You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@asterixdb.apache.org by "Preston Carman (Code Review)" <do...@asterix-gerrit.ics.uci.edu> on 2015/08/13 20:25:56 UTC

Change in hyracks[master]: Abstracting algebricks rules out of AsterixDB.

Preston Carman has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/352

Change subject: Abstracting algebricks rules out of AsterixDB.
......................................................................

Abstracting algebricks rules out of AsterixDB.

Change-Id: I80789e3827b4224b92b0b9d7247919f862f6f939
---
A algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
M algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
M algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/MoveFreeVariableOperatorOutOfSubplanRule.java
3 files changed, 253 insertions(+), 14 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/52/352/1

diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
new file mode 100644
index 0000000..8f8e62b
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
@@ -0,0 +1,155 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.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.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+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.FunctionIdentifier;
+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.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Factors out function expressions from each comparison function or similarity function in join condition by
+ * assigning them to a variables, and replacing the function expressions with references to those variables.
+ * Examples:
+ * Plan with function expressions in comparison or similarity condition of join expression.
+ * Generates one assign operator per extracted function expression.
+ *
+ * <pre>
+ * Before plan:
+ * 
+ *   join ( eq( funcX($$1), funcX($$2) ) )
+ * 
+ * After plan:
+ * 
+ *   join (eq($$3,$$4))
+ *   assign [$$4] <- [funcY($$2)]
+ *   assign [$$3] <- [funcX($$1)]
+ * </pre>
+ */
+public class ExtractFunctionsFromJoinConditionRule 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.INNERJOIN
+                && op.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+            return false;
+        }
+        AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) op;
+        ILogicalExpression expr = joinOp.getCondition().getValue();
+
+        return assignFunctionExpressions(joinOp, expr, context);
+
+    }
+
+    private boolean assignFunctionExpressions(AbstractLogicalOperator joinOp, ILogicalExpression expr,
+            IOptimizationContext context) throws AlgebricksException {
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) expr;
+        FunctionIdentifier fi = fexp.getFunctionIdentifier();
+
+        boolean modified = false;
+        if (fi.equals(AlgebricksBuiltinFunctions.AND) || fi.equals(AlgebricksBuiltinFunctions.OR)
+                || processArgumentsToFunction(fi)) {
+            for (Mutable<ILogicalExpression> a : fexp.getArguments()) {
+                if (assignFunctionExpressions(joinOp, a.getValue(), context)) {
+                    modified = true;
+                }
+            }
+            return modified;
+        } else if (AlgebricksBuiltinFunctions.isComparisonFunction(fi) || isComparisonFunction(fi)) {
+            for (Mutable<ILogicalExpression> exprRef : fexp.getArguments()) {
+                if (exprRef.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    LogicalVariable newVar = context.newVar();
+                    AssignOperator newAssign = new AssignOperator(newVar, new MutableObject<ILogicalExpression>(exprRef
+                            .getValue().cloneExpression()));
+                    newAssign.setExecutionMode(joinOp.getExecutionMode());
+
+                    // Place assign below joinOp.
+                    List<LogicalVariable> used = new ArrayList<LogicalVariable>();
+                    VariableUtilities.getUsedVariables(newAssign, used);
+
+                    Mutable<ILogicalOperator> leftBranchRef = joinOp.getInputs().get(0);
+                    ILogicalOperator leftBranch = leftBranchRef.getValue();
+                    List<LogicalVariable> leftBranchVariables = new ArrayList<LogicalVariable>();
+                    VariableUtilities.getLiveVariables(leftBranch, leftBranchVariables);
+                    if (leftBranchVariables.containsAll(used)) {
+                        // place assign on left branch
+                        newAssign.getInputs().add(new MutableObject<ILogicalOperator>(leftBranch));
+                        leftBranchRef.setValue(newAssign);
+                        modified = true;
+                    } else {
+                        Mutable<ILogicalOperator> rightBranchRef = joinOp.getInputs().get(1);
+                        ILogicalOperator rightBranch = rightBranchRef.getValue();
+                        List<LogicalVariable> rightBranchVariables = new ArrayList<LogicalVariable>();
+                        VariableUtilities.getLiveVariables(rightBranch, rightBranchVariables);
+                        if (rightBranchVariables.containsAll(used)) {
+                            // place assign on right branch
+                            newAssign.getInputs().add(new MutableObject<ILogicalOperator>(rightBranch));
+                            rightBranchRef.setValue(newAssign);
+                            modified = true;
+                        }
+                    }
+
+                    if (modified) {
+                        // Replace original expr with variable reference.
+                        exprRef.setValue(new VariableReferenceExpression(newVar));
+                        context.computeAndSetTypeEnvironmentForOperator(newAssign);
+                        context.computeAndSetTypeEnvironmentForOperator(joinOp);
+                    }
+                }
+            }
+            return modified;
+        } else {
+            return false;
+        }
+    }
+
+    protected boolean processArgumentsToFunction(FunctionIdentifier fi) {
+        return false;
+    }
+
+    protected boolean isComparisonFunction(FunctionIdentifier fi) {
+        return false;
+    }
+
+}
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
index ab7835d..08d3fbf 100644
--- 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
@@ -60,6 +60,36 @@
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
 
