ClickHouse/src/Functions/date_trunc.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

185 lines
7.4 KiB
C++
Raw Normal View History

2020-08-12 13:50:54 +00:00
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
2024-03-19 16:04:29 +00:00
#include <Columns/ColumnsNumber.h>
2020-08-12 13:50:54 +00:00
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeInterval.h>
2024-03-19 16:04:29 +00:00
#include <Formats/FormatSettings.h>
2020-08-12 13:50:54 +00:00
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
2020-08-26 14:28:54 +00:00
extern const int BAD_ARGUMENTS;
2020-08-12 13:50:54 +00:00
}
2020-09-07 18:00:37 +00:00
namespace
{
2020-08-12 13:50:54 +00:00
2023-04-17 15:06:07 +00:00
class FunctionDateTrunc : public IFunction
{
public:
static constexpr auto name = "dateTrunc";
explicit FunctionDateTrunc(ContextPtr context_) : context(context_) {}
2023-04-17 15:06:07 +00:00
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
{
2023-04-17 15:06:07 +00:00
/// The first argument is a constant string with the name of datepart.
2020-08-12 13:50:54 +00:00
enum ResultType
{
Date,
DateTime,
DateTime64,
};
ResultType result_type;
2023-04-17 15:06:07 +00:00
String datepart_param;
auto check_first_argument = [&] {
2023-04-17 15:06:07 +00:00
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());
2020-08-12 13:50:54 +00:00
2023-12-07 15:55:23 +00:00
datepart_param = Poco::toLower(datepart_column->getValue<String>());
2023-04-17 15:06:07 +00:00
if (datepart_param.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty",
getName());
2020-08-12 13:50:54 +00:00
2023-04-17 15:06:07 +00:00
if (!IntervalKind::tryParseString(datepart_param, datepart_kind))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName());
2020-08-12 13:50:54 +00:00
if ((datepart_kind == IntervalKind::Kind::Year) || (datepart_kind == IntervalKind::Kind::Quarter)
|| (datepart_kind == IntervalKind::Kind::Month) || (datepart_kind == IntervalKind::Kind::Week))
result_type = ResultType::Date;
else if ((datepart_kind == IntervalKind::Kind::Day) || (datepart_kind == IntervalKind::Kind::Hour)
|| (datepart_kind == IntervalKind::Kind::Minute) || (datepart_kind == IntervalKind::Kind::Second))
result_type = ResultType::DateTime;
else
result_type = ResultType::DateTime64;
2023-04-17 15:06:07 +00:00
};
2020-08-12 13:50:54 +00:00
2023-04-17 15:06:07 +00:00
bool second_argument_is_date = false;
auto check_second_argument = [&] {
2023-04-17 15:06:07 +00:00
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());
2023-04-17 15:06:07 +00:00
second_argument_is_date = isDate(arguments[1].type);
2024-02-27 13:37:21 +00:00
if (second_argument_is_date && ((datepart_kind == IntervalKind::Kind::Hour)
|| (datepart_kind == IntervalKind::Kind::Minute) || (datepart_kind == IntervalKind::Kind::Second)))
2023-04-17 15:06:07 +00:00
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", getName());
};
auto check_timezone_argument = [&] {
2023-04-17 15:06:07 +00:00
if (!WhichDataType(arguments[2].type).isString())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. "
2023-04-17 15:06:07 +00:00
"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 == ResultType::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);
2023-04-17 15:06:07 +00:00
};
2020-08-12 13:50:54 +00:00
2023-04-17 15:06:07 +00:00
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,
2023-04-17 15:06:07 +00:00
"Number of arguments for function {} doesn't match: passed {}, should be 2 or 3",
getName(), arguments.size());
2023-04-17 15:06:07 +00:00
}
2020-08-12 13:50:54 +00:00
if (result_type == ResultType::Date)
return std::make_shared<DataTypeDate>();
else if (result_type == ResultType::DateTime)
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1, false));
else
{
size_t scale;
if (datepart_kind == IntervalKind::Kind::Millisecond)
scale = 3;
else if (datepart_kind == IntervalKind::Kind::Microsecond)
scale = 6;
else if (datepart_kind == IntervalKind::Kind::Nanosecond)
scale = 9;
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 1, false));
}
2023-04-17 15:06:07 +00:00
}
2020-08-12 13:50:54 +00:00
2023-04-17 15:06:07 +00:00
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 2}; }
2020-08-12 13:50:54 +00:00
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
2023-04-17 15:06:07 +00:00
{
ColumnsWithTypeAndName temp_columns(arguments.size());
temp_columns[0] = arguments[1];
2020-08-12 13:50:54 +00:00
2023-04-17 15:06:07 +00:00
const UInt16 interval_value = 1;
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), ""};
2020-08-12 13:50:54 +00:00
2023-04-17 15:06:07 +00:00
auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context);
2020-08-12 13:50:54 +00:00
2023-04-17 15:06:07 +00:00
if (arguments.size() == 2)
return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count);
2020-08-12 13:50:54 +00:00
temp_columns[2] = arguments[2];
return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count);
2023-04-17 15:06:07 +00:00
}
bool hasInformationAboutMonotonicity() const override
{
return true;
}
2023-04-17 15:06:07 +00:00
Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override
{
return { .is_monotonic = true, .is_always_monotonic = true };
2023-04-17 15:06:07 +00:00
}
private:
ContextPtr context;
mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second;
};
2020-08-12 13:50:54 +00:00
2020-09-07 18:00:37 +00:00
}
2020-08-12 13:50:54 +00:00
2022-08-27 22:17:13 +00:00
REGISTER_FUNCTION(DateTrunc)
2020-08-12 13:50:54 +00:00
{
2022-08-27 22:17:13 +00:00
factory.registerFunction<FunctionDateTrunc>();
2020-08-12 13:50:54 +00:00
/// Compatibility alias.
2022-08-27 22:17:13 +00:00
factory.registerAlias("DATE_TRUNC", "dateTrunc", FunctionFactory::CaseInsensitive);
2020-08-12 13:50:54 +00:00
}
}