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 2023/01/14 02:20:16 UTC

[doris] branch master updated: [Bugfix] (ROLLUP) fix the coredump when add rollup by link schema change (#15654)

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 313e14d220 [Bugfix] (ROLLUP) fix the coredump when add rollup by link schema change (#15654)
313e14d220 is described below

commit 313e14d220e1e54f066162d8b38bf14f3a89cbf9
Author: Lightman <31...@users.noreply.github.com>
AuthorDate: Sat Jan 14 10:20:07 2023 +0800

    [Bugfix] (ROLLUP) fix the coredump when add rollup by link schema change (#15654)
    
    Because of the rollup has the same keys and the keys's order is same, BE will do linked schema change. The base tablet's segments will link to the new rollup tablet. But the unique id from the base tablet is starting from 0 and as the rollup tablet also. In this case, the unique id 4 in the base table is column 'city', but in the rollup tablet is 'cost'. It will decode the varcode page to bigint page so that be coredump. It needs to be rejected.
    
    I think that if a rollup add by link schema change, it means this rollup is redundant. It brings no additional revenue and wastes storage space. So It needs to be rejected.
---
 be/src/olap/schema_change.cpp                      | 18 ++++++
 .../org/apache/doris/task/AlterReplicaTask.java    | 11 ++++
 .../data/rollup_p0/test_rollup_agg_date.out        |  2 +-
 .../link_schema_change/test_rollup_agg_fail.groovy | 70 ++++++++++++++++++++++
 .../link_schema_change/test_rollup_dup_fail.groovy | 53 ++++++++++++++++
 .../link_schema_change/test_rollup_uni_fail.groovy | 53 ++++++++++++++++
 .../suites/rollup_p0/test_rollup_agg_date.groovy   |  2 +-
 .../test_agg_mv_schema_change.groovy               |  2 +-
 .../test_agg_rollup_schema_change.groovy           |  2 +-
 .../test_dup_mv_schema_change.groovy               |  2 +-
 .../test_dup_rollup_schema_change.groovy           |  2 +-
 .../test_uniq_mv_schema_change.groovy              |  2 +-
 .../test_uniq_rollup_schema_change.groovy          |  2 +-
 13 files changed, 213 insertions(+), 8 deletions(-)

diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp
index 3fed9c3c01..5144e940fa 100644
--- a/be/src/olap/schema_change.cpp
+++ b/be/src/olap/schema_change.cpp
@@ -823,6 +823,18 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2&
         sc_params.ref_rowset_readers = rs_readers;
         sc_params.delete_handler = &delete_handler;
         sc_params.base_tablet_schema = base_tablet_schema;
+        DCHECK(request.__isset.alter_tablet_type);
+        switch (request.alter_tablet_type) {
+        case TAlterTabletType::SCHEMA_CHANGE:
+            sc_params.alter_tablet_type = AlterTabletType::SCHEMA_CHANGE;
+            break;
+        case TAlterTabletType::ROLLUP:
+            sc_params.alter_tablet_type = AlterTabletType::ROLLUP;
+            break;
+        case TAlterTabletType::MIGRATION:
+            sc_params.alter_tablet_type = AlterTabletType::MIGRATION;
+            break;
+        }
         if (request.__isset.materialized_view_params) {
             for (auto item : request.materialized_view_params) {
                 AlterMaterializedViewParam mv_param;
@@ -1024,6 +1036,12 @@ Status SchemaChangeHandler::_convert_historical_rowsets(const SchemaChangeParams
         return process_alter_exit();
     }
 
+    if (!sc_sorting && !sc_directly && sc_params.alter_tablet_type == AlterTabletType::ROLLUP) {
+        res = Status::Error<SCHEMA_SCHEMA_INVALID>(
+                "Don't support to add materialized view by linked schema change");
+        return process_alter_exit();
+    }
+
     // b. Generate historical data converter
     auto sc_procedure = get_sc_procedure(rb_changer, sc_sorting, sc_directly);
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/AlterReplicaTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/AlterReplicaTask.java
index d9ac6f0bb4..191915833c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/AlterReplicaTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/AlterReplicaTask.java
@@ -24,6 +24,7 @@ import org.apache.doris.analysis.SlotRef;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.thrift.TAlterMaterializedViewParam;
 import org.apache.doris.thrift.TAlterTabletReqV2;
+import org.apache.doris.thrift.TAlterTabletType;
 import org.apache.doris.thrift.TColumn;
 import org.apache.doris.thrift.TTaskType;
 
@@ -109,6 +110,16 @@ public class AlterReplicaTask extends AgentTask {
     public TAlterTabletReqV2 toThrift() {
         TAlterTabletReqV2 req = new TAlterTabletReqV2(baseTabletId, signature, baseSchemaHash, newSchemaHash);
         req.setAlterVersion(version);
+        switch (jobType) {
+            case ROLLUP:
+                req.setAlterTabletType(TAlterTabletType.ROLLUP);
+                break;
+            case SCHEMA_CHANGE:
+                req.setAlterTabletType(TAlterTabletType.SCHEMA_CHANGE);
+                break;
+            default:
+                break;
+        }
         if (defineExprs != null) {
             for (Map.Entry<String, Expr> entry : defineExprs.entrySet()) {
                 List<SlotRef> slots = Lists.newArrayList();
diff --git a/regression-test/data/rollup_p0/test_rollup_agg_date.out b/regression-test/data/rollup_p0/test_rollup_agg_date.out
index 887cd41a3c..c03adfe3f8 100644
--- a/regression-test/data/rollup_p0/test_rollup_agg_date.out
+++ b/regression-test/data/rollup_p0/test_rollup_agg_date.out
@@ -11,8 +11,8 @@ test_rollup_agg_date	AGG_KEYS	datek1	DATEV2	Yes	true	\N		true
 		datetimev4	DATETIMEV2(3)	Yes	false	\N	MAX	true
 								
 rollup_date	AGG_KEYS	datek1	DATEV2	Yes	true	\N		true
-		datetimek1	DATETIMEV2(0)	Yes	true	\N		true
 		datetimek2	DATETIMEV2(3)	Yes	true	\N		true
+		datetimek1	DATETIMEV2(0)	Yes	true	\N		true
 		datetimek3	DATETIMEV2(6)	Yes	true	\N		true
 		datev1	DATEV2	No	false	\N	MAX	true
 		datetimev1	DATETIMEV2(0)	No	false	\N	MAX	true
diff --git a/regression-test/suites/rollup_p0/link_schema_change/test_rollup_agg_fail.groovy b/regression-test/suites/rollup_p0/link_schema_change/test_rollup_agg_fail.groovy
new file mode 100644
index 0000000000..598b0dfed8
--- /dev/null
+++ b/regression-test/suites/rollup_p0/link_schema_change/test_rollup_agg_fail.groovy
@@ -0,0 +1,70 @@
+// 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_rollup_add_fail") {
+    def tableName = "test_rollup_add_fail"
+
+    /* agg */
+    sql """ DROP TABLE IF EXISTS ${tableName} FORCE"""
+    sql """
+            CREATE TABLE IF NOT EXISTS ${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 (failed)
+    def result = "null"
+    def mvName = "mv1"
+    sql "create materialized view ${mvName} as select user_id, date, city, age, sex, sum(cost) from ${tableName} group by user_id, date, city, age, sex;"
+    while (!result.contains("CANCELLED")){
+        result = sql "SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+        result = result.toString()
+        logger.info("result: ${result}")
+        if(result.contains("FINISHED")){
+            assertTrue(false);
+        }
+        Thread.sleep(100)
+    }
+
+    Thread.sleep(1000)
+
+    //add rollup (failed)
+    result = "null"
+    def rollupName = "rollup_cost"
+    sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`age`, `sex`, cost);"
+    while (!result.contains("CANCELLED")){
+        result = sql "SHOW ALTER TABLE ROLLUP WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+        result = result.toString()
+        logger.info("result: ${result}")
+        if(result.contains("FINISHED")){
+            assertTrue(false);
+        }
+        Thread.sleep(100)
+    }
+}
diff --git a/regression-test/suites/rollup_p0/link_schema_change/test_rollup_dup_fail.groovy b/regression-test/suites/rollup_p0/link_schema_change/test_rollup_dup_fail.groovy
new file mode 100644
index 0000000000..a3a1c726b8
--- /dev/null
+++ b/regression-test/suites/rollup_p0/link_schema_change/test_rollup_dup_fail.groovy
@@ -0,0 +1,53 @@
+// 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_rollup_dup_fail") {
+    def tableName = "test_rollup_dup_fail"
+
+    sql """ DROP TABLE IF EXISTS ${tableName} FORCE"""
+    sql """
+        CREATE TABLE IF NOT EXISTS ${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 "用户最小停留时间")
+        DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+        BUCKETS 1
+        PROPERTIES ( "replication_num" = "1", "light_schema_change" = "true" );
+    """
+
+    //add rollup (failed)
+    result = "null"
+    rollupName = "rollup_cost"
+    sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`age`,`sex`) DUPLICATE KEY (`user_id`,`date`,`city`,`age`,`sex`);"
+    while (!result.contains("CANCELLED")){
+        result = sql "SHOW ALTER TABLE ROLLUP WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+        result = result.toString()
+        logger.info("result: ${result}")
+        if(result.contains("FINISHED")){
+            assertTrue(false);
+        }
+        Thread.sleep(100)
+    }
+}
diff --git a/regression-test/suites/rollup_p0/link_schema_change/test_rollup_uni_fail.groovy b/regression-test/suites/rollup_p0/link_schema_change/test_rollup_uni_fail.groovy
new file mode 100644
index 0000000000..41f1443bff
--- /dev/null
+++ b/regression-test/suites/rollup_p0/link_schema_change/test_rollup_uni_fail.groovy
@@ -0,0 +1,53 @@
+// 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_rollup_uni_fail") {
+    def tableName = "test_rollup_uni_fail"
+
+    /* unique */
+    sql """ DROP TABLE IF EXISTS ${tableName} FORCE"""
+    sql """
+        CREATE TABLE IF NOT EXISTS ${tableName} (
+                `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                `age` SMALLINT COMMENT "用户年龄",
+                `sex` TINYINT COMMENT "用户性别",
+                `city` VARCHAR(20) DEFAULT "beijing "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`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+            BUCKETS 1
+            PROPERTIES ( "replication_num" = "1", "light_schema_change" = "true");
+    """
+
+    result = "null"
+    rollupName = "rollup_cost"
+    sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`age`, `sex`, cost);"
+    while (!result.contains("CANCELLED")){
+        result = sql "SHOW ALTER TABLE ROLLUP WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+        result = result.toString()
+        logger.info("result: ${result}")
+        if(result.contains("FINISHED")){
+            assertTrue(false);
+        }
+        Thread.sleep(100)
+    }
+}
diff --git a/regression-test/suites/rollup_p0/test_rollup_agg_date.groovy b/regression-test/suites/rollup_p0/test_rollup_agg_date.groovy
index 40f1313c41..a65a03a97b 100644
--- a/regression-test/suites/rollup_p0/test_rollup_agg_date.groovy
+++ b/regression-test/suites/rollup_p0/test_rollup_agg_date.groovy
@@ -40,7 +40,7 @@ suite("test_rollup_agg_date", "rollup") {
             AGGREGATE KEY (datek1, datetimek1, datetimek2, datetimek3)
             DISTRIBUTED BY HASH(datek1) BUCKETS 5 properties("replication_num" = "1");
         """
-    sql """ALTER TABLE ${tbName} ADD ROLLUP rollup_date(datek1,datetimek1,datetimek2,datetimek3,datev1,datetimev1,datetimev2,datetimev3);"""
+    sql """ALTER TABLE ${tbName} ADD ROLLUP rollup_date(datek1,datetimek2,datetimek1,datetimek3,datev1,datetimev1,datetimev2,datetimev3);"""
     int max_try_secs = 60
     while (max_try_secs--) {
         String res = getJobRollupState(tbName)
diff --git a/regression-test/suites/schema_change_p0/test_agg_mv_schema_change.groovy b/regression-test/suites/schema_change_p0/test_agg_mv_schema_change.groovy
index 5c7f29b91b..5bcef9b8d3 100644
--- a/regression-test/suites/schema_change_p0/test_agg_mv_schema_change.groovy
+++ b/regression-test/suites/schema_change_p0/test_agg_mv_schema_change.groovy
@@ -79,7 +79,7 @@ suite ("test_agg_mv_schema_change") {
         //add materialized view
         def result = "null"
         def mvName = "mv1"
-        sql "create materialized view ${mvName} as select user_id, date, city, age, sex, sum(cost) from ${tableName} group by user_id, date, city, age, sex, cost;"
+        sql "create materialized view ${mvName} as select user_id, date, city, age, sum(cost) from ${tableName} group by user_id, date, city, age, sex;"
         while (!result.contains("FINISHED")){
             result = sql "SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
             result = result.toString()
diff --git a/regression-test/suites/schema_change_p0/test_agg_rollup_schema_change.groovy b/regression-test/suites/schema_change_p0/test_agg_rollup_schema_change.groovy
index 196f3a0071..acd9a82931 100644
--- a/regression-test/suites/schema_change_p0/test_agg_rollup_schema_change.groovy
+++ b/regression-test/suites/schema_change_p0/test_agg_rollup_schema_change.groovy
@@ -80,7 +80,7 @@ suite ("test_agg_rollup_schema_change") {
         //add rollup
         def result = "null"
         def rollupName = "rollup_cost"
-        sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`age`,`sex`, cost);"
+        sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`age`, cost);"
         while (!result.contains("FINISHED")){
             result = sql "SHOW ALTER TABLE ROLLUP WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
             result = result.toString()
diff --git a/regression-test/suites/schema_change_p0/test_dup_mv_schema_change.groovy b/regression-test/suites/schema_change_p0/test_dup_mv_schema_change.groovy
index d62415f67a..9e3c885a0c 100644
--- a/regression-test/suites/schema_change_p0/test_dup_mv_schema_change.groovy
+++ b/regression-test/suites/schema_change_p0/test_dup_mv_schema_change.groovy
@@ -79,7 +79,7 @@ suite ("test_dup_mv_schema_change") {
         //add materialized view
         def result = "null"
         def mvName = "mv1"
-        sql "create materialized view ${mvName} as select user_id, date, city, age,sex from ${tableName};"
+        sql "create materialized view ${mvName} as select user_id, date, city, age from ${tableName};"
         while (!result.contains("FINISHED")){
             result = sql "SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
             result = result.toString()
diff --git a/regression-test/suites/schema_change_p0/test_dup_rollup_schema_change.groovy b/regression-test/suites/schema_change_p0/test_dup_rollup_schema_change.groovy
index 67c4763b7e..94e6510555 100644
--- a/regression-test/suites/schema_change_p0/test_dup_rollup_schema_change.groovy
+++ b/regression-test/suites/schema_change_p0/test_dup_rollup_schema_change.groovy
@@ -78,7 +78,7 @@ suite ("test_dup_rollup_schema_change") {
         //add rollup
         def result = "null"
         def rollupName = "rollup_cost"
-        sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`age`,`sex`, cost);"
+        sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`age`, cost);"
         while (!result.contains("FINISHED")){
             result = sql "SHOW ALTER TABLE ROLLUP WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
             result = result.toString()
diff --git a/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy b/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy
index 70d836111e..cad59ed449 100644
--- a/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy
+++ b/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy
@@ -78,7 +78,7 @@ suite ("test_uniq_mv_schema_change") {
     //add materialized view
     def result = "null"
     def mvName = "mv1"
-    sql "create materialized view ${mvName} as select user_id, date, city, age, sex from ${tableName} group by user_id, date, city, age, sex;"
+    sql "create materialized view ${mvName} as select user_id, date, city, age from ${tableName} group by user_id, date, city, age;"
     while (!result.contains("FINISHED")){
         result = sql "SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
         result = result.toString()
diff --git a/regression-test/suites/schema_change_p0/test_uniq_rollup_schema_change.groovy b/regression-test/suites/schema_change_p0/test_uniq_rollup_schema_change.groovy
index fcbef30a85..ceef82590c 100644
--- a/regression-test/suites/schema_change_p0/test_uniq_rollup_schema_change.groovy
+++ b/regression-test/suites/schema_change_p0/test_uniq_rollup_schema_change.groovy
@@ -78,7 +78,7 @@ suite ("test_uniq_rollup_schema_change") {
     //add rollup
     def result = "null"
     def rollupName = "rollup_cost"
-    sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`age`,`sex`, cost);"
+    sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`sex`, `age`, cost);"
     while (!result.contains("FINISHED")){
         result = sql "SHOW ALTER TABLE ROLLUP WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
         result = result.toString()


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