+/**
+ * The rule searches for unnest(iterate) operator followed by an assign(child)
+ * operator and merges the assign into the unnest operator.
+ *
+ * <pre>
+ * Before
+ * 
+ *   plan__parent
+ *   SUBPLAN {
+ *     PROJECT?
+ *     AGGREGATE
+ *     INNER_JOIN | LEFT_OUTER_JOIN ($condition, $left, $right)
+ *   }
+ *   plan__child
+ * 
+ *   where $condition does not equal a constant true.
+ * 
+ * After
+ * 
+ *   plan__parent
+ *   GROUP_BY {
+ *     ASSIGN( $v2 : algebricks:not( is_null( $right ) ) )
+ *     NESTED_TUPLE_SOURCE
+ *   }
+ *   plan__child
+ * </pre>
+ *
+ * @author prestonc
+ */
+
 public class IntroduceGroupByForSubplanRule implements IAlgebraicRewriteRule {
 
     @Override
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/MoveFreeVariableOperatorOutOfSubplanRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/MoveFreeVariableOperatorOutOfSubplanRule.java
index b01f320..5867abd 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/MoveFreeVariableOperatorOutOfSubplanRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/MoveFreeVariableOperatorOutOfSubplanRule.java
@@ -1,11 +1,11 @@
 /*
- * Copyright 2009-2014 by The Regents of the University of California
+ * Copyright 2009-2013 by The Regents of the University of California
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * you may obtain a copy of the License from
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -30,19 +30,43 @@
 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.rewriter.rules.AbstractDecorrelationRule;
 
+/**
+ * The rule searches for operators that can be moved outside the subplan.
+ *
+ * <pre>
+ * Before
+ * 
+ *   %PARENT_PLAN
+ *   SUBPLAN{
+ *     %NESTED_OPERATORS_B+
+ *     ASSIGN || %SUBPLAN
+ *     %NESTED_OPERATORS_A*
+ *     NESTEDTUPLESOURCE
+ *   }
+ *   %CHILD_PLAN
+ * 
+ *   where
+ *     %SUBPLAN has one nested plan with a root AGGREGATE operator.
+ * 
+ * After
+ * 
+ *   %PARENT_PLAN
+ *   SUBPLAN{
+ *     %NESTED_OPERATORS_B+
+ *     %NESTED_OPERATORS_A*
+ *     NESTEDTUPLESOURCE
+ *   }
+ *   ASSIGN || %SUBPLAN
+ *   %CHILD_PLAN
+ * </pre>
+ */
 public class MoveFreeVariableOperatorOutOfSubplanRule extends AbstractDecorrelationRule {
 
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
-        /*
-         * This rule looks for an assign within a subplan that uses only 
-         * variables from outside of the subplan
-         * 
-         * It moves this assign outside of the subplan
-         * 
-         */
         AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getValue();
         if (op0.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
             return false;
@@ -90,11 +114,38 @@
 
             //Get its used variables
             Set<LogicalVariable> used = new HashSet<LogicalVariable>();
-            VariableUtilities.getUsedVariables(op2, used);
 
-            //not movable if the operator is not an assign
+            //not movable if the operator is not an assign or subplan
             //Might be helpful in the future for other operations in the future
-            if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+            if (movableOperator(op2.getOperatorTag())) {
+                if (op2.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                    VariableUtilities.getUsedVariables(op2, used);
+                } else if (op2.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+                    // Nested plan must have an aggregate root.
+                    ListIterator<ILogicalPlan> subplansIter = ((SubplanOperator) op2).getNestedPlans().listIterator();
+                    ILogicalPlan plan = null;
+                    while (subplansIter.hasNext()) {
+                        plan = subplansIter.next();
+                    }
+                    if (plan == null) {
+                        return false;
+                    }
+                    if (plan.getRoots().size() != 1) {
+                        return false;
+                    }
+                    ILogicalOperator op3 = plan.getRoots().get(0).getValue();
+                    if (op3.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+                        return false;
+                    }
+                    // Used variables do not include ones created in the subplan.
+                    VariableUtilities.getUsedVariables(op2, used);
+                    Set<LogicalVariable> subplanProducedAndDown = new HashSet<LogicalVariable>();
+                    VariableUtilities.getProducedVariablesInDescendantsAndSelf(op3, subplanProducedAndDown);
+                    used.removeAll(subplanProducedAndDown);
+                } else {
+                    notApplicable = true;
+                }
+            } else {
                 notApplicable = true;
             }
 
@@ -126,4 +177,7 @@
         return false;
     }
 
-}
+    protected boolean movableOperator(LogicalOperatorTag operatorTag) {
+        return (operatorTag == LogicalOperatorTag.ASSIGN || operatorTag == LogicalOperatorTag.SUBPLAN);
+    }
+}
\ No newline at end of file

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/352
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I80789e3827b4224b92b0b9d7247919f862f6f939
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>

