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

huajianlan 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 180cc35815 [Feature](nereids) support sub-query and alias in 
FromClause (#11035)
180cc35815 is described below

commit 180cc35815ed1023cbbd22d0c5a4b6016ef11495
Author: mch_ucchi <41606806+sohardforan...@users.noreply.github.com>
AuthorDate: Thu Aug 11 12:42:19 2022 +0800

    [Feature](nereids) support sub-query and alias in FromClause (#11035)
    
    Support sub-query and alias for TPC-H,for example:
    select * from (select * from (T1) A join T2 as B on T1.id = T2.id) T;
---
 .../org/apache/doris/nereids/NereidsPlanner.java   |   6 +
 .../glue/translator/PhysicalPlanTranslator.java    |   1 +
 .../batch/FinalizeAnalyzeJob.java}                 |  24 +--
 .../jobs/cascades/OptimizeGroupExpressionJob.java  |   1 -
 .../apache/doris/nereids/memo/GroupExpression.java |  10 +-
 .../doris/nereids/parser/LogicalPlanBuilder.java   |  58 ++++++-
 .../apache/doris/nereids/properties/OrderKey.java  |   2 +-
 .../org/apache/doris/nereids/rules/RuleType.java   |   9 ++
 .../doris/nereids/rules/analysis/BindRelation.java |  42 ++++-
 .../nereids/rules/analysis/BindSlotReference.java  |   4 +
 .../nereids/rules/analysis/EliminateAliasNode.java |  66 ++++++++
 .../trees/expressions/NamedExpressionUtil.java     |  18 +++
 .../doris/nereids/trees/expressions/Slot.java      |   6 +
 .../nereids/trees/expressions/SlotReference.java   |   5 +
 .../apache/doris/nereids/trees/plans/PlanType.java |   1 +
 .../trees/plans/logical/LogicalProject.java        |   1 +
 ...gicalProject.java => LogicalSubQueryAlias.java} |  84 +++++-----
 .../nereids/trees/plans/visitor/PlanVisitor.java   |   5 +
 .../doris/nereids/util/AnalyzeSubQueryTest.java    | 178 +++++++++++++++++++++
 .../apache/doris/nereids/util/FieldChecker.java    |  56 +++++++
 .../apache/doris/utframe/TestWithFeService.java    |   9 ++
 .../data/nereids_syntax_p0/sub_query_alias.out     |  26 +++
 .../nereids_syntax_p0/sub_query_alias.groovy       |  72 +++++++++
 23 files changed, 610 insertions(+), 74 deletions(-)

diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java
index 3a486c1898..173b68a69f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java
@@ -26,6 +26,7 @@ import org.apache.doris.nereids.glue.LogicalPlanAdapter;
 import org.apache.doris.nereids.glue.translator.PhysicalPlanTranslator;
 import org.apache.doris.nereids.glue.translator.PlanTranslatorContext;
 import org.apache.doris.nereids.jobs.batch.DisassembleRulesJob;
+import org.apache.doris.nereids.jobs.batch.FinalizeAnalyzeJob;
 import org.apache.doris.nereids.jobs.batch.JoinReorderRulesJob;
 import org.apache.doris.nereids.jobs.batch.NormalizeExpressionRulesJob;
 import org.apache.doris.nereids.jobs.batch.OptimizeRulesJob;
