2018-09-26 00:31:40 +00:00
|
|
|
#include <DataTypes/DataTypeDateTime.h>
|
2019-11-04 14:06:22 +00:00
|
|
|
#include <DataTypes/DataTypeDateTime64.h>
|
2018-09-26 00:31:40 +00:00
|
|
|
|
2021-05-17 07:30:42 +00:00
|
|
|
#include <Functions/IFunction.h>
|
2018-09-26 00:31:40 +00:00
|
|
|
#include <Functions/FunctionFactory.h>
|
|
|
|
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
|
|
|
|
2023-06-10 12:53:02 +00:00
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
|
2018-09-26 00:31:40 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2019-10-07 07:45:59 +00:00
|
|
|
#include <Common/assert_cast.h>
|
2018-09-26 00:31:40 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
|
|
|
}
|
|
|
|
|
2020-09-07 18:00:37 +00:00
|
|
|
namespace
|
|
|
|
{
|
2021-07-13 05:54:44 +00:00
|
|
|
class ExecutableFunctionToTimeZone : public IExecutableFunction
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
explicit ExecutableFunctionToTimeZone() = default;
|
2020-09-07 18:00:37 +00:00
|
|
|
|
2021-07-14 02:41:24 +00:00
|
|
|
String getName() const override { return "toTimezone"; }
|
2021-07-13 05:54:44 +00:00
|
|
|
|
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override
|
|
|
|
{
|
|
|
|
return arguments[0].column;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
class FunctionBaseToTimeZone : public IFunctionBase
|
2018-09-26 00:31:40 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-07-14 06:49:18 +00:00
|
|
|
FunctionBaseToTimeZone(
|
|
|
|
bool is_constant_timezone_,
|
|
|
|
DataTypes argument_types_,
|
|
|
|
DataTypePtr return_type_)
|
|
|
|
: is_constant_timezone(is_constant_timezone_)
|
|
|
|
, argument_types(std::move(argument_types_))
|
|
|
|
, return_type(std::move(return_type_)) {}
|
2021-07-13 05:54:44 +00:00
|
|
|
|
2021-07-14 02:41:24 +00:00
|
|
|
String getName() const override { return "toTimezone"; }
|
2021-07-13 05:54:44 +00:00
|
|
|
|
2021-08-10 11:31:15 +00:00
|
|
|
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
|
|
|
|
2021-07-13 05:54:44 +00:00
|
|
|
const DataTypes & getArgumentTypes() const override
|
|
|
|
{
|
|
|
|
return argument_types;
|
|
|
|
}
|
|
|
|
|
|
|
|
const DataTypePtr & getResultType() const override
|
|
|
|
{
|
|
|
|
return return_type;
|
|
|
|
}
|
|
|
|
|
|
|
|
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & /*arguments*/) const override
|
|
|
|
{
|
|
|
|
return std::make_unique<ExecutableFunctionToTimeZone>();
|
|
|
|
}
|
|
|
|
|
|
|
|
bool hasInformationAboutMonotonicity() const override { return is_constant_timezone; }
|
2018-09-26 00:31:40 +00:00
|
|
|
|
2021-07-13 05:54:44 +00:00
|
|
|
Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const override
|
2018-09-26 00:31:40 +00:00
|
|
|
{
|
2021-09-29 16:01:26 +00:00
|
|
|
const bool b = is_constant_timezone;
|
2022-09-22 15:49:27 +00:00
|
|
|
return { .is_monotonic = b, .is_positive = b, .is_always_monotonic = b, .is_strict = b };
|
2018-09-26 00:31:40 +00:00
|
|
|
}
|
|
|
|
|
2021-07-13 05:54:44 +00:00
|
|
|
private:
|
|
|
|
bool is_constant_timezone;
|
|
|
|
DataTypes argument_types;
|
|
|
|
DataTypePtr return_type;
|
|
|
|
};
|
|
|
|
|
|
|
|
/// Just changes time zone information for data type. The calculation is free.
|
|
|
|
class ToTimeZoneOverloadResolver : public IFunctionOverloadResolver
|
|
|
|
{
|
|
|
|
public:
|
2021-07-14 02:41:24 +00:00
|
|
|
static constexpr auto name = "toTimezone";
|
2021-07-13 05:54:44 +00:00
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
2018-09-26 00:31:40 +00:00
|
|
|
size_t getNumberOfArguments() const override { return 2; }
|
2023-06-10 12:53:02 +00:00
|
|
|
static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique<ToTimeZoneOverloadResolver>(context); }
|
|
|
|
explicit ToTimeZoneOverloadResolver(ContextPtr context)
|
|
|
|
: allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments)
|
|
|
|
{}
|
2021-07-13 05:54:44 +00:00
|
|
|
|
2021-07-14 02:41:24 +00:00
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
2018-09-26 00:31:40 +00:00
|
|
|
{
|
|
|
|
if (arguments.size() != 2)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
|
|
|
"Number of arguments for function {} doesn't match: passed {}, should be 2",
|
|
|
|
getName(), arguments.size());
|
2018-09-26 00:31:40 +00:00
|
|
|
|
2019-10-07 07:45:59 +00:00
|
|
|
const auto which_type = WhichDataType(arguments[0].type);
|
2019-10-08 04:50:13 +00:00
|
|
|
if (!which_type.isDateTime() && !which_type.isDateTime64())
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. "
|
|
|
|
"Should be DateTime or DateTime64", arguments[0].type->getName(), getName());
|
2021-07-13 05:54:44 +00:00
|
|
|
|
2023-06-10 12:53:02 +00:00
|
|
|
String time_zone_name = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, allow_nonconst_timezone_arguments);
|
2023-04-06 11:10:18 +00:00
|
|
|
|
2021-07-14 02:41:24 +00:00
|
|
|
if (which_type.isDateTime())
|
|
|
|
return std::make_shared<DataTypeDateTime>(time_zone_name);
|
|
|
|
|
|
|
|
const auto * date_time64 = assert_cast<const DataTypeDateTime64 *>(arguments[0].type.get());
|
|
|
|
return std::make_shared<DataTypeDateTime64>(date_time64->getScale(), time_zone_name);
|
|
|
|
}
|
|
|
|
|
|
|
|
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override
|
|
|
|
{
|
2021-07-13 05:54:44 +00:00
|
|
|
bool is_constant_timezone = false;
|
|
|
|
if (arguments[1].column)
|
|
|
|
is_constant_timezone = isColumnConst(*arguments[1].column);
|
2018-09-26 00:31:40 +00:00
|
|
|
|
2021-07-13 05:54:44 +00:00
|
|
|
DataTypes data_types(arguments.size());
|
|
|
|
for (size_t i = 0; i < arguments.size(); ++i)
|
|
|
|
data_types[i] = arguments[i].type;
|
|
|
|
|
2021-07-14 02:41:24 +00:00
|
|
|
return std::make_unique<FunctionBaseToTimeZone>(is_constant_timezone, data_types, result_type);
|
2018-09-26 00:31:40 +00:00
|
|
|
}
|
2023-06-10 12:53:02 +00:00
|
|
|
private:
|
|
|
|
const bool allow_nonconst_timezone_arguments;
|
2018-09-26 00:31:40 +00:00
|
|
|
};
|
|
|
|
|
2020-09-07 18:00:37 +00:00
|
|
|
}
|
|
|
|
|
2022-07-04 07:01:39 +00:00
|
|
|
REGISTER_FUNCTION(ToTimeZone)
|
2018-09-26 00:31:40 +00:00
|
|
|
{
|
2021-07-13 05:54:44 +00:00
|
|
|
factory.registerFunction<ToTimeZoneOverloadResolver>();
|
2021-07-14 02:41:24 +00:00
|
|
|
factory.registerAlias("toTimeZone", "toTimezone");
|
2018-09-26 00:31:40 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|