diff --git a/docs/en/sql-reference/sql-statements/Data Definition/CREATE TABLE LIKE.md b/docs/en/sql-reference/sql-statements/Data Definition/CREATE TABLE LIKE.md new file mode 100644 index 00000000000000..53239fd90088e5 --- /dev/null +++ b/docs/en/sql-reference/sql-statements/Data Definition/CREATE TABLE LIKE.md @@ -0,0 +1,61 @@ +--- +{ + "title": "CREATE TABLE LIKE", + "language": "en" +} +--- + + + +# CREATE TABLE LIKE + +## description + +Use CREATE TABLE ... LIKE to create an empty table based on the definition of another table, including any column attributes, table partitions and table properties defined in the original table: +Syntax: + +``` + CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [database.]table_name LIKE [database.]table_name +``` + +Explain: + 1. The replicated table structures include Column Defination, Partitions, Table Properties, and so on + 2. The SELECT privilege is required on the original table. + 3. Support to copy external table such as MySQL. + +## Example + 1. Under the test1 Database, create an empty table with the same table structure as table1, named table2 + + CREATE TABLE test1.table2 LIKE test1.table1 + + 2. Under the test2 Database, create an empty table with the same table structure as test1.table1, named table2 + + CREATE TABLE test2.table2 LIKE test1.table1 + + 3. Under the test1 Database, create an empty table with the same table structure as MySQL's external table1, called table2 + + CREATE TABLE test1.table2 LIKE test1.table1 + +## keyword + +``` + CREATE,TABLE,LIKE + +``` diff --git a/docs/zh-CN/sql-reference/sql-statements/Data Definition/CREATE TABLE LIKE.md b/docs/zh-CN/sql-reference/sql-statements/Data Definition/CREATE TABLE LIKE.md new file mode 100644 index 00000000000000..65c62f7eef1fe1 --- /dev/null +++ b/docs/zh-CN/sql-reference/sql-statements/Data Definition/CREATE TABLE LIKE.md @@ -0,0 +1,61 @@ +--- +{ + "title": "CREATE TABLE LIKE", + "language": "zh-CN" +} +--- + + + +# CREATE TABLE LIKE + +## description + +该语句用于创建一个表结构和另一张表完全相同的空表。 +语法: + +``` + CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [database.]table_name LIKE [database.]table_name +``` + +说明: + 1. 复制的表结构包括Column Defination、Partitions、Table Properties等 + 2. 用户需要对复制的原表有`SELECT`权限 + 3. 支持复制MySQL等外表 + +## Example + 1. 在test1库下创建一张表结构和table1相同的空表,表名为table2 + + CREATE TABLE test1.table2 LIKE test1.table1 + + 2. 在test2库下创建一张表结构和test1.table1相同的空表,表名为table2 + + CREATE TABLE test2.table2 LIKE test1.table1 + + 3. 在test1库下创建一张表结构和MySQL外表table1相同的空表,表名为table2 + + CREATE TABLE test1.table2 LIKE test1.table1 + +## keyword + +``` + CREATE,TABLE,LIKE + +``` diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 73b0b47d5bc47d..b9e75cf1c042f6 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -1097,11 +1097,15 @@ create_stmt ::= :}*/ /* Function */ | KW_CREATE opt_aggregate:isAggregate KW_FUNCTION function_name:functionName LPAREN func_args_def:args RPAREN - KW_RETURNS type_def:retrunType opt_intermediate_type:intermediateType opt_properties:properties + KW_RETURNS type_def:returnType opt_intermediate_type:intermediateType opt_properties:properties {: - RESULT = new CreateFunctionStmt(isAggregate, functionName, args, retrunType, intermediateType, properties); + RESULT = new CreateFunctionStmt(isAggregate, functionName, args, returnType, intermediateType, properties); :} /* Table */ + | KW_CREATE opt_external:isExternal KW_TABLE opt_if_not_exists:ifNotExists table_name:name KW_LIKE table_name:existed_name + {: + RESULT = new CreateTableLikeStmt(ifNotExists, name, existed_name); + :} | KW_CREATE opt_external:isExternal KW_TABLE opt_if_not_exists:ifNotExists table_name:name LPAREN column_definition_list:columns RPAREN opt_engine:engineName opt_keys:keys diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableLikeStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableLikeStmt.java new file mode 100644 index 00000000000000..11a3908c33bb99 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableLikeStmt.java @@ -0,0 +1,95 @@ +// 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.analysis; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.UserException; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +/** + * @author wangcong + * @version 1.0 + * @date 2020/10/7 10:32 上午 + */ +public class CreateTableLikeStmt extends DdlStmt { + private static final Logger LOG = LogManager.getLogger(CreateTableLikeStmt.class); + + private final boolean ifNotExists; + private final TableName tableName; + private final TableName existedTableName; + + public CreateTableLikeStmt(boolean ifNotExists, TableName tableName, TableName existedTableName) { + this.ifNotExists = ifNotExists; + this.tableName = tableName; + this.existedTableName = existedTableName; + } + + public boolean isSetIfNotExists() { + return ifNotExists; + } + + public String getDbName() { + return tableName.getDb(); + } + + public String getTableName() { + return tableName.getTbl(); + } + + public String getExistedDbName() { + return existedTableName.getDb(); + } + + public String getExistedTableName() { + return existedTableName.getTbl(); + } + + @Override + public void analyze(Analyzer analyzer) throws UserException { + super.analyze(analyzer); + existedTableName.analyze(analyzer); + ConnectContext ctx = ConnectContext.get(); + if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ctx, existedTableName.getDb(), + existedTableName.getTbl(), PrivPredicate.SELECT)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "SELECT"); + } + + tableName.analyze(analyzer); + FeNameFormat.checkTableName(getTableName()); + if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ctx, tableName.getDb(), + tableName.getTbl(), PrivPredicate.CREATE)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "CREATE"); + } + } + + @Override + public String toSql() { + return String.format("CREATE TABLE %s LIKE %s", tableName.toSql(), existedTableName.toSql()); + } + + @Override + public String toString() { + return toSql(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java index b59b2f0cda5083..af6a2fbe8a94e0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java @@ -561,9 +561,6 @@ public String toString() { @Override public boolean needAuditEncryption() { - if (!engineName.equals("olap")) { - return true; - } - return false; + return !engineName.equals("olap"); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index 8b47b1161171d3..82351c6de9aca9 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -48,6 +48,7 @@ import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateFunctionStmt; import org.apache.doris.analysis.CreateMaterializedViewStmt; +import org.apache.doris.analysis.CreateTableLikeStmt; import org.apache.doris.analysis.CreateTableStmt; import org.apache.doris.analysis.CreateUserStmt; import org.apache.doris.analysis.CreateViewStmt; @@ -121,6 +122,7 @@ import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.QueryableReentrantLock; import org.apache.doris.common.util.SmallFileMgr; +import org.apache.doris.common.util.SqlParserUtils; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.common.util.Util; import org.apache.doris.consistency.ConsistencyChecker; @@ -2274,6 +2276,7 @@ protected void runAfterCatalogReady() { public void createReplayer() { replayer = new Daemon("replayer", REPLAY_INTERVAL_MS) { + @Override protected void runOneCycle() { boolean err = false; boolean hasLog = false; @@ -2995,7 +2998,25 @@ public void replayRenameDatabase(String dbName, String newDbName) { LOG.info("replay rename database {} to {}", dbName, newDbName); } - public void createTable(CreateTableStmt stmt) throws DdlException { + /** + * Following is the step to create an olap table: + * 1. create columns + * 2. create partition info + * 3. create distribution info + * 4. set table id and base index id + * 5. set bloom filter columns + * 6. set and build TableProperty includes: + * 6.1. dynamicProperty + * 6.2. replicationNum + * 6.3. inMemory + * 6.4. storageFormat + * 7. set index meta + * 8. check colocation properties + * 9. create tablet in BE + * 10. add this table to FE's meta + * 11. add this table to ColocateGroup if necessary + */ + public void createTable(CreateTableStmt stmt) throws DdlException { String engineName = stmt.getEngineName(); String dbName = stmt.getDbName(); String tableName = stmt.getTableName(); @@ -3055,7 +3076,31 @@ public void createTable(CreateTableStmt stmt) throws DdlException { ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_STORAGE_ENGINE, engineName); } Preconditions.checkState(false); - return; + } + + public void createTableLike(CreateTableLikeStmt stmt) throws DdlException { + try { + Database db = Catalog.getCurrentCatalog().getDb(stmt.getExistedDbName()); + List createTableStmt = Lists.newArrayList(); + db.readLock(); + try { + Table table = db.getTable(stmt.getExistedTableName()); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, stmt.getExistedTableName()); + } + Catalog.getDdlStmt(stmt.getDbName(), table, createTableStmt, null, null, false, false); + if (createTableStmt.isEmpty()) { + ErrorReport.reportDdlException(ErrorCode.ERROR_CREATE_TABLE_LIKE_EMPTY, "CREATE"); + } + } finally { + db.readUnlock(); + } + CreateTableStmt parsedCreateTableStmt = (CreateTableStmt) SqlParserUtils.parseAndAnalyzeStmt(createTableStmt.get(0), ConnectContext.get()); + parsedCreateTableStmt.setTableName(stmt.getTableName()); + createTable(parsedCreateTableStmt); + } catch (UserException e) { + throw new DdlException("Failed to execute CREATE TABLE LIKE " + stmt.getExistedTableName() + ". Reason: " + e.getMessage()); + } } public void addPartition(Database db, String tableName, AddPartitionClause addPartitionClause) throws DdlException { @@ -3926,6 +3971,11 @@ private void createHiveTable(Database db, CreateTableStmt stmt) throws DdlExcept public static void getDdlStmt(Table table, List createTableStmt, List addPartitionStmt, List createRollupStmt, boolean separatePartition, boolean hidePassword) { + getDdlStmt(null, table, createTableStmt, addPartitionStmt, createRollupStmt, separatePartition, hidePassword); + } + + public static void getDdlStmt(String dbName, Table table, List createTableStmt, List addPartitionStmt, + List createRollupStmt, boolean separatePartition, boolean hidePassword) { StringBuilder sb = new StringBuilder(); // 1. create table @@ -3945,6 +3995,9 @@ public static void getDdlStmt(Table table, List createTableStmt, List getMultiStmts(SqlParser parser) throws Excepti } return stmts; } + + public static StatementBase parseAndAnalyzeStmt(String originStmt, ConnectContext ctx) throws UserException { + LOG.info("begin to parse stmt: " + originStmt); + SqlScanner input = new SqlScanner(new StringReader(originStmt), ctx.getSessionVariable().getSqlMode()); + SqlParser parser = new SqlParser(input); + Analyzer analyzer = new Analyzer(ctx.getCatalog(), ctx); + StatementBase statementBase; + try { + statementBase = SqlParserUtils.getFirstStmt(parser); + } catch (AnalysisException e) { + String errorMessage = parser.getErrorMsg(originStmt); + LOG.error("parse failed: " + errorMessage); + if (errorMessage == null) { + throw e; + } else { + throw new AnalysisException(errorMessage, e); + } + } catch (Exception e) { + String errorMsg = String.format("get exception when parse stmt. Origin stmt is %s . Error msg is %s.", + originStmt, e.getMessage()); + throw new AnalysisException(errorMsg); + } + statementBase.analyze(analyzer); + return statementBase; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java index dfa0b715c03bb0..60b9aae4addc81 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java @@ -43,6 +43,7 @@ import org.apache.doris.analysis.CreateResourceStmt; import org.apache.doris.analysis.CreateRoleStmt; import org.apache.doris.analysis.CreateRoutineLoadStmt; +import org.apache.doris.analysis.CreateTableLikeStmt; import org.apache.doris.analysis.CreateTableStmt; import org.apache.doris.analysis.CreateUserStmt; import org.apache.doris.analysis.CreateViewStmt; @@ -104,6 +105,8 @@ public static void execute(Catalog catalog, DdlStmt ddlStmt) throws Exception { catalog.dropFunction((DropFunctionStmt) ddlStmt); } else if (ddlStmt instanceof CreateTableStmt) { catalog.createTable((CreateTableStmt) ddlStmt); + } else if (ddlStmt instanceof CreateTableLikeStmt) { + catalog.createTableLike((CreateTableLikeStmt) ddlStmt); } else if (ddlStmt instanceof DropTableStmt) { catalog.dropTable((DropTableStmt) ddlStmt); } else if (ddlStmt instanceof CreateMaterializedViewStmt) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index 5e76f2443d593d..08f03f09a44a5a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -217,10 +217,7 @@ public ShowResultSet getShowResultSet() { } public boolean isQueryStmt() { - if (parsedStmt != null && parsedStmt instanceof QueryStmt) { - return true; - } - return false; + return parsedStmt != null && parsedStmt instanceof QueryStmt; } public StatementBase getParsedStmt() { diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableLikeTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableLikeTest.java new file mode 100644 index 00000000000000..b09c6b40f94b1e --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableLikeTest.java @@ -0,0 +1,244 @@ +// 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.catalog; + +import avro.shaded.com.google.common.collect.Lists; +import org.apache.doris.analysis.CreateDbStmt; +import org.apache.doris.analysis.CreateTableLikeStmt; +import org.apache.doris.analysis.CreateTableStmt; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ExceptionChecker; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.utframe.UtFrameUtils; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.File; +import java.util.List; +import java.util.UUID; + +/** + * @author wangcong + * @version 1.0 + * @date 2020/10/7 12:31 下午 + */ +public class CreateTableLikeTest { + private static String runningDir = "fe/mocked/CreateTableLikeTest/" + UUID.randomUUID().toString() + "/"; + + private static ConnectContext connectContext; + + @BeforeClass + public static void beforeClass() throws Exception { + UtFrameUtils.createMinDorisCluster(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); + String createDbStmtStr2 = "create database test2;"; + CreateDbStmt createDbStmt2 = (CreateDbStmt) UtFrameUtils.parseAndAnalyzeStmt(createDbStmtStr2, connectContext); + Catalog.getCurrentCatalog().createDb(createDbStmt2); + } + + @AfterClass + public static void tearDown() { + File file = new File(runningDir); + file.delete(); + } + + private static void createTable(String sql) throws Exception { + CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext); + Catalog.getCurrentCatalog().createTable(createTableStmt); + } + + private static void createTableLike(String sql) throws Exception { + CreateTableLikeStmt createTableLikeStmt = (CreateTableLikeStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext); + Catalog.getCurrentCatalog().createTableLike(createTableLikeStmt); + } + + private static void checkTableEqual(Table newTable, Table existedTable) { + List newCreateTableStmt = Lists.newArrayList(); + Catalog.getDdlStmt(newTable, newCreateTableStmt, null, null, false, true /* hide password */); + List existedTableStmt = Lists.newArrayList(); + Catalog.getDdlStmt(existedTable, existedTableStmt, null, null, false, true /* hide password */); + Assert.assertEquals(newCreateTableStmt.get(0).replace(newTable.getName(), existedTable.getName()), existedTableStmt.get(0)); + } + + private static void checkCreateOlapTableLike(String createTableSql, String createTableLikeSql, + String newDbName, String existedDbName, + String newTblName, String existedTblName) throws Exception { + createTable(createTableSql); + createTableLike(createTableLikeSql); + Database newDb = Catalog.getCurrentCatalog().getDb("default_cluster:" + newDbName); + Database existedDb = Catalog.getCurrentCatalog().getDb("default_cluster:" + existedDbName); + OlapTable newTbl = (OlapTable) newDb.getTable(newTblName); + OlapTable existedTbl = (OlapTable) existedDb.getTable(existedTblName); + checkTableEqual(newTbl, existedTbl); + } + + private static void checkCreateMysqlTableLike(String createTableSql, String createTableLikeSql, + String newDbName, String existedDbName, + String newTblName, String existedTblName) throws Exception { + + createTable(createTableSql); + createTableLike(createTableLikeSql); + Database newDb = Catalog.getCurrentCatalog().getDb("default_cluster:" + newDbName); + Database existedDb = Catalog.getCurrentCatalog().getDb("default_cluster:" + existedDbName); + MysqlTable newTbl = (MysqlTable) newDb.getTable(newTblName); + MysqlTable existedTbl = (MysqlTable) existedDb.getTable(existedTblName); + checkTableEqual(newTbl, existedTbl); + } + @Test + public void testNormal() throws Exception { + // 1. creat table with single partition + String createTableSql = "create table test.testTbl1\n" + "(k1 int, k2 int)\n" + "duplicate key(k1)\n" + + "distributed by hash(k2) buckets 1\n" + "properties('replication_num' = '1'); "; + String createTableLikeSql = "create table test.testTbl1_like like test.testTbl1"; + String newDbName = "test"; + String newTblName = "testTbl1_like"; + String existedTblName = "testTbl1"; + checkCreateOlapTableLike(createTableSql, createTableLikeSql, newDbName, newDbName, newTblName, existedTblName); + // 2. create table with hash partition + String createTableWithHashPartitionSql = "create table test.testTbl2\n" + "(k1 int, k2 int)\n" + + "duplicate key(k1)\n" + "partition by range(k2)\n" + "(partition p1 values less than(\"10\"))\n" + + "distributed by hash(k2) buckets 1\n" + "properties('replication_num' = '1'); "; + String createTableLikeSql2 = "create table test.testTbl2_like like test.testTbl2"; + String newDbName2 = "test"; + String newTblName2 = "testTbl2_like"; + String existedTblName2 = "testTbl2"; + checkCreateOlapTableLike(createTableWithHashPartitionSql, createTableLikeSql2, newDbName2, newDbName2, newTblName2, existedTblName2); + // 3. create aggregate table + String createAggTableSql3 = "create table test.testTbl3\n" + "(k1 varchar(40), k2 int, v1 int sum)\n" + + "partition by range(k2)\n" + "(partition p1 values less than(\"10\"))\n" + + "distributed by hash(k1) buckets 1\n" + "properties('replication_num' = '1');"; + String createTableLikeSql3 = "create table test.testTbl3_like like test.testTbl3"; + String newDbName3 = "test"; + String newTblName3 = "testTbl3_like"; + String existedTblName3 = "testTbl3"; + checkCreateOlapTableLike(createAggTableSql3, createTableLikeSql3, newDbName3, newDbName3, newTblName3, existedTblName3); + // 4. create aggregate table without partition + String createAggTableWithoutPartitionSql4 = "create table test.testTbl4\n" + "(k1 varchar(40), k2 int, k3 int)\n" + "duplicate key(k1, k2, k3)\n" + + "partition by range(k2)\n" + "()\n" + + "distributed by hash(k1) buckets 1\n" + "properties('replication_num' = '1');"; + String createTableLikeSql4 = "create table test.testTbl4_like like test.testTbl4"; + String newDbName4 = "test"; + String newTblName4 = "testTbl4_like"; + String existedTblName4 = "testTbl4"; + checkCreateOlapTableLike(createAggTableWithoutPartitionSql4, createTableLikeSql4, newDbName4, newDbName4, newTblName4, existedTblName4); + // 5. create table from different db + String createTableFromDiffDb5 = "create table test.testTbl5\n" + "(k1 varchar(40), k2 int, k3 int)\n" + "duplicate key(k1, k2, k3)\n" + + "partition by range(k2)\n" + "()\n" + + "distributed by hash(k1) buckets 1\n" + "properties('replication_num' = '1');"; + String createTableLikeSql5 = "create table test2.testTbl5_like like test.testTbl5"; + String newDbName5 = "test2"; + String existedDbName5 = "test"; + String newTblName5 = "testTbl5_like"; + String existedTblName5 = "testTbl5"; + checkCreateOlapTableLike(createTableFromDiffDb5, createTableLikeSql5, newDbName5, existedDbName5, newTblName5, existedTblName5); + // 6. create table from dynamic partition table + String createDynamicTblSql = "CREATE TABLE test.`dynamic_partition_normal` (\n" + + " `k1` date NULL COMMENT \"\",\n" + + " `k2` int NULL COMMENT \"\",\n" + + " `k3` smallint NULL COMMENT \"\",\n" + + " `v1` varchar(2048) NULL COMMENT \"\",\n" + + " `v2` datetime NULL COMMENT \"\"\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(`k1`, `k2`, `k3`)\n" + + "COMMENT \"OLAP\"\n" + + "PARTITION BY RANGE (k1)\n" + + "(\n" + + "PARTITION p1 VALUES LESS THAN (\"2014-01-01\"),\n" + + "PARTITION p2 VALUES LESS THAN (\"2014-06-01\"),\n" + + "PARTITION p3 VALUES LESS THAN (\"2014-12-01\")\n" + + ")\n" + + "DISTRIBUTED BY HASH(`k1`) BUCKETS 32\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\",\n" + + "\"dynamic_partition.enable\" = \"true\",\n" + + "\"dynamic_partition.start\" = \"-3\",\n" + + "\"dynamic_partition.end\" = \"3\",\n" + + "\"dynamic_partition.time_unit\" = \"day\",\n" + + "\"dynamic_partition.prefix\" = \"p\",\n" + + "\"dynamic_partition.buckets\" = \"1\"\n" + + ");"; + String createTableLikeSql6 = "create table test.dynamic_partition_normal_like like test.dynamic_partition_normal"; + String newDbName6 = "test"; + String newTblName6 = "dynamic_partition_normal_like"; + String existedTblName6 = "dynamic_partition_normal"; + checkCreateOlapTableLike(createDynamicTblSql, createTableLikeSql6, newDbName6, newDbName6, newTblName6, existedTblName6); + // 7. create table from colocate table + String createColocateTblSql = "create table test.colocateTbl (\n" + + " `k1` int NULL COMMENT \"\",\n" + + " `k2` varchar(10) NULL COMMENT \"\"\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(`k1`, `k2`)\n" + + "COMMENT \"OLAP\"\n" + + "DISTRIBUTED BY HASH(`k1`, `k2`) BUCKETS 1\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\",\n" + + " \"colocate_with\" = \"test_group\"\n" + + ");"; + String createTableLikeSql7 = "create table test.colocateTbl_like like test.colocateTbl"; + String newDbName7 = "test"; + String newTblName7 = "colocateTbl_like"; + String existedTblName7 = "colocateTbl"; + checkCreateOlapTableLike(createColocateTblSql, createTableLikeSql7, newDbName7, newDbName7, newTblName7, existedTblName7); + // 8. creat non-OLAP table + String createNonOlapTableSql = "create table test.testMysqlTbl\n" + + "(k1 DATE, k2 INT, k3 SMALLINT, k4 VARCHAR(2048), k5 DATETIME)\n" + + "ENGINE=mysql\nPROPERTIES(\n"+ + "\"host\" = \"127.0.0.1\",\n" + + "\"port\" = \"8239\",\n" + + "\"user\" = \"mysql_passwd\",\n" + + "\"password\" = \"mysql_passwd\",\n" + + "\"database\" = \"mysql_db_test\",\n" + + "\"table\" = \"mysql_table_test\");"; + String createTableLikeSql8 = "create table test.testMysqlTbl_like like test.testMysqlTbl"; + String newDbName8 = "test"; + String existedDbName8 = "test"; + String newTblName8 = "testMysqlTbl_like"; + String existedTblName8 = "testMysqlTbl"; + checkCreateMysqlTableLike(createNonOlapTableSql, createTableLikeSql8, newDbName8, existedDbName8, newTblName8, existedTblName8); + } + + @Test + public void testAbnormal() { + // 1. create table with same name + String createTableSql = "create table test.testAbTbl1\n" + "(k1 int, k2 int)\n" + "duplicate key(k1)\n" + + "distributed by hash(k2) buckets 1\n" + "properties('replication_num' = '1'); "; + String createTableLikeSql = "create table test.testAbTbl1 like test.testAbTbl1"; + String newDbName = "test"; + String newTblName = "testAbTbl1"; + ExceptionChecker.expectThrowsWithMsg(DdlException.class, "Table 'testAbTbl1' already exists", + () -> checkCreateOlapTableLike(createTableSql, createTableLikeSql, newDbName, newDbName, newTblName, newTblName)); + // 2. create table with not existed DB + String createTableSql2 = "create table test.testAbTbl2\n" + "(k1 int, k2 int)\n" + "duplicate key(k1)\n" + + "distributed by hash(k2) buckets 1\n" + "properties('replication_num' = '1'); "; + String createTableLikeSql2 = "create table fake_test.testAbTbl2_like like test.testAbTbl1"; + String newDbName2 = "fake_test"; + String existedDbName2 = "test"; + String newTblName2 = "testAbTbl2_like"; + String existedTblName2 = "testAbTbl1"; + ExceptionChecker.expectThrowsWithMsg(DdlException.class, "Unknown database 'default_cluster:fake_test'", + () -> checkCreateOlapTableLike(createTableSql2, createTableLikeSql2, newDbName2, existedDbName2, newTblName2, existedTblName2)); + } +}