Change in hyracks[master]: Abstracting algebricks rules out of AsterixDB.

Posted by "Steven Jacobs (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Steven Jacobs has posted comments on this change.

Change subject: Abstracting algebricks rules out of AsterixDB.
......................................................................


Patch Set 2: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/352
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I80789e3827b4224b92b0b9d7247919f862f6f939
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Abstracting algebricks rules out of AsterixDB.

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Abstracting algebricks rules out of AsterixDB.
......................................................................


Patch Set 2: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/406/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/352
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I80789e3827b4224b92b0b9d7247919f862f6f939
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Abstracting algebricks rules out of AsterixDB.

Posted by "Preston Carman (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Preston Carman has submitted this change and it was merged.

Change subject: Abstracting algebricks rules out of AsterixDB.
......................................................................


Abstracting algebricks rules out of AsterixDB.

Change-Id: I80789e3827b4224b92b0b9d7247919f862f6f939
Reviewed-on: https://asterix-gerrit.ics.uci.edu/352
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Steven Jacobs <sj...@ucr.edu>
---
A algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
M algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
M algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/MoveFreeVariableOperatorOutOfSubplanRule.java
3 files changed, 264 insertions(+), 16 deletions(-)

Approvals:
  Steven Jacobs: Looks good to me, approved
  Jenkins: Verified



diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
new file mode 100644
index 0000000..8f8e62b
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
@@ -0,0 +1,155 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.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.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+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.FunctionIdentifier;
+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.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Factors out function expressions from each comparison function or similarity function in join condition by
+ * assigning them to a variables, and replacing the function expressions with references to those variables.
+ * Examples:
+ * Plan with function expressions in comparison or similarity condition of join expression.
+ * Generates one assign operator per extracted function expression.
+ *
+ * <pre>
+ * Before plan:
+ * 
+ *   join ( eq( funcX($$1), funcX($$2) ) )
+ * 
+ * After plan:
+ * 
+ *   join (eq($$3,$$4))
+ *   assign [$$4] <- [funcY($$2)]
+ *   assign [$$3] <- [funcX($$1)]
+ * </pre>
+ */
+public class ExtractFunctionsFromJoinConditionRule 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.INNERJOIN
+                && op.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+            return false;
+        }
+        AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) op;
+        ILogicalExpression expr = joinOp.getCondition().getValue();
+
+        return assignFunctionExpressions(joinOp, expr, context);
+
+    }
+
+    private boolean assignFunctionExpressions(AbstractLogicalOperator joinOp, ILogicalExpression expr,
+            IOptimizationContext context) throws AlgebricksException {
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) expr;
+        FunctionIdentifier fi = fexp.getFunctionIdentifier();
+
+        boolean modified = false;
+        if (fi.equals(AlgebricksBuiltinFunctions.AND) || fi.equals(AlgebricksBuiltinFunctions.OR)
+                || processArgumentsToFunction(fi)) {
+            for (Mutable<ILogicalExpression> a : fexp.getArguments()) {
+                if (assignFunctionExpressions(joinOp, a.getValue(), context)) {
+                    modified = true;
+                }
+            }
+            return modified;
+        } else if (AlgebricksBuiltinFunctions.isComparisonFunction(fi) || isComparisonFunction(fi)) {
+            for (Mutable<ILogicalExpression> exprRef : fexp.getArguments()) {
+                if (exprRef.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    LogicalVariable newVar = context.newVar();
+                    AssignOperator newAssign = new AssignOperator(newVar, new MutableObject<ILogicalExpression>(exprRef
+                            .getValue().cloneExpression()));
+                    newAssign.setExecutionMode(joinOp.getExecutionMode());
+
+                    // Place assign below joinOp.
+                    List<LogicalVariable> used = new ArrayList<LogicalVariable>();
+                    VariableUtilities.getUsedVariables(newAssign, used);
+
+                    Mutable<ILogicalOperator> leftBranchRef = joinOp.getInputs().get(0);
+                    ILogicalOperator leftBranch = leftBranchRef.getValue();
+                    List<LogicalVariable> leftBranchVariables = new ArrayList<LogicalVariable>();
+                    VariableUtilities.getLiveVariables(leftBranch, leftBranchVariables);
+                    if (leftBranchVariables.containsAll(used)) {
+                        // place assign on left branch
+                        newAssign.getInputs().add(new MutableObject<ILogicalOperator>(leftBranch));
+                        leftBranchRef.setValue(newAssign);
+                        modified = true;
+                    } else {
+                        Mutable<ILogicalOperator> rightBranchRef = joinOp.getInputs().get(1);
+                        ILogicalOperator rightBranch = rightBranchRef.getValue();
+                        List<LogicalVariable> rightBranchVariables = new ArrayList<LogicalVariable>();
+                        VariableUtilities.getLiveVariables(rightBranch, rightBranchVariables);
+                        if (rightBranchVariables.containsAll(used)) {
+                            // place assign on right branch
+                            newAssign.getInputs().add(new MutableObject<ILogicalOperator>(rightBranch));
+                            rightBranchRef.setValue(newAssign);
+                            modified = true;
+                        }
+                    }
+
+                    if (modified) {
+                        // Replace original expr with variable reference.
+                        exprRef.setValue(new VariableReferenceExpression(newVar));
+                        context.computeAndSetTypeEnvironmentForOperator(newAssign);
+                        context.computeAndSetTypeEnvironmentForOperator(joinOp);
+                    }
+                }
+            }
+            return modified;
+        } else {
+            return false;
+        }
+    }
+
+    protected boolean processArgumentsToFunction(FunctionIdentifier fi) {
+        return false;
+    }
+
+    protected boolean isComparisonFunction(FunctionIdentifier fi) {
+        return false;
+    }
+
+}
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
index ab7835d..a6d9ab0 100644
--- 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
@@ -3,9 +3,9 @@
  * 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.
