You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by da...@apache.org on 2023/06/02 01:22:47 UTC

[doris] branch master updated: [feature-wip](duplicate-no-keys) schame change support for duplicate no keys (#19326)

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

dataroaring 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 ecdc5124be [feature-wip](duplicate-no-keys) schame change support for duplicate no keys (#19326)
ecdc5124be is described below

commit ecdc5124bed42a4fd33007a77d9ed3ab93497584
Author: xueweizhang <zx...@163.com>
AuthorDate: Fri Jun 2 09:22:41 2023 +0800

    [feature-wip](duplicate-no-keys) schame change support for duplicate no keys (#19326)
---
 docs/en/docs/data-table/data-model.md              | 44 ++++++++++++
 .../Create/CREATE-TABLE.md                         | 38 +++++++++++
 docs/zh-CN/docs/data-table/data-model.md           | 47 +++++++++++++
 .../Create/CREATE-TABLE.md                         | 38 +++++++++++
 .../main/java/org/apache/doris/common/Config.java  |  7 --
 .../doris/alter/MaterializedViewHandler.java       | 12 +++-
 .../apache/doris/alter/SchemaChangeHandler.java    |  8 ++-
 .../org/apache/doris/analysis/CreateTableStmt.java | 12 +++-
 .../analysis/ModifyTablePropertiesClause.java      |  2 +
 .../main/java/org/apache/doris/catalog/Env.java    |  8 +++
 .../apache/doris/common/util/PropertyAnalyzer.java | 22 ++++++
 .../apache/doris/datasource/InternalCatalog.java   |  5 +-
 .../org/apache/doris/alter/AlterJobV2Test.java     | 36 ++++++++++
 .../apache/doris/analysis/CreateTableStmtTest.java | 32 +++++++++
 .../test_duplicate_table_without_keys.out          | 54 +++++++++++++++
 .../test_duplicate_table_without_keys.groovy       | 78 ++++++++++++++++++++++
 16 files changed, 429 insertions(+), 14 deletions(-)

diff --git a/docs/en/docs/data-table/data-model.md b/docs/en/docs/data-table/data-model.md
index bc581e0629..86e6d8a0ba 100644
--- a/docs/en/docs/data-table/data-model.md
+++ b/docs/en/docs/data-table/data-model.md
@@ -437,6 +437,50 @@ Different from the Aggregate and Unique Models, the Duplicate Model stores the d
 
 The Duplicate Model is suitable for storing raw data without aggregation requirements or primary key uniqueness constraints. For more usage scenarios, see the [Limitations of Aggregate Model](#Limitations of Aggregate Model) section.
 
+### Duplicate Model without SORTING COLUMN (Since Doris 2.0)
+
+When creating a table without specifying Unique, Aggregate, or Duplicate, a table with a Duplicate model will be created by default, and the SORTING COLUMN will be automatically specified.
+
+When users do not need SORTING COLUMN or Prefix Index, they can configure the following table property:
+
+```
+"enable_duplicate_without_keys_by_default" = "true"
+```
+
+Then, when creating the default model, the sorting column will no longer be specified, and no prefix index will be created for the table to reduce additional overhead in importing and storing.
+
+The corresponding CREATE TABLE statement is as follows:
+
+```sql
+CREATE TABLE IF NOT EXISTS example_db.example_tbl
+(
+    `timestamp` DATETIME NOT NULL COMMENT "日志时间",
+    `type` INT NOT NULL COMMENT "日志类型",
+    `error_code` INT COMMENT "错误码",
+    `error_msg` VARCHAR(1024) COMMENT "错误详细信息",
+    `op_id` BIGINT COMMENT "负责人id",
+    `op_time` DATETIME COMMENT "处理时间"
+)
+DISTRIBUTED BY HASH(`type`) BUCKETS 1
+PROPERTIES (
+"replication_allocation" = "tag.location.default: 1",
+"enable_duplicate_without_keys_by_default" = "true"
+);
+
+MySQL > desc example_tbl;
++------------+---------------+------+-------+---------+-------+
+| Field      | Type          | Null | Key   | Default | Extra |
++------------+---------------+------+-------+---------+-------+
+| timestamp  | DATETIME      | No   | false | NULL    | NONE  |
+| type       | INT           | No   | false | NULL    | NONE  |
+| error_code | INT           | Yes  | false | NULL    | NONE  |
+| error_msg  | VARCHAR(1024) | Yes  | false | NULL    | NONE  |
+| op_id      | BIGINT        | Yes  | false | NULL    | NONE  |
+| op_time    | DATETIME      | Yes  | false | NULL    | NONE  |
++------------+---------------+------+-------+---------+-------+
+6 rows in set (0.01 sec)
+```
+
 ## Limitations of Aggregate Model
 
 This section is about the limitations of the Aggregate Model.
diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE.md
index 8aef9be367..f7c7c12f09 100644
--- a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE.md
+++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE.md
@@ -188,6 +188,10 @@ distribution_desc
     * AGGREGATE KEY: The specified column is the dimension column.
     * UNIQUE KEY: The subsequent specified column is the primary key column.
 
+    <version since="2.0">
+    NOTE: when set table property `"enable_duplicate_without_keys_by_default" = "true"`, will create a duplicate model without sorting columns and prefix indexes by default.
+    </version>
+
     Example:
 
     ```
@@ -378,6 +382,12 @@ distribution_desc
         If this property is set to 'true', all replicas of the tablet will only have one replica performing compaction, while the others fetch rowsets from that replica.
 
         `"enable_single_replica_compaction" = "false"`
+
+    * `enable_duplicate_without_keys_by_default`
+
+        When `true`, if Unique, Aggregate, or Duplicate is not specified when creating a table, a Duplicate model table without sorting columns and prefix indexes will be created by default.
+
+        `"enable_duplicate_without_keys_by_default" = "false"`
     
     * Dynamic partition related
     
@@ -698,6 +708,34 @@ NOTE: Multi Partition can be mixed with conventional manual creation of partitio
 
 </version>
 
+<version since="2.0">
+
+14. Add a duplicate without sorting column table
+
+```sql
+    CREATE TABLE example_db.table_hash
+    (
+        k1 DATE,
+        k2 DECIMAL(10, 2) DEFAULT "10.5",
+        k3 CHAR(10) COMMENT "string column",
+        k4 INT NOT NULL DEFAULT "1" COMMENT "int column"
+    )
+    COMMENT "duplicate without keys"
+    PARTITION BY RANGE(k1)
+    (
+        PARTITION p1 VALUES LESS THAN ("2020-02-01"),
+        PARTITION p2 VALUES LESS THAN ("2020-03-01"),
+        PARTITION p3 VALUES LESS THAN ("2020-04-01")
+    )
+    DISTRIBUTED BY HASH(k1) BUCKETS 32
+    PROPERTIES (
+        "replication_num" = "1",
+        "enable_duplicate_without_keys_by_default" = "true"
+    );
+```
+
+</version>
+
 ### Keywords
 
     CREATE, TABLE
diff --git a/docs/zh-CN/docs/data-table/data-model.md b/docs/zh-CN/docs/data-table/data-model.md
index ac6b7872ba..90a7ec4e52 100644
--- a/docs/zh-CN/docs/data-table/data-model.md
+++ b/docs/zh-CN/docs/data-table/data-model.md
@@ -438,6 +438,53 @@ PROPERTIES (
 
 这种数据模型适用于既没有聚合需求,又没有主键唯一性约束的原始数据的存储。更多使用场景,可参阅**聚合模型的局限性**小节。
 
+<version since="2.0">
+
+### 无排序列 Duplicate 模型
+
+当创建表的时候没有指定Unique、Aggregate或Duplicate时,会默认创建一个Duplicate模型的表,并自动指定排序列。
+
+当用户并没有排序需求的时候,可以通过在表属性中配置:
+
+```
+"enable_duplicate_without_keys_by_default" = "true"
+```
+
+然后再创建默认模型的时候,就会不再指定排序列,也不会给该表创建前缀索引,以此减少在导入和存储上额外的开销。
+
+建表语句如下:
+
+```sql
+CREATE TABLE IF NOT EXISTS example_db.example_tbl
+(
+    `timestamp` DATETIME NOT NULL COMMENT "日志时间",
+    `type` INT NOT NULL COMMENT "日志类型",
+    `error_code` INT COMMENT "错误码",
+    `error_msg` VARCHAR(1024) COMMENT "错误详细信息",
+    `op_id` BIGINT COMMENT "负责人id",
+    `op_time` DATETIME COMMENT "处理时间"
+)
+DISTRIBUTED BY HASH(`type`) BUCKETS 1
+PROPERTIES (
+"replication_allocation" = "tag.location.default: 1",
+"enable_duplicate_without_keys_by_default" = "true"
+);
+
+MySQL > desc example_tbl;
++------------+---------------+------+-------+---------+-------+
+| Field      | Type          | Null | Key   | Default | Extra |
++------------+---------------+------+-------+---------+-------+
+| timestamp  | DATETIME      | No   | false | NULL    | NONE  |
+| type       | INT           | No   | false | NULL    | NONE  |
+| error_code | INT           | Yes  | false | NULL    | NONE  |
+| error_msg  | VARCHAR(1024) | Yes  | false | NULL    | NONE  |
+| op_id      | BIGINT        | Yes  | false | NULL    | NONE  |
+| op_time    | DATETIME      | Yes  | false | NULL    | NONE  |
++------------+---------------+------+-------+---------+-------+
+6 rows in set (0.01 sec)
+```
+</version>
+
 ## 聚合模型的局限性
 
 这里我们针对 Aggregate 模型,来介绍下聚合模型的局限性。
diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE.md
index 1c57ed8143..2f76198cc5 100644
--- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE.md
+++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE.md
@@ -189,6 +189,10 @@ distribution_desc
     * AGGREGATE KEY:其后指定的列为维度列。
     * UNIQUE KEY:其后指定的列为主键列。
 
+    <version since="2.0">
+    注:当表属性`enable_duplicate_without_keys_by_default = true`时, 默认创建没有排序列的DUPLICATE表。
+    </version>
+
     示例:
     
     ```
@@ -376,6 +380,12 @@ distribution_desc
 
         `"enable_single_replica_compaction" = "false"`
 
+    * `enable_duplicate_without_keys_by_default`
+
+        当配置为`true`时,如果创建表的时候没有指定Unique、Aggregate或Duplicate时,会默认创建一个没有排序列和前缀索引的Duplicate模型的表。
+
+        `"enable_duplicate_without_keys_by_default" = "false"`
+
     * 动态分区相关
     
         动态分区相关参数如下:
@@ -695,6 +705,34 @@ distribution_desc
 
 </version>
 
+<version since="2.0">
+
+14. 批量无排序列Duplicate表
+
+```sql
+    CREATE TABLE example_db.table_hash
+    (
+        k1 DATE,
+        k2 DECIMAL(10, 2) DEFAULT "10.5",
+        k3 CHAR(10) COMMENT "string column",
+        k4 INT NOT NULL DEFAULT "1" COMMENT "int column"
+    )
+    COMMENT "duplicate without keys"
+    PARTITION BY RANGE(k1)
+    (
+        PARTITION p1 VALUES LESS THAN ("2020-02-01"),
+        PARTITION p2 VALUES LESS THAN ("2020-03-01"),
+        PARTITION p3 VALUES LESS THAN ("2020-04-01")
+    )
+    DISTRIBUTED BY HASH(k1) BUCKETS 32
+    PROPERTIES (
+        "replication_num" = "1",
+        "enable_duplicate_without_keys_by_default" = "true"
+    );
+```
+
+</version>
+
 ### Keywords
 
     CREATE, TABLE
diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
index 919218a249..d88fcfc8a8 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
@@ -1920,13 +1920,6 @@ public class Config extends ConfigBase {
     @ConfField
     public static boolean enable_stats = true;
 
-    /**
-     * Whether create a duplicate table without keys by default
-     * when creating a table which not set key type and key columns
-     */
-    @ConfField(mutable = true, masterOnly = true)
-    public static boolean experimental_enable_duplicate_without_keys_by_default = false;
-
     /**
      * To prevent different types (V1, V2, V3) of behavioral inconsistencies,
      * we may delete the DecimalV2 and DateV1 types in the future.
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
index 75a2ac0d7f..d9c2e5469e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
@@ -345,7 +345,12 @@ public class MaterializedViewHandler extends AlterHandler {
         // get rollup schema hash
         int mvSchemaHash = Util.generateSchemaHash();
         // get short key column count
-        short mvShortKeyColumnCount = Env.calcShortKeyColumnCount(mvColumns, properties, true/*isKeysRequired*/);
+        boolean isKeysRequired = !(mvKeysType == KeysType.DUP_KEYS);
+        short mvShortKeyColumnCount = Env.calcShortKeyColumnCount(mvColumns, properties, isKeysRequired);
+        if (mvShortKeyColumnCount <= 0 && olapTable.isDuplicateWithoutKey()) {
+            throw new DdlException("Not support create duplicate materialized view without order "
+                            + "by based on a duplicate table without keys");
+        }
         // get timeout
         long timeoutMs = PropertyAnalyzer.analyzeTimeout(properties, Config.alter_table_timeout_second) * 1000;
 
@@ -551,7 +556,7 @@ public class MaterializedViewHandler extends AlterHandler {
                         break;
                     }
                 }
-                if (allKeysMatch) {
+                if (allKeysMatch && !olapTable.isDuplicateWithoutKey()) {
                     throw new DdlException("MV contain the columns of the base table in prefix order for "
                             + "duplicate table is useless.");
                 }
@@ -1196,6 +1201,9 @@ public class MaterializedViewHandler extends AlterHandler {
     @Override
     public void process(List<AlterClause> alterClauses, String clusterName, Database db, OlapTable olapTable)
             throws DdlException, AnalysisException, MetaNotFoundException {
+        if (olapTable.isDuplicateWithoutKey()) {
+            throw new DdlException("Duplicate table without keys do not support alter rollup!");
+        }
         Optional<AlterClause> alterClauseOptional = alterClauses.stream().findAny();
         if (alterClauseOptional.isPresent()) {
             if (alterClauseOptional.get() instanceof AddRollupClause) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
index f16f45c968..0a06205963 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
@@ -923,6 +923,8 @@ public class SchemaChangeHandler extends AlterHandler {
                     throw new DdlException("Please add non-key column on base table directly");
                 }
                 newColumn.setAggregationType(AggregateType.NONE, true);
+            } else if (olapTable.isDuplicateWithoutKey()) {
+                throw new DdlException("Duplicate table without keys do not support add key column!");
             }
         }
 
@@ -1373,7 +1375,7 @@ public class SchemaChangeHandler extends AlterHandler {
                     hasKey = true;
                 }
             }
-            if (!hasKey) {
+            if (!hasKey && !olapTable.isDuplicateWithoutKey()) {
                 throw new DdlException("No key column left. index[" + olapTable.getIndexNameById(alterIndexId) + "]");
             }
 
@@ -1449,7 +1451,7 @@ public class SchemaChangeHandler extends AlterHandler {
 
             // 5. calc short key
             short newShortKeyColumnCount = Env.calcShortKeyColumnCount(alterSchema,
-                    indexIdToProperties.get(alterIndexId), true/*isKeysRequired*/);
+                    indexIdToProperties.get(alterIndexId), !olapTable.isDuplicateWithoutKey());
             LOG.debug("alter index[{}] short key column count: {}", alterIndexId, newShortKeyColumnCount);
             indexIdToShortKeyColumnCount.put(alterIndexId, newShortKeyColumnCount);
 
@@ -2607,7 +2609,7 @@ public class SchemaChangeHandler extends AlterHandler {
                     hasKey = true;
                 }
             }
-            if (!hasKey) {
+            if (!hasKey && !olapTable.isDuplicateWithoutKey()) {
                 throw new DdlException("No key column left. index[" + olapTable.getIndexNameById(alterIndexId) + "]");
             }
 
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 18db7b5847..2eae045e35 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
@@ -316,6 +316,11 @@ public class CreateTableStmt extends DdlStmt {
 
         analyzeEngineName();
 
+        boolean enableDuplicateWithoutKeysByDefault = false;
+        if (properties != null) {
+            enableDuplicateWithoutKeysByDefault =
+                                            PropertyAnalyzer.analyzeEnableDuplicateWithoutKeysByDefault(properties);
+        }
         //pre-block creation with column type ALL
         for (ColumnDef columnDef : columnDefs) {
             if (Objects.equals(columnDef.getType(), Type.ALL)) {
@@ -354,7 +359,7 @@ public class CreateTableStmt extends DdlStmt {
                     }
                     keysDesc = new KeysDesc(KeysType.AGG_KEYS, keysColumnNames);
                 } else {
-                    if (!Config.experimental_enable_duplicate_without_keys_by_default) {
+                    if (!enableDuplicateWithoutKeysByDefault) {
                         for (ColumnDef columnDef : columnDefs) {
                             keyLength += columnDef.getType().getIndexSize();
                             if (keysColumnNames.size() >= FeConstants.shortkey_max_column_count
@@ -393,6 +398,11 @@ public class CreateTableStmt extends DdlStmt {
                     }
                     keysDesc = new KeysDesc(KeysType.DUP_KEYS, keysColumnNames);
                 }
+            } else {
+                if (enableDuplicateWithoutKeysByDefault) {
+                    throw new AnalysisException("table property 'enable_duplicate_without_keys_by_default' only can"
+                                    + " set 'true' when create olap table by default.");
+                }
             }
 
             if (properties != null && properties.containsKey(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE)
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java
index abf873cea5..8bcd6a7d8f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java
@@ -127,6 +127,8 @@ public class ModifyTablePropertiesClause extends AlterTableClause {
             setStoragePolicy(storagePolicy);
         } else if (properties.containsKey(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE)) {
             throw new AnalysisException("Can not change UNIQUE KEY to Merge-On-Write mode");
+        } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_ENABLE_DUPLICATE_WITHOUT_KEYS_BY_DEFAULT)) {
+            throw new AnalysisException("Can not change enable_duplicate_without_keys_by_default");
         } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_ENABLE_LIGHT_SCHEMA_CHANGE)) {
             // do nothing, will be alter in SchemaChangeHandler.updateTableProperties
         } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_CCR_ENABLE)) {
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 11cd3f7612..a4276464d9 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
@@ -3092,6 +3092,14 @@ public class Env {
             sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_ENABLE_SINGLE_REPLICA_COMPACTION).append("\" = \"");
             sb.append(olapTable.enableSingleReplicaCompaction()).append("\"");
 
+            // enable duplicate without keys by default
+            if (olapTable.isDuplicateWithoutKey()) {
+                sb.append(",\n\"")
+                        .append(PropertyAnalyzer.PROPERTIES_ENABLE_DUPLICATE_WITHOUT_KEYS_BY_DEFAULT)
+                        .append("\" = \"");
+                sb.append(olapTable.isDuplicateWithoutKey()).append("\"");
+            }
+
             sb.append("\n)");
         } else if (table.getType() == TableType.MYSQL) {
             MysqlTable mysqlTable = (MysqlTable) table;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
index 47f15aa0bb..4edd8cfa41 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
@@ -136,6 +136,9 @@ public class PropertyAnalyzer {
     public static final String PROPERTIES_BINLOG_MAX_BYTES = "binlog.max_bytes";
     public static final String PROPERTIES_BINLOG_MAX_HISTORY_NUMS = "binlog.max_history_nums";
 
+    public static final String PROPERTIES_ENABLE_DUPLICATE_WITHOUT_KEYS_BY_DEFAULT =
+                                                                        "enable_duplicate_without_keys_by_default";
+
     private static final Logger LOG = LogManager.getLogger(PropertyAnalyzer.class);
     private static final String COMMA_SEPARATOR = ",";
     private static final double MAX_FPP = 0.05;
@@ -527,6 +530,25 @@ public class PropertyAnalyzer {
                 + " must be `true` or `false`");
     }
 
+    public static Boolean analyzeEnableDuplicateWithoutKeysByDefault(Map<String, String> properties)
+                            throws AnalysisException {
+        if (properties == null || properties.isEmpty()) {
+            return false;
+        }
+        String value = properties.get(PROPERTIES_ENABLE_DUPLICATE_WITHOUT_KEYS_BY_DEFAULT);
+        if (null == value) {
+            return false;
+        }
+        properties.remove(PROPERTIES_ENABLE_DUPLICATE_WITHOUT_KEYS_BY_DEFAULT);
+        if (value.equalsIgnoreCase("true")) {
+            return true;
+        } else if (value.equalsIgnoreCase("false")) {
+            return false;
+        }
+        throw new AnalysisException(PROPERTIES_ENABLE_DUPLICATE_WITHOUT_KEYS_BY_DEFAULT
+                + " must be `true` or `false`");
+    }
+
     public static Boolean analyzeStoreRowColumn(Map<String, String> properties) throws AnalysisException {
         if (properties == null || properties.isEmpty()) {
             return false;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
index 1355776a9d..9199c5fedf 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
@@ -1999,7 +1999,7 @@ public class InternalCatalog implements CatalogIf<Database> {
         // set in memory
         boolean isInMemory = PropertyAnalyzer.analyzeBooleanProp(properties, PropertyAnalyzer.PROPERTIES_INMEMORY,
                 false);
-        if (isInMemory == true) {
+        if (isInMemory) {
             throw new AnalysisException("Not support set 'in_memory'='true' now!");
         }
         olapTable.setIsInMemory(false);
@@ -2118,6 +2118,9 @@ public class InternalCatalog implements CatalogIf<Database> {
                 baseIndexStorageType, keysType, olapTable.getIndexes());
 
         for (AlterClause alterClause : stmt.getRollupAlterClauseList()) {
+            if (olapTable.isDuplicateWithoutKey()) {
+                throw new DdlException("Duplicate table without keys do not support add rollup!");
+            }
             AddRollupClause addRollupClause = (AddRollupClause) alterClause;
 
             Long baseRollupIndex = olapTable.getIndexIdByName(tableName);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java
index 3a144d1d08..5b92c4b616 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java
@@ -26,6 +26,7 @@ import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Table;
+import org.apache.doris.common.DdlException;
 import org.apache.doris.common.ExceptionChecker;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.qe.ConnectContext;
@@ -198,4 +199,39 @@ public class AlterJobV2Test {
         Map<Long, AlterJobV2> alterJobs = Env.getCurrentEnv().getMaterializedViewHandler().getAlterJobsV2();
         waitAlterJobDone(alterJobs);
     }
+
+    @Test
+    public void testDupTableWithoutKeysSchemaChange() throws Exception {
+
+        createTable("CREATE TABLE test.dup_table_without_keys (\n"
+                + "  k1 bigint(20) NULL ,\n"
+                + "  k2 bigint(20) NULL ,\n"
+                + "  k3 bigint(20) NULL,\n"
+                + "  v1 bigint(20) NULL ,\n"
+                + "  v2 varchar(1) NULL,\n"
+                + "  v3 varchar(1) NULL \n"
+                + ") ENGINE=OLAP\n"
+                + "PARTITION BY RANGE(k1, v1)\n"
+                + "(PARTITION p1 VALUES LESS THAN (\"10\", \"10\"))\n"
+                + "DISTRIBUTED BY HASH(v1,k2) BUCKETS 10\n"
+                + "PROPERTIES (\n"
+                + "\"replication_num\" = \"1\",\n"
+                + "\"enable_duplicate_without_keys_by_default\" = \"true\""
+                + ");");
+
+        ExceptionChecker.expectThrowsWithMsg(DdlException.class, "Duplicate table without keys do not support alter rollup!",
+                                () -> alterTable("alter table test.dup_table_without_keys add rollup r1(v1,v2,k2,k1);"));
+        ExceptionChecker.expectThrowsNoException(() -> alterTable("alter table test.dup_table_without_keys modify column v2 varchar(2);"));
+        ExceptionChecker.expectThrowsNoException(() -> alterTable("alter table test.dup_table_without_keys add column v4 varchar(2);"));
+        ExceptionChecker.expectThrowsWithMsg(DdlException.class, "Duplicate table without keys do not support add key column!",
+                                () -> alterTable("alter table test.dup_table_without_keys add column new_col INT KEY DEFAULT \"0\" AFTER k3;"));
+
+        createMaterializedView("create materialized view k1_k33 as select k2, k1 from test.dup_table_without_keys;");
+        Map<Long, AlterJobV2> alterJobs = Env.getCurrentEnv().getMaterializedViewHandler().getAlterJobsV2();
+        waitAlterJobDone(alterJobs);
+
+        createMaterializedView("create materialized view k1_k24 as select k2, k1 from test.dup_table_without_keys order by k2,k1;");
+        alterJobs = Env.getCurrentEnv().getMaterializedViewHandler().getAlterJobsV2();
+        waitAlterJobDone(alterJobs);
+    }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableStmtTest.java
index a3f13a97ba..d65a85e0ab 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableStmtTest.java
@@ -178,6 +178,38 @@ public class CreateTableStmtTest {
         cols.remove(col4);
     }
 
+    @Test
+    public void testCreateTableDuplicateWithoutKeys() throws UserException {
+        // setup
+        Map<String, String> properties = new HashMap<>();
+        properties.put(PropertyAnalyzer.PROPERTIES_ENABLE_DUPLICATE_WITHOUT_KEYS_BY_DEFAULT, "true");
+        ColumnDef col3 = new ColumnDef("col3", new TypeDef(ScalarType.createType(PrimitiveType.BIGINT)));
+        col3.setIsKey(false);
+        cols.add(col3);
+        ColumnDef col4 = new ColumnDef("col4", new TypeDef(ScalarType.createType(PrimitiveType.STRING)));
+        col4.setIsKey(false);
+        cols.add(col4);
+        // test merge-on-write
+        CreateTableStmt stmt1 = new CreateTableStmt(false, false, tblName, cols, "olap",
+                new KeysDesc(KeysType.DUP_KEYS, colsName), null,
+                new HashDistributionDesc(10, Lists.newArrayList("col3")), properties, null, "");
+        expectedEx.expect(AnalysisException.class);
+        expectedEx.expectMessage("table property 'enable_duplicate_without_keys_by_default' only can "
+                                                + "set 'true' when create olap table by default.");
+        stmt1.analyze(analyzer);
+
+        CreateTableStmt stmt2 = new CreateTableStmt(false, false, tblName, cols, "olap",
+                null, null,
+                new HashDistributionDesc(10, Lists.newArrayList("col3")), properties, null, "");
+        stmt2.analyze(analyzer);
+
+        Assert.assertEquals(col3.getAggregateType(), AggregateType.NONE);
+        Assert.assertEquals(col4.getAggregateType(), AggregateType.NONE);
+        // clear
+        cols.remove(col3);
+        cols.remove(col4);
+    }
+
     @Test
     public void testCreateTableWithRollup() throws UserException {
         List<AlterClause> ops = Lists.newArrayList();
diff --git a/regression-test/data/data_model_p0/duplicate/test_duplicate_table_without_keys.out b/regression-test/data/data_model_p0/duplicate/test_duplicate_table_without_keys.out
new file mode 100644
index 0000000000..2210dd8df9
--- /dev/null
+++ b/regression-test/data/data_model_p0/duplicate/test_duplicate_table_without_keys.out
@@ -0,0 +1,54 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select_dup_table --
+0	1	2	3
+0	1	2	4
+0	1	2	5
+
+-- !desc_dup_table --
+k1	INT	Yes	true	\N	
+k2	INT	Yes	true	\N	
+k3	INT	Yes	true	\N	
+int_value	INT	Yes	false	\N	NONE
+
+-- !show_dup_table --
+test_default_data_model	CREATE TABLE `test_default_data_model` (\n  `k1` int(11) NULL,\n  `k2` int(11) NULL,\n  `k3` int(11) NULL,\n  `int_value` int(11) NULL\n) ENGINE=OLAP\nDUPLICATE KEY(`k1`, `k2`, `k3`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`k1`) BUCKETS 5\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"storage_format" = "V2",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"enable_single_replica_compaction" = "false"\n);
+
+-- !select_dup_table --
+0	1	2	3
+0	1	2	4
+0	1	2	5
+
+-- !desc_dup_table --
+k1	INT	Yes	false	\N	NONE
+k2	INT	Yes	false	\N	NONE
+k3	INT	Yes	false	\N	NONE
+int_value	INT	Yes	false	\N	NONE
+
+-- !show_dup_table --
+test_default_data_model_no_keys	CREATE TABLE `test_default_data_model_no_keys` (\n  `k1` int(11) NULL,\n  `k2` int(11) NULL,\n  `k3` int(11) NULL,\n  `int_value` int(11) NULL\n) ENGINE=OLAP\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`k1`) BUCKETS 5\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"storage_format" = "V2",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"enable_single_replica_compaction" = "false",\n"enable_duplicate_without_keys_by_d [...]
+
+-- !select_dup_table --
+0	1	2	0	3
+0	1	2	0	4
+0	1	2	0	5
+
+-- !desc_dup_table --
+k1	INT	Yes	false	\N	NONE
+k2	INT	Yes	false	\N	NONE
+k3	INT	Yes	false	\N	NONE
+new_col1	INT	Yes	false	0	NONE
+int_value	INT	Yes	false	\N	NONE
+
+-- !show_dup_table --
+test_default_data_model_no_keys	CREATE TABLE `test_default_data_model_no_keys` (\n  `k1` int(11) NULL,\n  `k2` int(11) NULL,\n  `k3` int(11) NULL,\n  `new_col1` int(11) NULL DEFAULT "0",\n  `int_value` int(11) NULL\n) ENGINE=OLAP\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`k1`) BUCKETS 5\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"storage_format" = "V2",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"enable_single_replica_compaction" = "fals [...]
+
+-- !desc_dup_table --
+k1	INT	Yes	false	\N	NONE
+k2	INT	Yes	false	\N	NONE
+k3	INT	Yes	false	\N	NONE
+new_col1	INT	Yes	false	0	NONE
+int_value	INT	Yes	false	\N	NONE
+
+-- !show_dup_table --
+test_default_data_model_no_keys_like	CREATE TABLE `test_default_data_model_no_keys_like` (\n  `k1` int(11) NULL,\n  `k2` int(11) NULL,\n  `k3` int(11) NULL,\n  `new_col1` int(11) NULL DEFAULT "0",\n  `int_value` int(11) NULL\n) ENGINE=OLAP\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`k1`) BUCKETS 5\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"storage_format" = "V2",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"enable_single_replica_compactio [...]
+
diff --git a/regression-test/suites/data_model_p0/duplicate/test_duplicate_table_without_keys.groovy b/regression-test/suites/data_model_p0/duplicate/test_duplicate_table_without_keys.groovy
new file mode 100644
index 0000000000..0d8c2b3f4e
--- /dev/null
+++ b/regression-test/suites/data_model_p0/duplicate/test_duplicate_table_without_keys.groovy
@@ -0,0 +1,78 @@
+// 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_duplicate_table_without_keys") {
+    def dbName = "test_duplicate_table_without_keys"
+    List<List<Object>> db = sql "show databases like '${dbName}'"
+    if (db.size() == 0) {
+        sql "CREATE DATABASE  ${dbName}"
+    }
+    sql "use ${dbName}"
+
+    def tbName1 = "test_default_data_model"
+    sql "DROP TABLE IF EXISTS ${tbName1}"
+    sql """
+            CREATE TABLE IF NOT EXISTS ${tbName1} (
+                k1 int,
+                k2 int, 
+                k3 int,
+                int_value int
+            )
+            DISTRIBUTED BY HASH(k1) BUCKETS 5 
+            properties("replication_num" = "1","enable_duplicate_without_keys_by_default" = "false");
+        """
+    sql "insert into ${tbName1} values(0, 1, 2, 4)"
+    sql "insert into ${tbName1} values(0, 1, 2, 5)"
+    sql "insert into ${tbName1} values(0, 1, 2, 3)"
+    order_qt_select_dup_table "select * from ${tbName1}"
+    qt_desc_dup_table "desc ${tbName1}"
+    qt_show_dup_table "show create table ${tbName1}"
+    sql "DROP TABLE ${tbName1}"
+    
+    def tbName2 = "test_default_data_model_no_keys"
+    sql "DROP TABLE IF EXISTS ${tbName2}"
+    sql """
+            CREATE TABLE IF NOT EXISTS ${tbName2} (
+                k1 int,
+                k2 int, 
+                k3 int,
+                int_value int
+            )
+            DISTRIBUTED BY HASH(k1) BUCKETS 5 
+            properties("replication_num" = "1","enable_duplicate_without_keys_by_default" = "true");
+        """
+    sql "insert into ${tbName2} values(0, 1, 2, 4)"
+    sql "insert into ${tbName2} values(0, 1, 2, 5)"
+    sql "insert into ${tbName2} values(0, 1, 2, 3)"
+    order_qt_select_dup_table "select * from ${tbName2}"
+    qt_desc_dup_table "desc ${tbName2}"
+    qt_show_dup_table "show create table ${tbName2}"
+
+    sql "ALTER TABLE ${tbName2} ADD COLUMN new_col1 INT DEFAULT \"0\" AFTER k3"
+    order_qt_select_dup_table "select * from ${tbName2}"
+    qt_desc_dup_table "desc ${tbName2}"
+    qt_show_dup_table "show create table ${tbName2}"
+
+    def tbName3 = "test_default_data_model_no_keys_like"
+    sql "DROP TABLE IF EXISTS ${tbName3}"
+    sql """create table ${tbName3} like ${tbName2}"""
+    qt_desc_dup_table "desc ${tbName3}"
+    qt_show_dup_table "show create table ${tbName3}"    
+
+    sql "DROP TABLE ${tbName2}"
+    sql "DROP TABLE ${tbName3}"
+}


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