@@ -106,6 +107,7 @@ public class NereidsPlanner extends Planner {
                 // cascades style optimize phase.
                 .setJobContext(outputProperties);
 
+        finalizeAnalyze();
         rewrite();
         // TODO: remove this condition, when stats collector is fully 
developed.
         if (ConnectContext.get().getSessionVariable().isEnableNereidsCBO()) {
@@ -116,6 +118,10 @@ public class NereidsPlanner extends Planner {
         return getRoot().extractPlan();
     }
 
+    private void finalizeAnalyze() {
+        new FinalizeAnalyzeJob(plannerContext).execute();
+    }
+
     /**
      * Logical plan rewrite based on a series of heuristic rules.
      */
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java
index 9270d51a41..1799c70123 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java
@@ -402,6 +402,7 @@ public class PhysicalPlanTranslator extends 
DefaultPlanVisitor<PlanFragment, Pla
                 .stream()
                 .map(e -> ExpressionTranslator.translate(e, context))
                 .collect(Collectors.toList());
+        // TODO: fix the project alias of an aliased relation.
         PlanNode inputPlanNode = inputFragment.getPlanRoot();
         List<Expr> predicateList = inputPlanNode.getConjuncts();
         Set<Integer> requiredSlotIdList = new HashSet<>();
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpressionUtil.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/FinalizeAnalyzeJob.java
similarity index 58%
copy from 
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpressionUtil.java
copy to 
fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/FinalizeAnalyzeJob.java
index bbbc839ea3..ef0a8e5da4 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpressionUtil.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/FinalizeAnalyzeJob.java
@@ -15,20 +15,26 @@
 // specific language governing permissions and limitations
 // under the License.
 
-package org.apache.doris.nereids.trees.expressions;
+package org.apache.doris.nereids.jobs.batch;
 
-import org.apache.doris.common.IdGenerator;
+import org.apache.doris.nereids.PlannerContext;
+import org.apache.doris.nereids.rules.analysis.EliminateAliasNode;
+
+import com.google.common.collect.ImmutableList;
 
 /**
- * The util of named expression.
+ * Job to eliminate the logical node of sub query and alias
  */
-public class NamedExpressionUtil {
+public class FinalizeAnalyzeJob extends BatchRulesJob {
+
     /**
-     * Tool class for generate next ExprId.
+     * constructor
+     * @param plannerContext ctx
      */
-    private static final IdGenerator<ExprId> ID_GENERATOR = 
ExprId.createGenerator();
-
-    public static ExprId newExprId() {
-        return ID_GENERATOR.getNextId();
+    public FinalizeAnalyzeJob(PlannerContext plannerContext) {
+        super(plannerContext);
+        rulesJob.addAll(ImmutableList.of(
+                bottomUpBatch(ImmutableList.of(new EliminateAliasNode()))
+        ));
     }
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java
index d43c8a7c68..4a2a876452 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java
@@ -63,6 +63,5 @@ public class OptimizeGroupExpressionJob extends Job {
                 }
             }
         }
-
     }
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java
index 44f8d0f86c..587051d618 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java
@@ -22,6 +22,8 @@ import org.apache.doris.nereids.properties.PhysicalProperties;
 import org.apache.doris.nereids.rules.Rule;
 import org.apache.doris.nereids.rules.RuleType;
 import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalRelation;
+import org.apache.doris.nereids.trees.plans.physical.PhysicalRelation;
 import org.apache.doris.statistics.StatsDeriveResult;
 
 import com.google.common.base.Preconditions;
@@ -168,7 +170,13 @@ public class GroupExpression {
             return false;
         }
         GroupExpression that = (GroupExpression) o;
-        return plan.equals(that.plan) && children.equals(that.children);
+        // if the plan is LogicalRelation or PhysicalRelation, this == that 
should be true,
+        // when if one relation appear in plan more than once,
+        // we cannot distinguish them throw equals function, since equals 
function cannot use output info.
+        if (plan instanceof LogicalRelation || plan instanceof 
PhysicalRelation) {
+            return false;
+        }
+        return children.equals(that.children) && plan.equals(that.plan);
     }
 
     @Override
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
index 2453759199..5ea7202629 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
@@ -21,6 +21,8 @@ package org.apache.doris.nereids.parser;
 import org.apache.doris.analysis.ArithmeticExpr.Operator;
 import org.apache.doris.nereids.DorisParser;
 import org.apache.doris.nereids.DorisParser.AggClauseContext;
+import org.apache.doris.nereids.DorisParser.AliasedQueryContext;
+import org.apache.doris.nereids.DorisParser.AliasedRelationContext;
 import org.apache.doris.nereids.DorisParser.ArithmeticBinaryContext;
 import org.apache.doris.nereids.DorisParser.ArithmeticUnaryContext;
 import org.apache.doris.nereids.DorisParser.BooleanLiteralContext;
@@ -59,6 +61,7 @@ import org.apache.doris.nereids.DorisParser.SortItemContext;
 import org.apache.doris.nereids.DorisParser.StarContext;
 import org.apache.doris.nereids.DorisParser.StringLiteralContext;
 import org.apache.doris.nereids.DorisParser.SubqueryExpressionContext;
+import org.apache.doris.nereids.DorisParser.TableAliasContext;
 import org.apache.doris.nereids.DorisParser.TableNameContext;
 import org.apache.doris.nereids.DorisParser.TypeConstructorContext;
 import org.apache.doris.nereids.DorisParser.UnitIdentifierContext;
@@ -69,6 +72,8 @@ import org.apache.doris.nereids.analyzer.UnboundFunction;
 import org.apache.doris.nereids.analyzer.UnboundRelation;
 import org.apache.doris.nereids.analyzer.UnboundSlot;
 import org.apache.doris.nereids.analyzer.UnboundStar;
+import org.apache.doris.nereids.annotation.Developing;
+import org.apache.doris.nereids.exceptions.ParseException;
 import org.apache.doris.nereids.properties.OrderKey;
 import org.apache.doris.nereids.trees.expressions.Add;
 import org.apache.doris.nereids.trees.expressions.Alias;
@@ -118,6 +123,7 @@ import 
org.apache.doris.nereids.trees.plans.logical.LogicalLimit;
 import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
 import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
+import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
@@ -212,11 +218,33 @@ public class LogicalPlanBuilder extends 
DorisParserBaseVisitor<Object> {
     /**
      * Create an aliased table reference. This is typically used in FROM 
clauses.
      */
+    @Developing
+    private LogicalPlan withTableAlias(LogicalPlan plan, TableAliasContext 
ctx) {
+        String alias = ctx.strictIdentifier().getText();
+        if (null != ctx.identifierList()) {
+            throw new ParseException("Do not implemented", ctx);
+            // TODO: multi-colName
+        }
+        return new LogicalSubQueryAlias<>(alias, plan);
+    }
+
     @Override
     public LogicalPlan visitTableName(TableNameContext ctx) {
         List<String> tableId = 
visitMultipartIdentifier(ctx.multipartIdentifier());
-        // TODO: sample and time travel, alias, sub query
-        return new UnboundRelation(tableId);
+        if (null == ctx.tableAlias().strictIdentifier()) {
+            return new UnboundRelation(tableId);
+        }
+        return withTableAlias(new UnboundRelation(tableId), ctx.tableAlias());
+    }
+
+    @Override
+    public LogicalPlan visitAliasedQuery(AliasedQueryContext ctx) {
+        return withTableAlias(visitQuery(ctx.query()), ctx.tableAlias());
+    }
+
+    @Override
+    public LogicalPlan visitAliasedRelation(AliasedRelationContext ctx) {
+        return withTableAlias(visitRelation(ctx.relation()), ctx.tableAlias());
     }
 
     /**
@@ -568,19 +596,33 @@ public class LogicalPlanBuilder extends 
DorisParserBaseVisitor<Object> {
         return visit(namedCtx.namedExpression(), Expression.class);
     }
 
+    @Override
+    public LogicalPlan visitRelation(RelationContext ctx) {
+        LogicalPlan right = plan(ctx.relationPrimary());
+        if (ctx.LATERAL() != null) {
+            if (!(right instanceof LogicalSubQueryAlias)) {
+                throw new IllegalStateException("lateral join right table 
should be sub-query");
+            }
+        }
+        return right;
+    }
+
     @Override
     public LogicalPlan visitFromClause(FromClauseContext ctx) {
         return ParserUtils.withOrigin(ctx, () -> {
             LogicalPlan left = null;
-            // build left deep join tree
             for (RelationContext relation : ctx.relation()) {
-                LogicalPlan right = plan(relation.relationPrimary());
-                left = (left == null)
-                        ? right
-                        : new LogicalJoin<>(JoinType.CROSS_JOIN, 
Optional.empty(), left, right);
+                // build left deep join tree
+                LogicalPlan right = visitRelation(relation);
+                left = (left == null) ? right :
+                        new LogicalJoin<>(
+                                JoinType.CROSS_JOIN,
+                                Optional.empty(),
+                                left,
+                                right);
                 left = withJoinRelations(left, relation);
+                // TODO: pivot and lateral view
             }
-            // TODO: pivot and lateral view
             return left;
         });
     }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/OrderKey.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/OrderKey.java
index 8059b2ddfa..7ed9e8955f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/OrderKey.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/OrderKey.java
@@ -67,7 +67,7 @@ public class OrderKey {
 
     @Override
     public String toString() {
-        return expr.toSql();
+        return expr.toString();
     }
 
     @Override
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 18b077fd2b..92c306b2fd 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
@@ -33,6 +33,7 @@ public enum RuleType {
     BINDING_SORT_SLOT(RuleTypeClass.REWRITE),
     BINDING_PROJECT_FUNCTION(RuleTypeClass.REWRITE),
     BINDING_AGGREGATE_FUNCTION(RuleTypeClass.REWRITE),
+    BINDING_SUBQUERY_ALIAS_SLOT(RuleTypeClass.REWRITE),
     BINDING_FILTER_FUNCTION(RuleTypeClass.REWRITE),
 
     RESOLVE_PROJECT_ALIAS(RuleTypeClass.REWRITE),
@@ -41,6 +42,14 @@ public enum RuleType {
     // rewrite rules
     AGGREGATE_DISASSEMBLE(RuleTypeClass.REWRITE),
     COLUMN_PRUNE_PROJECTION(RuleTypeClass.REWRITE),
+    ELIMINATE_ALIAS_NODE(RuleTypeClass.REWRITE),
+
+    PROJECT_ELIMINATE_ALIAS_NODE(RuleTypeClass.REWRITE),
+    FILTER_ELIMINATE_ALIAS_NODE(RuleTypeClass.REWRITE),
+    JOIN_ELIMINATE_ALIAS_NODE(RuleTypeClass.REWRITE),
+    JOIN_LEFT_CHILD_ELIMINATE_ALIAS_NODE(RuleTypeClass.REWRITE),
+    JOIN_RIGHT_CHILD_ELIMINATE_ALIAS_NODE(RuleTypeClass.REWRITE),
+    AGGREGATE_ELIMINATE_ALIAS_NODE(RuleTypeClass.REWRITE),
     // predicate push down rules
     PUSH_DOWN_PREDICATE_THROUGH_JOIN(RuleTypeClass.REWRITE),
     PUSH_DOWN_PREDICATE_THROUGH_AGGREGATION(RuleTypeClass.REWRITE),
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 f68e3bdc40..88ca9598b1 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
@@ -20,9 +20,13 @@ package org.apache.doris.nereids.rules.analysis;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.Table;
+import org.apache.doris.catalog.TableIf.TableType;
+import org.apache.doris.nereids.analyzer.NereidsAnalyzer;
 import org.apache.doris.nereids.rules.Rule;
 import org.apache.doris.nereids.rules.RuleType;
 import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias;
 import org.apache.doris.qe.ConnectContext;
 
 import com.google.common.collect.ImmutableList;
@@ -41,16 +45,11 @@ public class BindRelation extends OneAnalysisRuleFactory {
             switch (nameParts.size()) {
                 case 1: {
                     // Use current database name from catalog.
-                    String dbName = connectContext.getDatabase();
-                    Table table = getTable(dbName, nameParts.get(0), 
connectContext.getEnv());
-                    // TODO: should generate different Scan sub class 
according to table's type
-                    return new LogicalOlapScan(table, 
ImmutableList.of(dbName));
+                    return bindWithCurrentDb(connectContext, nameParts);
                 }
                 case 2: {
                     // Use database name from table name parts.
-                    String dbName = connectContext.getClusterName() + ":" + 
nameParts.get(0);
-                    Table table = getTable(dbName, nameParts.get(1), 
connectContext.getEnv());
-                    return new LogicalOlapScan(table, 
ImmutableList.of(dbName));
+                    return bindWithDbNameFromNamePart(connectContext, 
nameParts);
                 }
                 default:
                     throw new IllegalStateException("Table name [" + 
ctx.root.getTableName() + "] is invalid.");
@@ -69,4 +68,33 @@ public class BindRelation extends OneAnalysisRuleFactory {
             db.readUnlock();
         }
     }
+
+    private LogicalPlan bindWithCurrentDb(ConnectContext ctx, List<String> 
nameParts) {
+        String dbName = ctx.getDatabase();
+        Table table = getTable(dbName, nameParts.get(0), ctx.getEnv());
+        // TODO: should generate different Scan sub class according to table's 
type
+        if (table.getType() == TableType.OLAP) {
+            return new LogicalOlapScan(table, ImmutableList.of(dbName));
+        } else if (table.getType() == TableType.VIEW) {
+            LogicalPlan viewPlan = new 
NereidsAnalyzer(ctx).analyze(table.getDdlSql());
+            return new LogicalSubQueryAlias<>(table.getName(), viewPlan);
+        }
+        throw new RuntimeException("Unsupported tableType:" + table.getType());
+    }
+
+    private LogicalPlan bindWithDbNameFromNamePart(ConnectContext ctx, 
List<String> nameParts) {
+        // if the relation is view, nameParts.get(0) is dbName.
+        String dbName = nameParts.get(0);
+        if (!dbName.equals(ctx.getDatabase())) {
+            dbName = ctx.getClusterName() + ":" + nameParts.get(0);
+        }
+        Table table = getTable(dbName, nameParts.get(1), ctx.getEnv());
+        if (table.getType() == TableType.OLAP) {
+            return new LogicalOlapScan(table, ImmutableList.of(dbName));
+        } else if (table.getType() == TableType.VIEW) {
+            LogicalPlan viewPlan = new 
NereidsAnalyzer(ctx).analyze(table.getDdlSql());
+            return new LogicalSubQueryAlias<>(table.getName(), viewPlan);
+        }
+        throw new RuntimeException("Unsupported tableType:" + table.getType());
+    }
 }
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 182a0c55b3..e38702f101 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
@@ -34,6 +34,7 @@ import 
org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
 import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
 import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
 import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
+import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
@@ -103,6 +104,9 @@ public class BindSlotReference implements 
AnalysisRuleFactory {
 
                     return new LogicalSort<>(sortItemList, sort.child());
                 })
+            ),
+            RuleType.BINDING_SUBQUERY_ALIAS_SLOT.build(
+                    logicalSubQueryAlias().then(alias -> new 
LogicalSubQueryAlias<>(alias.getAlias(), alias.child()))
             )
         );
     }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateAliasNode.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateAliasNode.java
new file mode 100644
index 0000000000..38b294a72a
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateAliasNode.java
@@ -0,0 +1,66 @@
+// 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 com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * Eliminate the logical sub query and alias node after analyze and before 
rewrite
+ * If we match the alias node and return its child node, in the execute() of 
the job
+ * <p>
+ * TODO: refactor group merge strategy to support the feature above
+ */
+public class EliminateAliasNode implements AnalysisRuleFactory {
+    @Override
+    public List<Rule> buildRules() {
+        return ImmutableList.of(
+                RuleType.PROJECT_ELIMINATE_ALIAS_NODE.build(
+                        logicalProject(logicalSubQueryAlias())
+                                .then(project -> 
project.withChildren(ImmutableList.of(project.child().child())))
+                ),
+                RuleType.FILTER_ELIMINATE_ALIAS_NODE.build(
+                        logicalFilter(logicalSubQueryAlias())
+                                .then(filter -> 
filter.withChildren(ImmutableList.of(filter.child().child())))
+                ),
+                RuleType.AGGREGATE_ELIMINATE_ALIAS_NODE.build(
+                        logicalAggregate(logicalSubQueryAlias())
+                                .then(agg -> 
agg.withChildren(ImmutableList.of(agg.child().child())))
+                ),
+                RuleType.JOIN_ELIMINATE_ALIAS_NODE.build(
+                        logicalJoin(logicalSubQueryAlias(), 
logicalSubQueryAlias())
+                                .then(join -> join.withChildren(
+                                        ImmutableList.of(join.left().child(), 
join.right().child())))
+                ),
+                RuleType.JOIN_LEFT_CHILD_ELIMINATE_ALIAS_NODE.build(
+                        logicalJoin(logicalSubQueryAlias(), group())
+                                .then(join -> join.withChildren(
+                                        ImmutableList.of(join.left().child(), 
join.right())))
+                ),
+                RuleType.JOIN_RIGHT_CHILD_ELIMINATE_ALIAS_NODE.build(
+                        logicalJoin(group(), logicalSubQueryAlias())
+                                .then(join -> join.withChildren(
+                                        ImmutableList.of(join.left(), 
join.right().child())))
+                )
+        );
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpressionUtil.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpressionUtil.java
index bbbc839ea3..9e5fa24062 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpressionUtil.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/NamedExpressionUtil.java
@@ -19,6 +19,11 @@ package org.apache.doris.nereids.trees.expressions;
 
 import org.apache.doris.common.IdGenerator;
 
+import com.google.common.annotations.VisibleForTesting;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+
 /**
  * The util of named expression.
  */
@@ -31,4 +36,17 @@ public class NamedExpressionUtil {
     public static ExprId newExprId() {
         return ID_GENERATOR.getNextId();
     }
+
+    /**
+     *  Reset Id Generator
+     */
+    @VisibleForTesting
+    public static void clear() throws Exception {
+        Field f = NamedExpressionUtil.class.getDeclaredField("ID_GENERATOR");
+        f.setAccessible(true);
+        Field mf = Field.class.getDeclaredField("modifiers");
+        mf.setAccessible(true);
+        mf.setInt(f, f.getModifiers() & ~Modifier.FINAL);
+        f.set(NamedExpressionUtil.class, ExprId.createGenerator());
+    }
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Slot.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Slot.java
index ccc3f4b5fc..232164b1f9 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Slot.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Slot.java
@@ -17,6 +17,8 @@
 
 package org.apache.doris.nereids.trees.expressions;
 
+import java.util.List;
+
 /**
  * Abstract class for all slot in expression.
  */
@@ -30,4 +32,8 @@ public abstract class Slot extends NamedExpression implements 
LeafExpression {
     public Slot withNullable(boolean newNullable) {
         throw new RuntimeException("Do not implement");
     }
+
+    public Slot withQualifier(List<String> qualifiers) {
+        throw new RuntimeException("Do not implement");
+    }
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java
index 310dcb6d9a..aa0a15631d 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java
@@ -149,4 +149,9 @@ public class SlotReference extends Slot {
         }
         return new SlotReference(exprId, name, dataType, newNullable, 
qualifier);
     }
+
+    @Override
+    public Slot withQualifier(List<String> qualifiers) {
+        return new SlotReference(exprId, name, dataType, nullable, qualifiers);
+    }
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java
index 53f45a7ab1..b920589f80 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java
@@ -24,6 +24,7 @@ public enum PlanType {
     UNKNOWN,
 
     // logical plan
+    LOGICAL_SUBQUERY_ALIAS,
     LOGICAL_UNBOUND_RELATION,
     LOGICAL_BOUND_RELATION,
     LOGICAL_PROJECT,
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java
index 1b3f57949c..e1aaca270c 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java
@@ -62,6 +62,7 @@ public class LogicalProject<CHILD_TYPE extends Plan> extends 
LogicalUnary<CHILD_
      *
      * @return all project of this node.
      */
+    @Override
     public List<NamedExpression> getProjects() {
         return projects;
     }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java
similarity index 53%
copy from 
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java
copy to 
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java
index 1b3f57949c..5bb98cf41f 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java
@@ -20,72 +20,52 @@ package org.apache.doris.nereids.trees.plans.logical;
 import org.apache.doris.nereids.memo.GroupExpression;
 import org.apache.doris.nereids.properties.LogicalProperties;
 import org.apache.doris.nereids.trees.expressions.Expression;
-import org.apache.doris.nereids.trees.expressions.NamedExpression;
 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.Project;
 import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
-import org.apache.commons.lang3.StringUtils;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 import java.util.Optional;
+import java.util.stream.Collectors;
 
 /**
- * Logical project plan.
+ * The node of logical plan for sub query and alias
+ *
+ * @param <CHILD_TYPE> param
  */
-public class LogicalProject<CHILD_TYPE extends Plan> extends 
LogicalUnary<CHILD_TYPE> implements Project {
+public class LogicalSubQueryAlias<CHILD_TYPE extends Plan> extends 
LogicalUnary<CHILD_TYPE> {
+    private final String alias;
 
-    private final List<NamedExpression> projects;
-
-    public LogicalProject(List<NamedExpression> projects, CHILD_TYPE child) {
-        this(projects, Optional.empty(), Optional.empty(), child);
-    }
-
-    /**
-     * Constructor for LogicalProject.
-     *
-     * @param projects project list
-     */
-    public LogicalProject(List<NamedExpression> projects, 
Optional<GroupExpression> groupExpression,
-                          Optional<LogicalProperties> logicalProperties, 
CHILD_TYPE child) {
-        super(PlanType.LOGICAL_PROJECT, groupExpression, logicalProperties, 
child);
-        this.projects = ImmutableList.copyOf(Objects.requireNonNull(projects, 
"projects can not be null"));
+    public LogicalSubQueryAlias(String tableAlias, CHILD_TYPE child) {
+        this(tableAlias, Optional.empty(), Optional.empty(), child);
     }
 
-    /**
-     * Get project list.
-     *
-     * @return all project of this node.
-     */
-    public List<NamedExpression> getProjects() {
-        return projects;
+    public LogicalSubQueryAlias(String tableAlias, Optional<GroupExpression> 
groupExpression,
+            Optional<LogicalProperties> logicalProperties, CHILD_TYPE child) {
+        super(PlanType.LOGICAL_SUBQUERY_ALIAS, groupExpression, 
logicalProperties, child);
+        this.alias = tableAlias;
     }
 
     @Override
     public List<Slot> computeOutput(Plan input) {
-        return projects.stream()
-                .map(NamedExpression::toSlot)
-                .collect(ImmutableList.toImmutableList());
-    }
-
-    @Override
-    public String toString() {
-        return "LogicalProject (" + StringUtils.join(projects, ", ") + ")";
+        return input.getOutput().stream()
+                .map(slot -> slot.withQualifier(ImmutableList.of(alias)))
+                .collect(Collectors.toList());
     }
 
-    @Override
-    public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
-        return visitor.visitLogicalProject((LogicalProject<Plan>) this, 
context);
+    public String getAlias() {
+        return alias;
     }
 
     @Override
-    public List<Expression> getExpressions() {
-        return new 
ImmutableList.Builder<Expression>().addAll(projects).build();
+    public String toString() {
+        return "LogicalSubQueryAlias (" + alias + ")";
     }
 
     @Override
@@ -96,28 +76,38 @@ public class LogicalProject<CHILD_TYPE extends Plan> 
extends LogicalUnary<CHILD_
         if (o == null || getClass() != o.getClass()) {
             return false;
         }
-        LogicalProject that = (LogicalProject) o;
-        return projects.equals(that.projects);
+        LogicalSubQueryAlias that = (LogicalSubQueryAlias) o;
+        return alias.equals(that.alias);
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(projects);
+        return Objects.hash(alias);
     }
 
     @Override
-    public LogicalUnary<Plan> withChildren(List<Plan> children) {
+    public Plan withChildren(List<Plan> children) {
         Preconditions.checkArgument(children.size() == 1);
-        return new LogicalProject<>(projects, children.get(0));
+        return new LogicalSubQueryAlias<>(alias, children.get(0));
+    }
+
+    @Override
+    public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
+        return visitor.visitSubQueryAlias((LogicalSubQueryAlias<Plan>) this, 
context);
+    }
+
+    @Override
+    public List<Expression> getExpressions() {
+        return Collections.emptyList();
     }
 
     @Override
     public Plan withGroupExpression(Optional<GroupExpression> groupExpression) 
{
-        return new LogicalProject<>(projects, groupExpression, 
Optional.of(logicalProperties), child());
+        return new LogicalSubQueryAlias<>(alias, groupExpression, 
Optional.of(logicalProperties), child());
     }
 
     @Override
     public Plan withLogicalProperties(Optional<LogicalProperties> 
logicalProperties) {
-        return new LogicalProject<>(projects, Optional.empty(), 
logicalProperties, child());
+        return new LogicalSubQueryAlias<>(alias, Optional.empty(), 
logicalProperties, child());
     }
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java
index 2f91e1f7f3..523ba554ee 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java
@@ -31,6 +31,7 @@ import 
org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
 import org.apache.doris.nereids.trees.plans.logical.LogicalRelation;
 import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
+import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalAggregate;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribution;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter;
@@ -56,6 +57,10 @@ public abstract class PlanVisitor<R, C> {
     // Logical plans
     // *******************************
 
+    public R visitSubQueryAlias(LogicalSubQueryAlias<Plan> alias, C context) {
+        return visit(alias, context);
+    }
+
     public R visitUnboundRelation(UnboundRelation relation, C context) {
         return visit(relation, context);
     }
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/AnalyzeSubQueryTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/AnalyzeSubQueryTest.java
new file mode 100644
index 0000000000..c1413d974a
--- /dev/null
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/AnalyzeSubQueryTest.java
@@ -0,0 +1,178 @@
+// 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.util;
+
+import org.apache.doris.nereids.NereidsPlanner;
+import org.apache.doris.nereids.analyzer.NereidsAnalyzer;
+import org.apache.doris.nereids.glue.translator.PhysicalPlanTranslator;
+import org.apache.doris.nereids.glue.translator.PlanTranslatorContext;
+import org.apache.doris.nereids.parser.NereidsParser;
+import org.apache.doris.nereids.properties.PhysicalProperties;
+import org.apache.doris.nereids.rules.analysis.EliminateAliasNode;
+import org.apache.doris.nereids.trees.expressions.EqualTo;
+import org.apache.doris.nereids.trees.expressions.ExprId;
+import org.apache.doris.nereids.trees.expressions.NamedExpressionUtil;
+import org.apache.doris.nereids.trees.expressions.SlotReference;
+import org.apache.doris.nereids.trees.plans.JoinType;
+import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan;
+import org.apache.doris.nereids.types.BigIntType;
+import org.apache.doris.utframe.TestWithFeService;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Optional;
+
+public class AnalyzeSubQueryTest extends TestWithFeService implements 
PatternMatchSupported {
+    private final NereidsParser parser = new NereidsParser();
+
+    private final List<String> testSql = ImmutableList.of(
+            "SELECT * FROM (SELECT * FROM T1 T) T2",
+            "SELECT * FROM T1 TT1 JOIN (SELECT * FROM T2 TT2) T ON TT1.ID = 
T.ID",
+            "SELECT * FROM T1 TT1 JOIN (SELECT TT2.ID FROM T2 TT2) T ON TT1.ID 
= T.ID",
+            "SELECT T.ID FROM T1 T",
+            "SELECT A.ID FROM T1 A, T2 B WHERE A.ID = B.ID",
+            "SELECT * FROM T1 JOIN T1 T2 ON T1.ID = T2.ID"
+    );
+
+    @Override
+    protected void runBeforeAll() throws Exception {
+        createDatabase("test");
+        connectContext.setDatabase("default_cluster:test");
+
+        createTables(
+                "CREATE TABLE IF NOT EXISTS T1 (\n"
+                        + "    id bigint,\n"
+                        + "    score bigint\n"
+                        + ")\n"
+                        + "DUPLICATE KEY(id)\n"
+                        + "DISTRIBUTED BY HASH(id) BUCKETS 1\n"
+                        + "PROPERTIES (\n"
+                        + "  \"replication_num\" = \"1\"\n"
+                        + ")\n",
+                "CREATE TABLE IF NOT EXISTS T2 (\n"
+                        + "    id bigint,\n"
+                        + "    score bigint\n"
+                        + ")\n"
+                        + "DUPLICATE KEY(id)\n"
+                        + "DISTRIBUTED BY HASH(id) BUCKETS 1\n"
+                        + "PROPERTIES (\n"
+                        + "  \"replication_num\" = \"1\"\n"
+                        + ")\n"
+        );
+    }
+
+    @Override
+    protected void runBeforeEach() throws Exception {
+        NamedExpressionUtil.clear();
+    }
+
+    @Test
+    public void testTranslateCase() throws Exception {
+        for (String sql : testSql) {
+            NamedExpressionUtil.clear();
+            System.out.println("\n\n***** " + sql + " *****\n\n");
+            PhysicalPlan plan = new NereidsPlanner(connectContext).plan(
+                    parser.parseSingle(sql),
+                    PhysicalProperties.ANY,
+                    connectContext
+            );
+            // Just to check whether translate will throw exception
+            new PhysicalPlanTranslator().translatePlan(plan, new 
PlanTranslatorContext());
+        }
+    }
+
+    @Test
+    public void testCaseSubQuery() {
+        FieldChecker projectChecker = new 
FieldChecker(ImmutableList.of("projects"));
+        new PlanChecker().plan(new 
NereidsAnalyzer(connectContext).analyze(testSql.get(0)))
+                .applyTopDown(new EliminateAliasNode())
+                .matches(
+                        logicalProject(
+                                logicalProject(
+                                        logicalOlapScan().when(o -> true)
+                                
).when(projectChecker.check(ImmutableList.of(ImmutableList.of(
+                                        new SlotReference(new ExprId(0), "id", 
new BigIntType(), true, ImmutableList.of("T")),
+                                        new SlotReference(new ExprId(1), 
"score", new BigIntType(), true, ImmutableList.of("T")))))
+                                )
+                        
).when(projectChecker.check(ImmutableList.of(ImmutableList.of(
+                                new SlotReference(new ExprId(0), "id", new 
BigIntType(), true, ImmutableList.of("T2")),
+                                new SlotReference(new ExprId(1), "score", new 
BigIntType(), true, ImmutableList.of("T2")))))
+                        )
+                );
+    }
+
+    @Test
+    public void testCaseMixed() {
+        FieldChecker projectChecker = new 
FieldChecker(ImmutableList.of("projects"));
+        FieldChecker joinChecker = new 
FieldChecker(ImmutableList.of("joinType", "condition"));
+        new PlanChecker().plan(new 
NereidsAnalyzer(connectContext).analyze(testSql.get(1)))
+                .applyTopDown(new EliminateAliasNode())
+                .matches(
+                        logicalProject(
+                                logicalJoin(
+                                        logicalOlapScan(),
+                                        logicalProject(
+                                                logicalOlapScan()
+                                        
).when(projectChecker.check(ImmutableList.of(ImmutableList.of(
+                                                new SlotReference(new 
ExprId(0), "id", new BigIntType(), true, ImmutableList.of("TT2")),
+                                                new SlotReference(new 
ExprId(1), "score", new BigIntType(), true, ImmutableList.of("TT2")))))
+                                        )
+                                ).when(joinChecker.check(ImmutableList.of(
+                                        JoinType.INNER_JOIN,
+                                        Optional.of(new EqualTo(
+                                                new SlotReference(new 
ExprId(2), "id", new BigIntType(), true, ImmutableList.of("TT1")),
+                                                new SlotReference(new 
ExprId(0), "id", new BigIntType(), true, ImmutableList.of("T"))))))
+                                )
+                        
).when(projectChecker.check(ImmutableList.of(ImmutableList.of(
+                                new SlotReference(new ExprId(2), "id", new 
BigIntType(), true, ImmutableList.of("TT1")),
+                                new SlotReference(new ExprId(3), "score", new 
BigIntType(), true, ImmutableList.of("TT1")),
+                                new SlotReference(new ExprId(0), "id", new 
BigIntType(), true, ImmutableList.of("T")),
+                                new SlotReference(new ExprId(1), "score", new 
BigIntType(), true, ImmutableList.of("T")))))
+                        )
+                );
+    }
+
+    @Test
+    public void testCaseJoinSameTable() {
+        FieldChecker projectChecker = new 
FieldChecker(ImmutableList.of("projects"));
+        FieldChecker joinChecker = new 
FieldChecker(ImmutableList.of("joinType", "condition"));
+        new PlanChecker().plan(new 
NereidsAnalyzer(connectContext).analyze(testSql.get(5)))
+                .applyTopDown(new EliminateAliasNode())
+                .matches(
+                        logicalProject(
+                                logicalJoin(
+                                        logicalOlapScan(),
+                                        logicalOlapScan()
+                                ).when(joinChecker.check(ImmutableList.of(
+                                        JoinType.INNER_JOIN,
+                                        Optional.of(new EqualTo(
+                                                new SlotReference(new 
ExprId(0), "id", new BigIntType(), true, 
ImmutableList.of("default_cluster:test", "T1")),
+                                                new SlotReference(new 
ExprId(2), "id", new BigIntType(), true, ImmutableList.of("T2"))))))
+                                )
+                        
).when(projectChecker.check(ImmutableList.of(ImmutableList.of(
+                                new SlotReference(new ExprId(0), "id", new 
BigIntType(), true, ImmutableList.of("default_cluster:test", "T1")),
+                                new SlotReference(new ExprId(1), "score", new 
BigIntType(), true, ImmutableList.of("default_cluster:test", "T1")),
+                                new SlotReference(new ExprId(2), "id", new 
BigIntType(), true, ImmutableList.of("T2")),
+                                new SlotReference(new ExprId(3), "score", new 
BigIntType(), true, ImmutableList.of("T2")))))
+                        )
+                );
+    }
+}
+
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/FieldChecker.java 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/FieldChecker.java
new file mode 100644
index 0000000000..da55e9ebeb
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/FieldChecker.java
@@ -0,0 +1,56 @@
+// 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.util;
+
+import org.junit.jupiter.api.Assertions;
+
+import java.lang.reflect.Field;
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.IntStream;
+
+public class FieldChecker {
+
+    public final List<String> fields;
+
+    public FieldChecker(List<String> fields) {
+        this.fields = fields;
+    }
+
+    public <T> Predicate<T> check(List<Object> valueList) {
+        return (o) -> {
+            Assertions.assertEquals(fields.size(), valueList.size());
+            Class<?> classInfo = o.getClass();
+            IntStream.range(0, valueList.size()).forEach(i -> {
+                Field field;
+                try {
+                    field = classInfo.getDeclaredField(this.fields.get(i));
+                } catch (NoSuchFieldException e) {
+                    throw new RuntimeException(e);
+                }
+                field.setAccessible(true);
+                try {
+                    Assertions.assertEquals(valueList.get(i), field.get(o));
+                } catch (IllegalAccessException e) {
+                    throw new RuntimeException(e);
+                }
+            });
+            return true;
+        };
+    }
+}
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java 
b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java
index 823bc4f776..a441eab88d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java
@@ -66,6 +66,7 @@ import org.apache.commons.io.FileUtils;
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.TestInstance;
 
 import java.io.File;
@@ -112,12 +113,20 @@ public abstract class TestWithFeService {
         cleanDorisFeDir(runningDir);
     }
 
+    @BeforeEach
+    public final void beforeEach() throws Exception {
+        runBeforeEach();
+    }
+
     protected void runBeforeAll() throws Exception {
     }
 
     protected void runAfterAll() throws Exception {
     }
 
+    protected void runBeforeEach() throws Exception {
+    }
+
     // Help to create a mocked ConnectContext.
     protected ConnectContext createDefaultCtx() throws IOException {
         return createCtx(UserIdentity.ROOT, "127.0.0.1");
diff --git a/regression-test/data/nereids_syntax_p0/sub_query_alias.out 
b/regression-test/data/nereids_syntax_p0/sub_query_alias.out
new file mode 100644
index 0000000000..bd9b380b01
--- /dev/null
+++ b/regression-test/data/nereids_syntax_p0/sub_query_alias.out
@@ -0,0 +1,26 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !select_1 --
+1303   1303
+1303   1303
+1312   1312
+
+-- !select_2 --
+1303   1303
+1303   1303
+1312   1312
+
+-- !select_3 --
+1303   1303
+1303   1303
+1312   1312
+
+-- !select_4 --
+1303   Customer#000001303      fQ Lp,FoozZe1   ETHIOPIA 3      ETHIOPIA        
AFRICA  15-658-234-7985 MACHINERY       1303    Customer#000001303      fQ 
Lp,FoozZe1   ETHIOPIA 3      ETHIOPIA        AFRICA  15-658-234-7985 MACHINERY
+1309   Customer#000001309      vQcJGUXPHMH2 5OWs1XUP0kx        IRAN     2      
IRAN    MIDDLE EAST     20-821-905-5952 AUTOMOBILE      1309    
Customer#000001309      vQcJGUXPHMH2 5OWs1XUP0kx        IRAN     2      IRAN    
MIDDLE EAST     20-821-905-5952 AUTOMOBILE
+1312   Customer#000001312      MVsKeqWejff8jQ30        CANADA   9      CANADA  
AMERICA 13-153-492-9898 BUILDING        1312    Customer#000001312      
MVsKeqWejff8jQ30        CANADA   9      CANADA  AMERICA 13-153-492-9898 BUILDING
+
+-- !select_5 --
+1303   Customer#000001303      fQ Lp,FoozZe1   ETHIOPIA 3      ETHIOPIA        
AFRICA  15-658-234-7985 MACHINERY       1309892 1       1303    1432    15      
19920517        3-MEDIUM        0       24      2959704 5119906 7       2752524 
73992   0       19920619        TRUCK
+1303   Customer#000001303      fQ Lp,FoozZe1   ETHIOPIA 3      ETHIOPIA        
AFRICA  15-658-234-7985 MACHINERY       1309892 2       1303    1165    9       
19920517        3-MEDIUM        0       21      2404899 5119906 8       2212507 
68711   7       19920616        RAIL
+1312   Customer#000001312      MVsKeqWejff8jQ30        CANADA   9      CANADA  
AMERICA 13-153-492-9898 BUILDING        1310179 6       1312    1455    29      
19921110        3-MEDIUM        0       15      1705830 20506457        10      
1535247 68233   8       19930114        FOB
+
diff --git a/regression-test/suites/nereids_syntax_p0/sub_query_alias.groovy 
b/regression-test/suites/nereids_syntax_p0/sub_query_alias.groovy
new file mode 100644
index 0000000000..8a50987089
--- /dev/null
+++ b/regression-test/suites/nereids_syntax_p0/sub_query_alias.groovy
@@ -0,0 +1,72 @@
+// 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.
+
+suite("sub_query_alias") {
+    sql """
+        SET enable_vectorized_engine=true
+    """
+
+    sql """
+        SET enable_nereids_planner=true
+    """
+
+    qt_select_1 """
+        select t.c_custkey, t.lo_custkey 
+        from (
+            select * 
+            from customer, lineorder 
+            where customer.c_custkey = lineorder.lo_custkey
+        ) t
+        order by t.c_custkey
+    """
+
+    qt_select_2 """
+        select c.c_custkey, l.lo_custkey 
+        from customer c, lineorder l 
+        where c.c_custkey = l.lo_custkey
+        order by c.c_custkey
+    """
+
+    qt_select_3 """
+        select t.c_custkey, t.lo_custkey 
+        from (
+            select * 
+            from customer c, lineorder l 
+            where c.c_custkey = l.lo_custkey
+        ) t
+        order by t.c_custkey
+    """
+
+    qt_select_4 """
+        select * 
+        from customer c 
+        join customer c1 
+        on c.c_custkey = c1.c_custkey
+        order by c.c_custkey
+    """
+
+    qt_select_5 """
+        select * 
+        from customer c 
+        join (
+            select * 
+            from lineorder l
+        ) t on c.c_custkey = t.lo_custkey
+        order by c.c_custkey
+    """
+}
+


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

Reply via email to