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:32:53 UTC

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

Preston Carman has uploaded a new change for review.

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

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

Abstracting algebricks rules out of AsterixDB.

Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
---
M asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
A asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
A asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java
D asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ExtractFunctionsFromJoinConditionRule.java
4 files changed, 62 insertions(+), 146 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/53/353/1

diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index 10ab856..a200874 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -19,8 +19,10 @@
 import java.util.List;
 
 import edu.uci.ics.asterix.optimizer.rules.AddEquivalenceClassForRecordConstructorRule;
+import edu.uci.ics.asterix.optimizer.rules.AsterixExtractFunctionsFromJoinConditionRule;
 import edu.uci.ics.asterix.optimizer.rules.AsterixInlineVariablesRule;
 import edu.uci.ics.asterix.optimizer.rules.AsterixIntroduceGroupByCombinerRule;
+import edu.uci.ics.asterix.optimizer.rules.AsterixMoveFreeVariableOperatorOutOfSubplanRule;
 import edu.uci.ics.asterix.optimizer.rules.ByNameToByIndexFieldAccessRule;
 import edu.uci.ics.asterix.optimizer.rules.CancelUnnestWithNestedListifyRule;
 import edu.uci.ics.asterix.optimizer.rules.CheckFilterExpressionTypeRule;
@@ -28,7 +30,6 @@
 import edu.uci.ics.asterix.optimizer.rules.CountVarToCountOneRule;
 import edu.uci.ics.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
 import edu.uci.ics.asterix.optimizer.rules.ExtractDistinctByExpressionsRule;
-import edu.uci.ics.asterix.optimizer.rules.ExtractFunctionsFromJoinConditionRule;
 import edu.uci.ics.asterix.optimizer.rules.ExtractOrderExpressionsRule;
 import edu.uci.ics.asterix.optimizer.rules.FeedScanCollectionToUnnest;
 import edu.uci.ics.asterix.optimizer.rules.FuzzyEqRule;
@@ -96,7 +97,6 @@
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceGroupByForSubplanRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceProjectsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IsolateHyracksOperatorsRule;
-import edu.uci.ics.hyracks.algebricks.rewriter.rules.MoveFreeVariableOperatorOutOfSubplanRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.NestedSubplanToJoinRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PullSelectOutOfEqJoin;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushAssignBelowUnionAllRule;
@@ -150,7 +150,7 @@
         normalization.add(new ExtractGbyExpressionsRule());
         normalization.add(new ExtractDistinctByExpressionsRule());
         normalization.add(new ExtractOrderExpressionsRule());
-        normalization.add(new MoveFreeVariableOperatorOutOfSubplanRule());
+        normalization.add(new AsterixMoveFreeVariableOperatorOutOfSubplanRule());
 
         // IntroduceStaticTypeCastRule should go before
         // IntroduceDynamicTypeCastRule to
@@ -186,7 +186,7 @@
         condPushDownAndJoinInference.add(new IntroduceGroupByForSubplanRule());
         condPushDownAndJoinInference.add(new SubplanOutOfGroupRule());
         condPushDownAndJoinInference.add(new InsertOuterJoinRule());
-        condPushDownAndJoinInference.add(new ExtractFunctionsFromJoinConditionRule());
+        condPushDownAndJoinInference.add(new AsterixExtractFunctionsFromJoinConditionRule());
 
         condPushDownAndJoinInference.add(new RemoveRedundantVariablesRule());
         condPushDownAndJoinInference.add(new AsterixInlineVariablesRule());
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
new file mode 100644
index 0000000..5457e55
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.optimizer.rules;
+
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractFunctionsFromJoinConditionRule;
+
+public class AsterixExtractFunctionsFromJoinConditionRule extends ExtractFunctionsFromJoinConditionRule {
+
+    @Override
+    protected boolean processArgumentsToFunction(FunctionIdentifier fi) {
+        return fi.equals(AsterixBuiltinFunctions.GET_ITEM);
+    }
+
+    @Override
+    protected boolean isComparisonFunction(FunctionIdentifier fi) {
+        return AsterixBuiltinFunctions.isSimilarityFunction(fi);
+    }
+
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java
new file mode 100644
index 0000000..a0035f5
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.optimizer.rules;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.MoveFreeVariableOperatorOutOfSubplanRule;
+
+public class AsterixMoveFreeVariableOperatorOutOfSubplanRule extends MoveFreeVariableOperatorOutOfSubplanRule {
+
+    protected boolean movableOperator(LogicalOperatorTag operatorTag) {
+        return (operatorTag == LogicalOperatorTag.ASSIGN);
+    }
+}
\ No newline at end of file
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ExtractFunctionsFromJoinConditionRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ExtractFunctionsFromJoinConditionRule.java
deleted file mode 100644
index 37066f1..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ExtractFunctionsFromJoinConditionRule.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.optimizer.rules;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.commons.lang3.mutable.MutableObject;
-
-import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
-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.
- * Example
- * Before plan:
- * join ( eq( funcX($$1), funcX($$2) ) )
- * After plan:
- * join (eq($$3,$$4))
- * assign [$$4] <- [funcY($$2)]
- * assign [$$3] <- [funcX($$1)]
- */
-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)
-                || fi.equals(AsterixBuiltinFunctions.GET_ITEM)) {
-            for (Mutable<ILogicalExpression> a : fexp.getArguments()) {
-                if (assignFunctionExpressions(joinOp, a.getValue(), context)) {
-                    modified = true;
-                }
-            }
-            return modified;
-        } else if (AlgebricksBuiltinFunctions.isComparisonFunction(fi)
-                || AsterixBuiltinFunctions.isSimilarityFunction(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;
-        }
-    }
-
-}

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

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

