Moving materialize operator and nested subplan to join rule to Hyracks.
Change-Id: I5a18d30a9b097ec890c03a0c0f30ebecfae46b3e
Reviewed-on: http://fulliautomatix.ics.uci.edu:8443/169
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <westmann@gmail.com>
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
index 15b64db..cedc962 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
@@ -46,6 +46,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
@@ -350,6 +351,12 @@
}
@Override
+ public ILogicalOperator visitMaterializeOperator(MaterializeOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
public ILogicalOperator visitRunningAggregateOperator(RunningAggregateOperator op, ILogicalOperator arg) {
throw new UnsupportedOperationException();
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/MaterializeOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/MaterializeOperator.java
deleted file mode 100644
index a8e8049..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/MaterializeOperator.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.algebra.operators;
-
-import java.util.Collection;
-
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractExtensibleLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorExtension;
-import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
-
-public class MaterializeOperator extends AbstractExtensibleLogicalOperator {
-
- @Override
- public boolean isMap() {
- return false;
- }
-
- @Override
- public IOperatorExtension newInstance() {
- return new MaterializeOperator();
- }
-
- @Override
- public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform) throws AlgebricksException {
- return false;
- }
-
- @Override
- public String toString() {
- return "materialize";
- }
-
- @Override
- public void getUsedVariables(Collection<LogicalVariable> usedVars) {
- // No used variables.
- }
-
- @Override
- public void getProducedVariables(Collection<LogicalVariable> producedVars) {
- // No produced variables.
- }
-}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java
deleted file mode 100644
index 542e36d..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.algebra.operators.physical;
-
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractPhysicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
-import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
-import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.misc.MaterializingOperatorDescriptor;
-
-public class MaterializePOperator extends AbstractPhysicalOperator {
-
- private final boolean isSingleActivity;
-
- public MaterializePOperator(boolean isSingleActivity) {
- this.isSingleActivity = isSingleActivity;
- }
-
- @Override
- public PhysicalOperatorTag getOperatorTag() {
- return PhysicalOperatorTag.EXTENSION_OPERATOR;
- }
-
- @Override
- public String toString() {
- return "MATERIALIZE";
- }
-
- @Override
- public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
- IPhysicalPropertiesVector reqdByParent) {
- return emptyUnaryRequirements();
- }
-
- @Override
- public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
- throws AlgebricksException {
- AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
- deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
- }
-
- @Override
- public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
- IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
- throws AlgebricksException {
- RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op),
- propagatedSchema, context);
- MaterializingOperatorDescriptor materializationOpDesc = new MaterializingOperatorDescriptor(
- builder.getJobSpec(), recDescriptor, isSingleActivity);
- contributeOpDesc(builder, (AbstractLogicalOperator) op, materializationOpDesc);
- ILogicalOperator src = op.getInputs().get(0).getValue();
- builder.contributeGraphEdge(src, 0, op, 0);
- }
-
- @Override
- public boolean isMicroOperator() {
- return true;
- }
-
- @Override
- public Pair<int[], int[]> getInputOutputDependencyLabels(ILogicalOperator op) {
- int[] inputDependencyLabels = new int[] { 0 };
- int[] outputDependencyLabels;
- if (isSingleActivity) {
- outputDependencyLabels = new int[] { 0 };
- } else {
- outputDependencyLabels = new int[] { 1 };
- }
- return new Pair<int[], int[]>(inputDependencyLabels, outputDependencyLabels);
- }
-
- @Override
- public boolean expensiveThanMaterialization() {
- return false;
- }
-}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index 5cc5ede..9626774 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -44,7 +44,6 @@
import edu.uci.ics.asterix.optimizer.rules.IntroduceUnnestForCollectionToSequenceRule;
import edu.uci.ics.asterix.optimizer.rules.LoadRecordFieldsRule;
import edu.uci.ics.asterix.optimizer.rules.NestGroupByRule;
-import edu.uci.ics.asterix.optimizer.rules.NestedSubplanToJoinRule;
import edu.uci.ics.asterix.optimizer.rules.PushAggFuncIntoStandaloneAggregateRule;
import edu.uci.ics.asterix.optimizer.rules.PushAggregateIntoGroupbyRule;
import edu.uci.ics.asterix.optimizer.rules.PushFieldAccessRule;
@@ -92,9 +91,10 @@
import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceProjectsRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.IsolateHyracksOperatorsRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.LeftOuterJoinToInnerJoinRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.NestedSubplanToJoinRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.PullSelectOutOfEqJoin;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushAssignBelowUnionAllRule;
-import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushAssignDownThroughProductRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushMapOperatorDownThroughProductRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushGroupByIntoSortRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushNestedOrderByUnderPreSortedGroupByRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushProjectDownRule;
@@ -102,7 +102,6 @@
import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSelectIntoJoinRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSubplanIntoGroupByRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSubplanWithAggregateDownThroughProductRule;
-import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushUnnestDownThroughProductRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.ReinferAllTypesRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.RemoveRedundantGroupByDecorVars;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.RemoveRedundantVariablesRule;
@@ -166,8 +165,7 @@
condPushDownAndJoinInference.add(new DisjunctivePredicateToJoinRule());
condPushDownAndJoinInference.add(new PushSelectIntoJoinRule());
condPushDownAndJoinInference.add(new IntroJoinInsideSubplanRule());
- condPushDownAndJoinInference.add(new PushAssignDownThroughProductRule());
- condPushDownAndJoinInference.add(new PushUnnestDownThroughProductRule());
+ condPushDownAndJoinInference.add(new PushMapOperatorDownThroughProductRule());
condPushDownAndJoinInference.add(new PushSubplanWithAggregateDownThroughProductRule());
condPushDownAndJoinInference.add(new IntroduceGroupByForSubplanRule());
condPushDownAndJoinInference.add(new SubplanOutOfGroupRule());
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
index c53e49a..06b203c 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
@@ -17,8 +17,6 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
-import edu.uci.ics.asterix.algebra.operators.MaterializeOperator;
-import edu.uci.ics.asterix.algebra.operators.physical.MaterializePOperator;
import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
@@ -34,9 +32,10 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.MaterializePOperator;
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
public class IntroduceMaterializationForInsertWithSelfScanRule implements IAlgebraicRewriteRule {
@@ -63,15 +62,12 @@
MaterializePOperator materializePOperator = new MaterializePOperator(true);
materializeOperator.setPhysicalOperator(materializePOperator);
- ExtensionOperator extensionOperator = new ExtensionOperator(materializeOperator);
- extensionOperator.setPhysicalOperator(materializePOperator);
-
- extensionOperator.getInputs().add(
+ materializeOperator.getInputs().add(
new MutableObject<ILogicalOperator>(insertOp.getInputs().get(0).getValue()));
- context.computeAndSetTypeEnvironmentForOperator(extensionOperator);
+ context.computeAndSetTypeEnvironmentForOperator(materializeOperator);
insertOp.getInputs().clear();
- insertOp.getInputs().add(new MutableObject<ILogicalOperator>(extensionOperator));
+ insertOp.getInputs().add(new MutableObject<ILogicalOperator>(materializeOperator));
context.computeAndSetTypeEnvironmentForOperator(insertOp);
return true;
} else {
@@ -105,7 +101,8 @@
} else if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
AqlDataSource ds = (AqlDataSource) dataSourceScanOp.getDataSource();
- if (ds.getDatasourceType() != AqlDataSourceType.FEED && ds.getDatasourceType() != AqlDataSourceType.ADAPTED_LOADABLE) {
+ if (ds.getDatasourceType() != AqlDataSourceType.FEED
+ && ds.getDatasourceType() != AqlDataSourceType.ADAPTED_LOADABLE) {
if (((DatasetDataSource) ds).getDataset().getDatasetName().compareTo(insertDatasetName) == 0) {
return true;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java
deleted file mode 100644
index 6929158..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.optimizer.rules;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.commons.lang3.mutable.MutableObject;
-
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-
-/**
- * replace Subplan operators with nested loop joins where the join condition is true, if the Subplan
- * does not contain free variables (does not have correlations to the input stream).
- *
- * @author yingyib
- */
-public class NestedSubplanToJoinRule implements IAlgebraicRewriteRule {
-
- @Override
- public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
- return false;
- }
-
- @Override
- public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
- throws AlgebricksException {
- if (context.checkIfInDontApplySet(this, opRef.getValue()))
- return false;
- context.addToDontApplySet(this, opRef.getValue());
-
- ILogicalOperator op1 = opRef.getValue();
- if (op1.getInputs().size() == 0) {
- return false;
- }
-
- boolean rewritten = false;
- for (int index = 0; index < op1.getInputs().size(); index++) {
- AbstractLogicalOperator child = (AbstractLogicalOperator) op1.getInputs().get(index).getValue();
- if (child.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
- continue;
- }
-
- AbstractOperatorWithNestedPlans subplan = (AbstractOperatorWithNestedPlans) child;
- Set<LogicalVariable> freeVars = new HashSet<LogicalVariable>();
- OperatorPropertiesUtil.getFreeVariablesInSubplans(subplan, freeVars);
- if (!freeVars.isEmpty()) {
- /**
- * the subplan is correlated with the outer plan, other rules can deal with it
- */
- continue;
- }
-
- /** get the input operator of the subplan operator */
- ILogicalOperator subplanInput = subplan.getInputs().get(0).getValue();
- AbstractLogicalOperator subplanInputOp = (AbstractLogicalOperator) subplanInput;
-
- /** If the other join branch is a trivial plan, do not do the rewriting. */
- if (subplanInputOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
- continue;
- }
-
- /** get all nested top operators */
- List<ILogicalPlan> nestedPlans = subplan.getNestedPlans();
- List<Mutable<ILogicalOperator>> nestedRoots = new ArrayList<Mutable<ILogicalOperator>>();
- for (ILogicalPlan nestedPlan : nestedPlans) {
- nestedRoots.addAll(nestedPlan.getRoots());
- }
- if (nestedRoots.size() == 0) {
- /** there is no nested top operators */
- continue;
- }
-
- /** expend the input and roots into a DAG of nested loop joins */
- Mutable<ILogicalExpression> expr = new MutableObject<ILogicalExpression>(ConstantExpression.TRUE);
- Mutable<ILogicalOperator> nestedRootRef = nestedRoots.get(0);
- ILogicalOperator join = new LeftOuterJoinOperator(expr, new MutableObject<ILogicalOperator>(subplanInput),
- nestedRootRef);
-
- /** rewrite the nested tuple source to be empty tuple source */
- rewriteNestedTupleSource(nestedRootRef);
-
- for (int i = 1; i < nestedRoots.size(); i++) {
- join = new LeftOuterJoinOperator(expr, new MutableObject<ILogicalOperator>(join), nestedRoots.get(i));
- }
- op1.getInputs().get(index).setValue(join);
- context.computeAndSetTypeEnvironmentForOperator(join);
- rewritten = true;
- }
- return rewritten;
- }
-
- /**
- * rewrite NestedTupleSource operators to EmptyTupleSource operators
- *
- * @param nestedRootRef
- */
- private void rewriteNestedTupleSource(Mutable<ILogicalOperator> nestedRootRef) {
- AbstractLogicalOperator nestedRoot = (AbstractLogicalOperator) nestedRootRef.getValue();
- if (nestedRoot.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
- nestedRootRef.setValue(new EmptyTupleSourceOperator());
- }
- List<Mutable<ILogicalOperator>> inputs = nestedRoot.getInputs();
- for (Mutable<ILogicalOperator> input : inputs) {
- rewriteNestedTupleSource(input);
- }
- }
-}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
index cfd25c0..3ebe6b6 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
@@ -27,6 +27,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
@@ -189,6 +190,11 @@
}
@Override
+ public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
public Void visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
return null;
}
diff --git a/pom.xml b/pom.xml
index 95f7fc9..e63c417 100644
--- a/pom.xml
+++ b/pom.xml
@@ -13,157 +13,160 @@
! See the License for the specific language governing permissions and
! limitations under the License.
!-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
- <modelVersion>4.0.0</modelVersion>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix</artifactId>
- <version>0.8.7-SNAPSHOT</version>
- <packaging>pom</packaging>
-
- <properties>
- <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
- <jvm.extraargs />
- <runSlowAQLTests>false</runSlowAQLTests>
+<project
+ xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <artifactId>asterix</artifactId>
+ <version>0.8.7-SNAPSHOT</version>
+ <packaging>pom</packaging>
+
+ <properties>
+ <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+ <jvm.extraargs />
+ <runSlowAQLTests>false</runSlowAQLTests>
<!-- Definition of tests in various categories which may be excluded -->
- <optimizer.tests>**/optimizer/**/*Test.java</optimizer.tests>
- <metadata.tests>**/metadata/*Test.java</metadata.tests>
- <execution.tests>**/ExecutionTest.java</execution.tests>
- <invalid.tests>**/DmlTest.java</invalid.tests>
- <global.test.includes>**/*TestSuite.java,**/*Test.java,${execution.tests}</global.test.includes>
- <global.test.excludes>${optimizer.tests},${metadata.tests},${invalid.tests}</global.test.excludes>
+ <optimizer.tests>**/optimizer/**/*Test.java</optimizer.tests>
+ <metadata.tests>**/metadata/*Test.java</metadata.tests>
+ <execution.tests>**/ExecutionTest.java</execution.tests>
+ <invalid.tests>**/DmlTest.java</invalid.tests>
+ <global.test.includes>**/*TestSuite.java,**/*Test.java,${execution.tests}</global.test.includes>
+ <global.test.excludes>${optimizer.tests},${metadata.tests},${invalid.tests}</global.test.excludes>
<!-- Versions under dependencymanagement or used in many projects via properties -->
- <algebricks.version>0.2.15-SNAPSHOT</algebricks.version>
- <hyracks.version>0.2.15-SNAPSHOT</hyracks.version>
- <hadoop.version>2.2.0</hadoop.version>
- <junit.version>4.8.1</junit.version>
- <commons.io.version>2.4</commons.io.version>
- <servlet.api.version>2.5</servlet.api.version>
- <json.version>20090211</json.version>
- </properties>
+ <algebricks.version>0.2.15-SNAPSHOT</algebricks.version>
+ <hyracks.version>0.2.15-SNAPSHOT</hyracks.version>
+ <hadoop.version>2.2.0</hadoop.version>
+ <junit.version>4.8.1</junit.version>
+ <commons.io.version>2.4</commons.io.version>
+ <servlet.api.version>2.5</servlet.api.version>
+ <json.version>20090211</json.version>
+ </properties>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-release-plugin</artifactId>
- <version>2.1</version>
- <configuration>
- <goals>package source:jar javadoc:jar deploy:deploy</goals>
- </configuration>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-surefire-plugin</artifactId>
- <version>2.16</version>
- <configuration>
- <failIfNoTests>false</failIfNoTests>
- <systemPropertyVariables>
- <skipFredSlowTests>true</skipFredSlowTests>
- </systemPropertyVariables>
- <forkCount>1</forkCount>
- <reuseForks>false</reuseForks>
- <argLine>-enableassertions -Xmx${test.heap.size}m
- -Dfile.encoding=UTF-8
- -Djava.util.logging.config.file=${user.home}/logging.properties
- -DrunSlowAQLTests=${runSlowAQLTests}
- -Xdebug
- -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n</argLine>
- <includes>
- <include>${global.test.includes},${test.includes}</include>
- </includes>
- <excludes>
- <exclude>${global.test.excludes},${test.excludes}</exclude>
- </excludes>
- </configuration>
- </plugin>
- </plugins>
- </build>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-release-plugin</artifactId>
+ <version>2.1</version>
+ <configuration>
+ <goals>package source:jar javadoc:jar deploy:deploy</goals>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <version>2.16</version>
+ <configuration>
+ <failIfNoTests>false</failIfNoTests>
+ <systemPropertyVariables>
+ <skipFredSlowTests>true</skipFredSlowTests>
+ </systemPropertyVariables>
+ <forkCount>1</forkCount>
+ <reuseForks>false</reuseForks>
+ <argLine>-enableassertions -Xmx${test.heap.size}m
+ -Dfile.encoding=UTF-8
+ -Djava.util.logging.config.file=${user.home}/logging.properties
+ -DrunSlowAQLTests=${runSlowAQLTests}
+ -Xdebug
+ -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n</argLine>
+ <includes>
+ <include>${global.test.includes},${test.includes}</include>
+ </includes>
+ <excludes>
+ <exclude>${global.test.excludes},${test.excludes}</exclude>
+ </excludes>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
- <scm>
- <connection>scm:git:https://code.google.com/p/asterixdb/</connection>
- <developerConnection>scm:git:ssh://fulliautomatix.ics.uci.edu:29418/asterixdb</developerConnection>
- </scm>
+ <scm>
+ <connection>scm:git:https://code.google.com/p/asterixdb/</connection>
+ <developerConnection>scm:git:ssh://fulliautomatix.ics.uci.edu:29418/asterixdb</developerConnection>
+ </scm>
- <distributionManagement>
- <repository>
- <id>asterix-releases</id>
- <url>http://obelix.ics.uci.edu/nexus/content/repositories/asterix-releases/</url>
- </repository>
- <snapshotRepository>
- <id>asterix-snapshots</id>
- <url>http://obelix.ics.uci.edu/nexus/content/repositories/asterix-snapshots/</url>
- </snapshotRepository>
- </distributionManagement>
+ <distributionManagement>
+ <repository>
+ <id>asterix-releases</id>
+ <url>http://obelix.ics.uci.edu/nexus/content/repositories/asterix-releases/</url>
+ </repository>
+ <snapshotRepository>
+ <id>asterix-snapshots</id>
+ <url>http://obelix.ics.uci.edu/nexus/content/repositories/asterix-snapshots/</url>
+ </snapshotRepository>
+ </distributionManagement>
<profiles>
- <profile>
- <id>slow-aql-tests</id>
- <properties>
- <runSlowAQLTests>true</runSlowAQLTests>
- </properties>
- </profile>
- <profile>
- <id>optimizer-tests</id>
- <properties>
- <optimizer.tests />
- </properties>
- </profile>
- <profile>
- <id>metadata-tests</id>
- <properties>
- <metadata.tests />
- </properties>
- </profile>
- <profile>
- <id>execution-tests</id>
- <properties>
- <execution.tests />
- </properties>
- </profile>
- <profile>
- <id>invalid-tests</id>
- <properties>
- <invalid.tests />
- </properties>
- </profile>
<profile>
- <id>32bitvm</id>
- <activation>
- <property>
- <name>sun.arch.data.model</name>
- <value>32</value>
- </property>
- </activation>
- <properties>
- <test.heap.size>2048</test.heap.size>
- </properties>
+ <id>slow-aql-tests</id>
+ <properties>
+ <runSlowAQLTests>true</runSlowAQLTests>
+ </properties>
+ </profile>
+ <profile>
+ <id>optimizer-tests</id>
+ <properties>
+ <optimizer.tests />
+ </properties>
+ </profile>
+ <profile>
+ <id>metadata-tests</id>
+ <properties>
+ <metadata.tests />
+ </properties>
+ </profile>
+ <profile>
+ <id>execution-tests</id>
+ <properties>
+ <execution.tests />
+ </properties>
+ </profile>
+ <profile>
+ <id>invalid-tests</id>
+ <properties>
+ <invalid.tests />
+ </properties>
+ </profile>
+ <profile>
+ <id>32bitvm</id>
+ <activation>
+ <property>
+ <name>sun.arch.data.model</name>
+ <value>32</value>
+ </property>
+ </activation>
+ <properties>
+ <test.heap.size>2048</test.heap.size>
+ </properties>
</profile>
<profile>
- <id>64bitvm</id>
- <activation>
- <property>
- <name>sun.arch.data.model</name>
- <value>64</value>
- </property>
- </activation>
- <properties>
- <test.heap.size>3072</test.heap.size>
- </properties>
+ <id>64bitvm</id>
+ <activation>
+ <property>
+ <name>sun.arch.data.model</name>
+ <value>64</value>
+ </property>
+ </activation>
+ <properties>
+ <test.heap.size>3072</test.heap.size>
+ </properties>
</profile>
</profiles>
- <modules>
- <module>asterix-common</module>
- <module>asterix-algebra</module>
- <module>asterix-app</module>
- <module>asterix-tools</module>
- <module>asterix-transactions</module>
- <module>asterix-runtime</module>
- <module>asterix-om</module>
- <module>asterix-aql</module>
+ <modules>
+ <module>asterix-common</module>
+ <module>asterix-algebra</module>
+ <module>asterix-app</module>
+ <module>asterix-tools</module>
+ <module>asterix-transactions</module>
+ <module>asterix-runtime</module>
+ <module>asterix-om</module>
+ <module>asterix-aql</module>
<module>asterix-external-data</module>
<module>asterix-examples</module>
<module>asterix-metadata</module>
@@ -174,197 +177,197 @@
<module>asterix-events</module>
<module>asterix-doc</module>
<module>asterix-fuzzyjoin</module>
- </modules>
+ </modules>
- <repositories>
- <repository>
- <releases>
- <enabled>true</enabled>
- <updatePolicy>always</updatePolicy>
- <checksumPolicy>warn</checksumPolicy>
- </releases>
- <snapshots>
- <enabled>true</enabled>
- <updatePolicy>always</updatePolicy>
- <checksumPolicy>fail</checksumPolicy>
- </snapshots>
- <id>asterix-public</id>
- <url>http://obelix.ics.uci.edu/nexus/content/groups/asterix-public/</url>
- </repository>
- <repository>
- <releases>
- <enabled>true</enabled>
- <updatePolicy>always</updatePolicy>
- <checksumPolicy>warn</checksumPolicy>
- </releases>
- <snapshots>
- <enabled>true</enabled>
- <updatePolicy>always</updatePolicy>
- <checksumPolicy>fail</checksumPolicy>
- </snapshots>
- <id>third-party</id>
- <url>http://obelix.ics.uci.edu/nexus/content/repositories/third-party/</url>
- </repository>
- <repository>
- <releases>
- <enabled>true</enabled>
- <updatePolicy>always</updatePolicy>
- <checksumPolicy>warn</checksumPolicy>
- </releases>
- <id>algebricks-releases</id>
- <url>http://obelix.ics.uci.edu/nexus/content/repositories/algebricks-releases/</url>
- </repository>
- <repository>
- <snapshots>
- <enabled>true</enabled>
- <updatePolicy>always</updatePolicy>
- <checksumPolicy>fail</checksumPolicy>
- </snapshots>
- <id>algebricks-snapshots</id>
- <url>http://obelix.ics.uci.edu/nexus/content/repositories/algebricks-snapshots/</url>
- </repository>
- </repositories>
- <dependencyManagement>
- <dependencies>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <version>${junit.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.maven</groupId>
- <artifactId>maven-plugin-api</artifactId>
- <version>2.2.1</version>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-yarn-common</artifactId>
- <version>${hadoop.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-yarn-client</artifactId>
- <version>${hadoop.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-client</artifactId>
- <version>${hadoop.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-hdfs</artifactId>
- <version>${hadoop.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-common</artifactId>
- <version>${hadoop.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>algebricks-compiler</artifactId>
- <version>${algebricks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-api</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-dataflow-std</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-control-cc</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-control-nc</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-server</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-cli</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-dataflow-hadoop</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-storage-am-btree</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-storage-am-rtree</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>
+ <repositories>
+ <repository>
+ <releases>
+ <enabled>true</enabled>
+ <updatePolicy>always</updatePolicy>
+ <checksumPolicy>warn</checksumPolicy>
+ </releases>
+ <snapshots>
+ <enabled>true</enabled>
+ <updatePolicy>always</updatePolicy>
+ <checksumPolicy>fail</checksumPolicy>
+ </snapshots>
+ <id>asterix-public</id>
+ <url>http://obelix.ics.uci.edu/nexus/content/groups/asterix-public/</url>
+ </repository>
+ <repository>
+ <releases>
+ <enabled>true</enabled>
+ <updatePolicy>always</updatePolicy>
+ <checksumPolicy>warn</checksumPolicy>
+ </releases>
+ <snapshots>
+ <enabled>true</enabled>
+ <updatePolicy>always</updatePolicy>
+ <checksumPolicy>fail</checksumPolicy>
+ </snapshots>
+ <id>third-party</id>
+ <url>http://obelix.ics.uci.edu/nexus/content/repositories/third-party/</url>
+ </repository>
+ <repository>
+ <releases>
+ <enabled>true</enabled>
+ <updatePolicy>always</updatePolicy>
+ <checksumPolicy>warn</checksumPolicy>
+ </releases>
+ <id>algebricks-releases</id>
+ <url>http://obelix.ics.uci.edu/nexus/content/repositories/algebricks-releases/</url>
+ </repository>
+ <repository>
+ <snapshots>
+ <enabled>true</enabled>
+ <updatePolicy>always</updatePolicy>
+ <checksumPolicy>fail</checksumPolicy>
+ </snapshots>
+ <id>algebricks-snapshots</id>
+ <url>http://obelix.ics.uci.edu/nexus/content/repositories/algebricks-snapshots/</url>
+ </repository>
+ </repositories>
+ <dependencyManagement>
+ <dependencies>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>${junit.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven</groupId>
+ <artifactId>maven-plugin-api</artifactId>
+ <version>2.2.1</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-common</artifactId>
+ <version>${hadoop.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-client</artifactId>
+ <version>${hadoop.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-client</artifactId>
+ <version>${hadoop.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ <version>${hadoop.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <version>${hadoop.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>algebricks-compiler</artifactId>
+ <version>${algebricks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-dataflow-std</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-cc</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-nc</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-server</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-cli</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-dataflow-hadoop</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-btree</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-rtree</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>
hyracks-storage-am-invertedindex
</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-storage-am-common</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-common</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-client</artifactId>
<version>${hyracks.version}</version>
</dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-storage-am-lsm-common</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-storage-am-lsm-btree</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-storage-am-lsm-rtree</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
- <version>${hyracks.version}</version>
- </dependency>
- <dependency>
- <groupId>org.json</groupId>
- <artifactId>json</artifactId>
- <version>${json.version}</version>
- <type>jar</type>
- </dependency>
- <dependency>
- <groupId>javax.servlet</groupId>
- <artifactId>servlet-api</artifactId>
- <version>${servlet.api.version}</version>
- <type>jar</type>
- </dependency>
- <dependency>
- <groupId>commons-io</groupId>
- <artifactId>commons-io</artifactId>
- <version>${commons.io.version}</version>
- </dependency>
- </dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-common</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-btree</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-rtree</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
+ <version>${hyracks.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.json</groupId>
+ <artifactId>json</artifactId>
+ <version>${json.version}</version>
+ <type>jar</type>
+ </dependency>
+ <dependency>
+ <groupId>javax.servlet</groupId>
+ <artifactId>servlet-api</artifactId>
+ <version>${servlet.api.version}</version>
+ <type>jar</type>
+ </dependency>
+ <dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ <version>${commons.io.version}</version>
+ </dependency>
+ </dependencies>
</dependencyManagement>
</project>