@@ -60,6 +60,45 @@
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
 
+/**
+ * The rule searches for SUBPLAN operator with a optional PROJECT operator and
+ * an AGGREGATE followed by a join operator.
+ *
+ * <pre>
+ * Before
+ * 
+ *   plan__parent
+ *   SUBPLAN {
+ *     PROJECT?
+ *     AGGREGATE
+ *     plan__nested_A
+ *     INNER_JOIN | LEFT_OUTER_JOIN ($condition, $left, $right)
+ *       plan__nested_B
+ *   }
+ *   plan__child
+ * 
+ *   where $condition does not equal a constant true.
+ * 
+ * After (This is a general application of the rule, specifics may vary based on the query plan.)
+ * 
+ *   plan__parent
+ *   GROUP_BY {
+ *     PROJECT?
+ *     AGGREGATE
+ *     plan__nested_A
+ *     SELECT( algebricks:not( is_null( $right ) ) )
+ *     NESTED_TUPLE_SOURCE
+ *   }
+ *   SUBPLAN {
+ *     INNER_JOIN | LEFT_OUTER_JOIN ($condition, $left, $right)
+ *       plan__nested_B
+ *   }
+ *   plan__child
+ * </pre>
+ *
+ * @author prestonc
+ */
+
 public class IntroduceGroupByForSubplanRule implements IAlgebraicRewriteRule {
 
     @Override
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/MoveFreeVariableOperatorOutOfSubplanRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/MoveFreeVariableOperatorOutOfSubplanRule.java
index b01f320..5867abd 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/MoveFreeVariableOperatorOutOfSubplanRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/MoveFreeVariableOperatorOutOfSubplanRule.java
@@ -1,11 +1,11 @@
 /*
- * Copyright 2009-2014 by The Regents of the University of California
+ * Copyright 2009-2013 by The Regents of the University of California
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * you may obtain a copy of the License from
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -30,19 +30,43 @@
 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.rewriter.rules.AbstractDecorrelationRule;
 
+/**
+ * The rule searches for operators that can be moved outside the subplan.
+ *
+ * <pre>
+ * Before
+ * 
+ *   %PARENT_PLAN
+ *   SUBPLAN{
+ *     %NESTED_OPERATORS_B+
+ *     ASSIGN || %SUBPLAN
+ *     %NESTED_OPERATORS_A*
+ *     NESTEDTUPLESOURCE
+ *   }
+ *   %CHILD_PLAN
+ * 
+ *   where
+ *     %SUBPLAN has one nested plan with a root AGGREGATE operator.
+ * 
+ * After
+ * 
+ *   %PARENT_PLAN
+ *   SUBPLAN{
+ *     %NESTED_OPERATORS_B+
+ *     %NESTED_OPERATORS_A*
+ *     NESTEDTUPLESOURCE
+ *   }
+ *   ASSIGN || %SUBPLAN
+ *   %CHILD_PLAN
+ * </pre>
+ */
 public class MoveFreeVariableOperatorOutOfSubplanRule extends AbstractDecorrelationRule {
 
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
-        /*
-         * This rule looks for an assign within a subplan that uses only 
-         * variables from outside of the subplan
-         * 
-         * It moves this assign outside of the subplan
-         * 
-         */
         AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getValue();
         if (op0.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
             return false;
@@ -90,11 +114,38 @@
 
             //Get its used variables
             Set<LogicalVariable> used = new HashSet<LogicalVariable>();
-            VariableUtilities.getUsedVariables(op2, used);
 
-            //not movable if the operator is not an assign
+            //not movable if the operator is not an assign or subplan
             //Might be helpful in the future for other operations in the future
-            if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+            if (movableOperator(op2.getOperatorTag())) {
+                if (op2.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                    VariableUtilities.getUsedVariables(op2, used);
+                } else if (op2.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+                    // Nested plan must have an aggregate root.
+                    ListIterator<ILogicalPlan> subplansIter = ((SubplanOperator) op2).getNestedPlans().listIterator();
+                    ILogicalPlan plan = null;
+                    while (subplansIter.hasNext()) {
+                        plan = subplansIter.next();
+                    }
+                    if (plan == null) {
+                        return false;
+                    }
+                    if (plan.getRoots().size() != 1) {
+                        return false;
+                    }
+                    ILogicalOperator op3 = plan.getRoots().get(0).getValue();
+                    if (op3.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+                        return false;
+                    }
+                    // Used variables do not include ones created in the subplan.
+                    VariableUtilities.getUsedVariables(op2, used);
+                    Set<LogicalVariable> subplanProducedAndDown = new HashSet<LogicalVariable>();
+                    VariableUtilities.getProducedVariablesInDescendantsAndSelf(op3, subplanProducedAndDown);
+                    used.removeAll(subplanProducedAndDown);
+                } else {
+                    notApplicable = true;
+                }
+            } else {
                 notApplicable = true;
             }
 
@@ -126,4 +177,7 @@
         return false;
     }
 
-}
+    protected boolean movableOperator(LogicalOperatorTag operatorTag) {
+        return (operatorTag == LogicalOperatorTag.ASSIGN || operatorTag == LogicalOperatorTag.SUBPLAN);
+    }
+}
\ No newline at end of file

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/352
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I80789e3827b4224b92b0b9d7247919f862f6f939
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>

