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/12/20 07:41:59 UTC

[GitHub] [doris] Jibing-Li opened a new pull request, #15201: [feature](multi catalog nereids)Add file scan node to nereids. WIP

Jibing-Li opened a new pull request, #15201:
URL: https://github.com/apache/doris/pull/15201

   # Proposed changes
   
   Issue Number: close #xxx
   
   ## Problem summary
   
   Add file scan node to nereids
   
   ## Checklist(Required)
   
   1. Does it affect the original behavior: 
       - [ ] Yes
       - [ ] No
       - [ ] I don't know
   2. Has unit tests been added:
       - [ ] Yes
       - [ ] No
       - [ ] No Need
   3. Has document been added or modified:
       - [ ] Yes
       - [ ] No
       - [ ] No Need
   4. Does it need to update dependencies:
       - [ ] Yes
       - [ ] No
   5. Are there any changes that cannot be rolled back:
       - [ ] Yes (If Yes, please explain WHY)
       - [ ] No
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at [dev@doris.apache.org](mailto:dev@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc...
   
   


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


[GitHub] [doris] 924060929 commented on a diff in pull request #15201: [feature](multi catalog nereids)Add file scan node to nereids.

Posted by GitBox <gi...@apache.org>.
924060929 commented on code in PR #15201:
URL: https://github.com/apache/doris/pull/15201#discussion_r1058035149


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java:
##########
@@ -82,48 +112,62 @@ private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, UnboundRe
             }
             return new LogicalSubQueryAlias<>(tableName, ctePlan);
         }
-
+        String catalogName = cascadesContext.getConnectContext().getCurrentCatalog().getName();
         String dbName = cascadesContext.getConnectContext().getDatabase();
-        Table table = cascadesContext.getTable(dbName, tableName, cascadesContext.getConnectContext().getEnv());
+        TableIf table = getTable(catalogName, dbName, tableName, cascadesContext.getConnectContext().getEnv());
         // TODO: should generate different Scan sub class according to table's type
-        List<Long> partIds = getPartitionIds(table, unboundRelation);
-        if (table.getType() == TableType.OLAP) {
-            if (!CollectionUtils.isEmpty(partIds)) {
-                return new LogicalOlapScan(RelationUtil.newRelationId(),
-                        (OlapTable) table, ImmutableList.of(dbName), partIds);
-            } else {
-                return new LogicalOlapScan(RelationUtil.newRelationId(),
-                        (OlapTable) table, ImmutableList.of(dbName));
-            }
-        } else if (table.getType() == TableType.VIEW) {
-            Plan viewPlan = parseAndAnalyzeView(table.getDdlSql(), cascadesContext);
-            return new LogicalSubQueryAlias<>(table.getName(), viewPlan);
-        }
-        throw new AnalysisException("Unsupported tableType:" + table.getType());
+        return getLogicalPlan(table, unboundRelation, dbName, cascadesContext);
     }
 
     private LogicalPlan bindWithDbNameFromNamePart(CascadesContext cascadesContext, UnboundRelation unboundRelation) {
         List<String> nameParts = unboundRelation.getNameParts();
         ConnectContext connectContext = cascadesContext.getConnectContext();
+        String catalogName = cascadesContext.getConnectContext().getCurrentCatalog().getName();
         // if the relation is view, nameParts.get(0) is dbName.
         String dbName = nameParts.get(0);
         if (!dbName.equals(connectContext.getDatabase())) {
             dbName = connectContext.getClusterName() + ":" + dbName;
         }
-        Table table = cascadesContext.getTable(dbName, nameParts.get(1), connectContext.getEnv());
-        List<Long> partIds = getPartitionIds(table, unboundRelation);
-        if (table.getType() == TableType.OLAP) {
-            if (!CollectionUtils.isEmpty(partIds)) {
-                return new LogicalOlapScan(RelationUtil.newRelationId(), (OlapTable) table,
-                        ImmutableList.of(dbName), partIds);
-            } else {
-                return new LogicalOlapScan(RelationUtil.newRelationId(), (OlapTable) table, ImmutableList.of(dbName));
-            }
-        } else if (table.getType() == TableType.VIEW) {
-            Plan viewPlan = parseAndAnalyzeView(table.getDdlSql(), cascadesContext);
-            return new LogicalSubQueryAlias<>(table.getName(), viewPlan);
+        String tableName = nameParts.get(1);
+        TableIf table = getTable(catalogName, dbName, tableName, connectContext.getEnv());
+        return getLogicalPlan(table, unboundRelation, dbName, cascadesContext);
+    }
+
+    private LogicalPlan bindWithCatalogNameFromNamePart(CascadesContext cascadesContext,
+                                                        UnboundRelation unboundRelation) {
+        List<String> nameParts = unboundRelation.getNameParts();
+        ConnectContext connectContext = cascadesContext.getConnectContext();
+        String catalogName = nameParts.get(0);
+        String dbName = nameParts.get(1);
+        if (!dbName.equals(connectContext.getDatabase())) {
+            dbName = connectContext.getClusterName() + ":" + dbName;
+        }
+        String tableName = nameParts.get(2);
+        TableIf table = getTable(catalogName, dbName, tableName, connectContext.getEnv());
+        return getLogicalPlan(table, unboundRelation, dbName, cascadesContext);
+    }
+
+    private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelation, String dbName,
+                                       CascadesContext cascadesContext) {
+        switch (table.getType()) {
+            case OLAP:
+                List<Long> partIds = getPartitionIds(table, unboundRelation);
+                if (!CollectionUtils.isEmpty(partIds)) {
+                    return new LogicalOlapScan(RelationUtil.newRelationId(),
+                        (OlapTable) table, ImmutableList.of(dbName), partIds);
+                } else {
+                    return new LogicalOlapScan(RelationUtil.newRelationId(),
+                        (OlapTable) table, ImmutableList.of(dbName));
+                }
+            case VIEW:
+                Plan viewPlan = parseAndAnalyzeView(((View) table).getDdlSql(), cascadesContext);
+                return new LogicalSubQueryAlias<>(table.getName(), viewPlan);
+            case HMS_EXTERNAL_TABLE:
+                return new LogicalFileScan(cascadesContext.getStatementContext().getNextRelationId(),
+                    (HMSExternalTable) table, ImmutableList.of(dbName));

Review Comment:
   I suggest rename LogicalFileScan to LogicalExternalRelation(with field: HMSExternalTable table) or LogicalHMSTable



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


[GitHub] [doris] github-actions[bot] commented on pull request #15201: [feature](multi catalog nereids)Add file scan node to nereids.

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #15201:
URL: https://github.com/apache/doris/pull/15201#issuecomment-1367033028

   PR approved by at least one committer and no changes requested.


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


[GitHub] [doris] morningman commented on a diff in pull request #15201: [feature](multi catalog nereids)Add file scan node to nereids.

Posted by GitBox <gi...@apache.org>.
morningman commented on code in PR #15201:
URL: https://github.com/apache/doris/pull/15201#discussion_r1057730020


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java:
##########
@@ -491,6 +495,29 @@ public PlanFragment visitPhysicalOlapScan(PhysicalOlapScan olapScan, PlanTransla
         return planFragment;
     }
 
+    @Override
+    public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTranslatorContext context) {
+        // Create OlapScanNode

Review Comment:
   ExternalFileScanNode



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java:
##########
@@ -63,12 +68,37 @@ public Rule build() {
                     // Use database name from table name parts.
                     return bindWithDbNameFromNamePart(ctx.cascadesContext, ctx.root);
                 }
+                case 3: { // catalog.db.table
+                    // Use catalog and database name from name parts.
+                    return bindWithCatalogNameFromNamePart(ctx.cascadesContext, ctx.root);
+                }
                 default:
                     throw new IllegalStateException("Table name [" + ctx.root.getTableName() + "] is invalid.");
             }
         }).toRule(RuleType.BINDING_RELATION);
     }
 
