You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2023/01/10 18:03:20 UTC

[doris] branch master updated: [fix](nereids) bind sort key priority problem (#15646)

This is an automated email from the ASF dual-hosted git repository.

morrysnow pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 280603b253 [fix](nereids) bind sort key priority problem (#15646)
280603b253 is described below

commit 280603b253b8dcc1ba15e730513f825d0b388400
Author: minghong <en...@gmail.com>
AuthorDate: Wed Jan 11 02:03:09 2023 +0800

    [fix](nereids) bind sort key priority problem (#15646)
    
    `a.b.c` should only bind on `a.b.c`, not on `b.c` or `c`
---
 .../doris/nereids/rules/analysis/BindRelation.java |  17 ++--
 .../nereids/rules/analysis/BindSlotReference.java  | 113 ++++++++++-----------
 .../trees/plans/logical/LogicalSubQueryAlias.java  |  37 ++++---
 .../data/nereids_syntax_p0/bind_priority.out       |  15 +++
 .../suites/nereids_syntax_p0/bind_priority.groovy  |  31 +++++-
 5 files changed, 131 insertions(+), 82 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java
index bfc9979b7f..2f4269d055 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java
@@ -43,6 +43,7 @@ import org.apache.doris.nereids.trees.plans.logical.RelationUtil;
 import org.apache.doris.qe.ConnectContext;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 import org.apache.commons.collections.CollectionUtils;
 
 import java.util.Collections;
@@ -107,13 +108,14 @@ public class BindRelation extends OneAnalysisRuleFactory {
                     && ((LogicalSubQueryAlias<?>) ctePlan).getAlias().equals(tableName)) {
                 return ctePlan;
             }
-            return new LogicalSubQueryAlias<>(tableName, ctePlan);
+            return new LogicalSubQueryAlias<>(unboundRelation.getNameParts(), ctePlan);
         }
         String catalogName = cascadesContext.getConnectContext().getCurrentCatalog().getName();
         String dbName = cascadesContext.getConnectContext().getDatabase();
         TableIf table = getTable(catalogName, dbName, tableName, cascadesContext.getConnectContext().getEnv());
         // TODO: should generate different Scan sub class according to table's type
-        return getLogicalPlan(table, unboundRelation, dbName, cascadesContext);
+        List<String> tableQualifier = Lists.newArrayList(catalogName, dbName, tableName);
+        return getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext);
     }
 
     private LogicalPlan bindWithDbNameFromNamePart(CascadesContext cascadesContext, UnboundRelation unboundRelation) {
@@ -127,7 +129,8 @@ public class BindRelation extends OneAnalysisRuleFactory {
         }
         String tableName = nameParts.get(1);
         TableIf table = getTable(catalogName, dbName, tableName, connectContext.getEnv());
-        return getLogicalPlan(table, unboundRelation, dbName, cascadesContext);
+        List<String> tableQualifier = Lists.newArrayList(catalogName, dbName, tableName);
+        return getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext);
     }
 
     private LogicalPlan bindWithCatalogNameFromNamePart(CascadesContext cascadesContext,
@@ -141,11 +144,13 @@ public class BindRelation extends OneAnalysisRuleFactory {
         }
         String tableName = nameParts.get(2);
         TableIf table = getTable(catalogName, dbName, tableName, connectContext.getEnv());
-        return getLogicalPlan(table, unboundRelation, dbName, cascadesContext);
+        List<String> tableQualifier = Lists.newArrayList(catalogName, dbName, tableName);
+        return getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext);
     }
 
