morrySnow commented on code in PR #40048: URL: https://github.com/apache/doris/pull/40048#discussion_r1766533006
########## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpQualifyMissingSlot.java: ########## @@ -0,0 +1,176 @@ +// 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.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.WindowExpression; +import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.algebra.Aggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalHaving; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalQualify; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * fill up missing slot for qualify + */ +public class FillUpQualifyMissingSlot extends FillUpMissingSlots { + @Override + public List<Rule> buildRules() { Review Comment: why impl a new class instead of put them into FillUpMissingSlots? ########## fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java: ########## @@ -1385,6 +1393,55 @@ public LogicalPlan visitRegularQuerySpecification(RegularQuerySpecificationConte }); } + private LogicalPlan withQualifyQuery(LogicalPlan input, RegularQuerySpecificationContext ctx) { + QualifyClauseContext qualifyClauseContext = ctx.qualifyClause(); + if (qualifyClauseContext != null) { + Expression qualifyExpr = getExpression(qualifyClauseContext.booleanExpression()); +// List<NamedExpression> windows = new ArrayList<>(); Review Comment: this comment block should be removed? ########## fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java: ########## @@ -1385,6 +1393,55 @@ public LogicalPlan visitRegularQuerySpecification(RegularQuerySpecificationConte }); } + private LogicalPlan withQualifyQuery(LogicalPlan input, RegularQuerySpecificationContext ctx) { + QualifyClauseContext qualifyClauseContext = ctx.qualifyClause(); Review Comment: maybe should impl in `withSelectQuerySpecification` ########## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/QualifyToFilter.java: ########## @@ -0,0 +1,91 @@ +// 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.SlotReference; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionVisitor; +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.LogicalProject; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * HavingToFilter + */ +public class QualifyToFilter extends OneAnalysisRuleFactory { Review Comment: we should not do check things in this rule, since its name is `AToB`. check should be apply before translate, and this rule only do Qualify to Filter thing ########## fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java: ########## @@ -1385,6 +1393,55 @@ public LogicalPlan visitRegularQuerySpecification(RegularQuerySpecificationConte }); } + private LogicalPlan withQualifyQuery(LogicalPlan input, RegularQuerySpecificationContext ctx) { + QualifyClauseContext qualifyClauseContext = ctx.qualifyClause(); + if (qualifyClauseContext != null) { + Expression qualifyExpr = getExpression(qualifyClauseContext.booleanExpression()); +// List<NamedExpression> windows = new ArrayList<>(); +// List<NamedExpression> excepts = new ArrayList<>(); +// qualifyExpr = qualifyExpr.accept(new DefaultExpressionRewriter<List<NamedExpression>>() { +// @Override +// public Expression visitWindow(WindowExpression windowExpression, List<NamedExpression> context) { +// String aliasName = ConnectContext.get().getStatementContext().generateColumnName(); +// UnboundAlias alias = new UnboundAlias(windowExpression, aliasName); +// windows.add(alias); +// UnboundSlot unboundSlot = new UnboundSlot(aliasName); +// excepts.add(unboundSlot); +// return unboundSlot; +// } +// }, windows); +// +// LogicalPlan plan = (LogicalPlan) input.accept(new DefaultPlanRewriter<Void>() { +// @Override +// public Plan visitLogicalProject(LogicalProject<? extends Plan> project, Void context) { +// if (!windows.isEmpty()) { +// project = project.withProjects(ImmutableList.<NamedExpression>builder() +// .addAll(project.getProjects()) +// .addAll(windows).build()); +// } +// return project; +// } +// +// @Override +// public Plan visitLogicalAggregate(LogicalAggregate<? extends Plan> aggregate, Void context) { +// if (!windows.isEmpty()) { +// aggregate = aggregate.withGroupByAndOutput( +// aggregate.getGroupByExpressions(), +// ImmutableList.<NamedExpression>builder() +// .addAll(aggregate.getOutputExpressions()) +// .addAll(windows).build()); +// } +// return aggregate; +// } +// }, null); + LogicalQualify<LogicalPlan> qualify = new LogicalQualify<>(Sets.newHashSet(qualifyExpr), input); + List<NamedExpression> output = + Lists.newArrayList(new UnboundStar(ImmutableList.of(), ImmutableList.of(), ImmutableList.of())); + return new LogicalProject<>(output, qualify); Review Comment: why add another project here? ########## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java: ########## @@ -697,6 +702,107 @@ private Plan bindFilter(MatchingContext<LogicalFilter<Plan>> ctx) { return new LogicalFilter<>(boundConjuncts.build(), filter.child()); } + private Plan bindQualify(MatchingContext<LogicalQualify<Plan>> ctx) { + LogicalQualify<Plan> qualify = ctx.root; + CascadesContext cascadesContext = ctx.cascadesContext; + ImmutableSet.Builder<Expression> boundConjuncts = ImmutableSet.builderWithExpectedSize( + qualify.getConjuncts().size()); + qualify.accept(new DefaultPlanVisitor<Void, Void>() { + @Override + public Void visitLogicalProject(LogicalProject<? extends Plan> project, Void context) { + if (project.child() instanceof LogicalHaving) { + return visit(project, context); + } + if (project.isDistinct()) { + Plan child = project.child(); + if (child instanceof Aggregate) { Review Comment: why check child here? i think we should always bind project's child's output first, if failed, then bind on project's output, right? ########## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/QualifyToFilter.java: ########## @@ -0,0 +1,91 @@ +// 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.SlotReference; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionVisitor; +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.LogicalProject; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * HavingToFilter + */ +public class QualifyToFilter extends OneAnalysisRuleFactory { + @Override + public Rule build() { + return logicalQualify() + .then(qualify -> { + Set<String> windowSlotName = new HashSet<>(); Review Comment: hard to understand. why all slot in qualify is window slot? how about `c2` in `rn over() > c2` ########## fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java: ########## @@ -0,0 +1,138 @@ +// 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.DataTrait.Builder; +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 org.apache.doris.nereids.util.Utils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +/** + * Logical qualify plan. + */ +public class LogicalQualify<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> implements Filter { + + private final Set<Expression> conjuncts; + + public LogicalQualify(Set<Expression> conjuncts, CHILD_TYPE child) { + this(conjuncts, Optional.empty(), Optional.empty(), child); + } + + private LogicalQualify(Set<Expression> conjuncts, Optional<GroupExpression> groupExpression, + Optional<LogicalProperties> logicalProperties, CHILD_TYPE child) { + super(PlanType.LOGICAL_QUALIFY, groupExpression, logicalProperties, child); + this.conjuncts = ImmutableSet.copyOf(Objects.requireNonNull(conjuncts, "conjuncts can not be null")); + } + + @Override + public Set<Expression> getConjuncts() { + return conjuncts; + } + + @Override + public List<Slot> computeOutput() { + return child().getOutput(); + } + + @Override + public Plan withGroupExpression(Optional<GroupExpression> groupExpression) { + return new LogicalQualify<>(conjuncts, groupExpression, Optional.of(getLogicalProperties()), child()); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional<GroupExpression> groupExpression, + Optional<LogicalProperties> logicalProperties, List<Plan> children) { + Preconditions.checkArgument(children.size() == 1); + return new LogicalQualify<>(conjuncts, groupExpression, logicalProperties, children.get(0)); + } + + public LogicalQualify<Plan> withConjuncts(Set<Expression> conjuncts) { + return new LogicalQualify<>(conjuncts, Optional.empty(), Optional.of(getLogicalProperties()), child()); + } + + @Override + public String toString() { + return Utils.toSqlString("LogicalQualify[" + id.asInt() + "]", + "predicates", getPredicate() + ); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LogicalQualify that = (LogicalQualify) o; + return conjuncts.equals(that.conjuncts); + } + + @Override + public int hashCode() { + return Objects.hash(conjuncts); + } + + @Override + public <R, C> R accept(PlanVisitor<R, C> visitor, C context) { + return visitor.visitLogicalQualify(this, context); + } + + @Override + public List<? extends Expression> getExpressions() { + return ImmutableList.copyOf(conjuncts); + } + + @Override + public LogicalQualify<Plan> withChildren(List<Plan> children) { + Preconditions.checkArgument(children.size() == 1); + return new LogicalQualify<>(conjuncts, children.get(0)); + } + + @Override + public void computeUnique(Builder builder) { + } + + @Override + public void computeUniform(Builder builder) { + } + + @Override + public void computeEqualSet(Builder builder) { + } + + @Override + public void computeFd(Builder builder) { + } Review Comment: these interface should impl same with filter and having ########## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java: ########## @@ -697,6 +702,107 @@ private Plan bindFilter(MatchingContext<LogicalFilter<Plan>> ctx) { return new LogicalFilter<>(boundConjuncts.build(), filter.child()); } + private Plan bindQualify(MatchingContext<LogicalQualify<Plan>> ctx) { + LogicalQualify<Plan> qualify = ctx.root; + CascadesContext cascadesContext = ctx.cascadesContext; + ImmutableSet.Builder<Expression> boundConjuncts = ImmutableSet.builderWithExpectedSize( + qualify.getConjuncts().size()); + qualify.accept(new DefaultPlanVisitor<Void, Void>() { + @Override + public Void visitLogicalProject(LogicalProject<? extends Plan> project, Void context) { Review Comment: we could use pattern to distinguish them? `logicalQualify(logicalProject())` and `logicalQualify(logicalAggregate())` ? ########## fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java: ########## @@ -697,6 +702,107 @@ private Plan bindFilter(MatchingContext<LogicalFilter<Plan>> ctx) { return new LogicalFilter<>(boundConjuncts.build(), filter.child()); } + private Plan bindQualify(MatchingContext<LogicalQualify<Plan>> ctx) { + LogicalQualify<Plan> qualify = ctx.root; + CascadesContext cascadesContext = ctx.cascadesContext; + ImmutableSet.Builder<Expression> boundConjuncts = ImmutableSet.builderWithExpectedSize( + qualify.getConjuncts().size()); + qualify.accept(new DefaultPlanVisitor<Void, Void>() { + @Override + public Void visitLogicalProject(LogicalProject<? extends Plan> project, Void context) { + if (project.child() instanceof LogicalHaving) { + return visit(project, context); + } + if (project.isDistinct()) { + Plan child = project.child(); + if (child instanceof Aggregate) { + bindSlotByAgg((Aggregate<? extends Plan>) child, cascadesContext, qualify, boundConjuncts); + } else { + bindSlotByProject(project, cascadesContext, qualify, boundConjuncts); + } + } else { + bindSlotByProject(project, cascadesContext, qualify, boundConjuncts); + } + return null; + } + + @Override + public Void visitLogicalAggregate(LogicalAggregate<? extends Plan> aggregate, Void context) { + bindSlotByAgg(aggregate, cascadesContext, qualify, boundConjuncts); + return null; + } + }, null); + return new LogicalQualify<>(boundConjuncts.build(), qualify.child()); + } + + private void bindSlotByProject(LogicalProject<? extends Plan> project, CascadesContext cascadesContext, Review Comment: since this class use to bind all plan node, the function name should be `bindQualifyProject` -- 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