+    private TableIf getTable(String catalogName, String dbName, String tableName, Env env) {
+        CatalogIf catalog = env.getCatalogMgr().getCatalog(catalogName);
+        if (catalog == null) {
+            throw new RuntimeException(String.format("Catalog %s does not exist.", catalogName));
+        }
+        DatabaseIf<TableIf> db = null;
+        try {
+            db = (DatabaseIf<TableIf>) catalog.getDb(dbName)
+                    .orElseThrow(() -> new RuntimeException("Database [" + dbName + "] does not exist."));
+        } catch (Throwable e) {
+            throw new RuntimeException(e);
+        }
+        db.readLock();

Review Comment:
   No need to add db lock



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


[GitHub] [doris] github-actions[bot] commented on pull request #15201: [feature](multi catalog nereids)Add file scan node to nereids.

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #15201:
URL: https://github.com/apache/doris/pull/15201#issuecomment-1367033044

   PR approved by anyone and no changes requested.


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


[GitHub] [doris] Jibing-Li commented on a diff in pull request #15201: [feature](multi catalog nereids)Add file scan node to nereids.

Posted by GitBox <gi...@apache.org>.
Jibing-Li commented on code in PR #15201:
URL: https://github.com/apache/doris/pull/15201#discussion_r1058432368


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java:
##########
@@ -82,48 +112,62 @@ private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, UnboundRe
             }
             return new LogicalSubQueryAlias<>(tableName, ctePlan);
         }
-
+        String catalogName = cascadesContext.getConnectContext().getCurrentCatalog().getName();
         String dbName = cascadesContext.getConnectContext().getDatabase();
-        Table table = cascadesContext.getTable(dbName, tableName, cascadesContext.getConnectContext().getEnv());
+        TableIf table = getTable(catalogName, dbName, tableName, cascadesContext.getConnectContext().getEnv());
         // TODO: should generate different Scan sub class according to table's type
