You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by yi...@apache.org on 2022/07/10 15:51:42 UTC

[doris] branch master updated: [Bug] CTAS varchar length lost (#10738)

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

yiguolei 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 46662bfee8 [Bug] CTAS varchar length lost (#10738)
46662bfee8 is described below

commit 46662bfee818a87380e5a2e4f4532c9926d164ac
Author: Stalary <st...@163.com>
AuthorDate: Sun Jul 10 23:51:36 2022 +0800

    [Bug] CTAS varchar length lost (#10738)
---
 .../Create/CREATE-TABLE-AS-SELECT.md               |   1 -
 .../Create/CREATE-TABLE-AS-SELECT.md               |   1 -
 .../doris/analysis/CreateTableAsSelectStmt.java    |   6 -
 .../java/org/apache/doris/catalog/ScalarType.java  |   1 +
 .../doris/datasource/InternalDataSource.java       |   9 +-
 .../analysis/CreateTableAsSelectStmtTest.java      | 420 +++++++--------------
 .../apache/doris/utframe/TestWithFeService.java    |  12 +
 7 files changed, 166 insertions(+), 284 deletions(-)

diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE-AS-SELECT.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE-AS-SELECT.md
index 2c40305867..e72329e8fa 100644
--- a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE-AS-SELECT.md
+++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE-AS-SELECT.md
@@ -43,7 +43,6 @@ opt_engine opt_partition opt_properties KW_AS query_stmt
 
 illustrate: 
 
-- Fields of type`decimal`are not currently supported
 - The user needs to have`SELECT`permission for the source table and`CREATE`permission for the target database
 - After a table is created, data is imported. If the import fails, the table is deleted
 
diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE-AS-SELECT.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE-AS-SELECT.md
index 21f15a3a8b..d10d252ff7 100644
--- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE-AS-SELECT.md
+++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE-AS-SELECT.md
@@ -43,7 +43,6 @@ opt_engine opt_partition opt_properties KW_AS query_stmt
 
 说明: 
 
-- 暂时不支持`decimal`类型的字段
 - 用户需要拥有来源表的`SELECT`权限和目标库的`CREATE`权限
 - 创建表成功后,会进行数据导入,如果导入失败,将会删除表
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableAsSelectStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableAsSelectStmt.java
index 235e497e80..5b682a06b2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableAsSelectStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableAsSelectStmt.java
@@ -67,12 +67,6 @@ public class CreateTableAsSelectStmt extends DdlStmt {
         tmpStmt.analyze(dummyRootAnalyzer);
         this.queryStmt = tmpStmt;
         ArrayList<Expr> resultExprs = getQueryStmt().getResultExprs();
-        // TODO: support decimal
-        for (Expr expr : resultExprs) {
-            if (expr.getType().isDecimalV2()) {
-                ErrorReport.reportAnalysisException(ErrorCode.ERR_UNSUPPORTED_TYPE_IN_CTAS, expr.getType());
-            }
-        }
         if (columnNames != null && columnNames.size() != resultExprs.size()) {
             ErrorReport.reportAnalysisException(ErrorCode.ERR_COL_NUMBER_NOT_MATCH);
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarType.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarType.java
index 601e7d833f..2cced324a5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarType.java
@@ -83,6 +83,7 @@ public class ScalarType extends Type {
     // Only used for type CHAR.
     @SerializedName(value = "len")
     private int len = -1;
+    @SerializedName(value = "isAssignedStrLenInColDefinition")
     private boolean isAssignedStrLenInColDefinition = false;
 
     // Only used if type is DECIMAL. -1 (for both) is used to represent a
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java
index 90797a0d66..332fa85078 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java
@@ -97,6 +97,7 @@ import org.apache.doris.catalog.RangePartitionItem;
 import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.Replica.ReplicaState;
 import org.apache.doris.catalog.ReplicaAllocation;
+import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.SinglePartitionInfo;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.TableIf;
@@ -1101,6 +1102,9 @@ public class InternalDataSource implements DataSourceIf<Database> {
         }
     }
 
+    /**
+     * Create table for select.
+     **/
     public void createTableAsSelect(CreateTableAsSelectStmt stmt) throws DdlException {
         try {
             List<String> columnNames = stmt.getColumnNames();
@@ -1126,8 +1130,11 @@ public class InternalDataSource implements DataSourceIf<Database> {
                 }
                 TypeDef typeDef;
                 Expr resultExpr = resultExprs.get(i);
-                if (resultExpr.getType().isStringType() && resultExpr.getType().getLength() < 0) {
+                Type resultType = resultExpr.getType();
+                if (resultType.isStringType() && resultType.getLength() < 0) {
                     typeDef = new TypeDef(Type.STRING);
+                } else if (resultType.isDecimalV2() && resultType.equals(ScalarType.DECIMALV2)) {
+                    typeDef = new TypeDef(ScalarType.createDecimalV2Type(27, 9));
                 } else {
                     typeDef = new TypeDef(resultExpr.getType());
                 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableAsSelectStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableAsSelectStmtTest.java
index 08b9b7dae0..88151b9c65 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableAsSelectStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableAsSelectStmtTest.java
@@ -17,113 +17,67 @@
 
 package org.apache.doris.analysis;
 
-import org.apache.doris.catalog.Catalog;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.ExceptionChecker;
-import org.apache.doris.datasource.InternalDataSource;
-import org.apache.doris.metric.MetricRepo;
-import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.qe.ShowExecutor;
 import org.apache.doris.qe.ShowResultSet;
-import org.apache.doris.utframe.UtFrameUtils;
+import org.apache.doris.utframe.TestWithFeService;
 
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.UUID;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 
 /**
  * test for CreateTableAsSelectStmt.
  **/
-public class CreateTableAsSelectStmtTest {
-
-    private static String runningDir = "fe/mocked/CreateTableAsSelectStmtTest/" + UUID.randomUUID() + "/";
-    private static ConnectContext connectContext;
-
-    /**
-     * clean file.
-     **/
-    @AfterClass
-    public static void tearDown() throws Exception {
-        UtFrameUtils.cleanDorisFeDir(runningDir);
-    }
+public class CreateTableAsSelectStmtTest extends TestWithFeService {
 
-    /**
-     * init.
-     **/
-    @BeforeClass
-    public static void setUp() throws Exception {
-        UtFrameUtils.createDorisCluster(runningDir);
-        // create connect context
-        connectContext = UtFrameUtils.createDefaultCtx();
-        // create database
-        String createDbStmtStr = "create database test";
-        CreateDbStmt createDbStmt = (CreateDbStmt) UtFrameUtils.parseAndAnalyzeStmt(createDbStmtStr, connectContext);
-        Catalog.getCurrentCatalog().createDb(createDbStmt);
+    @Override
+    protected void runBeforeAll() throws Exception {
+        createDatabase("test");
+        useDatabase("test");
         String varcharTable = "CREATE TABLE `test`.`varchar_table`\n" + "(\n"
-                + "    `userId`   varchar(255) NOT NULL,\n"
-                + "    `username` varchar(255) NOT NULL\n"
-                + ") ENGINE = OLAP unique KEY(`userId`)\n"
-                + "COMMENT 'varchar_table'\n"
-                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")";
-        MetricRepo.init();
+                + "    `userId`   varchar(255) NOT NULL,\n" + "    `username` varchar(255) NOT NULL\n"
+                + ") ENGINE = OLAP unique KEY(`userId`)\n" + "COMMENT 'varchar_table'\n"
+                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n" + "PROPERTIES (\n"
+                + "\"replication_allocation\" = \"tag.location.default: 1\",\n" + "\"in_memory\" = \"false\",\n"
+                + "\"storage_format\" = \"V2\"\n" + ")";
         createTable(varcharTable);
-        String decimalTable = "CREATE TABLE `test`.`decimal_table`\n"
-                + "(\n"
-                + "    `userId`   varchar(255) NOT NULL,\n"
-                + "    `amount_decimal` decimal(10, 2) NOT NULL\n"
-                + ") ENGINE = OLAP unique KEY(`userId`)\n"
-                + "COMMENT 'decimal_table'\n"
-                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
+        String decimalTable = "CREATE TABLE `test`.`decimal_table`\n" + "(\n"
+                + "    `userId`   varchar(255) NOT NULL,\n" + "    `amount_decimal` decimal(10, 2) NOT NULL\n"
+                + ") ENGINE = OLAP unique KEY(`userId`)\n" + "COMMENT 'decimal_table'\n"
+                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n" + "PROPERTIES (\n"
+                + "\"replication_allocation\" = \"tag.location.default: 1\",\n" + "\"in_memory\" = \"false\",\n"
                 + "\"storage_format\" = \"V2\")";
         createTable(decimalTable);
         String joinTable = "CREATE TABLE `test`.`join_table` (`userId`   varchar(255) NOT NULL,"
-                + " `status`   int NOT NULL)"
-                + " ENGINE = OLAP unique KEY(`userId`)\n"
-                + "COMMENT 'join_table'\n"
-                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
+                + " `status`   int NOT NULL)" + " ENGINE = OLAP unique KEY(`userId`)\n" + "COMMENT 'join_table'\n"
+                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n" + "PROPERTIES (\n"
+                + "\"replication_allocation\" = \"tag.location.default: 1\",\n" + "\"in_memory\" = \"false\",\n"
                 + "\"storage_format\" = \"V2\"\n)";
         createTable(joinTable);
     }
 
-    private static void createTable(String sql) throws Exception {
-        CreateTableStmt stmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext);
-        Catalog.getCurrentCatalog().createTable(stmt);
-    }
-
-    private static void createTableAsSelect(String sql) throws Exception {
-        CreateTableAsSelectStmt stmt = (CreateTableAsSelectStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext);
-        Catalog.getCurrentCatalog().createTableAsSelect(stmt);
-    }
-
-    private static ShowResultSet showCreateTable(String tableName) throws Exception {
-        ShowCreateTableStmt stmt = new ShowCreateTableStmt(new TableName(InternalDataSource.INTERNAL_DS_NAME, "test", tableName));
-        Analyzer dummyRootAnalyzer = new Analyzer(Catalog.getCurrentCatalog(), connectContext);
-        stmt.analyze(dummyRootAnalyzer);
-        ShowExecutor executor = new ShowExecutor(connectContext, stmt);
-        return executor.execute();
-    }
-
     @Test
-    public void testErrorType() {
-        String selectFromDecimal =
-                "create table `test`.`select_decimal_table` PROPERTIES(\"replication_num\" = \"1\") "
-                        + "as select * from `test`.`decimal_table`";
-        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "Unsupported type",
-                () -> UtFrameUtils.parseAndAnalyzeStmt(selectFromDecimal, connectContext));
+    public void testDecimal() throws Exception {
+        String selectFromDecimal = "create table `test`.`select_decimal_table` PROPERTIES(\"replication_num\" = \"1\") "
+                + "as select * from `test`.`decimal_table`";
+        createTableAsSelect(selectFromDecimal);
+        Assertions.assertEquals("CREATE TABLE `select_decimal_table` (\n" + "  `userId` varchar(255) NOT NULL,\n"
+                        + "  `amount_decimal` decimal(10, 2) REPLACE NOT NULL\n" + ") ENGINE=OLAP\n"
+                        + "AGGREGATE KEY(`userId`)\n" + "COMMENT 'OLAP'\n"
+                        + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n"
+                        + "PROPERTIES (\n" + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
+                        + "\"in_memory\" = \"false\",\n" + "\"storage_format\" = \"V2\"\n" + ")",
+                showCreateTableByName("select_decimal_table").getResultRows().get(0).get(1));
+        String selectFromDecimal1 =
+                "create table `test`.`select_decimal_table_1` PROPERTIES(\"replication_num\" = \"1\") "
+                        + "as select sum(amount_decimal) from `test`.`decimal_table`";
+        createTableAsSelect(selectFromDecimal1);
+        Assertions.assertEquals(
+                "CREATE TABLE `select_decimal_table_1` (\n" + "  `_col0` decimal(27, 9) NULL\n" + ") ENGINE=OLAP\n"
+                        + "DUPLICATE KEY(`_col0`)\n" + "COMMENT 'OLAP'\n" + "DISTRIBUTED BY HASH(`_col0`) BUCKETS 10\n"
+                        + "PROPERTIES (\n" + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
+                        + "\"in_memory\" = \"false\",\n" + "\"storage_format\" = \"V2\"\n" + ")",
+                showCreateTableByName("select_decimal_table_1").getResultRows().get(0).get(1));
     }
 
     @Test
@@ -133,107 +87,71 @@ public class CreateTableAsSelectStmtTest {
                         + "as select * from `test`.`varchar_table`";
         ExceptionChecker.expectThrowsWithMsg(AnalysisException.class,
                 "Number of columns don't equal number of SELECT statement's select list",
-                () -> UtFrameUtils.parseAndAnalyzeStmt(selectFromColumn, connectContext));
+                () -> parseAndAnalyzeStmt(selectFromColumn, connectContext));
     }
 
     @Test
     public void testVarchar() throws Exception {
-        String selectFromVarchar =
-                "create table `test`.`select_varchar` PROPERTIES(\"replication_num\" = \"1\") "
-                        + "as select * from `test`.`varchar_table`";
+        String selectFromVarchar = "create table `test`.`select_varchar` PROPERTIES(\"replication_num\" = \"1\") "
+                + "as select * from `test`.`varchar_table`";
         createTableAsSelect(selectFromVarchar);
-        ShowResultSet showResultSet = showCreateTable("select_varchar");
-        Assert.assertEquals("CREATE TABLE `select_varchar` (\n"
-                + "  `userId` varchar(255) NOT NULL,\n"
-                + "  `username` varchar(255) REPLACE NOT NULL\n"
-                + ") ENGINE=OLAP\n"
-                + "AGGREGATE KEY(`userId`)\n"
-                + "COMMENT 'OLAP'\n"
-                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")", showResultSet.getResultRows().get(0).get(1));
+        ShowResultSet showResultSet = showCreateTableByName("select_varchar");
+        Assertions.assertEquals("CREATE TABLE `select_varchar` (\n" + "  `userId` varchar(255) NOT NULL,\n"
+                + "  `username` varchar(255) REPLACE NOT NULL\n" + ") ENGINE=OLAP\n" + "AGGREGATE KEY(`userId`)\n"
+                + "COMMENT 'OLAP'\n" + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n" + "PROPERTIES (\n"
+                + "\"replication_allocation\" = \"tag.location.default: 1\",\n" + "\"in_memory\" = \"false\",\n"
+                + "\"storage_format\" = \"V2\"\n" + ")", showResultSet.getResultRows().get(0).get(1));
     }
 
     @Test
     public void testFunction() throws Exception {
-        String selectFromFunction1 =
-                "create table `test`.`select_function_1` PROPERTIES(\"replication_num\" = \"1\") "
-                        + "as select count(*) from `test`.`varchar_table`";
+        String selectFromFunction1 = "create table `test`.`select_function_1` PROPERTIES(\"replication_num\" = \"1\") "
+                + "as select count(*) from `test`.`varchar_table`";
         createTableAsSelect(selectFromFunction1);
-        ShowResultSet showResultSet1 = showCreateTable("select_function_1");
-        Assert.assertEquals("CREATE TABLE `select_function_1` (\n"
-                + "  `_col0` bigint(20) NULL\n"
-                + ") ENGINE=OLAP\n"
-                + "DUPLICATE KEY(`_col0`)\n"
-                + "COMMENT 'OLAP'\n"
-                + "DISTRIBUTED BY HASH(`_col0`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")", showResultSet1.getResultRows().get(0).get(1));
-
-        String selectFromFunction2 =
-                "create table `test`.`select_function_2` PROPERTIES(\"replication_num\" = \"1\") "
-                        + "as select sum(status), sum(status), sum(status), count(status), count(status) "
-                        + "from `test`.`join_table`";
+        ShowResultSet showResultSet1 = showCreateTableByName("select_function_1");
+        Assertions.assertEquals(
+                "CREATE TABLE `select_function_1` (\n" + "  `_col0` bigint(20) NULL\n" + ") ENGINE=OLAP\n"
+                        + "DUPLICATE KEY(`_col0`)\n" + "COMMENT 'OLAP'\n" + "DISTRIBUTED BY HASH(`_col0`) BUCKETS 10\n"
+                        + "PROPERTIES (\n" + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
+                        + "\"in_memory\" = \"false\",\n" + "\"storage_format\" = \"V2\"\n" + ")",
+                showResultSet1.getResultRows().get(0).get(1));
+
+        String selectFromFunction2 = "create table `test`.`select_function_2` PROPERTIES(\"replication_num\" = \"1\") "
+                + "as select sum(status), sum(status), sum(status), count(status), count(status) "
+                + "from `test`.`join_table`";
         createTableAsSelect(selectFromFunction2);
-        ShowResultSet showResultSet2 = showCreateTable("select_function_2");
-        Assert.assertEquals("CREATE TABLE `select_function_2` (\n"
-                + "  `_col0` bigint(20) NULL,\n"
-                + "  `_col1` bigint(20) NULL,\n"
-                + "  `_col2` bigint(20) NULL,\n"
-                + "  `_col3` bigint(20) NULL,\n"
-                + "  `_col4` bigint(20) NULL\n"
-                + ") ENGINE=OLAP\n"
-                + "DUPLICATE KEY(`_col0`, `_col1`, `_col2`)\n"
-                + "COMMENT 'OLAP'\n"
-                + "DISTRIBUTED BY HASH(`_col0`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")", showResultSet2.getResultRows().get(0).get(1));
+        ShowResultSet showResultSet2 = showCreateTableByName("select_function_2");
+        Assertions.assertEquals(
+                "CREATE TABLE `select_function_2` (\n" + "  `_col0` bigint(20) NULL,\n" + "  `_col1` bigint(20) NULL,\n"
+                        + "  `_col2` bigint(20) NULL,\n" + "  `_col3` bigint(20) NULL,\n"
+                        + "  `_col4` bigint(20) NULL\n" + ") ENGINE=OLAP\n"
+                        + "DUPLICATE KEY(`_col0`, `_col1`, `_col2`)\n" + "COMMENT 'OLAP'\n"
+                        + "DISTRIBUTED BY HASH(`_col0`) BUCKETS 10\n" + "PROPERTIES (\n"
+                        + "\"replication_allocation\" = \"tag.location.default: 1\",\n" + "\"in_memory\" = \"false\",\n"
+                        + "\"storage_format\" = \"V2\"\n" + ")", showResultSet2.getResultRows().get(0).get(1));
     }
 
     @Test
     public void testAlias() throws Exception {
-        String selectAlias1 =
-                "create table `test`.`select_alias_1` PROPERTIES(\"replication_num\" = \"1\") "
-                        + "as select count(*) as amount from `test`.`varchar_table`";
+        String selectAlias1 = "create table `test`.`select_alias_1` PROPERTIES(\"replication_num\" = \"1\") "
+                + "as select count(*) as amount from `test`.`varchar_table`";
         createTableAsSelect(selectAlias1);
-        ShowResultSet showResultSet1 = showCreateTable("select_alias_1");
-        Assert.assertEquals("CREATE TABLE `select_alias_1` (\n"
-                + "  `amount` bigint(20) NULL\n"
-                + ") ENGINE=OLAP\n"
-                + "DUPLICATE KEY(`amount`)\n"
-                + "COMMENT 'OLAP'\n"
-                + "DISTRIBUTED BY HASH(`amount`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")", showResultSet1.getResultRows().get(0).get(1));
-        String selectAlias2 =
-                "create table `test`.`select_alias_2` PROPERTIES(\"replication_num\" = \"1\") "
-                        + "as select userId as alias_name, username from `test`.`varchar_table`";
+        ShowResultSet showResultSet1 = showCreateTableByName("select_alias_1");
+        Assertions.assertEquals("CREATE TABLE `select_alias_1` (\n" + "  `amount` bigint(20) NULL\n" + ") ENGINE=OLAP\n"
+                        + "DUPLICATE KEY(`amount`)\n" + "COMMENT 'OLAP'\n"
+                        + "DISTRIBUTED BY HASH(`amount`) BUCKETS 10\n"
+                        + "PROPERTIES (\n" + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
+                        + "\"in_memory\" = \"false\",\n" + "\"storage_format\" = \"V2\"\n" + ")",
+                showResultSet1.getResultRows().get(0).get(1));
+        String selectAlias2 = "create table `test`.`select_alias_2` PROPERTIES(\"replication_num\" = \"1\") "
+                + "as select userId as alias_name, username from `test`.`varchar_table`";
         createTableAsSelect(selectAlias2);
-        ShowResultSet showResultSet2 = showCreateTable("select_alias_2");
-        Assert.assertEquals("CREATE TABLE `select_alias_2` (\n"
-                + "  `alias_name` varchar(255) NOT NULL,\n"
-                + "  `username` varchar(255) REPLACE NOT NULL\n"
-                + ") ENGINE=OLAP\n"
-                + "AGGREGATE KEY(`alias_name`)\n"
-                + "COMMENT 'OLAP'\n"
-                + "DISTRIBUTED BY HASH(`alias_name`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")", showResultSet2.getResultRows().get(0).get(1));
+        ShowResultSet showResultSet2 = showCreateTableByName("select_alias_2");
+        Assertions.assertEquals("CREATE TABLE `select_alias_2` (\n" + "  `alias_name` varchar(255) NOT NULL,\n"
+                + "  `username` varchar(255) REPLACE NOT NULL\n" + ") ENGINE=OLAP\n" + "AGGREGATE KEY(`alias_name`)\n"
+                + "COMMENT 'OLAP'\n" + "DISTRIBUTED BY HASH(`alias_name`) BUCKETS 10\n" + "PROPERTIES (\n"
+                + "\"replication_allocation\" = \"tag.location.default: 1\",\n" + "\"in_memory\" = \"false\",\n"
+                + "\"storage_format\" = \"V2\"\n" + ")", showResultSet2.getResultRows().get(0).get(1));
     }
 
     @Test
@@ -242,40 +160,24 @@ public class CreateTableAsSelectStmtTest {
                 + "as select vt.userId, vt.username, jt.status from `test`.`varchar_table` vt "
                 + "join `test`.`join_table` jt on vt.userId=jt.userId";
         createTableAsSelect(selectFromJoin);
-        ShowResultSet showResultSet = showCreateTable("select_join");
-        Assert.assertEquals("CREATE TABLE `select_join` (\n"
-                + "  `userId` varchar(255) NOT NULL,\n"
-                + "  `username` varchar(255) REPLACE NOT NULL,\n"
-                + "  `status` int(11) REPLACE NOT NULL\n"
-                + ") ENGINE=OLAP\n"
-                + "AGGREGATE KEY(`userId`)\n"
-                + "COMMENT 'OLAP'\n"
-                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")", showResultSet.getResultRows().get(0).get(1));
+        ShowResultSet showResultSet = showCreateTableByName("select_join");
+        Assertions.assertEquals("CREATE TABLE `select_join` (\n" + "  `userId` varchar(255) NOT NULL,\n"
+                + "  `username` varchar(255) REPLACE NOT NULL,\n" + "  `status` int(11) REPLACE NOT NULL\n"
+                + ") ENGINE=OLAP\n" + "AGGREGATE KEY(`userId`)\n" + "COMMENT 'OLAP'\n"
+                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n" + "PROPERTIES (\n"
+                + "\"replication_allocation\" = \"tag.location.default: 1\",\n" + "\"in_memory\" = \"false\",\n"
+                + "\"storage_format\" = \"V2\"\n" + ")", showResultSet.getResultRows().get(0).get(1));
         String selectFromJoin1 = "create table `test`.`select_join1` PROPERTIES(\"replication_num\" = \"1\") "
                 + "as select vt.userId as userId1, jt.userId as userId2, vt.username, jt.status "
-                + "from `test`.`varchar_table` vt "
-                + "join `test`.`join_table` jt on vt.userId=jt.userId";
+                + "from `test`.`varchar_table` vt " + "join `test`.`join_table` jt on vt.userId=jt.userId";
         createTableAsSelect(selectFromJoin1);
-        ShowResultSet showResultSet1 = showCreateTable("select_join1");
-        Assert.assertEquals("CREATE TABLE `select_join1` (\n"
-                + "  `userId1` varchar(255) NOT NULL,\n"
-                + "  `userId2` varchar(255) NOT NULL,\n"
-                + "  `username` varchar(255) REPLACE NOT NULL,\n"
-                + "  `status` int(11) REPLACE NOT NULL\n"
-                + ") ENGINE=OLAP\n"
-                + "AGGREGATE KEY(`userId1`, `userId2`)\n"
-                + "COMMENT 'OLAP'\n"
-                + "DISTRIBUTED BY HASH(`userId1`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")", showResultSet1.getResultRows().get(0).get(1));
+        ShowResultSet showResultSet1 = showCreateTableByName("select_join1");
+        Assertions.assertEquals("CREATE TABLE `select_join1` (\n" + "  `userId1` varchar(255) NOT NULL,\n"
+                + "  `userId2` varchar(255) NOT NULL,\n" + "  `username` varchar(255) REPLACE NOT NULL,\n"
+                + "  `status` int(11) REPLACE NOT NULL\n" + ") ENGINE=OLAP\n" + "AGGREGATE KEY(`userId1`, `userId2`)\n"
+                + "COMMENT 'OLAP'\n" + "DISTRIBUTED BY HASH(`userId1`) BUCKETS 10\n" + "PROPERTIES (\n"
+                + "\"replication_allocation\" = \"tag.location.default: 1\",\n" + "\"in_memory\" = \"false\",\n"
+                + "\"storage_format\" = \"V2\"\n" + ")", showResultSet1.getResultRows().get(0).get(1));
     }
 
     @Test
@@ -285,20 +187,13 @@ public class CreateTableAsSelectStmtTest {
                         + "as select vt.userId, vt.username, jt.status from `test`.`varchar_table` vt "
                         + "join `test`.`join_table` jt on vt.userId=jt.userId";
         createTableAsSelect(selectFromName);
-        ShowResultSet showResultSet = showCreateTable("select_name");
-        Assert.assertEquals("CREATE TABLE `select_name` (\n"
-                + "  `user` varchar(255) NOT NULL,\n"
-                + "  `testname` varchar(255) REPLACE NOT NULL,\n"
-                + "  `userstatus` int(11) REPLACE NOT NULL\n"
-                + ") ENGINE=OLAP\n"
-                + "AGGREGATE KEY(`user`)\n"
-                + "COMMENT 'OLAP'\n"
-                + "DISTRIBUTED BY HASH(`user`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")", showResultSet.getResultRows().get(0).get(1));
+        ShowResultSet showResultSet = showCreateTableByName("select_name");
+        Assertions.assertEquals("CREATE TABLE `select_name` (\n" + "  `user` varchar(255) NOT NULL,\n"
+                + "  `testname` varchar(255) REPLACE NOT NULL,\n" + "  `userstatus` int(11) REPLACE NOT NULL\n"
+                + ") ENGINE=OLAP\n" + "AGGREGATE KEY(`user`)\n" + "COMMENT 'OLAP'\n"
+                + "DISTRIBUTED BY HASH(`user`) BUCKETS 10\n" + "PROPERTIES (\n"
+                + "\"replication_allocation\" = \"tag.location.default: 1\",\n" + "\"in_memory\" = \"false\",\n"
+                + "\"storage_format\" = \"V2\"\n" + ")", showResultSet.getResultRows().get(0).get(1));
     }
 
     @Test
@@ -306,18 +201,13 @@ public class CreateTableAsSelectStmtTest {
         String selectFromName = "create table `test`.`select_union` PROPERTIES(\"replication_num\" = \"1\") "
                 + "as select userId  from `test`.`varchar_table` union select userId from `test`.`join_table`";
         createTableAsSelect(selectFromName);
-        ShowResultSet showResultSet = showCreateTable("select_union");
-        Assert.assertEquals("CREATE TABLE `select_union` (\n"
-                + "  `userId` varchar(255) NULL\n"
-                + ") ENGINE=OLAP\n"
-                + "DUPLICATE KEY(`userId`)\n"
-                + "COMMENT 'OLAP'\n"
-                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")", showResultSet.getResultRows().get(0).get(1));
+        ShowResultSet showResultSet = showCreateTableByName("select_union");
+        Assertions.assertEquals("CREATE TABLE `select_union` (\n" + "  `userId` varchar(255) NULL\n" + ") ENGINE=OLAP\n"
+                        + "DUPLICATE KEY(`userId`)\n" + "COMMENT 'OLAP'\n"
+                        + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n"
+                        + "PROPERTIES (\n" + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
+                        + "\"in_memory\" = \"false\",\n" + "\"storage_format\" = \"V2\"\n" + ")",
+                showResultSet.getResultRows().get(0).get(1));
     }
 
     @Test
@@ -325,58 +215,38 @@ public class CreateTableAsSelectStmtTest {
         String selectFromCte = "create table `test`.`select_cte` PROPERTIES(\"replication_num\" = \"1\") "
                 + "as with cte_name1 as (select userId from `test`.`varchar_table`) select * from cte_name1";
         createTableAsSelect(selectFromCte);
-        ShowResultSet showResultSet = showCreateTable("select_cte");
-        Assert.assertEquals("CREATE TABLE `select_cte` (\n"
-                + "  `userId` varchar(255) NOT NULL\n"
-                + ") ENGINE=OLAP\n"
-                + "DUPLICATE KEY(`userId`)\n"
-                + "COMMENT 'OLAP'\n"
-                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")", showResultSet.getResultRows().get(0).get(1));
-        String selectFromCteAndUnion =
-                "create table `test`.`select_cte_union` PROPERTIES(\"replication_num\" = \"1\")"
-                        + "as with source_data as (select 1 as id union all select 2 as id) select * from source_data;";
+        ShowResultSet showResultSet = showCreateTableByName("select_cte");
+        Assertions.assertEquals(
+                "CREATE TABLE `select_cte` (\n" + "  `userId` varchar(255) NOT NULL\n" + ") ENGINE=OLAP\n"
+                        + "DUPLICATE KEY(`userId`)\n" + "COMMENT 'OLAP'\n"
+                        + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n" + "PROPERTIES (\n"
+                        + "\"replication_allocation\" = \"tag.location.default: 1\",\n" + "\"in_memory\" = \"false\",\n"
+                        + "\"storage_format\" = \"V2\"\n" + ")", showResultSet.getResultRows().get(0).get(1));
+        String selectFromCteAndUnion = "create table `test`.`select_cte_union` PROPERTIES(\"replication_num\" = \"1\")"
+                + "as with source_data as (select 1 as id union all select 2 as id) select * from source_data;";
         createTableAsSelect(selectFromCteAndUnion);
-        ShowResultSet showResultSet1 = showCreateTable("select_cte_union");
-        Assert.assertEquals("CREATE TABLE `select_cte_union` (\n"
-                + "  `id` tinyint(4) NOT NULL\n"
-                + ") ENGINE=OLAP\n"
-                + "DUPLICATE KEY(`id`)\n"
-                + "COMMENT 'OLAP'\n"
-                + "DISTRIBUTED BY HASH(`id`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")", showResultSet1.getResultRows().get(0).get(1));
+        ShowResultSet showResultSet1 = showCreateTableByName("select_cte_union");
+        Assertions.assertEquals(
+                "CREATE TABLE `select_cte_union` (\n" + "  `id` tinyint(4) NOT NULL\n" + ") ENGINE=OLAP\n"
+                        + "DUPLICATE KEY(`id`)\n" + "COMMENT 'OLAP'\n" + "DISTRIBUTED BY HASH(`id`) BUCKETS 10\n"
+                        + "PROPERTIES (\n" + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
+                        + "\"in_memory\" = \"false\",\n" + "\"storage_format\" = \"V2\"\n" + ")",
+                showResultSet1.getResultRows().get(0).get(1));
     }
 
     @Test
     public void testPartition() throws Exception {
-        String selectFromPartition =
-                "create table `test`.`selectPartition` PARTITION BY LIST "
-                        + "(userId)(PARTITION p1 values in (\"CA\",\"GB\",\"US\",\"ZH\")) "
-                        + "PROPERTIES (\"replication_num\" = \"1\") as "
-                        + "select * from `test`.`varchar_table`;";
+        String selectFromPartition = "create table `test`.`selectPartition` PARTITION BY LIST "
+                + "(userId)(PARTITION p1 values in (\"CA\",\"GB\",\"US\",\"ZH\")) "
+                + "PROPERTIES (\"replication_num\" = \"1\") as " + "select * from `test`.`varchar_table`;";
         createTableAsSelect(selectFromPartition);
-        ShowResultSet showResultSet = showCreateTable("selectPartition");
-        Assert.assertEquals("CREATE TABLE `selectPartition` (\n"
-                + "  `userId` varchar(255) NOT NULL,\n"
-                + "  `username` varchar(255) REPLACE NOT NULL\n"
-                + ") ENGINE=OLAP\n"
-                + "AGGREGATE KEY(`userId`)\n"
-                + "COMMENT 'OLAP'\n"
-                + "PARTITION BY LIST(`userId`)\n"
+        ShowResultSet showResultSet = showCreateTableByName("selectPartition");
+        Assertions.assertEquals("CREATE TABLE `selectPartition` (\n" + "  `userId` varchar(255) NOT NULL,\n"
+                + "  `username` varchar(255) REPLACE NOT NULL\n" + ") ENGINE=OLAP\n" + "AGGREGATE KEY(`userId`)\n"
+                + "COMMENT 'OLAP'\n" + "PARTITION BY LIST(`userId`)\n"
                 + "(PARTITION p1 VALUES IN (\"CA\",\"GB\",\"US\",\"ZH\"))\n"
-                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n"
-                + "PROPERTIES (\n"
-                + "\"replication_allocation\" = \"tag.location.default: 1\",\n"
-                + "\"in_memory\" = \"false\",\n"
-                + "\"storage_format\" = \"V2\"\n"
-                + ")", showResultSet.getResultRows().get(0).get(1));
+                + "DISTRIBUTED BY HASH(`userId`) BUCKETS 10\n" + "PROPERTIES (\n"
+                + "\"replication_allocation\" = \"tag.location.default: 1\",\n" + "\"in_memory\" = \"false\",\n"
+                + "\"storage_format\" = \"V2\"\n" + ")", showResultSet.getResultRows().get(0).get(1));
     }
 }
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 0fe03dd5b6..2915547420 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
@@ -22,6 +22,7 @@ import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.CreateDbStmt;
 import org.apache.doris.analysis.CreatePolicyStmt;
 import org.apache.doris.analysis.CreateSqlBlockRuleStmt;
+import org.apache.doris.analysis.CreateTableAsSelectStmt;
 import org.apache.doris.analysis.CreateTableStmt;
 import org.apache.doris.analysis.CreateViewStmt;
 import org.apache.doris.analysis.DropPolicyStmt;
@@ -373,10 +374,21 @@ public abstract class TestWithFeService {
         return executor.execute();
     }
 
+    protected ShowResultSet showCreateTableByName(String table) throws Exception {
+        ShowCreateTableStmt stmt = (ShowCreateTableStmt) parseAndAnalyzeStmt("show create table " + table);
+        ShowExecutor executor = new ShowExecutor(connectContext, stmt);
+        return executor.execute();
+    }
+
     public void createTable(String sql) throws Exception {
         createTables(sql);
     }
 
+    public void createTableAsSelect(String sql) throws Exception {
+        CreateTableAsSelectStmt createTableAsSelectStmt = (CreateTableAsSelectStmt) parseAndAnalyzeStmt(sql);
+        Catalog.getCurrentCatalog().createTableAsSelect(createTableAsSelectStmt);
+    }
+
     public void createTables(String... sqls) throws Exception {
         for (String sql : sqls) {
             CreateTableStmt stmt = (CreateTableStmt) parseAndAnalyzeStmt(sql);


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