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/10/21 00:45:40 UTC

[doris] branch master updated: [feature](alter) support rename column for table with unique column id (#13410)

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 27d84eafc5 [feature](alter) support rename column for table with unique column id (#13410)
27d84eafc5 is described below

commit 27d84eafc5b549d5e07b2bf0ff6566a5126a75ba
Author: Xin Liao <li...@126.com>
AuthorDate: Fri Oct 21 08:45:34 2022 +0800

    [feature](alter) support rename column for table with unique column id (#13410)
---
 .../Alter/ALTER-TABLE-RENAME.md                    |  18 ++
 .../Alter/ALTER-TABLE-RENAME.md                    |  18 ++
 .../main/java/org/apache/doris/catalog/Env.java    | 109 ++++++++-
 .../org/apache/doris/journal/JournalEntity.java    |   6 +
 .../java/org/apache/doris/persist/EditLog.java     |   9 +
 .../org/apache/doris/persist/OperationType.java    |   1 +
 .../doris/persist/TableRenameColumnInfo.java       | 101 +++++++++
 .../data/schema_change_p0/test_rename_column.out   | Bin 0 -> 4101 bytes
 .../schema_change_p0/test_rename_column.groovy     | 245 +++++++++++++++++++++
 9 files changed, 504 insertions(+), 3 deletions(-)

diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-RENAME.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-RENAME.md
index 55e6376906..042488a335 100644
--- a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-RENAME.md
+++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-RENAME.md
@@ -66,6 +66,18 @@ grammar:
 RENAME PARTITION old_partition_name new_partition_name;
 ```
 
+4. Modify the column name
+
+grammar:
+
+```sql
+RENAME COLUMN old_column_name new_column_name;
+```
+
+Notice:
+- Currently only tables with column unique id are supported, which are created with property 'light_schema_change'.
+
+
 ### Example
 
 1. Modify the table named table1 to table2
@@ -86,6 +98,12 @@ ALTER TABLE example_table RENAME ROLLUP rollup1 rollup2;
 ALTER TABLE example_table RENAME PARTITION p1 p2;
 ```
 
+4. Modify the column named c1 in the table example_table to c2
+
+```sql
+ALTER TABLE example_table RENAME COLUMN c1 c2;
+```
+
 ### Keywords
 
 ```text
diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-RENAME.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-RENAME.md
index 9d3fa414b8..42b3c08fcc 100644
--- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-RENAME.md
+++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-RENAME.md
@@ -66,6 +66,18 @@ RENAME ROLLUP old_rollup_name new_rollup_name;
 RENAME PARTITION old_partition_name new_partition_name;    
 ```
 
+4.修改 column 名称
+
+语法:
+
+```sql
+RENAME COLUMN old_column_name new_column_name;    
+```
+
+注意:
+- 目前仅支持有 column unique id 的表, 通过 'light_schema_change' 属性创建。
+
+
 ### Example
 
 1. 将名为 table1 的表修改为 table2
@@ -86,6 +98,12 @@ ALTER TABLE example_table RENAME ROLLUP rollup1 rollup2;
 ALTER TABLE example_table RENAME PARTITION p1 p2;
 ```
 
+4. 将表 example_table 中名为 c1 的 column 修改为 c2
+
+```sql
+ALTER TABLE example_table RENAME COLUMN c1 c2;
+```
+
 ### Keywords
 
 ```text
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
index 28eb754b6b..f62f81ea54 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
@@ -190,6 +190,7 @@ import org.apache.doris.persist.StorageInfo;
 import org.apache.doris.persist.StorageInfoV2;
 import org.apache.doris.persist.TableInfo;
 import org.apache.doris.persist.TablePropertyInfo;
+import org.apache.doris.persist.TableRenameColumnInfo;
 import org.apache.doris.persist.TruncateTableInfo;
 import org.apache.doris.persist.meta.MetaHeader;
 import org.apache.doris.persist.meta.MetaReader;
@@ -4030,12 +4031,114 @@ public class Env {
         }
     }
 
+    public void renameColumn(Database db, OlapTable table, String colName,
+            String newColName, boolean isReplay) throws DdlException {
+        if (table.getState() != OlapTableState.NORMAL) {
+            throw new DdlException("Table[" + table.getName() + "] is under " + table.getState());
+        }
+
+        if (colName.equalsIgnoreCase(newColName)) {
+            throw new DdlException("Same column name");
+        }
+
+        Map<Long, MaterializedIndexMeta> indexIdToMeta = table.getIndexIdToMeta();
+        for (Map.Entry<Long, MaterializedIndexMeta> entry : indexIdToMeta.entrySet()) {
+            // rename column is not implemented for table without column unique id.
+            if (entry.getValue().getMaxColUniqueId() < 0) {
+                throw new DdlException("not implemented for table without column unique id,"
+                        + " which are created with property 'light_schema_change'.");
+            }
+            // check if new name is already used
+            if (entry.getValue().getColumnByName(newColName) != null) {
+                throw new DdlException("Column name[" + newColName + "] is already used");
+            }
+        }
+
+        // 1. modify MaterializedIndexMeta
+        boolean hasColumn = false;
+        for (Map.Entry<Long, MaterializedIndexMeta> entry : indexIdToMeta.entrySet()) {
+            Column column = entry.getValue().getColumnByName(colName);
+            if (column != null) {
+                column.setName(newColName);
+                hasColumn = true;
+            }
+        }
+        if (!hasColumn) {
+            throw new DdlException("Column[" + colName + "] does not exists");
+        }
+
+        // 2. modify partition key
+        PartitionInfo partitionInfo = table.getPartitionInfo();
+        List<Column> partitionColumns = partitionInfo.getPartitionColumns();
+        for (Column column : partitionColumns) {
+            if (column.getName().equalsIgnoreCase(colName)) {
+                column.setName(newColName);
+                break;
+            }
+        }
+
+        // 3. modify index
+        List<Index> indexes = table.getIndexes();
+        for (Index index : indexes) {
+            List<String> columns = index.getColumns();
+            for (int i = 0; i < columns.size(); i++) {
+                if (columns.get(i).equalsIgnoreCase(colName)) {
+                    columns.set(i, newColName);
+                }
+            }
+        }
+
+        // 4. modify distribution info
+        DistributionInfo distributionInfo = table.getDefaultDistributionInfo();
+        if (distributionInfo.getType() == DistributionInfoType.HASH) {
+            List<Column> distributionColumns = ((HashDistributionInfo) distributionInfo).getDistributionColumns();
+            for (Column column : distributionColumns) {
+                if (column.getName().equalsIgnoreCase(colName)) {
+                    column.setName(newColName);
+                    break;
+                }
+            }
+        }
+
+        table.rebuildFullSchema();
+
+        if (!isReplay) {
+            // log
+            TableRenameColumnInfo info = new TableRenameColumnInfo(db.getId(), table.getId(), colName, newColName);
+            editLog.logColumnRename(info);
+            LOG.info("rename coloumn[{}] to {}", colName, newColName);
+        }
+    }
+
     public void renameColumn(Database db, OlapTable table, ColumnRenameClause renameClause) throws DdlException {
-        throw new DdlException("not implemented");
+        table.writeLockOrDdlException();
+        try {
+            String colName = renameClause.getColName();
+            String newColName = renameClause.getNewColName();
+            renameColumn(db, table, colName, newColName, false);
+        } finally {
+            table.writeUnlock();
+        }
     }
 
-    public void replayRenameColumn(TableInfo tableInfo) throws DdlException {
-        throw new DdlException("not implemented");
+    public void replayRenameColumn(TableRenameColumnInfo info) throws MetaNotFoundException {
+        LOG.debug("info:{}", info);
+        long dbId = info.getDbId();
+        long tableId = info.getTableId();
+        String colName = info.getColName();
+        String newColName = info.getNewColName();
+
+        Database db = getCurrentEnv().getInternalCatalog().getDbOrMetaException(dbId);
+        OlapTable table = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
+        table.writeLock();
+        try {
+            renameColumn(db, table, colName, newColName, true);
+        } catch (DdlException e) {
+            // should not happen
+            LOG.warn("failed to replay rename column", e);
+        } finally {
+            table.writeUnlock();
+        }
     }
 
     public void modifyTableDynamicPartition(Database db, OlapTable table, Map<String, String> properties)
diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java
index 7782fdcd0f..d45ac59a35 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java
@@ -92,6 +92,7 @@ import org.apache.doris.persist.SetReplicaStatusOperationLog;
 import org.apache.doris.persist.TableAddOrDropColumnsInfo;
 import org.apache.doris.persist.TableInfo;
 import org.apache.doris.persist.TablePropertyInfo;
+import org.apache.doris.persist.TableRenameColumnInfo;
 import org.apache.doris.persist.TruncateTableInfo;
 import org.apache.doris.plugin.PluginInfo;
 import org.apache.doris.policy.DropPolicyLog;
@@ -256,6 +257,11 @@ public class JournalEntity implements Writable {
                 isRead = true;
                 break;
             }
+            case OperationType.OP_RENAME_COLUMN: {
+                data = TableRenameColumnInfo.read(in);
+                isRead = true;
+                break;
+            }
             case OperationType.OP_MODIFY_VIEW_DEF: {
                 data = AlterViewInfo.read(in);
                 isRead = true;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
index f47f1314cf..4d6c0314b3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
@@ -279,6 +279,11 @@ public class EditLog {
                     env.replayRenamePartition(info);
                     break;
                 }
+                case OperationType.OP_RENAME_COLUMN: {
+                    TableRenameColumnInfo info = (TableRenameColumnInfo) journal.getData();
+                    env.replayRenameColumn(info);
+                    break;
+                }
                 case OperationType.OP_BACKUP_JOB: {
                     BackupJob job = (BackupJob) journal.getData();
                     env.getBackupHandler().replayAddJob(job);
@@ -1220,6 +1225,10 @@ public class EditLog {
         logEdit(OperationType.OP_RENAME_PARTITION, tableInfo);
     }
 
+    public void logColumnRename(TableRenameColumnInfo info) {
+        logEdit(OperationType.OP_RENAME_COLUMN, info);
+    }
+
     public void logCreateCluster(Cluster cluster) {
         logEdit(OperationType.OP_CREATE_CLUSTER, cluster);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java
index aa11dfdbed..e1064e22b1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java
@@ -46,6 +46,7 @@ public class OperationType {
     public static final short OP_RECOVER_PARTITION = 18;
     public static final short OP_RENAME_TABLE = 19;
     public static final short OP_RENAME_PARTITION = 110;
+    public static final short OP_RENAME_COLUMN = 115;
     public static final short OP_BACKUP_JOB = 116;
     public static final short OP_RESTORE_JOB = 117;
     public static final short OP_TRUNCATE_TABLE = 118;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/TableRenameColumnInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/TableRenameColumnInfo.java
new file mode 100644
index 0000000000..f7a388756d
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/TableRenameColumnInfo.java
@@ -0,0 +1,101 @@
+// 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.persist;
+
+import org.apache.doris.common.io.Text;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.persist.gson.GsonUtils;
+
+import com.google.gson.annotations.SerializedName;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * PersistInfo for Table rename column info
+ */
+public class TableRenameColumnInfo implements Writable {
+    @SerializedName(value = "dbId")
+    private long dbId;
+    @SerializedName(value = "tableId")
+    private long tableId;
+    @SerializedName(value = "colName")
+    private String colName;
+    @SerializedName(value = "newColName")
+    private String newColName;
+
+    public TableRenameColumnInfo(long dbId, long tableId,
+            String colName, String newColName) {
+        this.dbId = dbId;
+        this.tableId = tableId;
+        this.colName = colName;
+        this.newColName = newColName;
+    }
+
+    public long getDbId() {
+        return dbId;
+    }
+
+    public long getTableId() {
+        return tableId;
+    }
+
+    public String getColName() {
+        return colName;
+    }
+
+    public String getNewColName() {
+        return newColName;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        Text.writeString(out, GsonUtils.GSON.toJson(this));
+    }
+
+    public static TableRenameColumnInfo read(DataInput in) throws IOException {
+        return GsonUtils.GSON.fromJson(Text.readString(in), TableRenameColumnInfo.class);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj == this) {
+            return true;
+        }
+
+        if (!(obj instanceof TableRenameColumnInfo)) {
+            return false;
+        }
+
+        TableRenameColumnInfo info = (TableRenameColumnInfo) obj;
+
+        return (dbId == info.dbId && tableId == tableId
+                && colName.equals(info.colName) && newColName.equals(info.newColName));
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append(" dbId: ").append(dbId);
+        sb.append(" tableId: ").append(tableId);
+        sb.append(" colName: ").append(colName);
+        sb.append(" newColName: ").append(newColName);
+        return sb.toString();
+    }
+}
diff --git a/regression-test/data/schema_change_p0/test_rename_column.out b/regression-test/data/schema_change_p0/test_rename_column.out
new file mode 100644
index 0000000000..5bfcba17d8
Binary files /dev/null and b/regression-test/data/schema_change_p0/test_rename_column.out differ
diff --git a/regression-test/suites/schema_change_p0/test_rename_column.groovy b/regression-test/suites/schema_change_p0/test_rename_column.groovy
new file mode 100644
index 0000000000..aeb2de2ca4
--- /dev/null
+++ b/regression-test/suites/schema_change_p0/test_rename_column.groovy
@@ -0,0 +1,245 @@
+// 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.
+
+suite ("test_rename_column") {
+    def tableName = "rename_column_test"
+    sql """ DROP TABLE IF EXISTS ${tableName} """
+    sql """
+        CREATE TABLE ${tableName} (
+            `user_id` LARGEINT NOT NULL COMMENT "用户id",
+            `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+            `city` VARCHAR(20) COMMENT "用户所在城市",
+            `age` SMALLINT COMMENT "用户年龄",
+            `sex` TINYINT COMMENT "用户性别",
+            `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+            `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间",
+            `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+            `cost` BIGINT DEFAULT "0" COMMENT "用户总消费",
+            `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间",
+            `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间")
+        UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+        PROPERTIES ( "replication_num" = "1" , "light_schema_change" = "true")
+        """
+    qt_desc """ desc ${tableName} """
+
+    sql """ 
+        INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20)
+        """
+    sql """
+        INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21)
+        """
+    sql """ sync """
+
+    qt_select """ SELECT * FROM ${tableName} order by user_id ASC, last_visit_date """
+
+    // rename key column
+    sql """ ALTER table ${tableName} RENAME COLUMN  user_id new_user_id """
+
+    sql """ sync """
+
+    qt_select """ SELECT * FROM ${tableName} order by new_user_id DESC, last_visit_date """
+
+    qt_desc """ desc ${tableName} """
+
+    sql """
+        INSERT INTO ${tableName} VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 32, 22)
+        """
+    qt_select """ SELECT * FROM ${tableName} order by new_user_id DESC, last_visit_date """
+
+    // rename value column
+    sql """
+        ALTER table ${tableName} RENAME COLUMN  max_dwell_time new_max_dwell_time
+        """
+    sql """
+        INSERT INTO ${tableName} VALUES
+                (4, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 33, 23)
+        """
+    sql """ sync """
+
+    qt_select """ SELECT * FROM ${tableName} order by new_user_id DESC, last_visit_date """
+
+    qt_desc """ desc ${tableName} """
+
+    test {
+        sql """ ALTER table ${tableName} RENAME COLUMN  date city """
+        exception "Column name[city] is already used"
+    }
+
+    test {
+        sql """ ALTER table ${tableName} RENAME COLUMN  aaa  bbb """
+        exception "Column[aaa] does not exists"
+    }
+
+    test {
+        sql """ ALTER table ${tableName} RENAME COLUMN  date  date """
+        exception "Same column name"
+    }
+
+    sql """ DROP TABLE ${tableName} """
+
+    // table without column unique id
+    sql """
+        CREATE TABLE ${tableName} (
+            `user_id` LARGEINT NOT NULL COMMENT "用户id",
+            `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+            `city` VARCHAR(20) COMMENT "用户所在城市",
+            `age` SMALLINT COMMENT "用户年龄",
+            `sex` TINYINT COMMENT "用户性别",
+            `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+            `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间",
+            `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+            `cost` BIGINT DEFAULT "0" COMMENT "用户总消费",
+            `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间",
+            `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间")
+        UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+        PROPERTIES ( "replication_num" = "1" , "light_schema_change" = "false")
+        """
+    test {
+        sql """ ALTER table ${tableName} RENAME COLUMN  date new_date """
+        exception "not implemented"
+    }
+    sql """ DROP TABLE ${tableName} """
+
+    // rollup
+    sql """
+            CREATE TABLE ${tableName} (
+                `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                `city` VARCHAR(20) COMMENT "用户所在城市",
+                `age` SMALLINT COMMENT "用户年龄",
+                `sex` TINYINT COMMENT "用户性别",
+                `cost` BIGINT SUM DEFAULT "0" COMMENT "用户总消费",
+                `max_dwell_time` INT MAX DEFAULT "0" COMMENT "用户最大停留时间",
+                `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "用户最小停留时间",
+                `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列",
+                `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列")
+            AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+            BUCKETS 1
+            PROPERTIES ( "replication_num" = "1", "light_schema_change" = "true" );
+        """
+    qt_desc """ desc ${tableName} """
+
+    //add rollup
+    def resRoll = "null"
+    def rollupName = "rollup_cost"
+    sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`, `cost`);"
+    while (!resRoll.contains("FINISHED")){
+        resRoll = sql "SHOW ALTER TABLE ROLLUP WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+        resRoll = resRoll.toString()
+        logger.info("result: ${resRoll}")
+        if(resRoll.contains("CANCELLED")){
+            return
+        }
+        Thread.sleep(500)
+    }
+
+    qt_select """ select user_id, cost from ${tableName} order by user_id """
+
+    sql """ INSERT INTO ${tableName} VALUES
+            (1, '2017-10-01', 'Beijing', 10, 1, 1, 30, 20, hll_hash(1), to_bitmap(1))
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+            (1, '2017-10-02', 'Beijing', 10, 1, 1, 31, 19, hll_hash(2), to_bitmap(2))
+        """
+    sql """ sync """
+
+    qt_select """ select * from ${tableName} order by user_id """
+
+    qt_select """ select user_id, sum(cost) from ${tableName} group by user_id order by user_id """
+
+    sql """ ALTER TABLE ${tableName} RENAME COLUMN user_id new_user_id """
+
+    sql """ INSERT INTO ${tableName} VALUES
+            (2, '2017-10-01', 'Beijing', 10, 1, 1, 31, 21, hll_hash(2), to_bitmap(2))
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+            (2, '2017-10-02', 'Beijing', 10, 1, 1, 32, 20, hll_hash(3), to_bitmap(3))
+        """
+    qt_desc """ desc ${tableName} """
+
+    qt_select""" select * from ${tableName} order by new_user_id """
+
+    qt_select """ select new_user_id, sum(cost) from ${tableName} group by new_user_id order by new_user_id """
+
+    sql """ DROP TABLE ${tableName} """
+
+    // materialized view
+    sql """
+            CREATE TABLE ${tableName} (
+                `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                `city` VARCHAR(20) COMMENT "用户所在城市",
+                `age` SMALLINT COMMENT "用户年龄",
+                `sex` TINYINT COMMENT "用户性别",
+                `cost` BIGINT SUM DEFAULT "0" COMMENT "用户总消费",
+                `max_dwell_time` INT MAX DEFAULT "0" COMMENT "用户最大停留时间",
+                `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "用户最小停留时间",
+                `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列",
+                `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列")
+            AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+            BUCKETS 1
+            PROPERTIES ( "replication_num" = "1", "light_schema_change" = "true" );
+        """
+
+    //add materialized view
+    def resMv = "null"
+    def mvName = "mv1"
+    sql "create materialized view ${mvName} as select user_id, sum(cost) from ${tableName} group by user_id;"
+    while (!resMv.contains("FINISHED")){
+        resMv = sql "SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+        resMv = resMv.toString()
+        logger.info("result: ${resMv}")
+        if(resMv.contains("CANCELLED")){
+            return
+        }
+        Thread.sleep(500)
+    }
+
+    qt_select """ select user_id, cost from ${tableName} order by user_id """
+
+    sql """ INSERT INTO ${tableName} VALUES
+            (1, '2017-10-01', 'Beijing', 10, 1, 1, 30, 20, hll_hash(1), to_bitmap(1))
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+            (1, '2017-10-02', 'Beijing', 10, 1, 1, 31, 19, hll_hash(2), to_bitmap(2))
+        """
+    sql """ sync """
+
+    qt_select """ select * from ${tableName} order by user_id """
+
+    qt_select """ select user_id, sum(cost) from ${tableName} group by user_id order by user_id """
+
+    sql """ ALTER TABLE ${tableName} RENAME COLUMN user_id new_user_id """
+
+    sql """ INSERT INTO ${tableName} VALUES
+            (2, '2017-10-01', 'Beijing', 10, 1, 1, 31, 21, hll_hash(2), to_bitmap(2))
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+            (2, '2017-10-02', 'Beijing', 10, 1, 1, 32, 20, hll_hash(3), to_bitmap(3))
+        """
+    qt_desc """ desc ${tableName} """
+
+    qt_select""" select * from ${tableName} order by new_user_id """
+
+    qt_select """ select new_user_id, sum(cost) from ${tableName} group by new_user_id order by new_user_id """
+
+    sql """ DROP TABLE ${tableName} """
+
+}


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