You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2021/08/31 12:33:51 UTC

[incubator-doris] branch master updated: [Feature] Create table like clause support copy rollup (#6475)

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

morningman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git


The following commit(s) were added to refs/heads/master by this push:
     new a949dcd  [Feature] Create table like clause support copy rollup (#6475)
a949dcd is described below

commit a949dcd9f6e4e2a87be8c61302d080f78b390ee9
Author: shee <13...@users.noreply.github.com>
AuthorDate: Tue Aug 31 20:33:26 2021 +0800

    [Feature] Create table like clause support copy rollup (#6475)
    
    for issue #6474
    
    ```sql
    create table test.table1 like test.table with rollup r1,r2 -- copy some rollup
    
    create table test.table1 like test.table with rollup all -- copy all rollup
    
    create table test.table1 like test.table  -- only copy base table
    ```
---
 .../Data Definition/CREATE TABLE LIKE.md           |  21 +++-
 .../Data Definition/CREATE TABLE LIKE.md           |  21 +++-
 fe/fe-core/src/main/cup/sql_parser.cup             |  10 +-
 .../apache/doris/analysis/CreateTableLikeStmt.java |  16 ++-
 .../java/org/apache/doris/catalog/Catalog.java     |  75 ++++++++++++-
 .../apache/doris/catalog/CreateTableLikeTest.java  | 118 ++++++++++++++++++++-
 6 files changed, 247 insertions(+), 14 deletions(-)

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
index 9507af5..47284de 100644
--- 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	
@@ -32,13 +32,14 @@ Use CREATE TABLE ... LIKE to create an empty table based on the definition of an
 Syntax:
 
 ```
-    CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [database.]table_name LIKE [database.]table_name
+    CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [database.]table_name LIKE [database.]table_name [WITH ROLLUP r1,r2,r3,...|ALL] 
 ```
 
 Explain:
     1. The replicated table structures include Column Definition, 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.
+    4. Support to copy OLAP table rollup
 
 ## Example
     1. Under the test1 Database, create an empty table with the same table structure as table1, named table2
@@ -49,7 +50,23 @@ Explain:
 
         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
+    3. Under the test1 Database, create an empty table with the same table structure as table1, named table2. copy r1 and r2 rollup of table1 simultaneously
+
+        CREATE TABLE test1.table2 LIKE test1.table1 WITH ROLLUP r1,r2
+
+    4. Under the test1 Database, create an empty table with the same table structure as table1, named table2. copy all rollup of table1 simultaneously
+
+        CREATE TABLE test1.table2 LIKE test1.table1 WITH ROLLUP ALL
+
+    5. Under the test2 Database, create an empty table with the same table structure as table1, named table2. copy r1 and r2 rollup of table1 simultaneously
+
+        CREATE TABLE test2.table2 LIKE test1.table1 WITH ROLLUP r1,r2
+
+    6. Under the test2 Database, create an empty table with the same table structure as table1, named table2. copy all rollup of table1 simultaneously
+
+        CREATE TABLE test2.table2 LIKE test1.table1 WITH ROLLUP ALL
+    
+    7. 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
 
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
index f15c5fd..82c1378 100644
--- 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	
@@ -28,7 +28,7 @@ under the License.
 
 ## description
 
-该语句用于创建一个表结构和另一张表完全相同的空表。
+该语句用于创建一个表结构和另一张表完全相同的空表,同时也能够可选复制一些rollup。
 语法:
 
 ```
@@ -39,6 +39,7 @@ under the License.
     1. 复制的表结构包括Column Definition、Partitions、Table Properties等
     2. 用户需要对复制的原表有`SELECT`权限
     3. 支持复制MySQL等外表
+    4. 支持复制OLAP Table的rollup
 
 ## Example
     1. 在test1库下创建一张表结构和table1相同的空表,表名为table2
@@ -49,7 +50,23 @@ under the License.
 
         CREATE TABLE test2.table2 LIKE test1.table1
 
-    3. 在test1库下创建一张表结构和MySQL外表table1相同的空表,表名为table2
+    3. 在test1库下创建一张表结构和table1相同的空表,表名为table2,同时复制table1的r1,r2两个rollup
+
+        CREATE TABLE test1.table2 LIKE test1.table1 WITH ROLLUP r1,r2
+
+    4. 在test1库下创建一张表结构和table1相同的空表,表名为table2,同时复制table1的所有rollup
+
+        CREATE TABLE test1.table2 LIKE test1.table1 WITH ROLLUP ALL
+
+    5. 在test2库下创建一张表结构和test1.table1相同的空表,表名为table2,同时复制table1的r1,r2两个rollup
+
+        CREATE TABLE test2.table2 LIKE test1.table1 WITH ROLLUP r1,r2
+
+    6. 在test2库下创建一张表结构和test1.table1相同的空表,表名为table2,同时复制table1的所有rollup
+
+        CREATE TABLE test2.table2 LIKE test1.table1 WITH ROLLUP ALL
+
+    7. 在test1库下创建一张表结构和MySQL外表table1相同的空表,表名为table2
 
         CREATE TABLE test1.table2 LIKE test1.table1
 
diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index e60c6c1..711f32a 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -1190,9 +1190,17 @@ create_stmt ::=
         RESULT = new CreateFunctionStmt(functionName, args, parameters, func);
     :}
     /* Table */
+    | KW_CREATE opt_external:isExternal KW_TABLE opt_if_not_exists:ifNotExists table_name:name KW_LIKE table_name:existed_name KW_WITH KW_ROLLUP ident_list:rollupNames
+    {:
+        RESULT = new CreateTableLikeStmt(ifNotExists, name, existed_name, rollupNames, false);
+    :}
+    | KW_CREATE opt_external:isExternal KW_TABLE opt_if_not_exists:ifNotExists table_name:name KW_LIKE table_name:existed_name KW_WITH KW_ROLLUP KW_ALL
+    {:
+        RESULT = new CreateTableLikeStmt(ifNotExists, name, existed_name, null, true);
+    :}
     | 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);
+        RESULT = new CreateTableLikeStmt(ifNotExists, name, existed_name, null, false);
     :}
     | KW_CREATE opt_external:isExternal KW_TABLE opt_if_not_exists:ifNotExists table_name:name
             LPAREN column_definition_list:columns RPAREN opt_engine:engineName
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
index 58f6e54..4a00f3b 100644
--- 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
@@ -27,6 +27,8 @@ import org.apache.doris.qe.ConnectContext;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import java.util.ArrayList;
+
 /**
  * @author wangcong
  * @version 1.0
@@ -38,11 +40,15 @@ public class CreateTableLikeStmt extends DdlStmt {
     private final boolean ifNotExists;
     private final TableName tableName;
     private final TableName existedTableName;
+    private final ArrayList<String> rollupNames;
+    private final boolean withAllRollup;
 
-    public CreateTableLikeStmt(boolean ifNotExists, TableName tableName, TableName existedTableName) {
+    public CreateTableLikeStmt(boolean ifNotExists, TableName tableName, TableName existedTableName, ArrayList<String> rollupNames, boolean withAllRollup) {
         this.ifNotExists = ifNotExists;
         this.tableName = tableName;
         this.existedTableName = existedTableName;
+        this.rollupNames = rollupNames;
+        this.withAllRollup = withAllRollup;
     }
 
     public boolean isIfNotExists() {
@@ -65,6 +71,14 @@ public class CreateTableLikeStmt extends DdlStmt {
         return existedTableName.getTbl();
     }
 
+    public ArrayList<String> getRollupNames() {
+        return rollupNames;
+    }
+
+    public boolean isWithAllRollup() {
+        return withAllRollup;
+    }
+
     @Override
     public void analyze(Analyzer analyzer) throws UserException {
         super.analyze(analyzer);
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 52eef91..35a881a 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
@@ -53,6 +53,7 @@ import org.apache.doris.analysis.CreateTableLikeStmt;
 import org.apache.doris.analysis.CreateTableStmt;
 import org.apache.doris.analysis.CreateUserStmt;
 import org.apache.doris.analysis.CreateViewStmt;
+import org.apache.doris.analysis.DdlStmt;
 import org.apache.doris.analysis.DecommissionBackendClause;
 import org.apache.doris.analysis.DistributionDesc;
 import org.apache.doris.analysis.DropClusterStmt;
@@ -3081,7 +3082,18 @@ public class Catalog {
             List<String> createTableStmt = Lists.newArrayList();
             table.readLock();
             try {
-                Catalog.getDdlStmt(stmt.getDbName(), table, createTableStmt, null, null, false, false);
+                if (table.getType() == TableType.OLAP){
+                    if (!CollectionUtils.isEmpty(stmt.getRollupNames())){
+                        OlapTable olapTable = (OlapTable) table;
+                        for (String rollupIndexName : stmt.getRollupNames()) {
+                            if (!olapTable.hasMaterializedIndex(rollupIndexName)) {
+                                throw new DdlException("Rollup index[" + rollupIndexName + "] not exists in Table[" + olapTable.getName() + "]");
+                            }
+                        }
+                    }
+                }
+
+                Catalog.getDdlStmt(stmt, stmt.getDbName(), table, createTableStmt, null, null, false, false);
                 if (createTableStmt.isEmpty()) {
                     ErrorReport.reportDdlException(ErrorCode.ERROR_CREATE_TABLE_LIKE_EMPTY, "CREATE");
                 }
@@ -4001,10 +4013,10 @@ public class Catalog {
 
     public static void getDdlStmt(Table table, List<String> createTableStmt, List<String> addPartitionStmt,
                                   List<String> createRollupStmt, boolean separatePartition, boolean hidePassword) {
-         getDdlStmt(null, table, createTableStmt, addPartitionStmt, createRollupStmt, separatePartition, hidePassword);
+         getDdlStmt(null, null, table, createTableStmt, addPartitionStmt, createRollupStmt, separatePartition, hidePassword);
     }
 
-    public static void getDdlStmt(String dbName, Table table, List<String> createTableStmt, List<String> addPartitionStmt,
+    public static void getDdlStmt(DdlStmt ddlStmt, String dbName, Table table, List<String> createTableStmt, List<String> addPartitionStmt,
                                   List<String> createRollupStmt, boolean separatePartition, boolean hidePassword) {
         StringBuilder sb = new StringBuilder();
 
@@ -4085,6 +4097,63 @@ public class Catalog {
             DistributionInfo distributionInfo = olapTable.getDefaultDistributionInfo();
             sb.append("\n").append(distributionInfo.toSql());
 
+            // rollup index
+            if (ddlStmt instanceof CreateTableLikeStmt){
+
+                CreateTableLikeStmt stmt = (CreateTableLikeStmt) ddlStmt;
+
+                ArrayList<String> rollupNames = stmt.getRollupNames();
+                boolean withAllRollup = stmt.isWithAllRollup();
+
+                Map<Long,MaterializedIndexMeta> addMVs = Maps.newHashMap();
+                Map<String, Long> indexNameToId = olapTable.getIndexNameToId();
+
+                boolean needAddRollup = false;
+                if (!CollectionUtils.isEmpty(rollupNames)) {
+                    for (String rollupName : rollupNames) {
+                        addMVs.put(indexNameToId.get(rollupName),olapTable.getIndexMetaByIndexId(indexNameToId.get(rollupName)));
+                    }
+                    needAddRollup = true;
+                }
+
+                if (withAllRollup && rollupNames == null) {
+                    for (Entry<Long, MaterializedIndexMeta> entry : olapTable.getIndexIdToMeta().entrySet()) {
+                        if (entry.getKey() == olapTable.getBaseIndexId()) {
+                            continue;
+                        }
+                        addMVs.put(entry.getKey(), entry.getValue());
+                    }
+                    needAddRollup = true;
+                }
+
+                if (needAddRollup){
+                    sb.append("\n").append("rollup (");
+                }
+
+                int size = addMVs.size();
+                int index = 1;
+                for (Map.Entry<Long, MaterializedIndexMeta> entry : addMVs.entrySet()) {
+                    MaterializedIndexMeta materializedIndexMeta = entry.getValue();
+                    String indexName = olapTable.getIndexNameById(entry.getKey());
+                    sb.append("\n").append(indexName).append("(");
+                    List<Column> indexSchema = materializedIndexMeta.getSchema();
+                    for (int i = 0; i < indexSchema.size(); i++) {
+                        Column column = indexSchema.get(i);
+                        sb.append(column.getName());
+                        if (i != indexSchema.size() - 1) {
+                            sb.append(", ");
+                        }
+                    }
+                    if (index != size){
+                        sb.append("),");
+                    } else {
+                        sb.append(")");
+                        sb.append("\n)");
+                    }
+                    index++;
+                }
+            }
+
             // properties
             sb.append("\nPROPERTIES (\n");
 
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
index a0eae7a..858eb76 100644
--- 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
@@ -18,11 +18,13 @@
 package org.apache.doris.catalog;
 
 import avro.shaded.com.google.common.collect.Lists;
+import org.apache.commons.collections.ListUtils;
 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.common.util.ListUtil;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.utframe.UtFrameUtils;
 import org.junit.AfterClass;
@@ -75,24 +77,44 @@ public class CreateTableLikeTest {
         Catalog.getCurrentCatalog().createTableLike(createTableLikeStmt);
     }
 
-    private static void checkTableEqual(Table newTable, Table existedTable) {
+    private static void checkTableEqual(Table newTable, Table existedTable, int rollupSize) {
         List<String> newCreateTableStmt = Lists.newArrayList();
-        Catalog.getDdlStmt(newTable, newCreateTableStmt, null, null, false, true /* hide password */);
+        List<String> newAddRollupStmt = Lists.newArrayList();
+        Catalog.getDdlStmt(newTable, newCreateTableStmt, null, newAddRollupStmt, false, true /* hide password */);
         List<String> existedTableStmt = Lists.newArrayList();
-        Catalog.getDdlStmt(existedTable, existedTableStmt, null, null, false, true /* hide password */);
+        List<String> existedAddRollupStmt = Lists.newArrayList();
+        Catalog.getDdlStmt(existedTable, existedTableStmt, null, existedAddRollupStmt, false, true /* hide password */);
         Assert.assertEquals(newCreateTableStmt.get(0).replace(newTable.getName(), existedTable.getName()), existedTableStmt.get(0));
+        checkTableRollup(existedAddRollupStmt, newAddRollupStmt, newTable.getName(), existedTable.getName(), rollupSize);
+    }
+
+    private static void checkTableRollup(List<String> existedAddRollupStmt, List<String> newAddRollupStmt, String newTableName, String existedTableName, int rollupSize){
+        if (rollupSize != 0) {
+            List<String> addRollupStmt = Lists.newArrayList();
+            for (String aaRollupStmt : newAddRollupStmt) {
+                addRollupStmt.add(aaRollupStmt.replace(newTableName, existedTableName));
+            }
+            Assert.assertEquals(addRollupStmt.size(), rollupSize);
+            Assert.assertTrue(existedAddRollupStmt.containsAll(addRollupStmt));
+        }
     }
 
     private static void checkCreateOlapTableLike(String createTableSql, String createTableLikeSql,
                                                  String newDbName, String existedDbName,
                                                  String newTblName, String existedTblName) throws Exception {
+        checkCreateOlapTableLike(createTableSql, createTableLikeSql, newDbName, existedDbName, newTblName, existedTblName,0);
+    }
+
+    private static void checkCreateOlapTableLike(String createTableSql, String createTableLikeSql,
+                                                 String newDbName, String existedDbName,
+                                                 String newTblName, String existedTblName, int rollupSize) 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);
+        checkTableEqual(newTbl, existedTbl, rollupSize);
     }
 
     private static void checkCreateMysqlTableLike(String createTableSql, String createTableLikeSql,
@@ -105,8 +127,9 @@ public class CreateTableLikeTest {
         Database existedDb = Catalog.getCurrentCatalog().getDb("default_cluster:" + existedDbName);
         MysqlTable newTbl = (MysqlTable) newDb.getTable(newTblName);
         MysqlTable existedTbl = (MysqlTable) existedDb.getTable(existedTblName);
-        checkTableEqual(newTbl, existedTbl);
+        checkTableEqual(newTbl, existedTbl, 0);
     }
+
     @Test
     public void testNormal() throws Exception {
         // 1. creat table with single partition
@@ -202,6 +225,7 @@ public class CreateTableLikeTest {
         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" +
@@ -233,6 +257,55 @@ public class CreateTableLikeTest {
         } catch (Exception e) {
             Assert.fail(e.getMessage());
         }
+
+        // 10. test create table like with rollup
+        String createTableWithRollup = "CREATE TABLE IF NOT EXISTS test.table_with_rollup\n" +
+                "(\n" +
+                "    event_day DATE,\n" +
+                "    siteid INT DEFAULT '10',\n" +
+                "    citycode SMALLINT,\n" +
+                "    username VARCHAR(32) DEFAULT '',\n" +
+                "    pv BIGINT SUM DEFAULT '0'\n" +
+                ")\n" +
+                "AGGREGATE KEY(event_day, siteid, citycode, username)\n" +
+                "PARTITION BY RANGE(event_day)\n" +
+                "(\n" +
+                "    PARTITION p201706 VALUES LESS THAN ('2021-07-01'),\n" +
+                "    PARTITION p201707 VALUES LESS THAN ('2021-08-01'),\n" +
+                "    PARTITION p201708 VALUES LESS THAN ('2021-09-01')\n" +
+                ")\n" +
+                "DISTRIBUTED BY HASH(siteid) BUCKETS 10\n" +
+                "ROLLUP\n" +
+                "(\n" +
+                "r(event_day,pv),\n" +
+                "r1(event_day,siteid,pv),\n" +
+                "r2(siteid,pv),\n" +
+                "r3(siteid,citycode,username,pv)\n" +
+                ")\n" +
+                "PROPERTIES(\"replication_num\" = \"1\");";
+
+        String createTableLikeWithRollupSql1_1 = "create table test.table_like_rollup like test.table_with_rollup with rollup r1,r2";
+        String createTableLikeWithRollupSql1_2 = "create table test.table_like_rollup1 like test.table_with_rollup with rollup all";
+
+        String newDbName10 = "test";
+        String existedDbName10 = "test";
+        String newTblName10_1 = "table_like_rollup";
+        String newTblName10_2 = "table_like_rollup1";
+        String existedTblName10 = "table_with_rollup";
+        checkCreateOlapTableLike(createTableWithRollup, createTableLikeWithRollupSql1_1, newDbName10, existedDbName10, newTblName10_1, existedTblName10, 2);
+        checkCreateOlapTableLike(createTableWithRollup, createTableLikeWithRollupSql1_2, newDbName10, existedDbName10, newTblName10_2, existedTblName10, 4);
+
+        String createTableLikeWithRollupSql2_1 = "create table test2.table_like_rollup like test.table_with_rollup with rollup r1,r2";
+        String createTableLikeWithRollupSql2_2 = "create table test2.table_like_rollup1 like test.table_with_rollup with rollup all";
+
+        String newDbName11 = "test2";
+        String existedDbName11 = "test";
+        String newTblName11_1 = "table_like_rollup";
+        String newTblName11_2 = "table_like_rollup1";
+        String existedTblName11 = "table_with_rollup";
+        checkCreateOlapTableLike(createTableWithRollup, createTableLikeWithRollupSql2_1, newDbName11, existedDbName11, newTblName11_1, existedTblName11, 2);
+        checkCreateOlapTableLike(createTableWithRollup, createTableLikeWithRollupSql2_2, newDbName11, existedDbName11, newTblName11_2, existedTblName11, 4);
+
     }
 
     @Test
@@ -255,5 +328,40 @@ public class CreateTableLikeTest {
         String existedTblName2 = "testAbTbl1";
         ExceptionChecker.expectThrowsWithMsg(DdlException.class, "Unknown database 'default_cluster:fake_test'",
                 () -> checkCreateOlapTableLike(createTableSql2, createTableLikeSql2, newDbName2, existedDbName2, newTblName2, existedTblName2));
+
+        //3. add not existed rollup
+        String createTableWithRollup = "CREATE TABLE IF NOT EXISTS test.table_with_rollup\n" +
+                "(\n" +
+                "    event_day DATE,\n" +
+                "    siteid INT DEFAULT '10',\n" +
+                "    citycode SMALLINT,\n" +
+                "    username VARCHAR(32) DEFAULT '',\n" +
+                "    pv BIGINT SUM DEFAULT '0'\n" +
+                ")\n" +
+                "AGGREGATE KEY(event_day, siteid, citycode, username)\n" +
+                "PARTITION BY RANGE(event_day)\n" +
+                "(\n" +
+                "    PARTITION p201706 VALUES LESS THAN ('2021-07-01'),\n" +
+                "    PARTITION p201707 VALUES LESS THAN ('2021-08-01'),\n" +
+                "    PARTITION p201708 VALUES LESS THAN ('2021-09-01')\n" +
+                ")\n" +
+                "DISTRIBUTED BY HASH(siteid) BUCKETS 10\n" +
+                "ROLLUP\n" +
+                "(\n" +
+                "r(event_day,pv),\n" +
+                "r1(event_day,siteid,pv),\n" +
+                "r2(siteid,pv),\n" +
+                "r3(siteid,citycode,username,pv)\n" +
+                ")\n" +
+                "PROPERTIES(\"replication_num\" = \"1\");";
+
+        String createTableLikeWithRollupSq3 = "create table test.table_like_rollup like test.table_with_rollup with rollup r11";
+        String newDbName3 = "test";
+        String existedDbName3 = "test";
+        String newTblName3 = "table_like_rollup";
+        String existedTblName3 = "table_with_rollup";
+        ExceptionChecker.expectThrowsWithMsg(DdlException.class, "Rollup index[r11] not exists in Table[table_with_rollup]",
+                () -> checkCreateOlapTableLike(createTableWithRollup, createTableLikeWithRollupSq3, newDbName3, existedDbName3, newTblName3, existedTblName3, 1));
+
     }
 }

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