Hoisted algebricks as a top-level project in fullstack
git-svn-id: https://hyracks.googlecode.com/svn/branches/fullstack_staging@1968 123451ca-8445-de46-9d55-352943316053
diff --git a/algebricks/algebricks-rewriter/pom.xml b/algebricks/algebricks-rewriter/pom.xml
new file mode 100644
index 0000000..cc71c33
--- /dev/null
+++ b/algebricks/algebricks-rewriter/pom.xml
@@ -0,0 +1,31 @@
+<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>
+ <artifactId>algebricks-rewriter</artifactId>
+
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>algebricks</artifactId>
+ <version>0.2.2-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.6</source>
+ <target>1.6</target>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>algebricks-core</artifactId>
+ <version>0.2.2-SNAPSHOT</version>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
new file mode 100644
index 0000000..3b7fa8c
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.Collection;
+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.common.utils.Pair;
+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.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
+
+public abstract class AbstractDecorrelationRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ protected boolean descOrSelfIsScanOrJoin(AbstractLogicalOperator op2) {
+ LogicalOperatorTag t = op2.getOperatorTag();
+ if (t == LogicalOperatorTag.DATASOURCESCAN || t == LogicalOperatorTag.INNERJOIN
+ || t == LogicalOperatorTag.LEFTOUTERJOIN) {
+ return true;
+ }
+ if (op2.getInputs().size() != 1) {
+ return false;
+ }
+ AbstractLogicalOperator alo = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
+ if (descOrSelfIsScanOrJoin(alo)) {
+ return true;
+ }
+ return false;
+ }
+
+ protected Set<LogicalVariable> computeGbyVarsUsingPksOnly(Set<LogicalVariable> varSet, AbstractLogicalOperator op,
+ IOptimizationContext context) throws AlgebricksException {
+ PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(op, context);
+ List<FunctionalDependency> fdList = context.getFDList(op);
+ if (fdList == null) {
+ return null;
+ }
+ // check if any of the FDs is a key
+ for (FunctionalDependency fd : fdList) {
+ if (fd.getTail().containsAll(varSet)) {
+ return new HashSet<LogicalVariable>(fd.getHead());
+ }
+ }
+ return null;
+ }
+
+ protected void buildVarExprList(Collection<LogicalVariable> vars, IOptimizationContext context, GroupByOperator g,
+ List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> outVeList) throws AlgebricksException {
+ for (LogicalVariable ov : vars) {
+ LogicalVariable newVar = context.newVar();
+ ILogicalExpression varExpr = new VariableReferenceExpression(newVar);
+ outVeList.add(new Pair<LogicalVariable, Mutable<ILogicalExpression>>(ov,
+ new MutableObject<ILogicalExpression>(varExpr)));
+ for (ILogicalPlan p : g.getNestedPlans()) {
+ for (Mutable<ILogicalOperator> r : p.getRoots()) {
+ OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getValue(), ov, newVar, true,
+ context);
+ }
+ }
+ // g.substituteVarInNestedPlans(ov, newVar);
+ // OperatorManipulationUtil.substituteVarRec(lojoin, ov, newVar);
+ }
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
new file mode 100644
index 0000000..553165a
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+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.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public abstract class AbstractExtractExprRule implements IAlgebraicRewriteRule {
+
+ protected LogicalVariable extractExprIntoAssignOpRef(ILogicalExpression gExpr, Mutable<ILogicalOperator> opRef2,
+ IOptimizationContext context) throws AlgebricksException {
+ LogicalVariable v = context.newVar();
+ AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(gExpr));
+ a.getInputs().add(new MutableObject<ILogicalOperator>(opRef2.getValue()));
+ opRef2.setValue(a);
+ if (gExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ context.addNotToBeInlinedVar(v);
+ }
+ context.computeAndSetTypeEnvironmentForOperator(a);
+ return v;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java
new file mode 100644
index 0000000..08271c1
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java
@@ -0,0 +1,144 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+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.common.utils.Pair;
+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.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public abstract class AbstractIntroduceCombinerRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ /**
+ * Replace the original aggregate functions with their corresponding global aggregate function.
+ */
+ public void replaceOriginalAggFuncs(Map<AggregateFunctionCallExpression, SimilarAggregatesInfo> toReplaceMap) {
+ for (Map.Entry<AggregateFunctionCallExpression, SimilarAggregatesInfo> entry : toReplaceMap.entrySet()) {
+ SimilarAggregatesInfo sai = entry.getValue();
+ for (AggregateExprInfo aei : sai.simAggs) {
+ AbstractFunctionCallExpression afce = (AbstractFunctionCallExpression) aei.aggExprRef.getValue();
+ afce.setFunctionInfo(aei.newFunInfo);
+ afce.getArguments().clear();
+ afce.getArguments().add(new MutableObject<ILogicalExpression>(sai.stepOneResult));
+ }
+ }
+ }
+
+ protected Pair<Boolean, Mutable<ILogicalOperator>> tryToPushAgg(AggregateOperator initAgg,
+ GroupByOperator newGbyOp, Map<AggregateFunctionCallExpression, SimilarAggregatesInfo> toReplaceMap,
+ IOptimizationContext context) throws AlgebricksException {
+
+ ArrayList<LogicalVariable> pushedVars = new ArrayList<LogicalVariable>();
+ ArrayList<Mutable<ILogicalExpression>> pushedExprs = new ArrayList<Mutable<ILogicalExpression>>();
+
+ List<LogicalVariable> initVars = initAgg.getVariables();
+ List<Mutable<ILogicalExpression>> initExprs = initAgg.getExpressions();
+ int numExprs = initVars.size();
+
+ // First make sure that all agg funcs are two step, otherwise we cannot use local aggs.
+ for (int i = 0; i < numExprs; i++) {
+ AggregateFunctionCallExpression aggFun = (AggregateFunctionCallExpression) initExprs.get(i).getValue();
+ if (!aggFun.isTwoStep()) {
+ return new Pair<Boolean, Mutable<ILogicalOperator>>(false, null);
+ }
+ }
+
+ boolean haveAggToReplace = false;
+ for (int i = 0; i < numExprs; i++) {
+ Mutable<ILogicalExpression> expRef = initExprs.get(i);
+ AggregateFunctionCallExpression aggFun = (AggregateFunctionCallExpression) expRef.getValue();
+ IFunctionInfo fi1 = aggFun.getStepOneAggregate();
+ // Clone the aggregate's args.
+ List<Mutable<ILogicalExpression>> newArgs = new ArrayList<Mutable<ILogicalExpression>>(aggFun
+ .getArguments().size());
+ for (Mutable<ILogicalExpression> er : aggFun.getArguments()) {
+ newArgs.add(new MutableObject<ILogicalExpression>(er.getValue().cloneExpression()));
+ }
+ IFunctionInfo fi2 = aggFun.getStepTwoAggregate();
+ SimilarAggregatesInfo inf = toReplaceMap.get(aggFun);
+ if (inf == null) {
+ inf = new SimilarAggregatesInfo();
+ LogicalVariable newAggVar = context.newVar();
+ pushedVars.add(newAggVar);
+ inf.stepOneResult = new VariableReferenceExpression(newAggVar);
+ inf.simAggs = new ArrayList<AggregateExprInfo>();
+ toReplaceMap.put(aggFun, inf);
+ AggregateFunctionCallExpression aggLocal = new AggregateFunctionCallExpression(fi1, false, newArgs);
+ pushedExprs.add(new MutableObject<ILogicalExpression>(aggLocal));
+ }
+ AggregateExprInfo aei = new AggregateExprInfo();
+ aei.aggExprRef = expRef;
+ aei.newFunInfo = fi2;
+ inf.simAggs.add(aei);
+ haveAggToReplace = true;
+ }
+
+ if (!pushedVars.isEmpty()) {
+ AggregateOperator pushedAgg = new AggregateOperator(pushedVars, pushedExprs);
+ pushedAgg.setExecutionMode(ExecutionMode.LOCAL);
+ // If newGbyOp is null, then we optimizing an aggregate without group by.
+ if (newGbyOp != null) {
+ // Hook up the nested aggregate op with the outer group by.
+ NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(
+ newGbyOp));
+ nts.setExecutionMode(ExecutionMode.LOCAL);
+ pushedAgg.getInputs().add(new MutableObject<ILogicalOperator>(nts));
+ } else {
+ // The local aggregate operator is fed by the input of the original aggregate operator.
+ pushedAgg.getInputs().add(new MutableObject<ILogicalOperator>(initAgg.getInputs().get(0).getValue()));
+ // Set the partitioning variable in the local agg to ensure it is not projected away.
+ context.computeAndSetTypeEnvironmentForOperator(pushedAgg);
+ LogicalVariable trueVar = context.newVar();
+ // Reintroduce assign op for the global agg partitioning var.
+ AssignOperator trueAssignOp = new AssignOperator(trueVar, new MutableObject<ILogicalExpression>(
+ ConstantExpression.TRUE));
+ trueAssignOp.getInputs().add(new MutableObject<ILogicalOperator>(pushedAgg));
+ context.computeAndSetTypeEnvironmentForOperator(trueAssignOp);
+ initAgg.setPartitioningVariable(trueVar);
+ initAgg.getInputs().get(0).setValue(trueAssignOp);
+ }
+ return new Pair<Boolean, Mutable<ILogicalOperator>>(true, new MutableObject<ILogicalOperator>(pushedAgg));
+ } else {
+ return new Pair<Boolean, Mutable<ILogicalOperator>>(haveAggToReplace, null);
+ }
+ }
+
+ protected class SimilarAggregatesInfo {
+ ILogicalExpression stepOneResult;
+ List<AggregateExprInfo> simAggs;
+ }
+
+ protected class AggregateExprInfo {
+ Mutable<ILogicalExpression> aggExprRef;
+ IFunctionInfo newFunInfo;
+ }
+
+ protected class BookkeepingInfo {
+ Map<AggregateFunctionCallExpression, SimilarAggregatesInfo> toReplaceMap = new HashMap<AggregateFunctionCallExpression, SimilarAggregatesInfo>();
+ Map<GroupByOperator, List<LogicalVariable>> modifyGbyMap = new HashMap<GroupByOperator, List<LogicalVariable>>();
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
new file mode 100644
index 0000000..53b361b
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+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.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class BreakSelectIntoConjunctsRule implements IAlgebraicRewriteRule {
+
+ private List<Mutable<ILogicalExpression>> conjs = new ArrayList<Mutable<ILogicalExpression>>();
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ SelectOperator select = (SelectOperator) op;
+
+ ILogicalExpression cond = select.getCondition().getValue();
+
+ conjs.clear();
+ if (!cond.splitIntoConjuncts(conjs)) {
+ return false;
+ }
+
+ Mutable<ILogicalOperator> childOfSelect = select.getInputs().get(0);
+ boolean fst = true;
+ ILogicalOperator botOp = select;
+ ILogicalExpression firstExpr = null;
+ for (Mutable<ILogicalExpression> eRef : conjs) {
+ ILogicalExpression e = eRef.getValue();
+ if (fst) {
+ fst = false;
+ firstExpr = e;
+ } else {
+ SelectOperator newSelect = new SelectOperator(new MutableObject<ILogicalExpression>(e));
+ List<Mutable<ILogicalOperator>> botInpList = botOp.getInputs();
+ botInpList.clear();
+ botInpList.add(new MutableObject<ILogicalOperator>(newSelect));
+ context.computeAndSetTypeEnvironmentForOperator(botOp);
+ botOp = newSelect;
+ }
+ }
+ botOp.getInputs().add(childOfSelect);
+ select.getCondition().setValue(firstExpr);
+ context.computeAndSetTypeEnvironmentForOperator(botOp);
+ context.computeAndSetTypeEnvironmentForOperator(select);
+
+ return true;
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
new file mode 100644
index 0000000..4f6699a
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
@@ -0,0 +1,112 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+
+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.AbstractScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class ComplexJoinInferenceRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ ILogicalOperator op = opRef.getValue();
+ if (!(op instanceof AbstractScanOperator)) {
+ return false;
+ }
+
+ Mutable<ILogicalOperator> opRef2 = op.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+ if (op2.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ SubplanOperator subplan = (SubplanOperator) op2;
+
+ Mutable<ILogicalOperator> opRef3 = subplan.getInputs().get(0);
+ AbstractLogicalOperator op3 = (AbstractLogicalOperator) opRef3.getValue();
+
+ if (op3.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE
+ || op3.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ return false;
+ }
+
+ if (subplanHasFreeVariables(subplan)) {
+ return false;
+ }
+
+ HashSet<LogicalVariable> varsUsedInUnnest = new HashSet<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op, varsUsedInUnnest);
+
+ HashSet<LogicalVariable> producedInSubplan = new HashSet<LogicalVariable>();
+ VariableUtilities.getLiveVariables(subplan, producedInSubplan);
+
+ if (!producedInSubplan.containsAll(varsUsedInUnnest)) {
+ return false;
+ }
+
+ ntsToEtsInSubplan(subplan, context);
+ InnerJoinOperator join = new InnerJoinOperator(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+ join.getInputs().add(opRef3);
+ opRef2.setValue(OperatorManipulationUtil.eliminateSingleSubplanOverEts(subplan));
+ join.getInputs().add(new MutableObject<ILogicalOperator>(op));
+ opRef.setValue(join);
+ context.computeAndSetTypeEnvironmentForOperator(join);
+ return true;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ private static void ntsToEtsInSubplan(SubplanOperator s, IOptimizationContext context) throws AlgebricksException {
+ for (ILogicalPlan p : s.getNestedPlans()) {
+ for (Mutable<ILogicalOperator> r : p.getRoots()) {
+ OperatorManipulationUtil.ntsToEts(r, context);
+ }
+ }
+ }
+
+ private static boolean subplanHasFreeVariables(SubplanOperator s) throws AlgebricksException {
+ for (ILogicalPlan p : s.getNestedPlans()) {
+ for (Mutable<ILogicalOperator> r : p.getRoots()) {
+ if (OperatorPropertiesUtil.hasFreeVariablesInSelfOrDesc((AbstractLogicalOperator) r.getValue())) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
new file mode 100644
index 0000000..4521d1a
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
@@ -0,0 +1,276 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+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.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.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Complex rewrite rule for producing joins from unnests.
+ * This rule is limited to creating left-deep trees.
+ */
+public class ComplexUnnestToProductRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN
+ && op.getOperatorTag() != LogicalOperatorTag.UNNEST) {
+ return false;
+ }
+
+ // We may pull selects above the join we create in order to eliminate possible dependencies between
+ // the outer and inner input plans of the join.
+ List<ILogicalOperator> topSelects = new ArrayList<ILogicalOperator>();
+
+ // Keep track of the operators and used variables participating in the inner input plan.
+ HashSet<LogicalVariable> innerUsedVars = new HashSet<LogicalVariable>();
+ List<ILogicalOperator> innerOps = new ArrayList<ILogicalOperator>();
+ HashSet<LogicalVariable> outerUsedVars = new HashSet<LogicalVariable>();
+ List<ILogicalOperator> outerOps = new ArrayList<ILogicalOperator>();
+ innerOps.add(op);
+ VariableUtilities.getUsedVariables(op, innerUsedVars);
+
+ Mutable<ILogicalOperator> opRef2 = op.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+
+ // Find an unnest or join and partition the plan between the first unnest and that operator into independent parts.
+ if (!findPlanPartition(op2, innerUsedVars, outerUsedVars, innerOps, outerOps, topSelects, false)) {
+ // We could not find an unnest or join.
+ return false;
+ }
+ // The last operator must be an unnest or join.
+ AbstractLogicalOperator unnestOrJoin = (AbstractLogicalOperator) outerOps.get(outerOps.size() - 1);
+
+ ILogicalOperator outerRoot = null;
+ ILogicalOperator innerRoot = null;
+ EmptyTupleSourceOperator ets = new EmptyTupleSourceOperator();
+ // If we found a join, simply use it as the outer root.
+ if (unnestOrJoin.getOperatorTag() != LogicalOperatorTag.INNERJOIN
+ && unnestOrJoin.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+ // We've found a second unnest. First, sanity check that the unnest does not produce any vars that are used by the plan above (until the first unnest).
+ List<LogicalVariable> producedVars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(unnestOrJoin, producedVars);
+ for (LogicalVariable producedVar : producedVars) {
+ if (innerUsedVars.contains(producedVar)) {
+ return false;
+ }
+ }
+ // Continue finding a partitioning of the plan such that the inner and outer partitions are independent, in order to feed a join.
+ // Now, we look below the second unnest or join.
+ VariableUtilities.getUsedVariables(unnestOrJoin, outerUsedVars);
+ AbstractLogicalOperator unnestChild = (AbstractLogicalOperator) unnestOrJoin.getInputs().get(0).getValue();
+ if (!findPlanPartition(unnestChild, innerUsedVars, outerUsedVars, innerOps, outerOps, topSelects, true)) {
+ // We could not find a suitable partitioning.
+ return false;
+ }
+ }
+ innerRoot = buildOperatorChain(innerOps, ets, context);
+ context.computeAndSetTypeEnvironmentForOperator(innerRoot);
+ outerRoot = buildOperatorChain(outerOps, null, context);
+ context.computeAndSetTypeEnvironmentForOperator(outerRoot);
+
+ InnerJoinOperator product = new InnerJoinOperator(
+ new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+ // Outer branch.
+ product.getInputs().add(new MutableObject<ILogicalOperator>(outerRoot));
+ // Inner branch.
+ product.getInputs().add(new MutableObject<ILogicalOperator>(innerRoot));
+ context.computeAndSetTypeEnvironmentForOperator(product);
+ // Put the selects on top of the join.
+ ILogicalOperator topOp = product;
+ if (!topSelects.isEmpty()) {
+ topOp = buildOperatorChain(topSelects, product, context);
+ }
+ // Plug the selects + product in the plan.
+ opRef.setValue(topOp);
+ context.computeAndSetTypeEnvironmentForOperator(topOp);
+ return true;
+ }
+
+ private ILogicalOperator buildOperatorChain(List<ILogicalOperator> ops, ILogicalOperator bottomOp,
+ IOptimizationContext context) throws AlgebricksException {
+ ILogicalOperator root = ops.get(0);
+ ILogicalOperator prevOp = root;
+ for (int i = 1; i < ops.size(); i++) {
+ ILogicalOperator inputOp = ops.get(i);
+ prevOp.getInputs().clear();
+ prevOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+ prevOp = inputOp;
+ }
+ if (bottomOp != null) {
+ context.computeAndSetTypeEnvironmentForOperator(bottomOp);
+ prevOp.getInputs().clear();
+ prevOp.getInputs().add(new MutableObject<ILogicalOperator>(bottomOp));
+ }
+ return root;
+ }
+
+ private boolean findPlanPartition(AbstractLogicalOperator op, HashSet<LogicalVariable> innerUsedVars,
+ HashSet<LogicalVariable> outerUsedVars, List<ILogicalOperator> innerOps, List<ILogicalOperator> outerOps,
+ List<ILogicalOperator> topSelects, boolean belowSecondUnnest) throws AlgebricksException {
+ if (belowSecondUnnest && innerUsedVars.isEmpty()) {
+ // Trivially joinable.
+ return true;
+ }
+ if (!belowSecondUnnest && op.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+ // Bail on subplan.
+ return false;
+ }
+ switch (op.getOperatorTag()) {
+ case UNNEST:
+ case DATASOURCESCAN: {
+ // We may have reached this state by descending through a subplan.
+ outerOps.add(op);
+ return true;
+ }
+ case INNERJOIN:
+ case LEFTOUTERJOIN: {
+ // Make sure that no variables that are live under this join are needed by the inner.
+ List<LogicalVariable> liveVars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(op, liveVars);
+ for (LogicalVariable liveVar : liveVars) {
+ if (innerUsedVars.contains(liveVar)) {
+ return false;
+ }
+ }
+ outerOps.add(op);
+ return true;
+ }
+ case SELECT: {
+ // Remember this select to pulling it above the join.
+ if (innerUsedVars.isEmpty()) {
+ outerOps.add(op);
+ } else {
+ topSelects.add(op);
+ }
+ break;
+ }
+ case PROJECT: {
+ // Throw away projects from the plan since we are pulling selects up.
+ break;
+ }
+ case EMPTYTUPLESOURCE:
+ case NESTEDTUPLESOURCE: {
+ if (belowSecondUnnest) {
+ // We have successfully partitioned the plan into independent parts to be plugged into the join.
+ return true;
+ } else {
+ // We could not find a second unnest or a join.
+ return false;
+ }
+ }
+ default: {
+ // The inner is trivially independent.
+ if (!belowSecondUnnest && innerUsedVars.isEmpty()) {
+ outerOps.add(op);
+ break;
+ }
+
+ // Examine produced vars to determine which partition uses them.
+ List<LogicalVariable> producedVars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op, producedVars);
+ int outerMatches = 0;
+ int innerMatches = 0;
+ for (LogicalVariable producedVar : producedVars) {
+ if (outerUsedVars.contains(producedVar)) {
+ outerMatches++;
+ } else if (innerUsedVars.contains(producedVar)) {
+ innerMatches++;
+ }
+ }
+
+ HashSet<LogicalVariable> targetUsedVars = null;
+ if (outerMatches == producedVars.size() && !producedVars.isEmpty()) {
+ // All produced vars used by outer partition.
+ outerOps.add(op);
+ targetUsedVars = outerUsedVars;
+ } else if (innerMatches == producedVars.size() && !producedVars.isEmpty()) {
+ // All produced vars used by inner partition.
+ innerOps.add(op);
+ targetUsedVars = innerUsedVars;
+ } else if (innerMatches == 0 && outerMatches == 0) {
+ // Op produces variables that are not used in the part of the plan we've seen (or it doesn't produce any vars).
+ // Try to figure out where it belongs by analyzing the used variables.
+ List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op, usedVars);
+ for (LogicalVariable usedVar : usedVars) {
+ if (outerUsedVars.contains(usedVar)) {
+ outerOps.add(op);
+ targetUsedVars = outerUsedVars;
+ break;
+ }
+ if (innerUsedVars.contains(usedVar)) {
+ innerOps.add(op);
+ targetUsedVars = innerUsedVars;
+ break;
+ }
+ }
+ // TODO: For now we bail here, but we could remember such ops and determine their target partition at a later point.
+ if (targetUsedVars == null) {
+ return false;
+ }
+ } else {
+ // The current operator produces variables that are used by both partitions, so the inner and outer are not independent and, therefore, we cannot create a join.
+ // TODO: We may still be able to split the operator to create a viable partitioning.
+ return false;
+ }
+ // Update used variables of partition that op belongs to.
+ if (op.hasNestedPlans() && op.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ AbstractOperatorWithNestedPlans opWithNestedPlans = (AbstractOperatorWithNestedPlans) op;
+ opWithNestedPlans.getUsedVariablesExceptNestedPlans(targetUsedVars);
+ } else {
+ VariableUtilities.getUsedVariables(op, targetUsedVars);
+ }
+ break;
+ }
+ }
+ if (!op.hasInputs()) {
+ if (!belowSecondUnnest) {
+ // We could not find a second unnest or a join.
+ return false;
+ } else {
+ // We have successfully partitioned the plan into independent parts to be plugged into the join.
+ return true;
+ }
+ }
+ return findPlanPartition((AbstractLogicalOperator) op.getInputs().get(0).getValue(), innerUsedVars,
+ outerUsedVars, innerOps, outerOps, topSelects, belowSecondUnnest);
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateAssignsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateAssignsRule.java
new file mode 100644
index 0000000..e0220cc
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateAssignsRule.java
@@ -0,0 +1,73 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class ConsolidateAssignsRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+ return false;
+ }
+ AssignOperator assign1 = (AssignOperator) op;
+
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) assign1.getInputs().get(0).getValue();
+ if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+ return false;
+ }
+ AssignOperator assign2 = (AssignOperator) op2;
+
+ HashSet<LogicalVariable> used1 = new HashSet<LogicalVariable>();
+ VariableUtilities.getUsedVariables(assign1, used1);
+ for (LogicalVariable v2 : assign2.getVariables()) {
+ if (used1.contains(v2)) {
+ return false;
+ }
+ }
+
+ assign1.getVariables().addAll(assign2.getVariables());
+ assign1.getExpressions().addAll(assign2.getExpressions());
+
+ Mutable<ILogicalOperator> botOpRef = assign2.getInputs().get(0);
+ List<Mutable<ILogicalOperator>> asgnInpList = assign1.getInputs();
+ asgnInpList.clear();
+ asgnInpList.add(botOpRef);
+ context.computeAndSetTypeEnvironmentForOperator(assign1);
+ return true;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
new file mode 100644
index 0000000..ff9d921
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+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.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Matches the following operator pattern:
+ * (select) <-- ((assign)* <-- (select)*)+
+ *
+ * Consolidates the selects to:
+ * (select) <-- (assign)*
+ *
+ */
+public class ConsolidateSelectsRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ SelectOperator firstSelect = (SelectOperator) op;
+
+ IFunctionInfo andFn = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
+ // New conjuncts for consolidated select.
+ AbstractFunctionCallExpression conj = null;
+ AbstractLogicalOperator topMostOp = null;
+ AbstractLogicalOperator selectParent = null;
+ AbstractLogicalOperator nextSelect = firstSelect;
+ do {
+ // Skip through assigns.
+ do {
+ selectParent = nextSelect;
+ nextSelect = (AbstractLogicalOperator) selectParent.getInputs().get(0).getValue();
+ } while (nextSelect.getOperatorTag() == LogicalOperatorTag.ASSIGN);
+ // Stop if the child op is not a select.
+ if (nextSelect.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ break;
+ }
+ // Remember the top-most op that we are not removing.
+ topMostOp = selectParent;
+
+ // Initialize the new conjuncts, if necessary.
+ if (conj == null) {
+ conj = new ScalarFunctionCallExpression(andFn);
+ // Add the first select's condition.
+ conj.getArguments().add(new MutableObject<ILogicalExpression>(firstSelect.getCondition().getValue()));
+ }
+
+ // Consolidate all following selects.
+ do {
+ // Add the condition nextSelect to the new list of conjuncts.
+ conj.getArguments().add(((SelectOperator) nextSelect).getCondition());
+ selectParent = nextSelect;
+ nextSelect = (AbstractLogicalOperator) nextSelect.getInputs().get(0).getValue();
+ } while (nextSelect.getOperatorTag() == LogicalOperatorTag.SELECT);
+
+ // Hook up the input of the top-most remaining op if necessary.
+ if (topMostOp.getOperatorTag() == LogicalOperatorTag.ASSIGN || topMostOp == firstSelect) {
+ topMostOp.getInputs().set(0, selectParent.getInputs().get(0));
+ }
+
+ // Prepare for next iteration.
+ nextSelect = selectParent;
+ } while (true);
+
+ // Did we consolidate any selects?
+ if (conj == null) {
+ return false;
+ }
+
+ // Set the new conjuncts.
+ firstSelect.getCondition().setValue(conj);
+ context.computeAndSetTypeEnvironmentForOperator(firstSelect);
+ return true;
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EliminateSubplanRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EliminateSubplanRule.java
new file mode 100644
index 0000000..3e6ab6f
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EliminateSubplanRule.java
@@ -0,0 +1,126 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.LinkedList;
+
+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.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class EliminateSubplanRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ /**
+ * Eliminate Subplan above ETS
+ * and Subplan that has only ops. with one input and no free vars. (could we
+ * modify it to consider free vars which are sources of Unnest or Assign, if
+ * there are no aggregates?)
+ */
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ SubplanOperator subplan = (SubplanOperator) op;
+
+ Mutable<ILogicalOperator> outerRef = subplan.getInputs().get(0);
+ AbstractLogicalOperator outerRefOp = (AbstractLogicalOperator) outerRef.getValue();
+ if (outerRefOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ elimSubplanOverEts(opRef, context);
+ return true;
+ }
+ if (subplan.getNestedPlans().size() == 1 && subplan.getNestedPlans().get(0).getRoots().size() == 1
+ && !OperatorPropertiesUtil.hasFreeVariables(subplan)) {
+ if (elimOneSubplanWithNoFreeVars(opRef)) {
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+ private boolean elimOneSubplanWithNoFreeVars(Mutable<ILogicalOperator> opRef) {
+ SubplanOperator subplan = (SubplanOperator) opRef.getValue();
+ AbstractLogicalOperator rootOp = (AbstractLogicalOperator) subplan.getNestedPlans().get(0).getRoots().get(0)
+ .getValue();
+ if (rootOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ opRef.setValue(subplan.getInputs().get(0).getValue());
+ return true;
+ } else {
+ AbstractLogicalOperator botOp = rootOp;
+ if (botOp.getInputs().size() != 1) {
+ return false;
+ }
+ do {
+ Mutable<ILogicalOperator> botRef = botOp.getInputs().get(0);
+ botOp = (AbstractLogicalOperator) botRef.getValue();
+ if (botOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ botRef.setValue(subplan.getInputs().get(0).getValue());
+ opRef.setValue(rootOp);
+ return true;
+ }
+ } while (botOp.getInputs().size() == 1);
+ return false;
+ }
+ }
+
+ private void elimSubplanOverEts(Mutable<ILogicalOperator> opRef, IOptimizationContext ctx)
+ throws AlgebricksException {
+ SubplanOperator subplan = (SubplanOperator) opRef.getValue();
+ for (ILogicalPlan p : subplan.getNestedPlans()) {
+ for (Mutable<ILogicalOperator> r : p.getRoots()) {
+ OperatorManipulationUtil.ntsToEts(r, ctx);
+ }
+ }
+ LinkedList<Mutable<ILogicalOperator>> allRoots = subplan.allRootsInReverseOrder();
+ if (allRoots.size() == 1) {
+ opRef.setValue(allRoots.get(0).getValue());
+ } else {
+ ILogicalOperator topOp = null;
+ for (Mutable<ILogicalOperator> r : allRoots) {
+ if (topOp == null) {
+ topOp = r.getValue();
+ } else {
+ LeftOuterJoinOperator j = new LeftOuterJoinOperator(new MutableObject<ILogicalExpression>(
+ ConstantExpression.TRUE));
+ j.getInputs().add(new MutableObject<ILogicalOperator>(topOp));
+ j.getInputs().add(r);
+ ctx.setOutputTypeEnvironment(j, j.computeOutputTypeEnvironment(ctx));
+ topOp = j;
+ }
+ }
+ opRef.setValue(topOp);
+ }
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
new file mode 100644
index 0000000..de2e6af
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -0,0 +1,583 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+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.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+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.IPhysicalOperator;
+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.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+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.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.FDsAndEquivClassesVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractStableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.BroadcastPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HashPartitionExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.InMemoryStableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.PreSortedDistinctByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.PreclusteredGroupByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.RandomMergeExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.RangePartitionPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SortMergeExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.AsterixNodeGroupDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty.PropertyType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty.PartitioningType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalGroupingProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderedPartitionedProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
+
+public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
+
+ private static final INodeDomain DEFAULT_DOMAIN = new AsterixNodeGroupDomain("__DEFAULT");
+
+ private PhysicalOptimizationConfig physicalOptimizationConfig;
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ // wait for the physical operators to be set first
+ if (op.getPhysicalOperator() == null) {
+ return false;
+ }
+ if (context.checkIfInDontApplySet(this, op)) {
+ return false;
+ }
+
+ List<FunctionalDependency> fds = context.getFDList(op);
+ if (fds != null && !fds.isEmpty()) {
+ return false;
+ }
+ // These are actually logical constraints, so they could be pre-computed
+ // somewhere else, too.
+
+ physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Optimizing operator " + op.getPhysicalOperator() + ".\n");
+
+ PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(op, context);
+
+ StructuralPropertiesVector pvector = new StructuralPropertiesVector(new RandomPartitioningProperty(null),
+ new LinkedList<ILocalStructuralProperty>());
+ boolean changed = physOptimizeOp(opRef, pvector, false, context);
+ op.computeDeliveredPhysicalProperties(context);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Structural properties for " + op.getPhysicalOperator() + ": "
+ + op.getDeliveredPhysicalProperties() + "\n");
+
+ context.addToDontApplySet(this, opRef.getValue());
+
+ return changed;
+ }
+
+ private boolean physOptimizePlan(ILogicalPlan plan, IPhysicalPropertiesVector pvector, boolean nestedPlan,
+ IOptimizationContext context) throws AlgebricksException {
+ boolean changed = false;
+ for (Mutable<ILogicalOperator> root : plan.getRoots()) {
+ if (physOptimizeOp(root, pvector, nestedPlan, context)) {
+ changed = true;
+ }
+ AbstractLogicalOperator op = (AbstractLogicalOperator) root.getValue();
+ op.computeDeliveredPhysicalProperties(context);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Structural properties for " + op.getPhysicalOperator()
+ + ": " + op.getDeliveredPhysicalProperties() + "\n");
+ }
+ return changed;
+ }
+
+ private boolean physOptimizeOp(Mutable<ILogicalOperator> opRef, IPhysicalPropertiesVector required,
+ boolean nestedPlan, IOptimizationContext context) throws AlgebricksException {
+
+ boolean changed = false;
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ optimizeUsingConstraintsAndEquivClasses(op);
+ PhysicalRequirements pr = op.getRequiredPhysicalPropertiesForChildren(required);
+ IPhysicalPropertiesVector[] reqdProperties = null;
+ if (pr != null) {
+ reqdProperties = pr.getRequiredProperties();
+ }
+ boolean opIsRedundantSort = false;
+
+ // compute properties and figure out the domain
+ INodeDomain childrenDomain = null;
+ {
+ int j = 0;
+ for (Mutable<ILogicalOperator> childRef : op.getInputs()) {
+ AbstractLogicalOperator child = (AbstractLogicalOperator) childRef.getValue();
+ // recursive call
+ if (physOptimizeOp(childRef, reqdProperties[j], nestedPlan, context)) {
+ changed = true;
+ }
+ child.computeDeliveredPhysicalProperties(context);
+ IPhysicalPropertiesVector delivered = child.getDeliveredPhysicalProperties();
+ if (childrenDomain == null) {
+ childrenDomain = delivered.getPartitioningProperty().getNodeDomain();
+ } else {
+ INodeDomain dom2 = delivered.getPartitioningProperty().getNodeDomain();
+ if (!childrenDomain.sameAs(dom2)) {
+ childrenDomain = DEFAULT_DOMAIN;
+ }
+ }
+ j++;
+ }
+ }
+
+ if (reqdProperties != null) {
+ for (int k = 0; k < reqdProperties.length; k++) {
+ IPhysicalPropertiesVector pv = reqdProperties[k];
+ IPartitioningProperty pp = pv.getPartitioningProperty();
+ if (pp != null && pp.getNodeDomain() == null) {
+ pp.setNodeDomain(childrenDomain);
+ }
+ }
+ }
+
+ IPartitioningProperty firstDeliveredPartitioning = null;
+ int i = 0;
+ for (Mutable<ILogicalOperator> childRef : op.getInputs()) {
+ AbstractLogicalOperator child = (AbstractLogicalOperator) childRef.getValue();
+ IPhysicalPropertiesVector delivered = child.getDeliveredPhysicalProperties();
+
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Properties delivered by " + child.getPhysicalOperator()
+ + ": " + delivered + "\n");
+ IPartitioningRequirementsCoordinator prc = pr.getPartitioningCoordinator();
+ Pair<Boolean, IPartitioningProperty> pbpp = prc.coordinateRequirements(
+ reqdProperties[i].getPartitioningProperty(), firstDeliveredPartitioning, op, context);
+ boolean mayExpandPartitioningProperties = pbpp.first;
+ IPhysicalPropertiesVector rqd = new StructuralPropertiesVector(pbpp.second,
+ reqdProperties[i].getLocalProperties());
+
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Required properties for " + child.getPhysicalOperator()
+ + ": " + rqd + "\n");
+ IPhysicalPropertiesVector diff = delivered.getUnsatisfiedPropertiesFrom(rqd,
+ mayExpandPartitioningProperties, context.getEquivalenceClassMap(child), context.getFDList(child));
+
+ if (isRedundantSort(opRef, delivered, diff, context)) {
+ opIsRedundantSort = true;
+ }
+
+ if (diff != null) {
+ changed = true;
+ addEnforcers(op, i, diff, rqd, delivered, childrenDomain, nestedPlan, context);
+
+ AbstractLogicalOperator newChild = ((AbstractLogicalOperator) op.getInputs().get(i).getValue());
+
+ if (newChild != child) {
+ delivered = newChild.getDeliveredPhysicalProperties();
+ IPhysicalPropertiesVector newDiff = newPropertiesDiff(newChild, rqd,
+ mayExpandPartitioningProperties, context);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> New properties diff: " + newDiff + "\n");
+
+ if (isRedundantSort(opRef, delivered, newDiff, context)) {
+ opIsRedundantSort = true;
+ break;
+ }
+ }
+
+ }
+ if (firstDeliveredPartitioning == null) {
+ IPartitioningProperty dpp = delivered.getPartitioningProperty();
+ if (dpp.getPartitioningType() == PartitioningType.ORDERED_PARTITIONED
+ || dpp.getPartitioningType() == PartitioningType.UNORDERED_PARTITIONED) {
+ firstDeliveredPartitioning = dpp;
+ }
+ }
+
+ i++;
+ }
+
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans nested = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan p : nested.getNestedPlans()) {
+ if (physOptimizePlan(p, required, true, context)) {
+ changed = true;
+ }
+ }
+ }
+
+ if (opIsRedundantSort) {
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Removing redundant SORT operator "
+ + op.getPhysicalOperator() + "\n");
+ printOp((AbstractLogicalOperator) op);
+ }
+ changed = true;
+ AbstractLogicalOperator nextOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ if (nextOp.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+ nextOp = (AbstractLogicalOperator) nextOp.getInputs().get(0).getValue();
+ }
+ opRef.setValue(nextOp);
+ // Now, transfer annotations from the original sort op. to this one.
+ AbstractLogicalOperator transferTo = nextOp;
+ if (transferTo.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ transferTo = (AbstractLogicalOperator) transferTo.getInputs().get(0).getValue();
+ }
+ transferTo.getAnnotations().putAll(op.getAnnotations());
+ physOptimizeOp(opRef, required, nestedPlan, context);
+ }
+ return changed;
+ }
+
+ private IPhysicalPropertiesVector newPropertiesDiff(AbstractLogicalOperator newChild,
+ IPhysicalPropertiesVector required, boolean mayExpandPartitioningProperties, IOptimizationContext context)
+ throws AlgebricksException {
+ IPhysicalPropertiesVector newDelivered = newChild.getDeliveredPhysicalProperties();
+
+ Map<LogicalVariable, EquivalenceClass> newChildEqClasses = context.getEquivalenceClassMap(newChild);
+ List<FunctionalDependency> newChildFDs = context.getFDList(newChild);
+ if (newChildEqClasses == null || newChildFDs == null) {
+ FDsAndEquivClassesVisitor fdsVisitor = new FDsAndEquivClassesVisitor();
+ newChild.accept(fdsVisitor, context);
+ newChildEqClasses = context.getEquivalenceClassMap(newChild);
+ newChildFDs = context.getFDList(newChild);
+ }
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Required properties for new op. "
+ + newChild.getPhysicalOperator() + ": " + required + "\n");
+
+ return newDelivered.getUnsatisfiedPropertiesFrom(required, mayExpandPartitioningProperties, newChildEqClasses,
+ newChildFDs);
+ }
+
+ private void optimizeUsingConstraintsAndEquivClasses(AbstractLogicalOperator op) {
+ IPhysicalOperator pOp = op.getPhysicalOperator();
+ switch (pOp.getOperatorTag()) {
+ case HASH_GROUP_BY:
+ case EXTERNAL_GROUP_BY: {
+ GroupByOperator gby = (GroupByOperator) op;
+ ExternalGroupByPOperator hgbyOp = (ExternalGroupByPOperator) pOp;
+ hgbyOp.computeColumnSet(gby.getGroupByList());
+ break;
+ }
+ case PRE_CLUSTERED_GROUP_BY: {
+ GroupByOperator gby = (GroupByOperator) op;
+ PreclusteredGroupByPOperator preSortedGby = (PreclusteredGroupByPOperator) pOp;
+ preSortedGby.setGbyColumns(gby.getGbyVarList());
+ break;
+ }
+ case PRE_SORTED_DISTINCT_BY: {
+ DistinctOperator d = (DistinctOperator) op;
+ PreSortedDistinctByPOperator preSortedDistinct = (PreSortedDistinctByPOperator) pOp;
+ preSortedDistinct.setDistinctByColumns(d.getDistinctByVarList());
+ break;
+ }
+ }
+ }
+
+ private List<OrderColumn> getOrderColumnsFromGroupingProperties(List<ILocalStructuralProperty> reqd,
+ List<ILocalStructuralProperty> dlvd) {
+ List<OrderColumn> returnedProperties = new ArrayList<OrderColumn>();
+ List<LogicalVariable> rqdCols = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> dlvdCols = new ArrayList<LogicalVariable>();
+ for (ILocalStructuralProperty r : reqd) {
+ r.getVariables(rqdCols);
+ }
+ for (ILocalStructuralProperty d : dlvd) {
+ d.getVariables(dlvdCols);
+ }
+
+ int prefix = dlvdCols.size() - 1;
+ for (; prefix >= 0;)
+ if (!rqdCols.contains(dlvdCols.get(prefix)))
+ prefix--;
+ else
+ break;
+ for (int j = 0; j <= prefix; j++) {
+ LocalOrderProperty orderProp = (LocalOrderProperty) dlvd.get(j);
+ returnedProperties.add(new OrderColumn(orderProp.getColumn(), orderProp.getOrder()));
+ }
+ return returnedProperties;
+ }
+
+ /*
+ * We assume delivered to be already normalized.
+ */
+ private boolean isRedundantSort(Mutable<ILogicalOperator> opRef, IPhysicalPropertiesVector delivered,
+ IPhysicalPropertiesVector diffOfProperties, IOptimizationContext context) {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.ORDER
+ || (op.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.STABLE_SORT && op
+ .getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.IN_MEMORY_STABLE_SORT)
+ || delivered.getLocalProperties() == null) {
+ return false;
+ }
+ AbstractStableSortPOperator sortOp = (AbstractStableSortPOperator) op.getPhysicalOperator();
+ sortOp.computeLocalProperties(op);
+ List<ILocalStructuralProperty> orderProps = sortOp.getOrderProperties();
+ return PropertiesUtil.matchLocalProperties(orderProps, delivered.getLocalProperties(),
+ context.getEquivalenceClassMap(op), context.getFDList(op));
+ }
+
+ private void addEnforcers(AbstractLogicalOperator op, int childIndex,
+ IPhysicalPropertiesVector diffPropertiesVector, IPhysicalPropertiesVector required,
+ IPhysicalPropertiesVector deliveredByChild, INodeDomain domain, boolean nestedPlan,
+ IOptimizationContext context) throws AlgebricksException {
+
+ IPartitioningProperty pp = diffPropertiesVector.getPartitioningProperty();
+ if (pp == null || pp.getPartitioningType() == PartitioningType.UNPARTITIONED) {
+ addLocalEnforcers(op, childIndex, diffPropertiesVector.getLocalProperties(), nestedPlan, context);
+ IPhysicalPropertiesVector deliveredByNewChild = ((AbstractLogicalOperator) op.getInputs().get(0).getValue())
+ .getDeliveredPhysicalProperties();
+ addPartitioningEnforcers(op, childIndex, pp, required, deliveredByNewChild, domain, context);
+ } else {
+ addPartitioningEnforcers(op, childIndex, pp, required, deliveredByChild, domain, context);
+ AbstractLogicalOperator newChild = (AbstractLogicalOperator) op.getInputs().get(childIndex).getValue();
+ IPhysicalPropertiesVector newDiff = newPropertiesDiff(newChild, required, true, context);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> New properties diff: " + newDiff + "\n");
+ if (newDiff != null) {
+ addLocalEnforcers(op, childIndex, newDiff.getLocalProperties(), nestedPlan, context);
+ }
+ }
+ }
+
+ private void addLocalEnforcers(AbstractLogicalOperator op, int i, List<ILocalStructuralProperty> localProperties,
+ boolean nestedPlan, IOptimizationContext context) throws AlgebricksException {
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Adding local enforcers for local props = " + localProperties
+ + "\n");
+ }
+
+ if (localProperties == null || localProperties.isEmpty()) {
+ return;
+ }
+
+ Mutable<ILogicalOperator> topOp = new MutableObject<ILogicalOperator>();
+ topOp.setValue(op.getInputs().get(i).getValue());
+ LinkedList<LocalOrderProperty> oList = new LinkedList<LocalOrderProperty>();
+
+ for (ILocalStructuralProperty prop : localProperties) {
+ switch (prop.getPropertyType()) {
+ case LOCAL_ORDER_PROPERTY: {
+ oList.add((LocalOrderProperty) prop);
+ break;
+ }
+ case LOCAL_GROUPING_PROPERTY: {
+ LocalGroupingProperty g = (LocalGroupingProperty) prop;
+ Collection<LogicalVariable> vars = (g.getPreferredOrderEnforcer() != null) ? g
+ .getPreferredOrderEnforcer() : g.getColumnSet();
+ for (LogicalVariable v : vars) {
+ OrderColumn oc = new OrderColumn(v, OrderKind.ASC);
+ LocalOrderProperty lop = new LocalOrderProperty(oc);
+ oList.add(lop);
+ }
+ break;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ }
+ if (!oList.isEmpty()) {
+ topOp = enforceOrderProperties(oList, topOp, nestedPlan, context);
+ }
+
+ op.getInputs().set(i, topOp);
+ OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull((AbstractLogicalOperator) topOp.getValue(), context);
+ printOp((AbstractLogicalOperator) topOp.getValue());
+ }
+
+ private Mutable<ILogicalOperator> enforceOrderProperties(List<LocalOrderProperty> oList,
+ Mutable<ILogicalOperator> topOp, boolean isMicroOp, IOptimizationContext context)
+ throws AlgebricksException {
+ List<Pair<IOrder, Mutable<ILogicalExpression>>> oe = new LinkedList<Pair<IOrder, Mutable<ILogicalExpression>>>();
+ for (LocalOrderProperty o : oList) {
+ IOrder ordType = (o.getOrder() == OrderKind.ASC) ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
+ Pair<IOrder, Mutable<ILogicalExpression>> pair = new Pair<IOrder, Mutable<ILogicalExpression>>(ordType,
+ new MutableObject<ILogicalExpression>(new VariableReferenceExpression(o.getColumn())));
+ oe.add(pair);
+ }
+ OrderOperator oo = new OrderOperator(oe);
+ oo.setExecutionMode(AbstractLogicalOperator.ExecutionMode.LOCAL);
+ if (isMicroOp) {
+ oo.setPhysicalOperator(new InMemoryStableSortPOperator());
+ } else {
+ oo.setPhysicalOperator(new StableSortPOperator(physicalOptimizationConfig.getMaxFramesExternalSort()));
+ }
+ oo.getInputs().add(topOp);
+ context.computeAndSetTypeEnvironmentForOperator(oo);
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Added sort enforcer " + oo.getPhysicalOperator() + ".\n");
+ }
+ return new MutableObject<ILogicalOperator>(oo);
+ }
+
+ private void addPartitioningEnforcers(ILogicalOperator op, int i, IPartitioningProperty pp,
+ IPhysicalPropertiesVector required, IPhysicalPropertiesVector deliveredByChild, INodeDomain domain,
+ IOptimizationContext context) throws AlgebricksException {
+ if (pp != null) {
+ IPhysicalOperator pop;
+ switch (pp.getPartitioningType()) {
+ case UNPARTITIONED: {
+ List<OrderColumn> ordCols = computeOrderColumns(deliveredByChild);
+ if (ordCols == null || ordCols.size() == 0) {
+ pop = new RandomMergeExchangePOperator();
+ } else {
+ OrderColumn[] sortColumns = new OrderColumn[ordCols.size()];
+ sortColumns = ordCols.toArray(sortColumns);
+ pop = new SortMergeExchangePOperator(sortColumns);
+ }
+ break;
+ }
+ case UNORDERED_PARTITIONED: {
+ List<LogicalVariable> varList = new ArrayList<LogicalVariable>(
+ ((UnorderedPartitionedProperty) pp).getColumnSet());
+ List<ILocalStructuralProperty> cldLocals = deliveredByChild.getLocalProperties();
+ List<ILocalStructuralProperty> reqdLocals = required.getLocalProperties();
+ boolean propWasSet = false;
+ pop = null;
+ if (reqdLocals != null && cldLocals != null && allAreOrderProps(cldLocals)) {
+ AbstractLogicalOperator c = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
+ Map<LogicalVariable, EquivalenceClass> ecs = context.getEquivalenceClassMap(c);
+ List<FunctionalDependency> fds = context.getFDList(c);
+ if (PropertiesUtil.matchLocalProperties(reqdLocals, cldLocals, ecs, fds)) {
+ List<OrderColumn> orderColumns = getOrderColumnsFromGroupingProperties(reqdLocals,
+ cldLocals);
+ pop = new HashPartitionMergeExchangePOperator(orderColumns, varList, domain);
+ propWasSet = true;
+ }
+ }
+ if (!propWasSet) {
+ pop = new HashPartitionExchangePOperator(varList, domain);
+ }
+ break;
+ }
+ case ORDERED_PARTITIONED: {
+ pop = new RangePartitionPOperator(((OrderedPartitionedProperty) pp).getOrderColumns(), domain);
+ break;
+ }
+ case BROADCAST: {
+ pop = new BroadcastPOperator(domain);
+ break;
+ }
+ case RANDOM: {
+ RandomPartitioningProperty rpp = (RandomPartitioningProperty) pp;
+ INodeDomain nd = rpp.getNodeDomain();
+ if (nd == null) {
+ throw new AlgebricksException("Unknown node domain for " + rpp);
+ }
+ if (nd.cardinality() == null) {
+ throw new AlgebricksException("Unknown cardinality for node domain " + nd);
+ }
+ if (nd.cardinality() != 1) {
+ throw new NotImplementedException(
+ "Random repartitioning is only implemented for target domains of"
+ + "cardinality equal to 1.");
+ }
+ pop = new BroadcastPOperator(nd);
+ break;
+ }
+ default: {
+ throw new NotImplementedException("Enforcer for " + pp.getPartitioningType()
+ + " partitioning type has not been implemented.");
+ }
+ }
+ Mutable<ILogicalOperator> ci = op.getInputs().get(i);
+ ExchangeOperator exchg = new ExchangeOperator();
+ exchg.setPhysicalOperator(pop);
+ setNewOp(ci, exchg, context);
+ exchg.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+ OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull(exchg, context);
+ context.computeAndSetTypeEnvironmentForOperator(exchg);
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Added partitioning enforcer "
+ + exchg.getPhysicalOperator() + ".\n");
+ printOp((AbstractLogicalOperator) op);
+ }
+ }
+ }
+
+ private boolean allAreOrderProps(List<ILocalStructuralProperty> cldLocals) {
+ for (ILocalStructuralProperty lsp : cldLocals) {
+ if (lsp.getPropertyType() != PropertyType.LOCAL_ORDER_PROPERTY) {
+ return false;
+ }
+ }
+ return !cldLocals.isEmpty();
+ }
+
+ private void printOp(AbstractLogicalOperator op) throws AlgebricksException {
+ StringBuilder sb = new StringBuilder();
+ LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
+ PlanPrettyPrinter.printOperator(op, sb, pvisitor, 0);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(sb.toString());
+ }
+
+ private List<OrderColumn> computeOrderColumns(IPhysicalPropertiesVector pv) {
+ List<OrderColumn> ordCols = new ArrayList<OrderColumn>();
+ List<ILocalStructuralProperty> localProps = pv.getLocalProperties();
+ if (localProps == null || localProps.size() == 0) {
+ return null;
+ } else {
+ for (ILocalStructuralProperty p : localProps) {
+ if (p.getPropertyType() == PropertyType.LOCAL_ORDER_PROPERTY) {
+ LocalOrderProperty lop = (LocalOrderProperty) p;
+ ordCols.add(lop.getOrderColumn());
+ } else {
+ return null;
+ }
+ }
+ return ordCols;
+ }
+
+ }
+
+ private void setNewOp(Mutable<ILogicalOperator> opRef, AbstractLogicalOperator newOp, IOptimizationContext context)
+ throws AlgebricksException {
+ ILogicalOperator oldOp = opRef.getValue();
+ opRef.setValue(newOp);
+ newOp.getInputs().add(new MutableObject<ILogicalOperator>(oldOp));
+ newOp.recomputeSchema();
+ newOp.computeDeliveredPhysicalProperties(context);
+ context.computeAndSetTypeEnvironmentForOperator(newOp);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Structural properties for " + newOp.getPhysicalOperator()
+ + ": " + newOp.getDeliveredPhysicalProperties() + "\n");
+
+ PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(newOp, context);
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
new file mode 100644
index 0000000..7e27b79
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
@@ -0,0 +1,391 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+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.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.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.IsomorphismUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.ReplicatePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class ExtractCommonOperatorsRule implements IAlgebraicRewriteRule {
+
+ private HashMap<Mutable<ILogicalOperator>, List<Mutable<ILogicalOperator>>> childrenToParents = new HashMap<Mutable<ILogicalOperator>, List<Mutable<ILogicalOperator>>>();
+ private List<Mutable<ILogicalOperator>> roots = new ArrayList<Mutable<ILogicalOperator>>();
+ private List<Mutable<ILogicalOperator>> joins = new ArrayList<Mutable<ILogicalOperator>>();
+ private List<List<Mutable<ILogicalOperator>>> equivalenceClasses = new ArrayList<List<Mutable<ILogicalOperator>>>();
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.WRITE && op.getOperatorTag() != LogicalOperatorTag.WRITE_RESULT) {
+ return false;
+ }
+ if (!roots.contains(op))
+ roots.add(new MutableObject<ILogicalOperator>(op));
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.WRITE && op.getOperatorTag() != LogicalOperatorTag.WRITE_RESULT) {
+ return false;
+ }
+ boolean rewritten = false;
+ boolean changed = false;
+ if (roots.size() > 0) {
+ do {
+ changed = false;
+ // applying the rewriting until fixpoint
+ topDownMaterialization(roots);
+ removeNonJoinBuildBranchCandidates();
+ genCandidates(context);
+ removeTrivialShare();
+ removeNonJoinBuildBranchCandidates();
+ if (equivalenceClasses.size() > 0)
+ changed = rewrite(context);
+ if (!rewritten)
+ rewritten = changed;
+ equivalenceClasses.clear();
+ childrenToParents.clear();
+ joins.clear();
+ } while (changed);
+ roots.clear();
+ }
+ return rewritten;
+ }
+
+ private void removeTrivialShare() {
+ for (List<Mutable<ILogicalOperator>> candidates : equivalenceClasses) {
+ for (int i = candidates.size() - 1; i >= 0; i--) {
+ Mutable<ILogicalOperator> opRef = candidates.get(i);
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) opRef.getValue();
+ if (aop.getOperatorTag() == LogicalOperatorTag.EXCHANGE)
+ aop = (AbstractLogicalOperator) aop.getInputs().get(0).getValue();
+ if (aop.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE)
+ candidates.remove(i);
+ }
+ }
+ for (int i = equivalenceClasses.size() - 1; i >= 0; i--)
+ if (equivalenceClasses.get(i).size() < 2)
+ equivalenceClasses.remove(i);
+ }
+
+ private void removeNonJoinBuildBranchCandidates() {
+ for (List<Mutable<ILogicalOperator>> candidates : equivalenceClasses) {
+ for (int i = candidates.size() - 1; i >= 0; i--) {
+ Mutable<ILogicalOperator> opRef = candidates.get(i);
+ boolean reserve = false;
+ for (Mutable<ILogicalOperator> join : joins)
+ if (isInJoinBuildBranch(join, opRef)) {
+ reserve = true;
+ }
+ if (!reserve)
+ candidates.remove(i);
+ }
+ }
+ for (int i = equivalenceClasses.size() - 1; i >= 0; i--)
+ if (equivalenceClasses.get(i).size() < 2)
+ equivalenceClasses.remove(i);
+ }
+
+ private boolean isInJoinBuildBranch(Mutable<ILogicalOperator> joinRef, Mutable<ILogicalOperator> opRef) {
+ Mutable<ILogicalOperator> buildBranch = joinRef.getValue().getInputs().get(1);
+ do {
+ if (buildBranch.equals(opRef)) {
+ return true;
+ } else {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) buildBranch.getValue();
+ if (aop.getOperatorTag() == LogicalOperatorTag.INNERJOIN
+ || aop.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN
+ || buildBranch.getValue().getInputs().size() == 0)
+ return false;
+ else
+ buildBranch = buildBranch.getValue().getInputs().get(0);
+ }
+ } while (true);
+ }
+
+ private boolean rewrite(IOptimizationContext context) throws AlgebricksException {
+ boolean changed = false;
+ for (List<Mutable<ILogicalOperator>> members : equivalenceClasses) {
+ if (rewriteForOneEquivalentClass(members, context))
+ changed = true;
+ }
+ return changed;
+ }
+
+ private boolean rewriteForOneEquivalentClass(List<Mutable<ILogicalOperator>> members, IOptimizationContext context)
+ throws AlgebricksException {
+ List<Mutable<ILogicalOperator>> group = new ArrayList<Mutable<ILogicalOperator>>();
+ boolean rewritten = false;
+ while (members.size() > 0) {
+ group.clear();
+ Mutable<ILogicalOperator> candidate = members.remove(members.size() - 1);
+ group.add(candidate);
+ for (int i = members.size() - 1; i >= 0; i--) {
+ Mutable<ILogicalOperator> peer = members.get(i);
+ if (IsomorphismUtilities.isOperatorIsomorphic(candidate.getValue(), peer.getValue())) {
+ group.add(peer);
+ members.remove(i);
+ }
+ }
+ AbstractLogicalOperator rop = new ReplicateOperator(group.size());
+ rop.setPhysicalOperator(new ReplicatePOperator());
+ rop.setExecutionMode(ExecutionMode.PARTITIONED);
+ Mutable<ILogicalOperator> ropRef = new MutableObject<ILogicalOperator>(rop);
+ AbstractLogicalOperator aopCandidate = (AbstractLogicalOperator) candidate.getValue();
+
+ if (aopCandidate.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ rop.getInputs().add(candidate);
+ } else {
+ AbstractLogicalOperator beforeExchange = new ExchangeOperator();
+ beforeExchange.setPhysicalOperator(new OneToOneExchangePOperator());
+ beforeExchange.getInputs().add(candidate);
+ context.computeAndSetTypeEnvironmentForOperator(beforeExchange);
+ rop.getInputs().add(new MutableObject<ILogicalOperator>(beforeExchange));
+ }
+ context.computeAndSetTypeEnvironmentForOperator(rop);
+
+ List<Mutable<ILogicalOperator>> parents = childrenToParents.get(candidate);
+ for (Mutable<ILogicalOperator> parentRef : parents) {
+ AbstractLogicalOperator parent = (AbstractLogicalOperator) parentRef.getValue();
+ int index = parent.getInputs().indexOf(candidate);
+ if (parent.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ parent.getInputs().set(index, ropRef);
+ } else {
+ AbstractLogicalOperator exchange = new ExchangeOperator();
+ exchange.setPhysicalOperator(new OneToOneExchangePOperator());
+ exchange.getInputs().add(ropRef);
+ context.computeAndSetTypeEnvironmentForOperator(exchange);
+ // parent.getInputs().get(index).setValue(exchange);
+ parent.getInputs().set(index, new MutableObject<ILogicalOperator>(exchange));
+ context.computeAndSetTypeEnvironmentForOperator(parent);
+ }
+ }
+
+ List<LogicalVariable> liveVarsNew = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(candidate.getValue(), liveVarsNew);
+ ArrayList<Mutable<ILogicalExpression>> assignExprs = new ArrayList<Mutable<ILogicalExpression>>();
+ for (LogicalVariable liveVar : liveVarsNew)
+ assignExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(liveVar)));
+ for (Mutable<ILogicalOperator> ref : group) {
+ if (ref.equals(candidate))
+ continue;
+ ArrayList<LogicalVariable> liveVars = new ArrayList<LogicalVariable>();
+ Map<LogicalVariable, LogicalVariable> variableMappingBack = new HashMap<LogicalVariable, LogicalVariable>();
+ IsomorphismUtilities.mapVariablesTopDown(ref.getValue(), candidate.getValue(), variableMappingBack);
+ for (int i = 0; i < liveVarsNew.size(); i++) {
+ liveVars.add(variableMappingBack.get(liveVarsNew.get(i)));
+ }
+
+ AbstractLogicalOperator assignOperator = new AssignOperator(liveVars, assignExprs);
+ assignOperator.setPhysicalOperator(new AssignPOperator());
+ AbstractLogicalOperator projectOperator = new ProjectOperator(liveVars);
+ projectOperator.setPhysicalOperator(new StreamProjectPOperator());
+ AbstractLogicalOperator exchOp = new ExchangeOperator();
+ exchOp.setPhysicalOperator(new OneToOneExchangePOperator());
+ exchOp.getInputs().add(ropRef);
+
+ assignOperator.getInputs().add(new MutableObject<ILogicalOperator>(exchOp));
+ projectOperator.getInputs().add(new MutableObject<ILogicalOperator>(assignOperator));
+ // set the types
+ context.computeAndSetTypeEnvironmentForOperator(exchOp);
+ context.computeAndSetTypeEnvironmentForOperator(assignOperator);
+ context.computeAndSetTypeEnvironmentForOperator(projectOperator);
+
+ List<Mutable<ILogicalOperator>> parentOpList = childrenToParents.get(ref);
+ for (Mutable<ILogicalOperator> parentOpRef : parentOpList) {
+ AbstractLogicalOperator parentOp = (AbstractLogicalOperator) parentOpRef.getValue();
+ int index = parentOp.getInputs().indexOf(ref);
+ if (parentOp.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ AbstractLogicalOperator parentOpNext = (AbstractLogicalOperator) childrenToParents
+ .get(parentOpRef).get(0).getValue();
+ if (parentOpNext.isMap()) {
+ index = parentOpNext.getInputs().indexOf(parentOpRef);
+ parentOp = parentOpNext;
+ }
+ }
+
+ AbstractLogicalOperator exchg = new ExchangeOperator();
+ exchg.setPhysicalOperator(new OneToOneExchangePOperator());
+
+ ILogicalOperator childOp = parentOp.getOperatorTag() == LogicalOperatorTag.PROJECT ? assignOperator
+ : projectOperator;
+ if (parentOp.isMap()) {
+ parentOp.getInputs().set(index, new MutableObject<ILogicalOperator>(childOp));
+ } else {
+ exchg.getInputs().add(new MutableObject<ILogicalOperator>(childOp));
+ parentOp.getInputs().set(index, new MutableObject<ILogicalOperator>(exchg));
+ }
+ context.computeAndSetTypeEnvironmentForOperator(exchg);
+ }
+ }
+ rewritten = true;
+ }
+ return rewritten;
+ }
+
+ private void genCandidates(IOptimizationContext context) throws AlgebricksException {
+ List<List<Mutable<ILogicalOperator>>> previousEquivalenceClasses = new ArrayList<List<Mutable<ILogicalOperator>>>();
+ while (equivalenceClasses.size() > 0) {
+ previousEquivalenceClasses.clear();
+ for (List<Mutable<ILogicalOperator>> candidates : equivalenceClasses) {
+ List<Mutable<ILogicalOperator>> candidatesCopy = new ArrayList<Mutable<ILogicalOperator>>();
+ candidatesCopy.addAll(candidates);
+ previousEquivalenceClasses.add(candidatesCopy);
+ }
+ List<Mutable<ILogicalOperator>> currentLevelOpRefs = new ArrayList<Mutable<ILogicalOperator>>();
+ for (List<Mutable<ILogicalOperator>> candidates : equivalenceClasses) {
+ if (candidates.size() > 0) {
+ for (Mutable<ILogicalOperator> opRef : candidates) {
+ List<Mutable<ILogicalOperator>> refs = childrenToParents.get(opRef);
+ if (refs != null)
+ currentLevelOpRefs.addAll(refs);
+ }
+ }
+ if (currentLevelOpRefs.size() == 0)
+ continue;
+ candidatesGrow(currentLevelOpRefs, candidates);
+ }
+ if (currentLevelOpRefs.size() == 0)
+ break;
+ prune(context);
+ }
+ if (equivalenceClasses.size() < 1 && previousEquivalenceClasses.size() > 0) {
+ equivalenceClasses.addAll(previousEquivalenceClasses);
+ prune(context);
+ }
+ }
+
+ private void topDownMaterialization(List<Mutable<ILogicalOperator>> tops) {
+ List<Mutable<ILogicalOperator>> candidates = new ArrayList<Mutable<ILogicalOperator>>();
+ List<Mutable<ILogicalOperator>> nextLevel = new ArrayList<Mutable<ILogicalOperator>>();
+ for (Mutable<ILogicalOperator> op : tops) {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) op.getValue();
+ if ((aop.getOperatorTag() == LogicalOperatorTag.INNERJOIN || aop.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN)
+ && !joins.contains(op)) {
+ joins.add(op);
+ }
+ for (Mutable<ILogicalOperator> opRef : op.getValue().getInputs()) {
+ List<Mutable<ILogicalOperator>> opRefList = childrenToParents.get(opRef);
+ if (opRefList == null) {
+ opRefList = new ArrayList<Mutable<ILogicalOperator>>();
+ childrenToParents.put(opRef, opRefList);
+ nextLevel.add(opRef);
+ }
+ opRefList.add(op);
+ }
+ if (op.getValue().getInputs().size() == 0)
+ candidates.add(op);
+ }
+ if (equivalenceClasses.size() > 0) {
+ equivalenceClasses.get(0).addAll(candidates);
+ } else {
+ equivalenceClasses.add(candidates);
+ }
+ if (nextLevel.size() > 0) {
+ topDownMaterialization(nextLevel);
+ }
+ }
+
+ private void candidatesGrow(List<Mutable<ILogicalOperator>> opList, List<Mutable<ILogicalOperator>> candidates) {
+ List<Mutable<ILogicalOperator>> previousCandidates = new ArrayList<Mutable<ILogicalOperator>>();
+ previousCandidates.addAll(candidates);
+ candidates.clear();
+ boolean validCandidate = false;
+ for (Mutable<ILogicalOperator> op : opList) {
+ for (Mutable<ILogicalOperator> inputRef : op.getValue().getInputs()) {
+ validCandidate = false;
+ // if current input is in candidates
+ for (Mutable<ILogicalOperator> candidate : previousCandidates)
+ if (inputRef.getValue().equals(candidate.getValue()))
+ validCandidate = true;
+ // if one input is not in candidates
+ if (!validCandidate)
+ break;
+ }
+ if (!validCandidate)
+ continue;
+ candidates.add(op);
+ }
+ }
+
+ private void prune(IOptimizationContext context) throws AlgebricksException {
+ List<List<Mutable<ILogicalOperator>>> previousEquivalenceClasses = new ArrayList<List<Mutable<ILogicalOperator>>>();
+ for (List<Mutable<ILogicalOperator>> candidates : equivalenceClasses) {
+ List<Mutable<ILogicalOperator>> candidatesCopy = new ArrayList<Mutable<ILogicalOperator>>();
+ candidatesCopy.addAll(candidates);
+ previousEquivalenceClasses.add(candidatesCopy);
+ }
+ equivalenceClasses.clear();
+ for (List<Mutable<ILogicalOperator>> candidates : previousEquivalenceClasses) {
+ boolean[] reserved = new boolean[candidates.size()];
+ for (int i = 0; i < reserved.length; i++)
+ reserved[i] = false;
+ for (int i = candidates.size() - 1; i >= 0; i--) {
+ if (reserved[i] == false) {
+ List<Mutable<ILogicalOperator>> equivalentClass = new ArrayList<Mutable<ILogicalOperator>>();
+ ILogicalOperator candidate = candidates.get(i).getValue();
+ equivalentClass.add(candidates.get(i));
+ for (int j = i - 1; j >= 0; j--) {
+ ILogicalOperator peer = candidates.get(j).getValue();
+ if (IsomorphismUtilities.isOperatorIsomorphic(candidate, peer)) {
+ reserved[i] = true;
+ reserved[j] = true;
+ equivalentClass.add(candidates.get(j));
+ }
+ }
+ if (equivalentClass.size() > 1) {
+ equivalenceClasses.add(equivalentClass);
+ Collections.reverse(equivalentClass);
+ }
+ }
+ }
+ for (int i = candidates.size() - 1; i >= 0; i--) {
+ if (!reserved[i]) {
+ candidates.remove(i);
+ }
+ }
+ }
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
new file mode 100644
index 0000000..8305bc9
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+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.ILogicalExpression;
+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.LogicalExpressionTag;
+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.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+
+/**
+ * Needed only bc. current Hyrax operators require keys to be fields.
+ */
+public class ExtractGbyExpressionsRule extends AbstractExtractExprRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
+ if (op1.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+
+ if (context.checkIfInDontApplySet(this, op1)) {
+ return false;
+ }
+ context.addToDontApplySet(this, op1);
+ GroupByOperator g = (GroupByOperator) op1;
+ boolean r1 = gbyExprWasRewritten(g, context);
+ boolean r2 = decorExprWasRewritten(g, context);
+ boolean fired = r1 || r2;
+ if (fired) {
+ context.computeAndSetTypeEnvironmentForOperator(g);
+ }
+ return fired;
+ }
+
+ private boolean gbyExprWasRewritten(GroupByOperator g, IOptimizationContext context) throws AlgebricksException {
+ if (!gbyHasComplexExpr(g)) {
+ return false;
+ }
+ Mutable<ILogicalOperator> opRef2 = g.getInputs().get(0);
+ for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gbyPair : g.getGroupByList()) {
+ ILogicalExpression expr = gbyPair.second.getValue();
+ if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
+ gbyPair.second.setValue(new VariableReferenceExpression(v));
+ }
+ }
+ return true;
+ }
+
+ private boolean decorExprWasRewritten(GroupByOperator g, IOptimizationContext context) throws AlgebricksException {
+ if (!decorHasComplexExpr(g)) {
+ return false;
+ }
+ Mutable<ILogicalOperator> opRef2 = g.getInputs().get(0);
+ for (Pair<LogicalVariable, Mutable<ILogicalExpression>> decorPair : g.getDecorList()) {
+ ILogicalExpression expr = decorPair.second.getValue();
+ if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
+ decorPair.second.setValue(new VariableReferenceExpression(v));
+ }
+ }
+ return true;
+ }
+
+ private boolean gbyHasComplexExpr(GroupByOperator g) {
+ for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gbyPair : g.getGroupByList()) {
+ if (gbyPair.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private boolean decorHasComplexExpr(GroupByOperator g) {
+ for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gbyPair : g.getDecorList()) {
+ if (gbyPair.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java
new file mode 100644
index 0000000..bfaff9f
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+
+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.common.utils.Pair;
+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.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+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.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class FactorRedundantGroupAndDecorVarsRule 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 {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+ GroupByOperator gby = (GroupByOperator) op;
+ Map<LogicalVariable, LogicalVariable> varRhsToLhs = new HashMap<LogicalVariable, LogicalVariable>();
+ boolean gvChanged = factorRedundantRhsVars(gby.getGroupByList(), opRef, varRhsToLhs, context);
+ boolean dvChanged = factorRedundantRhsVars(gby.getDecorList(), opRef, varRhsToLhs, context);
+
+ return gvChanged || dvChanged;
+ }
+
+ private boolean factorRedundantRhsVars(List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> veList,
+ Mutable<ILogicalOperator> opRef, Map<LogicalVariable, LogicalVariable> varRhsToLhs,
+ IOptimizationContext context) throws AlgebricksException {
+ varRhsToLhs.clear();
+ ListIterator<Pair<LogicalVariable, Mutable<ILogicalExpression>>> iter = veList.listIterator();
+ boolean changed = false;
+ while (iter.hasNext()) {
+ Pair<LogicalVariable, Mutable<ILogicalExpression>> p = iter.next();
+ if (p.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ continue;
+ }
+ LogicalVariable v = GroupByOperator.getDecorVariable(p);
+ LogicalVariable lhs = varRhsToLhs.get(v);
+ if (lhs != null) {
+ if (p.first != null) {
+ AssignOperator assign = new AssignOperator(p.first, new MutableObject<ILogicalExpression>(
+ new VariableReferenceExpression(lhs)));
+ ILogicalOperator op = opRef.getValue();
+ assign.getInputs().add(new MutableObject<ILogicalOperator>(op));
+ opRef.setValue(assign);
+ context.computeAndSetTypeEnvironmentForOperator(assign);
+ }
+ iter.remove();
+ changed = true;
+ } else {
+ varRhsToLhs.put(v, p.first);
+ }
+ }
+ return changed;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InferTypesRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InferTypesRule.java
new file mode 100644
index 0000000..703567a
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InferTypesRule.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.rewriter.base.IAlgebraicRewriteRule;
+
+public class InferTypesRule 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 {
+ ILogicalOperator op = opRef.getValue();
+ if (context.getOutputTypeEnvironment(op) != null) {
+ return false;
+ }
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ return true;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineAssignIntoAggregateRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineAssignIntoAggregateRule.java
new file mode 100644
index 0000000..ca45cbb
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineAssignIntoAggregateRule.java
@@ -0,0 +1,122 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+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.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.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.AbstractConstVarFunVisitor;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class InlineAssignIntoAggregateRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+ boolean changed = false;
+ GroupByOperator gbyOp = (GroupByOperator) op;
+ for (ILogicalPlan p : gbyOp.getNestedPlans()) {
+ for (Mutable<ILogicalOperator> r : p.getRoots()) {
+ if (inlined(r)) {
+ changed = true;
+ }
+ }
+ }
+ return changed;
+ }
+
+ private boolean inlined(Mutable<ILogicalOperator> r) throws AlgebricksException {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) r.getValue();
+ if (op1.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
+ if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+ return false;
+ }
+ AggregateOperator agg = (AggregateOperator) op1;
+ AssignOperator assign = (AssignOperator) op2;
+ VarExprSubstitution ves = new VarExprSubstitution(assign.getVariables(), assign.getExpressions());
+ for (Mutable<ILogicalExpression> exprRef : agg.getExpressions()) {
+ ILogicalExpression expr = exprRef.getValue();
+ Pair<Boolean, ILogicalExpression> p = expr.accept(ves, null);
+ if (p.first == true) {
+ exprRef.setValue(p.second);
+ }
+ // AbstractLogicalExpression ale = (AbstractLogicalExpression) expr;
+ // ale.accept(ves, null);
+ }
+ List<Mutable<ILogicalOperator>> op1InpList = op1.getInputs();
+ op1InpList.clear();
+ op1InpList.add(op2.getInputs().get(0));
+ return true;
+ }
+
+ private class VarExprSubstitution extends AbstractConstVarFunVisitor<Pair<Boolean, ILogicalExpression>, Void> {
+
+ private List<LogicalVariable> variables;
+ private List<Mutable<ILogicalExpression>> expressions;
+
+ public VarExprSubstitution(List<LogicalVariable> variables, List<Mutable<ILogicalExpression>> expressions) {
+ this.variables = variables;
+ this.expressions = expressions;
+ }
+
+ @Override
+ public Pair<Boolean, ILogicalExpression> visitConstantExpression(ConstantExpression expr, Void arg) {
+ return new Pair<Boolean, ILogicalExpression>(false, expr);
+ }
+
+ @Override
+ public Pair<Boolean, ILogicalExpression> visitFunctionCallExpression(AbstractFunctionCallExpression expr,
+ Void arg) throws AlgebricksException {
+ boolean changed = false;
+ for (Mutable<ILogicalExpression> eRef : expr.getArguments()) {
+ ILogicalExpression e = eRef.getValue();
+ Pair<Boolean, ILogicalExpression> p = e.accept(this, arg);
+ if (p.first) {
+ eRef.setValue(p.second);
+ changed = true;
+ }
+ }
+ return new Pair<Boolean, ILogicalExpression>(changed, expr);
+ }
+
+ @Override
+ public Pair<Boolean, ILogicalExpression> visitVariableReferenceExpression(VariableReferenceExpression expr,
+ Void arg) {
+ LogicalVariable v = expr.getVariableReference();
+ int idx = variables.indexOf(v);
+ if (idx < 0) {
+ return new Pair<Boolean, ILogicalExpression>(false, expr);
+ } else {
+ return new Pair<Boolean, ILogicalExpression>(true, expressions.get(idx).getValue());
+ }
+
+ }
+
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
new file mode 100644
index 0000000..8a79d81
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
@@ -0,0 +1,344 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+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.EquivalenceClass;
+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.LogicalExpressionTag;
+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.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+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.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class InlineVariablesRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ /**
+ *
+ * Does one big DFS sweep over the plan.
+ *
+ */
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ if (context.checkIfInDontApplySet(this, opRef.getValue())) {
+ return false;
+ }
+ VariableSubstitutionVisitor substVisitor = new VariableSubstitutionVisitor(false);
+ VariableSubstitutionVisitor substVisitorForWrites = new VariableSubstitutionVisitor(true);
+ substVisitor.setContext(context);
+ substVisitorForWrites.setContext(context);
+ Pair<Boolean, Boolean> bb = collectEqClassesAndRemoveRedundantOps(opRef, context, true,
+ new LinkedList<EquivalenceClass>(), substVisitor, substVisitorForWrites);
+ return bb.first;
+ }
+
+ private Pair<Boolean, Boolean> collectEqClassesAndRemoveRedundantOps(Mutable<ILogicalOperator> opRef,
+ IOptimizationContext context, boolean first, List<EquivalenceClass> equivClasses,
+ VariableSubstitutionVisitor substVisitor, VariableSubstitutionVisitor substVisitorForWrites)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ // if (context.checkIfInDontApplySet(this, opRef.getValue())) {
+ // return false;
+ // }
+ if (op.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+ return new Pair<Boolean, Boolean>(false, false);
+ }
+ boolean modified = false;
+ boolean ecChange = false;
+ int cnt = 0;
+ for (Mutable<ILogicalOperator> i : op.getInputs()) {
+ boolean isOuterInputBranch = op.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN && cnt == 1;
+ List<EquivalenceClass> eqc = isOuterInputBranch ? new LinkedList<EquivalenceClass>() : equivClasses;
+
+ Pair<Boolean, Boolean> bb = (collectEqClassesAndRemoveRedundantOps(i, context, false, eqc, substVisitor,
+ substVisitorForWrites));
+
+ if (bb.first) {
+ modified = true;
+ }
+ if (bb.second) {
+ ecChange = true;
+ }
+
+ if (isOuterInputBranch) {
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest("--- Equivalence classes for inner branch of outer op.: "
+ + eqc + "\n");
+ }
+ for (EquivalenceClass ec : eqc) {
+ if (!ec.representativeIsConst()) {
+ equivClasses.add(ec);
+ }
+ }
+ }
+
+ ++cnt;
+ }
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans n = (AbstractOperatorWithNestedPlans) op;
+ List<EquivalenceClass> eqc = equivClasses;
+ if (n.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+ eqc = new LinkedList<EquivalenceClass>();
+ } else {
+ eqc = equivClasses;
+ }
+ for (ILogicalPlan p : n.getNestedPlans()) {
+ for (Mutable<ILogicalOperator> r : p.getRoots()) {
+ Pair<Boolean, Boolean> bb = collectEqClassesAndRemoveRedundantOps(r, context, false, eqc,
+ substVisitor, substVisitorForWrites);
+ if (bb.first) {
+ modified = true;
+ }
+ if (bb.second) {
+ ecChange = true;
+ }
+ }
+ }
+ }
+ // we assume a variable is assigned a value only once
+ if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator a = (AssignOperator) op;
+ ILogicalExpression rhs = a.getExpressions().get(0).getValue();
+ if (rhs.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable varLeft = a.getVariables().get(0);
+ VariableReferenceExpression varRef = (VariableReferenceExpression) rhs;
+ LogicalVariable varRight = varRef.getVariableReference();
+
+ EquivalenceClass ecRight = findEquivClass(varRight, equivClasses);
+ if (ecRight != null) {
+ ecRight.addMember(varLeft);
+ } else {
+ List<LogicalVariable> m = new LinkedList<LogicalVariable>();
+ m.add(varRight);
+ m.add(varLeft);
+ EquivalenceClass ec = new EquivalenceClass(m, varRight);
+ equivClasses.add(ec);
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest("--- New equivalence class: " + ec + "\n");
+ }
+ }
+ ecChange = true;
+ } else if (((AbstractLogicalExpression) rhs).getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ LogicalVariable varLeft = a.getVariables().get(0);
+ List<LogicalVariable> m = new LinkedList<LogicalVariable>();
+ m.add(varLeft);
+ EquivalenceClass ec = new EquivalenceClass(m, (ConstantExpression) rhs);
+ // equivClassesForParent.add(ec);
+ equivClasses.add(ec);
+ ecChange = true;
+ }
+ } else if (op.getOperatorTag() == LogicalOperatorTag.GROUP && !(context.checkIfInDontApplySet(this, op))) {
+ GroupByOperator group = (GroupByOperator) op;
+ Pair<Boolean, Boolean> r1 = processVarExprPairs(group.getGroupByList(), equivClasses);
+ Pair<Boolean, Boolean> r2 = processVarExprPairs(group.getDecorList(), equivClasses);
+ modified = modified || r1.first || r2.first;
+ ecChange = r1.second || r2.second;
+ }
+ if (op.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+ assignVarsNeededByProject((ProjectOperator) op, equivClasses, context);
+ } else {
+ if (op.getOperatorTag() == LogicalOperatorTag.WRITE) {
+ substVisitorForWrites.setEquivalenceClasses(equivClasses);
+ if (op.acceptExpressionTransform(substVisitorForWrites)) {
+ modified = true;
+ }
+ } else {
+ substVisitor.setEquivalenceClasses(equivClasses);
+ if (op.acceptExpressionTransform(substVisitor)) {
+ modified = true;
+ if (op.getOperatorTag() == LogicalOperatorTag.GROUP) {
+ GroupByOperator group = (GroupByOperator) op;
+ for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gp : group.getGroupByList()) {
+ if (gp.first != null
+ && gp.second.getValue().getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable gv = ((VariableReferenceExpression) gp.second.getValue())
+ .getVariableReference();
+ Iterator<Pair<LogicalVariable, Mutable<ILogicalExpression>>> iter = group
+ .getDecorList().iterator();
+ while (iter.hasNext()) {
+ Pair<LogicalVariable, Mutable<ILogicalExpression>> dp = iter.next();
+ if (dp.first == null
+ && dp.second.getValue().getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable dv = ((VariableReferenceExpression) dp.second.getValue())
+ .getVariableReference();
+ if (dv == gv) {
+ // The decor variable is redundant,
+ // since it is
+ // propagated as a grouping
+ // variable.
+ EquivalenceClass ec1 = findEquivClass(gv, equivClasses);
+ if (ec1 != null) {
+ ec1.addMember(gp.first);
+ ec1.setVariableRepresentative(gp.first);
+ } else {
+ List<LogicalVariable> varList = new ArrayList<LogicalVariable>();
+ varList.add(gp.first);
+ varList.add(gv);
+ ec1 = new EquivalenceClass(varList, gp.first);
+ }
+ iter.remove();
+ break;
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ return new Pair<Boolean, Boolean>(modified, ecChange);
+ }
+
+ private Pair<Boolean, Boolean> processVarExprPairs(
+ List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> vePairs, List<EquivalenceClass> equivClasses) {
+ boolean ecFromGroup = false;
+ boolean modified = false;
+ for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : vePairs) {
+ ILogicalExpression expr = p.second.getValue();
+ if (p.first != null && expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
+ LogicalVariable rhsVar = varRef.getVariableReference();
+ ecFromGroup = true;
+ EquivalenceClass ecRight = findEquivClass(rhsVar, equivClasses);
+ if (ecRight != null) {
+ LogicalVariable replacingVar = ecRight.getVariableRepresentative();
+ if (replacingVar != null && replacingVar != rhsVar) {
+ varRef.setVariable(replacingVar);
+ modified = true;
+ }
+ }
+ }
+ }
+ return new Pair<Boolean, Boolean>(modified, ecFromGroup);
+ }
+
+ // Instead of doing this, we could make Projection to be more expressive and
+ // also take constants (or even expression), at the expense of a more
+ // complex project push down.
+ private void assignVarsNeededByProject(ProjectOperator op, List<EquivalenceClass> equivClasses,
+ IOptimizationContext context) throws AlgebricksException {
+ List<LogicalVariable> prVars = op.getVariables();
+ int sz = prVars.size();
+ for (int i = 0; i < sz; i++) {
+ EquivalenceClass ec = findEquivClass(prVars.get(i), equivClasses);
+ if (ec != null) {
+ if (!ec.representativeIsConst()) {
+ prVars.set(i, ec.getVariableRepresentative());
+ }
+ }
+ }
+ }
+
+ private final static EquivalenceClass findEquivClass(LogicalVariable var, List<EquivalenceClass> equivClasses) {
+ for (EquivalenceClass ec : equivClasses) {
+ if (ec.contains(var)) {
+ return ec;
+ }
+ }
+ return null;
+ }
+
+ private class VariableSubstitutionVisitor implements ILogicalExpressionReferenceTransform {
+ private List<EquivalenceClass> equivClasses;
+ private IOptimizationContext context;
+ private final boolean doNotSubstWithConst;
+
+ public VariableSubstitutionVisitor(boolean doNotSubstWithConst) {
+ this.doNotSubstWithConst = doNotSubstWithConst;
+ }
+
+ public void setContext(IOptimizationContext context) {
+ this.context = context;
+ }
+
+ public void setEquivalenceClasses(List<EquivalenceClass> equivClasses) {
+ this.equivClasses = equivClasses;
+ }
+
+ @Override
+ public boolean transform(Mutable<ILogicalExpression> exprRef) {
+ ILogicalExpression e = exprRef.getValue();
+ switch (((AbstractLogicalExpression) e).getExpressionTag()) {
+ case VARIABLE: {
+ // look for a required substitution
+ LogicalVariable var = ((VariableReferenceExpression) e).getVariableReference();
+ if (context.shouldNotBeInlined(var)) {
+ return false;
+ }
+ EquivalenceClass ec = findEquivClass(var, equivClasses);
+ if (ec == null) {
+ return false;
+ }
+ if (ec.representativeIsConst()) {
+ if (doNotSubstWithConst) {
+ return false;
+ }
+ exprRef.setValue(ec.getConstRepresentative());
+ return true;
+ } else {
+ LogicalVariable r = ec.getVariableRepresentative();
+ if (!r.equals(var)) {
+ exprRef.setValue(new VariableReferenceExpression(r));
+ return true;
+ } else {
+ return false;
+ }
+ }
+ }
+ case FUNCTION_CALL: {
+ AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) e;
+ boolean m = false;
+ for (Mutable<ILogicalExpression> arg : fce.getArguments()) {
+ if (transform(arg)) {
+ m = true;
+ }
+ }
+ return m;
+ }
+ default: {
+ return false;
+ }
+ }
+ }
+
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertOuterJoinRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertOuterJoinRule.java
new file mode 100644
index 0000000..bea8817
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertOuterJoinRule.java
@@ -0,0 +1,121 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.Iterator;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class InsertOuterJoinRule 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 {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getValue();
+ if (op0.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ SubplanOperator subplan = (SubplanOperator) op0;
+
+ Iterator<ILogicalPlan> plansIter = subplan.getNestedPlans().iterator();
+ ILogicalPlan p = null;
+ while (plansIter.hasNext()) {
+ p = plansIter.next();
+ }
+ if (p == null) {
+ return false;
+ }
+ if (p.getRoots().size() != 1) {
+ return false;
+ }
+ Mutable<ILogicalOperator> subplanRoot = p.getRoots().get(0);
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) subplanRoot.getValue();
+ Mutable<ILogicalOperator> opUnder = subplan.getInputs().get(0);
+
+ if (OperatorPropertiesUtil.isNullTest((AbstractLogicalOperator) opUnder.getValue())) {
+ return false;
+ }
+
+ switch (op1.getOperatorTag()) {
+ case INNERJOIN: {
+ InnerJoinOperator join = (InnerJoinOperator) op1;
+ Mutable<ILogicalOperator> leftRef = join.getInputs().get(0);
+ Mutable<ILogicalOperator> rightRef = join.getInputs().get(1);
+ Mutable<ILogicalOperator> ntsRef = getNtsAtEndOfPipeline(leftRef);
+ if (ntsRef == null) {
+ ntsRef = getNtsAtEndOfPipeline(rightRef);
+ if (ntsRef == null) {
+ return false;
+ } else {
+ Mutable<ILogicalOperator> t = leftRef;
+ leftRef = rightRef;
+ rightRef = t;
+ }
+ }
+ ntsRef.setValue(opUnder.getValue());
+ LeftOuterJoinOperator loj = new LeftOuterJoinOperator(join.getCondition());
+ loj.getInputs().add(leftRef);
+ loj.getInputs().add(rightRef);
+ opRef.setValue(loj);
+ context.computeAndSetTypeEnvironmentForOperator(loj);
+ return true;
+ }
+ case LEFTOUTERJOIN: {
+ LeftOuterJoinOperator join = (LeftOuterJoinOperator) op1;
+ Mutable<ILogicalOperator> leftRef = join.getInputs().get(0);
+ Mutable<ILogicalOperator> ntsRef = getNtsAtEndOfPipeline(leftRef);
+ if (ntsRef == null) {
+ return false;
+ }
+ ntsRef.setValue(opUnder.getValue());
+ opRef.setValue(join);
+ context.computeAndSetTypeEnvironmentForOperator(join);
+ return true;
+ }
+ default: {
+ return false;
+ }
+ }
+ }
+
+ private Mutable<ILogicalOperator> getNtsAtEndOfPipeline(Mutable<ILogicalOperator> opRef) {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ return opRef;
+ }
+ if (op.getInputs().size() != 1) {
+ return null;
+ }
+ return getNtsAtEndOfPipeline(op.getInputs().get(0));
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java
new file mode 100644
index 0000000..d54833e
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+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.common.utils.Triple;
+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.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class InsertProjectBeforeUnionRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ return false;
+ }
+
+ /**
+ * When the input schema to WriteOperator is different from the output
+ * schema in terms of variable order, add a project operator to get the
+ * write order
+ */
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.UNIONALL) {
+ return false;
+ }
+ UnionAllOperator opUnion = (UnionAllOperator) op;
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = opUnion.getVariableMappings();
+ ArrayList<LogicalVariable> usedVariablesFromOne = new ArrayList<LogicalVariable>();
+ ArrayList<LogicalVariable> usedVariablesFromTwo = new ArrayList<LogicalVariable>();
+
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple : varMap) {
+ usedVariablesFromOne.add(triple.first);
+ usedVariablesFromTwo.add(triple.second);
+ }
+
+ ArrayList<LogicalVariable> inputSchemaOne = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(opUnion.getInputs().get(0).getValue(), inputSchemaOne);
+
+ ArrayList<LogicalVariable> inputSchemaTwo = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(opUnion.getInputs().get(1).getValue(), inputSchemaTwo);
+
+ boolean rewritten = false;
+ if (!isIdentical(usedVariablesFromOne, inputSchemaOne)) {
+ insertProjectOperator(opUnion, 0, usedVariablesFromOne, context);
+ rewritten = true;
+ }
+ if (!isIdentical(usedVariablesFromTwo, inputSchemaTwo)) {
+ insertProjectOperator(opUnion, 1, usedVariablesFromTwo, context);
+ rewritten = true;
+ }
+ return rewritten;
+ }
+
+ private void insertProjectOperator(UnionAllOperator opUnion, int branch, ArrayList<LogicalVariable> usedVariables,
+ IOptimizationContext context) throws AlgebricksException {
+ ProjectOperator projectOp = new ProjectOperator(usedVariables);
+ ILogicalOperator parentOp = opUnion.getInputs().get(branch).getValue();
+ projectOp.getInputs().add(new MutableObject<ILogicalOperator>(parentOp));
+ opUnion.getInputs().get(branch).setValue(projectOp);
+ projectOp.setPhysicalOperator(new StreamProjectPOperator());
+ context.computeAndSetTypeEnvironmentForOperator(parentOp);
+ }
+
+ private boolean isIdentical(List<LogicalVariable> finalSchema, List<LogicalVariable> inputSchema)
+ throws AlgebricksException {
+ int finalSchemaSize = finalSchema.size();
+ int inputSchemaSize = inputSchema.size();
+ if (finalSchemaSize != inputSchemaSize) {
+ return false;
+ }
+ for (int i = 0; i < finalSchemaSize; i++) {
+ LogicalVariable var1 = finalSchema.get(i);
+ LogicalVariable var2 = inputSchema.get(i);
+ if (!var1.equals(var2))
+ return false;
+ }
+ return true;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroHashPartitionMergeExchange.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroHashPartitionMergeExchange.java
new file mode 100644
index 0000000..f1321d3
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroHashPartitionMergeExchange.java
@@ -0,0 +1,52 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.physical.HashPartitionExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SortMergeExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class IntroHashPartitionMergeExchange 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 {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
+ if (op1.getPhysicalOperator() == null
+ || op1.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.HASH_PARTITION_EXCHANGE) {
+ return false;
+ }
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
+ if (op2.getPhysicalOperator() == null
+ || op2.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.SORT_MERGE_EXCHANGE) {
+ return false;
+ }
+ HashPartitionExchangePOperator hpe = (HashPartitionExchangePOperator) op1.getPhysicalOperator();
+ SortMergeExchangePOperator sme = (SortMergeExchangePOperator) op2.getPhysicalOperator();
+ List<OrderColumn> ocList = new ArrayList<OrderColumn>();
+ for (OrderColumn oc : sme.getSortColumns()) {
+ ocList.add(oc);
+ }
+ HashPartitionMergeExchangePOperator hpme = new HashPartitionMergeExchangePOperator(ocList, hpe.getHashFields(),
+ hpe.getDomain());
+ op1.setPhysicalOperator(hpme);
+ op1.getInputs().get(0).setValue(op2.getInputs().get(0).getValue());
+ return true;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroJoinInsideSubplanRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroJoinInsideSubplanRule.java
new file mode 100644
index 0000000..d69cd14
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroJoinInsideSubplanRule.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+import java.util.ListIterator;
+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.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+
+public class IntroJoinInsideSubplanRule extends AbstractDecorrelationRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getValue();
+ if (op0.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ SubplanOperator subplan = (SubplanOperator) op0;
+
+ Mutable<ILogicalOperator> leftRef = subplan.getInputs().get(0);
+ if (((AbstractLogicalOperator) leftRef.getValue()).getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ return false;
+ }
+
+ ListIterator<ILogicalPlan> plansIter = subplan.getNestedPlans().listIterator();
+ ILogicalPlan p = null;
+ while (plansIter.hasNext()) {
+ p = plansIter.next();
+ }
+ if (p == null) {
+ return false;
+ }
+ if (p.getRoots().size() != 1) {
+ return false;
+ }
+ Mutable<ILogicalOperator> opRef1 = p.getRoots().get(0);
+
+ while (true) {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef1.getValue();
+ if (op1.getInputs().size() != 1) {
+ return false;
+ }
+ if (op1.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ Mutable<ILogicalOperator> op2Ref = op1.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op2Ref.getValue();
+ if (op2.getOperatorTag() != LogicalOperatorTag.SELECT && descOrSelfIsScanOrJoin(op2)) {
+ Set<LogicalVariable> free2 = new HashSet<LogicalVariable>();
+ OperatorPropertiesUtil.getFreeVariablesInSelfOrDesc(op2, free2);
+ if (free2.isEmpty()) {
+ Set<LogicalVariable> free1 = new HashSet<LogicalVariable>();
+ OperatorPropertiesUtil.getFreeVariablesInSelfOrDesc(op1, free1);
+ if (!free1.isEmpty()) {
+ OperatorManipulationUtil.ntsToEts(op2Ref, context);
+ NestedTupleSourceOperator nts = new NestedTupleSourceOperator(
+ new MutableObject<ILogicalOperator>(subplan));
+ Mutable<ILogicalOperator> ntsRef = new MutableObject<ILogicalOperator>(nts);
+ Mutable<ILogicalOperator> innerRef = new MutableObject<ILogicalOperator>(op2);
+ InnerJoinOperator join = new InnerJoinOperator(new MutableObject<ILogicalExpression>(
+ ConstantExpression.TRUE), ntsRef, innerRef);
+ op2Ref.setValue(join);
+ context.computeAndSetTypeEnvironmentForOperator(nts);
+ context.computeAndSetTypeEnvironmentForOperator(join);
+ return true;
+ }
+ }
+ }
+ }
+ opRef1 = op1.getInputs().get(0);
+ }
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceAggregateCombinerRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceAggregateCombinerRule.java
new file mode 100644
index 0000000..c3d935c
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceAggregateCombinerRule.java
@@ -0,0 +1,44 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+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.ILogicalOperator;
+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.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+
+public class IntroduceAggregateCombinerRule extends AbstractIntroduceCombinerRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (context.checkIfInDontApplySet(this, op)) {
+ return false;
+ }
+ context.addToDontApplySet(this, op);
+ if (op.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
+ AggregateOperator aggOp = (AggregateOperator) op;
+ if (aggOp.getExecutionMode() != ExecutionMode.PARTITIONED || aggOp.getPartitioningVariable() == null) {
+ return false;
+ }
+ Map<AggregateFunctionCallExpression, SimilarAggregatesInfo> toReplaceMap = new HashMap<AggregateFunctionCallExpression, SimilarAggregatesInfo>();
+ Pair<Boolean, Mutable<ILogicalOperator>> result = tryToPushAgg(aggOp, null, toReplaceMap, context);
+ if (!result.first || result.second == null) {
+ return false;
+ }
+ replaceOriginalAggFuncs(toReplaceMap);
+ context.computeAndSetTypeEnvironmentForOperator(aggOp);
+ return true;
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByCombinerRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByCombinerRule.java
new file mode 100644
index 0000000..5c5fdb1
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByCombinerRule.java
@@ -0,0 +1,218 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+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.common.utils.Pair;
+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.base.OperatorAnnotations;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+
+public class IntroduceGroupByCombinerRule extends AbstractIntroduceCombinerRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (context.checkIfInDontApplySet(this, op)) {
+ return false;
+ }
+ context.addToDontApplySet(this, op);
+ if (op.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+ GroupByOperator gbyOp = (GroupByOperator) op;
+ if (gbyOp.getExecutionMode() != ExecutionMode.PARTITIONED) {
+ return false;
+ }
+
+ BookkeepingInfo bi = new BookkeepingInfo();
+ GroupByOperator newGbyOp = opToPush(gbyOp, bi, context);
+ if (newGbyOp == null) {
+ return false;
+ }
+
+ replaceOriginalAggFuncs(bi.toReplaceMap);
+
+ for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gbyOp.getDecorList()) {
+ LogicalVariable newDecorVar = context.newVar();
+ newGbyOp.addDecorExpression(newDecorVar, p.second.getValue());
+ p.second.setValue(new VariableReferenceExpression(newDecorVar));
+ }
+ newGbyOp.setExecutionMode(ExecutionMode.LOCAL);
+ Object v = gbyOp.getAnnotations().get(OperatorAnnotations.USE_HASH_GROUP_BY);
+ newGbyOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY, v);
+
+ Object v2 = gbyOp.getAnnotations().get(OperatorAnnotations.USE_EXTERNAL_GROUP_BY);
+ newGbyOp.getAnnotations().put(OperatorAnnotations.USE_EXTERNAL_GROUP_BY, v2);
+
+ List<LogicalVariable> propagatedVars = new LinkedList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(newGbyOp, propagatedVars);
+
+ Set<LogicalVariable> freeVars = new HashSet<LogicalVariable>();
+ OperatorPropertiesUtil.getFreeVariablesInSubplans(gbyOp, freeVars);
+
+ for (LogicalVariable var : freeVars) {
+ if (!propagatedVars.contains(var)) {
+ LogicalVariable newDecorVar = context.newVar();
+ newGbyOp.addDecorExpression(newDecorVar, new VariableReferenceExpression(var));
+ VariableUtilities.substituteVariables(gbyOp.getNestedPlans().get(0).getRoots().get(0).getValue(), var,
+ newDecorVar, context);
+ }
+ }
+
+ Mutable<ILogicalOperator> opRef3 = gbyOp.getInputs().get(0);
+ opRef3.setValue(newGbyOp);
+ typeGby(newGbyOp, context);
+ typeGby(gbyOp, context);
+ return true;
+ }
+
+ private void typeGby(AbstractOperatorWithNestedPlans op, IOptimizationContext context) throws AlgebricksException {
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ OperatorPropertiesUtil.typePlan(p, context);
+ }
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ }
+
+ private GroupByOperator opToPush(GroupByOperator gbyOp, BookkeepingInfo bi, IOptimizationContext context)
+ throws AlgebricksException {
+ // Hook up input to new group-by.
+ Mutable<ILogicalOperator> opRef3 = gbyOp.getInputs().get(0);
+ ILogicalOperator op3 = opRef3.getValue();
+ GroupByOperator newGbyOp = new GroupByOperator();
+ newGbyOp.getInputs().add(new MutableObject<ILogicalOperator>(op3));
+ // Copy annotations.
+ Map<String, Object> annotations = newGbyOp.getAnnotations();
+ annotations.putAll(gbyOp.getAnnotations());
+
+ List<LogicalVariable> gbyVars = gbyOp.getGbyVarList();
+ for (ILogicalPlan p : gbyOp.getNestedPlans()) {
+ Pair<Boolean, ILogicalPlan> bip = tryToPushSubplan(p, gbyOp, newGbyOp, bi, gbyVars, context);
+ if (!bip.first) {
+ // For now, if we cannot push everything, give up.
+ return null;
+ }
+ ILogicalPlan pushedSubplan = bip.second;
+ if (pushedSubplan != null) {
+ newGbyOp.getNestedPlans().add(pushedSubplan);
+ }
+ }
+
+ ArrayList<LogicalVariable> newOpGbyList = new ArrayList<LogicalVariable>();
+ ArrayList<LogicalVariable> replGbyList = new ArrayList<LogicalVariable>();
+ // Find maximal sequence of variable.
+ for (Map.Entry<GroupByOperator, List<LogicalVariable>> e : bi.modifyGbyMap.entrySet()) {
+ List<LogicalVariable> varList = e.getValue();
+ boolean see1 = true;
+ int sz1 = newOpGbyList.size();
+ int i = 0;
+ for (LogicalVariable v : varList) {
+ if (see1) {
+ if (i < sz1) {
+ LogicalVariable v2 = newOpGbyList.get(i);
+ if (v != v2) {
+ // cannot linearize
+ return null;
+ }
+ } else {
+ see1 = false;
+ newOpGbyList.add(v);
+ replGbyList.add(context.newVar());
+ }
+ i++;
+ } else {
+ newOpGbyList.add(v);
+ replGbyList.add(context.newVar());
+ }
+ }
+ }
+ // set the vars in the new op
+ int n = newOpGbyList.size();
+ for (int i = 0; i < n; i++) {
+ newGbyOp.addGbyExpression(replGbyList.get(i), new VariableReferenceExpression(newOpGbyList.get(i)));
+ VariableUtilities.substituteVariables(gbyOp, newOpGbyList.get(i), replGbyList.get(i), false, context);
+ }
+ return newGbyOp;
+ }
+
+ private Pair<Boolean, ILogicalPlan> tryToPushSubplan(ILogicalPlan nestedPlan, GroupByOperator oldGbyOp,
+ GroupByOperator newGbyOp, BookkeepingInfo bi, List<LogicalVariable> gbyVars, IOptimizationContext context)
+ throws AlgebricksException {
+ List<Mutable<ILogicalOperator>> pushedRoots = new ArrayList<Mutable<ILogicalOperator>>();
+ for (Mutable<ILogicalOperator> r : nestedPlan.getRoots()) {
+ if (!tryToPushRoot(r, oldGbyOp, newGbyOp, bi, gbyVars, context, pushedRoots)) {
+ // For now, if we cannot push everything, give up.
+ return new Pair<Boolean, ILogicalPlan>(false, null);
+ }
+ }
+ if (pushedRoots.isEmpty()) {
+ return new Pair<Boolean, ILogicalPlan>(true, null);
+ } else {
+ return new Pair<Boolean, ILogicalPlan>(true, new ALogicalPlanImpl(pushedRoots));
+ }
+ }
+
+ private boolean tryToPushRoot(Mutable<ILogicalOperator> root, GroupByOperator oldGbyOp, GroupByOperator newGbyOp,
+ BookkeepingInfo bi, List<LogicalVariable> gbyVars, IOptimizationContext context,
+ List<Mutable<ILogicalOperator>> toPushAccumulate) throws AlgebricksException {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) root.getValue();
+ if (op1.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
+ if (op2.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ AggregateOperator initAgg = (AggregateOperator) op1;
+ Pair<Boolean, Mutable<ILogicalOperator>> pOpRef = tryToPushAgg(initAgg, newGbyOp, bi.toReplaceMap, context);
+ if (!pOpRef.first) {
+ return false;
+ }
+ Mutable<ILogicalOperator> opRef = pOpRef.second;
+ if (opRef != null) {
+ toPushAccumulate.add(opRef);
+ }
+ bi.modifyGbyMap.put(oldGbyOp, gbyVars);
+ return true;
+ } else {
+ while (op2.getOperatorTag() != LogicalOperatorTag.GROUP && op2.getInputs().size() == 1) {
+ op2 = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
+ }
+ if (op2.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+ GroupByOperator nestedGby = (GroupByOperator) op2;
+ List<LogicalVariable> gbyVars2 = nestedGby.getGbyVarList();
+ List<LogicalVariable> concatGbyVars = new ArrayList<LogicalVariable>(gbyVars);
+ concatGbyVars.addAll(gbyVars2);
+ for (ILogicalPlan p : nestedGby.getNestedPlans()) {
+ for (Mutable<ILogicalOperator> r2 : p.getRoots()) {
+ if (!tryToPushRoot(r2, nestedGby, newGbyOp, bi, concatGbyVars, context, toPushAccumulate)) {
+ return false;
+ }
+ }
+ }
+ return true;
+ }
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
new file mode 100644
index 0000000..18c483e
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
@@ -0,0 +1,270 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+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.common.utils.Pair;
+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.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
+
+public class IntroduceGroupByForSubplanRule 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 {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getValue();
+ if (op0.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ SubplanOperator subplan = (SubplanOperator) op0;
+
+ Iterator<ILogicalPlan> plansIter = subplan.getNestedPlans().iterator();
+ ILogicalPlan p = null;
+ while (plansIter.hasNext()) {
+ p = plansIter.next();
+ }
+ if (p == null) {
+ return false;
+ }
+ if (p.getRoots().size() != 1) {
+ return false;
+ }
+ Mutable<ILogicalOperator> subplanRoot = p.getRoots().get(0);
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) subplanRoot.getValue();
+
+ Mutable<ILogicalOperator> botRef = subplanRoot;
+ AbstractLogicalOperator op2;
+ // Project is optional
+ if (op1.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+ op2 = op1;
+ } else {
+ ProjectOperator project = (ProjectOperator) op1;
+ botRef = project.getInputs().get(0);
+ op2 = (AbstractLogicalOperator) botRef.getValue();
+ }
+ if (op2.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
+ AggregateOperator aggregate = (AggregateOperator) op2;
+
+ Set<LogicalVariable> free = new HashSet<LogicalVariable>();
+ VariableUtilities.getUsedVariables(aggregate, free);
+
+ Mutable<ILogicalOperator> op3Ref = aggregate.getInputs().get(0);
+ AbstractLogicalOperator op3 = (AbstractLogicalOperator) op3Ref.getValue();
+
+ while (op3.getInputs().size() == 1) {
+ Set<LogicalVariable> prod = new HashSet<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op3, prod);
+ free.removeAll(prod);
+ VariableUtilities.getUsedVariables(op3, free);
+ botRef = op3Ref;
+ op3Ref = op3.getInputs().get(0);
+ op3 = (AbstractLogicalOperator) op3Ref.getValue();
+ }
+
+ if (op3.getOperatorTag() != LogicalOperatorTag.INNERJOIN
+ && op3.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+ return false;
+ }
+ AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op3;
+ if (join.getCondition().getValue() == ConstantExpression.TRUE) {
+ return false;
+ }
+ VariableUtilities.getUsedVariables(join, free);
+
+ AbstractLogicalOperator b0 = (AbstractLogicalOperator) join.getInputs().get(0).getValue();
+ // see if there's an NTS at the end of the pipeline
+ NestedTupleSourceOperator outerNts = getNts(b0);
+ if (outerNts == null) {
+ AbstractLogicalOperator b1 = (AbstractLogicalOperator) join.getInputs().get(1).getValue();
+ outerNts = getNts(b1);
+ if (outerNts == null) {
+ return false;
+ }
+ }
+
+ Set<LogicalVariable> pkVars = computeGbyVars(outerNts, free, context);
+ if (pkVars == null || pkVars.size() < 1) {
+ // could not group only by primary keys
+ return false;
+ }
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("Found FD for introducing group-by: " + pkVars);
+
+ Mutable<ILogicalOperator> rightRef = join.getInputs().get(1);
+ LogicalVariable testForNull = null;
+ AbstractLogicalOperator right = (AbstractLogicalOperator) rightRef.getValue();
+ switch (right.getOperatorTag()) {
+ case UNNEST: {
+ UnnestOperator innerUnnest = (UnnestOperator) right;
+ // Select [ $y != null ]
+ testForNull = innerUnnest.getVariable();
+ break;
+ }
+ case DATASOURCESCAN: {
+ DataSourceScanOperator innerScan = (DataSourceScanOperator) right;
+ // Select [ $y != null ]
+ if (innerScan.getVariables().size() == 1) {
+ testForNull = innerScan.getVariables().get(0);
+ }
+ break;
+ }
+ }
+ if (testForNull == null) {
+ testForNull = context.newVar();
+ AssignOperator tmpAsgn = new AssignOperator(testForNull, new MutableObject<ILogicalExpression>(
+ ConstantExpression.TRUE));
+ tmpAsgn.getInputs().add(new MutableObject<ILogicalOperator>(rightRef.getValue()));
+ rightRef.setValue(tmpAsgn);
+ context.computeAndSetTypeEnvironmentForOperator(tmpAsgn);
+ }
+
+ IFunctionInfo finfoEq = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.IS_NULL);
+ ILogicalExpression isNullTest = new ScalarFunctionCallExpression(finfoEq,
+ new MutableObject<ILogicalExpression>(new VariableReferenceExpression(testForNull)));
+ IFunctionInfo finfoNot = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.NOT);
+ ScalarFunctionCallExpression nonNullTest = new ScalarFunctionCallExpression(finfoNot,
+ new MutableObject<ILogicalExpression>(isNullTest));
+ SelectOperator selectNonNull = new SelectOperator(new MutableObject<ILogicalExpression>(nonNullTest));
+ GroupByOperator g = new GroupByOperator();
+ Mutable<ILogicalOperator> newSubplanRef = new MutableObject<ILogicalOperator>(subplan);
+ NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(g));
+ opRef.setValue(g);
+ selectNonNull.getInputs().add(new MutableObject<ILogicalOperator>(nts));
+
+ List<Mutable<ILogicalOperator>> prodInpList = botRef.getValue().getInputs();
+ prodInpList.clear();
+ prodInpList.add(new MutableObject<ILogicalOperator>(selectNonNull));
+
+ ILogicalPlan gPlan = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(subplanRoot.getValue()));
+ g.getNestedPlans().add(gPlan);
+ subplanRoot.setValue(op3Ref.getValue());
+ g.getInputs().add(newSubplanRef);
+
+ HashSet<LogicalVariable> underVars = new HashSet<LogicalVariable>();
+ VariableUtilities.getLiveVariables(subplan.getInputs().get(0).getValue(), underVars);
+ underVars.removeAll(pkVars);
+ Map<LogicalVariable, LogicalVariable> mappedVars = buildVarExprList(pkVars, context, g, g.getGroupByList());
+ context.updatePrimaryKeys(mappedVars);
+ for (LogicalVariable uv : underVars) {
+ g.getDecorList().add(
+ new Pair<LogicalVariable, Mutable<ILogicalExpression>>(null, new MutableObject<ILogicalExpression>(
+ new VariableReferenceExpression(uv))));
+ }
+ OperatorPropertiesUtil.typeOpRec(subplanRoot, context);
+ OperatorPropertiesUtil.typeOpRec(gPlan.getRoots().get(0), context);
+ context.computeAndSetTypeEnvironmentForOperator(g);
+ return true;
+ }
+
+ private NestedTupleSourceOperator getNts(AbstractLogicalOperator op) {
+ AbstractLogicalOperator alo = op;
+ do {
+ if (alo.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ return (NestedTupleSourceOperator) alo;
+ }
+ if (alo.getInputs().size() != 1) {
+ return null;
+ }
+ alo = (AbstractLogicalOperator) alo.getInputs().get(0).getValue();
+ } while (true);
+ }
+
+ protected Set<LogicalVariable> computeGbyVars(AbstractLogicalOperator op, Set<LogicalVariable> freeVars,
+ IOptimizationContext context) throws AlgebricksException {
+ PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(op, context);
+ List<FunctionalDependency> fdList = context.getFDList(op);
+ if (fdList == null) {
+ return null;
+ }
+ // check if any of the FDs is a key
+ List<LogicalVariable> all = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(op, all);
+ all.retainAll(freeVars);
+ for (FunctionalDependency fd : fdList) {
+ if (fd.getTail().containsAll(all)) {
+ return new HashSet<LogicalVariable>(fd.getHead());
+ }
+ }
+ return null;
+ }
+
+ private Map<LogicalVariable, LogicalVariable> buildVarExprList(Collection<LogicalVariable> vars,
+ IOptimizationContext context, GroupByOperator g,
+ List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> outVeList) throws AlgebricksException {
+ Map<LogicalVariable, LogicalVariable> m = new HashMap<LogicalVariable, LogicalVariable>();
+ for (LogicalVariable ov : vars) {
+ LogicalVariable newVar = context.newVar();
+ ILogicalExpression varExpr = new VariableReferenceExpression(newVar);
+ outVeList.add(new Pair<LogicalVariable, Mutable<ILogicalExpression>>(ov,
+ new MutableObject<ILogicalExpression>(varExpr)));
+ for (ILogicalPlan p : g.getNestedPlans()) {
+ for (Mutable<ILogicalOperator> r : p.getRoots()) {
+ OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getValue(), ov, newVar, true,
+ context);
+ }
+ }
+ AbstractLogicalOperator opUnder = (AbstractLogicalOperator) g.getInputs().get(0).getValue();
+ OperatorManipulationUtil.substituteVarRec(opUnder, ov, newVar, true, context);
+ m.put(ov, newVar);
+ }
+ return m;
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
new file mode 100644
index 0000000..ae57fb7
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
@@ -0,0 +1,140 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+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.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+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.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class IsolateHyracksOperatorsRule implements IAlgebraicRewriteRule {
+
+ private final PhysicalOperatorTag[] operatorsBelowWhichJobGenIsDisabled;
+
+ public IsolateHyracksOperatorsRule(PhysicalOperatorTag[] operatorsBelowWhichJobGenIsDisabled) {
+ this.operatorsBelowWhichJobGenIsDisabled = operatorsBelowWhichJobGenIsDisabled;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ IPhysicalOperator pt = op.getPhysicalOperator();
+
+ if (pt == null || op.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ return false;
+ }
+ if (!pt.isMicroOperator()) {
+ return testIfExchangeBelow(opRef, context);
+ } else {
+ return testIfExchangeAbove(opRef, context);
+ }
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ private boolean testIfExchangeBelow(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ boolean exchInserted = false;
+
+ for (Mutable<ILogicalOperator> i : op.getInputs()) {
+ AbstractLogicalOperator c = (AbstractLogicalOperator) i.getValue();
+ if (c.getOperatorTag() != LogicalOperatorTag.EXCHANGE) {
+ if (c.getPhysicalOperator() == null) {
+ return false;
+ }
+ insertOneToOneExchange(i, context);
+ exchInserted = true;
+ }
+ }
+ IPhysicalOperator pt = op.getPhysicalOperator();
+ if (pt.isJobGenDisabledBelowMe() || arrayContains(operatorsBelowWhichJobGenIsDisabled, pt.getOperatorTag())) {
+ for (Mutable<ILogicalOperator> i : op.getInputs()) {
+ disableJobGenRec(i.getValue());
+ }
+ }
+ return exchInserted;
+ }
+
+ private void disableJobGenRec(ILogicalOperator operator) {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) operator;
+ op.disableJobGen();
+ for (Mutable<ILogicalOperator> i : op.getInputs()) {
+ disableJobGenRec(i.getValue());
+ }
+ }
+
+ private boolean testIfExchangeAbove(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ return false;
+ }
+ boolean exchInserted = false;
+ for (Mutable<ILogicalOperator> i : op.getInputs()) {
+ AbstractLogicalOperator c = (AbstractLogicalOperator) i.getValue();
+ IPhysicalOperator cpop = c.getPhysicalOperator();
+ if (c.getOperatorTag() == LogicalOperatorTag.EXCHANGE || cpop == null) {
+ continue;
+ }
+ if (!cpop.isMicroOperator()) {
+ insertOneToOneExchange(i, context);
+ exchInserted = true;
+ }
+ }
+ return exchInserted;
+ }
+
+ private final static <T> boolean arrayContains(T[] array, T tag) {
+ for (int i = 0; i < array.length; i++) {
+ if (array[i] == tag) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private final static void insertOneToOneExchange(Mutable<ILogicalOperator> i, IOptimizationContext context)
+ throws AlgebricksException {
+ ExchangeOperator e = new ExchangeOperator();
+ e.setPhysicalOperator(new OneToOneExchangePOperator());
+ ILogicalOperator inOp = i.getValue();
+
+ e.getInputs().add(new MutableObject<ILogicalOperator>(inOp));
+ i.setValue(e);
+ // e.recomputeSchema();
+ OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull(e, context);
+ ExecutionMode em = ((AbstractLogicalOperator) inOp).getExecutionMode();
+ e.setExecutionMode(em);
+ e.computeDeliveredPhysicalProperties(context);
+ context.computeAndSetTypeEnvironmentForOperator(e);
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
new file mode 100644
index 0000000..75862cf
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
@@ -0,0 +1,118 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+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.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PullSelectOutOfEqJoin implements IAlgebraicRewriteRule {
+
+ private List<Mutable<ILogicalExpression>> eqVarVarComps = new ArrayList<Mutable<ILogicalExpression>>();
+ private List<Mutable<ILogicalExpression>> otherPredicates = new ArrayList<Mutable<ILogicalExpression>>();
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
+ if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN
+ && op.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+ return false;
+ }
+ AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op;
+
+ ILogicalExpression expr = join.getCondition().getValue();
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) expr;
+ FunctionIdentifier fi = fexp.getFunctionIdentifier();
+ if (!fi.equals(AlgebricksBuiltinFunctions.AND)) {
+ return false;
+ }
+ eqVarVarComps.clear();
+ otherPredicates.clear();
+ for (Mutable<ILogicalExpression> arg : fexp.getArguments()) {
+ if (isEqVarVar(arg.getValue())) {
+ eqVarVarComps.add(arg);
+ } else {
+ otherPredicates.add(arg);
+ }
+ }
+ if (eqVarVarComps.isEmpty() || otherPredicates.isEmpty()) {
+ return false;
+ }
+ // pull up
+ ILogicalExpression pulledCond = makeCondition(otherPredicates, context);
+ SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond));
+ ILogicalExpression newJoinCond = makeCondition(eqVarVarComps, context);
+ join.getCondition().setValue(newJoinCond);
+ select.getInputs().add(new MutableObject<ILogicalOperator>(join));
+ opRef.setValue(select);
+ context.computeAndSetTypeEnvironmentForOperator(select);
+ return true;
+ }
+
+ private ILogicalExpression makeCondition(List<Mutable<ILogicalExpression>> predList, IOptimizationContext context) {
+ if (predList.size() > 1) {
+ IFunctionInfo finfo = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
+ return new ScalarFunctionCallExpression(finfo, predList);
+ } else {
+ return predList.get(0).getValue();
+ }
+ }
+
+ private boolean isEqVarVar(ILogicalExpression expr) {
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+ if (!f.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.EQ)) {
+ return false;
+ }
+ ILogicalExpression e1 = f.getArguments().get(0).getValue();
+ if (e1.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return false;
+ } else {
+ ILogicalExpression e2 = f.getArguments().get(1).getValue();
+ return e2.getExpressionTag() == LogicalExpressionTag.VARIABLE;
+ }
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushAssignDownThroughProductRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushAssignDownThroughProductRule.java
new file mode 100644
index 0000000..c105b51
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushAssignDownThroughProductRule.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.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.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushAssignDownThroughProductRule 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 {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
+ if (op1.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+ return false;
+ }
+ Mutable<ILogicalOperator> op2Ref = op1.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op2Ref.getValue();
+ if (op2.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+ return false;
+ }
+ AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op2;
+ if (join.getCondition().getValue() != ConstantExpression.TRUE) {
+ return false;
+ }
+
+ List<LogicalVariable> used = new ArrayList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op1, used);
+
+ Mutable<ILogicalOperator> b0Ref = op2.getInputs().get(0);
+ ILogicalOperator b0 = b0Ref.getValue();
+ List<LogicalVariable> b0Scm = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(b0, b0Scm);
+ if (b0Scm.containsAll(used)) {
+ // push assign on left branch
+ op2Ref.setValue(b0);
+ b0Ref.setValue(op1);
+ opRef.setValue(op2);
+ return true;
+ } else {
+ Mutable<ILogicalOperator> b1Ref = op2.getInputs().get(1);
+ ILogicalOperator b1 = b1Ref.getValue();
+ List<LogicalVariable> b1Scm = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(b1, b1Scm);
+ if (b1Scm.containsAll(used)) {
+ // push assign on right branch
+ op2Ref.setValue(b1);
+ b1Ref.setValue(op1);
+ opRef.setValue(op2);
+ return true;
+ } else {
+ return false;
+ }
+ }
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushDieUpRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushDieUpRule.java
new file mode 100644
index 0000000..c4dd78d
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushDieUpRule.java
@@ -0,0 +1,52 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+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.ILogicalOperator;
+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.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushDieUpRule 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 {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getValue();
+ if (op0.getInputs().size() == 0)
+ return false;
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) op0.getInputs().get(0).getValue();
+
+ if (op1.getInputs().size() == 0)
+ return false;
+ LogicalOperatorTag tag = op1.getOperatorTag();
+ if (tag == LogicalOperatorTag.SINK || tag == LogicalOperatorTag.WRITE
+ || tag == LogicalOperatorTag.INSERT_DELETE || tag == LogicalOperatorTag.WRITE_RESULT)
+ return false;
+
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
+ if (op2.getOperatorTag() == LogicalOperatorTag.DIE) {
+ op0.getInputs().get(0).setValue(op2);
+ op1.getInputs().clear();
+ for (Mutable<ILogicalOperator> ref : op2.getInputs())
+ op1.getInputs().add(ref);
+ op2.getInputs().clear();
+ op2.getInputs().add(new MutableObject<ILogicalOperator>(op1));
+
+ context.computeAndSetTypeEnvironmentForOperator(op0);
+ context.computeAndSetTypeEnvironmentForOperator(op1);
+ context.computeAndSetTypeEnvironmentForOperator(op2);
+ return true;
+ } else {
+ return false;
+ }
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushLimitDownRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushLimitDownRule.java
new file mode 100644
index 0000000..acfdbd3
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushLimitDownRule.java
@@ -0,0 +1,118 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.LinkedList;
+
+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.ILogicalOperator;
+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.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamLimitPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushLimitDownRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ /**
+ * When a global Limit over a merge-exchange is found, a local Limit is
+ * pushed down.
+ */
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.LIMIT) {
+ return false;
+ }
+ LimitOperator opLim = (LimitOperator) op;
+ if (!opLim.isTopmostLimitOp()) {
+ return false;
+ }
+
+ Mutable<ILogicalOperator> opRef2 = opLim.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+
+ if (context.checkAndAddToAlreadyCompared(op, op2)) {
+ return false;
+ }
+ if (op2.getOperatorTag() != LogicalOperatorTag.EXCHANGE) {
+ return false;
+ }
+ PhysicalOperatorTag op2PTag = op2.getPhysicalOperator().getOperatorTag();
+ // we should test for any kind of merge
+ if (op2PTag != PhysicalOperatorTag.RANDOM_MERGE_EXCHANGE && op2PTag != PhysicalOperatorTag.SORT_MERGE_EXCHANGE) {
+ return false;
+ }
+
+ LinkedList<LogicalVariable> usedVars1 = new LinkedList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(opLim, usedVars1);
+
+ do {
+ if (op2.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE
+ || op2.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE
+ || op2.getOperatorTag() == LogicalOperatorTag.LIMIT) {
+ return false;
+ }
+ if (op2.getInputs().size() > 1 || !op2.isMap()) {
+ break;
+ }
+ LinkedList<LogicalVariable> vars2 = new LinkedList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op2, vars2);
+ if (!OperatorPropertiesUtil.disjoint(vars2, usedVars1)) {
+ return false;
+ }
+ // we assume pipelineable ops. have only one input
+ opRef2 = op2.getInputs().get(0);
+ op2 = (AbstractLogicalOperator) opRef2.getValue();
+ } while (true);
+
+ LimitOperator clone2 = null;
+ if (opLim.getOffset().getValue() == null) {
+ clone2 = new LimitOperator(opLim.getMaxObjects().getValue(), false);
+ } else {
+ // push limit (max+offset)
+ IFunctionInfo finfoAdd = context.getMetadataProvider().lookupFunction(
+ AlgebricksBuiltinFunctions.NUMERIC_ADD);
+ ScalarFunctionCallExpression maxPlusOffset = new ScalarFunctionCallExpression(finfoAdd,
+ opLim.getMaxObjects(), opLim.getOffset());
+ clone2 = new LimitOperator(maxPlusOffset, false);
+ }
+ clone2.setPhysicalOperator(new StreamLimitPOperator(false));
+ clone2.getInputs().add(new MutableObject<ILogicalOperator>(op2));
+ clone2.setExecutionMode(op2.getExecutionMode());
+ clone2.recomputeSchema();
+ opRef2.setValue(clone2);
+ context.computeAndSetTypeEnvironmentForOperator(clone2);
+ return true;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushNestedOrderByUnderPreSortedGroupByRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushNestedOrderByUnderPreSortedGroupByRule.java
new file mode 100644
index 0000000..95ad030
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushNestedOrderByUnderPreSortedGroupByRule.java
@@ -0,0 +1,119 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+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.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.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushNestedOrderByUnderPreSortedGroupByRule 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 {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+ if (op.getPhysicalOperator() == null) {
+ return false;
+ }
+ AbstractPhysicalOperator pOp = (AbstractPhysicalOperator) op.getPhysicalOperator();
+ if (pOp.getOperatorTag() != PhysicalOperatorTag.PRE_CLUSTERED_GROUP_BY) {
+ return false;
+ }
+ GroupByOperator gby = (GroupByOperator) op;
+ ILogicalPlan plan = gby.getNestedPlans().get(0);
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) plan.getRoots().get(0).getValue();
+ if (op1.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
+ Mutable<ILogicalOperator> opRef2 = op1.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+ if (op2.getOperatorTag() != LogicalOperatorTag.ORDER) {
+ return false;
+ }
+ OrderOperator order1 = (OrderOperator) op2;
+ if (!isIndependentFromChildren(order1)) {
+ return false;
+ }
+ AbstractPhysicalOperator pOrder1 = (AbstractPhysicalOperator) op2.getPhysicalOperator();
+ if (pOrder1.getOperatorTag() != PhysicalOperatorTag.STABLE_SORT
+ && pOrder1.getOperatorTag() != PhysicalOperatorTag.IN_MEMORY_STABLE_SORT) {
+ return false;
+ }
+ // StableSortPOperator sort1 = (StableSortPOperator) pOrder1;
+ AbstractLogicalOperator op3 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ if (op3.getOperatorTag() != LogicalOperatorTag.ORDER) {
+ return false;
+ }
+ AbstractPhysicalOperator pOp3 = (AbstractPhysicalOperator) op3.getPhysicalOperator();
+ if (pOp3.getOperatorTag() != PhysicalOperatorTag.STABLE_SORT) {
+ return false;
+ }
+ OrderOperator order2 = (OrderOperator) op3;
+ StableSortPOperator sort2 = (StableSortPOperator) pOp3;
+ // int n1 = sort1.getSortColumns().length;
+ // int n2 = sort2.getSortColumns().length;
+ // OrderColumn[] sortColumns = new OrderColumn[n2 + n1];
+ // System.arraycopy(sort2.getSortColumns(), 0, sortColumns, 0, n2);
+ // int k = 0;
+ for (Pair<IOrder, Mutable<ILogicalExpression>> oe : order1.getOrderExpressions()) {
+ order2.getOrderExpressions().add(oe);
+ // sortColumns[n2 + k] = sort1.getSortColumns()[k];
+ // ++k;
+ }
+ // sort2.setSortColumns(sortColumns);
+ sort2.computeDeliveredProperties(order2, null);
+ // remove order1
+ ILogicalOperator underOrder1 = order1.getInputs().get(0).getValue();
+ opRef2.setValue(underOrder1);
+ return true;
+ }
+
+ private boolean isIndependentFromChildren(OrderOperator order1) throws AlgebricksException {
+ Set<LogicalVariable> free = new HashSet<LogicalVariable>();
+ OperatorPropertiesUtil.getFreeVariablesInSelfOrDesc(order1, free);
+ Set<LogicalVariable> usedInOrder = new HashSet<LogicalVariable>();
+ VariableUtilities.getUsedVariables(order1, usedInOrder);
+ return free.containsAll(usedInOrder);
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java
new file mode 100644
index 0000000..f223844
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java
@@ -0,0 +1,219 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+
+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.common.utils.Pair;
+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.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Pushes projections through its input operator, provided that operator does
+ * not produce the projected variables.
+ *
+ * @author Nicola
+ */
+public class PushProjectDownRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+ return false;
+ }
+ ProjectOperator pi = (ProjectOperator) op;
+ Mutable<ILogicalOperator> opRef2 = pi.getInputs().get(0);
+
+ HashSet<LogicalVariable> toPush = new HashSet<LogicalVariable>();
+ toPush.addAll(pi.getVariables());
+
+ Pair<Boolean, Boolean> p = pushThroughOp(toPush, opRef2, op, context);
+ boolean smthWasPushed = p.first;
+ if (p.second) { // the original projection is redundant
+ opRef.setValue(op.getInputs().get(0).getValue());
+ smthWasPushed = true;
+ }
+
+ return smthWasPushed;
+ }
+
+ private static Pair<Boolean, Boolean> pushThroughOp(HashSet<LogicalVariable> toPush,
+ Mutable<ILogicalOperator> opRef2, ILogicalOperator initialOp, IOptimizationContext context)
+ throws AlgebricksException {
+ List<LogicalVariable> initProjectList = new ArrayList<LogicalVariable>(toPush);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+ do {
+ if (op2.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE
+ || op2.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE
+ || op2.getOperatorTag() == LogicalOperatorTag.PROJECT
+ || op2.getOperatorTag() == LogicalOperatorTag.REPLICATE
+ || op2.getOperatorTag() == LogicalOperatorTag.UNIONALL) {
+ return new Pair<Boolean, Boolean>(false, false);
+ }
+ if (!op2.isMap()) {
+ break;
+ }
+ LinkedList<LogicalVariable> usedVars = new LinkedList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op2, usedVars);
+ toPush.addAll(usedVars);
+ LinkedList<LogicalVariable> producedVars = new LinkedList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op2, producedVars);
+ toPush.removeAll(producedVars);
+ // we assume pipelineable ops. have only one input
+ opRef2 = op2.getInputs().get(0);
+ op2 = (AbstractLogicalOperator) opRef2.getValue();
+ } while (true);
+
+ LinkedList<LogicalVariable> produced2 = new LinkedList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op2, produced2);
+ LinkedList<LogicalVariable> used2 = new LinkedList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op2, used2);
+
+ boolean canCommuteProjection = initProjectList.containsAll(toPush) && initProjectList.containsAll(produced2)
+ && initProjectList.containsAll(used2);
+ // if true, we can get rid of the initial projection
+
+ // get rid of useless decor vars.
+ if (!canCommuteProjection && op2.getOperatorTag() == LogicalOperatorTag.GROUP) {
+ boolean gbyChanged = false;
+ GroupByOperator gby = (GroupByOperator) op2;
+ List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> newDecorList = new ArrayList<Pair<LogicalVariable, Mutable<ILogicalExpression>>>();
+ for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gby.getDecorList()) {
+ LogicalVariable decorVar = GroupByOperator.getDecorVariable(p);
+ if (!toPush.contains(decorVar)) {
+ used2.remove(decorVar);
+ gbyChanged = true;
+ } else {
+ newDecorList.add(p);
+ }
+ }
+ gby.getDecorList().clear();
+ gby.getDecorList().addAll(newDecorList);
+ if (gbyChanged) {
+ context.computeAndSetTypeEnvironmentForOperator(gby);
+ }
+ }
+ used2.clear();
+ VariableUtilities.getUsedVariables(op2, used2);
+
+ toPush.addAll(used2); // remember that toPush is a Set
+ toPush.removeAll(produced2);
+
+ if (toPush.isEmpty()) {
+ return new Pair<Boolean, Boolean>(false, false);
+ }
+
+ boolean smthWasPushed = false;
+ for (Mutable<ILogicalOperator> c : op2.getInputs()) {
+ if (pushNeededProjections(toPush, c, context, initialOp)) {
+ smthWasPushed = true;
+ }
+ }
+ if (op2.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans n = (AbstractOperatorWithNestedPlans) op2;
+ for (ILogicalPlan p : n.getNestedPlans()) {
+ for (Mutable<ILogicalOperator> r : p.getRoots()) {
+ if (pushNeededProjections(toPush, r, context, initialOp)) {
+ smthWasPushed = true;
+ }
+ }
+ }
+ }
+ return new Pair<Boolean, Boolean>(smthWasPushed, canCommuteProjection);
+ }
+
+ // It does not try to push above another Projection.
+ private static boolean pushNeededProjections(HashSet<LogicalVariable> toPush, Mutable<ILogicalOperator> opRef,
+ IOptimizationContext context, ILogicalOperator initialOp) throws AlgebricksException {
+ HashSet<LogicalVariable> allP = new HashSet<LogicalVariable>();
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ VariableUtilities.getLiveVariables(op, allP);
+
+ HashSet<LogicalVariable> toProject = new HashSet<LogicalVariable>();
+ for (LogicalVariable v : toPush) {
+ if (allP.contains(v)) {
+ toProject.add(v);
+ }
+ }
+ if (toProject.equals(allP)) {
+ // projection would be redundant, since we would project everything
+ // but we can try with the children
+ boolean push = false;
+ if (pushThroughOp(toProject, opRef, initialOp, context).first) {
+ push = true;
+ }
+ return push;
+ } else {
+ return pushAllProjectionsOnTopOf(toProject, opRef, context, initialOp);
+ }
+ }
+
+ // It does not try to push above another Projection.
+ private static boolean pushAllProjectionsOnTopOf(Collection<LogicalVariable> toPush,
+ Mutable<ILogicalOperator> opRef, IOptimizationContext context, ILogicalOperator initialOp)
+ throws AlgebricksException {
+ if (toPush.isEmpty()) {
+ return false;
+ }
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
+ if (context.checkAndAddToAlreadyCompared(initialOp, op)) {
+ return false;
+ }
+
+ switch (op.getOperatorTag()) {
+ case EXCHANGE: {
+ opRef = opRef.getValue().getInputs().get(0);
+ op = (AbstractLogicalOperator) opRef.getValue();
+ break;
+ }
+ case PROJECT: {
+ return false;
+ }
+ }
+
+ ProjectOperator pi2 = new ProjectOperator(new ArrayList<LogicalVariable>(toPush));
+ pi2.getInputs().add(new MutableObject<ILogicalOperator>(op));
+ opRef.setValue(pi2);
+ pi2.setExecutionMode(op.getExecutionMode());
+ context.computeAndSetTypeEnvironmentForOperator(pi2);
+ return true;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectIntoDataSourceScanRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectIntoDataSourceScanRule.java
new file mode 100644
index 0000000..2cbbdbf
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectIntoDataSourceScanRule.java
@@ -0,0 +1,46 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.LogicalOperatorTag;
+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.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushProjectIntoDataSourceScanRule 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 {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getInputs().size() <= 0)
+ return false;
+ AbstractLogicalOperator project = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ if (project.getOperatorTag() != LogicalOperatorTag.PROJECT)
+ return false;
+ AbstractLogicalOperator exchange = (AbstractLogicalOperator) project.getInputs().get(0).getValue();
+ if (exchange.getOperatorTag() != LogicalOperatorTag.EXCHANGE)
+ return false;
+ AbstractLogicalOperator inputOp = (AbstractLogicalOperator) exchange.getInputs().get(0).getValue();
+ if (inputOp.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN)
+ return false;
+ DataSourceScanOperator scanOp = (DataSourceScanOperator) inputOp;
+ ProjectOperator projectOp = (ProjectOperator) project;
+ scanOp.addProjectVariables(projectOp.getVariables());
+ if (op.getOperatorTag() != LogicalOperatorTag.EXCHANGE) {
+ op.getInputs().set(0, project.getInputs().get(0));
+ } else {
+ op.getInputs().set(0, exchange.getInputs().get(0));
+ }
+ return true;
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectDownRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectDownRule.java
new file mode 100644
index 0000000..ef4fd2e
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectDownRule.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushSelectDownRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+
+ Mutable<ILogicalOperator> opRef2 = op.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+
+ if (context.checkAndAddToAlreadyCompared(op, op2)) {
+ return false;
+ }
+
+ LogicalOperatorTag tag2 = op2.getOperatorTag();
+
+ if (tag2 == LogicalOperatorTag.INNERJOIN || tag2 == LogicalOperatorTag.LEFTOUTERJOIN
+ || tag2 == LogicalOperatorTag.REPLICATE) {
+ return false;
+ } else { // not a join
+ boolean res = propagateSelectionRec(opRef, opRef2);
+ if (res) {
+ OperatorPropertiesUtil.typeOpRec(opRef, context);
+ }
+ return res;
+ }
+ }
+
+ private static boolean propagateSelectionRec(Mutable<ILogicalOperator> sigmaRef, Mutable<ILogicalOperator> opRef2)
+ throws AlgebricksException {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+ if (op2.getInputs().size() != 1 || op2.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ return false;
+ }
+
+ SelectOperator sigma = (SelectOperator) sigmaRef.getValue();
+ LinkedList<LogicalVariable> usedInSigma = new LinkedList<LogicalVariable>();
+ sigma.getCondition().getValue().getUsedVariables(usedInSigma);
+
+ LinkedList<LogicalVariable> produced2 = new LinkedList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op2, produced2);
+ if (OperatorPropertiesUtil.disjoint(produced2, usedInSigma)) {
+ // just swap
+ opRef2.setValue(sigma);
+ sigmaRef.setValue(op2);
+ List<Mutable<ILogicalOperator>> sigmaInpList = sigma.getInputs();
+ sigmaInpList.clear();
+ sigmaInpList.addAll(op2.getInputs());
+ List<Mutable<ILogicalOperator>> op2InpList = op2.getInputs();
+ op2InpList.clear();
+ op2InpList.add(opRef2);
+ propagateSelectionRec(opRef2, sigma.getInputs().get(0));
+ return true;
+
+ }
+ return false;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
new file mode 100644
index 0000000..8c679c5
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
@@ -0,0 +1,240 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+
+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.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+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.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushSelectIntoJoinRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ Collection<LogicalVariable> joinLiveVarsLeft = new HashSet<LogicalVariable>();
+ Collection<LogicalVariable> joinLiveVarsRight = new HashSet<LogicalVariable>();
+ Collection<LogicalVariable> liveInOpsToPushLeft = new HashSet<LogicalVariable>();
+ Collection<LogicalVariable> liveInOpsToPushRight = new HashSet<LogicalVariable>();
+
+ List<ILogicalOperator> pushedOnLeft = new ArrayList<ILogicalOperator>();
+ List<ILogicalOperator> pushedOnRight = new ArrayList<ILogicalOperator>();
+ LinkedList<ILogicalOperator> notPushedStack = new LinkedList<ILogicalOperator>();
+ Collection<LogicalVariable> usedVars = new HashSet<LogicalVariable>();
+ Collection<LogicalVariable> producedVars = new HashSet<LogicalVariable>();
+
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ SelectOperator select = (SelectOperator) op;
+ Mutable<ILogicalOperator> opRef2 = op.getInputs().get(0);
+ AbstractLogicalOperator son = (AbstractLogicalOperator) opRef2.getValue();
+ AbstractLogicalOperator op2 = son;
+ boolean needToPushOps = false;
+ while (son.isMap()) {
+ needToPushOps = true;
+ Mutable<ILogicalOperator> opRefLink = son.getInputs().get(0);
+ son = (AbstractLogicalOperator) opRefLink.getValue();
+ }
+
+ if (son.getOperatorTag() != LogicalOperatorTag.INNERJOIN
+ && son.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+ return false;
+ }
+ boolean isLoj = son.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN;
+ AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) son;
+
+ Mutable<ILogicalOperator> joinBranchLeftRef = join.getInputs().get(0);
+ Mutable<ILogicalOperator> joinBranchRightRef = join.getInputs().get(1);
+
+ if (needToPushOps) {
+ ILogicalOperator joinBranchLeft = joinBranchLeftRef.getValue();
+ ILogicalOperator joinBranchRight = joinBranchRightRef.getValue();
+ VariableUtilities.getLiveVariables(joinBranchLeft, joinLiveVarsLeft);
+ VariableUtilities.getLiveVariables(joinBranchRight, joinLiveVarsRight);
+ Mutable<ILogicalOperator> opIterRef = opRef2;
+ ILogicalOperator opIter = op2;
+ while (opIter != join) {
+ LogicalOperatorTag tag = ((AbstractLogicalOperator) opIter).getOperatorTag();
+ if (tag == LogicalOperatorTag.PROJECT) {
+ notPushedStack.addFirst(opIter);
+ } else {
+ VariableUtilities.getUsedVariables(opIter, usedVars);
+ VariableUtilities.getProducedVariables(opIter, producedVars);
+ if (joinLiveVarsLeft.containsAll(usedVars)) {
+ pushedOnLeft.add(opIter);
+ liveInOpsToPushLeft.addAll(producedVars);
+ } else if (joinLiveVarsRight.containsAll(usedVars)) {
+ pushedOnRight.add(opIter);
+ liveInOpsToPushRight.addAll(producedVars);
+ } else {
+ return false;
+ }
+ }
+ opIterRef = opIter.getInputs().get(0);
+ opIter = opIterRef.getValue();
+ }
+ if (isLoj && pushedOnLeft.isEmpty()) {
+ return false;
+ }
+ }
+
+ boolean intersectsAllBranches = true;
+ boolean[] intersectsBranch = new boolean[join.getInputs().size()];
+ LinkedList<LogicalVariable> selectVars = new LinkedList<LogicalVariable>();
+ select.getCondition().getValue().getUsedVariables(selectVars);
+ int i = 0;
+ for (Mutable<ILogicalOperator> branch : join.getInputs()) {
+ LinkedList<LogicalVariable> branchVars = new LinkedList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(branch.getValue(), branchVars);
+ if (i == 0) {
+ branchVars.addAll(liveInOpsToPushLeft);
+ } else {
+ branchVars.addAll(liveInOpsToPushRight);
+ }
+ if (OperatorPropertiesUtil.disjoint(selectVars, branchVars)) {
+ intersectsAllBranches = false;
+ } else {
+ intersectsBranch[i] = true;
+ }
+ i++;
+ }
+ if (!intersectsBranch[0] && !intersectsBranch[1]) {
+ return false;
+ }
+ if (intersectsAllBranches) {
+ if (needToPushOps) {
+ pushOps(pushedOnLeft, joinBranchLeftRef, context);
+ pushOps(pushedOnRight, joinBranchRightRef, context);
+ }
+ addCondToJoin(select, join, context);
+ } else { // push down
+ Iterator<Mutable<ILogicalOperator>> branchIter = join.getInputs().iterator();
+
+ for (int j = 0; j < intersectsBranch.length; j++) {
+ Mutable<ILogicalOperator> branch = branchIter.next();
+ boolean inter = intersectsBranch[j];
+ if (inter) {
+ if (needToPushOps) {
+ if (j == 0) {
+ pushOps(pushedOnLeft, joinBranchLeftRef, context);
+ } else {
+ pushOps(pushedOnRight, joinBranchRightRef, context);
+ }
+ }
+ copySelectToBranch(select, branch, context);
+ }
+
+ // if a left outer join, we can only push conditions into the
+ // outer branch.
+ if (j == 0 && isLoj) {
+ // stop at this branch
+ break;
+ }
+ }
+ }
+ ILogicalOperator top = join;
+ for (ILogicalOperator npOp : notPushedStack) {
+ List<Mutable<ILogicalOperator>> npInpList = npOp.getInputs();
+ npInpList.clear();
+ npInpList.add(new MutableObject<ILogicalOperator>(top));
+ context.computeAndSetTypeEnvironmentForOperator(npOp);
+ top = npOp;
+ }
+ opRef.setValue(top);
+ return true;
+
+ }
+
+ private void pushOps(List<ILogicalOperator> opList, Mutable<ILogicalOperator> joinBranch,
+ IOptimizationContext context) throws AlgebricksException {
+ ILogicalOperator topOp = joinBranch.getValue();
+ ListIterator<ILogicalOperator> iter = opList.listIterator(opList.size());
+ while (iter.hasPrevious()) {
+ ILogicalOperator op = iter.previous();
+ List<Mutable<ILogicalOperator>> opInpList = op.getInputs();
+ opInpList.clear();
+ opInpList.add(new MutableObject<ILogicalOperator>(topOp));
+ topOp = op;
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ }
+ joinBranch.setValue(topOp);
+ }
+
+ private static void addCondToJoin(SelectOperator select, AbstractBinaryJoinOperator join,
+ IOptimizationContext context) {
+ ILogicalExpression cond = join.getCondition().getValue();
+ if (OperatorPropertiesUtil.isAlwaysTrueCond(cond)) { // the join was a product
+ join.getCondition().setValue(select.getCondition().getValue());
+ } else {
+ boolean bAddedToConj = false;
+ if (cond.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ AbstractFunctionCallExpression fcond = (AbstractFunctionCallExpression) cond;
+ if (fcond.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.AND)) {
+ AbstractFunctionCallExpression newCond = new ScalarFunctionCallExpression(context
+ .getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND));
+ newCond.getArguments().add(select.getCondition());
+ newCond.getArguments().addAll(fcond.getArguments());
+ join.getCondition().setValue(newCond);
+ bAddedToConj = true;
+ }
+ }
+ if (!bAddedToConj) {
+ AbstractFunctionCallExpression newCond = new ScalarFunctionCallExpression(context.getMetadataProvider()
+ .lookupFunction(AlgebricksBuiltinFunctions.AND), select.getCondition(),
+ new MutableObject<ILogicalExpression>(join.getCondition().getValue()));
+ join.getCondition().setValue(newCond);
+ }
+ }
+ }
+
+ private static void copySelectToBranch(SelectOperator select, Mutable<ILogicalOperator> branch,
+ IOptimizationContext context) throws AlgebricksException {
+ ILogicalOperator newSelect = new SelectOperator(select.getCondition());
+ Mutable<ILogicalOperator> newRef = new MutableObject<ILogicalOperator>(branch.getValue());
+ newSelect.getInputs().add(newRef);
+ branch.setValue(newSelect);
+ context.computeAndSetTypeEnvironmentForOperator(newSelect);
+ }
+}
\ No newline at end of file
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSubplanWithAggregateDownThroughProductRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSubplanWithAggregateDownThroughProductRule.java
new file mode 100644
index 0000000..d7bd543
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSubplanWithAggregateDownThroughProductRule.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.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 edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushSubplanWithAggregateDownThroughProductRule 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 {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
+ if (op1.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ SubplanOperator subplan = (SubplanOperator) op1;
+ if (subplan.getNestedPlans().size() != 1) {
+ return false;
+ }
+ ILogicalPlan p = subplan.getNestedPlans().get(0);
+ if (p.getRoots().size() != 1) {
+ return false;
+ }
+ Mutable<ILogicalOperator> r = p.getRoots().get(0);
+ if (((AbstractLogicalOperator) r.getValue()).getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
+
+ Set<LogicalVariable> free = new HashSet<LogicalVariable>();
+ OperatorPropertiesUtil.getFreeVariablesInSubplans(subplan, free);
+
+ Mutable<ILogicalOperator> op2Ref = op1.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op2Ref.getValue();
+ if (op2.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+ return false;
+ }
+ AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op2;
+ if (!OperatorPropertiesUtil.isAlwaysTrueCond(join.getCondition().getValue())) {
+ return false;
+ }
+
+ Mutable<ILogicalOperator> b0Ref = op2.getInputs().get(0);
+ ILogicalOperator b0 = b0Ref.getValue();
+ List<LogicalVariable> b0Scm = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(b0, b0Scm);
+ if (b0Scm.containsAll(free)) {
+ // push subplan on left branch
+ op2Ref.setValue(b0);
+ b0Ref.setValue(op1);
+ opRef.setValue(op2);
+ return true;
+ } else {
+ Mutable<ILogicalOperator> b1Ref = op2.getInputs().get(1);
+ ILogicalOperator b1 = b1Ref.getValue();
+ List<LogicalVariable> b1Scm = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(b1, b1Scm);
+ if (b1Scm.containsAll(free)) {
+ // push subplan on right branch
+ op2Ref.setValue(b1);
+ b1Ref.setValue(op1);
+ opRef.setValue(op2);
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ReinferAllTypesRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ReinferAllTypesRule.java
new file mode 100644
index 0000000..e18a380
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ReinferAllTypesRule.java
@@ -0,0 +1,50 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class ReinferAllTypesRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ if (context.checkIfInDontApplySet(this, opRef.getValue())) {
+ return false;
+ }
+ typeOpRec(opRef, context);
+ return true;
+ }
+
+ private void typePlan(ILogicalPlan p, IOptimizationContext context) throws AlgebricksException {
+ for (Mutable<ILogicalOperator> r : p.getRoots()) {
+ typeOpRec(r, context);
+ }
+ }
+
+ private void typeOpRec(Mutable<ILogicalOperator> r, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) r.getValue();
+ for (Mutable<ILogicalOperator> i : op.getInputs()) {
+ typeOpRec(i, context);
+ }
+ if (op.hasNestedPlans()) {
+ for (ILogicalPlan p : ((AbstractOperatorWithNestedPlans) op).getNestedPlans()) {
+ typePlan(p, context);
+ }
+ }
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ context.addToDontApplySet(this, op);
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveRedundantProjectionRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveRedundantProjectionRule.java
new file mode 100644
index 0000000..f5ebe77
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveRedundantProjectionRule.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/*
+ * project [var-list1]
+ * project [var-list2]
+ * P
+ *
+ * if var-list1.equals(var-list2) becomes
+ *
+ * project [var-list1]
+ * P
+ *
+ */
+
+public class RemoveRedundantProjectionRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
+ if (op1.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+ Mutable<ILogicalOperator> opRef2 = op1.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+ if (op2.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+ return false;
+ }
+ ProjectOperator pi2 = (ProjectOperator) op2;
+ opRef2.setValue(pi2.getInputs().get(0).getValue());
+ } else {
+ if (op1.getInputs().size() <= 0)
+ return false;
+ Mutable<ILogicalOperator> opRef2 = op1.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+ if (op2.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+ return false;
+ }
+ if (op2.getInputs().size() <= 0)
+ return false;
+ Mutable<ILogicalOperator> opRef3 = op2.getInputs().get(0);
+ AbstractLogicalOperator op3 = (AbstractLogicalOperator) opRef3.getValue();
+
+ List<LogicalVariable> liveVars2 = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> liveVars3 = new ArrayList<LogicalVariable>();
+
+ VariableUtilities.getLiveVariables(op2, liveVars2);
+ VariableUtilities.getLiveVariables(op3, liveVars3);
+
+ if (!VariableUtilities.varListEqualUnordered(liveVars2, liveVars3))
+ return false;
+ opRef2.setValue(op3);
+ }
+
+ return true;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
new file mode 100644
index 0000000..c53ea0a
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
@@ -0,0 +1,163 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+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.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class RemoveUnusedAssignAndAggregateRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ if (context.checkIfInDontApplySet(this, opRef.getValue())) {
+ return false;
+ }
+ Set<LogicalVariable> toRemove = new HashSet<LogicalVariable>();
+ collectUnusedAssignedVars((AbstractLogicalOperator) opRef.getValue(), toRemove, true, context);
+ boolean smthToRemove = !toRemove.isEmpty();
+ if (smthToRemove) {
+ removeUnusedAssigns(opRef, toRemove, context);
+ }
+ return smthToRemove;
+ }
+
+ private void removeUnusedAssigns(Mutable<ILogicalOperator> opRef, Set<LogicalVariable> toRemove,
+ IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ while (removeFromAssigns(op, toRemove, context) == 0) {
+ if (op.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+ break;
+ }
+ op = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ opRef.setValue(op);
+ }
+ Iterator<Mutable<ILogicalOperator>> childIter = op.getInputs().iterator();
+ while (childIter.hasNext()) {
+ Mutable<ILogicalOperator> cRef = childIter.next();
+ removeUnusedAssigns(cRef, toRemove, context);
+ }
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans opWithNest = (AbstractOperatorWithNestedPlans) op;
+ Iterator<ILogicalPlan> planIter = opWithNest.getNestedPlans().iterator();
+ while (planIter.hasNext()) {
+ ILogicalPlan p = planIter.next();
+ for (Mutable<ILogicalOperator> r : p.getRoots()) {
+ removeUnusedAssigns(r, toRemove, context);
+ }
+ }
+ }
+ }
+
+ private int removeFromAssigns(AbstractLogicalOperator op, Set<LogicalVariable> toRemove,
+ IOptimizationContext context) throws AlgebricksException {
+ if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator assign = (AssignOperator) op;
+ if (removeUnusedVarsAndExprs(toRemove, assign.getVariables(), assign.getExpressions())) {
+ context.computeAndSetTypeEnvironmentForOperator(assign);
+ }
+ return assign.getVariables().size();
+ } else if (op.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+ AggregateOperator agg = (AggregateOperator) op;
+ if (removeUnusedVarsAndExprs(toRemove, agg.getVariables(), agg.getExpressions())) {
+ context.computeAndSetTypeEnvironmentForOperator(agg);
+ }
+ return agg.getVariables().size();
+ } else if (op.getOperatorTag() == LogicalOperatorTag.UNNEST) {
+ UnnestOperator uOp = (UnnestOperator) op;
+ LogicalVariable pVar = uOp.getPositionalVariable();
+ if (pVar != null && toRemove.contains(pVar)) {
+ uOp.setPositionalVariable(null);
+ }
+ }
+ return -1;
+ }
+
+ private boolean removeUnusedVarsAndExprs(Set<LogicalVariable> toRemove, List<LogicalVariable> varList,
+ List<Mutable<ILogicalExpression>> exprList) {
+ boolean changed = false;
+ Iterator<LogicalVariable> varIter = varList.iterator();
+ Iterator<Mutable<ILogicalExpression>> exprIter = exprList.iterator();
+ while (varIter.hasNext()) {
+ LogicalVariable v = varIter.next();
+ exprIter.next();
+ if (toRemove.contains(v)) {
+ varIter.remove();
+ exprIter.remove();
+ changed = true;
+ }
+ }
+ return changed;
+ }
+
+ private void collectUnusedAssignedVars(AbstractLogicalOperator op, Set<LogicalVariable> toRemove, boolean first,
+ IOptimizationContext context) throws AlgebricksException {
+ if (!first) {
+ context.addToDontApplySet(this, op);
+ }
+ for (Mutable<ILogicalOperator> c : op.getInputs()) {
+ collectUnusedAssignedVars((AbstractLogicalOperator) c.getValue(), toRemove, false, context);
+ }
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans opWithNested = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan plan : opWithNested.getNestedPlans()) {
+ for (Mutable<ILogicalOperator> r : plan.getRoots()) {
+ collectUnusedAssignedVars((AbstractLogicalOperator) r.getValue(), toRemove, false, context);
+ }
+ }
+ }
+ if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator assign = (AssignOperator) op;
+ toRemove.addAll(assign.getVariables());
+ } else if (op.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+ AggregateOperator agg = (AggregateOperator) op;
+ toRemove.addAll(agg.getVariables());
+ } else if (op.getOperatorTag() == LogicalOperatorTag.UNNEST) {
+ UnnestOperator uOp = (UnnestOperator) op;
+ LogicalVariable pVar = uOp.getPositionalVariable();
+ if (pVar != null) {
+ toRemove.add(pVar);
+ }
+ }
+ List<LogicalVariable> used = new LinkedList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op, used);
+ toRemove.removeAll(used);
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
new file mode 100644
index 0000000..38cf96e
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -0,0 +1,350 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+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.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+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.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+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.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AggregatePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.DataSourceScanPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.EmptyTupleSourcePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.InMemoryStableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.IndexInsertDeletePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.InsertDeletePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.MicroPreclusteredGroupByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.NestedTupleSourcePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.PreSortedDistinctByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.PreclusteredGroupByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.ReplicatePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.RunningAggregatePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SinkPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SinkWritePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamDiePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamLimitPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamSelectPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StringStreamingScriptPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SubplanPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.UnionAllPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.UnnestPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.WriteResultPOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.algebricks.rewriter.util.JoinUtils;
+
+public class SetAlgebricksPhysicalOperatorsRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ // if (context.checkIfInDontApplySet(this, op)) {
+ // return false;
+ // }
+ if (op.getPhysicalOperator() != null) {
+ return false;
+ }
+
+ computeDefaultPhysicalOp(op, true, context);
+ // context.addToDontApplySet(this, op);
+ return true;
+ }
+
+ private static void setPhysicalOperators(ILogicalPlan plan, boolean topLevelOp, IOptimizationContext context)
+ throws AlgebricksException {
+ for (Mutable<ILogicalOperator> root : plan.getRoots()) {
+ computeDefaultPhysicalOp((AbstractLogicalOperator) root.getValue(), topLevelOp, context);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ private static void computeDefaultPhysicalOp(AbstractLogicalOperator op, boolean topLevelOp,
+ IOptimizationContext context) throws AlgebricksException {
+ PhysicalOptimizationConfig physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
+ if (op.getPhysicalOperator() == null) {
+ switch (op.getOperatorTag()) {
+ case AGGREGATE: {
+ op.setPhysicalOperator(new AggregatePOperator());
+ break;
+ }
+ case ASSIGN: {
+ op.setPhysicalOperator(new AssignPOperator());
+ break;
+ }
+ case DISTINCT: {
+ DistinctOperator distinct = (DistinctOperator) op;
+ distinct.setPhysicalOperator(new PreSortedDistinctByPOperator(distinct.getDistinctByVarList()));
+ break;
+ }
+ case EMPTYTUPLESOURCE: {
+ op.setPhysicalOperator(new EmptyTupleSourcePOperator());
+ break;
+ }
+ case EXCHANGE: {
+ if (op.getPhysicalOperator() == null) {
+ throw new AlgebricksException("Implementation for EXCHANGE operator was not set.");
+ }
+ // implem. choice for exchange should be set by a parent op.
+ break;
+ }
+ case GROUP: {
+ GroupByOperator gby = (GroupByOperator) op;
+
+ if (gby.getNestedPlans().size() == 1) {
+ ILogicalPlan p0 = gby.getNestedPlans().get(0);
+ if (p0.getRoots().size() == 1) {
+ if (gby.getAnnotations().get(OperatorAnnotations.USE_HASH_GROUP_BY) == Boolean.TRUE
+ || gby.getAnnotations().get(OperatorAnnotations.USE_EXTERNAL_GROUP_BY) == Boolean.TRUE) {
+ if (!topLevelOp) {
+ throw new NotImplementedException(
+ "External hash group-by for nested grouping is not implemented.");
+ }
+ ExternalGroupByPOperator externalGby = new ExternalGroupByPOperator(
+ gby.getGroupByList(), physicalOptimizationConfig.getMaxFramesExternalGroupBy(),
+ physicalOptimizationConfig.getExternalGroupByTableSize());
+ op.setPhysicalOperator(externalGby);
+ generateMergeAggregationExpressions(gby, context);
+ break;
+ }
+ }
+ }
+
+ List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gbyList = gby.getGroupByList();
+ List<LogicalVariable> columnList = new ArrayList<LogicalVariable>(gbyList.size());
+ for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gbyList) {
+ ILogicalExpression expr = p.second.getValue();
+ if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
+ columnList.add(varRef.getVariableReference());
+ }
+ }
+ if (topLevelOp) {
+ op.setPhysicalOperator(new PreclusteredGroupByPOperator(columnList));
+ } else {
+ op.setPhysicalOperator(new MicroPreclusteredGroupByPOperator(columnList));
+ }
+ break;
+ }
+ case INNERJOIN: {
+ JoinUtils.setJoinAlgorithmAndExchangeAlgo((InnerJoinOperator) op, context);
+ break;
+ }
+ case LEFTOUTERJOIN: {
+ JoinUtils.setJoinAlgorithmAndExchangeAlgo((LeftOuterJoinOperator) op, context);
+ break;
+ }
+ case LIMIT: {
+ LimitOperator opLim = (LimitOperator) op;
+ op.setPhysicalOperator(new StreamLimitPOperator(opLim.isTopmostLimitOp()
+ && opLim.getExecutionMode() == ExecutionMode.PARTITIONED));
+ break;
+ }
+ case NESTEDTUPLESOURCE: {
+ op.setPhysicalOperator(new NestedTupleSourcePOperator());
+ break;
+ }
+ case ORDER: {
+ OrderOperator oo = (OrderOperator) op;
+ for (Pair<IOrder, Mutable<ILogicalExpression>> p : oo.getOrderExpressions()) {
+ ILogicalExpression e = p.second.getValue();
+ if (e.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new AlgebricksException("Order expression " + e + " has not been normalized.");
+ }
+ }
+ if (topLevelOp) {
+ op.setPhysicalOperator(new StableSortPOperator(physicalOptimizationConfig
+ .getMaxFramesExternalSort()));
+ } else {
+ op.setPhysicalOperator(new InMemoryStableSortPOperator());
+ }
+ break;
+ }
+ case PROJECT: {
+ op.setPhysicalOperator(new StreamProjectPOperator());
+ break;
+ }
+ case RUNNINGAGGREGATE: {
+ op.setPhysicalOperator(new RunningAggregatePOperator());
+ break;
+ }
+ case REPLICATE: {
+ op.setPhysicalOperator(new ReplicatePOperator());
+ break;
+ }
+ case SCRIPT: {
+ op.setPhysicalOperator(new StringStreamingScriptPOperator());
+ break;
+ }
+ case SELECT: {
+ op.setPhysicalOperator(new StreamSelectPOperator());
+ break;
+ }
+ case SUBPLAN: {
+ op.setPhysicalOperator(new SubplanPOperator());
+ break;
+ }
+ case UNIONALL: {
+ op.setPhysicalOperator(new UnionAllPOperator());
+ break;
+ }
+
+ case UNNEST: {
+ op.setPhysicalOperator(new UnnestPOperator());
+ break;
+ }
+ case DATASOURCESCAN: {
+ DataSourceScanOperator scan = (DataSourceScanOperator) op;
+ IDataSource dataSource = scan.getDataSource();
+ DataSourceScanPOperator dss = new DataSourceScanPOperator(dataSource);
+ IMetadataProvider mp = context.getMetadataProvider();
+ if (mp.scannerOperatorIsLeaf(dataSource)) {
+ dss.disableJobGenBelowMe();
+ }
+ op.setPhysicalOperator(dss);
+ break;
+ }
+ case WRITE: {
+ op.setPhysicalOperator(new SinkWritePOperator());
+ break;
+ }
+ case WRITE_RESULT: {
+ WriteResultOperator opLoad = (WriteResultOperator) op;
+ LogicalVariable payload;
+ List<LogicalVariable> keys = new ArrayList<LogicalVariable>();
+ payload = getKeysAndLoad(opLoad.getPayloadExpression(), opLoad.getKeyExpressions(), keys);
+ op.setPhysicalOperator(new WriteResultPOperator(opLoad.getDataSource(), payload, keys));
+ break;
+ }
+ case INSERT_DELETE: {
+ InsertDeleteOperator opLoad = (InsertDeleteOperator) op;
+ LogicalVariable payload;
+ List<LogicalVariable> keys = new ArrayList<LogicalVariable>();
+ payload = getKeysAndLoad(opLoad.getPayloadExpression(), opLoad.getPrimaryKeyExpressions(), keys);
+ op.setPhysicalOperator(new InsertDeletePOperator(payload, keys, opLoad.getDataSource()));
+ break;
+ }
+ case INDEX_INSERT_DELETE: {
+ IndexInsertDeleteOperator opInsDel = (IndexInsertDeleteOperator) op;
+ List<LogicalVariable> primaryKeys = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> secondaryKeys = new ArrayList<LogicalVariable>();
+ getKeys(opInsDel.getPrimaryKeyExpressions(), primaryKeys);
+ getKeys(opInsDel.getSecondaryKeyExpressions(), secondaryKeys);
+ op.setPhysicalOperator(new IndexInsertDeletePOperator(primaryKeys, secondaryKeys,
+ opInsDel.getFilterExpression(), opInsDel.getDataSourceIndex()));
+ break;
+ }
+ case SINK: {
+ op.setPhysicalOperator(new SinkPOperator());
+ break;
+ }
+ case DIE: {
+ op.setPhysicalOperator(new StreamDiePOperator());
+ break;
+ }
+ }
+ }
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans nested = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan p : nested.getNestedPlans()) {
+ setPhysicalOperators(p, false, context);
+ }
+ }
+ for (Mutable<ILogicalOperator> opRef : op.getInputs()) {
+ computeDefaultPhysicalOp((AbstractLogicalOperator) opRef.getValue(), topLevelOp, context);
+ }
+ }
+
+ private static void getKeys(List<Mutable<ILogicalExpression>> keyExpressions, List<LogicalVariable> keys) {
+ for (Mutable<ILogicalExpression> kExpr : keyExpressions) {
+ ILogicalExpression e = kExpr.getValue();
+ if (e.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new NotImplementedException();
+ }
+ keys.add(((VariableReferenceExpression) e).getVariableReference());
+ }
+ }
+
+ private static LogicalVariable getKeysAndLoad(Mutable<ILogicalExpression> payloadExpr,
+ List<Mutable<ILogicalExpression>> keyExpressions, List<LogicalVariable> keys) {
+ LogicalVariable payload;
+ if (payloadExpr.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new NotImplementedException();
+ }
+ payload = ((VariableReferenceExpression) payloadExpr.getValue()).getVariableReference();
+
+ for (Mutable<ILogicalExpression> kExpr : keyExpressions) {
+ ILogicalExpression e = kExpr.getValue();
+ if (e.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new NotImplementedException();
+ }
+ keys.add(((VariableReferenceExpression) e).getVariableReference());
+ }
+ return payload;
+ }
+
+ private static void generateMergeAggregationExpressions(GroupByOperator gby, IOptimizationContext context)
+ throws AlgebricksException {
+ if (gby.getNestedPlans().size() != 1) {
+ throw new AlgebricksException(
+ "External group-by currently works only for one nested plan with one root containing"
+ + "an aggregate and a nested-tuple-source.");
+ }
+ ILogicalPlan p0 = gby.getNestedPlans().get(0);
+ if (p0.getRoots().size() != 1) {
+ throw new AlgebricksException(
+ "External group-by currently works only for one nested plan with one root containing"
+ + "an aggregate and a nested-tuple-source.");
+ }
+ IMergeAggregationExpressionFactory mergeAggregationExpressionFactory = context
+ .getMergeAggregationExpressionFactory();
+ Mutable<ILogicalOperator> r0 = p0.getRoots().get(0);
+ AggregateOperator aggOp = (AggregateOperator) r0.getValue();
+ List<Mutable<ILogicalExpression>> aggFuncRefs = aggOp.getExpressions();
+ int n = aggOp.getExpressions().size();
+ List<Mutable<ILogicalExpression>> mergeExpressionRefs = new ArrayList<Mutable<ILogicalExpression>>();
+ for (int i = 0; i < n; i++) {
+ ILogicalExpression mergeExpr = mergeAggregationExpressionFactory.createMergeAggregation(aggFuncRefs.get(i)
+ .getValue(), context);
+ mergeExpressionRefs.add(new MutableObject<ILogicalExpression>(mergeExpr));
+ }
+ aggOp.setMergeExpressions(mergeExpressionRefs);
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetExecutionModeRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetExecutionModeRule.java
new file mode 100644
index 0000000..5bc3432
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetExecutionModeRule.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+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.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * This rule sets the executionMode property of an operator, w/o introducing
+ * EXCHANGE operators in the plan. Previously, i.e. before having physical
+ * optimizations in place, we were using the IntroduceExchangeRule, which was
+ * doing both, to both set excutionMode and introduce data exchange ops.
+ *
+ * @author Nicola
+ */
+public class SetExecutionModeRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ boolean changed = OperatorManipulationUtil.setOperatorMode(op);
+ if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.UNPARTITIONED
+ || op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.LOCAL) {
+ return changed;
+ }
+ switch (op.getOperatorTag()) {
+ // case DISTINCT:
+ // case AGGREGATE:
+ // case GROUP:
+ // case ORDER:
+ // case INNERJOIN:
+ // case LEFTOUTERJOIN: {
+ // op.setExecutionMode(ExecutionMode.GLOBAL);
+ // return true;
+ // }
+
+ case PARTITIONINGSPLIT: {
+ throw new NotImplementedException();
+ }
+ default: {
+ return changed;
+ }
+ }
+
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SimpleUnnestToProductRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SimpleUnnestToProductRule.java
new file mode 100644
index 0000000..b79c31b
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SimpleUnnestToProductRule.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.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.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.AbstractScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class SimpleUnnestToProductRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN
+ && op.getOperatorTag() != LogicalOperatorTag.UNNEST) {
+ return false;
+ }
+
+ Mutable<ILogicalOperator> opRef2 = op.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+
+ if (!(op2 instanceof AbstractScanOperator) && !descOrSelfIsSourceScan(op2)) {
+ return false;
+ }
+ // Make sure that op does not use any variables produced by op2.
+ if (!opsAreIndependent(op, op2)) {
+ return false;
+ }
+
+ InnerJoinOperator product = new InnerJoinOperator(
+ new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+
+ EmptyTupleSourceOperator ets = new EmptyTupleSourceOperator();
+ context.computeAndSetTypeEnvironmentForOperator(ets);
+ Mutable<ILogicalOperator> emptySrc = new MutableObject<ILogicalOperator>(ets);
+ List<Mutable<ILogicalOperator>> opInpList = op.getInputs();
+ opInpList.clear();
+ opInpList.add(emptySrc);
+ product.getInputs().add(opRef2); // outer branch
+ product.getInputs().add(new MutableObject<ILogicalOperator>(op));
+ opRef.setValue(product); // plug the product in the plan
+ context.computeAndSetTypeEnvironmentForOperator(product);
+ return true;
+ }
+
+ private boolean descOrSelfIsSourceScan(AbstractLogicalOperator op2) {
+ // Disregard data source scans in a subplan.
+ if (op2.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ if (op2.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN
+ && op2.getOperatorTag() != LogicalOperatorTag.UNNEST) {
+ return true;
+ }
+ for (Mutable<ILogicalOperator> cRef : op2.getInputs()) {
+ AbstractLogicalOperator alo = (AbstractLogicalOperator) cRef.getValue();
+ if (descOrSelfIsSourceScan(alo)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private boolean opsAreIndependent(ILogicalOperator unnestOp, ILogicalOperator outer) throws AlgebricksException {
+ List<LogicalVariable> opUsedVars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(unnestOp, opUsedVars);
+ Set<LogicalVariable> op2LiveVars = new HashSet<LogicalVariable>();
+ VariableUtilities.getLiveVariables(outer, op2LiveVars);
+ for (LogicalVariable usedVar : opUsedVars) {
+ if (op2LiveVars.contains(usedVar)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SubplanOutOfGroupRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SubplanOutOfGroupRule.java
new file mode 100644
index 0000000..09e03b5
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SubplanOutOfGroupRule.java
@@ -0,0 +1,121 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+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.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.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Looks for a nested group-by plan ending in
+ * subplan {
+ * ...
+ * }
+ * select (function-call: algebricks:not, Args:[function-call:
+ * algebricks:is-null, Args:[...]])
+ * nested tuple source -- |UNPARTITIONED|
+ */
+
+public class SubplanOutOfGroupRule 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 {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getValue();
+ if (op0.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+ GroupByOperator gby = (GroupByOperator) op0;
+
+ Iterator<ILogicalPlan> plansIter = gby.getNestedPlans().iterator();
+ ILogicalPlan p = null;
+ while (plansIter.hasNext()) {
+ p = plansIter.next();
+ }
+ if (p == null) {
+ return false;
+ }
+ if (p.getRoots().size() != 1) {
+ return false;
+ }
+ Mutable<ILogicalOperator> op1Ref = p.getRoots().get(0);
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) op1Ref.getValue();
+ boolean found = false;
+ while (op1.getInputs().size() == 1) {
+ if (op1.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+ SubplanOperator subplan = (SubplanOperator) op1;
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) subplan.getInputs().get(0).getValue();
+ if (OperatorPropertiesUtil.isNullTest(op2)) {
+ if (subplan.getNestedPlans().size() == 1) {
+ ILogicalPlan p1 = subplan.getNestedPlans().get(0);
+ if (p1.getRoots().size() == 1) {
+ AbstractLogicalOperator r1 = (AbstractLogicalOperator) p1.getRoots().get(0).getValue();
+ if (r1.getOperatorTag() == LogicalOperatorTag.INNERJOIN
+ || r1.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
+ // now, check that it propagates all variables,
+ // so it can be pushed
+ List<LogicalVariable> op2Vars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(op2, op2Vars);
+ List<LogicalVariable> op1Vars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(subplan, op1Vars);
+ if (op1Vars.containsAll(op2Vars)) {
+ found = true;
+ break;
+ }
+ }
+ }
+ }
+ }
+ }
+ op1Ref = op1.getInputs().get(0);
+ op1 = (AbstractLogicalOperator) op1Ref.getValue();
+ }
+ if (!found) {
+ return false;
+ }
+
+ ILogicalOperator subplan = op1;
+ ILogicalOperator op2 = op1.getInputs().get(0).getValue();
+ op1Ref.setValue(op2);
+ Mutable<ILogicalOperator> opUnderRef = gby.getInputs().get(0);
+ ILogicalOperator opUnder = opUnderRef.getValue();
+ subplan.getInputs().clear();
+ subplan.getInputs().add(new MutableObject<ILogicalOperator>(opUnder));
+ opUnderRef.setValue(subplan);
+
+ return true;
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/util/JoinUtils.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/util/JoinUtils.java
new file mode 100644
index 0000000..ddc00e3
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/util/JoinUtils.java
@@ -0,0 +1,216 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.util;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+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.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation.BroadcastSide;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.LogicalPropertiesVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator.JoinPartitioningType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HybridHashJoinPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.InMemoryHashJoinPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.NLJoinPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILogicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+
+public class JoinUtils {
+
+ private final static int MB = 1048576;
+
+ private final static double DEFAULT_FUDGE_FACTOR = 1.3;
+ private final static int MAX_RECORDS_PER_FRAME = 512;
+ private final static int DEFAULT_FRAME_SIZE = 32768;
+ private final static int MAX_LEFT_INPUT_SIZE_HYBRID_HASH = (int) (140L * 1024 * MB / DEFAULT_FRAME_SIZE);
+ private final static int DEFAULT_MEMORY_SIZE_HYBRID_HASH = (int) (256L * MB / DEFAULT_FRAME_SIZE);
+
+ public static void setJoinAlgorithmAndExchangeAlgo(AbstractBinaryJoinOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ List<LogicalVariable> sideLeft = new LinkedList<LogicalVariable>();
+ List<LogicalVariable> sideRight = new LinkedList<LogicalVariable>();
+ List<LogicalVariable> varsLeft = op.getInputs().get(0).getValue().getSchema();
+ List<LogicalVariable> varsRight = op.getInputs().get(1).getValue().getSchema();
+ if (isHashJoinCondition(op.getCondition().getValue(), varsLeft, varsRight, sideLeft, sideRight)) {
+ BroadcastSide side = getBroadcastJoinSide(op.getCondition().getValue(), varsLeft, varsRight);
+ if (side == null) {
+ setHashJoinOp(op, JoinPartitioningType.PAIRWISE, sideLeft, sideRight, context);
+ } else {
+ switch (side) {
+ case RIGHT:
+ setHashJoinOp(op, JoinPartitioningType.BROADCAST, sideLeft, sideRight, context);
+ break;
+ case LEFT:
+ Mutable<ILogicalOperator> opRef0 = op.getInputs().get(0);
+ Mutable<ILogicalOperator> opRef1 = op.getInputs().get(1);
+ ILogicalOperator tmp = opRef0.getValue();
+ opRef0.setValue(opRef1.getValue());
+ opRef1.setValue(tmp);
+ setHashJoinOp(op, JoinPartitioningType.BROADCAST, sideRight, sideLeft, context);
+ break;
+ default:
+ setHashJoinOp(op, JoinPartitioningType.PAIRWISE, sideLeft, sideRight, context);
+ }
+ }
+ } else {
+ setNLJoinOp(op);
+ }
+ }
+
+ private static void setNLJoinOp(AbstractBinaryJoinOperator op) {
+ op.setPhysicalOperator(new NLJoinPOperator(op.getJoinKind(), JoinPartitioningType.BROADCAST,
+ DEFAULT_MEMORY_SIZE_HYBRID_HASH));
+ }
+
+ private static void setHashJoinOp(AbstractBinaryJoinOperator op, JoinPartitioningType partitioningType,
+ List<LogicalVariable> sideLeft, List<LogicalVariable> sideRight, IOptimizationContext context)
+ throws AlgebricksException {
+ op.setPhysicalOperator(new HybridHashJoinPOperator(op.getJoinKind(), partitioningType, sideLeft, sideRight,
+ DEFAULT_MEMORY_SIZE_HYBRID_HASH, MAX_LEFT_INPUT_SIZE_HYBRID_HASH, MAX_RECORDS_PER_FRAME,
+ DEFAULT_FUDGE_FACTOR));
+ if (partitioningType == JoinPartitioningType.BROADCAST) {
+ hybridToInMemHashJoin(op, context);
+ }
+ // op.setPhysicalOperator(new
+ // InMemoryHashJoinPOperator(op.getJoinKind(), partitioningType,
+ // sideLeft, sideRight,
+ // 1024 * 512));
+ }
+
+ private static void hybridToInMemHashJoin(AbstractBinaryJoinOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ ILogicalOperator opBuild = op.getInputs().get(1).getValue();
+ LogicalPropertiesVisitor.computeLogicalPropertiesDFS(opBuild, context);
+ ILogicalPropertiesVector v = context.getLogicalPropertiesVector(opBuild);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("// HybridHashJoin inner branch -- Logical properties for " + opBuild
+ + ": " + v + "\n");
+ if (v != null) {
+ int size2 = v.getMaxOutputFrames();
+ HybridHashJoinPOperator hhj = (HybridHashJoinPOperator) op.getPhysicalOperator();
+ if (size2 > 0 && size2 * hhj.getFudgeFactor() <= hhj.getMemSizeInFrames()) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("// HybridHashJoin inner branch " + opBuild
+ + " fits in memory\n");
+ // maintains the local properties on the probe side
+ op.setPhysicalOperator(new InMemoryHashJoinPOperator(hhj.getKind(), hhj.getPartitioningType(), hhj
+ .getKeysLeftBranch(), hhj.getKeysRightBranch(), v.getNumberOfTuples() * 2));
+ }
+ }
+
+ }
+
+ private static boolean isHashJoinCondition(ILogicalExpression e, Collection<LogicalVariable> inLeftAll,
+ Collection<LogicalVariable> inRightAll, Collection<LogicalVariable> outLeftFields,
+ Collection<LogicalVariable> outRightFields) {
+ switch (e.getExpressionTag()) {
+ case FUNCTION_CALL: {
+ AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) e;
+ FunctionIdentifier fi = fexp.getFunctionIdentifier();
+ if (fi.equals(AlgebricksBuiltinFunctions.AND)) {
+ for (Mutable<ILogicalExpression> a : fexp.getArguments()) {
+ if (!isHashJoinCondition(a.getValue(), inLeftAll, inRightAll, outLeftFields,
+ outRightFields)) {
+ return false;
+ }
+ }
+ return true;
+ } else {
+ ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(fi);
+ if (ck != ComparisonKind.EQ) {
+ return false;
+ }
+ ILogicalExpression opLeft = fexp.getArguments().get(0).getValue();
+ ILogicalExpression opRight = fexp.getArguments().get(1).getValue();
+ if (opLeft.getExpressionTag() != LogicalExpressionTag.VARIABLE
+ || opRight.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return false;
+ }
+ LogicalVariable var1 = ((VariableReferenceExpression) opLeft).getVariableReference();
+ if (inLeftAll.contains(var1) && !outLeftFields.contains(var1)) {
+ outLeftFields.add(var1);
+ } else if (inRightAll.contains(var1) && !outRightFields.contains(var1)) {
+ outRightFields.add(var1);
+ } else {
+ return false;
+ }
+ LogicalVariable var2 = ((VariableReferenceExpression) opRight).getVariableReference();
+ if (inLeftAll.contains(var2) && !outLeftFields.contains(var2)) {
+ outLeftFields.add(var2);
+ } else if (inRightAll.contains(var2) && !outRightFields.contains(var2)) {
+ outRightFields.add(var2);
+ } else {
+ return false;
+ }
+ return true;
+ }
+ }
+ default: {
+ return false;
+ }
+ }
+ }
+
+ private static BroadcastSide getBroadcastJoinSide(ILogicalExpression e, List<LogicalVariable> varsLeft,
+ List<LogicalVariable> varsRight) {
+ if (e.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return null;
+ }
+ AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) e;
+ IExpressionAnnotation ann = fexp.getAnnotations().get(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY);
+ if (ann == null) {
+ return null;
+ }
+ BroadcastSide side = (BroadcastSide) ann.getObject();
+ if (side == null) {
+ return null;
+ }
+ int i;
+ switch (side) {
+ case LEFT:
+ i = 0;
+ break;
+ case RIGHT:
+ i = 1;
+ break;
+ default:
+ return null;
+ }
+ ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
+ fexp.getArguments().get(i).getValue().getUsedVariables(vars);
+ if (varsLeft.containsAll(vars)) {
+ return BroadcastSide.LEFT;
+ } else if (varsRight.containsAll(vars)) {
+ return BroadcastSide.RIGHT;
+ } else {
+ return null;
+ }
+ }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/util/PhysicalOptimizationsUtil.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/util/PhysicalOptimizationsUtil.java
new file mode 100644
index 0000000..d7ebfb4
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/util/PhysicalOptimizationsUtil.java
@@ -0,0 +1,57 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.util;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.FDsAndEquivClassesVisitor;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+
+public class PhysicalOptimizationsUtil {
+
+ public static void computeFDsAndEquivalenceClasses(AbstractLogicalOperator op, IOptimizationContext ctx)
+ throws AlgebricksException {
+ FDsAndEquivClassesVisitor visitor = new FDsAndEquivClassesVisitor();
+ Set<ILogicalOperator> visitSet = new HashSet<ILogicalOperator>();
+ computeFDsAndEqClassesWithVisitorRec(op, ctx, visitor, visitSet);
+ }
+
+ private static void computeFDsAndEqClassesWithVisitorRec(AbstractLogicalOperator op, IOptimizationContext ctx,
+ FDsAndEquivClassesVisitor visitor, Set<ILogicalOperator> visitSet) throws AlgebricksException {
+ visitSet.add(op);
+ for (Mutable<ILogicalOperator> i : op.getInputs()) {
+ computeFDsAndEqClassesWithVisitorRec((AbstractLogicalOperator) i.getValue(), ctx, visitor, visitSet);
+ }
+ if (op.hasNestedPlans()) {
+ for (ILogicalPlan p : ((AbstractOperatorWithNestedPlans) op).getNestedPlans()) {
+ for (Mutable<ILogicalOperator> r : p.getRoots()) {
+ AbstractLogicalOperator rootOp = (AbstractLogicalOperator) r.getValue();
+ computeFDsAndEqClassesWithVisitorRec(rootOp, ctx, visitor, visitSet);
+ }
+ }
+ }
+ if (op.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ NestedTupleSourceOperator nts = (NestedTupleSourceOperator) op;
+ ILogicalOperator source = nts.getDataSourceReference().getValue().getInputs().get(0).getValue();
+ if (!visitSet.contains(source)) {
+ computeFDsAndEqClassesWithVisitorRec((AbstractLogicalOperator) source, ctx, visitor, visitSet);
+ }
+ }
+ op.accept(visitor, ctx);
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("--> op. type = " + op.getOperatorTag() + "\n"
+ + " equiv. classes = " + ctx.getEquivalenceClassMap(op) + "\n" + " FDs = "
+ + ctx.getFDList(op) + "\n");
+ }
+ }
+
+}