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 2024/04/09 07:51:09 UTC

(doris) branch branch-2.1 updated: [Enhancement](partition) Refine some auto partition behaviours (#32737) (#33412)

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

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/branch-2.1 by this push:
     new e574b35833d [Enhancement](partition) Refine some auto partition behaviours (#32737) (#33412)
e574b35833d is described below

commit e574b35833dccaaf89c00e6b3e35f021e3d61731
Author: zclllyybb <zh...@selectdb.com>
AuthorDate: Tue Apr 9 15:51:02 2024 +0800

    [Enhancement](partition) Refine some auto partition behaviours (#32737) (#33412)
    
    fix legacy planner grammer
    fix nereids planner parsing
    fix cases
    forbid auto range partition with null column
    fix CreateTableStmt with auto partition and some partition items.
    1 and 2 are about #31585
    doc pr: apache/doris-website#488
---
 be/src/exec/tablet_info.cpp                        |  3 ++-
 be/src/vec/sink/vrow_distribution.cpp              |  6 ++++-
 fe/fe-core/src/main/cup/sql_parser.cup             |  6 ++---
 .../apache/doris/analysis/ListPartitionDesc.java   |  2 +-
 .../org/apache/doris/analysis/PartitionDesc.java   |  9 +++++--
 .../apache/doris/analysis/RangePartitionDesc.java  |  3 ++-
 .../apache/doris/datasource/InternalCatalog.java   |  3 +--
 .../doris/nereids/parser/PartitionTableInfo.java   | 29 +++++++++++++++++-----
 .../trees/plans/commands/info/CreateTableInfo.java |  3 ++-
 .../doris/analysis/RangePartitionPruneTest.java    |  2 +-
 .../exploration/mv/MaterializedViewUtilsTest.java  |  2 +-
 .../trees/plans/CreateTableCommandTest.java        |  2 +-
 .../doris/service/FrontendServiceImplTest.java     |  2 +-
 .../mv/dimension/dimension_1.groovy                |  4 +--
 .../mv/dimension/dimension_2_3.groovy              |  4 +--
 .../mv/dimension/dimension_2_4.groovy              |  4 +--
 .../mv/dimension/dimension_2_5.groovy              |  4 +--
 .../mv/dimension/dimension_2_6.groovy              |  4 +--
 .../mv/dimension/dimension_2_full_join.groovy      |  4 +--
 .../mv/dimension/dimension_2_inner_join.groovy     |  4 +--
 .../mv/dimension/dimension_2_left_anti_join.groovy |  4 +--
 .../mv/dimension/dimension_2_left_join.groovy      |  4 +--
 .../mv/dimension/dimension_2_left_semi_join.groovy |  4 +--
 .../dimension/dimension_2_right_anti_join.groovy   |  4 +--
 .../mv/dimension/dimension_2_right_join.groovy     |  4 +--
 .../dimension/dimension_2_right_semi_join.groovy   |  4 +--
 .../dimension_2_join_agg.groovy                    |  4 +--
 .../filter_equal_or_notequal.groovy                |  4 +--
 .../cross_join_range_date_increment_create.groovy  |  4 +--
 .../full_join_range_date_increment_create.groovy   |  4 +--
 .../inner_join_range_date_increment_create.groovy  |  4 +--
 ...ft_anti_join_range_date_increment_create.groovy |  4 +--
 .../left_join_range_date_increment_create.groovy   |  4 +--
 ...ft_semi_join_range_date_increment_create.groovy |  4 +--
 ...ht_anti_join_range_date_increment_create.groovy |  4 +--
 .../right_join_range_date_increment_create.groovy  |  4 +--
 ...ht_semi_join_range_date_increment_create.groovy |  4 +--
 .../self_conn_range_date_increment_create.groovy   |  4 +--
 .../mv/negative/negative_test.groovy               |  4 +--
 .../test_auto_partition_behavior.groovy            | 10 ++++----
 .../auto_partition/test_auto_partition_load.groovy |  2 +-
 .../test_auto_range_partition.groovy               | 21 ++++++++--------
 .../partition_p1/auto_partition/ddl/concurrent.sql |  2 +-
 .../auto_partition/ddl/stress_destination.sql      |  2 +-
 .../ddl/small_data_high_concurrrent_load.sql       |  2 +-
 .../ddl/stream_load_range_test_table.sql           |  2 +-
 46 files changed, 122 insertions(+), 95 deletions(-)

diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp
index af065f332a0..2bec0113ace 100644
--- a/be/src/exec/tablet_info.cpp
+++ b/be/src/exec/tablet_info.cpp
@@ -75,6 +75,8 @@ bool VOlapTablePartKeyComparator::operator()(const BlockRowWithIndicator lhs,
     bool l_use_new = std::get<2>(lhs);
     bool r_use_new = std::get<2>(rhs);
 
+    VLOG_TRACE << '\n' << l_block->dump_data() << '\n' << r_block->dump_data();
+
     if (l_row == -1) {
         return false;
     } else if (r_row == -1) {
@@ -93,7 +95,6 @@ bool VOlapTablePartKeyComparator::operator()(const BlockRowWithIndicator lhs,
         DCHECK(_slot_locs.size() == _param_locs.size())
                 << _slot_locs.size() << ' ' << _param_locs.size();
 
-        //TODO: use template to accelerate this for older compiler.
         const std::vector<uint16_t>* l_index = l_use_new ? &_param_locs : &_slot_locs;
         const std::vector<uint16_t>* r_index = r_use_new ? &_param_locs : &_slot_locs;
 
diff --git a/be/src/vec/sink/vrow_distribution.cpp b/be/src/vec/sink/vrow_distribution.cpp
index 9ddee3b8f0b..052194a15a0 100644
--- a/be/src/vec/sink/vrow_distribution.cpp
+++ b/be/src/vec/sink/vrow_distribution.cpp
@@ -29,6 +29,7 @@
 #include "runtime/runtime_state.h"
 #include "util/doris_metrics.h"
 #include "util/thrift_rpc_helper.h"
+#include "vec/columns/column.h"
 #include "vec/columns/column_const.h"
 #include "vec/columns/column_nullable.h"
 #include "vec/columns/column_vector.h"
@@ -331,7 +332,10 @@ Status VRowDistribution::generate_rows_distribution(
         for (int i = 0; i < func_size; ++i) {
             int result_idx = -1;
             RETURN_IF_ERROR(part_funcs[i]->execute(part_ctxs[i].get(), block.get(), &result_idx));
-            VLOG_DEBUG << "Partition-calculated block:" << block->dump_data();
+
+            VLOG_DEBUG << "Partition-calculated block:" << block->dump_data(0, 1);
+            DCHECK(result_idx != -1);
+
             partition_cols_idx.push_back(result_idx);
         }
 
diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index 8d6c43ff6c2..0b822601ae6 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -3164,15 +3164,15 @@ opt_partition ::=
          RESULT = new ListPartitionDesc(columns, list);
     :}
     /* expr range partition */
-    | KW_AUTO KW_PARTITION KW_BY KW_RANGE function_call_expr:fnExpr
+    | KW_AUTO KW_PARTITION KW_BY KW_RANGE LPAREN function_call_expr:fnExpr RPAREN
             LPAREN opt_all_partition_desc_list:list RPAREN
     {:
         ArrayList<Expr> exprs = new ArrayList<Expr>();
         exprs.add(fnExpr);
         RESULT = RangePartitionDesc.createRangePartitionDesc(exprs, list);
     :}
-    | KW_AUTO KW_PARTITION KW_BY KW_RANGE function_name:functionName LPAREN expr_list:l COMMA
-        KW_INTERVAL expr:v ident:u RPAREN LPAREN opt_all_partition_desc_list:list RPAREN
+    | KW_AUTO KW_PARTITION KW_BY KW_RANGE LPAREN function_name:functionName LPAREN expr_list:l COMMA
+        KW_INTERVAL expr:v ident:u RPAREN RPAREN LPAREN opt_all_partition_desc_list:list RPAREN
     {:
         Expr fnExpr = FunctionCallExpr.functionWithIntervalConvert(functionName.getFunction().toLowerCase(), l.get(0), v, u); 
         ArrayList<Expr> exprs = new ArrayList<Expr>();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ListPartitionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ListPartitionDesc.java
index bf94b227d83..90f9c054624 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ListPartitionDesc.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ListPartitionDesc.java
@@ -56,7 +56,7 @@ public class ListPartitionDesc extends PartitionDesc {
 
     public static ListPartitionDesc createListPartitionDesc(ArrayList<Expr> exprs,
             List<AllPartitionDesc> allPartitionDescs) throws AnalysisException {
-        List<String> colNames = getColNamesFromExpr(exprs, true);
+        List<String> colNames = getColNamesFromExpr(exprs, true, true);
         return new ListPartitionDesc(exprs, colNames, allPartitionDescs);
     }
 
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 39569d0b3e5..eb1bc7af747 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
@@ -102,7 +102,9 @@ public class PartitionDesc {
     // 1. partition by list (column) : now support one slotRef
     // 2. partition by range(column/function(column)) : support slotRef and some
     // special function eg: date_trunc, date_floor/ceil
-    public static List<String> getColNamesFromExpr(ArrayList<Expr> exprs, boolean isListPartition)
+    // not only for auto partition. maybe we should check for project partitiion also
+    public static List<String> getColNamesFromExpr(ArrayList<Expr> exprs, boolean isListPartition,
+            boolean isAutoPartition)
             throws AnalysisException {
         List<String> colNames = new ArrayList<>();
         for (Expr expr : exprs) {
@@ -128,7 +130,7 @@ public class PartitionDesc {
                                     + expr.toSql());
                 }
             } else if (expr instanceof SlotRef) {
-                if (!colNames.isEmpty() && !isListPartition) {
+                if (isAutoPartition && !colNames.isEmpty() && !isListPartition) {
                     throw new AnalysisException(
                             "auto create partition only support one slotRef in expr of RANGE partition. "
                                     + expr.toSql());
@@ -208,6 +210,9 @@ public class PartitionDesc {
                         throw new AnalysisException(
                                 "The partition column must be NOT NULL with allow_partition_column_nullable OFF");
                     }
+                    if (this instanceof RangePartitionDesc && isAutoCreatePartitions && columnDef.isAllowNull()) {
+                        throw new AnalysisException("AUTO RANGE PARTITION doesn't support NULL column");
+                    }
                     if (this instanceof RangePartitionDesc && partitionExprs != null) {
                         if (partitionExprs.get(0) instanceof FunctionCallExpr) {
                             if (!columnDef.getType().isDateType()) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java
index 57d696c37a5..176322766bf 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java
@@ -51,9 +51,10 @@ public class RangePartitionDesc extends PartitionDesc {
         this.isAutoCreatePartitions = true;
     }
 
+    // for parse auto partition
     public static RangePartitionDesc createRangePartitionDesc(ArrayList<Expr> exprs,
             List<AllPartitionDesc> allPartitionDescs) throws AnalysisException {
-        List<String> colNames = getColNamesFromExpr(exprs, false);
+        List<String> colNames = getColNamesFromExpr(exprs, false, true);
         return new RangePartitionDesc(exprs, colNames, allPartitionDescs);
     }
 
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 4e306a246c7..2bd2f21478b 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
@@ -2085,8 +2085,7 @@ public class InternalCatalog implements CatalogIf<Database> {
         PartitionInfo partitionInfo = null;
         Map<String, Long> partitionNameToId = Maps.newHashMap();
         if (partitionDesc != null) {
-            PartitionDesc partDesc = partitionDesc;
-            for (SinglePartitionDesc desc : partDesc.getSinglePartitionDescs()) {
+            for (SinglePartitionDesc desc : partitionDesc.getSinglePartitionDescs()) {
                 long partitionId = idGeneratorBuffer.getNextId();
                 partitionNameToId.put(desc.getPartitionName(), partitionId);
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java
index 33223a39ecc..c16d5a57024 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java
@@ -143,6 +143,9 @@ public class PartitionTableInfo {
             throw new AnalysisException(
                 "The partition column must be NOT NULL with allow_partition_column_nullable OFF");
         }
+        if (isAutoPartition && partitionType.equalsIgnoreCase(PartitionType.RANGE.name()) && column.isNullable()) {
+            throw new AnalysisException("AUTO RANGE PARTITION doesn't support NULL column");
+        }
     }
 
     /**
@@ -238,19 +241,17 @@ public class PartitionTableInfo {
             }
 
             try {
+                ArrayList<Expr> exprs = convertToLegacyAutoPartitionExprs(partitionList);
+                // here we have already extracted partitionColumns
                 if (partitionType.equals(PartitionType.RANGE.name())) {
                     if (isAutoPartition) {
-                        partitionDesc = new RangePartitionDesc(
-                            convertToLegacyAutoPartitionExprs(partitionList),
-                            partitionColumns, partitionDescs);
+                        partitionDesc = new RangePartitionDesc(exprs, partitionColumns, partitionDescs);
                     } else {
                         partitionDesc = new RangePartitionDesc(partitionColumns, partitionDescs);
                     }
                 } else {
                     if (isAutoPartition) {
-                        partitionDesc = new ListPartitionDesc(
-                            convertToLegacyAutoPartitionExprs(partitionList),
-                            partitionColumns, partitionDescs);
+                        partitionDesc = new ListPartitionDesc(exprs, partitionColumns, partitionDescs);
                     } else {
                         partitionDesc = new ListPartitionDesc(partitionColumns, partitionDescs);
                     }
@@ -289,4 +290,20 @@ public class PartitionTableInfo {
             }
         }).collect(Collectors.toList());
     }
+
+    /**
+     *  Get column names and put in partitionColumns
+     */
+    public void extractPartitionColumns() throws AnalysisException {
+        if (partitionList == null) {
+            return;
+        }
+        ArrayList<Expr> exprs = convertToLegacyAutoPartitionExprs(partitionList);
+        try {
+            partitionColumns = PartitionDesc.getColNamesFromExpr(exprs,
+                    partitionType.equalsIgnoreCase(PartitionType.LIST.name()), isAutoPartition);
+        } catch (Exception e) {
+            throw new AnalysisException(e.getMessage(), e.getCause());
+        }
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
index b67d1db27c3..33ec21ed66a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
@@ -90,7 +90,7 @@ public class CreateTableInfo {
     private String clusterName = null;
     private List<String> clusterKeysColumnNames = null;
     private List<Integer> clusterKeysColumnIds = null;
-    private PartitionTableInfo partitionTableInfo;
+    private PartitionTableInfo partitionTableInfo; // get when validate
 
     /**
      * constructor for create table
@@ -427,6 +427,7 @@ public class CreateTableInfo {
             });
 
             // validate partition
+            partitionTableInfo.extractPartitionColumns();
             partitionTableInfo.validatePartitionInfo(columnMap, properties, ctx, isEnableMergeOnWrite, isExternal);
 
             // validate distribution descriptor
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/RangePartitionPruneTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/RangePartitionPruneTest.java
index bdeda248908..4cd7f8d2049 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/RangePartitionPruneTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/RangePartitionPruneTest.java
@@ -108,7 +108,7 @@ public class RangePartitionPruneTest extends PartitionPruneTestBase {
                 + "    event_day DATETIME NOT NULL\n"
                 + ")\n"
                 + "DUPLICATE KEY(event_day)\n"
-                + "AUTO PARTITION BY range date_trunc(event_day, \"day\") (\n"
+                + "AUTO PARTITION BY range (date_trunc(event_day, \"day\")) (\n"
                 + "\tPARTITION `p20230807` values [(20230807 ), (20230808 )),\n"
                 + "\tPARTITION `p20020106` values [(20020106 ), (20020107 ))\n"
                 + ")\n"
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java
index 02fb18edbf7..bb5a6c8cb75 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java
@@ -194,7 +194,7 @@ public class MaterializedViewUtilsTest extends TestWithFeService {
                 + "    ) ENGINE=OLAP\n"
                 + "    DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )\n"
                 + "    COMMENT 'OLAP'\n"
-                + "    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()\n"
+                + "    AUTO PARTITION BY range (date_trunc(`l_shipdate`, 'day')) ()\n"
                 + "    DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 3\n"
                 + "    PROPERTIES (\n"
                 + "       \"replication_num\" = \"1\"\n"
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java
index b90d8ded6fd..8276f7fabb6 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/CreateTableCommandTest.java
@@ -724,7 +724,7 @@ public class CreateTableCommandTest extends TestWithFeService {
                     + "distributed by hash (id) properties (\"a\"=\"b\")");
         } catch (Exception e) {
             Assertions.assertEquals(
-                    "internal catalog does not support functions in 'LIST' partition",
+                    "errCode = 2, detailMessage = auto create partition only support slotRef in list partitions. func1(`id2`, '1')",
                     e.getMessage());
         }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/service/FrontendServiceImplTest.java b/fe/fe-core/src/test/java/org/apache/doris/service/FrontendServiceImplTest.java
index 318926167c9..1306a2bff88 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/service/FrontendServiceImplTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/service/FrontendServiceImplTest.java
@@ -99,7 +99,7 @@ public class FrontendServiceImplTest {
                 + "    city_code VARCHAR(100)\n"
                 + ")\n"
                 + "DUPLICATE KEY(event_day, site_id, city_code)\n"
-                + "AUTO PARTITION BY range date_trunc( event_day,'day') (\n"
+                + "AUTO PARTITION BY range (date_trunc( event_day,'day')) (\n"
                 + "\n"
                 + ")\n"
                 + "DISTRIBUTED BY HASH(event_day, site_id) BUCKETS 2\n"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy
index 3a60f5e90e4..50689b2733d 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy
@@ -43,7 +43,7 @@ suite("partition_mv_rewrite_dimension_1") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -73,7 +73,7 @@ suite("partition_mv_rewrite_dimension_1") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy
index 3a9d25b3f40..047779caa45 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy
@@ -44,7 +44,7 @@ suite("partition_mv_rewrite_dimension_2_3") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -74,7 +74,7 @@ suite("partition_mv_rewrite_dimension_2_3") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_4.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_4.groovy
index 180cf9b9095..85679ac99ac 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_4.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_4.groovy
@@ -44,7 +44,7 @@ suite("partition_mv_rewrite_dimension_2_4") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -74,7 +74,7 @@ suite("partition_mv_rewrite_dimension_2_4") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_5.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_5.groovy
index 6e6bb4bccdf..3efd9da11ea 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_5.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_5.groovy
@@ -44,7 +44,7 @@ suite("partition_mv_rewrite_dimension_2_5") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -74,7 +74,7 @@ suite("partition_mv_rewrite_dimension_2_5") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_6.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_6.groovy
index 59977c70503..efa219bf7dc 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_6.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_6.groovy
@@ -44,7 +44,7 @@ suite("partition_mv_rewrite_dimension_2_6") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -74,7 +74,7 @@ suite("partition_mv_rewrite_dimension_2_6") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy
index 82f3c0b0de8..496345eaa43 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy
@@ -44,7 +44,7 @@ suite("partition_mv_rewrite_dimension_2_full_join") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -74,7 +74,7 @@ suite("partition_mv_rewrite_dimension_2_full_join") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy
index 74046d9b3e4..58d550b45ba 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy
@@ -44,7 +44,7 @@ suite("partition_mv_rewrite_dimension_2_2") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -74,7 +74,7 @@ suite("partition_mv_rewrite_dimension_2_2") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy
index 13c6bc9c82c..8eed21914af 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy
@@ -44,7 +44,7 @@ suite("partition_mv_rewrite_dimension_2_left_anti_join") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -74,7 +74,7 @@ suite("partition_mv_rewrite_dimension_2_left_anti_join") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy
index cc3591c6f44..89a867013da 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy
@@ -44,7 +44,7 @@ suite("partition_mv_rewrite_dimension_2_1") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -74,7 +74,7 @@ suite("partition_mv_rewrite_dimension_2_1") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy
index e018aab7d79..9b0abb43554 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy
@@ -44,7 +44,7 @@ suite("partition_mv_rewrite_dimension_2_left_semi_join") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -74,7 +74,7 @@ suite("partition_mv_rewrite_dimension_2_left_semi_join") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy
index b6b576b1926..41701d9cc8c 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy
@@ -44,7 +44,7 @@ suite("partition_mv_rewrite_dimension_2_right_anti_join") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -74,7 +74,7 @@ suite("partition_mv_rewrite_dimension_2_right_anti_join") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy
index 996ca5ae78b..fb200034a99 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy
@@ -44,7 +44,7 @@ suite("partition_mv_rewrite_dimension_2_right_join") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -74,7 +74,7 @@ suite("partition_mv_rewrite_dimension_2_right_join") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy
index 39b94976106..fa2e0ace0f0 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy
@@ -44,7 +44,7 @@ suite("partition_mv_rewrite_dimension_2_right_semi_join") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -74,7 +74,7 @@ suite("partition_mv_rewrite_dimension_2_right_semi_join") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_2_join_agg.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_2_join_agg.groovy
index 055861fd344..3e083af5ab2 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_2_join_agg.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_2_join_agg.groovy
@@ -43,7 +43,7 @@ suite("dimension_2_join_agg_replenish") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -73,7 +73,7 @@ suite("dimension_2_join_agg_replenish") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy
index 1800a91b84b..b8fc1a347cb 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy
@@ -41,7 +41,7 @@ suite("filter_equal_or_notequal_case") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -71,7 +71,7 @@ suite("filter_equal_or_notequal_case") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_range_date_increment_create.groovy
index 56d8069b124..1cd76faef72 100644
--- a/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_range_date_increment_create.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_range_date_increment_create.groovy
@@ -40,7 +40,7 @@ suite("cross_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -70,7 +70,7 @@ suite("cross_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/full_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/full_join_range_date_increment_create.groovy
index 7c9f6e4d58b..26a2c0bb567 100644
--- a/regression-test/suites/nereids_rules_p0/mv/increment_create/full_join_range_date_increment_create.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/full_join_range_date_increment_create.groovy
@@ -40,7 +40,7 @@ suite("full_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -70,7 +70,7 @@ suite("full_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_range_date_increment_create.groovy
index 31b4d8ba9b3..c1de9c7f09e 100644
--- a/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_range_date_increment_create.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_range_date_increment_create.groovy
@@ -40,7 +40,7 @@ suite("inner_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -70,7 +70,7 @@ suite("inner_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_range_date_increment_create.groovy
index 0dbbe3abb57..436b574e7b9 100644
--- a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_range_date_increment_create.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_range_date_increment_create.groovy
@@ -40,7 +40,7 @@ suite("left_anti_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -70,7 +70,7 @@ suite("left_anti_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_range_date_increment_create.groovy
index a8eacdc4a69..a42a2e96372 100644
--- a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_range_date_increment_create.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_range_date_increment_create.groovy
@@ -40,7 +40,7 @@ suite("left_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -70,7 +70,7 @@ suite("left_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_range_date_increment_create.groovy
index a78ad12b3ff..01dc2eda881 100644
--- a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_range_date_increment_create.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_range_date_increment_create.groovy
@@ -40,7 +40,7 @@ suite("left_semi_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -70,7 +70,7 @@ suite("left_semi_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/right_anti_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/right_anti_join_range_date_increment_create.groovy
index 7e9e00a6cdc..c82579adbfc 100644
--- a/regression-test/suites/nereids_rules_p0/mv/increment_create/right_anti_join_range_date_increment_create.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/right_anti_join_range_date_increment_create.groovy
@@ -40,7 +40,7 @@ suite("right_anti_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -70,7 +70,7 @@ suite("right_anti_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/right_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/right_join_range_date_increment_create.groovy
index 9c00f72a39d..54f0d446d03 100644
--- a/regression-test/suites/nereids_rules_p0/mv/increment_create/right_join_range_date_increment_create.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/right_join_range_date_increment_create.groovy
@@ -40,7 +40,7 @@ suite("right_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -70,7 +70,7 @@ suite("right_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/right_semi_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/right_semi_join_range_date_increment_create.groovy
index cfc176f5471..77d09fadbe9 100644
--- a/regression-test/suites/nereids_rules_p0/mv/increment_create/right_semi_join_range_date_increment_create.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/right_semi_join_range_date_increment_create.groovy
@@ -40,7 +40,7 @@ suite("right_semi_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -70,7 +70,7 @@ suite("right_semi_join_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_range_date_increment_create.groovy
index 23ca627e7ed..8537198f54a 100644
--- a/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_range_date_increment_create.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_range_date_increment_create.groovy
@@ -40,7 +40,7 @@ suite("self_conn_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -70,7 +70,7 @@ suite("self_conn_range_date_increment_create") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/nereids_rules_p0/mv/negative/negative_test.groovy b/regression-test/suites/nereids_rules_p0/mv/negative/negative_test.groovy
index 672adc769fc..d9c61b989e1 100644
--- a/regression-test/suites/nereids_rules_p0/mv/negative/negative_test.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/negative/negative_test.groovy
@@ -43,7 +43,7 @@ suite("negative_partition_mv_rewrite") {
     ) ENGINE=OLAP
     DUPLICATE KEY(`o_orderkey`, `o_custkey`)
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`o_orderdate`, 'day') ()
+    auto partition by range (date_trunc(`o_orderdate`, 'day')) ()
     DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
@@ -73,7 +73,7 @@ suite("negative_partition_mv_rewrite") {
     ) ENGINE=OLAP
     DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )
     COMMENT 'OLAP'
-    AUTO PARTITION BY range date_trunc(`l_shipdate`, 'day') ()
+    auto partition by range (date_trunc(`l_shipdate`, 'day')) ()
     DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96
     PROPERTIES (
     "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy
index 951f656f55f..41a243501fb 100644
--- a/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy
+++ b/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy
@@ -116,7 +116,7 @@ suite("test_auto_partition_behavior") {
         ) ENGINE=OLAP
         AGGREGATE KEY(`k0`)
         COMMENT 'OLAP'
-        AUTO PARTITION BY RANGE date_trunc(`k0`, 'year')
+        auto partition by range (date_trunc(`k0`, 'year'))
         (
         )
         DISTRIBUTED BY HASH(`k0`) BUCKETS 10
@@ -257,7 +257,7 @@ suite("test_auto_partition_behavior") {
                 k1 DATETIME NOT NULL,
                 col1 int 
             )
-            auto PARTITION BY RANGE date_trunc(`k1`, 'year') ()
+            auto partition by range (date_trunc(`k1`, 'year')) ()
             DISTRIBUTED BY HASH(k1)
             PROPERTIES
             (
@@ -282,7 +282,7 @@ suite("test_auto_partition_behavior") {
                 k1 DATETIME NOT NULL,
                 col1 int 
             )
-            auto PARTITION BY RANGE date_trunc(`k1`, 'year') ()
+            auto partition by range (date_trunc(`k1`, 'year')) ()
             DISTRIBUTED BY HASH(k1)
             PROPERTIES
             (
@@ -327,7 +327,7 @@ suite("test_auto_partition_behavior") {
                 k0 datetime(6) NOT null,
                 k1 datetime(6) NOT null
             )
-            auto partition by range date_trunc(k0, k1, 'hour')
+            auto partition by range (date_trunc(k0, k1, 'hour'))
             (
             )
             DISTRIBUTED BY HASH(`k0`) BUCKETS 2
@@ -350,7 +350,7 @@ suite("test_auto_partition_behavior") {
             DISTRIBUTED BY HASH(`k0`) BUCKETS 2
             properties("replication_num" = "1");
         """
-        exception "partition expr date_trunc is illegal!"
+        exception "auto create partition only support one slotRef in function expr"
     }
     // test displacement of partition function
     test{
diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_partition_load.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_partition_load.groovy
index 81b440e0f80..6f20a6c33e7 100644
--- a/regression-test/suites/partition_p0/auto_partition/test_auto_partition_load.groovy
+++ b/regression-test/suites/partition_p0/auto_partition/test_auto_partition_load.groovy
@@ -26,7 +26,7 @@ suite("test_auto_partition_load") {
         ) ENGINE=OLAP
         DUPLICATE KEY(`k1`)
         COMMENT 'OLAP'
-        AUTO PARTITION BY RANGE date_trunc(`k2`, 'year')
+        auto partition by range (date_trunc(`k2`, 'year'))
         (
         )
         DISTRIBUTED BY HASH(`k1`) BUCKETS 16
diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_range_partition.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_range_partition.groovy
index 33574990bc0..cf11ceacba9 100644
--- a/regression-test/suites/partition_p0/auto_partition/test_auto_range_partition.groovy
+++ b/regression-test/suites/partition_p0/auto_partition/test_auto_range_partition.groovy
@@ -24,7 +24,7 @@ suite("test_auto_range_partition") {
         ) ENGINE=OLAP
         DUPLICATE KEY(`TIME_STAMP`)
         COMMENT 'OLAP'
-        AUTO PARTITION BY RANGE date_trunc(`TIME_STAMP`, 'day')
+        auto partition by range (date_trunc(`TIME_STAMP`, 'day'))
         (
         )
         DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10
@@ -39,15 +39,14 @@ suite("test_auto_range_partition") {
     qt_select01 """ select * from ${tblName1} WHERE TIME_STAMP = '2022-12-15' order by TIME_STAMP """
     qt_select02 """ select * from ${tblName1} WHERE TIME_STAMP > '2022-12-15' order by TIME_STAMP """
 
-    def tblDate = "range_table_date"
-    sql "drop table if exists ${tblDate}"
+    sql "drop table if exists range_table_date"
     sql """
-        CREATE TABLE `${tblDate}` (
+        CREATE TABLE `range_table_date` (
         `TIME_STAMP` datev2 NOT NULL COMMENT '采集日期'
         ) ENGINE=OLAP
         DUPLICATE KEY(`TIME_STAMP`)
         COMMENT 'OLAP'
-        AUTO PARTITION BY RANGE date_trunc(`TIME_STAMP`, 'month')
+        auto partition by range (date_trunc(`TIME_STAMP`, 'month'))
         (
         )
         DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10
@@ -55,12 +54,12 @@ suite("test_auto_range_partition") {
         "replication_allocation" = "tag.location.default: 1"
         );
         """
-    sql """ insert into ${tblDate} values ('2022-11-14'), ('2022-12-15'), ('2022-12-16'), ('2022-12-17'), ('2022-05-18'), ('2022-12-19'), ('2022-12-20') """
-    sql """ insert into ${tblDate} values ('2122-12-14'), ('2122-12-15'), ('2122-12-16'), ('2122-12-17'), ('2122-09-18'), ('2122-12-19'), ('2122-12-20') """
+    sql """ insert into range_table_date values ('2022-11-14'), ('2022-12-15'), ('2022-12-16'), ('2022-12-17'), ('2022-05-18'), ('2022-12-19'), ('2022-12-20') """
+    sql """ insert into range_table_date values ('2122-12-14'), ('2122-12-15'), ('2122-12-16'), ('2122-12-17'), ('2122-09-18'), ('2122-12-19'), ('2122-12-20') """
 
-    qt_date1 """ select * from ${tblDate} order by TIME_STAMP """
-    qt_date2 """ select * from ${tblDate} WHERE TIME_STAMP = '2022-12-15' order by TIME_STAMP """
-    qt_date3 """ select * from ${tblDate} WHERE TIME_STAMP > '2022-12-15' order by TIME_STAMP """
+    qt_date1 """ select * from range_table_date order by TIME_STAMP """
+    qt_date2 """ select * from range_table_date WHERE TIME_STAMP = '2022-12-15' order by TIME_STAMP """
+    qt_date3 """ select * from range_table_date WHERE TIME_STAMP > '2022-12-15' order by TIME_STAMP """
 
     def tblName2 = "range_table2"
     sql "drop table if exists ${tblName2}"
@@ -70,7 +69,7 @@ suite("test_auto_range_partition") {
         ) ENGINE=OLAP
         DUPLICATE KEY(`TIME_STAMP`)
         COMMENT 'OLAP'
-        AUTO PARTITION BY RANGE date_trunc(`TIME_STAMP`, 'day')
+        auto partition by range (date_trunc(`TIME_STAMP`, 'day'))
         (
         )
         DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10
diff --git a/regression-test/suites/partition_p1/auto_partition/ddl/concurrent.sql b/regression-test/suites/partition_p1/auto_partition/ddl/concurrent.sql
index cb7694654e1..1e2ebf7b597 100644
--- a/regression-test/suites/partition_p1/auto_partition/ddl/concurrent.sql
+++ b/regression-test/suites/partition_p1/auto_partition/ddl/concurrent.sql
@@ -10,7 +10,7 @@ CREATE TABLE `concurrent`(
   `col9` decimal(9, 3),
   `col10` char(128)
 ) duplicate KEY(`col1`)
-AUTO PARTITION BY range date_trunc(`col1`, 'day')
+auto partition by range (date_trunc(`col1`, 'day'))
 (
 )
 DISTRIBUTED BY HASH(`col1`) BUCKETS 10
diff --git a/regression-test/suites/partition_p1/auto_partition/ddl/stress_destination.sql b/regression-test/suites/partition_p1/auto_partition/ddl/stress_destination.sql
index f1dbb790987..275d65f79cb 100644
--- a/regression-test/suites/partition_p1/auto_partition/ddl/stress_destination.sql
+++ b/regression-test/suites/partition_p1/auto_partition/ddl/stress_destination.sql
@@ -13,7 +13,7 @@ CREATE TABLE `stress_destination` (
   `order_num` INT NULL
 ) ENGINE=OLAP
 UNIQUE KEY(`create_date`, `parent_org_id`, `org_id`)
-AUTO PARTITION BY RANGE date_trunc(`create_date`,'day')()
+auto partition by range (date_trunc(`create_date`,'day')) ()
 DISTRIBUTED BY HASH(`create_date`, `org_id`) BUCKETS AUTO
 PROPERTIES (
 "replication_allocation" = "tag.location.default: 1"
diff --git a/regression-test/suites/partition_p2/auto_partition/ddl/small_data_high_concurrrent_load.sql b/regression-test/suites/partition_p2/auto_partition/ddl/small_data_high_concurrrent_load.sql
index a85d7ce6c68..c3f7895dfa1 100644
--- a/regression-test/suites/partition_p2/auto_partition/ddl/small_data_high_concurrrent_load.sql
+++ b/regression-test/suites/partition_p2/auto_partition/ddl/small_data_high_concurrrent_load.sql
@@ -4,7 +4,7 @@ CREATE TABLE `small_data_high_concurrent_load_range`(
   `col3` decimal(9, 3),
   `col4` date
 ) duplicate KEY(`col1`)
-AUTO PARTITION BY range date_trunc(`col1`, 'day')
+auto partition by range (date_trunc(`col1`, 'day'))
 (
 )
 DISTRIBUTED BY HASH(`col1`) BUCKETS 10
diff --git a/regression-test/suites/partition_p2/auto_partition/ddl/stream_load_range_test_table.sql b/regression-test/suites/partition_p2/auto_partition/ddl/stream_load_range_test_table.sql
index f64b1a65a50..48100691945 100644
--- a/regression-test/suites/partition_p2/auto_partition/ddl/stream_load_range_test_table.sql
+++ b/regression-test/suites/partition_p2/auto_partition/ddl/stream_load_range_test_table.sql
@@ -40,7 +40,7 @@ CREATE TABLE `stream_load_range_test_table`(
   `col39` decimal(9, 3),
   `col40` char(128)
 ) UNIQUE KEY(`col1`)
-AUTO PARTITION BY range date_trunc(`col1`, 'day')
+auto partition by range (date_trunc(`col1`, 'day'))
 (
 )
 DISTRIBUTED BY HASH(`col1`) BUCKETS 10


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