Skip to content

Commit

Permalink
Merge pull request #56088 from jh0x/feature-from-days-zero
Browse files Browse the repository at this point in the history
Add function `fromDaysSinceYearZero()`
  • Loading branch information
rschu1ze committed Nov 3, 2023
2 parents 35d7855 + a43647b commit 480e284
Show file tree
Hide file tree
Showing 10 changed files with 260 additions and 11 deletions.
54 changes: 52 additions & 2 deletions docs/en/sql-reference/functions/date-time-functions.md
Expand Up @@ -1557,10 +1557,10 @@ Returns for a given date, the number of days passed since [1 January 0000](https
toDaysSinceYearZero(date[, time_zone])
```

Aliases: `TO_DAYS`

Alias: `TO_DAYS`

**Arguments**

- `date` — The date to calculate the number of days passed since year zero from. [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
- `time_zone` — A String type const value or a expression represent the time zone. [String types](../../sql-reference/data-types/string.md)

Expand All @@ -1584,6 +1584,56 @@ Result:
└────────────────────────────────────────────┘
```

**See Also**

- [fromDaysSinceYearZero](#fromDaysSinceYearZero)

## fromDaysSinceYearZero

Returns for a given number of days passed since [1 January 0000](https://en.wikipedia.org/wiki/Year_zero) the corresponding date in the [proleptic Gregorian calendar defined by ISO 8601](https://en.wikipedia.org/wiki/Gregorian_calendar#Proleptic_Gregorian_calendar). The calculation is the same as in MySQL's [`FROM_DAYS()`](https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_from-days) function.

The result is undefined if it cannot be represented within the bounds of the [Date](../../sql-reference/data-types/date.md) type.

**Syntax**

``` sql
fromDaysSinceYearZero(days)
```

Alias: `FROM_DAYS`

**Arguments**

- `days` — The number of days passed since year zero.

**Returned value**

The date corresponding to the number of days passed since year zero.

Type: [Date](../../sql-reference/data-types/date.md).

**Example**

``` sql
SELECT fromDaysSinceYearZero(739136), fromDaysSinceYearZero(toDaysSinceYearZero(toDate('2023-09-08')));
```

Result:

``` text
┌─fromDaysSinceYearZero(739136)─┬─fromDaysSinceYearZero(toDaysSinceYearZero(toDate('2023-09-08')))─┐
│ 2023-09-08 │ 2023-09-08 │
└───────────────────────────────┴──────────────────────────────────────────────────────────────────┘
```

**See Also**

- [toDaysSinceYearZero](#toDaysSinceYearZero)

## fromDaysSinceYearZero32

Like [fromDaysSinceYearZero](#fromDaysSinceYearZero) but returns a [Date32](../../sql-reference/data-types/date32.md).

## age

Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 microsecond.
Expand Down
2 changes: 2 additions & 0 deletions src/DataTypes/IDataType.h
Expand Up @@ -458,6 +458,8 @@ inline bool isUInt32(const T & data_type) { return WhichDataType(data_type).isUI
template <typename T>
inline bool isUInt64(const T & data_type) { return WhichDataType(data_type).isUInt64(); }
template <typename T>
inline bool isNativeUnsignedInteger(const T & data_type) { return WhichDataType(data_type).isNativeUInt(); }
template <typename T>
inline bool isUnsignedInteger(const T & data_type) { return WhichDataType(data_type).isUInt(); }

template <typename T>
Expand Down
3 changes: 2 additions & 1 deletion src/Functions/DateTimeTransforms.h
Expand Up @@ -1387,10 +1387,11 @@ struct ToDayOfYearImpl
struct ToDaysSinceYearZeroImpl
{
private:
static constexpr auto DAYS_BETWEEN_YEARS_0_AND_1970 = 719'528; /// 01 January, each. Constant taken from Java LocalDate. Consistent with MySQL's TO_DAYS().
static constexpr auto SECONDS_PER_DAY = 60 * 60 * 24;

public:
static constexpr auto DAYS_BETWEEN_YEARS_0_AND_1970 = 719'528; /// 01 January, each. Constant taken from Java LocalDate. Consistent with MySQL's TO_DAYS().

static constexpr auto name = "toDaysSinceYearZero";

static UInt32 execute(Int64 t, const DateLUTImpl & time_zone)
Expand Down
137 changes: 137 additions & 0 deletions src/Functions/fromDaysSinceYearZero.cpp
@@ -0,0 +1,137 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/DateTimeTransforms.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnsDateTime.h>
#include <Columns/ColumnsNumber.h>
#include <Interpreters/castColumn.h>

#include <Common/DateLUT.h>
#include <Common/typeid_cast.h>

#include <array>
#include <cmath>

namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}

namespace
{

struct DateTraits
{
static constexpr auto name = "fromDaysSinceYearZero";
using ReturnDataType = DataTypeDate;
};

struct DateTraits32
{
static constexpr auto name = "fromDaysSinceYearZero32";
using ReturnDataType = DataTypeDate32;
};

template <typename Traits>
class FunctionFromDaysSinceYearZero : public IFunction
{

public:
static constexpr auto name = Traits::name;
using RawReturnType = typename Traits::ReturnDataType::FieldType;

static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionFromDaysSinceYearZero>(); }

String getName() const override { return name; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
bool useDefaultImplementationForConstants() const override { return true; }
size_t getNumberOfArguments() const override { return 1; }

DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
FunctionArgumentDescriptors args{
{"days", &isNativeUnsignedInteger<IDataType>, nullptr, "UInt*"}
};

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
{
auto res_column = Traits::ReturnDataType::ColumnType::create(input_rows_count);
const auto & src_column = arguments[0];

auto try_type = [&]<typename T>(T)
{
using ColVecType = ColumnVector<T>;

if (const ColVecType * col_vec = checkAndGetColumn<ColVecType>(src_column.column.get()))
{
execute<T>(*col_vec, *res_column, input_rows_count);
return true;
}
return false;
};

const bool success = try_type(UInt8{}) || try_type(UInt16{}) || try_type(UInt32{}) || try_type(UInt64{});

if (!success)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column while execute function {}", getName());

return res_column;
}

template <typename T, typename ColVecType, typename ResCol>
void execute(const ColVecType & col, ResCol & result_column, size_t rows_count) const
{
const auto & src_data = col.getData();
auto & dst_data = result_column.getData();
dst_data.resize(rows_count);

using equivalent_integer = typename std::conditional_t<sizeof(T) == 4, UInt32, UInt64>;

for (size_t i = 0; i < rows_count; ++i)
{
auto raw_value = src_data[i];
auto value = static_cast<equivalent_integer>(raw_value);
dst_data[i] = static_cast<RawReturnType>(value - ToDaysSinceYearZeroImpl::DAYS_BETWEEN_YEARS_0_AND_1970);
}
}
};


}

REGISTER_FUNCTION(FromDaysSinceYearZero)
{
factory.registerFunction<FunctionFromDaysSinceYearZero<DateTraits>>(FunctionDocumentation{
.description = R"(
Given the number of days passed since 1 January 0000 in the proleptic Gregorian calendar defined by ISO 8601 return a corresponding date.
The calculation is the same as in MySQL's FROM_DAYS() function.
)",
.examples{{"typical", "SELECT fromDaysSinceYearZero(713569)", "2023-09-08"}},
.categories{"Dates and Times"}});

factory.registerFunction<FunctionFromDaysSinceYearZero<DateTraits32>>(FunctionDocumentation{
.description = R"(
Given the number of days passed since 1 January 0000 in the proleptic Gregorian calendar defined by ISO 8601 return a corresponding date.
The calculation is the same as in MySQL's FROM_DAYS() function.
)",
.examples{{"typical", "SELECT fromDaysSinceYearZero32(713569)", "2023-09-08"}},
.categories{"Dates and Times"}});

factory.registerAlias("FROM_DAYS", FunctionFromDaysSinceYearZero<DateTraits>::name, FunctionFactory::CaseInsensitive);
}

}
8 changes: 0 additions & 8 deletions src/Functions/makeDate.cpp
Expand Up @@ -36,15 +36,8 @@ namespace
class FunctionWithNumericParamsBase : public IFunction
{
public:
bool isInjective(const ColumnsWithTypeAndName &) const override
{
return false; /// invalid argument values and timestamps that are out of supported range are converted into a default value
}

bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }

bool useDefaultImplementationForNulls() const override { return true; }

bool useDefaultImplementationForConstants() const override { return true; }

bool isVariadic() const override { return true; }
Expand Down Expand Up @@ -191,7 +184,6 @@ class FunctionYYYYYMMDDToDate : public FunctionWithNumericParamsBase

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
Expand Down
3 changes: 3 additions & 0 deletions tests/fuzz/all.dict
Expand Up @@ -461,6 +461,9 @@
"FREEZE"
"FROM"
"FROM_BASE64"
"toDaysSinceYearZero"
"fromDaysSinceYearZero"
"fromDaysSinceYearZero32"
"fromModifiedJulianDay"
"fromModifiedJulianDayOrNull"
"FROM_UNIXTIME"
Expand Down
3 changes: 3 additions & 0 deletions tests/fuzz/dictionaries/functions.dict
Expand Up @@ -336,6 +336,9 @@
"arraySort"
"arrayPartialSort"
"arrayPartialReverseSort"
"toDaysSinceYearZero"
"fromDaysSinceYearZero"
"fromDaysSinceYearZero32"
"dumpColumnStructure"
"multiSearchFirstIndex"
"arrayReverseSplit"
Expand Down
22 changes: 22 additions & 0 deletions tests/queries/0_stateless/02907_fromDaysSinceYearZero.reference
@@ -0,0 +1,22 @@
-- negative tests
-- const and non-const arguments
719527 2149-06-06 2149-06-06
719528 1970-01-01 1970-01-01
719529 1970-01-02 1970-01-02
785062 2149-06-05 2149-06-05
785063 2149-06-06 2149-06-06
785064 1970-01-01 1970-01-01
693960 2299-12-31 2299-12-31
693961 1900-01-01 1900-01-01
693962 1900-01-02 1900-01-02
840056 2299-12-30 2299-12-30
840057 2299-12-31 2299-12-31
840058 2299-12-31 2299-12-31
-- integer types != UInt32
255 1974-06-12 2299-12-31
65535 1973-09-29 2299-12-31
719529 1970-01-02 1970-01-02
-- NULL handling
\N \N
-- Alias
1973-10-01
38 changes: 38 additions & 0 deletions tests/queries/0_stateless/02907_fromDaysSinceYearZero.sql
@@ -0,0 +1,38 @@
SET session_timezone = 'Europe/Amsterdam'; -- disable time zone randomization in CI

SELECT '-- negative tests';
SELECT fromDaysSinceYearZero(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT fromDaysSinceYearZero32(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT fromDaysSinceYearZero(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT fromDaysSinceYearZero32(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT fromDaysSinceYearZero('needs a number'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT fromDaysSinceYearZero32('needs a number'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT fromDaysSinceYearZero(-3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT fromDaysSinceYearZero32(-3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }

SELECT '-- const and non-const arguments';

SELECT 719527 AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)); -- outside Date's range
SELECT 719528 AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x));
SELECT 719529 AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x));
SELECT 785062 AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x));
SELECT 785063 AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x));
SELECT 785064 AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero(materialize(x)); -- outside Date's range

