mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-13 01:41:59 +00:00
date_trunc function to always return DateTime type
This commit is contained in:
parent
8bc0a3a899
commit
f2f47fc24d
@ -1,6 +1,6 @@
|
|||||||
#include <Columns/ColumnConst.h>
|
#include <Columns/ColumnConst.h>
|
||||||
#include <Columns/ColumnsNumber.h>
|
|
||||||
#include <Columns/ColumnString.h>
|
#include <Columns/ColumnString.h>
|
||||||
|
#include <Columns/ColumnsNumber.h>
|
||||||
#include <DataTypes/DataTypeDate.h>
|
#include <DataTypes/DataTypeDate.h>
|
||||||
#include <DataTypes/DataTypeDateTime.h>
|
#include <DataTypes/DataTypeDateTime.h>
|
||||||
#include <DataTypes/DataTypeInterval.h>
|
#include <DataTypes/DataTypeInterval.h>
|
||||||
@ -20,131 +20,163 @@ namespace ErrorCodes
|
|||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
class FunctionDateTrunc : public IFunction
|
class FunctionDateTrunc : public IFunction
|
||||||
{
|
|
||||||
public:
|
|
||||||
static constexpr auto name = "dateTrunc";
|
|
||||||
|
|
||||||
explicit FunctionDateTrunc(ContextPtr context_) : context(context_) {}
|
|
||||||
|
|
||||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionDateTrunc>(context); }
|
|
||||||
|
|
||||||
String getName() const override { return name; }
|
|
||||||
|
|
||||||
bool isVariadic() const override { return true; }
|
|
||||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
|
||||||
size_t getNumberOfArguments() const override { return 0; }
|
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
|
||||||
{
|
{
|
||||||
/// The first argument is a constant string with the name of datepart.
|
public:
|
||||||
|
static constexpr auto name = "dateTrunc";
|
||||||
|
|
||||||
auto result_type_is_date = false;
|
explicit FunctionDateTrunc(ContextPtr context_) : context(context_) { }
|
||||||
String datepart_param;
|
|
||||||
auto check_first_argument = [&] {
|
|
||||||
const ColumnConst * datepart_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
|
|
||||||
if (!datepart_column)
|
|
||||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be constant string: "
|
|
||||||
"name of datepart", getName());
|
|
||||||
|
|
||||||
datepart_param = datepart_column->getValue<String>();
|
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionDateTrunc>(context); }
|
||||||
if (datepart_param.empty())
|
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty",
|
|
||||||
getName());
|
|
||||||
|
|
||||||
if (!IntervalKind::tryParseString(datepart_param, datepart_kind))
|
String getName() const override { return name; }
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName());
|
|
||||||
|
|
||||||
result_type_is_date = (datepart_kind == IntervalKind::Year)
|
bool isVariadic() const override { return true; }
|
||||||
|| (datepart_kind == IntervalKind::Quarter) || (datepart_kind == IntervalKind::Month)
|
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||||
|| (datepart_kind == IntervalKind::Week);
|
size_t getNumberOfArguments() const override { return 0; }
|
||||||
};
|
|
||||||
|
|
||||||
bool second_argument_is_date = false;
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||||
auto check_second_argument = [&] {
|
|
||||||
if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type))
|
|
||||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}. "
|
|
||||||
"Should be a date or a date with time", arguments[1].type->getName(), getName());
|
|
||||||
|
|
||||||
second_argument_is_date = isDate(arguments[1].type);
|
|
||||||
|
|
||||||
if (second_argument_is_date && ((datepart_kind == IntervalKind::Hour)
|
|
||||||
|| (datepart_kind == IntervalKind::Minute) || (datepart_kind == IntervalKind::Second)))
|
|
||||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", getName());
|
|
||||||
};
|
|
||||||
|
|
||||||
auto check_timezone_argument = [&] {
|
|
||||||
if (!WhichDataType(arguments[2].type).isString())
|
|
||||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. "
|
|
||||||
"This argument is optional and must be a constant string with timezone name",
|
|
||||||
arguments[2].type->getName(), getName());
|
|
||||||
|
|
||||||
if (second_argument_is_date && result_type_is_date)
|
|
||||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
|
||||||
"The timezone argument of function {} with datepart '{}' "
|
|
||||||
"is allowed only when the 2nd argument has the type DateTime",
|
|
||||||
getName(), datepart_param);
|
|
||||||
};
|
|
||||||
|
|
||||||
if (arguments.size() == 2)
|
|
||||||
{
|
{
|
||||||
check_first_argument();
|
/// The first argument is a constant string with the name of datepart.
|
||||||
check_second_argument();
|
|
||||||
}
|
result_type_is_date = false;
|
||||||
else if (arguments.size() == 3)
|
String datepart_param;
|
||||||
{
|
auto check_first_argument = [&]
|
||||||
check_first_argument();
|
{
|
||||||
check_second_argument();
|
const ColumnConst * datepart_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
|
||||||
check_timezone_argument();
|
if (!datepart_column)
|
||||||
}
|
throw Exception(
|
||||||
else
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||||
{
|
"First argument for function {} must be constant string: "
|
||||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
"name of datepart",
|
||||||
"Number of arguments for function {} doesn't match: passed {}, should be 2 or 3",
|
getName());
|
||||||
getName(), arguments.size());
|
|
||||||
|
datepart_param = datepart_column->getValue<String>();
|
||||||
|
if (datepart_param.empty())
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty", getName());
|
||||||
|
|
||||||
|
if (!IntervalKind::tryParseString(datepart_param, datepart_kind))
|
||||||
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName());
|
||||||
|
|
||||||
|
result_type_is_date = (datepart_kind == IntervalKind::Year) || (datepart_kind == IntervalKind::Quarter)
|
||||||
|
|| (datepart_kind == IntervalKind::Month) || (datepart_kind == IntervalKind::Week);
|
||||||
|
};
|
||||||
|
|
||||||
|
bool second_argument_is_date = false;
|
||||||
|
auto check_second_argument = [&]
|
||||||
|
{
|
||||||
|
if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type))
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||||
|
"Illegal type {} of 2nd argument of function {}. "
|
||||||
|
"Should be a date or a date with time",
|
||||||
|
arguments[1].type->getName(),
|
||||||
|
getName());
|
||||||
|
|
||||||
|
second_argument_is_date = isDate(arguments[1].type);
|
||||||
|
|
||||||
|
if (second_argument_is_date
|
||||||
|
&& ((datepart_kind == IntervalKind::Hour) || (datepart_kind == IntervalKind::Minute)
|
||||||
|
|| (datepart_kind == IntervalKind::Second)))
|
||||||
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", getName());
|
||||||
|
};
|
||||||
|
|
||||||
|
auto check_timezone_argument = [&]
|
||||||
|
{
|
||||||
|
if (!WhichDataType(arguments[2].type).isString())
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||||
|
"Illegal type {} of argument of function {}. "
|
||||||
|
"This argument is optional and must be a constant string with timezone name",
|
||||||
|
arguments[2].type->getName(),
|
||||||
|
getName());
|
||||||
|
|
||||||
|
if (second_argument_is_date && result_type_is_date)
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||||
|
"The timezone argument of function {} with datepart '{}' "
|
||||||
|
"is allowed only when the 2nd argument has the type DateTime",
|
||||||
|
getName(),
|
||||||
|
datepart_param);
|
||||||
|
};
|
||||||
|
|
||||||
|
if (arguments.size() == 2)
|
||||||
|
{
|
||||||
|
check_first_argument();
|
||||||
|
check_second_argument();
|
||||||
|
}
|
||||||
|
else if (arguments.size() == 3)
|
||||||
|
{
|
||||||
|
check_first_argument();
|
||||||
|
check_second_argument();
|
||||||
|
check_timezone_argument();
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||||
|
"Number of arguments for function {} doesn't match: passed {}, should be 2 or 3",
|
||||||
|
getName(),
|
||||||
|
arguments.size());
|
||||||
|
}
|
||||||
|
|
||||||
|
if (result_type_is_date)
|
||||||
|
return std::make_shared<DataTypeDateTime>();
|
||||||
|
else
|
||||||
|
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1));
|
||||||
}
|
}
|
||||||
|
|
||||||
if (result_type_is_date)
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
return std::make_shared<DataTypeDate>();
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 2}; }
|
||||||
else
|
|
||||||
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1));
|
|
||||||
}
|
|
||||||
|
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
ColumnPtr
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 2}; }
|
executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||||
|
{
|
||||||
|
ColumnsWithTypeAndName temp_columns(arguments.size());
|
||||||
|
temp_columns[0] = arguments[1];
|
||||||
|
|
||||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
const UInt16 interval_value = 1;
|
||||||
{
|
const ColumnPtr interval_column = ColumnConst::create(ColumnInt64::create(1, interval_value), input_rows_count);
|
||||||
ColumnsWithTypeAndName temp_columns(arguments.size());
|
temp_columns[1] = {interval_column, std::make_shared<DataTypeInterval>(datepart_kind), ""};
|
||||||
temp_columns[0] = arguments[1];
|
|
||||||
|
|
||||||
const UInt16 interval_value = 1;
|
auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context);
|
||||||
const ColumnPtr interval_column = ColumnConst::create(ColumnInt64::create(1, interval_value), input_rows_count);
|
|
||||||
temp_columns[1] = {interval_column, std::make_shared<DataTypeInterval>(datepart_kind), ""};
|
|
||||||
|
|
||||||
auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context);
|
ColumnPtr truncated_column;
|
||||||
|
auto date_type = std::make_shared<DataTypeDate>();
|
||||||
|
|
||||||
if (arguments.size() == 2)
|
if (arguments.size() == 2)
|
||||||
return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count);
|
truncated_column = to_start_of_interval->build(temp_columns)
|
||||||
|
->execute(temp_columns, result_type_is_date ? date_type : result_type, input_rows_count);
|
||||||
|
else
|
||||||
|
{
|
||||||
|
temp_columns[2] = arguments[2];
|
||||||
|
truncated_column = to_start_of_interval->build(temp_columns)
|
||||||
|
->execute(temp_columns, result_type_is_date ? date_type : result_type, input_rows_count);
|
||||||
|
}
|
||||||
|
|
||||||
temp_columns[2] = arguments[2];
|
if (!result_type_is_date)
|
||||||
return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count);
|
return truncated_column;
|
||||||
}
|
|
||||||
|
|
||||||
bool hasInformationAboutMonotonicity() const override
|
ColumnsWithTypeAndName temp_truncated_column(1);
|
||||||
{
|
temp_truncated_column[0] = {truncated_column, date_type, ""};
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override
|
auto to_date_time_or_default = FunctionFactory::instance().get("toDateTime", context);
|
||||||
{
|
return to_date_time_or_default->build(temp_truncated_column)->execute(temp_truncated_column, result_type, input_rows_count);
|
||||||
return { .is_monotonic = true, .is_always_monotonic = true };
|
}
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
bool hasInformationAboutMonotonicity() const override { return true; }
|
||||||
ContextPtr context;
|
|
||||||
mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second;
|
Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override
|
||||||
};
|
{
|
||||||
|
return {.is_monotonic = true, .is_always_monotonic = true};
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
ContextPtr context;
|
||||||
|
mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second;
|
||||||
|
mutable bool result_type_is_date = false;
|
||||||
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user