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/07/07 06:30:50 UTC

[GitHub] [doris] qzsee opened a new pull request, #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

qzsee opened a new pull request, #10667:
URL: https://github.com/apache/doris/pull/10667

   # Proposed changes
   
   Issue Number: close #xxx
   
   ## Problem Summary:
   
   Describe the overview of changes.
   
   ## Checklist(Required)
   
   1. Does it affect the original behavior: (Yes/No/I Don't know)
   2. Has unit tests been added: (Yes/No/No Need)
   3. Has document been added or modified: (Yes/No/No Need)
   4. Does it need to update dependencies: (Yes/No)
   5. Are there any changes that cannot be rolled back: (Yes/No)
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at [dev@doris.apache.org](mailto:dev@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc...
   


-- 
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


[GitHub] [doris] englefly commented on pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
englefly commented on PR #10667:
URL: https://github.com/apache/doris/pull/10667#issuecomment-1191015808

   about the  "extract common expr"
   (a or b) and (a or c) -> a or (b and c)
   
   this rule may affect predicate push down. consider a and b are from left child, and c  is from right child. We cannot push down a or (b and c), however, we could push (a or b) to left child.


-- 
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


[GitHub] [doris] qzsee commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
qzsee commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r924087386


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/rules/ExtractCommonFactorRule.java:
##########
@@ -0,0 +1,76 @@
+// 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.expressions.CompoundPredicate;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.util.ExpressionUtils;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Extract common expr for `CompoundPredicate`.
+ * for example:
+ * transform (a or b) and (a or c) to a or (b and c)
+ * transform (a and b) or (a and c) to a and (b or c)
+ */
+public class ExtractCommonFactorRule extends AbstractExpressionRewriteRule {
+
+    public static final ExtractCommonFactorRule INSTANCE = new ExtractCommonFactorRule();
+
+    @Override
+    public Expression visitCompoundPredicate(CompoundPredicate expr, ExpressionRewriteContext context) {
+
+        Expression rewrittenChildren = ExpressionUtils.combine(expr.getType(), ExpressionUtils.extract(expr).stream()
+                .map(predicate -> rewrite(predicate, context)).collect(Collectors.toList()));
+
+        if (!(rewrittenChildren instanceof CompoundPredicate)) {
+            return rewrittenChildren;
+        }
+
+        CompoundPredicate compoundPredicate = (CompoundPredicate) rewrittenChildren;
+
+        List<List<Expression>> partitions = ExpressionUtils.extract(compoundPredicate).stream()
+                .map(predicate -> predicate instanceof CompoundPredicate ? ExpressionUtils.extract(
+                        (CompoundPredicate) predicate) : Lists.newArrayList(predicate)).collect(Collectors.toList());

Review Comment:
   nothing



-- 
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


[GitHub] [doris] 924060929 commented on pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
924060929 commented on PR #10667:
URL: https://github.com/apache/doris/pull/10667#issuecomment-1191026785

   
   
   
   > about the "extract common expr" (a or b) and (a or c) -> a or (b and c)
   > 
   > this rule may affect predicate push down. consider a and b are from left child, and c is from right child. We cannot push down a or (b and c), however, we could push (a or b) to left child.
   
   fix later


-- 
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


[GitHub] [doris] 924060929 commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
924060929 commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r925257955


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionOfPlanRewrite.java:
##########
@@ -0,0 +1,117 @@
+// 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 org.apache.doris.nereids.rules.Rule;
+import org.apache.doris.nereids.rules.RuleType;
+import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory;
+import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * expression of plan rewrite rule.
+ */
+public class ExpressionOfPlanRewrite implements RewriteRuleFactory {
+    private final ExpressionRuleExecutor rewriter;
+
+    public ExpressionOfPlanRewrite(ExpressionRuleExecutor rewriter) {
+        this.rewriter = Objects.requireNonNull(rewriter, "rewriter is null");
+    }
+
+    @Override
+    public List<Rule<Plan>> buildRules() {
+        return ImmutableList.of(
+                new ProjectExpressionRewrite().build(),
+                new AggExpressionRewrite().build(),
+                new FilterExpressionRewrite().build(),
+                new JoinExpressionRewrite().build());
+    }
+
+
+    private class ProjectExpressionRewrite extends OneRewriteRuleFactory {
+        @Override
+        public Rule<Plan> build() {
+            return logicalProject().then(project -> {
+                List<NamedExpression> origin = project.getProjects();
+                List<NamedExpression> newProjects = origin.stream()
+                        .map(expr -> (NamedExpression) rewriter.rewrite(expr)).collect(Collectors.toList());
+                return new LogicalProject<>(newProjects, project.child());
+            }).toRule(RuleType.REWRITE_PROJECT_EXPRESSION);
+        }
+    }
+
+    private class FilterExpressionRewrite extends OneRewriteRuleFactory {
+        @Override
+        public Rule<Plan> build() {
+            return logicalFilter().then(filter -> {
+                Expression newExpr = rewriter.rewrite(filter.getPredicates());
+                if (newExpr.equals(filter.getPredicates())) {
+                    return filter;
+                }
+                return new LogicalFilter<>(newExpr, filter.child());
+            }).toRule(RuleType.REWRITE_FILTER_EXPRESSION);
+        }
+    }
+
+    private class AggExpressionRewrite extends OneRewriteRuleFactory {
+        @Override
+        public Rule<Plan> build() {
+            return logicalAggregate().then(agg -> {
+                List<Expression> groupByExprs = agg.getGroupByExpressionList();
+                List<Expression> newGroupByExprs = rewriter.rewrite(groupByExprs);
+
+                List<NamedExpression> outputExpressions = agg.getOutputExpressionList();
+                List<NamedExpression> newOutputExpressions = outputExpressions.stream()

Review Comment:
   need check groupByExprs and newGroupByExprs not change and return origin aggregate?



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionOfPlanRewrite.java:
##########
@@ -0,0 +1,117 @@
+// 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 org.apache.doris.nereids.rules.Rule;
+import org.apache.doris.nereids.rules.RuleType;
+import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory;
+import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * expression of plan rewrite rule.
+ */
+public class ExpressionOfPlanRewrite implements RewriteRuleFactory {
+    private final ExpressionRuleExecutor rewriter;
+
+    public ExpressionOfPlanRewrite(ExpressionRuleExecutor rewriter) {
+        this.rewriter = Objects.requireNonNull(rewriter, "rewriter is null");
+    }
+
+    @Override
+    public List<Rule<Plan>> buildRules() {
+        return ImmutableList.of(
+                new ProjectExpressionRewrite().build(),
+                new AggExpressionRewrite().build(),
+                new FilterExpressionRewrite().build(),
+                new JoinExpressionRewrite().build());
+    }
+
+
+    private class ProjectExpressionRewrite extends OneRewriteRuleFactory {
+        @Override
+        public Rule<Plan> build() {
+            return logicalProject().then(project -> {
+                List<NamedExpression> origin = project.getProjects();
+                List<NamedExpression> newProjects = origin.stream()
+                        .map(expr -> (NamedExpression) rewriter.rewrite(expr)).collect(Collectors.toList());
+                return new LogicalProject<>(newProjects, project.child());

Review Comment:
   need check expression not change and return origin project?



-- 
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


[GitHub] [doris] qzsee commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
qzsee commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r925279448


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java:
##########
@@ -102,8 +104,13 @@ public static Expression combine(ExpressionType op, List<Expression> expressions
             }
         }
 
-        Optional<Expression> result =
-                distinctExpressions.stream().reduce((left, right) -> new CompoundPredicate(op, left, right));
+        Optional<Expression> result = distinctExpressions.stream().reduce((left, right) -> {
+            if (op == ExpressionType.AND) {
+                return new And(left, right);
+            } else {
+                return new Or(left, right);
+            }
+        });

Review Comment:
   logical and uniformly use And
   logical or uniformly use Or
   



-- 
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


[GitHub] [doris] adonis0147 commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
adonis0147 commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r925269681


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java:
##########
@@ -102,8 +104,13 @@ public static Expression combine(ExpressionType op, List<Expression> expressions
             }
         }
 
-        Optional<Expression> result =
-                distinctExpressions.stream().reduce((left, right) -> new CompoundPredicate(op, left, right));
+        Optional<Expression> result = distinctExpressions.stream().reduce((left, right) -> {
+            if (op == ExpressionType.AND) {
+                return new And(left, right);
+            } else {
+                return new Or(left, right);
+            }
+        });

Review Comment:
   Due to the previous check `Preconditions.checkArgument(op == ExpressionType.AND || op == ExpressionType.OR)`, here we can safely use `new CompoundPredicate(op, left, right))` to simplify the code.



-- 
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


[GitHub] [doris] morrySnow commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
morrySnow commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r923302421


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionOfPlanRewrite.java:
##########
@@ -0,0 +1,136 @@
+// 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 org.apache.doris.nereids.rules.PlanRuleFactory;
+import org.apache.doris.nereids.rules.Rule;
+import org.apache.doris.nereids.rules.RulePromise;
+import org.apache.doris.nereids.rules.RuleType;
+import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory;
+import org.apache.doris.nereids.trees.expressions.Alias;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * expression of plan rewrite rule.
+ */
+public class ExpressionOfPlanRewrite implements PlanRuleFactory {

Review Comment:
   ```suggestion
   public class ExpressionOfPlanRewrite implements RewriteRuleFactory {
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionOfPlanRewrite.java:
##########
@@ -0,0 +1,136 @@
+// 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 org.apache.doris.nereids.rules.PlanRuleFactory;
+import org.apache.doris.nereids.rules.Rule;
+import org.apache.doris.nereids.rules.RulePromise;
+import org.apache.doris.nereids.rules.RuleType;
+import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory;
+import org.apache.doris.nereids.trees.expressions.Alias;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * expression of plan rewrite rule.
+ */
+public class ExpressionOfPlanRewrite implements PlanRuleFactory {
+    ExpressionRuleExecutor rewriter;
+
+    public ExpressionOfPlanRewrite(ExpressionRuleExecutor rewriter) {
+        this.rewriter = Objects.requireNonNull(rewriter, "rewriter is null");
+    }
+
+    @Override
+    public List<Rule<Plan>> buildRules() {
+        return ImmutableList.of(new ProjectExpressionRewrite().build(), new AggExpressionRewrite().build(),
+                new FilterExpressionRewrite().build(), new JoinExpressionRewrite().build());
+    }
+
+    @Override
+    public RulePromise defaultPromise() {
+        return RulePromise.REWRITE;
+    }
+
+    private class ProjectExpressionRewrite extends OneRewriteRuleFactory {
+        @Override
+        public Rule<Plan> build() {
+            return logicalProject().then(project -> {
+                List<NamedExpression> origin = project.getProjects();
+                for (int i = 0; i < origin.size(); i++) {
+                    if (origin.get(i) instanceof Alias) {
+                        Alias alias = (Alias) origin.get(i);
+                        Expression newExpr = rewriter.rewrite((alias).child());
+                        origin.set(i, (NamedExpression) alias.withChildren(Lists.newArrayList(newExpr)));
+                    }
+                }
+                return new LogicalProject<>(origin, project.child());
+            }).toRule(RuleType.REWRITE_PROJECT_EXPRESSION);
+        }
+    }
+
+    private class FilterExpressionRewrite extends OneRewriteRuleFactory {
+        @Override
+        public Rule<Plan> build() {
+            return logicalFilter().then(filter -> {
+                Expression newExpr = rewriter.rewrite(filter.getPredicates());
+                if (newExpr.equals(filter.getPredicates())) {
+                    return filter;
+                }
+                return new LogicalFilter<>(newExpr, filter.child());
+            }).toRule(RuleType.REWRITE_FILTER_EXPRESSION);
+        }
+    }
+
+    private class AggExpressionRewrite extends OneRewriteRuleFactory {
+        @Override
+        public Rule<Plan> build() {
+            return logicalAggregate().then(agg -> {
+                List<Expression> groupByExprs = agg.getGroupByExpressionList();
+                List<Expression> newGroupByExprs = rewriter.rewrite(groupByExprs);
+
+                List<NamedExpression> outputExpressions = agg.getOutputExpressionList();
+                List<NamedExpression> newOutputExpressions = Lists.newArrayList();
+                for (NamedExpression expr : outputExpressions) {
+                    if (expr instanceof Alias) {
+                        Alias alias = (Alias) expr;

Review Comment:
   we don't need to do cast explicitly.
   BTW, why we could not just rewrite all expressions?



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionOfPlanRewrite.java:
##########
@@ -0,0 +1,136 @@
+// 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 org.apache.doris.nereids.rules.PlanRuleFactory;
+import org.apache.doris.nereids.rules.Rule;
+import org.apache.doris.nereids.rules.RulePromise;
+import org.apache.doris.nereids.rules.RuleType;
+import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory;
+import org.apache.doris.nereids.trees.expressions.Alias;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * expression of plan rewrite rule.
+ */
+public class ExpressionOfPlanRewrite implements PlanRuleFactory {
+    ExpressionRuleExecutor rewriter;
+
+    public ExpressionOfPlanRewrite(ExpressionRuleExecutor rewriter) {
+        this.rewriter = Objects.requireNonNull(rewriter, "rewriter is null");
+    }
+
+    @Override
+    public List<Rule<Plan>> buildRules() {
+        return ImmutableList.of(new ProjectExpressionRewrite().build(), new AggExpressionRewrite().build(),
+                new FilterExpressionRewrite().build(), new JoinExpressionRewrite().build());
+    }
+
+    @Override
+    public RulePromise defaultPromise() {
+        return RulePromise.REWRITE;
+    }
+
+    private class ProjectExpressionRewrite extends OneRewriteRuleFactory {
+        @Override
+        public Rule<Plan> build() {
+            return logicalProject().then(project -> {
+                List<NamedExpression> origin = project.getProjects();
+                for (int i = 0; i < origin.size(); i++) {
+                    if (origin.get(i) instanceof Alias) {
+                        Alias alias = (Alias) origin.get(i);
+                        Expression newExpr = rewriter.rewrite((alias).child());
+                        origin.set(i, (NamedExpression) alias.withChildren(Lists.newArrayList(newExpr)));

Review Comment:
   origin List is ImmutableList we could not do In place modification. we need to do like
   ```java
   origin.stream().map(e -> {...}).collect(...)
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionOfPlanRewrite.java:
##########
@@ -0,0 +1,136 @@
+// 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 org.apache.doris.nereids.rules.PlanRuleFactory;
+import org.apache.doris.nereids.rules.Rule;
+import org.apache.doris.nereids.rules.RulePromise;
+import org.apache.doris.nereids.rules.RuleType;
+import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory;
+import org.apache.doris.nereids.trees.expressions.Alias;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * expression of plan rewrite rule.
+ */
+public class ExpressionOfPlanRewrite implements PlanRuleFactory {
+    ExpressionRuleExecutor rewriter;
+
+    public ExpressionOfPlanRewrite(ExpressionRuleExecutor rewriter) {
+        this.rewriter = Objects.requireNonNull(rewriter, "rewriter is null");
+    }
+
+    @Override
+    public List<Rule<Plan>> buildRules() {
+        return ImmutableList.of(new ProjectExpressionRewrite().build(), new AggExpressionRewrite().build(),
+                new FilterExpressionRewrite().build(), new JoinExpressionRewrite().build());
+    }
+
+    @Override
+    public RulePromise defaultPromise() {
+        return RulePromise.REWRITE;
+    }
+

Review Comment:
   if we implement RewriteRuleFactory, then we don't need these



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/OptimizeExpressionOfPlan.java:
##########
@@ -0,0 +1,39 @@
+// 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 org.apache.doris.nereids.rules.expression.rewrite.rules.DistinctPredicatesRule;
+import org.apache.doris.nereids.rules.expression.rewrite.rules.ExtractCommonFactorRule;
+import org.apache.doris.nereids.rules.expression.rewrite.rules.SimplifyNotExprRule;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * optimize expression of plan rule set.
+ */
+public class OptimizeExpressionOfPlan extends ExpressionOfPlanRewrite {
+    public static final List<ExpressionRewriteRule> NORMALIZE_REWRITE_RULES = ImmutableList.of(

Review Comment:
   ```suggestion
       public static final List<ExpressionRewriteRule> OPTIMIZE_REWRITE_RULES = ImmutableList.of(
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionOfPlanRewrite.java:
##########
@@ -0,0 +1,136 @@
+// 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 org.apache.doris.nereids.rules.PlanRuleFactory;
+import org.apache.doris.nereids.rules.Rule;
+import org.apache.doris.nereids.rules.RulePromise;
+import org.apache.doris.nereids.rules.RuleType;
+import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory;
+import org.apache.doris.nereids.trees.expressions.Alias;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * expression of plan rewrite rule.
+ */
+public class ExpressionOfPlanRewrite implements PlanRuleFactory {
+    ExpressionRuleExecutor rewriter;

Review Comment:
   private?



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/OptimizeExpressionOfPlan.java:
##########
@@ -0,0 +1,39 @@
+// 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 org.apache.doris.nereids.rules.expression.rewrite.rules.DistinctPredicatesRule;
+import org.apache.doris.nereids.rules.expression.rewrite.rules.ExtractCommonFactorRule;
+import org.apache.doris.nereids.rules.expression.rewrite.rules.SimplifyNotExprRule;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * optimize expression of plan rule set.
+ */
+public class OptimizeExpressionOfPlan extends ExpressionOfPlanRewrite {
+    public static final List<ExpressionRewriteRule> NORMALIZE_REWRITE_RULES = ImmutableList.of(
+            SimplifyNotExprRule.INSTANCE, ExtractCommonFactorRule.INSTANCE, DistinctPredicatesRule.INSTANCE);

Review Comment:
   one rule in one line is better for reading



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/rules/ExtractCommonFactorRule.java:
##########
@@ -0,0 +1,76 @@
+// 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.expressions.CompoundPredicate;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.util.ExpressionUtils;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Extract common expr for `CompoundPredicate`.
+ * for example:
+ * transform (a or b) and (a or c) to a or (b and c)
+ * transform (a and b) or (a and c) to a and (b or c)
+ */
+public class ExtractCommonFactorRule extends AbstractExpressionRewriteRule {
+
+    public static final ExtractCommonFactorRule INSTANCE = new ExtractCommonFactorRule();
+
+    @Override
+    public Expression visitCompoundPredicate(CompoundPredicate expr, ExpressionRewriteContext context) {
+
+        Expression rewrittenChildren = ExpressionUtils.combine(expr.getType(), ExpressionUtils.extract(expr).stream()
+                .map(predicate -> rewrite(predicate, context)).collect(Collectors.toList()));
+
+        if (!(rewrittenChildren instanceof CompoundPredicate)) {
+            return rewrittenChildren;
+        }
+
+        CompoundPredicate compoundPredicate = (CompoundPredicate) rewrittenChildren;
+
+        List<List<Expression>> partitions = ExpressionUtils.extract(compoundPredicate).stream()
+                .map(predicate -> predicate instanceof CompoundPredicate ? ExpressionUtils.extract(
+                        (CompoundPredicate) predicate) : Lists.newArrayList(predicate)).collect(Collectors.toList());

Review Comment:
   what will happen if we have `(a AND b AND c) AND (a OR d OR e)`



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java:
##########
@@ -102,8 +104,13 @@ public static Expression combine(ExpressionType op, List<Expression> expressions
             }
         }
 
-        Optional<Expression> result =
-                distinctExpressions.stream().reduce((left, right) -> new CompoundPredicate(op, left, right));
+        Optional<Expression> result = distinctExpressions.stream().reduce((left, right) -> {
+            if (op == ExpressionType.AND) {
+                return new And(left, right);

Review Comment:
   why rewrite like this?



-- 
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


[GitHub] [doris] morrySnow commented on pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
morrySnow commented on PR #10667:
URL: https://github.com/apache/doris/pull/10667#issuecomment-1189906970

   @924060929 PTAL


-- 
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


[GitHub] [doris] adonis0147 commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
adonis0147 commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r925359964


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java:
##########
@@ -102,8 +104,13 @@ public static Expression combine(ExpressionType op, List<Expression> expressions
             }
         }
 
-        Optional<Expression> result =
-                distinctExpressions.stream().reduce((left, right) -> new CompoundPredicate(op, left, right));
+        Optional<Expression> result = distinctExpressions.stream().reduce((left, right) -> {
+            if (op == ExpressionType.AND) {
+                return new And(left, right);
+            } else {
+                return new Or(left, right);
+            }
+        });

Review Comment:
   ```suggestion
           Optional<Expression> result =
                   distinctExpressions.stream().reduce(op == ExpressionType.AND ? And::new : Or::new);
   ```



-- 
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


[GitHub] [doris] adonis0147 commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
adonis0147 commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r925354681


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java:
##########
@@ -102,8 +104,13 @@ public static Expression combine(ExpressionType op, List<Expression> expressions
             }
         }
 
-        Optional<Expression> result =
-                distinctExpressions.stream().reduce((left, right) -> new CompoundPredicate(op, left, right));
+        Optional<Expression> result = distinctExpressions.stream().reduce((left, right) -> {
+            if (op == ExpressionType.AND) {
+                return new And(left, right);
+            } else {
+                return new Or(left, right);
+            }
+        });

Review Comment:
   OK



-- 
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


[GitHub] [doris] github-actions[bot] commented on pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #10667:
URL: https://github.com/apache/doris/pull/10667#issuecomment-1191010892

   PR approved by at least one committer and no changes requested.


-- 
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


[GitHub] [doris] 924060929 merged pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
924060929 merged PR #10667:
URL: https://github.com/apache/doris/pull/10667


-- 
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


[GitHub] [doris] github-actions[bot] commented on pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #10667:
URL: https://github.com/apache/doris/pull/10667#issuecomment-1189907610

   PR approved by anyone and no changes requested.


-- 
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


[GitHub] [doris] qzsee commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
qzsee commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r924087759


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java:
##########
@@ -102,8 +104,13 @@ public static Expression combine(ExpressionType op, List<Expression> expressions
             }
         }
 
-        Optional<Expression> result =
-                distinctExpressions.stream().reduce((left, right) -> new CompoundPredicate(op, left, right));
+        Optional<Expression> result = distinctExpressions.stream().reduce((left, right) -> {
+            if (op == ExpressionType.AND) {
+                return new And(left, right);

Review Comment:
   && use AND
   || use Or



-- 
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


[GitHub] [doris] qzsee commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
qzsee commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r924097995


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionOfPlanRewrite.java:
##########
@@ -0,0 +1,136 @@
+// 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 org.apache.doris.nereids.rules.PlanRuleFactory;
+import org.apache.doris.nereids.rules.Rule;
+import org.apache.doris.nereids.rules.RulePromise;
+import org.apache.doris.nereids.rules.RuleType;
+import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory;
+import org.apache.doris.nereids.trees.expressions.Alias;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * expression of plan rewrite rule.
+ */
+public class ExpressionOfPlanRewrite implements PlanRuleFactory {
+    ExpressionRuleExecutor rewriter;
+
+    public ExpressionOfPlanRewrite(ExpressionRuleExecutor rewriter) {
+        this.rewriter = Objects.requireNonNull(rewriter, "rewriter is null");
+    }
+
+    @Override
+    public List<Rule<Plan>> buildRules() {
+        return ImmutableList.of(new ProjectExpressionRewrite().build(), new AggExpressionRewrite().build(),
+                new FilterExpressionRewrite().build(), new JoinExpressionRewrite().build());
+    }
+
+    @Override
+    public RulePromise defaultPromise() {
+        return RulePromise.REWRITE;
+    }
+
+    private class ProjectExpressionRewrite extends OneRewriteRuleFactory {
+        @Override
+        public Rule<Plan> build() {
+            return logicalProject().then(project -> {
+                List<NamedExpression> origin = project.getProjects();
+                for (int i = 0; i < origin.size(); i++) {
+                    if (origin.get(i) instanceof Alias) {
+                        Alias alias = (Alias) origin.get(i);
+                        Expression newExpr = rewriter.rewrite((alias).child());
+                        origin.set(i, (NamedExpression) alias.withChildren(Lists.newArrayList(newExpr)));

Review Comment:
   you are right



-- 
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


[GitHub] [doris] adonis0147 commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
adonis0147 commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r925383561


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/CompoundPredicate.java:
##########
@@ -58,7 +58,7 @@ public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
 
     @Override
     public Expression withChildren(List<Expression> children) {
-        return new CompoundPredicate(getType(), children.get(0), children.get(1));
+        throw  new RuntimeException("Do not implement this method");

Review Comment:
   By the way, why not just delete the method?



-- 
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


[GitHub] [doris] qzsee commented on pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
qzsee commented on PR #10667:
URL: https://github.com/apache/doris/pull/10667#issuecomment-1191015199

   > LGTM, but maybe we should add more ut for testExtractCommonFactorRewrite? @qzsee
   
   I added some extra


-- 
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


[GitHub] [doris] 924060929 commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
924060929 commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r926238689


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/OptimizeExpressionOfPlan.java:
##########
@@ -0,0 +1,41 @@
+// 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 org.apache.doris.nereids.rules.expression.rewrite.rules.DistinctPredicatesRule;
+import org.apache.doris.nereids.rules.expression.rewrite.rules.ExtractCommonFactorRule;
+import org.apache.doris.nereids.rules.expression.rewrite.rules.SimplifyNotExprRule;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * optimize expression of plan rule set.
+ */
+public class OptimizeExpressionOfPlan extends ExpressionOfPlanRewrite {

Review Comment:
   missing register this RuleSet into planner?



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/rules/ExtractCommonFactorRule.java:
##########
@@ -0,0 +1,76 @@
+// 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.expressions.CompoundPredicate;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.util.ExpressionUtils;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Extract common expr for `CompoundPredicate`.
+ * for example:
+ * transform (a or b) and (a or c) to a or (b and c)

Review Comment:
   this case maybe can not pushdown predicate to OlapScanNode because of disjunctive expression



-- 
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


[GitHub] [doris] qzsee commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
qzsee commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r924090467


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionOfPlanRewrite.java:
##########
@@ -0,0 +1,136 @@
+// 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 org.apache.doris.nereids.rules.PlanRuleFactory;
+import org.apache.doris.nereids.rules.Rule;
+import org.apache.doris.nereids.rules.RulePromise;
+import org.apache.doris.nereids.rules.RuleType;
+import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory;
+import org.apache.doris.nereids.trees.expressions.Alias;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * expression of plan rewrite rule.
+ */
+public class ExpressionOfPlanRewrite implements PlanRuleFactory {
+    ExpressionRuleExecutor rewriter;

Review Comment:
   ok 



-- 
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


[GitHub] [doris] wangshuo128 commented on pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
wangshuo128 commented on PR #10667:
URL: https://github.com/apache/doris/pull/10667#issuecomment-1189909816

   Please add some description for the PR.


-- 
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


[GitHub] [doris] adonis0147 commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
adonis0147 commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r925272126


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/ExpressionOfPlanRewrite.java:
##########
@@ -0,0 +1,117 @@
+// 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 org.apache.doris.nereids.rules.Rule;
+import org.apache.doris.nereids.rules.RuleType;
+import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory;
+import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * expression of plan rewrite rule.
+ */
+public class ExpressionOfPlanRewrite implements RewriteRuleFactory {
+    private final ExpressionRuleExecutor rewriter;
+
+    public ExpressionOfPlanRewrite(ExpressionRuleExecutor rewriter) {
+        this.rewriter = Objects.requireNonNull(rewriter, "rewriter is null");
+    }
+
+    @Override
+    public List<Rule<Plan>> buildRules() {

Review Comment:
   Please rebase the branch `master` and remove the generic type `<Plan>`.



-- 
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


[GitHub] [doris] 924060929 commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
924060929 commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r926241742


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rewrite/OptimizeExpressionOfPlan.java:
##########
@@ -0,0 +1,41 @@
+// 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 org.apache.doris.nereids.rules.expression.rewrite.rules.DistinctPredicatesRule;
+import org.apache.doris.nereids.rules.expression.rewrite.rules.ExtractCommonFactorRule;
+import org.apache.doris.nereids.rules.expression.rewrite.rules.SimplifyNotExprRule;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * optimize expression of plan rule set.
+ */
+public class OptimizeExpressionOfPlan extends ExpressionOfPlanRewrite {

Review Comment:
   fix later



-- 
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


[GitHub] [doris] adonis0147 commented on a diff in pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
adonis0147 commented on code in PR #10667:
URL: https://github.com/apache/doris/pull/10667#discussion_r925381603


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/CompoundPredicate.java:
##########
@@ -58,7 +58,7 @@ public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
 
     @Override
     public Expression withChildren(List<Expression> children) {
-        return new CompoundPredicate(getType(), children.get(0), children.get(1));
+        throw  new RuntimeException("Do not implement this method");

Review Comment:
   Use `Do not implement this method` as a comment. For the exception message, I think it is better to use another description.



-- 
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


[GitHub] [doris] github-actions[bot] commented on pull request #10667: [Enhancement](Nereids) add some expr rewrite rule and plan rewrite rule of rewrite its expression

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #10667:
URL: https://github.com/apache/doris/pull/10667#issuecomment-1191023404

   PR approved by at least one committer and no changes requested.


-- 
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