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/10/26 13:12:07 UTC

[GitHub] [doris] 924060929 commented on a diff in pull request #12742: [feature](nereids) support common table expression

924060929 commented on code in PR #12742:
URL: https://github.com/apache/doris/pull/12742#discussion_r1005395890


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java:
##########
@@ -68,7 +68,7 @@ public static UnboundSlot quoted(String name) {
 
     @Override
     public String toString() {
-        return "'" + getName();
+        return "'" + getName() + "'";

Review Comment:
   Left single quotation marks denote the UnboundXxx. So you should not add the single quotation marks on the right.



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundAlias.java:
##########
@@ -45,7 +54,10 @@ public DataType getDataType() throws UnboundException {
 
     @Override
     public String toString() {
-        return "UnboundAlias(" + child() + ")";
+        StringBuilder stringBuilder = new StringBuilder();
+        stringBuilder.append("UnboundAlias(" + child() + ")");
+        alias.ifPresent(name -> stringBuilder.append(" AS " + name));
+        return stringBuilder.toString();

Review Comment:
   use `Utils.toSqlString()` to implements `toString()`, see `UnboundRelation`



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java:
##########
@@ -73,9 +74,15 @@ private Table getTable(String dbName, String tableName, Env env) {
         }
     }
 
-    private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, List<String> nameParts) {
+    private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, String nameParts) {

Review Comment:
   nameParts -> tableName
   ```suggestion
       private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, String tableName) {
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotReference.java:
##########
@@ -218,9 +219,15 @@ public Expression bind(Expression expression) {
         public Expression visitUnboundAlias(UnboundAlias unboundAlias, PlannerContext context) {
             Expression child = unboundAlias.child().accept(this, context);
             if (child instanceof NamedExpression) {
+                if (unboundAlias.getAlias().isPresent()) {
+                    return new Alias(child, unboundAlias.getAlias().get());
+                }
                 return new Alias(child, ((NamedExpression) child).getName());
             } else {
                 // TODO: resolve aliases
+                if (unboundAlias.getAlias().isPresent()) {
+                    return new Alias(child, unboundAlias.getAlias().get());
+                }
                 return new Alias(child, child.toSql());
             }
         }

Review Comment:
   this two conditions can merge to one condition
   ```suggestion
           public Expression visitUnboundAlias(UnboundAlias unboundAlias, PlannerContext context) {
               Expression child = unboundAlias.child().accept(this, context);
               if (unboundAlias.getAlias().isPresent()) {
                   return new Alias(child, unboundAlias.getAlias().get());
               }
               if (child instanceof NamedExpression) {
                   return new Alias(child, ((NamedExpression) child).getName());
               } else {
                   // TODO: resolve aliases
                   return new Alias(child, child.toSql());
               }
           }
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/RegisterCTEJob.java:
##########
@@ -0,0 +1,36 @@
+// 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.jobs.batch;
+
+import org.apache.doris.nereids.CascadesContext;
+import org.apache.doris.nereids.rules.analysis.RegisterCTE;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Register CTE
+ */
+public class RegisterCTEJob extends BatchRulesJob {
+
+    public RegisterCTEJob(CascadesContext cascadesContext) {
+        super(cascadesContext);
+        rulesJob.addAll(ImmutableList.of(
+                bottomUpBatch(ImmutableList.of(new RegisterCTE()))
+        ));
+    }
+}

Review Comment:
   I think you don't need add a BatchJob class. You can register the `RegisterCTE` in the top of `AnalyzeRulesJob`



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java:
##########
@@ -216,6 +227,26 @@ public List<Pair<LogicalPlan, StatementContext>> visitMultiStatements(MultiState
      * Plan parsing
      * ******************************************************************************************** */
 
+    /**
+     * process withClauses and store the results in a logical plan node LogicalCTE
+     */
+    public LogicalPlan withCte(CteContext ctx, LogicalPlan plan) {
+        return new LogicalCTE<>(visit(ctx.withClause(), WithClause.class), plan);
+    }
+
+    @Override
+    public WithClause visitWithClause(WithClauseContext ctx) {
+        return ParserUtils.withOrigin(ctx, () -> {
+            WithSubquery withSubquery = new WithSubquery(plan(ctx.query()));
+            List<String> columnNames = null;
+            if (ctx.columnAliases() != null) {
+                columnNames = ctx.columnAliases().identifier().stream()
+                    .map(id -> id.getText()).collect(Collectors.toList());
+            }
+            return new WithClause(ctx.identifier().getText(), withSubquery, Optional.ofNullable(columnNames));

Review Comment:
   WithClause sounds like a syntax tree node. Nereids not have any syntax tree node, so I think you should rename `WithClause` to `AliasQuery` which contains relation name, output column names and a `Plan`.
   So  `WithQuery` is useless and can be removed.
   `AliasQuery` can used not only in CTE, but also FromSubquery for future. 
   e.g.
   ```sql
   select *
   from (select ...) as table_alias (column1_alias, column2_alias, ...)
   ```
   



-- 
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