SELECT 693960 AS x, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)); -- outside Date32's range
SELECT 693961 AS x, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x));
SELECT 693962 AS x, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x));
SELECT 840056 AS x, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x));
SELECT 840057 AS x, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x));
SELECT 840058 AS x, fromDaysSinceYearZero32(x), fromDaysSinceYearZero32(materialize(x)); -- outside Date32's range

SELECT '-- integer types != UInt32';
SELECT toUInt8(255) AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero32(x); -- outside Date's range for all UInt8-s
SELECT toUInt16(65535) AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero32(x); -- outside Date's range for all UInt16-s
SELECT toUInt64(719529) AS x, fromDaysSinceYearZero(x), fromDaysSinceYearZero32(x); -- something useful

SELECT '-- NULL handling';
SELECT fromDaysSinceYearZero(NULL), fromDaysSinceYearZero32(NULL);

SELECT '-- Alias';
SELECT FROM_DAYS(1);
1 change: 1 addition & 0 deletions utils/check-style/aspell-ignore/en/aspell-dict.txt
Expand Up @@ -1500,6 +1500,7 @@ formated
formatschema
formatter
freezed
fromDaysSinceYearZero
fromModifiedJulianDay
fromModifiedJulianDayOrNull
fromUTCTimestamp
Expand Down

0 comments on commit 480e284

Please sign in to comment.