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/18 12:20:07 UTC

[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

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