Change in asterixdb[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 Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/197/ : FAILURE

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

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

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

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

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


Patch Set 4:

Is there a way todo this through the web interface for gerrit?

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
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>
Gerrit-HasComments: No

Change in asterixdb[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/asterix-gerrit-topic/201/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
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 asterixdb[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: Id184cc1cdfa738964705007ac42027d90b84cf58
Reviewed-on: https://asterix-gerrit.ics.uci.edu/353
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Preston Carman <pr...@apache.org>
---
M asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
A asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
A asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java
D asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ExtractFunctionsFromJoinConditionRule.java
4 files changed, 63 insertions(+), 146 deletions(-)

Approvals:
  Preston Carman: Looks good to me, approved
  Jenkins: Verified



diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index 10ab856..a200874 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -19,8 +19,10 @@
 import java.util.List;
 
 import edu.uci.ics.asterix.optimizer.rules.AddEquivalenceClassForRecordConstructorRule;
+import edu.uci.ics.asterix.optimizer.rules.AsterixExtractFunctionsFromJoinConditionRule;
 import edu.uci.ics.asterix.optimizer.rules.AsterixInlineVariablesRule;
 import edu.uci.ics.asterix.optimizer.rules.AsterixIntroduceGroupByCombinerRule;
+import edu.uci.ics.asterix.optimizer.rules.AsterixMoveFreeVariableOperatorOutOfSubplanRule;
 import edu.uci.ics.asterix.optimizer.rules.ByNameToByIndexFieldAccessRule;
 import edu.uci.ics.asterix.optimizer.rules.CancelUnnestWithNestedListifyRule;
 import edu.uci.ics.asterix.optimizer.rules.CheckFilterExpressionTypeRule;
@@ -28,7 +30,6 @@
 import edu.uci.ics.asterix.optimizer.rules.CountVarToCountOneRule;
 import edu.uci.ics.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
 import edu.uci.ics.asterix.optimizer.rules.ExtractDistinctByExpressionsRule;
-import edu.uci.ics.asterix.optimizer.rules.ExtractFunctionsFromJoinConditionRule;
 import edu.uci.ics.asterix.optimizer.rules.ExtractOrderExpressionsRule;
 import edu.uci.ics.asterix.optimizer.rules.FeedScanCollectionToUnnest;
 import edu.uci.ics.asterix.optimizer.rules.FuzzyEqRule;
@@ -96,7 +97,6 @@
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceGroupByForSubplanRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceProjectsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IsolateHyracksOperatorsRule;
-import edu.uci.ics.hyracks.algebricks.rewriter.rules.MoveFreeVariableOperatorOutOfSubplanRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.NestedSubplanToJoinRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PullSelectOutOfEqJoin;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushAssignBelowUnionAllRule;
@@ -150,7 +150,7 @@
         normalization.add(new ExtractGbyExpressionsRule());
         normalization.add(new ExtractDistinctByExpressionsRule());
         normalization.add(new ExtractOrderExpressionsRule());
-        normalization.add(new MoveFreeVariableOperatorOutOfSubplanRule());
+        normalization.add(new AsterixMoveFreeVariableOperatorOutOfSubplanRule());
 
         // IntroduceStaticTypeCastRule should go before
         // IntroduceDynamicTypeCastRule to
@@ -186,7 +186,7 @@
         condPushDownAndJoinInference.add(new IntroduceGroupByForSubplanRule());
         condPushDownAndJoinInference.add(new SubplanOutOfGroupRule());
         condPushDownAndJoinInference.add(new InsertOuterJoinRule());
-        condPushDownAndJoinInference.add(new ExtractFunctionsFromJoinConditionRule());
+        condPushDownAndJoinInference.add(new AsterixExtractFunctionsFromJoinConditionRule());
 
         condPushDownAndJoinInference.add(new RemoveRedundantVariablesRule());
         condPushDownAndJoinInference.add(new AsterixInlineVariablesRule());
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
new file mode 100644
index 0000000..5457e55
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.optimizer.rules;
+
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractFunctionsFromJoinConditionRule;
+
+public class AsterixExtractFunctionsFromJoinConditionRule extends ExtractFunctionsFromJoinConditionRule {
+
+    @Override
+    protected boolean processArgumentsToFunction(FunctionIdentifier fi) {
+        return fi.equals(AsterixBuiltinFunctions.GET_ITEM);
+    }
+
+    @Override
+    protected boolean isComparisonFunction(FunctionIdentifier fi) {
+        return AsterixBuiltinFunctions.isSimilarityFunction(fi);
+    }
+
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java
new file mode 100644
index 0000000..c42fa15
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.optimizer.rules;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.MoveFreeVariableOperatorOutOfSubplanRule;
+
+public class AsterixMoveFreeVariableOperatorOutOfSubplanRule extends MoveFreeVariableOperatorOutOfSubplanRule {
+
+    @Override
+    protected boolean movableOperator(LogicalOperatorTag operatorTag) {
+        return (operatorTag == LogicalOperatorTag.ASSIGN);
+    }
+}
\ No newline at end of file
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ExtractFunctionsFromJoinConditionRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ExtractFunctionsFromJoinConditionRule.java
deleted file mode 100644
index 37066f1..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ExtractFunctionsFromJoinConditionRule.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.optimizer.rules;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.commons.lang3.mutable.MutableObject;
-
-import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
-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.
- * Example
- * Before plan:
- * join ( eq( funcX($$1), funcX($$2) ) )
- * After plan:
- * join (eq($$3,$$4))
- * assign [$$4] <- [funcY($$2)]
- * assign [$$3] <- [funcX($$1)]
- */
-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)
-                || fi.equals(AsterixBuiltinFunctions.GET_ITEM)) {
-            for (Mutable<ILogicalExpression> a : fexp.getArguments()) {
-                if (assignFunctionExpressions(joinOp, a.getValue(), context)) {
-                    modified = true;
-                }
-            }
-            return modified;
-        } else if (AlgebricksBuiltinFunctions.isComparisonFunction(fi)
-                || AsterixBuiltinFunctions.isSimilarityFunction(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;
-        }
-    }
-
-}

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

Gerrit-MessageType: merged
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
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 asterixdb[master]: Abstracting algebricks rules out of AsterixDB.

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

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


Patch Set 4:

Yeah, usually there's a "Rebase Change" button, if it's not based on current master. It works as long as the rebase is trivial (i.e. no files are changed between the candidate commit and the new commits included by the rebase)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
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>
Gerrit-HasComments: No

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

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

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


Patch Set 4:

The 'Rebase' button didn't work?

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
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>
Gerrit-HasComments: No

Change in asterixdb[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/asterix-gerrit-topic/200/ : SUCCESS

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

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

Change in asterixdb[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/353
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
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 asterixdb[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 1: Code-Review+1

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-HasComments: No

Change in asterixdb[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

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/200/

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

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

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

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

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

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

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

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

Abstracting algebricks rules out of AsterixDB.

Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
---
M asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
A asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
A asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java
D asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ExtractFunctionsFromJoinConditionRule.java
4 files changed, 63 insertions(+), 146 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/53/353/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/353
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
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>

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

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

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


Patch Set 1:

(1 comment)

One question :)

https://asterix-gerrit.ics.uci.edu/#/c/353/1/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java
File asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java:

Line 21: 
Why no @Override annotation?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
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: Yes

Change in asterixdb[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 Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/198/ : FAILURE

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

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

Change in asterixdb[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/asterix-gerrit-topic/201/ : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
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 asterixdb[master]: Abstracting algebricks rules out of AsterixDB.

Posted by "Ian Maxon (Code Review)" <do...@asterix-gerrit.ics.uci.edu>.
Ian Maxon 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/353
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
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 asterixdb[master]: Abstracting algebricks rules out of AsterixDB.

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

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

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

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

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

Abstracting algebricks rules out of AsterixDB.

Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
---
M asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
A asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
A asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java
D asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ExtractFunctionsFromJoinConditionRule.java
4 files changed, 63 insertions(+), 146 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/53/353/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/353
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
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>

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

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

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


Patch Set 3: Code-Review+2

Only merged changes from master. Keeping previous +2.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
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>
Gerrit-HasComments: No

Change in asterixdb[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/asterix-gerrit-notopic/197/

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

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

Change in asterixdb[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 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/206/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
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 asterixdb[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 3: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/206/ : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Id184cc1cdfa738964705007ac42027d90b84cf58
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
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 asterixdb[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

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/198/

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

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