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