morrySnow commented on code in PR #12129: URL: https://github.com/apache/doris/pull/12129#discussion_r959557878
########## fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java: ########## @@ -69,9 +69,10 @@ public Optional<GroupExpression> getGroupExpression() { @Override public boolean canBind() { - return !bound() + boolean result = !bound() Review Comment: unnecessary change ########## fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Alias.java: ########## @@ -49,7 +49,7 @@ public Alias(Expression child, String name) { } @VisibleForTesting - Alias(ExprId exprId, Expression child, String name) { + public Alias(ExprId exprId, Expression child, String name) { Review Comment: unnecessary change ########## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotReference.java: ########## @@ -123,11 +126,23 @@ public List<Rule> buildRules() { return new LogicalSort<>(sortItemList, sort.child()); }) ), - + RuleType.BINDING_HAVING_SLOT.build( + logicalHaving(logicalAggregate()).thenApply(ctx -> { + LogicalHaving<LogicalAggregate<GroupPlan>> having = ctx.root; + LogicalAggregate<GroupPlan> aggregate = having.child(); + Set<Slot> boundSlots = Stream.concat(Stream.of(aggregate), aggregate.children().stream()) + .flatMap(plan -> plan.getOutput().stream()) + .collect(Collectors.toSet()); + Expression boundPredicates = new SlotBinder( + toScope(new ArrayList<>(boundSlots)), having, ctx.cascadesContext + ).bind(having.getPredicates()); + return new LogicalHaving<>(boundPredicates, having.child()); + }) + ), RuleType.BINDING_NON_LEAF_LOGICAL_PLAN.build( - logicalPlan() - .when(plan -> plan.canBind() && !(plan instanceof LeafPlan)) - .then(LogicalPlan::recomputeLogicalProperties) + logicalPlan() + .when(plan -> plan.canBind() && !(plan instanceof LeafPlan)) + .then(LogicalPlan::recomputeLogicalProperties) Review Comment: do not change indentions ########## fe/fe-core/src/test/java/org/apache/doris/nereids/parser/HavingClauseTest.java: ########## @@ -0,0 +1,327 @@ +// 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.parser; + +import org.apache.doris.common.ExceptionChecker; +import org.apache.doris.nereids.datasets.tpch.AnalyzeCheckTestBase; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.expressions.Add; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.GreaterThan; +import org.apache.doris.nereids.trees.expressions.NamedExpressionUtil; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.functions.Count; +import org.apache.doris.nereids.trees.expressions.functions.Min; +import org.apache.doris.nereids.trees.expressions.functions.Sum; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.util.FieldChecker; +import org.apache.doris.nereids.util.PatternMatchSupported; +import org.apache.doris.nereids.util.PlanChecker; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.junit.jupiter.api.Test; + +public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMatchSupported { Review Comment: please add a case like this: ```sql SELECT pk + 1, pk + 1 + 1, pk + 2, SUM(a1), COUNT(a1) + 1, SUM(a1 + a2), COUNT(a2) as v1 GROUP BY pk, pk + 1 HAVING pk > 0 AND COUNT(a1) + 1 > 0 AND SUM(a1 + a2) + 1 > 0 AND v1 + 1 > 0 AND v1 > 0 ``` ########## fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java: ########## @@ -0,0 +1,111 @@ +// 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.trees.plans.logical; + +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Filter; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * Logical Having plan + * @param <CHILD_TYPE> Types which inherit from {@link Plan} + */ +public class LogicalHaving<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> implements Filter { + + private final Expression predicates; + + public LogicalHaving(Expression predicates, CHILD_TYPE child) { + this(predicates, Optional.empty(), Optional.empty(), child); + } + + public LogicalHaving(Expression predicates, Optional<GroupExpression> groupExpression, + Optional<LogicalProperties> logicalProperties, CHILD_TYPE child) { + super(PlanType.LOGICAL_HAVING, groupExpression, logicalProperties, child); + this.predicates = Objects.requireNonNull(predicates, "predicates can not be null"); + } + + @Override + public Expression getPredicates() { + return predicates; + } + + @Override + public Plan withChildren(List<Plan> children) { + Preconditions.checkArgument(children.size() == 1); + return new LogicalHaving<>(predicates, children.get(0)); + } + + @Override + public <R, C> R accept(PlanVisitor<R, C> visitor, C context) { + return visitor.visitLogicalHaving((LogicalHaving<Plan>) this, context); + } + + @Override + public List<Expression> getExpressions() { + return ImmutableList.of(predicates); + } + + @Override + public Plan withGroupExpression(Optional<GroupExpression> groupExpression) { + return new LogicalHaving<>(predicates, groupExpression, Optional.of(logicalProperties), child()); + } + + @Override + public Plan withLogicalProperties(Optional<LogicalProperties> logicalProperties) { + return new LogicalHaving<>(predicates, Optional.empty(), logicalProperties, child()); + } + + @Override + public List<Slot> computeOutput() { + return child().getOutput(); + } + + @Override + public int hashCode() { + return Objects.hashCode(predicates); + } + + @Override + public boolean equals(Object object) { + if (this == object) { + return true; + } + if (!(object instanceof LogicalHaving) || getClass() != object.getClass()) { + return false; + } + LogicalHaving<CHILD_TYPE> other = (LogicalHaving<CHILD_TYPE>) object; Review Comment: other's child type maybe different with this: so ```suggestion LogicalHaving other = (LogicalHaving) object; ``` ########## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotReference.java.orig: ########## @@ -0,0 +1,345 @@ +// 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.analysis; + +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.analyzer.UnboundAlias; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.analyzer.UnboundStar; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.properties.OrderKey; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +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.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.visitor.DefaultSubExprRewriter; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.LeafPlan; +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.LogicalHaving; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import org.apache.commons.lang.StringUtils; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * BindSlotReference. + */ +public class BindSlotReference implements AnalysisRuleFactory { Review Comment: useless file? ########## fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java: ########## @@ -0,0 +1,111 @@ +// 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.trees.plans.logical; + +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Filter; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * Logical Having plan + * @param <CHILD_TYPE> Types which inherit from {@link Plan} + */ +public class LogicalHaving<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> implements Filter { + + private final Expression predicates; + + public LogicalHaving(Expression predicates, CHILD_TYPE child) { + this(predicates, Optional.empty(), Optional.empty(), child); + } + + public LogicalHaving(Expression predicates, Optional<GroupExpression> groupExpression, + Optional<LogicalProperties> logicalProperties, CHILD_TYPE child) { + super(PlanType.LOGICAL_HAVING, groupExpression, logicalProperties, child); + this.predicates = Objects.requireNonNull(predicates, "predicates can not be null"); + } + + @Override + public Expression getPredicates() { + return predicates; + } + + @Override + public Plan withChildren(List<Plan> children) { + Preconditions.checkArgument(children.size() == 1); + return new LogicalHaving<>(predicates, children.get(0)); + } + + @Override + public <R, C> R accept(PlanVisitor<R, C> visitor, C context) { + return visitor.visitLogicalHaving((LogicalHaving<Plan>) this, context); + } + + @Override + public List<Expression> getExpressions() { + return ImmutableList.of(predicates); + } + + @Override + public Plan withGroupExpression(Optional<GroupExpression> groupExpression) { + return new LogicalHaving<>(predicates, groupExpression, Optional.of(logicalProperties), child()); + } + + @Override + public Plan withLogicalProperties(Optional<LogicalProperties> logicalProperties) { + return new LogicalHaving<>(predicates, Optional.empty(), logicalProperties, child()); + } + + @Override + public List<Slot> computeOutput() { + return child().getOutput(); + } + + @Override + public int hashCode() { + return Objects.hashCode(predicates); + } + + @Override + public boolean equals(Object object) { + if (this == object) { + return true; + } + if (!(object instanceof LogicalHaving) || getClass() != object.getClass()) { + return false; + } + LogicalHaving<CHILD_TYPE> other = (LogicalHaving<CHILD_TYPE>) object; + return predicates.equals(other.predicates); + } + + @Override + public String toString() { + return "LogicalHaving (" + predicates + ")"; Review Comment: we have a new Util function to generate value for toString: https://github.com/apache/doris/blob/a49bde8a718ec96f107ac7d44dda8a9017f7ebc4/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java#L131 ########## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotReference.java: ########## @@ -123,11 +126,23 @@ public List<Rule> buildRules() { return new LogicalSort<>(sortItemList, sort.child()); }) ), - + RuleType.BINDING_HAVING_SLOT.build( + logicalHaving(logicalAggregate()).thenApply(ctx -> { + LogicalHaving<LogicalAggregate<GroupPlan>> having = ctx.root; + LogicalAggregate<GroupPlan> aggregate = having.child(); + Set<Slot> boundSlots = Stream.concat(Stream.of(aggregate), aggregate.children().stream()) + .flatMap(plan -> plan.getOutput().stream()) + .collect(Collectors.toSet()); + Expression boundPredicates = new SlotBinder( + toScope(new ArrayList<>(boundSlots)), having, ctx.cascadesContext Review Comment: ```suggestion List<Slot> boundSlots = Stream.concat(Stream.of(aggregate), aggregate.children().stream()) .flatMap(plan -> plan.getOutput().stream()) .collect(Collectors.toLists()); Expression boundPredicates = new SlotBinder( toScope(boundSlots), having, ctx.cascadesContext ``` ########## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveHaving.java: ########## @@ -0,0 +1,152 @@ +// 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.analysis; + +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +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.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.functions.AggregateFunction; +import org.apache.doris.nereids.trees.plans.GroupPlan; +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.LogicalHaving; + +import com.google.common.collect.Maps; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Resolve having clause to the aggregation. + */ +public class ResolveHaving extends OneAnalysisRuleFactory { Review Comment: ```suggestion public class BindHaving extends OneAnalysisRuleFactory { ``` please use **bind** for consistency with other rules ########## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveHaving.java: ########## @@ -0,0 +1,152 @@ +// 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.analysis; + +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +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.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.functions.AggregateFunction; +import org.apache.doris.nereids.trees.plans.GroupPlan; +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.LogicalHaving; + +import com.google.common.collect.Maps; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Resolve having clause to the aggregation. + */ +public class ResolveHaving extends OneAnalysisRuleFactory { + @Override + public Rule build() { + return RuleType.RESOLVE_HAVING.build( + logicalHaving(logicalAggregate()).thenApply(ctx -> { + LogicalHaving<LogicalAggregate<GroupPlan>> having = ctx.root; + LogicalAggregate<GroupPlan> aggregate = having.child(); + Set<Expression> groupByExpressions = new HashSet<>(aggregate.getGroupByExpressions()); + Set<Expression> tokens = extractTokensForAggregation(having.getPredicates()) + .collect(Collectors.toSet()); + tokens.forEach(token -> { + if (token instanceof SlotReference && (!groupByExpressions.contains(token) + && aggregate.getOutputExpressions().stream() + .noneMatch(output -> { + if (output instanceof SlotReference) { + return output.equals(token); + } else if (output instanceof Alias) { + Alias alias = (Alias) output; + return alias.toSlot().equals(token) || alias.child().equals(token); + } + return false; + }))) { + throw new AnalysisException(token.toSql() + " in having clause should be grouped by."); + } else if (token instanceof AggregateFunction + && checkWhetherNestedAggregateFunctionsExist((AggregateFunction) token)) { + throw new AnalysisException("Aggregate functions in having clause can't be nested: " + + token.toSql() + "."); + } + }); + return createLogicalFilter(having, tokens); Review Comment: if we put some expression to child's output expression, we need add a project on filter to keep the output constant -- 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-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org