You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/06/06 04:13:38 UTC

[GitHub] [incubator-doris] morrySnow commented on a diff in pull request #9942: [Enhancement] (Nereids) scalar expression rewrite framework

morrySnow commented on code in PR #9942:
URL: https://github.com/apache/incubator-doris/pull/9942#discussion_r889790374


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Literal.java:
##########
@@ -91,11 +97,41 @@ public boolean nullable() throws UnboundException {
         return value == null;
     }
 
+    @Override
+    public boolean isVariable() {
+        // for test

Review Comment:
   test code?



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java:
##########
@@ -61,4 +66,13 @@ public Expression child(int index) {
     public EXPR_TYPE newChildren(List<TreeNode> children) {
         throw new RuntimeException();
     }
+
+    public boolean isVariable() {

Review Comment:
   please add comment to this function



##########
fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4:
##########
@@ -147,6 +147,7 @@ booleanExpression
 valueExpression
     : primaryExpression                                                                      #valueExpressionDefault
     | left=valueExpression comparisonOperator right=valueExpression                          #comparison
+    | NOT valueExpression                                                                    #not

Review Comment:
   not should be placed under booleanExpression as 'NOT booleanExpression'



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionRewriter.java:
##########
@@ -0,0 +1,73 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.rules.expression.rewrite;
+
+import com.google.common.collect.Lists;
+import org.apache.doris.nereids.rules.expression.rewrite.rules.NormalizeExpressionRule;
+import org.apache.doris.nereids.rules.expression.rewrite.rules.NotExpressionRule;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.LeafExpression;
+
+import java.util.List;
+
+public class ExpressionRewriter {
+
+    public static final List<ExpressionRewriteRule> REWRITE_RULES = Lists.newArrayList(
+        new NotExpressionRule(),
+        new NormalizeExpressionRule()
+    );
+
+    private final ExpressionRewriteContext ctx;
+    private final List<ExpressionRewriteRule> rules;
+
+    public ExpressionRewriter(List<ExpressionRewriteRule> rules) {
+        this.rules = rules;
+        this.ctx = new ExpressionRewriteContext();
+    }
+
+    public ExpressionRewriter(ExpressionRewriteRule rule) {
+        this.rules = Lists.newArrayList(rule);
+        this.ctx = new ExpressionRewriteContext();
+    }
+
+
+    public Expression rewrite(Expression root) {
+        Expression result = root;
+        for (ExpressionRewriteRule rule : rules) {
+            result = applyRule(result, rule);
+        }
+        return result;
+    }
+
+    private Expression applyRule(Expression expr, ExpressionRewriteRule rule) {
+        Expression rewrittenExpr = expr;
+        rewrittenExpr = applyRuleBottomUp(rewrittenExpr, rule);
+        return rewrittenExpr;
+    }
+
+    private Expression applyRuleBottomUp(Expression expr, ExpressionRewriteRule rule) {
+        List<Expression> children = Lists.newArrayList();

Review Comment:
   add a todo: apply function could return a flag  indicating whether expression has changed. We could not replace children if no child has changed.



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/rules/NotExpressionRule.java:
##########
@@ -0,0 +1,74 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.rules.expression.rewrite.rules;
+
+import org.apache.doris.nereids.rules.expression.rewrite.AbstractExpressionRewriteRule;
+import org.apache.doris.nereids.rules.expression.rewrite.ExpressionRewriteContext;
+import org.apache.doris.nereids.trees.NodeType;
+import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.GreaterThan;
+import org.apache.doris.nereids.trees.expressions.GreaterThanEqual;
+import org.apache.doris.nereids.trees.expressions.LessThan;
+import org.apache.doris.nereids.trees.expressions.LessThanEqual;
+import org.apache.doris.nereids.trees.expressions.Literal;
+import org.apache.doris.nereids.trees.expressions.Not;
+
+public class NotExpressionRule extends AbstractExpressionRewriteRule {
+
+    public static NotExpressionRule INSTANCE = new NotExpressionRule();
+
+
+    @Override
+    public Expression visitNotExpression(Not expr, ExpressionRewriteContext context) {
+
+        Expression child = expr.child();
+
+        if (child instanceof ComparisonPredicate) {
+            ComparisonPredicate cp = (ComparisonPredicate) expr.child();
+            Expression left =  cp.left();
+            Expression right = cp.right();
+            NodeType type = cp.getType();
+            switch (type) {
+                case GREATER_THAN:
+                    return new LessThanEqual(left, right);
+                case GREATER_THAN_EQUAL:
+                    return new LessThan(left, right);
+                case LESS_THAN:
+                    return new GreaterThanEqual(left, right);
+                case LESS_THAN_EQUAL:
+                    return new GreaterThan(left, right);
+                default:
+                    return expr;
+            }
+        }
+
+        if (child instanceof Not) {
+            Not son = (Not) child;
+            return son.child();
+        }
+
+        return expr;
+    }
+
+    @Override
+    public Expression visitLiteral(Literal expr, ExpressionRewriteContext context) {

Review Comment:
   useless code



##########
fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4:
##########
@@ -173,6 +174,10 @@ comparisonOperator
     : EQ | NEQ | LT | LTE | GT | GTE | NSEQ
     ;
 
+logicalOperator

Review Comment:
   not use anymore?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org