This is an automated email from the ASF dual-hosted git repository. kxiao pushed a commit to branch branch-2.0 in repository https://gitbox.apache.org/repos/asf/doris.git
commit cee0a6ccf00209923afd93c2a0a50e029ccdcc0c Author: morrySnow <101034200+morrys...@users.noreply.github.com> AuthorDate: Thu Sep 7 20:15:59 2023 +0800 [opt](Nereids) optimize error msg of unbound slot (#23933) for example: ```sql select avg(c3) from (select c2 from t2) v; ``` the error msg before this PR ``` Invalid call to c3.getDataType() on unbound object ``` the error msg after this PR ``` Unknown column 'c3' in 'table list' in AGGREGATE clause ``` --- .../apache/doris/nereids/analyzer/UnboundSlot.java | 5 ++ .../doris/nereids/jobs/executor/Analyzer.java | 4 +- .../org/apache/doris/nereids/rules/RuleType.java | 1 - .../nereids/rules/analysis/BindExpression.java | 70 ++++++++++++---------- .../{CheckBound.java => CheckAfterBind.java} | 35 +---------- .../rules/analysis/BindSlotReferenceTest.java | 22 +++---- .../nereids/rules/analysis/CheckAnalysisTest.java | 13 ---- .../nereids_p0/except/test_bound_exception.groovy | 15 ++--- .../suites/nereids_syntax_p0/bind_priority.groovy | 2 +- 9 files changed, 64 insertions(+), 103 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java index 09eb1c94f5..47bad57f58 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java @@ -58,6 +58,11 @@ public class UnboundSlot extends Slot implements Unbound, PropagateNullable { }).reduce((left, right) -> left + "." + right).orElse(""); } + @Override + public List<String> getQualifier() { + return nameParts.subList(0, nameParts.size() - 1); + } + @Override public String toSql() { return nameParts.stream().map(Utils::quoteIfNeeded).reduce((left, right) -> left + "." + right).orElse(""); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java index 6d63039d04..a4a8dd1098 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java @@ -25,8 +25,8 @@ import org.apache.doris.nereids.rules.analysis.BindExpression; import org.apache.doris.nereids.rules.analysis.BindRelation; import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; import org.apache.doris.nereids.rules.analysis.BindSink; +import org.apache.doris.nereids.rules.analysis.CheckAfterBind; import org.apache.doris.nereids.rules.analysis.CheckAnalysis; -import org.apache.doris.nereids.rules.analysis.CheckBound; import org.apache.doris.nereids.rules.analysis.CheckPolicy; import org.apache.doris.nereids.rules.analysis.EliminateGroupByConstant; import org.apache.doris.nereids.rules.analysis.FillUpMissingSlots; @@ -89,7 +89,7 @@ public class Analyzer extends AbstractBatchJobExecutor { new BindExpression() ), topDown(new BindSink()), - bottomUp(new CheckBound()), + bottomUp(new CheckAfterBind()), bottomUp( new ProjectToGlobalAggregate(), // this rule check's the logicalProject node's isDistinct property diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index 33617deddb..3ede58eaab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -92,7 +92,6 @@ public enum RuleType { CHECK_AGGREGATE_ANALYSIS(RuleTypeClass.CHECK), CHECK_ANALYSIS(RuleTypeClass.CHECK), CHECK_OBJECT_TYPE_ANALYSIS(RuleTypeClass.CHECK), - CHECK_BOUND(RuleTypeClass.CHECK), CHECK_DATA_TYPES(RuleTypeClass.CHECK), // rewrite rules diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java index 1afecc4147..96c6ed1dd5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java @@ -77,8 +77,8 @@ import org.apache.doris.qe.ConnectContext; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import org.apache.commons.lang3.StringUtils; import java.util.ArrayList; import java.util.Collection; @@ -88,7 +88,6 @@ 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; @@ -140,7 +139,7 @@ public class BindExpression implements AnalysisRuleFactory { boundProjections = flatBoundStar(boundProjections, boundExceptions); } boundProjections = boundProjections.stream() - .map(expr -> bindFunction(expr, ctx.cascadesContext)) + .map(expr -> bindFunction(expr, ctx.root, ctx.cascadesContext)) .collect(ImmutableList.toImmutableList()); return new LogicalProject<>(boundProjections, project.isDistinct(), project.child()); }) @@ -150,7 +149,7 @@ public class BindExpression implements AnalysisRuleFactory { LogicalFilter<Plan> filter = ctx.root; Set<Expression> boundConjuncts = filter.getConjuncts().stream() .map(expr -> bindSlot(expr, filter.child(), ctx.cascadesContext)) - .map(expr -> bindFunction(expr, ctx.cascadesContext)) + .map(expr -> bindFunction(expr, ctx.root, ctx.cascadesContext)) .collect(ImmutableSet.toImmutableSet()); return new LogicalFilter<>(boundConjuncts, filter.child()); }) @@ -203,11 +202,11 @@ public class BindExpression implements AnalysisRuleFactory { LogicalJoin<Plan, Plan> join = ctx.root; List<Expression> cond = join.getOtherJoinConjuncts().stream() .map(expr -> bindSlot(expr, join.children(), ctx.cascadesContext)) - .map(expr -> bindFunction(expr, ctx.cascadesContext)) + .map(expr -> bindFunction(expr, ctx.root, ctx.cascadesContext)) .collect(Collectors.toList()); List<Expression> hashJoinConjuncts = join.getHashJoinConjuncts().stream() .map(expr -> bindSlot(expr, join.children(), ctx.cascadesContext)) - .map(expr -> bindFunction(expr, ctx.cascadesContext)) + .map(expr -> bindFunction(expr, ctx.root, ctx.cascadesContext)) .collect(Collectors.toList()); return new LogicalJoin<>(join.getJoinType(), hashJoinConjuncts, cond, join.getHint(), join.getMarkJoinSlotReference(), @@ -219,7 +218,7 @@ public class BindExpression implements AnalysisRuleFactory { LogicalAggregate<Plan> agg = ctx.root; List<NamedExpression> output = agg.getOutputExpressions().stream() .map(expr -> bindSlot(expr, agg.child(), ctx.cascadesContext)) - .map(expr -> bindFunction(expr, ctx.cascadesContext)) + .map(expr -> bindFunction(expr, ctx.root, ctx.cascadesContext)) .collect(ImmutableList.toImmutableList()); // The columns referenced in group by are first obtained from the child's output, @@ -327,20 +326,9 @@ public class BindExpression implements AnalysisRuleFactory { return e; }) .collect(Collectors.toList()); - List<Expression> unboundGroupBys = Lists.newArrayList(); - Predicate<List<Expression>> hasUnBound = (exprs) -> exprs.stream().anyMatch( - expression -> { - if (expression.anyMatch(UnboundSlot.class::isInstance)) { - unboundGroupBys.add(expression); - return true; - } - return false; - }); - if (hasUnBound.test(groupBy)) { - throw new AnalysisException("cannot bind GROUP BY KEY: " + unboundGroupBys.get(0).toSql()); - } + groupBy.forEach(expression -> checkBound(expression, ctx.root)); groupBy = groupBy.stream() - .map(expr -> bindFunction(expr, ctx.cascadesContext)) + .map(expr -> bindFunction(expr, ctx.root, ctx.cascadesContext)) .collect(ImmutableList.toImmutableList()); checkIfOutputAliasNameDuplicatedForGroupBy(groupBy, output); return agg.withGroupByAndOutput(groupBy, output); @@ -351,7 +339,7 @@ public class BindExpression implements AnalysisRuleFactory { LogicalRepeat<Plan> repeat = ctx.root; List<NamedExpression> output = repeat.getOutputExpressions().stream() .map(expr -> bindSlot(expr, repeat.child(), ctx.cascadesContext)) - .map(expr -> bindFunction(expr, ctx.cascadesContext)) + .map(expr -> bindFunction(expr, ctx.root, ctx.cascadesContext)) .collect(ImmutableList.toImmutableList()); // The columns referenced in group by are first obtained from the child's output, @@ -384,7 +372,7 @@ public class BindExpression implements AnalysisRuleFactory { .stream() .map(groupingSet -> groupingSet.stream() .map(expr -> bindSlot(expr, repeat.child(), ctx.cascadesContext)) - .map(expr -> bindFunction(expr, ctx.cascadesContext)) + .map(expr -> bindFunction(expr, ctx.root, ctx.cascadesContext)) .collect(ImmutableList.toImmutableList())) .collect(ImmutableList.toImmutableList()); List<NamedExpression> newOutput = adjustNullableForRepeat(groupingSets, output); @@ -439,7 +427,7 @@ public class BindExpression implements AnalysisRuleFactory { .stream() .map(orderKey -> { Expression item = bindSlot(orderKey.getExpr(), sort.child(), ctx.cascadesContext); - item = bindFunction(item, ctx.cascadesContext); + item = bindFunction(item, ctx.root, ctx.cascadesContext); return new OrderKey(item, orderKey.isAsc(), orderKey.isNullFirst()); }).collect(Collectors.toList()); return new LogicalSort<>(sortItemList, sort.child()); @@ -454,7 +442,7 @@ public class BindExpression implements AnalysisRuleFactory { expr = bindSlot(expr, childPlan.child(), ctx.cascadesContext, false); return bindSlot(expr, childPlan, ctx.cascadesContext, false); }) - .map(expr -> bindFunction(expr, ctx.cascadesContext)) + .map(expr -> bindFunction(expr, ctx.root, ctx.cascadesContext)) .collect(Collectors.toSet()); checkIfOutputAliasNameDuplicatedForGroupBy(ImmutableList.copyOf(boundConjuncts), childPlan.getOutputExpressions()); @@ -470,7 +458,7 @@ public class BindExpression implements AnalysisRuleFactory { expr = bindSlot(expr, childPlan, ctx.cascadesContext, false); return bindSlot(expr, childPlan.children(), ctx.cascadesContext, false); }) - .map(expr -> bindFunction(expr, ctx.cascadesContext)) + .map(expr -> bindFunction(expr, ctx.root, ctx.cascadesContext)) .collect(Collectors.toSet()); checkIfOutputAliasNameDuplicatedForGroupBy(ImmutableList.copyOf(boundConjuncts), childPlan.getOutput().stream().map(NamedExpression.class::cast) @@ -485,7 +473,7 @@ public class BindExpression implements AnalysisRuleFactory { List<NamedExpression> projects = oneRowRelation.getProjects() .stream() .map(project -> bindSlot(project, ImmutableList.of(), ctx.cascadesContext)) - .map(project -> bindFunction(project, ctx.cascadesContext)) + .map(project -> bindFunction(project, ctx.root, ctx.cascadesContext)) .collect(Collectors.toList()); return new LogicalOneRowRelation(oneRowRelation.getRelationId(), projects); }) @@ -531,7 +519,7 @@ public class BindExpression implements AnalysisRuleFactory { List<Function> boundSlotGenerators = bindSlot(generate.getGenerators(), generate.child(), ctx.cascadesContext); List<Function> boundFunctionGenerators = boundSlotGenerators.stream() - .map(f -> bindTableGeneratingFunction((UnboundFunction) f, ctx.cascadesContext)) + .map(f -> bindTableGeneratingFunction((UnboundFunction) f, ctx.root, ctx.cascadesContext)) .collect(Collectors.toList()); ImmutableList.Builder<Slot> slotBuilder = ImmutableList.builder(); for (int i = 0; i < generate.getGeneratorOutput().size(); i++) { @@ -591,7 +579,7 @@ public class BindExpression implements AnalysisRuleFactory { .map(orderKey -> { Expression item = bindSlot(orderKey.getExpr(), plan, ctx, true, false); item = bindSlot(item, plan.children(), ctx, true, false); - item = bindFunction(item, ctx); + item = bindFunction(item, sort, ctx); return new OrderKey(item, orderKey.isAsc(), orderKey.isNullFirst()); }).collect(Collectors.toList()); return new LogicalSort<>(sortItemList, sort.child()); @@ -660,8 +648,9 @@ public class BindExpression implements AnalysisRuleFactory { } @SuppressWarnings("unchecked") - private <E extends Expression> E bindFunction(E expr, CascadesContext cascadesContext) { - return (E) FunctionBinder.INSTANCE.rewrite(expr, new ExpressionRewriteContext(cascadesContext)); + private <E extends Expression> E bindFunction(E expr, Plan plan, CascadesContext cascadesContext) { + return (E) FunctionBinder.INSTANCE.rewrite(checkBound(expr, plan), + new ExpressionRewriteContext(cascadesContext)); } /** @@ -720,10 +709,10 @@ public class BindExpression implements AnalysisRuleFactory { } } - private BoundFunction bindTableGeneratingFunction(UnboundFunction unboundFunction, + private BoundFunction bindTableGeneratingFunction(UnboundFunction unboundFunction, Plan plan, CascadesContext cascadesContext) { List<Expression> boundArguments = unboundFunction.getArguments().stream() - .map(e -> bindFunction(e, cascadesContext)) + .map(e -> bindFunction(e, plan, cascadesContext)) .collect(Collectors.toList()); FunctionRegistry functionRegistry = cascadesContext.getConnectContext().getEnv().getFunctionRegistry(); @@ -762,4 +751,21 @@ public class BindExpression implements AnalysisRuleFactory { throw new AnalysisException("group_by_and_having_use_alias=true is unsupported for Nereids"); } } + + private <E extends Expression> E checkBound(E expression, Plan plan) { + expression.foreachUp(e -> { + if (e instanceof UnboundSlot) { + UnboundSlot unboundSlot = (UnboundSlot) e; + String tableName = StringUtils.join(unboundSlot.getQualifier(), "."); + if (tableName.isEmpty()) { + tableName = "table list"; + } + throw new AnalysisException("Unknown column '" + + unboundSlot.getNameParts().get(unboundSlot.getNameParts().size() - 1) + + "' in '" + tableName + "' in " + + plan.getType().toString().substring("LOGICAL_".length()) + " clause"); + } + }); + return expression; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckBound.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterBind.java similarity index 64% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckBound.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterBind.java index 674f6977fe..9658bfef20 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckBound.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterBind.java @@ -18,9 +18,6 @@ package org.apache.doris.nereids.rules.analysis; import org.apache.doris.catalog.Type; -import org.apache.doris.nereids.analyzer.Unbound; -import org.apache.doris.nereids.analyzer.UnboundFunction; -import org.apache.doris.nereids.analyzer.UnboundSlot; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; @@ -31,11 +28,9 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalHaving; import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.collect.ImmutableList; -import org.apache.commons.lang3.StringUtils; import java.util.List; import java.util.Set; -import java.util.stream.Collectors; /** * Check bound rule to check semantic correct after bounding of expression by Nereids. @@ -43,17 +38,11 @@ import java.util.stream.Collectors; * When we need to check original semantic of Having expression in sql, we need to check * here cause Having expression would be changed to Filter expression in analyze */ -public class CheckBound implements AnalysisRuleFactory { +public class CheckAfterBind implements AnalysisRuleFactory { @Override public List<Rule> buildRules() { return ImmutableList.of( - RuleType.CHECK_BOUND.build( - any().then(plan -> { - checkBound(plan); - return null; - }) - ), RuleType.CHECK_OBJECT_TYPE_ANALYSIS.build( logicalHaving().thenApply(ctx -> { LogicalHaving<Plan> having = ctx.root; @@ -64,28 +53,6 @@ public class CheckBound implements AnalysisRuleFactory { ); } - private void checkBound(Plan plan) { - Set<Unbound> unbounds = plan.getExpressions().stream() - .<Set<Unbound>>map(e -> e.collect(Unbound.class::isInstance)) - .flatMap(Set::stream) - .collect(Collectors.toSet()); - if (!unbounds.isEmpty()) { - throw new AnalysisException(String.format("unbounded object %s in %s clause.", - StringUtils.join(unbounds.stream() - .map(unbound -> { - if (unbound instanceof UnboundSlot) { - return ((UnboundSlot) unbound).toSql(); - } else if (unbound instanceof UnboundFunction) { - return ((UnboundFunction) unbound).toSql(); - } - return unbound.toString(); - }) - .collect(Collectors.toSet()), ", "), - plan.getType().toString().substring("LOGICAL_".length()) - )); - } - } - private void checkHavingObjectTypeExpression(LogicalHaving<Plan> having) { Set<Expression> havingConjuncts = having.getConjuncts(); for (Expression predicate : havingConjuncts) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindSlotReferenceTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindSlotReferenceTest.java index dc05ec0626..1e03874cf5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindSlotReferenceTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindSlotReferenceTest.java @@ -49,11 +49,11 @@ class BindSlotReferenceTest { @Test public void testCannotFindSlot() { - LogicalProject project = new LogicalProject<>(ImmutableList.of(new UnboundSlot("foo")), + LogicalProject<?> project = new LogicalProject<>(ImmutableList.of(new UnboundSlot("foo")), new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student)); AnalysisException exception = Assertions.assertThrows(AnalysisException.class, () -> PlanChecker.from(MemoTestUtils.createConnectContext()).analyze(project)); - Assertions.assertEquals("unbounded object foo in PROJECT clause.", exception.getMessage()); + Assertions.assertEquals("Unknown column 'foo' in 'table list' in PROJECT clause", exception.getMessage()); } @Test @@ -79,22 +79,22 @@ class BindSlotReferenceTest { @Test public void testGroupByOnJoin() { LogicalOlapScan scan1 = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); - LogicalSubQueryAlias sub1 = new LogicalSubQueryAlias("t1", scan1); + LogicalSubQueryAlias<LogicalOlapScan> sub1 = new LogicalSubQueryAlias<>("t1", scan1); LogicalOlapScan scan2 = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); - LogicalSubQueryAlias sub2 = new LogicalSubQueryAlias("t2", scan2); + LogicalSubQueryAlias<LogicalOlapScan> sub2 = new LogicalSubQueryAlias<>("t2", scan2); LogicalJoin<LogicalSubQueryAlias<LogicalOlapScan>, LogicalSubQueryAlias<LogicalOlapScan>> join = new LogicalJoin<>(JoinType.CROSS_JOIN, sub1, sub2); - LogicalAggregate<LogicalJoin> aggregate = new LogicalAggregate<>( + LogicalAggregate<?> aggregate = new LogicalAggregate<>( Lists.newArrayList(new UnboundSlot("id")), //group by Lists.newArrayList(new UnboundSlot("t1", "id")), //output join ); PlanChecker checker = PlanChecker.from(MemoTestUtils.createConnectContext()).analyze(aggregate); - LogicalAggregate plan = (LogicalAggregate) ((LogicalProject) checker.getCascadesContext() + LogicalAggregate<?> plan = (LogicalAggregate<?>) ((LogicalProject<?>) checker.getCascadesContext() .getMemo().copyOut()).child(); SlotReference groupByKey = (SlotReference) plan.getGroupByExpressions().get(0); - SlotReference t1id = (SlotReference) ((LogicalJoin) plan.child().child(0)).left().getOutput().get(0); - SlotReference t2id = (SlotReference) ((LogicalJoin) plan.child().child(0)).right().getOutput().get(0); + SlotReference t1id = (SlotReference) ((LogicalJoin<?, ?>) plan.child().child(0)).left().getOutput().get(0); + SlotReference t2id = (SlotReference) ((LogicalJoin<?, ?>) plan.child().child(0)).right().getOutput().get(0); Assertions.assertEquals(groupByKey.getExprId(), t1id.getExprId()); Assertions.assertNotEquals(t1id.getExprId(), t2id.getExprId()); } @@ -106,12 +106,12 @@ class BindSlotReferenceTest { @Test public void testGroupByOnJoinAmbiguous() { LogicalOlapScan scan1 = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); - LogicalSubQueryAlias sub1 = new LogicalSubQueryAlias("t1", scan1); + LogicalSubQueryAlias<LogicalOlapScan> sub1 = new LogicalSubQueryAlias<>("t1", scan1); LogicalOlapScan scan2 = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); - LogicalSubQueryAlias sub2 = new LogicalSubQueryAlias("t2", scan2); + LogicalSubQueryAlias<LogicalOlapScan> sub2 = new LogicalSubQueryAlias<>("t2", scan2); LogicalJoin<LogicalSubQueryAlias<LogicalOlapScan>, LogicalSubQueryAlias<LogicalOlapScan>> join = new LogicalJoin<>(JoinType.CROSS_JOIN, sub1, sub2); - LogicalAggregate<LogicalJoin> aggregate = new LogicalAggregate<>( + LogicalAggregate<LogicalJoin<?, ?>> aggregate = new LogicalAggregate<>( Lists.newArrayList(new UnboundSlot("id")), //group by Lists.newArrayList(new Alias(new Count(new IntegerLiteral(1)), "count(1)")), //output join diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckAnalysisTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckAnalysisTest.java index f470c0aa1f..9551fa771f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckAnalysisTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckAnalysisTest.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.rules.analysis; import org.apache.doris.nereids.CascadesContext; -import org.apache.doris.nereids.analyzer.UnboundFunction; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.And; @@ -33,7 +32,6 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; import mockit.Mocked; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -60,15 +58,4 @@ public class CheckAnalysisTest { Assertions.assertThrows(AnalysisException.class, () -> checkAnalysis.buildRules().forEach(rule -> rule.transform(plan, cascadesContext))); } - - @Test - public void testUnbound() { - UnboundFunction func = new UnboundFunction("now", Lists.newArrayList(new IntegerLiteral(1))); - Plan plan = new LogicalOneRowRelation(StatementScopeIdGenerator.newRelationId(), - ImmutableList.of(new Alias(func, "unboundFunction"))); - CheckBound checkBound = new CheckBound(); - Assertions.assertThrows(AnalysisException.class, () -> - checkBound.buildRules().forEach(rule -> rule.transform(plan, cascadesContext))); - } - } diff --git a/regression-test/suites/nereids_p0/except/test_bound_exception.groovy b/regression-test/suites/nereids_p0/except/test_bound_exception.groovy index 02b29b6517..b4c3fd82dc 100644 --- a/regression-test/suites/nereids_p0/except/test_bound_exception.groovy +++ b/regression-test/suites/nereids_p0/except/test_bound_exception.groovy @@ -19,9 +19,6 @@ suite("test_bound_exception") { sql "SET enable_nereids_planner=true" sql "SET enable_fallback_to_original_planner=false" def tbName = "test_bound_exception" - def dbName = "test_bound_db" - sql "CREATE DATABASE IF NOT EXISTS ${dbName}" - sql "USE ${dbName}" sql """ DROP TABLE IF EXISTS ${tbName} """ sql """ @@ -30,26 +27,26 @@ suite("test_bound_exception") { """ test { sql "SELECT id FROM ${tbName} GROUP BY id ORDER BY id123" - exception "errCode = 2, detailMessage = Unexpected exception: unbounded object id123 in SORT clause." + exception "Unknown column 'id123' in 'table list' in SORT clause" } test { sql "SELECT id123 FROM ${tbName} ORDER BY id" - exception "errCode = 2, detailMessage = Unexpected exception: unbounded object id123 in PROJECT clause." + exception "Unknown column 'id123' in 'table list' in PROJECT clause" } test { sql "SELECT id123 FROM ${tbName} GROUP BY id ORDER BY id" - exception "errCode = 2, detailMessage = Unexpected exception: unbounded object id123 in AGGREGATE clause." + exception "Unknown column 'id123' in 'table list' in AGGREGATE clause" } test { sql "SELECT id FROM ${tbName} GROUP BY id123 ORDER BY id" - exception "errCode = 2, detailMessage = Unexpected exception: cannot bind GROUP BY KEY: id123" + exception "Unknown column 'id123' in 'table list' in AGGREGATE clause" } test { sql "SELECT id FROM ${tbName} WHERE id = (SELECT id from ${tbName} ORDER BY id123 LIMIT 1) ORDER BY id" - exception "errCode = 2, detailMessage = Unexpected exception: unbounded object id123 in SORT clause." + exception "Unknown column 'id123' in 'table list' in SORT clause" } test { sql "SELECT id FROM ${tbName} WHERE id123 = 123 ORDER BY id" - exception "errCode = 2, detailMessage = Unexpected exception: Invalid call to id123.getDataType() on unbound object" + exception "Unknown column 'id123' in 'table list' in FILTER clause" } } diff --git a/regression-test/suites/nereids_syntax_p0/bind_priority.groovy b/regression-test/suites/nereids_syntax_p0/bind_priority.groovy index 01a08892cd..19b630854c 100644 --- a/regression-test/suites/nereids_syntax_p0/bind_priority.groovy +++ b/regression-test/suites/nereids_syntax_p0/bind_priority.groovy @@ -53,7 +53,7 @@ suite("bind_priority") { sql """ select sum(a) as v from bind_priority_tbl group by v; """ - exception "Unexpected exception: cannot bind GROUP BY KEY: v" + exception "Unknown column 'v' in 'table list' in AGGREGATE clause" } sql "drop table if exists bind_priority_tbl" --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org