-        List<Long> partIds = getPartitionIds(table, unboundRelation);
-        if (table.getType() == TableType.OLAP) {
-            if (!CollectionUtils.isEmpty(partIds)) {
-                return new LogicalOlapScan(RelationUtil.newRelationId(),
-                        (OlapTable) table, ImmutableList.of(dbName), partIds);
-            } else {
-                return new LogicalOlapScan(RelationUtil.newRelationId(),
-                        (OlapTable) table, ImmutableList.of(dbName));
-            }
-        } else if (table.getType() == TableType.VIEW) {
-            Plan viewPlan = parseAndAnalyzeView(table.getDdlSql(), cascadesContext);
-            return new LogicalSubQueryAlias<>(table.getName(), viewPlan);
-        }
-        throw new AnalysisException("Unsupported tableType:" + table.getType());
+        return getLogicalPlan(table, unboundRelation, dbName, cascadesContext);
     }
 
     private LogicalPlan bindWithDbNameFromNamePart(CascadesContext cascadesContext, UnboundRelation unboundRelation) {
         List<String> nameParts = unboundRelation.getNameParts();
         ConnectContext connectContext = cascadesContext.getConnectContext();
+        String catalogName = cascadesContext.getConnectContext().getCurrentCatalog().getName();
         // if the relation is view, nameParts.get(0) is dbName.
         String dbName = nameParts.get(0);
         if (!dbName.equals(connectContext.getDatabase())) {
             dbName = connectContext.getClusterName() + ":" + dbName;
         }
-        Table table = cascadesContext.getTable(dbName, nameParts.get(1), connectContext.getEnv());
-        List<Long> partIds = getPartitionIds(table, unboundRelation);
-        if (table.getType() == TableType.OLAP) {
-            if (!CollectionUtils.isEmpty(partIds)) {
-                return new LogicalOlapScan(RelationUtil.newRelationId(), (OlapTable) table,
-                        ImmutableList.of(dbName), partIds);
-            } else {
-                return new LogicalOlapScan(RelationUtil.newRelationId(), (OlapTable) table, ImmutableList.of(dbName));
-            }
-        } else if (table.getType() == TableType.VIEW) {
-            Plan viewPlan = parseAndAnalyzeView(table.getDdlSql(), cascadesContext);
-            return new LogicalSubQueryAlias<>(table.getName(), viewPlan);
+        String tableName = nameParts.get(1);
+        TableIf table = getTable(catalogName, dbName, tableName, connectContext.getEnv());
+        return getLogicalPlan(table, unboundRelation, dbName, cascadesContext);
+    }
+
+    private LogicalPlan bindWithCatalogNameFromNamePart(CascadesContext cascadesContext,
+                                                        UnboundRelation unboundRelation) {
+        List<String> nameParts = unboundRelation.getNameParts();
+        ConnectContext connectContext = cascadesContext.getConnectContext();
+        String catalogName = nameParts.get(0);
+        String dbName = nameParts.get(1);
+        if (!dbName.equals(connectContext.getDatabase())) {
+            dbName = connectContext.getClusterName() + ":" + dbName;
+        }
+        String tableName = nameParts.get(2);
+        TableIf table = getTable(catalogName, dbName, tableName, connectContext.getEnv());
+        return getLogicalPlan(table, unboundRelation, dbName, cascadesContext);
+    }
+
+    private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelation, String dbName,
+                                       CascadesContext cascadesContext) {
+        switch (table.getType()) {
+            case OLAP:
+                List<Long> partIds = getPartitionIds(table, unboundRelation);
+                if (!CollectionUtils.isEmpty(partIds)) {
+                    return new LogicalOlapScan(RelationUtil.newRelationId(),
+                        (OlapTable) table, ImmutableList.of(dbName), partIds);
+                } else {
+                    return new LogicalOlapScan(RelationUtil.newRelationId(),
+                        (OlapTable) table, ImmutableList.of(dbName));
+                }
+            case VIEW:
+                Plan viewPlan = parseAndAnalyzeView(((View) table).getDdlSql(), cascadesContext);
+                return new LogicalSubQueryAlias<>(table.getName(), viewPlan);
+            case HMS_EXTERNAL_TABLE:
+                return new LogicalFileScan(cascadesContext.getStatementContext().getNextRelationId(),
+                    (HMSExternalTable) table, ImmutableList.of(dbName));

Review Comment:
   I'll take a look at this, probably fix it in next PR.



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


[GitHub] [doris] morningman merged pull request #15201: [feature](multi catalog nereids)Add file scan node to nereids.

Posted by GitBox <gi...@apache.org>.
morningman merged PR #15201:
URL: https://github.com/apache/doris/pull/15201


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


[GitHub] [doris] hello-stephen commented on pull request #15201: [feature](multi catalog nereids)Add file scan node to nereids. WIP

Posted by GitBox <gi...@apache.org>.
hello-stephen commented on PR #15201:
URL: https://github.com/apache/doris/pull/15201#issuecomment-1359010822

   TeamCity pipeline, clickbench performance test result:
    the sum of best hot time: 34.62 seconds
    load time: 671 seconds
    storage size: 17123660262 Bytes
    https://doris-community-test-1308700295.cos.ap-hongkong.myqcloud.com/tmp/20221220084030_clickbench_pr_65541.html


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