Some fixups

This commit is contained in:
Robert Schulze 2024-02-28 11:14:53 +00:00
parent abbe596a2d
commit 613e426484
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
7 changed files with 97 additions and 57 deletions

View File

@ -394,8 +394,7 @@ Result:
## toYear
Converts a date or date with time to the year number (AD) as `UInt16` value.
Returns the year component (AD) of a date or date with time.
**Syntax**
@ -431,7 +430,7 @@ Result:
## toQuarter
Converts a date or date with time to the quarter number (1-4) as `UInt8` value.
Returns the quarter (1-4) of a date or date with time.
**Syntax**
@ -465,10 +464,9 @@ Result:
└──────────────────────────────────────────────┘
```
## toMonth
Converts a date or date with time to the month number (1-12) as `UInt8` value.
Returns the month component (1-12) of a date or date with time.
**Syntax**
@ -504,7 +502,7 @@ Result:
## toDayOfYear
Converts a date or date with time to the number of the day of the year (1-366) as `UInt16` value.
Returns the number of the day within the year (1-366) of a date or date with time.
**Syntax**
@ -540,7 +538,7 @@ Result:
## toDayOfMonth
Converts a date or date with time to the number of the day in the month (1-31) as `UInt8` value.
Returns the number of the day within the month (1-31) of a date or date with time.
**Syntax**
@ -576,7 +574,7 @@ Result:
## toDayOfWeek
Converts a date or date with time to the number of the day in the week as `UInt8` value.
Returns the number of the day within the week of a date or date with time.
The two-argument form of `toDayOfWeek()` enables you to specify whether the week starts on Monday or Sunday, and whether the return value should be in the range from 0 to 6 or 1 to 7. If the mode argument is omitted, the default mode is 0. The time zone of the date can be specified as the third argument.
@ -627,7 +625,7 @@ Result:
## toHour
Converts a date with time to the number of the hour in 24-hour time (0-23) as `UInt8` value.
Returns the hour component (0-24) of a date with time.
Assumes that if clocks are moved ahead, it is by one hour and occurs at 2 a.m., and if clocks are moved back, it is by one hour and occurs at 3 a.m. (which is not always exactly when it occurs - it depends on the timezone).
@ -641,7 +639,7 @@ Alias: `HOUR`
**Arguments**
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
**Returned value**
@ -665,7 +663,7 @@ Result:
## toMinute
Converts a date with time to the number of the minute of the hour (0-59) as `UInt8` value.
Returns the minute componentn (0-59) a date with time.
**Syntax**
@ -677,7 +675,7 @@ Alias: `MINUTE`
**Arguments**
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
**Returned value**
@ -701,7 +699,7 @@ Result:
## toSecond
Converts a date with time to the second in the minute (0-59) as `UInt8` value. Leap seconds are not considered.
Returns the second component (0-59) of a date with time. Leap seconds are not considered.
**Syntax**
@ -713,7 +711,7 @@ Alias: `SECOND`
**Arguments**
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
**Returned value**
@ -737,7 +735,7 @@ Result:
## toMillisecond
Similar to the `toSecond` function, it converts a date with time to the milliseconds in the minute of the time as `UInt16` value.
Returns the millisecond component (0-999) of a date with time.
**Syntax**
@ -747,7 +745,7 @@ toMillisecond(value)
*Arguments**
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
Alias: `MILLISECOND`

View File

@ -9,7 +9,6 @@
#include <string>
#include <type_traits>
#include <iostream>
#define DATE_SECONDS_PER_DAY 86400 /// Number of seconds in a day, 60 * 60 * 24
#define DATE_LUT_MIN_YEAR 1900 /// 1900 since majority of financial organizations consider 1900 as an initial year.
@ -281,9 +280,9 @@ private:
static_assert(std::is_integral_v<DateOrTime> && std::is_integral_v<Divisor>);
assert(divisor > 0);
if (likely(offset_is_whole_number_of_hours_during_epoch))
if (offset_is_whole_number_of_hours_during_epoch) [[likely]]
{
if (likely(x >= 0))
if (x >= 0) [[likely]]
return static_cast<DateOrTime>(x / divisor * divisor);
/// Integer division for negative numbers rounds them towards zero (up).
@ -577,10 +576,10 @@ public:
unsigned toSecond(Time t) const
{
if (likely(offset_is_whole_number_of_minutes_during_epoch))
if (offset_is_whole_number_of_minutes_during_epoch) [[likely]]
{
Time res = t % 60;
if (likely(res >= 0))
if (res >= 0) [[likely]]
return static_cast<unsigned>(res);
return static_cast<unsigned>(res) + 60;
}
@ -597,14 +596,14 @@ public:
unsigned toMillisecond(Time t) const
{
if (t >= 0 && offset_is_whole_number_of_hours_during_epoch)
return (t%60)*1000;
return (t % 60) * 1000;
LUTIndex index = findIndex(t);
Time time = t - lut[index].date;
if (time >= lut[index].time_at_offset_change())
time += lut[index].amount_of_offset_change();
return (time%60)*1000;
return (time % 60) * 1000;
}
unsigned toMinute(Time t) const
@ -1136,9 +1135,9 @@ public:
DateOrTime toStartOfMinuteInterval(DateOrTime t, UInt64 minutes) const
{
Int64 divisor = 60 * minutes;
if (likely(offset_is_whole_number_of_minutes_during_epoch))
if (offset_is_whole_number_of_minutes_during_epoch) [[likely]]
{
if (likely(t >= 0))
if (t >= 0) [[likely]]
return static_cast<DateOrTime>(t / divisor * divisor);
return static_cast<DateOrTime>((t + 1 - divisor) / divisor * divisor);
}
@ -1353,7 +1352,7 @@ public:
UInt8 saturateDayOfMonth(Int16 year, UInt8 month, UInt8 day_of_month) const
{
if (likely(day_of_month <= 28))
if (day_of_month <= 28) [[likely]]
return day_of_month;
UInt8 days_in_month = daysInMonth(year, month);

View File

@ -10,16 +10,17 @@ namespace ErrorCodes
void throwDateIsNotSupported(const char * name)
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", name);
}
void throwDateTimeIsNotSupported(const char * name)
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type DateTime of argument for function {}", name);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument of type Date for function {}", name);
}
void throwDate32IsNotSupported(const char * name)
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date32 of argument for function {}", name);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument of type Date32 for function {}", name);
}
void throwDateTimeIsNotSupported(const char * name)
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument of type DateTime for function {}", name);
}
}

View File

@ -54,8 +54,8 @@ constexpr time_t MAX_DATE_TIMESTAMP = 5662310399; // 2149-06-06 23:59:59 U
constexpr time_t MAX_DATETIME_DAY_NUM = 49710; // 2106-02-07
[[noreturn]] void throwDateIsNotSupported(const char * name);
[[noreturn]] void throwDateTimeIsNotSupported(const char * name);
[[noreturn]] void throwDate32IsNotSupported(const char * name);
[[noreturn]] void throwDateTimeIsNotSupported(const char * name);
/// This factor transformation will say that the function is monotone everywhere.
struct ZeroTransform
@ -481,7 +481,7 @@ struct ToStartOfInterval<IntervalKind::Nanosecond>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64)
{
@ -516,7 +516,7 @@ struct ToStartOfInterval<IntervalKind::Microsecond>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64)
{
@ -559,7 +559,7 @@ struct ToStartOfInterval<IntervalKind::Millisecond>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64)
{
@ -602,7 +602,7 @@ struct ToStartOfInterval<IntervalKind::Second>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, Int64 seconds, const DateLUTImpl & time_zone, Int64)
{
@ -623,7 +623,7 @@ struct ToStartOfInterval<IntervalKind::Minute>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, Int64 minutes, const DateLUTImpl & time_zone, Int64)
{
@ -644,7 +644,7 @@ struct ToStartOfInterval<IntervalKind::Hour>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, Int64 hours, const DateLUTImpl & time_zone, Int64)
{
@ -777,7 +777,7 @@ struct ToTimeImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -802,7 +802,7 @@ struct ToStartOfMinuteImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -849,7 +849,7 @@ struct ToStartOfSecondImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -897,7 +897,7 @@ struct ToStartOfMillisecondImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -941,7 +941,7 @@ struct ToStartOfMicrosecondImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -979,7 +979,7 @@ struct ToStartOfNanosecondImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -1004,7 +1004,7 @@ struct ToStartOfFiveMinutesImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -1036,7 +1036,7 @@ struct ToStartOfTenMinutesImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -1068,7 +1068,7 @@ struct ToStartOfFifteenMinutesImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -1103,7 +1103,7 @@ struct TimeSlotImpl
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
@ -1142,7 +1142,7 @@ struct ToStartOfHourImpl
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
@ -1429,7 +1429,7 @@ struct ToHourImpl
}
static UInt8 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt8 execute(UInt16, const DateLUTImpl &)
{
@ -1456,7 +1456,7 @@ struct TimezoneOffsetImpl
static time_t execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static time_t execute(UInt16, const DateLUTImpl &)
@ -1482,7 +1482,7 @@ struct ToMinuteImpl
}
static UInt8 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt8 execute(UInt16, const DateLUTImpl &)
{
@ -1507,7 +1507,7 @@ struct ToSecondImpl
}
static UInt8 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt8 execute(UInt16, const DateLUTImpl &)
{
@ -1532,7 +1532,7 @@ struct ToMillisecondImpl
}
static UInt16 execute(Int32, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
throwDate32IsNotSupported(name);
}
static UInt16 execute(UInt16, const DateLUTImpl &)
{

View File

@ -0,0 +1,20 @@
#include <Functions/FunctionFactory.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionDateOrDateTimeToSomething.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
using FunctionToMillisecond = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToMillisecondImpl>;
REGISTER_FUNCTION(ToMillisecond)
{
factory.registerFunction<FunctionToMillisecond>();
/// MySQL compatibility alias.
factory.registerAlias("MILLISECOND", "toMillisecond", FunctionFactory::CaseInsensitive);
}
}

View File

@ -1 +1,7 @@
2023-04-21 10:20:30 30000 30000
2023-04-21 10:20:30 30000 30000
2023-04-21 10:20:30.123 30000 30000
2023-04-21 10:20:30.123456 30000 30000
30000
2023-04-21 10:20:30 30000
2023-04-21 10:20:30 30000

View File

@ -1 +1,17 @@
SELECT toMillisecond(toDateTime64('2023-04-21 10:20:30.123456', 2));
-- Negative tests
SELECT toMillisecond(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT toMillisecond('string'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toMillisecond(toDate('2024-02-28')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toMillisecond(toDate32('2024-02-28')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- Tests with constant and non-constant arguments
SELECT toDateTime('2023-04-21 10:20:30') AS dt, toMillisecond(dt), toMillisecond(materialize(dt)); -- wrong, needs to return 0
SELECT toDateTime64('2023-04-21 10:20:30', 0) AS dt64, toMillisecond(dt64), toMillisecond(materialize(dt64)); -- wrong, needs to return 0
SELECT toDateTime64('2023-04-21 10:20:30.123', 3) AS dt64, toMillisecond(dt64), toMillisecond(materialize(dt64)); -- wrong, needs to return 123
SELECT toDateTime64('2023-04-21 10:20:30.123456', 6) AS dt64, toMillisecond(dt64), toMillisecond(materialize(dt64)); -- wrong, needs to return 123
SELECT toDateTime64('2023-04-21 10:20:30.123456789', 9) AS dt64, toMillisecond(dt64), toMillisecond(materialize(dt64)); -- wrong, needs to return 123
-- Special cases
SELECT MILLISECOND(toDateTime64('2023-04-21 10:20:30.123456', 2)); -- Alias
SELECT toNullable(toDateTime('2023-04-21 10:20:30')) AS dt, toMillisecond(dt); -- Nullable
SELECT toLowCardinality(toDateTime('2023-04-21 10:20:30')) AS dt, toMillisecond(dt); -- LowCardinality