[Enhancement](function) Support date_trunc(date) and use it in auto partition (#24341)

Support date_trunc(date) and use it in auto partition
This commit is contained in:
zclllyybb
2023-09-14 16:53:09 +08:00
committed by GitHub
parent f303a99250
commit 4fbb25bc55
19 changed files with 371 additions and 44 deletions

View File

@ -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 <typename DataType>
struct DatatypeToColumn {};
template <>
struct DatatypeToColumn<vectorized::DataTypeDate> {
using type = vectorized::ColumnDate;
};
template <>
struct DatatypeToColumn<vectorized::DataTypeDateTime> {
using type = vectorized::ColumnDateTime;
};
template <>
struct DatatypeToColumn<vectorized::DataTypeDateV2> {
using type = vectorized::ColumnDateV2;
};
template <>
struct DatatypeToColumn<vectorized::DataTypeDateTimeV2> {
using type = vectorized::ColumnDateTimeV2;
};
template <typename DataType>
using DateToColumnV = DatatypeToColumn<DataType>::type;
// DateTypeDate -> VecDateTimeValue
template <typename DataType>
struct DateToDateValueType {};
template <>
struct DateToDateValueType<vectorized::DataTypeDate> {
using type = vectorized::VecDateTimeValue;
};
template <>
struct DateToDateValueType<vectorized::DataTypeDateTime> {
using type = vectorized::VecDateTimeValue;
};
template <>
struct DateToDateValueType<vectorized::DataTypeDateV2> {
using type = vectorized::DateV2Value<vectorized::DateV2ValueType>;
};
template <>
struct DateToDateValueType<vectorized::DataTypeDateTimeV2> {
using type = vectorized::DateV2Value<vectorized::DateTimeV2ValueType>;
};
template <typename DataType>
using DateToDateValueTypeV = DateToDateValueType<DataType>::type;
// ColumnDate -> Int64 (see also columns_number.h)
template <typename ColumnType>
requires requires { typename ColumnType::value_type; }
struct ValueTypeOfDateColumn {
using type = ColumnType::value_type;
};
template <typename ColumnType>
using ValueTypeOfDateColumnV = ValueTypeOfDateColumn<ColumnType>::type;
} // namespace doris::date_cast

View File

@ -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 <typename DateValueType, typename ArgType>
template <typename DateType>
struct DateTrunc {
static constexpr auto name = "date_trunc";
using ColumnType = date_cast::DateToColumnV<DateType>;
using DateValueType = date_cast::DateToDateValueTypeV<DateType>;
using ArgType = date_cast::ValueTypeOfDateColumnV<ColumnType>;
static bool is_variadic() { return true; }
static DataTypes get_variadic_argument_types() {
if constexpr (std::is_same_v<DateValueType, VecDateTimeValue>) {
return {std::make_shared<DataTypeDateTime>(), std::make_shared<DataTypeString>()};
} else {
return {std::make_shared<DataTypeDateTimeV2>(), std::make_shared<DataTypeString>()};
}
return {std::make_shared<DateType>(), std::make_shared<DataTypeString>()};
}
static DataTypePtr get_return_type_impl(const DataTypes& arguments) {
if constexpr (std::is_same_v<DateValueType, VecDateTimeValue>) {
return make_nullable(std::make_shared<DataTypeDateTime>());
} else {
return make_nullable(std::make_shared<DataTypeDateTimeV2>());
}
return make_nullable(std::make_shared<DateType>());
}
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<const ColumnVector<ArgType>*>(argument_columns[0].get());
auto datetime_column = static_cast<const ColumnType*>(argument_columns[0].get());
auto str_column = static_cast<const ColumnString*>(argument_columns[1].get());
auto& rdata = str_column->get_chars();
auto& roffsets = str_column->get_offsets();
ColumnPtr res = ColumnVector<ArgType>::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<ColumnVector<ArgType>*>(res->assume_mutable().get())->get_data(),
static_cast<ColumnType*>(res->assume_mutable().get())->get_data(),
null_map->get_data(), input_rows_count);
} else {
execute_impl(
datetime_column->get_data(), rdata, roffsets,
static_cast<ColumnVector<ArgType>*>(res->assume_mutable().get())->get_data(),
null_map->get_data(), input_rows_count);
execute_impl(datetime_column->get_data(), rdata, roffsets,
static_cast<ColumnType*>(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<StrToDate>;
using FunctionMakeDate = FunctionOtherTypesToDateType<MakeDateImpl>;
using FunctionDateTrunc = FunctionOtherTypesToDateType<DateTrunc<VecDateTimeValue, Int64>>;
using FunctionDateTruncV2 =
FunctionOtherTypesToDateType<DateTrunc<DateV2Value<DateTimeV2ValueType>, UInt64>>;
using FunctionDateTruncDate = FunctionOtherTypesToDateType<DateTrunc<DataTypeDate>>;
using FunctionDateTruncDateV2 = FunctionOtherTypesToDateType<DateTrunc<DataTypeDateV2>>;
using FunctionDateTruncDatetime = FunctionOtherTypesToDateType<DateTrunc<DataTypeDateTime>>;
using FunctionDateTruncDatetimeV2 = FunctionOtherTypesToDateType<DateTrunc<DataTypeDateTimeV2>>;
void register_function_timestamp(SimpleFunctionFactory& factory) {
factory.register_function<FunctionStrToDate>();
factory.register_function<FunctionMakeDate>();
factory.register_function<FromDays>();
factory.register_function<FunctionDateTrunc>();
factory.register_function<FunctionDateTruncV2>();
factory.register_function<FunctionDateTruncDate>();
factory.register_function<FunctionDateTruncDateV2>();
factory.register_function<FunctionDateTruncDatetime>();
factory.register_function<FunctionDateTruncDatetimeV2>();
factory.register_function<FunctionUnixTimestamp<UnixTimeStampImpl>>();
factory.register_function<FunctionUnixTimestamp<UnixTimeStampDateImpl<DataTypeDate>>>();

View File

@ -3059,9 +3059,48 @@ bool DateV2Value<T>::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<DAY>(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 <typename T>

View File

@ -26,6 +26,7 @@
#include <cstddef>
#include <iostream>
#include <iterator>
#include <shared_mutex>
#include <string>
#include <string_view>
#include <tuple>
@ -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;

View File

@ -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);
}

View File

@ -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);

View File

@ -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());
}
}

View File

@ -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());

View File

@ -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();

View File

@ -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<FunctionSignature> 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)
);
/**

View File

@ -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();

View File

@ -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'],

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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"

View File

@ -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) {

View File

@ -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 """

View File

@ -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())
}
}