Change in hyracks[master]: Abstracting algebricks rules out of AsterixDB.

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Abstracting algebricks rules out of AsterixDB.
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/405/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/352
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I80789e3827b4224b92b0b9d7247919f862f6f939
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Abstracting algebricks rules out of AsterixDB.

Posted by "Steven Jacobs (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Steven Jacobs has posted comments on this change.

Change subject: Abstracting algebricks rules out of AsterixDB.
......................................................................


Patch Set 2: Code-Review+1

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/352
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I80789e3827b4224b92b0b9d7247919f862f6f939
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Abstracting algebricks rules out of AsterixDB.

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Abstracting algebricks rules out of AsterixDB.
......................................................................


Patch Set 1: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/405/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/352
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I80789e3827b4224b92b0b9d7247919f862f6f939
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Abstracting algebricks rules out of AsterixDB.

Posted by "Jenkins (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Jenkins has posted comments on this change.

Change subject: Abstracting algebricks rules out of AsterixDB.
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/406/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/352
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I80789e3827b4224b92b0b9d7247919f862f6f939
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Abstracting algebricks rules out of AsterixDB.

Posted by "Preston Carman (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/352

to look at the new patch set (#2).

Change subject: Abstracting algebricks rules out of AsterixDB.
......................................................................

Abstracting algebricks rules out of AsterixDB.

Change-Id: I80789e3827b4224b92b0b9d7247919f862f6f939
---
A algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
M algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
M algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/MoveFreeVariableOperatorOutOfSubplanRule.java
3 files changed, 264 insertions(+), 16 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/52/352/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/352
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I80789e3827b4224b92b0b9d7247919f862f6f939
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>