-    private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelation, String dbName,
+    private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelation, List<String> tableQualifier,
                                        CascadesContext cascadesContext) {
+        String dbName = tableQualifier.get(1); //[catalogName, dbName, tableName]
         switch (table.getType()) {
             case OLAP:
                 List<Long> partIds = getPartitionIds(table, unboundRelation);
@@ -158,7 +163,7 @@ public class BindRelation extends OneAnalysisRuleFactory {
                 }
             case VIEW:
                 Plan viewPlan = parseAndAnalyzeView(((View) table).getDdlSql(), cascadesContext);
-                return new LogicalSubQueryAlias<>(table.getName(), viewPlan);
+                return new LogicalSubQueryAlias<>(tableQualifier, viewPlan);
             case HMS_EXTERNAL_TABLE:
                 return new LogicalFileScan(cascadesContext.getStatementContext().getNextRelationId(),
                     (HMSExternalTable) table, ImmutableList.of(dbName));
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotReference.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotReference.java
index 7ff04b6c24..bc0c9325ae 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotReference.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotReference.java
@@ -152,7 +152,7 @@ public class BindSlotReference implements AnalysisRuleFactory {
                                 .peek(s -> slotNames.add(s.getName())).collect(
                                         Collectors.toList()));
                         for (Expression unboundSlot : unboundSlots) {
-                            Expression expression = new SlotBinder(scope, lj, ctx.cascadesContext).bind(unboundSlot);
+                            Expression expression = new SlotBinder(scope, ctx.cascadesContext).bind(unboundSlot);
                             leftSlots.add(expression);
                         }
                         slotNames.clear();
@@ -162,7 +162,7 @@ public class BindSlotReference implements AnalysisRuleFactory {
                                         Collectors.toList()));
                         List<Expression> rightSlots = new ArrayList<>();
                         for (Expression unboundSlot : unboundSlots) {
-                            Expression expression = new SlotBinder(scope, lj, ctx.cascadesContext).bind(unboundSlot);
+                            Expression expression = new SlotBinder(scope, ctx.cascadesContext).bind(unboundSlot);
                             rightSlots.add(expression);
                         }
                         int size = leftSlots.size();
@@ -178,10 +178,10 @@ public class BindSlotReference implements AnalysisRuleFactory {
                                 .whenNot(j -> j.getJoinType().equals(JoinType.USING_JOIN)).thenApply(ctx -> {
                                     LogicalJoin<GroupPlan, GroupPlan> join = ctx.root;
                                     List<Expression> cond = join.getOtherJoinConjuncts().stream()
-                                            .map(expr -> bind(expr, join.children(), join, ctx.cascadesContext))
+                                            .map(expr -> bind(expr, join.children(), ctx.cascadesContext))
                                             .collect(Collectors.toList());
                                     List<Expression> hashJoinConjuncts = join.getHashJoinConjuncts().stream()
-                                            .map(expr -> bind(expr, join.children(), join, ctx.cascadesContext))
+                                            .map(expr -> bind(expr, join.children(), ctx.cascadesContext))
                                             .collect(Collectors.toList());
                                     return new LogicalJoin<>(join.getJoinType(),
                                             hashJoinConjuncts, cond, join.getHint(), join.left(), join.right());
@@ -341,9 +341,9 @@ public class BindSlotReference implements AnalysisRuleFactory {
                     List<OrderKey> sortItemList = sort.getOrderKeys()
                             .stream()
                             .map(orderKey -> {
-                                Expression item = bind(orderKey.getExpr(), sort.children(), sort, ctx.cascadesContext);
+                                Expression item = bind(orderKey.getExpr(), sort.children(), ctx.cascadesContext);
                                 if (item.containsType(UnboundSlot.class)) {
-                                    item = bind(item, sort.child().children(), sort, ctx.cascadesContext);
+                                    item = bind(item, sort.child().children(), ctx.cascadesContext);
                                 }
                                 return new OrderKey(item, orderKey.isAsc(), orderKey.isNullFirst());
                             }).collect(Collectors.toList());
@@ -354,13 +354,20 @@ public class BindSlotReference implements AnalysisRuleFactory {
             RuleType.BINDING_SORT_SLOT.build(
                 logicalSort(logicalProject()).when(Plan::canBind).thenApply(ctx -> {
                     LogicalSort<LogicalProject<GroupPlan>> sort = ctx.root;
+                    Set<Slot> projectOutput = sort.child().getOutputSet();
+                    SlotBinder binderOnProject = new SlotBinder(toScope(Lists.newArrayList(projectOutput)),
+                            ctx.cascadesContext);
+                    Set<Slot> projectChildrenOutput = sort.child().children().stream()
+                            .flatMap(plan -> plan.getOutputSet().stream())
+                            .collect(Collectors.toSet());
+                    SlotBinder binderOnProjectChild = new SlotBinder(
+                            toScope(Lists.newArrayList(projectChildrenOutput)),
+                            ctx.cascadesContext);
                     List<OrderKey> sortItemList = sort.getOrderKeys()
                             .stream()
                             .map(orderKey -> {
-                                Expression item = bind(orderKey.getExpr(), sort.children(), sort, ctx.cascadesContext);
-                                if (item.containsType(UnboundSlot.class)) {
-                                    item = bind(item, sort.child().children(), sort, ctx.cascadesContext);
-                                }
+                                Expression item = binderOnProject.bind(orderKey.getExpr());
+                                item = binderOnProjectChild.bind(item);
                                 return new OrderKey(item, orderKey.isAsc(), orderKey.isNullFirst());
                             }).collect(Collectors.toList());
 
@@ -373,9 +380,9 @@ public class BindSlotReference implements AnalysisRuleFactory {
                     List<OrderKey> sortItemList = sort.getOrderKeys()
                             .stream()
                             .map(orderKey -> {
-                                Expression item = bind(orderKey.getExpr(), sort.children(), sort, ctx.cascadesContext);
+                                Expression item = bind(orderKey.getExpr(), sort.children(), ctx.cascadesContext);
                                 if (item.containsType(UnboundSlot.class)) {
-                                    item = bind(item, sort.child().children(), sort, ctx.cascadesContext);
+                                    item = bind(item, sort.child().children(), ctx.cascadesContext);
                                 }
                                 return new OrderKey(item, orderKey.isAsc(), orderKey.isNullFirst());
                             }).collect(Collectors.toList());
@@ -392,11 +399,11 @@ public class BindSlotReference implements AnalysisRuleFactory {
                     List<Slot> childChildSlots = childPlan.children().stream()
                             .flatMap(plan -> plan.getOutputSet().stream())
                             .collect(Collectors.toList());
-                    SlotBinder childChildBinder = new SlotBinder(toScope(childChildSlots), having,
+                    SlotBinder childChildBinder = new SlotBinder(toScope(childChildSlots),
                             ctx.cascadesContext);
                     List<Slot> childSlots = childPlan.getOutputSet().stream()
                             .collect(Collectors.toList());
-                    SlotBinder childBinder = new SlotBinder(toScope(childSlots), having,
+                    SlotBinder childBinder = new SlotBinder(toScope(childSlots),
                             ctx.cascadesContext);
                     Set<Expression> boundConjuncts = having.getConjuncts().stream().map(
                             expr -> {
@@ -413,7 +420,7 @@ public class BindSlotReference implements AnalysisRuleFactory {
                     UnboundOneRowRelation oneRowRelation = ctx.root;
                     List<NamedExpression> projects = oneRowRelation.getProjects()
                             .stream()
-                            .map(project -> bind(project, ImmutableList.of(), oneRowRelation, ctx.cascadesContext))
+                            .map(project -> bind(project, ImmutableList.of(), ctx.cascadesContext))
                             .collect(Collectors.toList());
                     return new LogicalOneRowRelation(projects);
                 })
@@ -492,9 +499,9 @@ public class BindSlotReference implements AnalysisRuleFactory {
         //        order by col1;     # order by order_col1
         //    bind order_col1 with alias_col1, then, bind it with inner_col1
         SlotBinder outputBinder = new SlotBinder(
-                toScope(aggregate.getOutputSet().stream().collect(Collectors.toList())), sort, ctx);
+                toScope(aggregate.getOutputSet().stream().collect(Collectors.toList())), ctx);
         List<Slot> childOutputSlots = aggregate.child().getOutputSet().stream().collect(Collectors.toList());
-        SlotBinder childOutputBinder = new SlotBinder(toScope(childOutputSlots), sort, ctx);
+        SlotBinder childOutputBinder = new SlotBinder(toScope(childOutputSlots), ctx);
         List<OrderKey> sortItemList = sort.getOrderKeys()
                 .stream()
                 .map(orderKey -> {
@@ -520,30 +527,28 @@ public class BindSlotReference implements AnalysisRuleFactory {
     private <E extends Expression> List<E> bind(List<E> exprList, List<Plan> inputs, Plan plan,
             CascadesContext cascadesContext) {
         return exprList.stream()
-            .map(expr -> bind(expr, inputs, plan, cascadesContext))
+            .map(expr -> bind(expr, inputs, cascadesContext))
             .collect(Collectors.toList());
     }
 
     private <E extends Expression> Set<E> bind(Set<E> exprList, List<Plan> inputs, Plan plan,
             CascadesContext cascadesContext) {
         return exprList.stream()
-                .map(expr -> bind(expr, inputs, plan, cascadesContext))
+                .map(expr -> bind(expr, inputs, cascadesContext))
                 .collect(Collectors.toSet());
     }
 
-    private <E extends Expression> E bind(E expr, List<Plan> inputs, Plan plan, CascadesContext cascadesContext) {
+    private <E extends Expression> E bind(E expr, List<Plan> inputs, CascadesContext cascadesContext) {
         List<Slot> boundedSlots = inputs.stream()
                 .flatMap(input -> input.getOutput().stream())
                 .collect(Collectors.toList());
-        return (E) new SlotBinder(toScope(boundedSlots), plan, cascadesContext).bind(expr);
+        return (E) new SlotBinder(toScope(boundedSlots), cascadesContext).bind(expr);
     }
 
     private class SlotBinder extends SubExprAnalyzer {
-        private final Plan plan;
 
-        public SlotBinder(Scope scope, Plan plan, CascadesContext cascadesContext) {
+        public SlotBinder(Scope scope, CascadesContext cascadesContext) {
             super(scope, cascadesContext);
-            this.plan = plan;
         }
 
         public Expression bind(Expression expression) {
@@ -572,9 +577,9 @@ public class BindSlotReference implements AnalysisRuleFactory {
             if (!foundInThisScope && getScope().getOuterScope().isPresent()) {
                 boundedOpt = Optional.of(bindSlot(unboundSlot,
                         getScope()
-                        .getOuterScope()
-                        .get()
-                        .getSlots()));
+                                .getOuterScope()
+                                .get()
+                                .getSlots()));
             }
             List<Slot> bounded = boundedOpt.get();
             switch (bounded.size()) {
@@ -607,7 +612,7 @@ public class BindSlotReference implements AnalysisRuleFactory {
                     return bindQualifiedStar(qualifier);
                 default:
                     throw new AnalysisException("Not supported qualifier: "
-                        + StringUtils.join(qualifier, "."));
+                            + StringUtils.join(qualifier, "."));
             }
         }
 
@@ -621,14 +626,15 @@ public class BindSlotReference implements AnalysisRuleFactory {
                         List<String> boundSlotQualifier = boundSlot.getQualifier();
                         switch (boundSlotQualifier.size()) {
                             // bound slot is `column` and no qualified
-                            case 0: return false;
+                            case 0:
+                                return false;
                             case 1: // bound slot is `table`.`column`
                                 return qualifierStar.get(0).equalsIgnoreCase(boundSlotQualifier.get(0));
                             case 2:// bound slot is `db`.`table`.`column`
                                 return qualifierStar.get(0).equalsIgnoreCase(boundSlotQualifier.get(1));
                             default:
                                 throw new AnalysisException("Not supported qualifier: "
-                                    + StringUtils.join(qualifierStar, "."));
+                                        + StringUtils.join(qualifierStar, "."));
                         }
                     case 2: // db.table.*
                         boundSlotQualifier = boundSlot.getQualifier();
@@ -642,11 +648,11 @@ public class BindSlotReference implements AnalysisRuleFactory {
                                         && qualifierStar.get(1).equalsIgnoreCase(boundSlotQualifier.get(1));
                             default:
                                 throw new AnalysisException("Not supported qualifier: "
-                                    + StringUtils.join(qualifierStar, ".") + ".*");
+                                        + StringUtils.join(qualifierStar, ".") + ".*");
                         }
                     default:
                         throw new AnalysisException("Not supported name: "
-                            + StringUtils.join(qualifierStar, ".") + ".*");
+                                + StringUtils.join(qualifierStar, ".") + ".*");
                 }
             }).collect(Collectors.toList());
 
@@ -656,12 +662,24 @@ public class BindSlotReference implements AnalysisRuleFactory {
         private List<Slot> bindSlot(UnboundSlot unboundSlot, List<Slot> boundSlots) {
             return boundSlots.stream().filter(boundSlot -> {
                 List<String> nameParts = unboundSlot.getNameParts();
-                if (nameParts.size() == 1) {
+                int qualifierSize = boundSlot.getQualifier().size();
+                int namePartsSize = nameParts.size();
+                if (namePartsSize > qualifierSize + 1) {
+                    return false;
+                }
+                if (namePartsSize == 1) {
                     return nameParts.get(0).equalsIgnoreCase(boundSlot.getName());
-                } else if (nameParts.size() <= 3) {
-                    int size = nameParts.size();
-                    // if nameParts.size() == 3, nameParts.get(0) is cluster name.
-                    return handleNamePartsTwoOrThree(boundSlot, nameParts.subList(size - 2, size));
+                }
+                if (namePartsSize == 2) {
+                    String qualifierDbName = boundSlot.getQualifier().get(qualifierSize - 1);
+                    return qualifierDbName.equalsIgnoreCase(nameParts.get(0))
+                            && boundSlot.getName().equalsIgnoreCase(nameParts.get(1));
+                } else if (nameParts.size() == 3) {
+                    String qualifierDbName = boundSlot.getQualifier().get(qualifierSize - 1);
+                    String qualifierClusterName = boundSlot.getQualifier().get(qualifierSize - 2);
+                    return qualifierClusterName.equalsIgnoreCase(nameParts.get(0))
+                            && qualifierDbName.equalsIgnoreCase(nameParts.get(1))
+                            && boundSlot.getName().equalsIgnoreCase(nameParts.get(2));
                 }
                 //TODO: handle name parts more than three.
                 throw new AnalysisException("Not supported name: "
@@ -670,27 +688,6 @@ public class BindSlotReference implements AnalysisRuleFactory {
         }
     }
 
-    private boolean handleNamePartsTwoOrThree(Slot boundSlot, List<String> nameParts) {
-        List<String> qualifier = boundSlot.getQualifier();
-        String name = boundSlot.getName();
-        switch (qualifier.size()) {
-            case 2:
-                // qualifier is `db`.`table`
-                return nameParts.get(0).equalsIgnoreCase(qualifier.get(1))
-                        && nameParts.get(1).equalsIgnoreCase(name);
-            case 1:
-                // qualifier is `table`
-                return nameParts.get(0).equalsIgnoreCase(qualifier.get(0))
-                        && nameParts.get(1).equalsIgnoreCase(name);
-            case 0:
-                // has no qualifiers
-                return nameParts.get(1).equalsIgnoreCase(name);
-            default:
-                throw new AnalysisException("Not supported qualifier: "
-                        + StringUtils.join(qualifier, "."));
-        }
-    }
-
     /** BoundStar is used to wrap list of slots for temporary. */
     public static class BoundStar extends NamedExpression implements PropagateNullable {
         public BoundStar(List<Slot> children) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java
index a02467a52f..30b4fc5df5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java
@@ -40,23 +40,31 @@ import java.util.Optional;
  * @param <CHILD_TYPE> param
  */
 public class LogicalSubQueryAlias<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> {
-    private final String alias;
+    private final List<String> qualifier;
 
     private final Optional<List<String>> columnAliases;
 
     public LogicalSubQueryAlias(String tableAlias, CHILD_TYPE child) {
-        this(tableAlias, Optional.empty(), Optional.empty(), Optional.empty(), child);
+        this(ImmutableList.of(tableAlias), Optional.empty(), Optional.empty(), Optional.empty(), child);
+    }
+
+    public LogicalSubQueryAlias(List<String> qualifier, CHILD_TYPE child) {
+        this(qualifier, Optional.empty(), Optional.empty(), Optional.empty(), child);
     }
 
     public LogicalSubQueryAlias(String tableAlias, Optional<List<String>> columnAliases, CHILD_TYPE child) {
-        this(tableAlias, columnAliases, Optional.empty(), Optional.empty(), child);
+        this(ImmutableList.of(tableAlias), columnAliases, Optional.empty(), Optional.empty(), child);
+    }
+
+    public LogicalSubQueryAlias(List<String> qualifier, Optional<List<String>> columnAliases, CHILD_TYPE child) {
+        this(qualifier, columnAliases, Optional.empty(), Optional.empty(), child);
     }
 
-    public LogicalSubQueryAlias(String tableAlias, Optional<List<String>> columnAliases,
+    public LogicalSubQueryAlias(List<String> qualifier, Optional<List<String>> columnAliases,
                                 Optional<GroupExpression> groupExpression,
                                 Optional<LogicalProperties> logicalProperties, CHILD_TYPE child) {
         super(PlanType.LOGICAL_SUBQUERY_ALIAS, groupExpression, logicalProperties, child);
-        this.alias = tableAlias;
+        this.qualifier = ImmutableList.copyOf(Objects.requireNonNull(qualifier));
         this.columnAliases = columnAliases;
     }
 
@@ -67,7 +75,6 @@ public class LogicalSubQueryAlias<CHILD_TYPE extends Plan> extends LogicalUnary<
                 ? this.columnAliases.get()
                 : ImmutableList.of();
         ImmutableList.Builder<Slot> currentOutput = ImmutableList.builder();
-        String qualifier = alias;
         for (int i = 0; i < childOutput.size(); i++) {
             Slot originSlot = childOutput.get(i);
             String columnAlias;
@@ -77,7 +84,7 @@ public class LogicalSubQueryAlias<CHILD_TYPE extends Plan> extends LogicalUnary<
                 columnAlias = originSlot.getName();
             }
             Slot qualified = originSlot
-                    .withQualifier(ImmutableList.of(qualifier))
+                    .withQualifier(qualifier)
                     .withName(columnAlias);
             currentOutput.add(qualified);
         }
@@ -85,7 +92,7 @@ public class LogicalSubQueryAlias<CHILD_TYPE extends Plan> extends LogicalUnary<
     }
 
     public String getAlias() {
-        return alias;
+        return qualifier.get(qualifier.size() - 1);
     }
 
     public Optional<List<String>> getColumnAliases() {
@@ -96,12 +103,12 @@ public class LogicalSubQueryAlias<CHILD_TYPE extends Plan> extends LogicalUnary<
     public String toString() {
         if (columnAliases.isPresent()) {
             return Utils.toSqlString("LogicalSubQueryAlias",
-                "alias", alias,
+                "qualifier", qualifier,
                 "columnAliases", StringUtils.join(columnAliases.get(), ",")
             );
         }
         return Utils.toSqlString("LogicalSubQueryAlias",
-                "alias", alias
+                "qualifier", qualifier
         );
     }
 
@@ -114,18 +121,18 @@ public class LogicalSubQueryAlias<CHILD_TYPE extends Plan> extends LogicalUnary<
             return false;
         }
         LogicalSubQueryAlias that = (LogicalSubQueryAlias) o;
-        return alias.equals(that.alias);
+        return qualifier.equals(that.qualifier);
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(alias);
+        return Objects.hash(qualifier);
     }
 
     @Override
     public LogicalSubQueryAlias<Plan> withChildren(List<Plan> children) {
         Preconditions.checkArgument(children.size() == 1);
-        return new LogicalSubQueryAlias<>(alias, columnAliases, children.get(0));
+        return new LogicalSubQueryAlias<>(qualifier, columnAliases, children.get(0));
     }
 
     @Override
@@ -140,13 +147,13 @@ public class LogicalSubQueryAlias<CHILD_TYPE extends Plan> extends LogicalUnary<
 
     @Override
     public LogicalSubQueryAlias<CHILD_TYPE> withGroupExpression(Optional<GroupExpression> groupExpression) {
-        return new LogicalSubQueryAlias<>(alias, columnAliases, groupExpression,
+        return new LogicalSubQueryAlias<>(qualifier, columnAliases, groupExpression,
                 Optional.of(getLogicalProperties()), child());
     }
 
     @Override
     public LogicalSubQueryAlias<CHILD_TYPE> withLogicalProperties(Optional<LogicalProperties> logicalProperties) {
-        return new LogicalSubQueryAlias<>(alias, columnAliases, Optional.empty(),
+        return new LogicalSubQueryAlias<>(qualifier, columnAliases, Optional.empty(),
                 logicalProperties, child());
     }
 }
diff --git a/regression-test/data/nereids_syntax_p0/bind_priority.out b/regression-test/data/nereids_syntax_p0/bind_priority.out
index 90d639c228..5b20034991 100644
--- a/regression-test/data/nereids_syntax_p0/bind_priority.out
+++ b/regression-test/data/nereids_syntax_p0/bind_priority.out
@@ -11,3 +11,18 @@ all	2
 -- !select --
 0
 
+-- !bind_sort_scan --
+3
+1
+2
+
+-- !bind_sort_alias --
+1
+2
+3
+
+-- !bind_sort_aliasAndScan --
+1
+3
+2
+
diff --git a/regression-test/suites/nereids_syntax_p0/bind_priority.groovy b/regression-test/suites/nereids_syntax_p0/bind_priority.groovy
index 89e878e89b..00e6e91e99 100644
--- a/regression-test/suites/nereids_syntax_p0/bind_priority.groovy
+++ b/regression-test/suites/nereids_syntax_p0/bind_priority.groovy
@@ -16,8 +16,6 @@
 // under the License.
 
 suite("bind_priority") {
-    sql "SET enable_nereids_planner=true"
-
     sql """
         DROP TABLE IF EXISTS bind_priority_tbl
        """
@@ -33,7 +31,8 @@ suite("bind_priority") {
     sql """
     insert into bind_priority_tbl values(1, 2),(3, 4)
     """
-
+    
+    sql "SET enable_nereids_planner=true"
     sql "SET enable_fallback_to_original_planner=false"
 
     sql """sync"""
@@ -56,4 +55,30 @@ suite("bind_priority") {
             """
         exception "Unexpected exception: cannot bind GROUP BY KEY: v"
     }
+
+    sql "drop table if exists t"
+    sql """
+        CREATE TABLE `t` (
+            `a` int(11) NOT NULL,
+            `b` int(11) NOT NULL
+        ) ENGINE=OLAP
+        DUPLICATE KEY(`a`)
+        COMMENT 'OLAP'
+        DISTRIBUTED BY HASH(`a`) BUCKETS 1
+        PROPERTIES (
+        "replication_allocation" = "tag.location.default: 1",
+        "in_memory" = "false",
+        "storage_format" = "V2",
+        "light_schema_change" = "true",
+        "disable_auto_compaction" = "false"
+        );
+    """
+
+    sql"insert into t values(1,5),(2, 6),(3,4);"
+
+    qt_bind_sort_scan "select a as b from t order by t.b;"
+
+    qt_bind_sort_alias "select a as b from t order by b;"
+
+    qt_bind_sort_aliasAndScan "select a as b from t order by t.b + b;"
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org