Merge pull request #60644 from ClickHouse/revert-60281-54746_to_millisecond_function

Revert "Add `toMillisecond` function"
This commit is contained in:
Alexander Tokmakov 2024-03-01 14:01:12 +01:00 committed by GitHub
commit 1b042be13b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
8 changed files with 50 additions and 178 deletions

View File

@ -394,7 +394,8 @@ Result:
## toYear
Returns the year component (AD) of a date or date with time.
Converts a date or date with time to the year number (AD) as `UInt16` value.
**Syntax**
@ -430,7 +431,7 @@ Result:
## toQuarter
Returns the quarter (1-4) of a date or date with time.
Converts a date or date with time to the quarter number (1-4) as `UInt8` value.
**Syntax**
@ -464,9 +465,10 @@ Result:
└──────────────────────────────────────────────┘
```
## toMonth
Returns the month component (1-12) of a date or date with time.
Converts a date or date with time to the month number (1-12) as `UInt8` value.
**Syntax**
@ -502,7 +504,7 @@ Result:
## toDayOfYear
Returns the number of the day within the year (1-366) of a date or date with time.
Converts a date or date with time to the number of the day of the year (1-366) as `UInt16` value.
**Syntax**
@ -538,7 +540,7 @@ Result:
## toDayOfMonth
Returns the number of the day within the month (1-31) of a date or date with time.
Converts a date or date with time to the number of the day in the month (1-31) as `UInt8` value.
**Syntax**
@ -574,7 +576,7 @@ Result:
## toDayOfWeek
Returns the number of the day within the week of a date or date with time.
Converts a date or date with time to the number of the day in the week as `UInt8` value.
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.
@ -625,7 +627,7 @@ Result:
## toHour
Returns the hour component (0-24) of a date with time.
Converts a date with time to the number of the hour in 24-hour time (0-23) as `UInt8` value.
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).
@ -639,7 +641,7 @@ Alias: `HOUR`
**Arguments**
- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
**Returned value**
@ -663,7 +665,7 @@ Result:
## toMinute
Returns the minute component (0-59) a date with time.
Converts a date with time to the number of the minute of the hour (0-59) as `UInt8` value.
**Syntax**
@ -675,7 +677,7 @@ Alias: `MINUTE`
**Arguments**
- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
**Returned value**
@ -699,7 +701,7 @@ Result:
## toSecond
Returns the second component (0-59) of a date with time. Leap seconds are not considered.
Converts a date with time to the second in the minute (0-59) as `UInt8` value. Leap seconds are not considered.
**Syntax**
@ -711,7 +713,7 @@ Alias: `SECOND`
**Arguments**
- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
**Returned value**
@ -733,40 +735,6 @@ Result:
└─────────────────────────────────────────────┘
```
## toMillisecond
Returns the millisecond component (0-999) of a date with time.
**Syntax**
```sql
toMillisecond(value)
```
*Arguments**
- `value` - [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
Alias: `MILLISECOND`
```sql
SELECT toMillisecond(toDateTime64('2023-04-21 10:20:30.456', 3))
```
Result:
```response
┌──toMillisecond(toDateTime64('2023-04-21 10:20:30.456', 3))─┐
│ 456 │
└────────────────────────────────────────────────────────────┘
```
**Returned value**
- The millisecond in the minute (0 - 59) of the given date/time
Type: `UInt16`
## toUnixTimestamp
Converts a string, a date or a date with time to the [Unix Timestamp](https://en.wikipedia.org/wiki/Unix_time) in `UInt32` representation.

View File

@ -3,13 +3,13 @@
#include <base/DayNum.h>
#include <base/defines.h>
#include <base/types.h>
#include <Core/DecimalFunctions.h>
#include <ctime>
#include <cassert>
#include <string>
#include <type_traits>
#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.
@ -280,9 +280,9 @@ private:
static_assert(std::is_integral_v<DateOrTime> && std::is_integral_v<Divisor>);
assert(divisor > 0);
if (offset_is_whole_number_of_hours_during_epoch) [[likely]]
if (likely(offset_is_whole_number_of_hours_during_epoch))
{
if (x >= 0) [[likely]]
if (likely(x >= 0))
return static_cast<DateOrTime>(x / divisor * divisor);
/// Integer division for negative numbers rounds them towards zero (up).
@ -576,10 +576,10 @@ public:
unsigned toSecond(Time t) const
{
if (offset_is_whole_number_of_minutes_during_epoch) [[likely]]
if (likely(offset_is_whole_number_of_minutes_during_epoch))
{
Time res = t % 60;
if (res >= 0) [[likely]]
if (likely(res >= 0))
return static_cast<unsigned>(res);
return static_cast<unsigned>(res) + 60;
}
@ -593,30 +593,6 @@ public:
return time % 60;
}
template <typename DateOrTime>
unsigned toMillisecond(const DateOrTime & datetime, Int64 scale_multiplier) const
{
constexpr Int64 millisecond_multiplier = 1'000;
constexpr Int64 microsecond_multiplier = 1'000 * millisecond_multiplier;
constexpr Int64 divider = microsecond_multiplier / millisecond_multiplier;
auto components = DB::DecimalUtils::splitWithScaleMultiplier(datetime, scale_multiplier);
if (datetime.value < 0 && components.fractional)
{
components.fractional = scale_multiplier + (components.whole ? Int64(-1) : Int64(1)) * components.fractional;
--components.whole;
}
Int64 fractional = components.fractional;
if (scale_multiplier > microsecond_multiplier)
fractional = fractional / (scale_multiplier / microsecond_multiplier);
else if (scale_multiplier < microsecond_multiplier)
fractional = fractional * (microsecond_multiplier / scale_multiplier);
UInt16 millisecond = static_cast<UInt16>(fractional / divider);
return millisecond;
}
unsigned toMinute(Time t) const
{
if (t >= 0 && offset_is_whole_number_of_hours_during_epoch)
@ -1146,9 +1122,9 @@ public:
DateOrTime toStartOfMinuteInterval(DateOrTime t, UInt64 minutes) const
{
Int64 divisor = 60 * minutes;
if (offset_is_whole_number_of_minutes_during_epoch) [[likely]]
if (likely(offset_is_whole_number_of_minutes_during_epoch))
{
if (t >= 0) [[likely]]
if (likely(t >= 0))
return static_cast<DateOrTime>(t / divisor * divisor);
return static_cast<DateOrTime>((t + 1 - divisor) / divisor * divisor);
}
@ -1363,7 +1339,7 @@ public:
UInt8 saturateDayOfMonth(Int16 year, UInt8 month, UInt8 day_of_month) const
{
if (day_of_month <= 28) [[likely]]
if (likely(day_of_month <= 28))
return day_of_month;
UInt8 days_in_month = daysInMonth(year, month);

View File

@ -10,17 +10,16 @@ namespace ErrorCodes
void throwDateIsNotSupported(const char * 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 argument of type Date32 for function {}", 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 argument of type DateTime for function {}", name);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type DateTime of argument for function {}", name);
}
void throwDate32IsNotSupported(const char * name)
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date32 of argument for function {}", name);
}
}

View File

@ -6,7 +6,6 @@
#include <Common/DateLUTImpl.h>
#include <Common/DateLUT.h>
#include <Common/IntervalKind.h>
#include "base/Decimal.h"
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnVector.h>
@ -55,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 throwDate32IsNotSupported(const char * name);
[[noreturn]] void throwDateTimeIsNotSupported(const char * name);
[[noreturn]] void throwDate32IsNotSupported(const char * name);
/// This factor transformation will say that the function is monotone everywhere.
struct ZeroTransform
@ -482,7 +481,7 @@ struct ToStartOfInterval<IntervalKind::Kind::Nanosecond>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64)
{
@ -517,7 +516,7 @@ struct ToStartOfInterval<IntervalKind::Kind::Microsecond>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64)
{
@ -560,7 +559,7 @@ struct ToStartOfInterval<IntervalKind::Kind::Millisecond>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64)
{
@ -603,7 +602,7 @@ struct ToStartOfInterval<IntervalKind::Kind::Second>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, Int64 seconds, const DateLUTImpl & time_zone, Int64)
{
@ -624,7 +623,7 @@ struct ToStartOfInterval<IntervalKind::Kind::Minute>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, Int64 minutes, const DateLUTImpl & time_zone, Int64)
{
@ -645,7 +644,7 @@ struct ToStartOfInterval<IntervalKind::Kind::Hour>
}
static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64)
{
throwDate32IsNotSupported(TO_START_OF_INTERVAL_NAME);
throwDateIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static UInt32 execute(UInt32 t, Int64 hours, const DateLUTImpl & time_zone, Int64)
{
@ -778,7 +777,7 @@ struct ToTimeImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -803,7 +802,7 @@ struct ToStartOfMinuteImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -850,7 +849,7 @@ struct ToStartOfSecondImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -898,7 +897,7 @@ struct ToStartOfMillisecondImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -942,7 +941,7 @@ struct ToStartOfMicrosecondImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -980,7 +979,7 @@ struct ToStartOfNanosecondImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -1005,7 +1004,7 @@ struct ToStartOfFiveMinutesImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -1037,7 +1036,7 @@ struct ToStartOfTenMinutesImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -1069,7 +1068,7 @@ struct ToStartOfFifteenMinutesImpl
}
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
{
@ -1104,7 +1103,7 @@ struct TimeSlotImpl
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
@ -1143,7 +1142,7 @@ struct ToStartOfHourImpl
static UInt32 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt32 execute(UInt16, const DateLUTImpl &)
@ -1430,7 +1429,7 @@ struct ToHourImpl
}
static UInt8 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt8 execute(UInt16, const DateLUTImpl &)
{
@ -1457,7 +1456,7 @@ struct TimezoneOffsetImpl
static time_t execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static time_t execute(UInt16, const DateLUTImpl &)
@ -1483,7 +1482,7 @@ struct ToMinuteImpl
}
static UInt8 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt8 execute(UInt16, const DateLUTImpl &)
{
@ -1508,7 +1507,7 @@ struct ToSecondImpl
}
static UInt8 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
throwDateIsNotSupported(name);
}
static UInt8 execute(UInt16, const DateLUTImpl &)
{
@ -1519,32 +1518,6 @@ struct ToSecondImpl
using FactorTransform = ToStartOfMinuteImpl;
};
struct ToMillisecondImpl
{
static constexpr auto name = "toMillisecond";
static UInt16 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl & time_zone)
{
return time_zone.toMillisecond<DateTime64>(datetime64, scale_multiplier);
}
static UInt16 execute(UInt32, const DateLUTImpl &)
{
return 0;
}
static UInt16 execute(Int32, const DateLUTImpl &)
{
throwDate32IsNotSupported(name);
}
static UInt16 execute(UInt16, const DateLUTImpl &)
{
throwDateIsNotSupported(name);
}
static constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
struct ToISOYearImpl
{
static constexpr auto name = "toISOYear";

View File

@ -1,18 +0,0 @@
#include <Functions/FunctionFactory.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionDateOrDateTimeToSomething.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,8 +0,0 @@
2023-04-21 10:20:30 0 0
2023-04-21 10:20:30 0 0
2023-04-21 10:20:30.123 123 123
2023-04-21 10:20:30.123456 123 123
2023-04-21 10:20:30.123456789 123 123
120
2023-04-21 10:20:30 0
2023-04-21 10:20:30 0

View File

@ -1,17 +0,0 @@
-- 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));
SELECT toDateTime64('2023-04-21 10:20:30', 0) AS dt64, toMillisecond(dt64), toMillisecond(materialize(dt64));
SELECT toDateTime64('2023-04-21 10:20:30.123', 3) AS dt64, toMillisecond(dt64), toMillisecond(materialize(dt64));
SELECT toDateTime64('2023-04-21 10:20:30.123456', 6) AS dt64, toMillisecond(dt64), toMillisecond(materialize(dt64));
SELECT toDateTime64('2023-04-21 10:20:30.123456789', 9) AS dt64, toMillisecond(dt64), toMillisecond(materialize(dt64));
-- 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

View File

@ -2542,7 +2542,6 @@ toRelativeSecondNum
toRelativeWeekNum
toRelativeYearNum
toSecond
toMillisecond
toStartOfDay
toStartOfFifteenMinutes
toStartOfFiveMinutes