diff --git a/be/src/util/datetype_cast.hpp b/be/src/util/datetype_cast.hpp new file mode 100644 index 0000000000..02ab94aeee --- /dev/null +++ b/be/src/util/datetype_cast.hpp @@ -0,0 +1,83 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include "vec/columns/columns_number.h" +#include "vec/data_types/data_type_date.h" +#include "vec/data_types/data_type_time_v2.h" +#include "vec/runtime/vdatetime_value.h" + +namespace doris::date_cast { + +// DataTypeDate -> ColumnDate +template +struct DatatypeToColumn {}; +template <> +struct DatatypeToColumn { + using type = vectorized::ColumnDate; +}; +template <> +struct DatatypeToColumn { + using type = vectorized::ColumnDateTime; +}; +template <> +struct DatatypeToColumn { + using type = vectorized::ColumnDateV2; +}; +template <> +struct DatatypeToColumn { + using type = vectorized::ColumnDateTimeV2; +}; + +template +using DateToColumnV = DatatypeToColumn::type; + +// DateTypeDate -> VecDateTimeValue +template +struct DateToDateValueType {}; +template <> +struct DateToDateValueType { + using type = vectorized::VecDateTimeValue; +}; +template <> +struct DateToDateValueType { + using type = vectorized::VecDateTimeValue; +}; +template <> +struct DateToDateValueType { + using type = vectorized::DateV2Value; +}; +template <> +struct DateToDateValueType { + using type = vectorized::DateV2Value; +}; + +template +using DateToDateValueTypeV = DateToDateValueType::type; + +// ColumnDate -> Int64 (see also columns_number.h) +template + requires requires { typename ColumnType::value_type; } +struct ValueTypeOfDateColumn { + using type = ColumnType::value_type; +}; + +template +using ValueTypeOfDateColumnV = ValueTypeOfDateColumn::type; + +} // namespace doris::date_cast diff --git a/be/src/vec/functions/function_timestamp.cpp b/be/src/vec/functions/function_timestamp.cpp index 3510365f46..5222b529a0 100644 --- a/be/src/vec/functions/function_timestamp.cpp +++ b/be/src/vec/functions/function_timestamp.cpp @@ -33,6 +33,7 @@ #include "runtime/types.h" #include "udf/udf.h" #include "util/binary_cast.hpp" +#include "util/datetype_cast.hpp" #include "util/time_lut.h" #include "vec/aggregate_functions/aggregate_function.h" #include "vec/columns/column.h" @@ -368,26 +369,22 @@ private: } }; -template +template struct DateTrunc { static constexpr auto name = "date_trunc"; + using ColumnType = date_cast::DateToColumnV; + using DateValueType = date_cast::DateToDateValueTypeV; + using ArgType = date_cast::ValueTypeOfDateColumnV; + static bool is_variadic() { return true; } static DataTypes get_variadic_argument_types() { - if constexpr (std::is_same_v) { - return {std::make_shared(), std::make_shared()}; - } else { - return {std::make_shared(), std::make_shared()}; - } + return {std::make_shared(), std::make_shared()}; } static DataTypePtr get_return_type_impl(const DataTypes& arguments) { - if constexpr (std::is_same_v) { - return make_nullable(std::make_shared()); - } else { - return make_nullable(std::make_shared()); - } + return make_nullable(std::make_shared()); } static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments, @@ -404,22 +401,21 @@ struct DateTrunc { std::tie(argument_columns[1], col_const[1]) = unpack_if_const(block.get_by_position(arguments[1]).column); - auto datetime_column = static_cast*>(argument_columns[0].get()); + auto datetime_column = static_cast(argument_columns[0].get()); auto str_column = static_cast(argument_columns[1].get()); auto& rdata = str_column->get_chars(); auto& roffsets = str_column->get_offsets(); - ColumnPtr res = ColumnVector::create(); + ColumnPtr res = ColumnType::create(); if (col_const[1]) { execute_impl_right_const( datetime_column->get_data(), str_column->get_data_at(0), - static_cast*>(res->assume_mutable().get())->get_data(), + static_cast(res->assume_mutable().get())->get_data(), null_map->get_data(), input_rows_count); } else { - execute_impl( - datetime_column->get_data(), rdata, roffsets, - static_cast*>(res->assume_mutable().get())->get_data(), - null_map->get_data(), input_rows_count); + execute_impl(datetime_column->get_data(), rdata, roffsets, + static_cast(res->assume_mutable().get())->get_data(), + null_map->get_data(), input_rows_count); } block.get_by_position(result).column = ColumnNullable::create(res, std::move(null_map)); @@ -1269,16 +1265,19 @@ public: using FunctionStrToDate = FunctionOtherTypesToDateType; using FunctionMakeDate = FunctionOtherTypesToDateType; -using FunctionDateTrunc = FunctionOtherTypesToDateType>; -using FunctionDateTruncV2 = - FunctionOtherTypesToDateType, UInt64>>; +using FunctionDateTruncDate = FunctionOtherTypesToDateType>; +using FunctionDateTruncDateV2 = FunctionOtherTypesToDateType>; +using FunctionDateTruncDatetime = FunctionOtherTypesToDateType>; +using FunctionDateTruncDatetimeV2 = FunctionOtherTypesToDateType>; void register_function_timestamp(SimpleFunctionFactory& factory) { factory.register_function(); factory.register_function(); factory.register_function(); - factory.register_function(); - factory.register_function(); + factory.register_function(); + factory.register_function(); + factory.register_function(); + factory.register_function(); factory.register_function>(); factory.register_function>>(); diff --git a/be/src/vec/runtime/vdatetime_value.cpp b/be/src/vec/runtime/vdatetime_value.cpp index 32a93424e7..3c1ebf2720 100644 --- a/be/src/vec/runtime/vdatetime_value.cpp +++ b/be/src/vec/runtime/vdatetime_value.cpp @@ -3059,9 +3059,48 @@ bool DateV2Value::datetime_trunc() { default: return false; } - return true; + } else { // is_datev2 + if (!is_valid_date()) { + return false; + } + switch (unit) { + case SECOND: + case MINUTE: + case HOUR: + case DAY: + break; + case WEEK: { + TimeInterval interval(DAY, weekday(), true); + date_add_interval(interval); + break; + } + case MONTH: { + date_v2_value_.day_ = 1; + break; + } + case QUARTER: { + date_v2_value_.day_ = 1; + if (date_v2_value_.month_ <= 3) { + date_v2_value_.month_ = 1; + } else if (date_v2_value_.month_ <= 6) { + date_v2_value_.month_ = 4; + } else if (date_v2_value_.month_ <= 9) { + date_v2_value_.month_ = 7; + } else { + date_v2_value_.month_ = 10; + } + break; + } + case YEAR: { + date_v2_value_.day_ = 1; + date_v2_value_.month_ = 1; + break; + } + default: + return false; + } } - return false; + return true; } template diff --git a/be/src/vec/runtime/vdatetime_value.h b/be/src/vec/runtime/vdatetime_value.h index aa2b23d942..edede1708f 100644 --- a/be/src/vec/runtime/vdatetime_value.h +++ b/be/src/vec/runtime/vdatetime_value.h @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -34,7 +35,6 @@ #include "util/hash_util.hpp" #include "util/time_lut.h" #include "util/timezone_utils.h" -#include "vec/common/hash_table/phmap_fwd_decl.h" namespace cctz { class time_zone; diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java index d48b82ef16..e75aec0d2a 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java @@ -398,13 +398,6 @@ public abstract class Type { return isScalarType(PrimitiveType.DATETIMEV2); } - public boolean isDateLike() { - return isScalarType(PrimitiveType.DATETIME) - || isScalarType(PrimitiveType.DATETIMEV2) - || isScalarType(PrimitiveType.DATE) - || isScalarType(PrimitiveType.DATEV2); - } - public boolean isTimeV2() { return isScalarType(PrimitiveType.TIMEV2); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/IntLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/IntLiteral.java index b37dc648db..6e7c5d6f21 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/IntLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/IntLiteral.java @@ -314,7 +314,7 @@ public class IntLiteral extends LiteralExpr { return res; } return this; - } else if (targetType.isDateLike()) { + } else if (targetType.isDateType()) { try { //int like 20200101 can be cast to date(2020,01,01) DateLiteral res = new DateLiteral("" + value, targetType); 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 8c5ff8b0ef..ea66d4a0f2 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 @@ -198,9 +198,10 @@ public class PartitionDesc { } if (this instanceof RangePartitionDesc && partitionExprs != null) { if (partitionExprs.get(0) instanceof FunctionCallExpr) { - if (!columnDef.getType().isDatetime() && !columnDef.getType().isDatetimeV2()) { + if (!columnDef.getType().isDateType()) { throw new AnalysisException( - "auto create partition function expr need datetime/datetimev2 type. " + "Auto range partition needs Date/DateV2/" + + "Datetime/DatetimeV2 column as partition column" + partitionExprs.get(0).toSql()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionExprUtil.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionExprUtil.java index dadf74b27c..750386162f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionExprUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionExprUtil.java @@ -121,7 +121,7 @@ public class PartitionExprUtil { filterPartitionValues.add(value); if (partitionType == PartitionType.RANGE) { String beginTime = value; - DateLiteral beginDateTime = new DateLiteral(beginTime, Type.DATETIMEV2); + DateLiteral beginDateTime = new DateLiteral(beginTime, partitionColumnType); partitionName += String.format(DATETIME_NAME_FORMATTER, beginDateTime.getYear(), beginDateTime.getMonth(), beginDateTime.getDay(), beginDateTime.getHour(), beginDateTime.getMinute(), beginDateTime.getSecond()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java index 4c7f80ab6c..db824e1935 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java @@ -320,6 +320,16 @@ public class DateTimeExtractAndTransform { return DateTimeV2Literal.fromJavaDateType(dateTruncHelper(date.toJavaDateType(), trunc.getValue())); } + @ExecFunction(name = "date_trunc", argTypes = { "DATE", "VARCHAR" }, returnType = "DATE") + public static Expression dateTrunc(DateLiteral date, VarcharLiteral trunc) { + return DateLiteral.fromJavaDateType(dateTruncHelper(date.toJavaDateType(), trunc.getValue())); + } + + @ExecFunction(name = "date_trunc", argTypes = { "DATEV2", "VARCHAR" }, returnType = "DATEV2") + public static Expression dateTrunc(DateV2Literal date, VarcharLiteral trunc) { + return DateV2Literal.fromJavaDateType(dateTruncHelper(date.toJavaDateType(), trunc.getValue())); + } + private static LocalDateTime dateTruncHelper(LocalDateTime dateTime, String trunc) { int year = dateTime.getYear(); int month = dateTime.getMonthValue(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateTrunc.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateTrunc.java index 53c24ff37d..03408b48dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateTrunc.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DateTrunc.java @@ -27,6 +27,8 @@ import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DateTimeType; import org.apache.doris.nereids.types.DateTimeV2Type; +import org.apache.doris.nereids.types.DateType; +import org.apache.doris.nereids.types.DateV2Type; import org.apache.doris.nereids.types.VarcharType; import com.google.common.base.Preconditions; @@ -44,7 +46,10 @@ public class DateTrunc extends ScalarFunction public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT) .args(DateTimeV2Type.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT), - FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, VarcharType.SYSTEM_DEFAULT) + FunctionSignature.ret(DateTimeType.INSTANCE).args(DateTimeType.INSTANCE, VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(DateV2Type.INSTANCE) + .args(DateV2Type.INSTANCE, VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(DateType.INSTANCE).args(DateType.INSTANCE, VarcharType.SYSTEM_DEFAULT) ); /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/rewrite/FEFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/rewrite/FEFunctions.java index dd1bce88cc..cc883e28d0 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/rewrite/FEFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rewrite/FEFunctions.java @@ -523,8 +523,8 @@ public class FEFunctions { } @FEFunction(name = "date_trunc", argTypes = {"DATETIME", "VARCHAR"}, returnType = "DATETIME") - public static DateLiteral dateTrunc(LiteralExpr date, LiteralExpr truncate) { - if (date.getType().isDateLike()) { + public static DateLiteral dateTruncDatetime(LiteralExpr date, LiteralExpr truncate) { + if (date.getType().isDateType()) { DateLiteral dateLiteral = ((DateLiteral) date); LocalDateTime localDate = dateTruncHelper(LocalDateTime.of( (int) dateLiteral.getYear(), (int) dateLiteral.getMonth(), (int) dateLiteral.getDay(), @@ -538,8 +538,8 @@ public class FEFunctions { } @FEFunction(name = "date_trunc", argTypes = {"DATETIMEV2", "VARCHAR"}, returnType = "DATETIMEV2") - public static DateLiteral dateTruncV2(LiteralExpr date, LiteralExpr truncate) { - if (date.getType().isDateLike()) { + public static DateLiteral dateTruncDatetimeV2(LiteralExpr date, LiteralExpr truncate) { + if (date.getType().isDateType()) { DateLiteral dateLiteral = ((DateLiteral) date); LocalDateTime localDate = dateTruncHelper(LocalDateTime.of( (int) dateLiteral.getYear(), (int) dateLiteral.getMonth(), (int) dateLiteral.getDay(), @@ -552,6 +552,34 @@ public class FEFunctions { return null; } + @FEFunction(name = "date_trunc", argTypes = { "DATE", "VARCHAR" }, returnType = "DATE") + public static DateLiteral dateTruncDate(LiteralExpr date, LiteralExpr truncate) { + if (date.getType().isDateType()) { + DateLiteral dateLiteral = ((DateLiteral) date); + LocalDateTime localDate = dateTruncHelper(LocalDateTime.of( + (int) dateLiteral.getYear(), (int) dateLiteral.getMonth(), (int) dateLiteral.getDay(), 0, 0, 0), + truncate.getStringValue()); + + return new DateLiteral(localDate.getYear(), localDate.getMonthValue(), localDate.getDayOfMonth(), + localDate.getHour(), localDate.getMinute(), localDate.getSecond(), date.getType()); + } + return null; + } + + @FEFunction(name = "date_trunc", argTypes = { "DATEV2", "VARCHAR" }, returnType = "DATEV2") + public static DateLiteral dateTruncDateV2(LiteralExpr date, LiteralExpr truncate) { + if (date.getType().isDateType()) { + DateLiteral dateLiteral = ((DateLiteral) date); + LocalDateTime localDate = dateTruncHelper(LocalDateTime.of( + (int) dateLiteral.getYear(), (int) dateLiteral.getMonth(), (int) dateLiteral.getDay(), 0, 0, 0), + truncate.getStringValue()); + + return new DateLiteral(localDate.getYear(), localDate.getMonthValue(), localDate.getDayOfMonth(), + localDate.getHour(), localDate.getMinute(), localDate.getSecond(), date.getType()); + } + return null; + } + private static LocalDateTime dateTruncHelper(LocalDateTime dateTime, String trunc) { int year = dateTime.getYear(); int month = dateTime.getMonthValue(); diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 4b24edd69a..a1673e3f38 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -901,6 +901,8 @@ visible_functions = { [['date_ceil'], 'DATETIMEV2', ['DATETIMEV2', 'INT'], 'ALWAYS_NULLABLE'], [['date_trunc'], 'DATETIMEV2', ['DATETIMEV2', 'VARCHAR'], 'ALWAYS_NULLABLE'], [['date_trunc'], 'DATETIME', ['DATETIME', 'VARCHAR'], 'ALWAYS_NULLABLE'], + [['date_trunc'], 'DATEV2', ['DATEV2', 'VARCHAR'], 'ALWAYS_NULLABLE'], + [['date_trunc'], 'DATE', ['DATE', 'VARCHAR'], 'ALWAYS_NULLABLE'], [['year'], 'SMALLINT', ['DATETIME'], 'ALWAYS_NULLABLE'], [['month'], 'TINYINT', ['DATETIME'], 'ALWAYS_NULLABLE'], diff --git a/regression-test/data/nereids_function_p0/scalar_function/D.out b/regression-test/data/nereids_function_p0/scalar_function/D.out index a7fb554e20..9d9ed55dd4 100644 --- a/regression-test/data/nereids_function_p0/scalar_function/D.out +++ b/regression-test/data/nereids_function_p0/scalar_function/D.out @@ -231,6 +231,64 @@ 2012-03-11T11:00 2012-03-12T12:00 +-- !sql_date_trunc_Date_Varchar -- +\N +2012-03-01 +2012-03-01 +2012-03-01 +2012-03-01 +2012-03-01 +2012-03-01 +2012-03-01 +2012-03-01 +2012-03-01 +2012-03-01 +2012-03-01 +2012-03-01 + +-- !sql_date_trunc_Date_Varchar_notnull -- +2012-02-27 +2012-02-27 +2012-02-27 +2012-02-27 +2012-03-05 +2012-03-05 +2012-03-05 +2012-03-05 +2012-03-05 +2012-03-05 +2012-03-05 +2012-03-12 + +-- !sql_date_trunc_DateV2_Varchar -- +\N +2012-03-01 +2012-03-02 +2012-03-03 +2012-03-04 +2012-03-05 +2012-03-06 +2012-03-07 +2012-03-08 +2012-03-09 +2012-03-10 +2012-03-11 +2012-03-12 + +-- !sql_date_trunc_DateV2_Varchar_not_null -- +2012-01-01 +2012-01-01 +2012-01-01 +2012-01-01 +2012-01-01 +2012-01-01 +2012-01-01 +2012-01-01 +2012-01-01 +2012-01-01 +2012-01-01 +2012-01-01 + -- !sql_datediff_DateTime_DateTime -- \N 0 diff --git a/regression-test/data/partition_p0/auto_partition/test_auto_range_partition.out b/regression-test/data/partition_p0/auto_partition/test_auto_range_partition.out index f359c996ec..5cab9d69b1 100644 --- a/regression-test/data/partition_p0/auto_partition/test_auto_range_partition.out +++ b/regression-test/data/partition_p0/auto_partition/test_auto_range_partition.out @@ -32,6 +32,38 @@ 2122-12-19T00:00 2122-12-20T00:00 +-- !date1 -- +2022-05-18 +2022-11-14 +2022-12-15 +2022-12-16 +2022-12-17 +2022-12-19 +2022-12-20 +2122-09-18 +2122-12-14 +2122-12-15 +2122-12-16 +2122-12-17 +2122-12-19 +2122-12-20 + +-- !date2 -- +2022-12-15 + +-- !date3 -- +2022-12-16 +2022-12-17 +2022-12-19 +2022-12-20 +2122-09-18 +2122-12-14 +2122-12-15 +2122-12-16 +2122-12-17 +2122-12-19 +2122-12-20 + -- !select10 -- 2022-11-14T22:22:22.222 2022-11-15T22:22:22.222 diff --git a/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_trunc.out b/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_trunc.out index cc78cac91f..a9492698e0 100644 --- a/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_trunc.out +++ b/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_trunc.out @@ -143,3 +143,39 @@ 2015-04-02T00:00 2015-01-01T00:00 2015-04-02T00:00 2015-01-01T00:00 +-- !date_week -- +\N \N +1989-03-21 1989-03-20 +1901-12-31 1901-12-30 +2012-03-14 2012-03-12 +3124-10-10 3124-10-06 +2015-01-01 2014-12-29 +2014-11-11 2014-11-10 +1988-03-21 1988-03-21 +1989-03-21 1989-03-20 +1991-08-11 1991-08-05 +2015-04-02 2015-03-30 +2015-04-02 2015-03-30 +1991-08-11 1991-08-05 +2015-04-02 2015-03-30 +2015-04-02 2015-03-30 +9999-12-12 9999-12-06 + +-- !date_year -- +\N \N +1989-03-21 1989-01-01 +1901-12-31 1901-01-01 +2012-03-14 2012-01-01 +3124-10-10 3124-01-01 +2015-01-01 2015-01-01 +2014-11-11 2014-01-01 +1988-03-21 1988-01-01 +1989-03-21 1989-01-01 +1991-08-11 1991-01-01 +2015-04-02 2015-01-01 +2015-04-02 2015-01-01 +1991-08-11 1991-01-01 +2015-04-02 2015-01-01 +2015-04-02 2015-01-01 +9999-12-12 9999-01-01 + diff --git a/regression-test/suites/nereids_function_p0/scalar_function/D.groovy b/regression-test/suites/nereids_function_p0/scalar_function/D.groovy index 219680bf2f..fb4025cfcc 100644 --- a/regression-test/suites/nereids_function_p0/scalar_function/D.groovy +++ b/regression-test/suites/nereids_function_p0/scalar_function/D.groovy @@ -37,6 +37,10 @@ suite("nereids_scalar_fn_D") { qt_sql_date_trunc_DateTime_Varchar_notnull "select date_trunc(kdtm, 'month') from fn_test_not_nullable order by kdtm, kvchrs1" qt_sql_date_trunc_DateTimeV2_Varchar "select date_trunc(kdtmv2s1, 'day') from fn_test order by kdtmv2s1, kvchrs1" qt_sql_date_trunc_DateTimeV2_Varchar_notnull "select date_trunc(kdtmv2s1, 'hour') from fn_test_not_nullable order by kdtmv2s1, kvchrs1" + qt_sql_date_trunc_Date_Varchar "select date_trunc(kdt, 'month') from fn_test order by kdt, kvchrs1" + qt_sql_date_trunc_Date_Varchar_notnull "select date_trunc(kdt, 'week') from fn_test_not_nullable order by kdt, kvchrs1" + qt_sql_date_trunc_DateV2_Varchar "select date_trunc(kdtv2, 'day') from fn_test order by kdtv2, kvchrs1" + qt_sql_date_trunc_DateV2_Varchar_not_null "select date_trunc(kdtv2, 'year') from fn_test_not_nullable order by kdtv2, kvchrs1" qt_sql_datediff_DateTime_DateTime "select datediff(kdtm, kdtm) from fn_test order by kdtm, kdtm" qt_sql_datediff_DateTime_DateTime_notnull "select datediff(kdtm, kdtm) from fn_test_not_nullable order by kdtm, kdtm" qt_sql_datediff_DateTimeV2_DateTimeV2 "select datediff(kdtmv2s1, kdtmv2s1) from fn_test order by kdtmv2s1, kdtmv2s1" diff --git a/regression-test/suites/nereids_p0/expression/fold_constant/fold_constant_by_fe.groovy b/regression-test/suites/nereids_p0/expression/fold_constant/fold_constant_by_fe.groovy index fbd4d1a326..7e6a42ac60 100644 --- a/regression-test/suites/nereids_p0/expression/fold_constant/fold_constant_by_fe.groovy +++ b/regression-test/suites/nereids_p0/expression/fold_constant/fold_constant_by_fe.groovy @@ -124,9 +124,11 @@ suite("test_fold_constant_by_fe") { res = sql "explain select date_trunc('${date}', 'year'), date_trunc('${date}', 'month'), date_trunc('${date}', 'day')" res = res.split('VUNION')[1] assertFalse(res.contains("date_trunc")) + assertFalse(res.contains("cast")) res = sql "explain select date_trunc('${date}', 'hour'), date_trunc('${date}', 'minute'), date_trunc('${date}', 'second')" res = res.split('VUNION')[1] assertFalse(res.contains("date_trunc")) + assertFalse(res.contains("cast")) } for (date in test_date) { 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 874704ea8f..1c70a84228 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 @@ -18,7 +18,6 @@ suite("test_auto_range_partition") { def tblName1 = "range_table1" sql "drop table if exists ${tblName1}" - // not support datev2 now. need impl date_trunc(datev2) sql """ CREATE TABLE `${tblName1}` ( `TIME_STAMP` datetimev2 NOT NULL COMMENT '采集日期' @@ -40,6 +39,29 @@ 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 """ + CREATE TABLE `${tblDate}` ( + `TIME_STAMP` datev2 NOT NULL COMMENT '采集日期' + ) ENGINE=OLAP + DUPLICATE KEY(`TIME_STAMP`) + COMMENT 'OLAP' + AUTO PARTITION BY RANGE date_trunc(`TIME_STAMP`, 'month') + ( + ) + DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10 + PROPERTIES ( + "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') """ + + 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 """ + def tblName2 = "range_table2" sql "drop table if exists ${tblName2}" sql """ diff --git a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_trunc.groovy b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_trunc.groovy index b6943b9670..7ef142d410 100644 --- a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_trunc.groovy +++ b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_trunc.groovy @@ -60,15 +60,28 @@ suite("test_date_trunc") { qt_select_date_trunc_quarter """ SELECT k11, date_trunc(k11,'quarter') FROM baseall order by k1,k2,k3;""" qt_select_date_trunc_year """ SELECT k11, date_trunc(k11,'YeaR') FROM baseall order by k1,k2,k3;""" + qt_date_week """ SELECT k10, date_trunc(k10,'Week') FROM baseall order by k1,k2,k3;""" + qt_date_year """ SELECT k10, date_trunc(k10,'YeaR') FROM baseall order by k1,k2,k3;""" + try { sql """ SELECT date_trunc(k11,k7) FROM baseall ; """ } catch (Exception e) { assertTrue(e.getMessage().contains("must be a string constant"), e.getMessage()) } - try { sql """ SELECT date_trunc(k11,'AAAA') FROM baseall ; """ } catch (Exception e) { assertTrue(e.getMessage().contains("param only support argument"), e.getMessage()) } + + try { + sql """ SELECT date_trunc(k10,k7) FROM baseall ; """ + } catch (Exception e) { + assertTrue(e.getMessage().contains("must be a string constant"), e.getMessage()) + } + try { + sql """ SELECT date_trunc(k10,'yearr') FROM baseall ; """ + } catch (Exception e) { + assertTrue(e.getMessage().contains("param only support argument"), e.getMessage()) + } }