You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by kx...@apache.org on 2023/06/21 10:36:52 UTC

[doris] branch master updated: Revert "[feature](merge-on-write) enable merge on write by default (#… (#21041)

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

kxiao 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 cc53391c9a Revert "[feature](merge-on-write) enable merge on write by default (#… (#21041)
cc53391c9a is described below

commit cc53391c9a7bb30449927a436805b9dece2310e0
Author: zhannngchen <48...@users.noreply.github.com>
AuthorDate: Wed Jun 21 18:36:46 2023 +0800

    Revert "[feature](merge-on-write) enable merge on write by default (#… (#21041)
---
 .../doris/alter/MaterializedViewHandler.java       |   8 +-
 .../org/apache/doris/analysis/CreateTableStmt.java |   4 +-
 .../org/apache/doris/analysis/PartitionDesc.java   |  20 +++--
 .../main/java/org/apache/doris/catalog/Env.java    |   2 +-
 .../apache/doris/common/util/PropertyAnalyzer.java |   8 +-
 .../apache/doris/analysis/CreateTableStmtTest.java |   2 +-
 .../doris/analysis/ShowCreateTableStmtTest.java    |  25 ------
 .../org/apache/doris/catalog/CreateTableTest.java  |   4 +-
 .../nereids/trees/plans/DeleteCommandTest.java     |   3 +-
 .../unique/test_unique_mow_table_debug_data.out    |  36 --------
 .../test_unique_mow_table_debug_data_delete.csv    |   1 -
 .../data_model_p0/unique/test_unique_table.out     |   6 +-
 .../unique/test_unique_table_like.out              |  24 +++---
 .../unique/test_unique_table_new_sequence.out      |  14 +--
 .../data/index_p0/test_bitmap_index.out            |  10 +--
 .../data/inverted_index_p0/test_bitmap_index.out   |   2 +-
 .../data/schema_change_p0/test_rename_column.out   | Bin 4043 -> 4097 bytes
 .../data/schema_change_p0/test_schema_change.out   |   6 +-
 .../test_uniq_delete_sign_schema_change.out        |  20 ++---
 .../unsigned/test_unsigned_int_compatibility.out   |   6 +-
 regression-test/data/update/test_update_unique.out |   6 +-
 .../alter_table_add_policy.groovy                  |   3 +-
 .../create_table_use_policy.groovy                 |   6 +-
 .../suites/cold_heat_separation/policy/drop.groovy |   3 +-
 .../unique/test_unique_mow_table_debug_data.groovy |  96 ---------------------
 .../unique/test_unique_table_debug_data.groovy     |   3 +-
 .../suites/nereids_p0/delete/load.groovy           |   3 +-
 .../unique_with_mow_p0/test_create_table.groovy    |  10 +--
 .../suites/update/test_update_mow.groovy           |   5 +-
 29 files changed, 91 insertions(+), 245 deletions(-)

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 baf0fd60f3..10dddcd94d 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
@@ -564,14 +564,10 @@ public class MaterializedViewHandler extends AlterHandler {
             }
         }
         if (KeysType.UNIQUE_KEYS == olapTable.getKeysType() && olapTable.hasDeleteSign()) {
-            Column newColumn = new Column(olapTable.getDeleteSignColumn());
-            newColumn.setAggregationType(AggregateType.REPLACE, true);
-            newMVColumns.add(newColumn);
+            newMVColumns.add(new Column(olapTable.getDeleteSignColumn()));
         }
         if (KeysType.UNIQUE_KEYS == olapTable.getKeysType() && olapTable.hasSequenceCol()) {
-            Column newColumn = new Column(olapTable.getSequenceCol());
-            newColumn.setAggregationType(AggregateType.REPLACE, true);
-            newMVColumns.add(newColumn);
+            newMVColumns.add(new Column(olapTable.getSequenceCol()));
         }
         if (olapTable.storeRowColumn()) {
             Column newColumn = new Column(olapTable.getRowStoreCol());
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 5e511176e0..5ec43781eb 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
@@ -429,7 +429,7 @@ public class CreateTableStmt extends DdlStmt {
                 if (keysDesc.getKeysType() == KeysType.DUP_KEYS) {
                     type = AggregateType.NONE;
                 }
-                if (enableUniqueKeyMergeOnWrite) {
+                if (keysDesc.getKeysType() == KeysType.UNIQUE_KEYS && enableUniqueKeyMergeOnWrite) {
                     type = AggregateType.NONE;
                 }
                 for (int i = keysDesc.keysColumnSize(); i < columnDefs.size(); ++i) {
@@ -528,7 +528,7 @@ public class CreateTableStmt extends DdlStmt {
             if (partitionDesc != null) {
                 if (partitionDesc instanceof ListPartitionDesc || partitionDesc instanceof RangePartitionDesc
                         || partitionDesc instanceof ColumnPartitionDesc) {
-                    partitionDesc.analyze(columnDefs, properties, keysDesc);
+                    partitionDesc.analyze(columnDefs, properties);
                 } else {
                     throw new AnalysisException("Currently only support range"
                             + " and list partition with engine type olap");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionDesc.java
index 8ea13cc50f..11cb795fd6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionDesc.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionDesc.java
@@ -17,13 +17,14 @@
 
 package org.apache.doris.analysis;
 
+import org.apache.doris.catalog.AggregateType;
 import org.apache.doris.catalog.Column;
-import org.apache.doris.catalog.KeysType;
 import org.apache.doris.catalog.PartitionInfo;
 import org.apache.doris.catalog.PartitionType;
 import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
+import org.apache.doris.common.util.PropertyAnalyzer;
 import org.apache.doris.qe.ConnectContext;
 
 import com.google.common.collect.Lists;
@@ -84,12 +85,18 @@ public class PartitionDesc {
         return partitionColNames;
     }
 
-    public void analyze(List<ColumnDef> columnDefs, Map<String, String> otherProperties, KeysDesc keysDesc)
-            throws AnalysisException {
+    public void analyze(List<ColumnDef> columnDefs, Map<String, String> otherProperties) throws AnalysisException {
         if (partitionColNames == null || partitionColNames.isEmpty()) {
             throw new AnalysisException("No partition columns.");
         }
 
+        // `analyzeUniqueKeyMergeOnWrite` would modify `properties`, which will be used later,
+        // so we just clone a properties map here.
+        boolean enableUniqueKeyMergeOnWrite = false;
+        if (otherProperties != null) {
+            enableUniqueKeyMergeOnWrite =
+                PropertyAnalyzer.analyzeUniqueKeyMergeOnWrite(Maps.newHashMap(otherProperties));
+        }
         Set<String> partColNames = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
         for (String partitionCol : partitionColNames) {
             if (!partColNames.add(partitionCol)) {
@@ -99,10 +106,9 @@ public class PartitionDesc {
             boolean found = false;
             for (ColumnDef columnDef : columnDefs) {
                 if (columnDef.getName().equals(partitionCol)) {
-                    if (!columnDef.isKey() && keysDesc.getKeysType() != KeysType.DUP_KEYS) {
-                        throw new AnalysisException(
-                                "The partition column must be key column in " + keysDesc.getKeysType().toSql()
-                                        + " table");
+                    if (!columnDef.isKey() && (columnDef.getAggregateType() != AggregateType.NONE
+                            || enableUniqueKeyMergeOnWrite)) {
+                        throw new AnalysisException("The partition column could not be aggregated column");
                     }
                     if (columnDef.getType().isFloatingPointType()) {
                         throw new AnalysisException("Floating point type column can not be partition column");
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 b6ffd0985d..90e4d06118 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
@@ -3068,7 +3068,7 @@ public class Env {
             }
 
             // unique key table with merge on write
-            if (olapTable.getKeysType() == KeysType.UNIQUE_KEYS && !olapTable.getEnableUniqueKeyMergeOnWrite()) {
+            if (olapTable.getKeysType() == KeysType.UNIQUE_KEYS && olapTable.getEnableUniqueKeyMergeOnWrite()) {
                 sb.append(",\n\"").append(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE).append("\" = \"");
                 sb.append(olapTable.getEnableUniqueKeyMergeOnWrite()).append("\"");
             }
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 d18d866b43..beb6d3b893 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
@@ -900,10 +900,12 @@ public class PropertyAnalyzer {
 
     public static boolean analyzeUniqueKeyMergeOnWrite(Map<String, String> properties) throws AnalysisException {
         if (properties == null || properties.isEmpty()) {
-            // enable merge on write by default
-            return true;
+            return false;
+        }
+        String value = properties.get(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE);
+        if (value == null) {
+            return false;
         }
-        String value = properties.getOrDefault(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE, "true");
         properties.remove(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE);
         if (value.equals("true")) {
             return true;
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 d65a85e0ab..4086b207e6 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
@@ -129,7 +129,6 @@ public class CreateTableStmtTest {
     public void testCreateTableUniqueKeyNormal() throws UserException {
         // setup
         Map<String, String> properties = new HashMap<>();
-        properties.put(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE, "false");
         ColumnDef col3 = new ColumnDef("col3", new TypeDef(ScalarType.createType(PrimitiveType.BIGINT)));
         col3.setIsKey(false);
         cols.add(col3);
@@ -152,6 +151,7 @@ public class CreateTableStmtTest {
     public void testCreateTableUniqueKeyMoW() throws UserException {
         // setup
         Map<String, String> properties = new HashMap<>();
+        properties.put(PropertyAnalyzer.ENABLE_UNIQUE_KEY_MERGE_ON_WRITE, "true");
         ColumnDef col3 = new ColumnDef("col3", new TypeDef(ScalarType.createType(PrimitiveType.BIGINT)));
         col3.setIsKey(false);
         cols.add(col3);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateTableStmtTest.java
index 6bb8f0b170..fe9adf9ccd 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateTableStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateTableStmtTest.java
@@ -40,14 +40,6 @@ public class ShowCreateTableStmtTest extends TestWithFeService {
                 + ") "
                 + "distributed by hash(k1) buckets 1\n"
                 + "properties(\"replication_num\" = \"1\");");
-        createTable("create table table2\n"
-                + "(k1 int comment 'test column k1', k2 int comment 'test column k2', v1 string comment 'test column v1') "
-                + "unique key(k1, k2) distributed by hash(k1) buckets 1\n"
-                + "properties(\"replication_num\" = \"1\");");
-        createTable("create table table3\n"
-                + "(k1 int comment 'test column k1', k2 int comment 'test column k2', v1 string comment 'test column v1') "
-                + "unique key(k1, k2) distributed by hash(k1) buckets 1\n"
-                + "properties(\"replication_num\" = \"1\", \"enable_unique_key_merge_on_write\" = \"false\");");
     }
 
 
@@ -68,21 +60,4 @@ public class ShowCreateTableStmtTest extends TestWithFeService {
         Assertions.assertTrue(!showSql.contains("PARTITION BY"));
         Assertions.assertTrue(!showSql.contains("PARTITION `p01`"));
     }
-
-    @Test
-    public void testUniqueKeyMoW() throws Exception {
-        String propertyStr = "\"enable_unique_key_merge_on_write\" = \"false\"";
-
-        String sql1 = "show create table table2";
-        ShowResultSet showResultSet1 = showCreateTable(sql1);
-        String showSql1 = showResultSet1.getResultRows().get(0).get(1);
-        Assertions.assertTrue(showSql1.contains("`k1` int(11) NULL COMMENT 'test column k1'"));
-        Assertions.assertFalse(showSql1.contains(propertyStr));
-
-        String sql2 = "show create table table3";
-        ShowResultSet showResultSet2 = showCreateTable(sql2);
-        String showSql2 = showResultSet2.getResultRows().get(0).get(1);
-        Assertions.assertTrue(showSql2.contains("`k1` int(11) NULL COMMENT 'test column k1'"));
-        Assertions.assertTrue(showSql2.contains(propertyStr));
-    }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
index 35abb9b2e0..5037327005 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
@@ -233,10 +233,10 @@ public class CreateTableTest {
         Assert.assertTrue(tbl8.getColumn("k1").isKey());
         Assert.assertTrue(tbl8.getColumn("k2").isKey());
         Assert.assertFalse(tbl8.getColumn("v1").isKey());
-        Assert.assertTrue(tbl8.getColumn(Column.SEQUENCE_COL).getAggregationType() == AggregateType.NONE);
+        Assert.assertTrue(tbl8.getColumn(Column.SEQUENCE_COL).getAggregationType() == AggregateType.REPLACE);
 
         OlapTable tbl13 = (OlapTable) db.getTableOrDdlException("tbl13");
-        Assert.assertTrue(tbl13.getColumn(Column.SEQUENCE_COL).getAggregationType() == AggregateType.NONE);
+        Assert.assertTrue(tbl13.getColumn(Column.SEQUENCE_COL).getAggregationType() == AggregateType.REPLACE);
         Assert.assertTrue(tbl13.getColumn(Column.SEQUENCE_COL).getType() == Type.INT);
         Assert.assertEquals(tbl13.getSequenceMapCol(), "v1");
     }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/DeleteCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/DeleteCommandTest.java
index 12a7ba48ce..c0a563085e 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/DeleteCommandTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/DeleteCommandTest.java
@@ -42,7 +42,8 @@ public class DeleteCommandTest extends TestWithFeService implements PlanPatternM
                 + "unique key(k1, k2)\n"
                 + "distributed by hash(k1) buckets 4\n"
                 + "properties(\n"
-                + "    \"replication_num\"=\"1\"\n"
+                + "    \"replication_num\"=\"1\",\n"
+                + "    \"enable_unique_key_merge_on_write\" = \"true\" \n"
                 + ")");
         createTable("create table t2 (\n"
                 + "    k1 int,\n"
diff --git a/regression-test/data/data_model_p0/unique/test_unique_mow_table_debug_data.out b/regression-test/data/data_model_p0/unique/test_unique_mow_table_debug_data.out
deleted file mode 100644
index 097eeffaed..0000000000
--- a/regression-test/data/data_model_p0/unique/test_unique_mow_table_debug_data.out
+++ /dev/null
@@ -1,36 +0,0 @@
--- This file is automatically generated. You should know what you did if you want to edit this
--- !select_init --
-1	11
-2	11
-3	1
-
--- !select_skip_delete_bitmap --
-1	1	0	2
-1	11	0	3
-2	1	0	2
-2	11	0	3
-3	1	0	4
-
--- !select_batch_delete --
-2	11
-3	1
-
--- !select_sql_delete --
-3	1
-
--- !select_skip_delete1 --
-2	11
-3	1
-
--- !select_skip_delete_bitmap_after_batch_delete --
-1	1	0	2
-1	11	0	3
-3	1	0	4
-
--- !select_skip_delete2 --
-1	1	0	2
-1	11	0	3
-2	1	0	2
-2	11	0	3
-3	1	0	4
-
diff --git a/regression-test/data/data_model_p0/unique/test_unique_mow_table_debug_data_delete.csv b/regression-test/data/data_model_p0/unique/test_unique_mow_table_debug_data_delete.csv
deleted file mode 100644
index 1c19a02704..0000000000
--- a/regression-test/data/data_model_p0/unique/test_unique_mow_table_debug_data_delete.csv
+++ /dev/null
@@ -1 +0,0 @@
-1|111
diff --git a/regression-test/data/data_model_p0/unique/test_unique_table.out b/regression-test/data/data_model_p0/unique/test_unique_table.out
index 9745f9fc6b..ccc1edc356 100644
--- a/regression-test/data/data_model_p0/unique/test_unique_table.out
+++ b/regression-test/data/data_model_p0/unique/test_unique_table.out
@@ -4,7 +4,7 @@
 
 -- !desc_uniq_table --
 k	INT	Yes	true	\N	
-int_value	INT	Yes	false	\N	NONE
-char_value	CHAR(10)	Yes	false	\N	NONE
-date_value	DATE	Yes	false	\N	NONE
+int_value	INT	Yes	false	\N	REPLACE
+char_value	CHAR(10)	Yes	false	\N	REPLACE
+date_value	DATE	Yes	false	\N	REPLACE
 
diff --git a/regression-test/data/data_model_p0/unique/test_unique_table_like.out b/regression-test/data/data_model_p0/unique/test_unique_table_like.out
index f6e03d26df..dd68e453ff 100644
--- a/regression-test/data/data_model_p0/unique/test_unique_table_like.out
+++ b/regression-test/data/data_model_p0/unique/test_unique_table_like.out
@@ -1,19 +1,19 @@
 -- This file is automatically generated. You should know what you did if you want to edit this
 -- !desc_uniq_table --
 k	INT	Yes	true	\N	
-int_value	INT	Yes	false	\N	NONE
-char_value	CHAR(10)	Yes	false	\N	NONE
-date_value	DATE	Yes	false	\N	NONE
-__DORIS_DELETE_SIGN__	TINYINT	No	false	0	NONE
-__DORIS_VERSION_COL__	BIGINT	No	false	0	NONE
-__DORIS_SEQUENCE_COL__	INT	Yes	false	\N	NONE
+int_value	INT	Yes	false	\N	REPLACE
+char_value	CHAR(10)	Yes	false	\N	REPLACE
+date_value	DATE	Yes	false	\N	REPLACE
+__DORIS_DELETE_SIGN__	TINYINT	No	false	0	REPLACE
+__DORIS_VERSION_COL__	BIGINT	No	false	0	REPLACE
+__DORIS_SEQUENCE_COL__	INT	Yes	false	\N	REPLACE
 
 -- !desc_uniq_table --
 k	INT	Yes	true	\N	
-int_value	INT	Yes	false	\N	NONE
-char_value	CHAR(10)	Yes	false	\N	NONE
-date_value	DATE	Yes	false	\N	NONE
-__DORIS_DELETE_SIGN__	TINYINT	No	false	0	NONE
-__DORIS_VERSION_COL__	BIGINT	No	false	0	NONE
-__DORIS_SEQUENCE_COL__	INT	Yes	false	\N	NONE
+int_value	INT	Yes	false	\N	REPLACE
+char_value	CHAR(10)	Yes	false	\N	REPLACE
+date_value	DATE	Yes	false	\N	REPLACE
+__DORIS_DELETE_SIGN__	TINYINT	No	false	0	REPLACE
+__DORIS_VERSION_COL__	BIGINT	No	false	0	REPLACE
+__DORIS_SEQUENCE_COL__	INT	Yes	false	\N	REPLACE
 
diff --git a/regression-test/data/data_model_p0/unique/test_unique_table_new_sequence.out b/regression-test/data/data_model_p0/unique/test_unique_table_new_sequence.out
index 5a5dd5b2be..54d3b70218 100644
--- a/regression-test/data/data_model_p0/unique/test_unique_table_new_sequence.out
+++ b/regression-test/data/data_model_p0/unique/test_unique_table_new_sequence.out
@@ -47,11 +47,11 @@
 
 -- !desc --
 k1	INT	Yes	true	\N	
-v1	TINYINT	Yes	false	\N	NONE
-v2	INT	Yes	false	\N	NONE
-v3	INT	Yes	false	\N	NONE
-v4	INT	Yes	false	\N	NONE
-__DORIS_DELETE_SIGN__	TINYINT	No	false	0	NONE
-__DORIS_VERSION_COL__	BIGINT	No	false	0	NONE
-__DORIS_SEQUENCE_COL__	INT	Yes	false	\N	NONE
+v1	TINYINT	Yes	false	\N	REPLACE
+v2	INT	Yes	false	\N	REPLACE
+v3	INT	Yes	false	\N	REPLACE
+v4	INT	Yes	false	\N	REPLACE
+__DORIS_DELETE_SIGN__	TINYINT	No	false	0	REPLACE
+__DORIS_VERSION_COL__	BIGINT	No	false	0	REPLACE
+__DORIS_SEQUENCE_COL__	INT	Yes	false	\N	REPLACE
 
diff --git a/regression-test/data/index_p0/test_bitmap_index.out b/regression-test/data/index_p0/test_bitmap_index.out
index bdc8f2b997..37a1668805 100644
--- a/regression-test/data/index_p0/test_bitmap_index.out
+++ b/regression-test/data/index_p0/test_bitmap_index.out
@@ -86,11 +86,11 @@ k8	DATETIME	Yes	true	\N
 k9	LARGEINT	Yes	true	\N	
 k10	DECIMALV3(9, 0)	Yes	true	\N	
 k11	BOOLEAN	Yes	true	\N	
-k12	DATE	Yes	false	\N	NONE
-k13	DATETIME	Yes	false	\N	NONE
-k14	DATETIME	Yes	false	\N	NONE
-k15	DATETIME	Yes	false	\N	NONE
-v1	INT	Yes	false	\N	NONE
+k12	DATE	Yes	false	\N	REPLACE
+k13	DATETIME	Yes	false	\N	REPLACE
+k14	DATETIME	Yes	false	\N	REPLACE
+k15	DATETIME	Yes	false	\N	REPLACE
+v1	INT	Yes	false	\N	REPLACE
 
 -- !sql --
 default_cluster:regression_test_index_p0.test_bitmap_index_unique		index1		k1						BITMAP		
diff --git a/regression-test/data/inverted_index_p0/test_bitmap_index.out b/regression-test/data/inverted_index_p0/test_bitmap_index.out
index ed7e2d1553..e03cc9f29e 100644
--- a/regression-test/data/inverted_index_p0/test_bitmap_index.out
+++ b/regression-test/data/inverted_index_p0/test_bitmap_index.out
@@ -70,7 +70,7 @@ k8	DATETIME	Yes	true	\N
 k9	LARGEINT	Yes	true	\N	
 k10	DECIMALV3(9, 0)	Yes	true	\N	
 k11	BOOLEAN	Yes	true	\N	
-v1	INT	Yes	false	\N	NONE
+v1	INT	Yes	false	\N	REPLACE
 
 -- !sql --
 default_cluster:regression_test_inverted_index_p0.test_bitmap_index_unique		index1		k1						BITMAP		
diff --git a/regression-test/data/schema_change_p0/test_rename_column.out b/regression-test/data/schema_change_p0/test_rename_column.out
index bc1966690b..3e8d56d043 100644
Binary files a/regression-test/data/schema_change_p0/test_rename_column.out and b/regression-test/data/schema_change_p0/test_rename_column.out differ
diff --git a/regression-test/data/schema_change_p0/test_schema_change.out b/regression-test/data/schema_change_p0/test_schema_change.out
index 43857c0a39..84ee227ec5 100644
--- a/regression-test/data/schema_change_p0/test_schema_change.out
+++ b/regression-test/data/schema_change_p0/test_schema_change.out
@@ -2,9 +2,9 @@
 -- !desc_uniq_table --
 event_day	DATE	Yes	true	\N	
 siteid	INT	Yes	true	10	
-citycode	TEXT	Yes	false	\N	NONE
-username	VARCHAR(32)	Yes	false		NONE
-pv	BIGINT	Yes	false	0	NONE
+citycode	TEXT	Yes	false	\N	REPLACE
+username	VARCHAR(32)	Yes	false		REPLACE
+pv	BIGINT	Yes	false	0	REPLACE
 
 -- !sql --
 2021-11-01	1	1	用户A	3
diff --git a/regression-test/data/schema_change_p0/test_uniq_delete_sign_schema_change.out b/regression-test/data/schema_change_p0/test_uniq_delete_sign_schema_change.out
index 6cf1255622..bc07e62e29 100644
--- a/regression-test/data/schema_change_p0/test_uniq_delete_sign_schema_change.out
+++ b/regression-test/data/schema_change_p0/test_uniq_delete_sign_schema_change.out
@@ -16,12 +16,12 @@
 
 -- !sql --
 k1	INT	Yes	true	\N	
-value1	INT	Yes	false	\N	NONE
-value2	INT	Yes	false	\N	NONE
-value3	INT	Yes	false	\N	NONE
-value4	INT	Yes	false	\N	NONE
-__DORIS_DELETE_SIGN__	TINYINT	No	false	0	NONE
-__DORIS_VERSION_COL__	BIGINT	No	false	0	NONE
+value1	INT	Yes	false	\N	REPLACE
+value2	INT	Yes	false	\N	REPLACE
+value3	INT	Yes	false	\N	REPLACE
+value4	INT	Yes	false	\N	REPLACE
+__DORIS_DELETE_SIGN__	TINYINT	No	false	0	REPLACE
+__DORIS_VERSION_COL__	BIGINT	No	false	0	REPLACE
 
 -- !sql --
 1	1	1	1	\N	1	7
@@ -47,10 +47,10 @@ __DORIS_VERSION_COL__	BIGINT	No	false	0	NONE
 
 -- !sql --
 k1	INT	Yes	true	\N	
-value2	INT	Yes	false	\N	NONE
-value4	INT	Yes	false	\N	NONE
-__DORIS_DELETE_SIGN__	TINYINT	No	false	0	NONE
-__DORIS_VERSION_COL__	BIGINT	No	false	0	NONE
+value2	INT	Yes	false	\N	REPLACE
+value4	INT	Yes	false	\N	REPLACE
+__DORIS_DELETE_SIGN__	TINYINT	No	false	0	REPLACE
+__DORIS_VERSION_COL__	BIGINT	No	false	0	REPLACE
 
 -- !sql --
 1	1	\N	1	7
diff --git a/regression-test/data/types_p0/unsigned/test_unsigned_int_compatibility.out b/regression-test/data/types_p0/unsigned/test_unsigned_int_compatibility.out
index b76de33e63..0f8cd0f99b 100644
--- a/regression-test/data/types_p0/unsigned/test_unsigned_int_compatibility.out
+++ b/regression-test/data/types_p0/unsigned/test_unsigned_int_compatibility.out
@@ -2,7 +2,7 @@
 -- !desc_tb --
 user_id	LARGEINT	No	true	\N	
 city	VARCHAR(20)	Yes	true	\N	
-value1	BIGINT	Yes	false	\N	NONE
+value1	BIGINT	Yes	false	\N	REPLACE
 
 -- !select_tb --
 1	Beijing	21474836478
@@ -10,8 +10,8 @@ value1	BIGINT	Yes	false	\N	NONE
 -- !desc_tb --
 user_id	LARGEINT	No	true	\N	
 city	VARCHAR(20)	Yes	true	\N	
-value1	BIGINT	Yes	false	\N	NONE
-value2	BIGINT	Yes	false	\N	NONE
+value1	BIGINT	Yes	false	\N	REPLACE
+value2	BIGINT	Yes	false	\N	REPLACE
 
 -- !select_tb --
 1	Beijing	21474836478	\N
diff --git a/regression-test/data/update/test_update_unique.out b/regression-test/data/update/test_update_unique.out
index 615ee7b07d..051bd679ab 100644
--- a/regression-test/data/update/test_update_unique.out
+++ b/regression-test/data/update/test_update_unique.out
@@ -9,9 +9,9 @@
 
 -- !desc_uniq_table --
 k	INT	Yes	true	\N	
-value1	INT	Yes	false	\N	NONE
-value2	INT	Yes	false	\N	NONE
-date_value	DATE	Yes	false	\N	NONE
+value1	INT	Yes	false	\N	REPLACE
+value2	INT	Yes	false	\N	REPLACE
+date_value	DATE	Yes	false	\N	REPLACE
 
 -- !complex_update --
 1	10	1	1000.0	2000-01-01
diff --git a/regression-test/suites/cold_heat_separation/empty_table_use_policy/alter_table_add_policy.groovy b/regression-test/suites/cold_heat_separation/empty_table_use_policy/alter_table_add_policy.groovy
index 73fb57ecb8..c4b257953d 100644
--- a/regression-test/suites/cold_heat_separation/empty_table_use_policy/alter_table_add_policy.groovy
+++ b/regression-test/suites/cold_heat_separation/empty_table_use_policy/alter_table_add_policy.groovy
@@ -26,8 +26,7 @@ suite("add_table_policy_by_alter_table") {
         UNIQUE KEY(k1)
         DISTRIBUTED BY HASH (k1) BUCKETS 3
         PROPERTIES(
-            "replication_num" = "1",
-            "enable_unique_key_merge_on_write" = "false"
+            "replication_num" = "1"
         );
     """
     assertEquals(create_table_not_have_policy_result.size(), 1);
diff --git a/regression-test/suites/cold_heat_separation/empty_table_use_policy/create_table_use_policy.groovy b/regression-test/suites/cold_heat_separation/empty_table_use_policy/create_table_use_policy.groovy
index 77e9f5a186..d4e3aa1774 100644
--- a/regression-test/suites/cold_heat_separation/empty_table_use_policy/create_table_use_policy.groovy
+++ b/regression-test/suites/cold_heat_separation/empty_table_use_policy/create_table_use_policy.groovy
@@ -29,8 +29,7 @@ suite("create_table_use_policy") {
         DISTRIBUTED BY HASH (k1) BUCKETS 3
         PROPERTIES(
             "storage_policy" = "not_exist_policy",
-            "replication_num" = "1",
-            "enable_unique_key_merge_on_write" = "false"
+            "replication_num" = "1"
         );
     """
 
@@ -86,8 +85,7 @@ suite("create_table_use_policy") {
         DISTRIBUTED BY HASH (k1) BUCKETS 3
         PROPERTIES(
             "storage_policy" = "test_create_table_use_policy",
-            "replication_num" = "1",
-            "enable_unique_key_merge_on_write" = "false"
+            "replication_num" = "1"
         );
     """
 
diff --git a/regression-test/suites/cold_heat_separation/policy/drop.groovy b/regression-test/suites/cold_heat_separation/policy/drop.groovy
index 9d5a46e4a4..8157df8f11 100644
--- a/regression-test/suites/cold_heat_separation/policy/drop.groovy
+++ b/regression-test/suites/cold_heat_separation/policy/drop.groovy
@@ -122,8 +122,7 @@ suite("drop_policy") {
             DISTRIBUTED BY HASH (k1) BUCKETS 3
             PROPERTIES(
                 "storage_policy" = "drop_policy_test_has_table_binded",
-                "replication_num" = "1",
-                "enable_unique_key_merge_on_write" = "false"
+                "replication_num" = "1"
             );
         """
 
diff --git a/regression-test/suites/data_model_p0/unique/test_unique_mow_table_debug_data.groovy b/regression-test/suites/data_model_p0/unique/test_unique_mow_table_debug_data.groovy
deleted file mode 100644
index 715e2e714f..0000000000
--- a/regression-test/suites/data_model_p0/unique/test_unique_mow_table_debug_data.groovy
+++ /dev/null
@@ -1,96 +0,0 @@
-// 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.
-
-import org.codehaus.groovy.runtime.IOGroovyMethods
-
-suite("test_unique_mow_table_debug_data") {
-
-    sql "ADMIN SET FRONTEND CONFIG ('enable_batch_delete_by_default' = 'true')"
-    sql "SET show_hidden_columns=false"
-    sql "SET skip_delete_predicate=false"
-    sql "SET skip_delete_bitmap=false"
-
-    def tbName = "test_unique_mow_table_debug_data"
-    sql "DROP TABLE IF EXISTS ${tbName}"
-    sql """
-            CREATE TABLE IF NOT EXISTS ${tbName} (
-                a int, b int
-            )
-            unique key (a)
-            distributed by hash(a) buckets 16
-            properties(
-                "replication_allocation" = "tag.location.default:1",
-                "disable_auto_compaction" = "true"
-            );
-        """
-
-    //BackendId,Cluster,IP,HeartbeatPort,BePort,HttpPort,BrpcPort,LastStartTime,LastHeartbeat,Alive,SystemDecommissioned,ClusterDecommissioned,TabletNum,DataUsedCapacity,AvailCapacity,TotalCapacity,UsedPct,MaxDiskUsedPct,Tag,ErrMsg,Version,Status
-    String[][] backends = sql """ show backends; """
-    assertTrue(backends.size() > 0)
-    StringBuilder sbCommand = new StringBuilder();
-
-    sql "insert into ${tbName} values(1,1),(2,1);"
-    sql "insert into ${tbName} values(1,11),(2,11);"
-    sql "insert into ${tbName} values(3,1);"
-    sql "sync"
-
-    qt_select_init "select * from ${tbName} order by a, b"
-
-    // enable skip_delete_bitmap and check select result,
-    // the rows that have duplicate primary key and marked delete, will be returned
-    sql "SET skip_delete_bitmap=true"
-    qt_select_skip_delete_bitmap "select * from ${tbName} order by a, b"
-
-    // turn off skip_delete_bitmap
-    sql "SET skip_delete_bitmap=false"
-
-    // batch delete and select again:
-    // curl --location-trusted -uroot: -H "column_separator:|" -H "columns:a, b" -H "merge_type: delete" -T delete.csv http://127.0.0.1:8030/api/test_skip/t1/_stream_load
-    streamLoad {
-        table "${tbName}"
-
-        set 'column_separator', '|'
-        set 'columns', 'a, b'
-        set 'merge_type', 'delete'
-
-        file 'test_unique_mow_table_debug_data_delete.csv'
-
-        time 10000 // limit inflight 10s
-    }
-    sql "sync"
-    qt_select_batch_delete "select * from ${tbName} order by a, b"
-
-    // delete rows with a = 2:
-    sql "delete from ${tbName} where a = 2;"
-    sql "sync"
-    qt_select_sql_delete "select * from ${tbName} order by a, b"
-
-    // enable skip_delete_predicate, rows deleted with delete statement is returned:
-    sql "SET skip_delete_predicate=true"
-    qt_select_skip_delete1 "select * from ${tbName} order by a, b"
-
-    sql "SET skip_delete_predicate=false"
-
-    sql "SET skip_delete_bitmap=true"
-    qt_select_skip_delete_bitmap_after_batch_delete "select * from ${tbName} order by a, b"
-
-    // enable skip_delete_predicate, rows deleted with delete statement is also returned:
-    sql "SET skip_delete_predicate=true"
-    qt_select_skip_delete2 "select * from ${tbName} order by a, b"
-
-    sql "DROP TABLE ${tbName}"
-}
diff --git a/regression-test/suites/data_model_p0/unique/test_unique_table_debug_data.groovy b/regression-test/suites/data_model_p0/unique/test_unique_table_debug_data.groovy
index e631146e5c..db0c4e4ac7 100644
--- a/regression-test/suites/data_model_p0/unique/test_unique_table_debug_data.groovy
+++ b/regression-test/suites/data_model_p0/unique/test_unique_table_debug_data.groovy
@@ -34,8 +34,7 @@ suite("test_unique_table_debug_data") {
             distributed by hash(a) buckets 16
             properties(
                 "replication_allocation" = "tag.location.default:1",
-                "disable_auto_compaction" = "true",
-                "enable_unique_key_merge_on_write" = "false"
+                "disable_auto_compaction" = "true"
             );
         """
 
diff --git a/regression-test/suites/nereids_p0/delete/load.groovy b/regression-test/suites/nereids_p0/delete/load.groovy
index 6415361b09..6fe3a5ad10 100644
--- a/regression-test/suites/nereids_p0/delete/load.groovy
+++ b/regression-test/suites/nereids_p0/delete/load.groovy
@@ -29,7 +29,8 @@ suite("load") {
         distributed by hash(id, id1)
         properties(
             'replication_num'='1',
-            "function_column.sequence_col" = "c4"
+            "function_column.sequence_col" = "c4",
+            "enable_unique_key_merge_on_write" = "true"
         );
     '''
 
diff --git a/regression-test/suites/unique_with_mow_p0/test_create_table.groovy b/regression-test/suites/unique_with_mow_p0/test_create_table.groovy
index fdc382fbb0..5326672db2 100644
--- a/regression-test/suites/unique_with_mow_p0/test_create_table.groovy
+++ b/regression-test/suites/unique_with_mow_p0/test_create_table.groovy
@@ -35,7 +35,7 @@ suite("test_create_table") {
                     "enable_unique_key_merge_on_write" = "true"
              );
         """
-        exception "only support unique key table"
+        exception "enable_unique_key_merge_on_write property only support unique key table"
     }
 
     // duplicate table with enable_unique_key_merge_on_write property
@@ -54,7 +54,7 @@ suite("test_create_table") {
                     "enable_unique_key_merge_on_write" = "false"
              );
         """
-        exception "only support unique key table"
+        exception "enable_unique_key_merge_on_write property only support unique key table"
     }
 
     // agg table with enable_unique_key_merge_on_write property
@@ -73,7 +73,7 @@ suite("test_create_table") {
                     "enable_unique_key_merge_on_write" = "true"
              );
         """
-        exception "only support unique key table"
+        exception "enable_unique_key_merge_on_write property only support unique key table"
     }
 
     // agg table with enable_unique_key_merge_on_write property
@@ -92,6 +92,6 @@ suite("test_create_table") {
                     "enable_unique_key_merge_on_write" = "false"
              );
         """
-        exception "only support unique key table"
+        exception "enable_unique_key_merge_on_write property only support unique key table"
     }
-}
\ No newline at end of file
+}
diff --git a/regression-test/suites/update/test_update_mow.groovy b/regression-test/suites/update/test_update_mow.groovy
index c8d120ff9f..df0d1a4e57 100644
--- a/regression-test/suites/update/test_update_mow.groovy
+++ b/regression-test/suites/update/test_update_mow.groovy
@@ -29,7 +29,10 @@ suite("test_update_mow", "p0") {
                 date_value date
             )
             UNIQUE KEY(k)
-            DISTRIBUTED BY HASH(k) BUCKETS 5 properties("replication_num" = "1");
+            DISTRIBUTED BY HASH(k) BUCKETS 5 properties(
+                "replication_num" = "1",
+                "enable_unique_key_merge_on_write" = "true"
+            );
         """
     sql "insert into ${tbName1} values(1, 1, 1, '2000-01-01');"
     sql "insert into ${tbName1} values(2, 1, 1, '2000-01-01');"


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