[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:
83
be/src/util/datetype_cast.hpp
Normal file
83
be/src/util/datetype_cast.hpp
Normal 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
|
||||
@ -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>>>();
|
||||
|
||||
@ -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>
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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);
|
||||
}
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
@ -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());
|
||||
|
||||
@ -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();
|
||||
|
||||
@ -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)
|
||||
);
|
||||
|
||||
/**
|
||||
|
||||
@ -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();
|
||||
|
||||
@ -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'],
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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"
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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 """
|
||||
|
||||
@ -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())
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user