You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/06/27 04:45:12 UTC

[GitHub] [doris] morrySnow commented on a diff in pull request #10415: [feature](nereids) Support analyze for test SSB

morrySnow commented on code in PR #10415:
URL: https://github.com/apache/doris/pull/10415#discussion_r906932104


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundAlias.java:
##########
@@ -34,18 +38,28 @@
         extends NamedExpression
         implements UnaryExpression<CHILD_TYPE>, Unbound {
 
-    public UnboundAlias(CHILD_TYPE child) {
+    private final ExprId exprId;
+    private final String name;
+
+    public UnboundAlias(ExprId exprId, CHILD_TYPE child) {
         super(NodeType.UNBOUND_ALIAS, child);
+        this.exprId = Objects.requireNonNull(exprId, "exprId can not be null");
+        this.name = "expr_" + exprId;
     }
 
     @Override
     public String sql() {
-        return null;
+        return child().sql();

Review Comment:
   i think all unbound should not generate sql



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java:
##########
@@ -143,265 +130,47 @@ public Object visitChildren(RuleNode node) {
 
     @Override
     public LogicalPlan visitSingleStatement(SingleStatementContext ctx) {
-        Supplier<LogicalPlan> f = () -> (LogicalPlan) visit(ctx.statement());
-        return ParserUtils.withOrigin(ctx, f);
+        return ParserUtils.withOrigin(ctx, () -> (LogicalPlan) visit(ctx.statement()));
     }
 
     /* ********************************************************************************************
      * Plan parsing
      * ******************************************************************************************** */
-    private LogicalPlan plan(ParserRuleContext tree) {
-        return (LogicalPlan) tree.accept(this);
-    }
-
     @Override
     public LogicalPlan visitQuery(QueryContext ctx) {
-        Supplier<LogicalPlan> f = () -> {
+        return ParserUtils.withOrigin(ctx, () -> {
             // TODO: need to add withQueryResultClauses and withCTE
             LogicalPlan query = plan(ctx.queryTerm());
-            LogicalPlan queryOrganization = withQueryOrganization(ctx.queryOrganization(), query);
+            LogicalPlan queryOrganization = withOrganization(query, ctx.queryOrganization());
             return queryOrganization;
-        };
-        return ParserUtils.withOrigin(ctx, f);
-    }
-
-    private LogicalPlan withQueryOrganization(QueryOrganizationContext ctx, LogicalPlan children) {
-        List<SortItems> sortItems = visitQueryOrganization(ctx);
-        return sortItems == null ? children : new LogicalUnaryPlan(new LogicalSort(sortItems), children);
+        });
     }
 
     @Override
     public LogicalPlan visitRegularQuerySpecification(RegularQuerySpecificationContext ctx) {
-        Supplier<LogicalPlan> f = () -> {
+        return ParserUtils.withOrigin(ctx, () -> {
             // TODO: support on row relation
-            LogicalPlan from = visitFromClause(ctx.fromClause());
+            LogicalPlan relation = withRelation(Optional.ofNullable(ctx.fromClause()));
             return withSelectQuerySpecification(
-                    ctx,
-                    ctx.selectClause(),
-                    ctx.whereClause(),
-                    from,
-                    ctx.aggClause());
-        };
-        return ParserUtils.withOrigin(ctx, f);
-    }
-
-    @Override
-    public Expression visitExpression(ExpressionContext ctx) {
-        Supplier<Expression> f = () -> (Expression) visit(ctx.booleanExpression());
-        return ParserUtils.withOrigin(ctx, f);
-    }
-
-    @Override
-    public List<Expression> visitNamedExpressionSeq(NamedExpressionSeqContext ctx) {
-        List<Expression> expressions = Lists.newArrayList();
-        if (ctx != null) {
-            for (NamedExpressionContext namedExpressionContext : ctx.namedExpression()) {
-                Expression expression = typedVisit(namedExpressionContext);
-                expressions.add(expression);
-            }
-        }
-        return expressions;
-    }
-
-    /**
-     * Add a regular (SELECT) query specification to a logical plan. The query specification
-     * is the core of the logical plan, this is where sourcing (FROM clause), projection (SELECT),
-     * aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
-     *
-     * <p>Note that query hints are ignored (both by the parser and the builder).
-     */
-    private LogicalPlan withSelectQuerySpecification(
-            ParserRuleContext ctx,
-            SelectClauseContext selectClause,
-            WhereClauseContext whereClause,
-            LogicalPlan relation,
-            AggClauseContext aggClause) {
-        Supplier<LogicalPlan> f = () -> {
-            //        Filter(expression(ctx.booleanExpression), plan);
-            LogicalPlan plan = visitCommonSelectQueryClausePlan(
-                    relation,
-                    visitNamedExpressionSeq(selectClause.namedExpressionSeq()),
-                    whereClause,
-                    aggClause);
-            // TODO: process hint
-            return plan;
-        };
-        return ParserUtils.withOrigin(ctx, f);
-    }
-
-    private LogicalPlan visitCommonSelectQueryClausePlan(
-            LogicalPlan relation,
-            List<Expression> expressions,
-            WhereClauseContext whereClause,
-            AggClauseContext aggClause) {
-        // TODO: add lateral views
-        // val withLateralView = lateralView.asScala.foldLeft(relation)(withGenerate)
-
-        // add where
-        LogicalPlan withFilter = relation.optionalMap(whereClause, withWhereClause);
-
-        List<NamedExpression> namedExpressions = expressions.stream().map(expression -> {
-            if (expression instanceof NamedExpression) {
-                return (NamedExpression) expression;
-            } else {
-                return new UnboundAlias(expression);
-            }
-        }).collect(Collectors.toList());
-
-        LogicalPlan withProject;
-        if (CollectionUtils.isNotEmpty(namedExpressions)) {
-            withProject = new LogicalUnaryPlan(new LogicalProject(namedExpressions), withFilter);
-        } else {
-            withProject = withFilter;
-        }
-
-        LogicalPlan withAgg;
-        if (aggClause != null) {
-            withAgg = withAggClause(namedExpressions, aggClause.groupByItem(), withFilter);
-        } else {
-            withAgg = withProject;
-        }
-
-        return withAgg;
-    }
-
-    @Override
-    public LogicalPlan visitFromClause(FromClauseContext ctx) {

Review Comment:
   should not remove this function, from clause use in more than one place in the future



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java:
##########
@@ -143,265 +130,47 @@ public Object visitChildren(RuleNode node) {
 
     @Override
     public LogicalPlan visitSingleStatement(SingleStatementContext ctx) {
-        Supplier<LogicalPlan> f = () -> (LogicalPlan) visit(ctx.statement());
-        return ParserUtils.withOrigin(ctx, f);
+        return ParserUtils.withOrigin(ctx, () -> (LogicalPlan) visit(ctx.statement()));
     }
 
     /* ********************************************************************************************
      * Plan parsing
      * ******************************************************************************************** */
-    private LogicalPlan plan(ParserRuleContext tree) {
-        return (LogicalPlan) tree.accept(this);
-    }
-
     @Override
     public LogicalPlan visitQuery(QueryContext ctx) {
-        Supplier<LogicalPlan> f = () -> {
+        return ParserUtils.withOrigin(ctx, () -> {
             // TODO: need to add withQueryResultClauses and withCTE
             LogicalPlan query = plan(ctx.queryTerm());
-            LogicalPlan queryOrganization = withQueryOrganization(ctx.queryOrganization(), query);
+            LogicalPlan queryOrganization = withOrganization(query, ctx.queryOrganization());
             return queryOrganization;
-        };
-        return ParserUtils.withOrigin(ctx, f);
-    }
-
-    private LogicalPlan withQueryOrganization(QueryOrganizationContext ctx, LogicalPlan children) {
-        List<SortItems> sortItems = visitQueryOrganization(ctx);
-        return sortItems == null ? children : new LogicalUnaryPlan(new LogicalSort(sortItems), children);
+        });
     }
 
     @Override
     public LogicalPlan visitRegularQuerySpecification(RegularQuerySpecificationContext ctx) {
-        Supplier<LogicalPlan> f = () -> {
+        return ParserUtils.withOrigin(ctx, () -> {
             // TODO: support on row relation
-            LogicalPlan from = visitFromClause(ctx.fromClause());
+            LogicalPlan relation = withRelation(Optional.ofNullable(ctx.fromClause()));

Review Comment:
   from clause is not only include relation, but also other plans, i think the original name `from` is better than `relation`



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundAlias.java:
##########
@@ -34,18 +38,28 @@
         extends NamedExpression
         implements UnaryExpression<CHILD_TYPE>, Unbound {
 
-    public UnboundAlias(CHILD_TYPE child) {
+    private final ExprId exprId;
+    private final String name;
+
+    public UnboundAlias(ExprId exprId, CHILD_TYPE child) {

Review Comment:
   unbound alias do not need ExprId at all, We only create unbound alias when we cannot ensure this expression represent one column, so no slot reference could refer to one UnboundAlias



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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