ClickHouse/src/Functions/toDaysSinceYearZero.cpp

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

92 lines
3.6 KiB
C++
Raw Normal View History

#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include "DataTypes/IDataType.h"
#include "Functions/TransformDateTime64.h"
namespace DB
{
namespace ErrorCodes
{
2023-09-21 14:16:03 +00:00
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{
2023-09-10 12:55:15 +00:00
/** Returns number of days passed since 0000-01-01 */
class FunctionToDaysSinceYearZero : public IFunction
{
using ResultType = DataTypeUInt32;
public:
static constexpr auto name = "toDaysSinceYearZero";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionToDaysSinceYearZero>(context); }
2023-09-21 14:16:03 +00:00
explicit FunctionToDaysSinceYearZero(ContextPtr /*context*/) { }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
FunctionArgumentDescriptors mandatory_args{
2023-09-22 11:52:02 +00:00
{"date", &isDateOrDate32OrDateTimeOrDateTime64<IDataType>, nullptr, "Date or Date32 or DateTime or DateTime64"}
};
validateFunctionArgumentTypes(*this, arguments, mandatory_args);
2023-09-22 11:52:02 +00:00
return std::make_shared<ResultType>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
const IDataType * from_type = arguments[0].type.get();
WhichDataType which(from_type);
if (which.isDate())
2023-09-22 11:52:02 +00:00
return DateTimeTransformImpl<DataTypeDate, ResultType, ToDaysSinceYearZeroImpl>::execute(arguments, result_type, input_rows_count);
else if (which.isDate32())
2023-09-22 11:52:02 +00:00
return DateTimeTransformImpl<DataTypeDate32, ResultType, ToDaysSinceYearZeroImpl>::execute(arguments, result_type, input_rows_count);
else if (which.isDateTime())
2023-09-22 11:52:02 +00:00
return DateTimeTransformImpl<DataTypeDateTime, ResultType, ToDaysSinceYearZeroImpl>::execute(arguments, result_type, input_rows_count);
else if (which.isDateTime64())
{
const auto scale = static_cast<const DataTypeDateTime64 *>(from_type)->getScale();
2023-09-22 11:52:02 +00:00
const TransformDateTime64<ToDaysSinceYearZeroImpl> transformer(scale);
return DateTimeTransformImpl<DataTypeDateTime64, ResultType, decltype(transformer)>::execute(arguments, result_type, input_rows_count, transformer);
}
2023-09-22 11:52:02 +00:00
throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}",
2023-09-21 14:16:03 +00:00
arguments[0].type->getName(),
this->getName());
}
};
}
REGISTER_FUNCTION(ToDaysSinceYearZero)
{
2023-09-21 14:16:03 +00:00
factory.registerFunction<FunctionToDaysSinceYearZero>(FunctionDocumentation{
.description = R"(
2023-09-22 11:52:02 +00:00
Returns for a given date or date with time, the number of days passed since 1 January 0000 in the proleptic Gregorian calendar defined by ISO 8601.
2023-09-10 12:55:15 +00:00
The calculation is the same as in MySQL's TO_DAYS() function.
)",
2023-09-21 14:16:03 +00:00
.examples{{"typical", "SELECT toDaysSinceYearZero(toDate('2023-09-08'))", "713569"}},
.categories{"Dates and Times"}});
2023-09-10 12:55:15 +00:00
/// MySQL compatibility alias.
factory.registerAlias("TO_DAYS", FunctionToDaysSinceYearZero::name, FunctionFactory::CaseInsensitive);
}
}