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

Reply via email to