2022-03-28 16:56:47 +00:00
|
|
|
#include <Functions/IFunction.h>
|
|
|
|
#include <Functions/FunctionFactory.h>
|
2023-05-07 13:06:35 +00:00
|
|
|
#include <Functions/FunctionHelpers.h>
|
2022-03-28 16:56:47 +00:00
|
|
|
#include <DataTypes/DataTypeDate.h>
|
2022-03-28 16:04:12 +00:00
|
|
|
#include <DataTypes/DataTypeDate32.h>
|
2022-04-04 18:12:10 +00:00
|
|
|
#include <DataTypes/DataTypeDateTime.h>
|
|
|
|
#include <DataTypes/DataTypeDateTime64.h>
|
2022-03-28 16:56:47 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2022-04-04 17:57:04 +00:00
|
|
|
#include <Columns/ColumnConst.h>
|
|
|
|
#include <Columns/ColumnDecimal.h>
|
2022-10-19 09:13:02 +00:00
|
|
|
#include <Columns/ColumnsDateTime.h>
|
2022-03-28 16:56:47 +00:00
|
|
|
#include <Columns/ColumnsNumber.h>
|
2022-03-28 16:04:12 +00:00
|
|
|
#include <Interpreters/castColumn.h>
|
2022-03-28 16:56:47 +00:00
|
|
|
|
|
|
|
#include <Common/DateLUT.h>
|
|
|
|
#include <Common/typeid_cast.h>
|
|
|
|
|
|
|
|
#include <array>
|
2023-09-12 11:10:17 +00:00
|
|
|
#include <cmath>
|
2022-03-28 16:56:47 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2023-09-12 16:28:08 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
2022-04-04 17:57:04 +00:00
|
|
|
extern const int ARGUMENT_OUT_OF_BOUND;
|
2023-09-12 16:28:08 +00:00
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
2022-03-28 16:56:47 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
2023-09-11 12:40:49 +00:00
|
|
|
/// Functionality common to
|
|
|
|
/// - makeDate, makeDate32, makeDateTime, makeDateTime64,
|
|
|
|
/// - YYYYMMDDToDate, YYYYMMDDToDate32, YYYYMMDDhhmmssToDateTime, YYYYMMDDhhmmssToDateTime64
|
2022-04-04 17:57:04 +00:00
|
|
|
class FunctionWithNumericParamsBase : public IFunction
|
2022-03-28 16:56:47 +00:00
|
|
|
{
|
|
|
|
public:
|
|
|
|
bool isInjective(const ColumnsWithTypeAndName &) const override
|
|
|
|
{
|
2022-04-04 17:57:04 +00:00
|
|
|
return false; /// invalid argument values and timestamps that are out of supported range are converted into a default value
|
2022-03-28 16:56:47 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
|
|
|
|
|
|
|
bool useDefaultImplementationForNulls() const override { return true; }
|
|
|
|
|
|
|
|
bool useDefaultImplementationForConstants() const override { return true; }
|
|
|
|
|
2023-05-06 18:50:25 +00:00
|
|
|
bool isVariadic() const override { return true; }
|
2022-03-28 16:56:47 +00:00
|
|
|
|
2023-05-06 18:50:25 +00:00
|
|
|
size_t getNumberOfArguments() const override { return 0; }
|
2022-03-28 16:56:47 +00:00
|
|
|
|
2023-05-06 18:50:25 +00:00
|
|
|
protected:
|
2023-09-11 12:40:49 +00:00
|
|
|
template <class DataType = DataTypeFloat32, class ArgumentNames>
|
2023-05-07 13:06:35 +00:00
|
|
|
Columns convertMandatoryArguments(const ColumnsWithTypeAndName & arguments, const ArgumentNames & argument_names) const
|
|
|
|
{
|
|
|
|
Columns converted_arguments;
|
2023-09-11 12:40:49 +00:00
|
|
|
const DataTypePtr converted_argument_type = std::make_shared<DataType>();
|
2023-05-07 13:06:35 +00:00
|
|
|
for (size_t i = 0; i < argument_names.size(); ++i)
|
|
|
|
{
|
|
|
|
ColumnPtr argument_column = castColumn(arguments[i], converted_argument_type);
|
|
|
|
argument_column = argument_column->convertToFullColumnIfConst();
|
|
|
|
converted_arguments.push_back(argument_column);
|
|
|
|
}
|
|
|
|
return converted_arguments;
|
|
|
|
}
|
2022-04-04 17:57:04 +00:00
|
|
|
};
|
|
|
|
|
2023-09-11 12:40:49 +00:00
|
|
|
/// Implementation of makeDate, makeDate32
|
2022-04-04 17:57:04 +00:00
|
|
|
template <typename Traits>
|
|
|
|
class FunctionMakeDate : public FunctionWithNumericParamsBase
|
|
|
|
{
|
|
|
|
private:
|
2023-05-07 13:06:35 +00:00
|
|
|
static constexpr std::array mandatory_argument_names_year_month_day = {"year", "month", "day"};
|
|
|
|
static constexpr std::array mandatory_argument_names_year_dayofyear = {"year", "dayofyear"};
|
2022-04-04 17:57:04 +00:00
|
|
|
|
|
|
|
public:
|
2023-09-11 12:40:49 +00:00
|
|
|
static constexpr auto name = Traits::makeDateName;
|
2022-04-04 17:57:04 +00:00
|
|
|
|
|
|
|
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMakeDate>(); }
|
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
|
|
|
{
|
2023-09-11 12:40:49 +00:00
|
|
|
const bool is_year_month_variant = (arguments.size() == 3);
|
2023-05-06 18:50:25 +00:00
|
|
|
|
2023-09-11 12:40:49 +00:00
|
|
|
if (is_year_month_variant)
|
2023-05-06 18:50:25 +00:00
|
|
|
{
|
2023-05-07 13:06:35 +00:00
|
|
|
FunctionArgumentDescriptors args{
|
|
|
|
{mandatory_argument_names_year_month_day[0], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names_year_month_day[1], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names_year_month_day[2], &isNumber<IDataType>, nullptr, "Number"}
|
|
|
|
};
|
|
|
|
validateFunctionArgumentTypes(*this, arguments, args);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
FunctionArgumentDescriptors args{
|
|
|
|
{mandatory_argument_names_year_dayofyear[0], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names_year_dayofyear[1], &isNumber<IDataType>, nullptr, "Number"}
|
|
|
|
};
|
|
|
|
validateFunctionArgumentTypes(*this, arguments, args);
|
2023-05-06 18:50:25 +00:00
|
|
|
}
|
2022-04-04 17:57:04 +00:00
|
|
|
|
|
|
|
return std::make_shared<typename Traits::ReturnDataType>();
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
|
|
|
{
|
2023-09-11 12:40:49 +00:00
|
|
|
const bool is_year_month_day_variant = (arguments.size() == 3);
|
2023-05-07 13:06:35 +00:00
|
|
|
|
2022-04-04 17:57:04 +00:00
|
|
|
Columns converted_arguments;
|
2023-09-11 12:40:49 +00:00
|
|
|
if (is_year_month_day_variant)
|
2023-05-07 13:06:35 +00:00
|
|
|
converted_arguments = convertMandatoryArguments(arguments, mandatory_argument_names_year_month_day);
|
|
|
|
else
|
|
|
|
converted_arguments = convertMandatoryArguments(arguments, mandatory_argument_names_year_dayofyear);
|
2022-03-28 16:56:47 +00:00
|
|
|
|
2023-04-26 12:17:47 +00:00
|
|
|
auto res_column = Traits::ReturnDataType::ColumnType::create(input_rows_count);
|
2022-03-28 16:56:47 +00:00
|
|
|
auto & result_data = res_column->getData();
|
|
|
|
|
|
|
|
const auto & date_lut = DateLUT::instance();
|
2023-05-06 18:50:25 +00:00
|
|
|
const Int32 max_days_since_epoch = date_lut.makeDayNum(Traits::MAX_DATE[0], Traits::MAX_DATE[1], Traits::MAX_DATE[2]);
|
2022-03-28 16:56:47 +00:00
|
|
|
|
2023-09-11 12:40:49 +00:00
|
|
|
if (is_year_month_day_variant)
|
2022-03-28 16:04:12 +00:00
|
|
|
{
|
2023-05-06 18:50:25 +00:00
|
|
|
const auto & year_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[0]).getData();
|
|
|
|
const auto & month_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[1]).getData();
|
|
|
|
const auto & day_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[2]).getData();
|
2022-03-28 16:56:47 +00:00
|
|
|
|
2023-05-06 18:50:25 +00:00
|
|
|
for (size_t i = 0; i < input_rows_count; ++i)
|
2022-03-28 16:56:47 +00:00
|
|
|
{
|
2023-05-06 18:50:25 +00:00
|
|
|
const auto year = year_data[i];
|
|
|
|
const auto month = month_data[i];
|
|
|
|
const auto day = day_data[i];
|
|
|
|
|
|
|
|
Int32 day_num = 0;
|
|
|
|
|
2023-09-11 12:40:49 +00:00
|
|
|
if (year >= Traits::MIN_YEAR && year <= Traits::MAX_YEAR &&
|
2023-05-06 18:50:25 +00:00
|
|
|
month >= 1 && month <= 12 &&
|
|
|
|
day >= 1 && day <= 31)
|
|
|
|
{
|
|
|
|
Int32 days_since_epoch = date_lut.makeDayNum(static_cast<Int16>(year), static_cast<UInt8>(month), static_cast<UInt8>(day));
|
|
|
|
if (days_since_epoch <= max_days_since_epoch)
|
|
|
|
day_num = days_since_epoch;
|
|
|
|
}
|
|
|
|
|
|
|
|
result_data[i] = day_num;
|
2022-03-28 16:56:47 +00:00
|
|
|
}
|
2023-05-06 18:50:25 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
const auto & year_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[0]).getData();
|
|
|
|
const auto & dayofyear_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[1]).getData();
|
|
|
|
|
|
|
|
for (size_t i = 0; i < input_rows_count; ++i)
|
|
|
|
{
|
|
|
|
const auto year = year_data[i];
|
|
|
|
const auto dayofyear = dayofyear_data[i];
|
|
|
|
|
|
|
|
Int32 day_num = 0;
|
|
|
|
|
2023-09-11 12:40:49 +00:00
|
|
|
if (year >= Traits::MIN_YEAR && year <= Traits::MAX_YEAR &&
|
2023-05-06 18:50:25 +00:00
|
|
|
dayofyear >= 1 && dayofyear <= 365)
|
|
|
|
{
|
|
|
|
Int32 days_since_epoch = date_lut.makeDayNum(static_cast<Int16>(year), 1, 1) + static_cast<Int32>(dayofyear) - 1;
|
|
|
|
if (days_since_epoch <= max_days_since_epoch)
|
|
|
|
day_num = days_since_epoch;
|
|
|
|
}
|
2022-03-28 16:56:47 +00:00
|
|
|
|
2023-05-06 18:50:25 +00:00
|
|
|
result_data[i] = day_num;
|
|
|
|
}
|
2022-03-28 16:56:47 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return res_column;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2023-09-11 12:40:49 +00:00
|
|
|
/// Implementation of YYYYMMDDToDate, YYYYMMDDToDate32
|
|
|
|
template<typename Traits>
|
|
|
|
class FunctionYYYYYMMDDToDate : public FunctionWithNumericParamsBase
|
2022-03-28 16:56:47 +00:00
|
|
|
{
|
2023-09-11 12:40:49 +00:00
|
|
|
private:
|
|
|
|
static constexpr std::array mandatory_argument_names = { "YYYYMMDD" };
|
|
|
|
|
|
|
|
public:
|
|
|
|
static constexpr auto name = Traits::YYYYMMDDName;
|
|
|
|
|
|
|
|
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionYYYYYMMDDToDate>(); }
|
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
|
|
|
bool isVariadic() const override { return false; }
|
|
|
|
size_t getNumberOfArguments() const override { return mandatory_argument_names.size(); }
|
|
|
|
|
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
|
|
|
{
|
|
|
|
FunctionArgumentDescriptors args{
|
|
|
|
{mandatory_argument_names[0], &isNumber<IDataType>, nullptr, "Number"}
|
|
|
|
};
|
|
|
|
|
|
|
|
validateFunctionArgumentTypes(*this, arguments, args);
|
|
|
|
|
|
|
|
return std::make_shared<typename Traits::ReturnDataType>();
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
|
|
|
{
|
|
|
|
Columns converted_arguments = convertMandatoryArguments<DataTypeFloat64>(arguments, mandatory_argument_names);
|
|
|
|
|
|
|
|
auto res_column = Traits::ReturnDataType::ColumnType::create(input_rows_count);
|
|
|
|
auto & result_data = res_column->getData();
|
|
|
|
|
|
|
|
const auto & yyyymmdd_data = typeid_cast<const ColumnFloat64 &>(*converted_arguments[0]).getData();
|
|
|
|
|
|
|
|
const auto & date_lut = DateLUT::instance();
|
|
|
|
const Int32 max_days_since_epoch = date_lut.makeDayNum(Traits::MAX_DATE[0], Traits::MAX_DATE[1], Traits::MAX_DATE[2]);
|
|
|
|
|
|
|
|
for (size_t i = 0; i < input_rows_count; ++i)
|
|
|
|
{
|
2023-09-12 16:28:08 +00:00
|
|
|
if (std::isinf(yyyymmdd_data[i]) || std::isnan(yyyymmdd_data[i])) [[unlikely]]
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument for function {} must be finite", getName());
|
|
|
|
|
2023-09-12 11:10:17 +00:00
|
|
|
const auto yyyymmdd = static_cast<UInt64>(static_cast<Int64>(yyyymmdd_data[i])); /// Float64-to-UInt64 is UB, double-cast avoids it
|
2023-09-11 12:40:49 +00:00
|
|
|
|
|
|
|
const auto year = yyyymmdd / 10'000;
|
|
|
|
const auto month = yyyymmdd / 100 % 100;
|
|
|
|
const auto day = yyyymmdd % 100;
|
|
|
|
|
|
|
|
Int32 day_num = 0;
|
|
|
|
|
|
|
|
if (year >= Traits::MIN_YEAR && year <= Traits::MAX_YEAR &&
|
|
|
|
month >= 1 && month <= 12 &&
|
|
|
|
day >= 1 && day <= 31)
|
|
|
|
{
|
|
|
|
Int32 days_since_epoch = date_lut.makeDayNum(static_cast<Int16>(year), static_cast<UInt8>(month), static_cast<UInt8>(day));
|
|
|
|
if (days_since_epoch <= max_days_since_epoch)
|
|
|
|
day_num = days_since_epoch;
|
|
|
|
}
|
|
|
|
|
|
|
|
result_data[i] = day_num;
|
|
|
|
}
|
|
|
|
|
|
|
|
return res_column;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
struct DateTraits
|
|
|
|
{
|
|
|
|
static constexpr auto makeDateName = "makeDate";
|
|
|
|
static constexpr auto YYYYMMDDName = "YYYYMMDDToDate";
|
2022-03-28 16:56:47 +00:00
|
|
|
using ReturnDataType = DataTypeDate;
|
|
|
|
|
|
|
|
static constexpr auto MIN_YEAR = 1970;
|
|
|
|
static constexpr auto MAX_YEAR = 2149;
|
2023-05-06 18:50:25 +00:00
|
|
|
static constexpr std::array MAX_DATE = {MAX_YEAR, 6, 6};
|
2022-03-28 16:56:47 +00:00
|
|
|
};
|
|
|
|
|
2023-09-11 12:40:49 +00:00
|
|
|
struct Date32Traits
|
2022-03-28 16:56:47 +00:00
|
|
|
{
|
2023-09-11 12:40:49 +00:00
|
|
|
static constexpr auto makeDateName = "makeDate32";
|
|
|
|
static constexpr auto YYYYMMDDName = "YYYYMMDDToDate32";
|
2022-03-28 16:56:47 +00:00
|
|
|
using ReturnDataType = DataTypeDate32;
|
|
|
|
|
2022-07-15 10:33:52 +00:00
|
|
|
static constexpr auto MIN_YEAR = 1900;
|
|
|
|
static constexpr auto MAX_YEAR = 2299;
|
2023-05-06 18:50:25 +00:00
|
|
|
static constexpr std::array MAX_DATE = {MAX_YEAR, 12, 31};
|
2022-03-28 16:56:47 +00:00
|
|
|
};
|
|
|
|
|
2023-09-11 12:40:49 +00:00
|
|
|
/// Functionality common to makeDateTime, makeDateTime64, YYYYMMDDhhmmssToDateTime, YYYYMMDDhhmmssToDateTime64
|
|
|
|
class FunctionDateTimeBase : public FunctionWithNumericParamsBase
|
2022-04-04 18:12:10 +00:00
|
|
|
{
|
|
|
|
protected:
|
2023-09-11 12:40:49 +00:00
|
|
|
static constexpr UInt32 DEFAULT_PRECISION = 3;
|
2022-04-04 18:12:10 +00:00
|
|
|
|
|
|
|
template <typename T>
|
|
|
|
static Int64 dateTime(T year, T month, T day_of_month, T hour, T minute, T second, const DateLUTImpl & lut)
|
|
|
|
{
|
|
|
|
/// Note that hour, minute and second are checked against 99 to behave consistently with parsing DateTime from String
|
|
|
|
/// E.g. "select cast('1984-01-01 99:99:99' as DateTime);" returns "1984-01-05 04:40:39"
|
2023-04-26 12:17:47 +00:00
|
|
|
if (std::isnan(year) || std::isnan(month) || std::isnan(day_of_month) ||
|
2022-04-07 08:20:38 +00:00
|
|
|
std::isnan(hour) || std::isnan(minute) || std::isnan(second) ||
|
|
|
|
year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31 ||
|
2023-04-26 12:17:47 +00:00
|
|
|
hour < 0 || hour > 99 || minute < 0 || minute > 99 || second < 0 || second > 99) [[unlikely]]
|
2022-04-07 08:20:38 +00:00
|
|
|
return minDateTime(lut);
|
2022-04-04 18:12:10 +00:00
|
|
|
|
2023-04-26 12:17:47 +00:00
|
|
|
if (year > DATE_LUT_MAX_YEAR) [[unlikely]]
|
2022-04-07 08:20:38 +00:00
|
|
|
return maxDateTime(lut);
|
2022-04-04 18:12:10 +00:00
|
|
|
|
2022-09-11 02:49:43 +00:00
|
|
|
return lut.makeDateTime(
|
|
|
|
static_cast<Int16>(year), static_cast<UInt8>(month), static_cast<UInt8>(day_of_month),
|
|
|
|
static_cast<UInt8>(hour), static_cast<UInt8>(minute), static_cast<UInt8>(second));
|
2022-04-04 18:12:10 +00:00
|
|
|
}
|
|
|
|
|
2022-04-07 08:20:38 +00:00
|
|
|
static Int64 minDateTime(const DateLUTImpl & lut)
|
|
|
|
{
|
|
|
|
return lut.makeDateTime(DATE_LUT_MIN_YEAR - 1, 1, 1, 0, 0, 0);
|
|
|
|
}
|
|
|
|
|
|
|
|
static Int64 maxDateTime(const DateLUTImpl & lut)
|
|
|
|
{
|
|
|
|
return lut.makeDateTime(DATE_LUT_MAX_YEAR + 1, 1, 1, 23, 59, 59);
|
|
|
|
}
|
|
|
|
|
2022-04-04 18:12:10 +00:00
|
|
|
std::string extractTimezone(const ColumnWithTypeAndName & timezone_argument) const
|
|
|
|
{
|
|
|
|
if (!isStringOrFixedString(timezone_argument.type) || !timezone_argument.column || (timezone_argument.column->size() != 1 && !typeid_cast<const ColumnConst*>(timezone_argument.column.get())))
|
|
|
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
|
|
|
"Argument 'timezone' for function {} must be const string", getName());
|
2023-09-11 12:40:49 +00:00
|
|
|
|
|
|
|
String timezone = timezone_argument.column->getDataAt(0).toString();
|
2022-04-04 18:12:10 +00:00
|
|
|
|
|
|
|
return timezone;
|
|
|
|
}
|
2023-09-11 12:40:49 +00:00
|
|
|
|
|
|
|
UInt32 extractPrecision(const ColumnWithTypeAndName & precision_argument) const
|
|
|
|
{
|
|
|
|
if (!isNumber(precision_argument.type) || !precision_argument.column || (precision_argument.column->size() != 1 && !typeid_cast<const ColumnConst*>(precision_argument.column.get())))
|
|
|
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
|
|
|
"Argument 'precision' for function {} must be constant number", getName());
|
|
|
|
|
|
|
|
Int64 precision = precision_argument.column->getInt(0);
|
|
|
|
|
|
|
|
if (precision < 0 || precision > 9)
|
|
|
|
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND,
|
|
|
|
"Argument 'precision' for function {} must be in range [0, 9]", getName());
|
|
|
|
|
|
|
|
return static_cast<UInt32>(precision);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
class FunctionMakeDateTimeBase : public FunctionDateTimeBase
|
|
|
|
{
|
|
|
|
protected:
|
|
|
|
static constexpr std::array mandatory_argument_names = {"year", "month", "day", "hour", "minute", "second"};
|
2022-04-04 18:12:10 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
/// makeDateTime(year, month, day, hour, minute, second, [timezone])
|
|
|
|
class FunctionMakeDateTime : public FunctionMakeDateTimeBase
|
|
|
|
{
|
|
|
|
private:
|
2023-05-06 18:10:17 +00:00
|
|
|
static constexpr std::array optional_argument_names = {"timezone"};
|
2022-04-04 18:12:10 +00:00
|
|
|
|
|
|
|
public:
|
|
|
|
static constexpr auto name = "makeDateTime";
|
|
|
|
|
|
|
|
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMakeDateTime>(); }
|
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
|
|
|
{
|
2023-05-07 13:06:35 +00:00
|
|
|
FunctionArgumentDescriptors mandatory_args{
|
|
|
|
{mandatory_argument_names[0], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names[1], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names[2], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names[3], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names[4], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names[5], &isNumber<IDataType>, nullptr, "Number"}
|
|
|
|
};
|
|
|
|
|
|
|
|
FunctionArgumentDescriptors optional_args{
|
2023-09-11 12:40:49 +00:00
|
|
|
{optional_argument_names[0], &isString<IDataType>, isColumnConst, "const String"}
|
2023-05-07 13:06:35 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args);
|
2022-04-04 18:12:10 +00:00
|
|
|
|
|
|
|
/// Optional timezone argument
|
|
|
|
std::string timezone;
|
2023-05-07 13:06:35 +00:00
|
|
|
if (arguments.size() == mandatory_argument_names.size() + 1)
|
2022-04-04 18:12:10 +00:00
|
|
|
timezone = extractTimezone(arguments.back());
|
|
|
|
|
|
|
|
return std::make_shared<DataTypeDateTime>(timezone);
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
|
|
|
{
|
|
|
|
/// Optional timezone argument
|
|
|
|
std::string timezone;
|
2023-05-07 13:06:35 +00:00
|
|
|
if (arguments.size() == mandatory_argument_names.size() + 1)
|
2022-04-04 18:12:10 +00:00
|
|
|
timezone = extractTimezone(arguments.back());
|
|
|
|
|
2023-05-07 13:06:35 +00:00
|
|
|
Columns converted_arguments = convertMandatoryArguments(arguments, mandatory_argument_names);
|
2022-04-04 18:12:10 +00:00
|
|
|
|
2022-10-19 09:13:02 +00:00
|
|
|
auto res_column = ColumnDateTime::create(input_rows_count);
|
2022-04-04 18:12:10 +00:00
|
|
|
auto & result_data = res_column->getData();
|
|
|
|
|
|
|
|
const auto & year_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[0]).getData();
|
|
|
|
const auto & month_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[1]).getData();
|
|
|
|
const auto & day_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[2]).getData();
|
|
|
|
const auto & hour_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[3]).getData();
|
|
|
|
const auto & minute_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[4]).getData();
|
|
|
|
const auto & second_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[5]).getData();
|
|
|
|
|
|
|
|
const auto & date_lut = DateLUT::instance(timezone);
|
|
|
|
|
|
|
|
for (size_t i = 0; i < input_rows_count; ++i)
|
|
|
|
{
|
|
|
|
const auto year = year_data[i];
|
|
|
|
const auto month = month_data[i];
|
|
|
|
const auto day = day_data[i];
|
|
|
|
const auto hour = hour_data[i];
|
|
|
|
const auto minute = minute_data[i];
|
|
|
|
const auto second = second_data[i];
|
|
|
|
|
|
|
|
auto date_time = dateTime(year, month, day, hour, minute, second, date_lut);
|
2023-04-26 12:17:47 +00:00
|
|
|
if (date_time < 0) [[unlikely]]
|
2022-04-04 18:12:10 +00:00
|
|
|
date_time = 0;
|
2023-04-26 12:17:47 +00:00
|
|
|
else if (date_time > 0x0ffffffffll) [[unlikely]]
|
2022-04-04 18:12:10 +00:00
|
|
|
date_time = 0x0ffffffffll;
|
|
|
|
|
2022-10-07 10:46:45 +00:00
|
|
|
result_data[i] = static_cast<UInt32>(date_time);
|
2022-04-04 18:12:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return res_column;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2023-05-07 13:06:35 +00:00
|
|
|
/// makeDateTime64(year, month, day, hour, minute, second[, fraction[, precision[, timezone]]])
|
2022-04-04 18:12:10 +00:00
|
|
|
class FunctionMakeDateTime64 : public FunctionMakeDateTimeBase
|
|
|
|
{
|
|
|
|
private:
|
2023-05-06 18:10:17 +00:00
|
|
|
static constexpr std::array optional_argument_names = {"fraction", "precision", "timezone"};
|
2022-04-04 18:12:10 +00:00
|
|
|
|
|
|
|
public:
|
|
|
|
static constexpr auto name = "makeDateTime64";
|
|
|
|
|
|
|
|
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMakeDateTime64>(); }
|
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
|
|
|
{
|
2023-05-07 13:06:35 +00:00
|
|
|
FunctionArgumentDescriptors mandatory_args{
|
|
|
|
{mandatory_argument_names[0], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names[1], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names[2], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names[3], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names[4], &isNumber<IDataType>, nullptr, "Number"},
|
|
|
|
{mandatory_argument_names[5], &isNumber<IDataType>, nullptr, "Number"}
|
|
|
|
};
|
|
|
|
|
|
|
|
FunctionArgumentDescriptors optional_args{
|
2023-09-11 12:40:49 +00:00
|
|
|
{optional_argument_names[0], &isNumber<IDataType>, isColumnConst, "const Number"},
|
|
|
|
{optional_argument_names[1], &isNumber<IDataType>, isColumnConst, "const Number"},
|
|
|
|
{optional_argument_names[2], &isString<IDataType>, isColumnConst, "const String"}
|
2023-05-07 13:06:35 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args);
|
|
|
|
|
|
|
|
if (arguments.size() >= mandatory_argument_names.size() + 1)
|
2022-04-04 18:12:10 +00:00
|
|
|
{
|
2023-05-07 13:06:35 +00:00
|
|
|
const auto& fraction_argument = arguments[mandatory_argument_names.size()];
|
2022-04-04 18:12:10 +00:00
|
|
|
if (!isNumber(fraction_argument.type))
|
|
|
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
2022-08-17 19:09:56 +00:00
|
|
|
"Argument 'fraction' for function {} must be a number", getName());
|
2022-04-04 18:12:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Optional precision argument
|
|
|
|
Int64 precision = DEFAULT_PRECISION;
|
2023-05-07 13:06:35 +00:00
|
|
|
if (arguments.size() >= mandatory_argument_names.size() + 2)
|
|
|
|
precision = extractPrecision(arguments[mandatory_argument_names.size() + 1]);
|
2022-04-04 18:12:10 +00:00
|
|
|
|
|
|
|
/// Optional timezone argument
|
|
|
|
std::string timezone;
|
2023-05-07 13:06:35 +00:00
|
|
|
if (arguments.size() == mandatory_argument_names.size() + 3)
|
2022-04-04 18:12:10 +00:00
|
|
|
timezone = extractTimezone(arguments.back());
|
|
|
|
|
|
|
|
return std::make_shared<DataTypeDateTime64>(precision, timezone);
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
|
|
|
{
|
|
|
|
/// Optional precision argument
|
|
|
|
Int64 precision = DEFAULT_PRECISION;
|
2023-05-07 13:06:35 +00:00
|
|
|
if (arguments.size() >= mandatory_argument_names.size() + 2)
|
|
|
|
precision = extractPrecision(arguments[mandatory_argument_names.size() + 1]);
|
2022-04-04 18:12:10 +00:00
|
|
|
|
|
|
|
/// Optional timezone argument
|
|
|
|
std::string timezone;
|
2023-05-07 13:06:35 +00:00
|
|
|
if (arguments.size() == mandatory_argument_names.size() + 3)
|
2022-04-04 18:12:10 +00:00
|
|
|
timezone = extractTimezone(arguments.back());
|
|
|
|
|
2023-05-07 13:06:35 +00:00
|
|
|
Columns converted_arguments = convertMandatoryArguments(arguments, mandatory_argument_names);
|
2022-04-04 18:12:10 +00:00
|
|
|
|
|
|
|
/// Optional fraction argument
|
|
|
|
const ColumnVector<Float64>::Container * fraction_data = nullptr;
|
2023-05-07 13:06:35 +00:00
|
|
|
if (arguments.size() >= mandatory_argument_names.size() + 1)
|
2022-04-04 18:12:10 +00:00
|
|
|
{
|
2023-05-07 13:06:35 +00:00
|
|
|
ColumnPtr fraction_column = castColumn(arguments[mandatory_argument_names.size()], std::make_shared<DataTypeFloat64>());
|
2022-04-04 18:12:10 +00:00
|
|
|
fraction_column = fraction_column->convertToFullColumnIfConst();
|
|
|
|
converted_arguments.push_back(fraction_column);
|
|
|
|
fraction_data = &typeid_cast<const ColumnFloat64 &>(*converted_arguments[6]).getData();
|
|
|
|
}
|
|
|
|
|
2022-10-19 09:13:02 +00:00
|
|
|
auto res_column = ColumnDateTime64::create(input_rows_count, static_cast<UInt32>(precision));
|
2022-04-04 18:12:10 +00:00
|
|
|
auto & result_data = res_column->getData();
|
|
|
|
|
|
|
|
const auto & year_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[0]).getData();
|
|
|
|
const auto & month_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[1]).getData();
|
|
|
|
const auto & day_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[2]).getData();
|
|
|
|
const auto & hour_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[3]).getData();
|
|
|
|
const auto & minute_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[4]).getData();
|
|
|
|
const auto & second_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[5]).getData();
|
|
|
|
|
|
|
|
const auto & date_lut = DateLUT::instance(timezone);
|
|
|
|
|
|
|
|
const auto max_fraction = pow(10, precision) - 1;
|
2022-04-07 08:20:38 +00:00
|
|
|
const auto min_date_time = minDateTime(date_lut);
|
|
|
|
const auto max_date_time = maxDateTime(date_lut);
|
2022-04-04 18:12:10 +00:00
|
|
|
|
|
|
|
for (size_t i = 0; i < input_rows_count; ++i)
|
|
|
|
{
|
|
|
|
const auto year = year_data[i];
|
|
|
|
const auto month = month_data[i];
|
|
|
|
const auto day = day_data[i];
|
|
|
|
const auto hour = hour_data[i];
|
|
|
|
const auto minute = minute_data[i];
|
|
|
|
const auto second = second_data[i];
|
|
|
|
|
|
|
|
auto date_time = dateTime(year, month, day, hour, minute, second, date_lut);
|
|
|
|
|
2022-04-07 08:20:38 +00:00
|
|
|
double fraction = 0;
|
2023-04-26 12:17:47 +00:00
|
|
|
if (date_time == min_date_time) [[unlikely]]
|
2022-04-04 18:12:10 +00:00
|
|
|
fraction = 0;
|
2023-04-26 12:17:47 +00:00
|
|
|
else if (date_time == max_date_time) [[unlikely]]
|
2022-09-11 02:49:43 +00:00
|
|
|
fraction = 999999999;
|
2022-04-07 08:20:38 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
fraction = fraction_data ? (*fraction_data)[i] : 0;
|
2023-04-26 12:17:47 +00:00
|
|
|
if (std::isnan(fraction)) [[unlikely]]
|
2022-04-07 08:20:38 +00:00
|
|
|
{
|
|
|
|
date_time = min_date_time;
|
|
|
|
fraction = 0;
|
|
|
|
}
|
2023-04-26 12:17:47 +00:00
|
|
|
else if (fraction < 0) [[unlikely]]
|
2022-04-07 08:20:38 +00:00
|
|
|
fraction = 0;
|
2023-04-26 12:17:47 +00:00
|
|
|
else if (fraction > max_fraction) [[unlikely]]
|
2022-04-07 08:20:38 +00:00
|
|
|
fraction = max_fraction;
|
|
|
|
}
|
2022-04-04 18:12:10 +00:00
|
|
|
|
2022-10-07 10:46:45 +00:00
|
|
|
result_data[i] = DecimalUtils::decimalFromComponents<DateTime64>(
|
|
|
|
date_time,
|
|
|
|
static_cast<Int64>(fraction),
|
|
|
|
static_cast<UInt32>(precision));
|
2022-04-04 18:12:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return res_column;
|
|
|
|
}
|
2023-09-11 12:40:49 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
class FunctionYYYYMMDDhhmmssToDateTimeBase : public FunctionDateTimeBase
|
|
|
|
{
|
|
|
|
protected:
|
|
|
|
static constexpr std::array mandatory_argument_names = { "YYYYMMDDhhmmss" };
|
|
|
|
};
|
2022-04-04 18:12:10 +00:00
|
|
|
|
2023-09-11 12:40:49 +00:00
|
|
|
/// YYYYMMDDhhmmssToDateTime
|
|
|
|
class FunctionYYYYMMDDhhmmssToDateTime : public FunctionYYYYMMDDhhmmssToDateTimeBase
|
|
|
|
{
|
2022-04-04 18:12:10 +00:00
|
|
|
private:
|
2023-09-11 12:40:49 +00:00
|
|
|
static constexpr std::array optional_argument_names = { "timezone" };
|
|
|
|
|
|
|
|
public:
|
|
|
|
static constexpr auto name = "YYYYMMDDhhmmssToDateTime";
|
|
|
|
|
|
|
|
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionYYYYMMDDhhmmssToDateTime>(); }
|
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
2022-04-04 18:12:10 +00:00
|
|
|
{
|
2023-09-11 12:40:49 +00:00
|
|
|
FunctionArgumentDescriptors mandatory_args{
|
|
|
|
{mandatory_argument_names[0], &isNumber<IDataType>, nullptr, "Number"}
|
|
|
|
};
|
2022-04-04 18:12:10 +00:00
|
|
|
|
2023-09-11 12:40:49 +00:00
|
|
|
FunctionArgumentDescriptors optional_args{
|
|
|
|
{optional_argument_names[0], &isString<IDataType>, isColumnConst, "const String"}
|
|
|
|
};
|
|
|
|
|
|
|
|
validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args);
|
|
|
|
|
|
|
|
/// Optional timezone argument
|
|
|
|
std::string timezone;
|
|
|
|
if (arguments.size() == mandatory_argument_names.size() + 1)
|
|
|
|
timezone = extractTimezone(arguments.back());
|
|
|
|
|
|
|
|
return std::make_shared<DataTypeDateTime>(timezone);
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
|
|
|
{
|
|
|
|
std::string timezone;
|
|
|
|
if (arguments.size() == mandatory_argument_names.size() + 1)
|
|
|
|
timezone = extractTimezone(arguments.back());
|
|
|
|
|
|
|
|
Columns converted_arguments = convertMandatoryArguments<DataTypeFloat64>(arguments, mandatory_argument_names);
|
|
|
|
|
|
|
|
auto res_column = ColumnDateTime::create(input_rows_count);
|
|
|
|
auto & result_data = res_column->getData();
|
|
|
|
|
|
|
|
const auto & yyyymmddhhmmss_data = typeid_cast<const ColumnFloat64 &>(*converted_arguments[0]).getData();
|
|
|
|
|
|
|
|
const auto & date_lut = DateLUT::instance(timezone);
|
|
|
|
|
|
|
|
for (size_t i = 0; i < input_rows_count; i++)
|
|
|
|
{
|
2023-09-12 16:28:08 +00:00
|
|
|
if (std::isinf(yyyymmddhhmmss_data[i]) || std::isnan(yyyymmddhhmmss_data[i])) [[unlikely]]
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument for function {} must be finite", getName());
|
|
|
|
|
2023-09-11 12:40:49 +00:00
|
|
|
const auto yyyymmddhhmmss = static_cast<Int64>(yyyymmddhhmmss_data[i]);
|
|
|
|
|
|
|
|
const auto yyyymmdd = yyyymmddhhmmss / 1'000'000;
|
|
|
|
const auto hhmmss = yyyymmddhhmmss % 1'000'000;
|
|
|
|
|
|
|
|
const auto year = yyyymmdd / 10'000;
|
|
|
|
const auto month = yyyymmdd / 100 % 100;
|
|
|
|
const auto day = yyyymmdd % 100;
|
|
|
|
const auto hour = hhmmss / 10'000;
|
|
|
|
const auto minute = hhmmss / 100 % 100;
|
|
|
|
const auto second = hhmmss % 100;
|
|
|
|
|
|
|
|
auto date_time = dateTime(year, month, day, hour, minute, second, date_lut);
|
|
|
|
|
|
|
|
if (date_time < 0) [[unlikely]]
|
|
|
|
date_time = 0;
|
|
|
|
else if (date_time > 0x0ffffffffll) [[unlikely]]
|
|
|
|
date_time = 0x0ffffffffll;
|
|
|
|
|
|
|
|
result_data[i] = static_cast<UInt32>(date_time);
|
|
|
|
}
|
|
|
|
|
|
|
|
return res_column;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
/// YYYYMMDDhhmmssToDateTime64
|
|
|
|
class FunctionYYYYMMDDhhmmssToDateTime64 : public FunctionYYYYMMDDhhmmssToDateTimeBase
|
|
|
|
{
|
|
|
|
private:
|
|
|
|
static constexpr std::array optional_argument_names = { "precision", "timezone" };
|
|
|
|
|
|
|
|
public:
|
|
|
|
static constexpr auto name = "YYYYMMDDhhmmssToDateTime64";
|
|
|
|
|
|
|
|
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionYYYYMMDDhhmmssToDateTime64>(); }
|
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
|
|
|
{
|
|
|
|
FunctionArgumentDescriptors mandatory_args{
|
|
|
|
{mandatory_argument_names[0], &isNumber<IDataType>, nullptr, "Number"}
|
|
|
|
};
|
|
|
|
|
|
|
|
FunctionArgumentDescriptors optional_args{
|
|
|
|
{optional_argument_names[0], &isNumber<IDataType>, isColumnConst, "const Number"},
|
|
|
|
{optional_argument_names[0], &isString<IDataType>, isColumnConst, "const String"}
|
|
|
|
};
|
|
|
|
|
|
|
|
validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args);
|
|
|
|
|
|
|
|
/// Optional precision argument
|
|
|
|
auto precision = DEFAULT_PRECISION;
|
|
|
|
if (arguments.size() >= mandatory_argument_names.size() + 1)
|
|
|
|
precision = extractPrecision(arguments[mandatory_argument_names.size()]);
|
|
|
|
|
|
|
|
/// Optional timezone argument
|
|
|
|
std::string timezone;
|
|
|
|
if (arguments.size() == mandatory_argument_names.size() + 2)
|
|
|
|
timezone = extractTimezone(arguments.back());
|
|
|
|
|
|
|
|
return std::make_shared<DataTypeDateTime64>(precision, timezone);
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
|
|
|
{
|
|
|
|
UInt32 precision = DEFAULT_PRECISION;
|
|
|
|
if (arguments.size() >= mandatory_argument_names.size() + 1)
|
|
|
|
precision = extractPrecision(arguments[mandatory_argument_names.size()]);
|
|
|
|
|
|
|
|
std::string timezone;
|
|
|
|
if (arguments.size() == mandatory_argument_names.size() + 2)
|
|
|
|
timezone = extractTimezone(arguments.back());
|
|
|
|
|
|
|
|
Columns converted_arguments = convertMandatoryArguments<DataTypeFloat64>(arguments, mandatory_argument_names);
|
|
|
|
|
|
|
|
auto res_column = ColumnDateTime64::create(input_rows_count, static_cast<UInt32>(precision));
|
|
|
|
auto & result_data = res_column->getData();
|
|
|
|
|
|
|
|
const auto & yyyymmddhhmmss_data = typeid_cast<const ColumnFloat64 &>(*converted_arguments[0]).getData();
|
|
|
|
|
|
|
|
const auto & date_lut = DateLUT::instance(timezone);
|
|
|
|
|
|
|
|
const auto fraction_pow = common::exp10_i32(precision);
|
|
|
|
|
|
|
|
for (size_t i = 0; i < input_rows_count; i++)
|
|
|
|
{
|
|
|
|
const auto float_date = yyyymmddhhmmss_data[i];
|
|
|
|
|
2023-09-12 16:28:08 +00:00
|
|
|
if (std::isinf(float_date) || std::isnan(float_date)) [[unlikely]]
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument for function {} must be finite", getName());
|
|
|
|
|
2023-09-12 11:10:17 +00:00
|
|
|
const auto yyyymmddhhmmss = static_cast<UInt64>(static_cast<Int64>(float_date)); /// Float64-to-UInt64 is UB, double-cast avoids it
|
2023-09-11 12:40:49 +00:00
|
|
|
|
|
|
|
const auto yyyymmdd = yyyymmddhhmmss / 1'000'000;
|
|
|
|
const auto hhmmss = yyyymmddhhmmss % 1'000'000;
|
|
|
|
|
|
|
|
const auto decimal = float_date - yyyymmddhhmmss;
|
|
|
|
|
|
|
|
const auto year = yyyymmdd / 10'000;
|
|
|
|
const auto month = yyyymmdd / 100 % 100;
|
|
|
|
const auto day = yyyymmdd % 100;
|
|
|
|
const auto hour = hhmmss / 10'000;
|
|
|
|
const auto minute = hhmmss / 100 % 100;
|
|
|
|
const auto second = hhmmss % 100;
|
|
|
|
|
|
|
|
auto date_time = dateTime(year, month, day, hour, minute, second, date_lut);
|
|
|
|
|
2023-09-12 11:10:17 +00:00
|
|
|
auto fraction = std::llround(decimal * fraction_pow);
|
|
|
|
|
|
|
|
result_data[i] = DecimalUtils::decimalFromComponents<DateTime64>(date_time, fraction, precision);
|
2023-09-11 12:40:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return res_column;
|
2022-04-04 18:12:10 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2022-03-28 16:56:47 +00:00
|
|
|
}
|
|
|
|
|
2022-07-04 07:01:39 +00:00
|
|
|
REGISTER_FUNCTION(MakeDate)
|
2022-03-28 16:56:47 +00:00
|
|
|
{
|
2023-09-11 12:40:49 +00:00
|
|
|
factory.registerFunction<FunctionMakeDate<DateTraits>>({}, FunctionFactory::CaseInsensitive);
|
|
|
|
factory.registerFunction<FunctionMakeDate<Date32Traits>>();
|
2022-04-04 18:12:10 +00:00
|
|
|
factory.registerFunction<FunctionMakeDateTime>();
|
|
|
|
factory.registerFunction<FunctionMakeDateTime64>();
|
2023-09-11 12:40:49 +00:00
|
|
|
|
|
|
|
factory.registerFunction<FunctionYYYYYMMDDToDate<DateTraits>>();
|
|
|
|
factory.registerFunction<FunctionYYYYYMMDDToDate<Date32Traits>>();
|
|
|
|
factory.registerFunction<FunctionYYYYMMDDhhmmssToDateTime>();
|
|
|
|
factory.registerFunction<FunctionYYYYMMDDhhmmssToDateTime64>();
|
2022-03-28 16:56:47 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|