This is an automated email from the ASF dual-hosted git repository.
zhangstar333 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 0afa283651d [Enhancement](partition) Refine some auto partition
behaviours (#32737)
0afa283651d is described below
commit 0afa283651d6f46e6ed5eb98bff5cc7dc09d2433
Author: zclllyybb <[email protected]>
AuthorDate: Mon Apr 1 11:04:03 2024 +0800
[Enhancement](partition) Refine some auto partition behaviours (#32737)
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 | 6 ++-
be/src/vec/sink/vrow_distribution.cpp | 8 +++-
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 | 51 ++++++++++++++--------
.../auto_partition/test_auto_partition_load.groovy | 2 +-
.../test_auto_range_partition.groovy | 50 ++++++---------------
.../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, 157 insertions(+), 135 deletions(-)
diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp
index ab8c3d562e7..c594b6ad7c6 100644
--- a/be/src/exec/tablet_info.cpp
+++ b/be/src/exec/tablet_info.cpp
@@ -46,7 +46,7 @@
#include "util/string_util.h"
#include "vec/columns/column.h"
// NOLINTNEXTLINE(unused-includes)
-#include "vec/exprs/vexpr_context.h"
+#include "vec/exprs/vexpr_context.h" // IWYU pragma: keep
#include "vec/exprs/vliteral.h"
#include "vec/runtime/vdatetime_value.h"
@@ -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;
@@ -330,6 +331,7 @@
VOlapTablePartitionParam::VOlapTablePartitionParam(std::shared_ptr<OlapTableSche
slot->get_data_type_ptr(),
slot->col_name()});
}
}
+ VLOG_TRACE << _partition_block.dump_structure();
} else {
// we insert all. but not all will be used. it will controlled by
_partition_slot_locs
for (auto* slot : _slots) {
diff --git a/be/src/vec/sink/vrow_distribution.cpp
b/be/src/vec/sink/vrow_distribution.cpp
index 8ec8814293b..27fa66c6a12 100644
--- a/be/src/vec/sink/vrow_distribution.cpp
+++ b/be/src/vec/sink/vrow_distribution.cpp
@@ -33,6 +33,7 @@
#include "service/backend_options.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"
@@ -142,7 +143,7 @@ Status VRowDistribution::_replace_overwriting_partition() {
std::set<int64_t> id_deduper;
for (const auto* part : _partitions) {
if (part == nullptr) [[unlikely]] {
- return Status::EndOfFile(
+ return Status::InternalError(
"Cannot found origin partitions in auto detect
overwriting, stop processing");
}
if (_new_partition_ids.contains(part->id)) {
@@ -441,7 +442,10 @@ Status VRowDistribution::generate_rows_distribution(
int result_idx = -1;
// we just calc left range here. leave right to FE to avoid dup
calc.
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 629eba3dfda..0809f4825d6 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -3192,15 +3192,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 9cb0a5e36a6..288d5a8457a 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());
@@ -204,6 +206,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 31c882a05f0..e836d3a1446 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
@@ -2137,8 +2137,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 a59d7c61a23..fcf35b739c4 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
@@ -140,6 +140,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");
+ }
}
/**
@@ -235,19 +238,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);
}
@@ -286,4 +287,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 967b81ebb08..ada3e08fdc9 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
@@ -425,6 +425,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 8416bd396b0..51fed437708 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 56891782b65..ed7cc927273 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 5c2b5c6f79c..e3300266ada 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
@@ -192,7 +192,7 @@ suite("test_auto_partition_behavior") {
`k0` int not null,
`k1` datetime(6) not null
)
- AUTO PARTITION BY RANGE date_trunc(`k1`, 'year')
+ auto partition by range (date_trunc(`k1`, 'year'))
(
)
DISTRIBUTED BY HASH(`k0`) BUCKETS 10
@@ -209,20 +209,35 @@ suite("test_auto_partition_behavior") {
qt_sql_non_order3 """ select * from non_order where k1 = '2013-12-12'; """
// range partition can't auto create null partition
+ sql " set experimental_enable_nereids_planner=true "
sql "drop table if exists invalid_null_range"
- sql """
- create table invalid_null_range(
- k0 datetime(6) null
- )
- auto partition by range date_trunc(k0, 'hour')
- (
- )
- DISTRIBUTED BY HASH(`k0`) BUCKETS 2
- properties("replication_num" = "1");
- """
test {
- sql " insert into invalid_null_range values (null); "
- exception "Can't create partition for NULL Range"
+ sql """
+ create table invalid_null_range(
+ k0 datetime(6) null
+ )
+ auto partition by range (date_trunc(k0, 'hour'))
+ (
+ )
+ DISTRIBUTED BY HASH(`k0`) BUCKETS 2
+ properties("replication_num" = "1");
+ """
+ exception "AUTO RANGE PARTITION doesn't support NULL column"
+ }
+ sql " set experimental_enable_nereids_planner=false "
+ sql "drop table if exists invalid_null_range"
+ test {
+ sql """
+ create table invalid_null_range(
+ k0 datetime(6) null
+ )
+ auto partition by range (date_trunc(k0, 'hour'))
+ (
+ )
+ DISTRIBUTED BY HASH(`k0`) BUCKETS 2
+ properties("replication_num" = "1");
+ """
+ exception "AUTO RANGE PARTITION doesn't support NULL column"
}
// PROHIBIT different timeunit of interval when use both auto & dynamic
partition
@@ -234,7 +249,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
(
@@ -259,7 +274,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
(
@@ -304,7 +319,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
@@ -327,7 +342,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 aa1a1b1f5d3..d728ba9a145 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
@@ -25,7 +25,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 9273851e72c..52325690ce7 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
@@ -23,7 +23,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
@@ -38,15 +38,14 @@ suite("test_auto_range_partition") {
qt_select01 """ select * from range_table1 WHERE TIME_STAMP = '2022-12-15'
order by TIME_STAMP """
qt_select02 """ select * from range_table1 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
@@ -54,12 +53,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 """
sql "drop table if exists range_table2"
sql """
@@ -68,7 +67,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
@@ -85,35 +84,14 @@ suite("test_auto_range_partition") {
qt_select11 """ select * from range_table2 WHERE TIME_STAMP = '2022-12-15
22:22:22.222' order by TIME_STAMP """
qt_select12 """ select * from range_table2 WHERE TIME_STAMP > '2022-12-15
22:22:22.222' order by TIME_STAMP """
- sql "drop table if exists range_table3"
- sql """
- CREATE TABLE `range_table3` (
- `k1` INT,
- `k2` DATETIMEV2(3) NOT NULL,
- `k3` DATETIMEV2(6)
- ) ENGINE=OLAP
- DUPLICATE KEY(`k1`)
- COMMENT 'OLAP'
- AUTO PARTITION BY RANGE date_trunc(`k2`, 'day')
- (
- )
- DISTRIBUTED BY HASH(`k1`) BUCKETS 16
- PROPERTIES (
- "replication_allocation" = "tag.location.default: 1"
- );
- """
- sql """ insert into range_table3 values (1, '1990-01-01', '2000-01-01
12:12:12.123456'), (2, '1991-02-01', '2000-01-01'), (3, '1991-01-01',
'2000-01-01'), (3, '1991-01-01', '2000-01-01') """
- result1 = sql "show partitions from range_table3"
- logger.info("${result1}")
- assertEquals(result1.size(), 3)
-
sql "drop table if exists right_bound"
sql """
create table right_bound(
- k0 datetime(6) null
+ k0 datetime(6) not null
)
- auto partition by range date_trunc(k0, 'second')
+ auto partition by range (date_trunc(k0, 'second'))
(
+ partition pX values less than ("1970-01-01")
)
DISTRIBUTED BY HASH(`k0`) BUCKETS auto
properties("replication_num" = "1");
@@ -123,5 +101,5 @@ suite("test_auto_range_partition") {
qt_right_bound " select * from right_bound order by k0; "
result2 = sql "show partitions from right_bound"
logger.info("${result2}")
- assertEquals(result2.size(), 1)
+ assertEquals(result2.size(), 2)
}
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: [email protected]
For additional commands, e-mail: [email protected]