Support string arguments

This commit is contained in:
Robert Schulze 2023-10-19 21:52:23 +00:00
parent 69a387b778
commit c2215cb5f0
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
6 changed files with 529 additions and 112 deletions

View File

@ -2393,44 +2393,48 @@ Accepts an additional, optional `precision` parameter after the `timezone` param
## addYears, addQuarters, addMonths, addWeeks, addDays, addHours, addMinutes, addSeconds, addMilliseconds, addMicroseconds, addNanoseconds
These functions add units of the interval specified by the function name to a date or a date with time. A date or date with time is returned.
These functions add units of the interval specified by the function name to a date, a date with time or a date or date with time encoded as string. A date or date with time is returned.
Example:
``` sql
WITH
toDate('2018-01-01') AS date,
toDateTime('2018-01-01 00:00:00') AS date_time
toDate('2024-01-01') AS date,
toDateTime('2024-01-01 00:00:00') AS date_time,
'2024-01-01 00:00:00' AS date_time_string
SELECT
addYears(date, 1) AS add_years_with_date,
addYears(date_time, 1) AS add_years_with_date_time
addYears(date_time, 1) AS add_years_with_date_time,
addYears(date_time_string, 1) AS add_years_with_date_time_string
```
``` text
┌─add_years_with_date─┬─add_years_with_date_time─┐
│ 2019-01-01 │ 2019-01-01 00:00:00 │
└─────────────────────┴──────────────────────────┘
┌─add_years_with_date─┬─add_years_with_date_time─┬─add_years_with_date_time_string─
│ 2025-01-01 │ 2025-01-01 00:00:00 │ 2025-01-01 00:00:00.000 │
└─────────────────────┴──────────────────────────┴─────────────────────────────────
```
## subtractYears, subtractQuarters, subtractMonths, subtractWeeks, subtractDays, subtractHours, subtractMinutes, subtractSeconds, subtractMilliseconds, subtractMicroseconds, subtractNanoseconds
These functions subtract units of the interval specified by the function name from a date or a date with time. A date or date with time is returned.
These functions subtract units of the interval specified by the function name from a date, a date with time or a date or date with time encoded as string. A date or date with time is returned.
Example:
``` sql
WITH
toDate('2019-01-01') AS date,
toDateTime('2019-01-01 00:00:00') AS date_time
toDate('2024-01-01') AS date,
toDateTime('2024-01-01 00:00:00') AS date_time,
'2024-01-01 00:00:00' AS date_time_string
SELECT
subtractYears(date, 1) AS subtract_years_with_date,
subtractYears(date_time, 1) AS subtract_years_with_date_time
subtractYears(date_time, 1) AS subtract_years_with_date_time,
subtractYears(date_time_string, 1) AS subtract_years_with_date_time_string
```
``` text
┌─subtract_years_with_date─┬─subtract_years_with_date_time─┐
│ 2018-01-01 │ 2018-01-01 00:00:00 │
└──────────────────────────┴───────────────────────────────┘
┌─subtract_years_with_date─┬─subtract_years_with_date_time─┬─subtract_years_with_date_time_string─
│ 2023-01-01 │ 2023-01-01 00:00:00 │ 2023-01-01 00:00:00.000 │
└──────────────────────────┴───────────────────────────────┴──────────────────────────────────────
```
## timeSlots(StartTime, Duration,\[, Size\])

View File

@ -796,21 +796,21 @@ class FunctionBinaryArithmetic : public IFunction
static FunctionOverloadResolverPtr
getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context)
{
bool first_is_date_or_datetime = isDateOrDate32(type0) || isDateTime(type0) || isDateTime64(type0);
bool second_is_date_or_datetime = isDateOrDate32(type1) || isDateTime(type1) || isDateTime64(type1);
bool first_arg_is_date_or_datetime_or_string = isDateOrDate32OrDateTimeOrDateTime64(type0) || isString(type0);
bool second_arg_is_date_or_datetime_or_string = isDateOrDate32OrDateTimeOrDateTime64(type1) || isString(type1);
/// Exactly one argument must be Date or DateTime
if (first_is_date_or_datetime == second_is_date_or_datetime)
/// Exactly one argument must be Date or DateTime or String
if (first_arg_is_date_or_datetime_or_string == second_arg_is_date_or_datetime_or_string)
return {};
/// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval.
/// Special case when the function is plus or minus, one of arguments is Date or DateTime or String and another is Interval.
/// We construct another function (example: addMonths) and call it.
if constexpr (!is_plus && !is_minus)
return {};
const DataTypePtr & type_time = first_is_date_or_datetime ? type0 : type1;
const DataTypePtr & type_interval = first_is_date_or_datetime ? type1 : type0;
const DataTypePtr & type_time = first_arg_is_date_or_datetime_or_string ? type0 : type1;
const DataTypePtr & type_interval = first_arg_is_date_or_datetime_or_string ? type1 : type0;
bool interval_is_number = isNumber(type_interval);
@ -823,7 +823,7 @@ class FunctionBinaryArithmetic : public IFunction
return {};
}
if (second_is_date_or_datetime && is_minus)
if (second_arg_is_date_or_datetime_or_string && is_minus)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong order of arguments for function {}: "
"argument of type Interval cannot be first", name);
@ -848,11 +848,11 @@ class FunctionBinaryArithmetic : public IFunction
static FunctionOverloadResolverPtr
getFunctionForDateTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context)
{
bool first_is_date_or_datetime = isDateOrDate32(type0) || isDateTime(type0) || isDateTime64(type0);
bool second_is_date_or_datetime = isDateOrDate32(type1) || isDateTime(type1) || isDateTime64(type1);
bool first_arg_is_date_or_datetime = isDateOrDate32OrDateTimeOrDateTime64(type0);
bool second_arg_is_date_or_datetime = isDateOrDate32OrDateTimeOrDateTime64(type1);
/// Exactly one argument must be Date or DateTime
if (first_is_date_or_datetime == second_is_date_or_datetime)
if (first_arg_is_date_or_datetime == second_arg_is_date_or_datetime)
return {};
if (!isTuple(type0) && !isTuple(type1))
@ -863,7 +863,7 @@ class FunctionBinaryArithmetic : public IFunction
if constexpr (!is_plus && !is_minus)
return {};
if (isTuple(type0) && second_is_date_or_datetime && is_minus)
if (isTuple(type0) && second_arg_is_date_or_datetime && is_minus)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong order of arguments for function {}: "
"argument of Tuple type cannot be first", name);
@ -1123,7 +1123,7 @@ class FunctionBinaryArithmetic : public IFunction
ColumnsWithTypeAndName new_arguments = arguments;
/// Interval argument must be second.
if (isDateOrDate32(arguments[1].type) || isDateTime(arguments[1].type) || isDateTime64(arguments[1].type))
if (isDateOrDate32OrDateTimeOrDateTime64(arguments[1].type) || isString(arguments[1].type))
std::swap(new_arguments[0], new_arguments[1]);
/// Change interval argument type to its representation
@ -1511,7 +1511,7 @@ public:
}
}
/// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval.
/// Special case when the function is plus or minus, one of arguments is Date/DateTime/String and another is Interval.
if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1], context))
{
ColumnsWithTypeAndName new_arguments(2);
@ -1520,7 +1520,7 @@ public:
new_arguments[i].type = arguments[i];
/// Interval argument must be second.
if (isDateOrDate32(new_arguments[1].type) || isDateTime(new_arguments[1].type) || isDateTime64(new_arguments[1].type))
if (isDateOrDate32OrDateTimeOrDateTime64(new_arguments[1].type) || isString(new_arguments[1].type))
std::swap(new_arguments[0], new_arguments[1]);
/// Change interval argument to its representation
@ -2104,7 +2104,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A
return executeAggregateAddition(arguments, result_type, input_rows_count);
}
/// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval.
/// Special case when the function is plus or minus, one of arguments is Date/DateTime/String and another is Interval.
if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0].type, arguments[1].type, context))
{
return executeDateTimeIntervalPlusMinus(arguments, result_type, input_rows_count, function_builder);

View File

@ -8,7 +8,9 @@
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Functions/IFunction.h>
@ -16,7 +18,9 @@
#include <Functions/castTypeToEither.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <IO/parseDateTimeBestEffort.h>
namespace DB
@ -47,268 +51,345 @@ struct AddNanosecondsImpl
{
static constexpr auto name = "addNanoseconds";
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale)
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16 scale)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(9 - scale);
return DateTime64(DecimalUtils::multiplyAdd(t.value, multiplier, delta));
}
static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(9);
return DateTime64(DecimalUtils::multiplyAdd(static_cast<Int64>(t), multiplier, delta));
}
static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "addNanoseconds() cannot be used with Date");
}
static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "addNanoseconds() cannot be used with Date32");
}
static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale)
{
ReadBufferFromString buf(s);
DateTime64 t;
parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone);
return execute(t, delta, time_zone, utc_time_zone, scale);
}
};
struct AddMicrosecondsImpl
{
static constexpr auto name = "addMicroseconds";
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale)
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16 scale)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(std::abs(6 - scale));
return DateTime64(scale <= 6
? DecimalUtils::multiplyAdd(t.value, multiplier, delta)
: DecimalUtils::multiplyAdd(delta, multiplier, t.value));
}
static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(6);
return DateTime64(DecimalUtils::multiplyAdd(static_cast<Int64>(t), multiplier, delta));
}
static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "addMicroseconds() cannot be used with Date");
}
static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "addMicroseconds() cannot be used with Date32");
}
static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale)
{
ReadBufferFromString buf(s);
DateTime64 t;
parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone);
return execute(t, delta, time_zone, utc_time_zone, scale);
}
};
struct AddMillisecondsImpl
{
static constexpr auto name = "addMilliseconds";
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale)
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16 scale)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(std::abs(3 - scale));
return DateTime64(scale <= 3
? DecimalUtils::multiplyAdd(t.value, multiplier, delta)
: DecimalUtils::multiplyAdd(delta, multiplier, t.value));
}
static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
Int64 multiplier = DecimalUtils::scaleMultiplier<DateTime64>(3);
return DateTime64(DecimalUtils::multiplyAdd(static_cast<Int64>(t), multiplier, delta));
}
static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "addMilliseconds() cannot be used with Date");
}
static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "addMilliseconds() cannot be used with Date32");
}
static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale)
{
ReadBufferFromString buf(s);
DateTime64 t;
parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone);
return execute(t, delta, time_zone, utc_time_zone, scale);
}
};
struct AddSecondsImpl
{
static constexpr auto name = "addSeconds";
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale)
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16 scale)
{
return DateTime64(DecimalUtils::multiplyAdd(delta, DecimalUtils::scaleMultiplier<DateTime64>(scale), t.value));
}
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
return static_cast<UInt32>(t + delta);
}
static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
// use default datetime64 scale
static_assert(DataTypeDateTime64::default_scale == 3);
return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta) * 1000;
}
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return static_cast<UInt32>(time_zone.fromDayNum(DayNum(d)) + delta);
}
static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale)
{
ReadBufferFromString buf(s);
DateTime64 t;
parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone);
return execute(t, delta, time_zone, utc_time_zone, scale);
}
};
struct AddMinutesImpl
{
static constexpr auto name = "addMinutes";
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale)
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16 scale)
{
return t + 60 * delta * DecimalUtils::scaleMultiplier<DateTime64>(scale);
}
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
return static_cast<UInt32>(t + delta * 60);
}
static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
// use default datetime64 scale
static_assert(DataTypeDateTime64::default_scale == 3);
return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 60) * 1000;
}
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return static_cast<UInt32>(time_zone.fromDayNum(DayNum(d)) + delta * 60);
}
static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale)
{
ReadBufferFromString buf(s);
DateTime64 t;
parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone);
return execute(t, delta, time_zone, utc_time_zone, scale);
}
};
struct AddHoursImpl
{
static constexpr auto name = "addHours";
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale)
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16 scale)
{
return t + 3600 * delta * DecimalUtils::scaleMultiplier<DateTime64>(scale);
}
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
return static_cast<UInt32>(t + delta * 3600);
}
static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
// use default datetime64 scale
static_assert(DataTypeDateTime64::default_scale == 3);
return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 3600) * 1000;
}
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return static_cast<UInt32>(time_zone.fromDayNum(DayNum(d)) + delta * 3600);
}
static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale)
{
ReadBufferFromString buf(s);
DateTime64 t;
parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone);
return execute(t, delta, time_zone, utc_time_zone, scale);
}
};
struct AddDaysImpl
{
static constexpr auto name = "addDays";
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale)
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16 scale)
{
auto multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
auto d = std::div(t, multiplier);
return time_zone.addDays(d.quot, delta) * multiplier + d.rem;
}
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return static_cast<UInt32>(time_zone.addDays(t, delta));
}
static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
return d + delta;
}
static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
return static_cast<Int32>(d + delta);
}
static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale)
{
ReadBufferFromString buf(s);
DateTime64 t;
parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone);
return execute(t, delta, time_zone, utc_time_zone, scale);
}
};
struct AddWeeksImpl
{
static constexpr auto name = "addWeeks";
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale)
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16 scale)
{
auto multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
auto d = std::div(t, multiplier);
return time_zone.addDays(d.quot, delta * 7) * multiplier + d.rem;
}
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return static_cast<UInt32>(time_zone.addWeeks(t, delta));
}
static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
return static_cast<UInt16>(d + delta * 7);
}
static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, UInt16)
static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16)
{
return static_cast<Int32>(d + delta * 7);
}
static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale)
{
ReadBufferFromString buf(s);
DateTime64 t;
parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone);
return execute(t, delta, time_zone, utc_time_zone, scale);
}
};
struct AddMonthsImpl
{
static constexpr auto name = "addMonths";
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale)
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16 scale)
{
auto multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
auto d = std::div(t, multiplier);
return time_zone.addMonths(d.quot, delta) * multiplier + d.rem;
}
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return static_cast<UInt32>(time_zone.addMonths(t, delta));
}
static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return time_zone.addMonths(DayNum(d), delta);
}
static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return time_zone.addMonths(ExtendedDayNum(d), delta);
}
static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale)
{
ReadBufferFromString buf(s);
DateTime64 t;
parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone);
return execute(t, delta, time_zone, utc_time_zone, scale);
}
};
struct AddQuartersImpl
{
static constexpr auto name = "addQuarters";
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale)
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16 scale)
{
auto multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
auto d = std::div(t, multiplier);
return time_zone.addQuarters(d.quot, delta) * multiplier + d.rem;
}
static UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return static_cast<UInt32>(time_zone.addQuarters(t, delta));
}
static UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return time_zone.addQuarters(DayNum(d), delta);
}
static Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return time_zone.addQuarters(ExtendedDayNum(d), delta);
}
static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale)
{
ReadBufferFromString buf(s);
DateTime64 t;
parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone);
return execute(t, delta, time_zone, utc_time_zone, scale);
}
};
struct AddYearsImpl
{
static constexpr auto name = "addYears";
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale)
static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16 scale)
{
auto multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
auto d = std::div(t, multiplier);
return time_zone.addYears(d.quot, delta) * multiplier + d.rem;
}
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return static_cast<UInt32>(time_zone.addYears(t, delta));
}
static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return time_zone.addYears(DayNum(d), delta);
}
static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16)
static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16)
{
return time_zone.addYears(ExtendedDayNum(d), delta);
}
static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale)
{
ReadBufferFromString buf(s);
DateTime64 t;
parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone);
return execute(t, delta, time_zone, utc_time_zone, scale);
}
};
template <typename Transform>
@ -317,10 +398,10 @@ struct SubtractIntervalImpl : public Transform
using Transform::Transform;
template <typename T>
NO_SANITIZE_UNDEFINED auto execute(T t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) const
NO_SANITIZE_UNDEFINED auto execute(T t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) const
{
/// Signed integer overflow is Ok.
return Transform::execute(t, -delta, time_zone, scale);
return Transform::execute(t, -delta, time_zone, utc_time_zone, scale);
}
};
@ -346,40 +427,56 @@ struct Processor
: transform(std::move(transform_))
{}
template <typename FromVectorType, typename ToVectorType>
void NO_INLINE vectorConstant(const FromVectorType & vec_from, ToVectorType & vec_to, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) const
template <typename FromColumnType, typename ToColumnType>
void NO_INLINE vectorConstant(const FromColumnType & col_from, ToColumnType & col_to, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) const
{
size_t size = vec_from.size();
vec_to.resize(size);
static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC");
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(vec_from[i], checkOverflow(delta), time_zone, scale);
if constexpr (std::is_same_v<FromColumnType, ColumnString>)
{
const auto & offsets_from = col_from.getOffsets();
auto & vec_to = col_to.getData();
size_t size = offsets_from.size();
vec_to.resize(size);
for (size_t i = 0 ; i < size; ++i)
{
std::string_view from = col_from.getDataAt(i).toView();
vec_to[i] = transform.execute(from, checkOverflow(delta), time_zone, utc_time_zone, scale);
}
}
else
{
const auto & vec_from = col_from.getData();
auto & vec_to = col_to.getData();
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(vec_from[i], checkOverflow(delta), time_zone, utc_time_zone, scale);
}
}
template <typename FromVectorType, typename ToVectorType>
void vectorVector(const FromVectorType & vec_from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone, UInt16 scale) const
template <typename FromColumnType, typename ToColumnType>
void vectorVector(const FromColumnType & col_from, ToColumnType & col_to, const IColumn & delta, const DateLUTImpl & time_zone, UInt16 scale) const
{
size_t size = vec_from.size();
vec_to.resize(size);
castTypeToEither<
ColumnUInt8, ColumnUInt16, ColumnUInt32, ColumnUInt64,
ColumnInt8, ColumnInt16, ColumnInt32, ColumnInt64,
ColumnFloat32, ColumnFloat64>(
&delta, [&](const auto & column){ vectorVector(vec_from, vec_to, column, time_zone, scale, size); return true; });
&delta, [&](const auto & column){ vectorVector(col_from, col_to, column, time_zone, scale); return true; });
}
template <typename FromType, typename ToVectorType>
void constantVector(const FromType & from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone, UInt16 scale) const
template <typename FromType, typename ToColumnType>
void constantVector(const FromType & from, ToColumnType & col_to, const IColumn & delta, const DateLUTImpl & time_zone, UInt16 scale) const
{
size_t size = delta.size();
vec_to.resize(size);
castTypeToEither<
ColumnUInt8, ColumnUInt16, ColumnUInt32, ColumnUInt64,
ColumnInt8, ColumnInt16, ColumnInt32, ColumnInt64,
ColumnFloat32, ColumnFloat64>(
&delta, [&](const auto & column){ constantVector(from, vec_to, column, time_zone, scale, size); return true; });
&delta, [&](const auto & column){ constantVector(from, col_to, column, time_zone, scale); return true; });
}
private:
@ -392,20 +489,65 @@ private:
throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow");
}
template <typename FromVectorType, typename ToVectorType, typename DeltaColumnType>
template <typename FromColumnType, typename ToColumnType, typename DeltaColumnType>
NO_INLINE NO_SANITIZE_UNDEFINED void vectorVector(
const FromVectorType & vec_from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, UInt16 scale, size_t size) const
const FromColumnType & col_from, ToColumnType & col_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, UInt16 scale) const
{
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(vec_from[i], checkOverflow(delta.getData()[i]), time_zone, scale);
static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC");
if constexpr (std::is_same_v<FromColumnType, ColumnString>)
{
const auto & offsets_from = col_from.getOffsets();
auto & vec_to = col_to.getData();
size_t size = offsets_from.size();
vec_to.resize(size);
for (size_t i = 0 ; i < size; ++i)
{
std::string_view from = col_from.getDataAt(i).toView();
vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone, utc_time_zone, scale);
}
}
else
{
const auto & vec_from = col_from.getData();
auto & vec_to = col_to.getData();
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(vec_from[i], checkOverflow(delta.getData()[i]), time_zone, utc_time_zone, scale);
}
}
template <typename FromType, typename ToVectorType, typename DeltaColumnType>
template <typename FromType, typename ToColumnType, typename DeltaColumnType>
NO_INLINE NO_SANITIZE_UNDEFINED void constantVector(
const FromType & from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, UInt16 scale, size_t size) const
const FromType & from, ToColumnType & col_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, UInt16 scale) const
{
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone, scale);
static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC");
if constexpr (std::is_same_v<FromType, String>)
{
auto & vec_to = col_to.getData();
size_t size = delta.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone, utc_time_zone, scale);
}
else
{
auto & vec_to = col_to.getData();
size_t size = delta.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone, utc_time_zone, scale);
}
}
};
@ -432,15 +574,15 @@ struct DateTimeAddIntervalImpl
if (const auto * sources = checkAndGetColumn<FromColumnType>(&source_column))
{
if (const auto * delta_const_column = typeid_cast<const ColumnConst *>(&delta_column))
processor.vectorConstant(sources->getData(), col_to->getData(), delta_const_column->getInt(0), time_zone, scale);
processor.vectorConstant(*sources, *col_to, delta_const_column->getInt(0), time_zone, scale);
else
processor.vectorVector(sources->getData(), col_to->getData(), delta_column, time_zone, scale);
processor.vectorVector(*sources, *col_to, delta_column, time_zone, scale);
}
else if (const auto * sources_const = checkAndGetColumnConst<FromColumnType>(&source_column))
{
processor.constantVector(
sources_const->template getValue<FromValueType>(),
col_to->getData(), delta_column, time_zone, scale);
*col_to, delta_column, time_zone, scale);
}
else
{
@ -486,15 +628,15 @@ public:
if (arguments.size() == 2)
{
if (!isDateOrDate32OrDateTimeOrDateTime64(arguments[0].type))
if (!isDateOrDate32OrDateTimeOrDateTime64(arguments[0].type) && !isString(arguments[0].type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. "
"Should be a date or a date with time", arguments[0].type->getName(), getName());
"Must be a date, a date with time or a String", arguments[0].type->getName(), getName());
}
else
{
if (!WhichDataType(arguments[0].type).isDateTime())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. "
"Should be a DateTime", arguments[0].type->getName(), getName());
"Must be a DateTime", arguments[0].type->getName(), getName());
if (!WhichDataType(arguments[2].type).isString())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of third argument of function {}. "
@ -516,6 +658,8 @@ public:
return resolveReturnType<DataTypeDateTime>(arguments);
case TypeIndex::DateTime64:
return resolveReturnType<DataTypeDateTime64>(arguments);
case TypeIndex::String:
return resolveReturnType<DataTypeString>(arguments);
default:
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid type of 1st argument of function {}: "
"{}, expected: Date, DateTime or DateTime64.", getName(), arguments[0].type->getName());
@ -525,7 +669,7 @@ public:
/// Helper templates to deduce return type based on argument type, since some overloads may promote or denote types,
/// e.g. addSeconds(Date, 1) => DateTime
template <typename FieldType>
using TransformExecuteReturnType = decltype(std::declval<Transform>().execute(FieldType(), 0, std::declval<DateLUTImpl>(), 0));
using TransformExecuteReturnType = decltype(std::declval<Transform>().execute(FieldType(), 0, std::declval<DateLUTImpl>(), std::declval<DateLUTImpl>(), 0));
// Deduces RETURN DataType from INPUT DataType, based on return type of Transform{}.execute(INPUT_TYPE, UInt64, DateLUTImpl).
// e.g. for Transform-type that has execute()-overload with 'UInt16' input and 'UInt32' return,
@ -588,11 +732,14 @@ public:
return DateTimeAddIntervalImpl<DataTypeDate32, TransformResultDataType<DataTypeDate32>, Transform>::execute(Transform{}, arguments, result_type, 0);
else if (which.isDateTime())
return DateTimeAddIntervalImpl<DataTypeDateTime, TransformResultDataType<DataTypeDateTime>, Transform>::execute(Transform{}, arguments, result_type, 0);
else if (const auto * datetime64_type = assert_cast<const DataTypeDateTime64 *>(from_type))
else if (which.isDateTime64())
{
const auto * datetime64_type = assert_cast<const DataTypeDateTime64 *>(from_type);
auto from_scale = datetime64_type->getScale();
return DateTimeAddIntervalImpl<DataTypeDateTime64, TransformResultDataType<DataTypeDateTime64>, Transform>::execute(Transform{}, arguments, result_type, from_scale);
}
else if (which.isString())
return DateTimeAddIntervalImpl<DataTypeString, DataTypeDateTime64, Transform>::execute(Transform{}, arguments, result_type, 3);
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}", arguments[0].type->getName(), getName());
}

View File

@ -54,13 +54,14 @@ template <typename T>
static FillColumnDescription::StepFunction getStepFunction(
IntervalKind kind, Int64 step, const DateLUTImpl & date_lut, UInt16 scale = DataTypeDateTime64::default_scale)
{
static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC");
switch (kind) // NOLINT(bugprone-switch-missing-default-case)
{
#define DECLARE_CASE(NAME) \
case IntervalKind::NAME: \
return [step, scale, &date_lut](Field & field) { \
field = Add##NAME##sImpl::execute(static_cast<T>(\
field.get<T>()), static_cast<Int32>(step), date_lut, scale); };
field.get<T>()), static_cast<Int32>(step), date_lut, utc_time_zone, scale); };
FOR_EACH_INTERVAL_KIND(DECLARE_CASE)
#undef DECLARE_CASE
@ -154,6 +155,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr &
{
const auto & step_dec = descr.fill_step.get<const DecimalField<Decimal64> &>();
Int64 step = DecimalUtils::convertTo<Int64>(step_dec.getValue(), step_dec.getScale());
static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC");
switch (*descr.step_kind) // NOLINT(bugprone-switch-missing-default-case)
{
@ -162,7 +164,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr &
descr.step_func = [step, &time_zone = date_time64->getTimeZone()](Field & field) \
{ \
auto field_decimal = field.get<DecimalField<DateTime64>>(); \
auto res = Add##NAME##sImpl::execute(field_decimal.getValue(), step, time_zone, field_decimal.getScale()); \
auto res = Add##NAME##sImpl::execute(field_decimal.getValue(), step, time_zone, utc_time_zone, field_decimal.getScale()); \
field = DecimalField(res, field_decimal.getScale()); \
}; \
break;

View File

@ -0,0 +1,125 @@
-- const date, const delta
-- add
2024-10-22 00:00:00.000 2024-10-22 00:00:00.000
2024-01-22 00:00:00.000 2024-01-22 00:00:00.000
2023-11-22 00:00:00.000 2023-11-22 00:00:00.000
2023-10-29 00:00:00.000 2023-10-29 00:00:00.000
2023-10-23 00:00:00.000 2023-10-23 00:00:00.000
2023-10-22 01:00:00.000 2023-10-22 01:00:00.000
2023-10-22 00:01:00.000 2023-10-22 00:01:00.000
2023-10-22 00:00:01.000 2023-10-22 00:00:01.000
2023-10-22 00:00:00.001 2023-10-22 00:00:00.001
2023-10-22 00:00:00.000001 2023-10-22 00:00:00.000001
2023-10-22 00:00:00.000000001 2023-10-22 00:00:00.000000001
-- subtract
2022-10-22 00:00:00.000 2022-10-22 00:00:00.000
2023-07-22 00:00:00.000 2023-07-22 00:00:00.000
2023-09-22 00:00:00.000 2023-09-22 00:00:00.000
2023-10-15 00:00:00.000 2023-10-15 00:00:00.000
2023-10-21 00:00:00.000 2023-10-21 00:00:00.000
2023-10-21 23:00:00.000 2023-10-21 23:00:00.000
2023-10-21 23:59:00.000 2023-10-21 23:59:00.000
2023-10-21 23:59:59.000 2023-10-21 23:59:59.000
2023-10-21 23:59:59.999 2023-10-21 23:59:59.999
2023-10-21 23:59:59.999999 2023-10-21 23:59:59.999999
2023-10-21 23:59:59.999999999 2023-10-21 23:59:59.999999999
-- non-const date, const delta
-- add
2024-10-22 00:00:00.000 2024-10-22 00:00:00.000
2024-01-22 00:00:00.000 2024-01-22 00:00:00.000
2023-11-22 00:00:00.000 2023-11-22 00:00:00.000
2023-10-29 00:00:00.000 2023-10-29 00:00:00.000
2023-10-23 00:00:00.000 2023-10-23 00:00:00.000
2023-10-22 01:00:00.000 2023-10-22 01:00:00.000
2023-10-22 00:01:00.000 2023-10-22 00:01:00.000
2023-10-22 00:00:01.000 2023-10-22 00:00:01.000
2023-10-22 00:00:00.001 2023-10-22 00:00:00.001
2023-10-22 00:00:00.000001 2023-10-22 00:00:00.000001
2023-10-22 00:00:00.000000001 2023-10-22 00:00:00.000000001
-- subtract
2022-10-22 00:00:00.000 2022-10-22 00:00:00.000
2023-07-22 00:00:00.000 2023-07-22 00:00:00.000
2023-09-22 00:00:00.000 2023-09-22 00:00:00.000
2023-10-15 00:00:00.000 2023-10-15 00:00:00.000
2023-10-21 00:00:00.000 2023-10-21 00:00:00.000
2023-10-21 23:00:00.000 2023-10-21 23:00:00.000
2023-10-21 23:59:00.000 2023-10-21 23:59:00.000
2023-10-21 23:59:59.000 2023-10-21 23:59:59.000
2023-10-21 23:59:59.999 2023-10-21 23:59:59.999
2023-10-21 23:59:59.999999 2023-10-21 23:59:59.999999
2023-10-21 23:59:59.999999999 2023-10-21 23:59:59.999999999
-- const date, non-const delta
-- add
2024-10-22 00:00:00.000 2024-10-22 00:00:00.000
2024-01-22 00:00:00.000 2024-01-22 00:00:00.000
2023-11-22 00:00:00.000 2023-11-22 00:00:00.000
2023-10-29 00:00:00.000 2023-10-29 00:00:00.000
2023-10-23 00:00:00.000 2023-10-23 00:00:00.000
2023-10-22 01:00:00.000 2023-10-22 01:00:00.000
2023-10-22 00:01:00.000 2023-10-22 00:01:00.000
2023-10-22 00:00:01.000 2023-10-22 00:00:01.000
2023-10-22 00:00:00.001 2023-10-22 00:00:00.001
2023-10-22 00:00:00.000001 2023-10-22 00:00:00.000001
2023-10-22 00:00:00.000000001 2023-10-22 00:00:00.000000001
-- subtract
2022-10-22 00:00:00.000 2022-10-22 00:00:00.000
2023-07-22 00:00:00.000 2023-07-22 00:00:00.000
2023-09-22 00:00:00.000 2023-09-22 00:00:00.000
2023-10-15 00:00:00.000 2023-10-15 00:00:00.000
2023-10-21 00:00:00.000 2023-10-21 00:00:00.000
2023-10-21 23:00:00.000 2023-10-21 23:00:00.000
2023-10-21 23:59:00.000 2023-10-21 23:59:00.000
2023-10-21 23:59:59.000 2023-10-21 23:59:59.000
2023-10-21 23:59:59.999 2023-10-21 23:59:59.999
2023-10-21 23:59:59.999999 2023-10-21 23:59:59.999999
2023-10-21 23:59:59.999999999 2023-10-21 23:59:59.999999999
-- non-const date, non-const delta
-- add
2024-10-22 00:00:00.000 2024-10-22 00:00:00.000
2024-01-22 00:00:00.000 2024-01-22 00:00:00.000
2023-11-22 00:00:00.000 2023-11-22 00:00:00.000
2023-10-29 00:00:00.000 2023-10-29 00:00:00.000
2023-10-23 00:00:00.000 2023-10-23 00:00:00.000
2023-10-22 01:00:00.000 2023-10-22 01:00:00.000
2023-10-22 00:01:00.000 2023-10-22 00:01:00.000
2023-10-22 00:00:01.000 2023-10-22 00:00:01.000
2023-10-22 00:00:00.001 2023-10-22 00:00:00.001
2023-10-22 00:00:00.000001 2023-10-22 00:00:00.000001
2023-10-22 00:00:00.000000001 2023-10-22 00:00:00.000000001
-- subtract
2022-10-22 00:00:00.000 2022-10-22 00:00:00.000
2023-07-22 00:00:00.000 2023-07-22 00:00:00.000
2023-09-22 00:00:00.000 2023-09-22 00:00:00.000
2023-10-15 00:00:00.000 2023-10-15 00:00:00.000
2023-10-21 00:00:00.000 2023-10-21 00:00:00.000
2023-10-21 23:00:00.000 2023-10-21 23:00:00.000
2023-10-21 23:59:00.000 2023-10-21 23:59:00.000
2023-10-21 23:59:59.000 2023-10-21 23:59:59.000
2023-10-21 23:59:59.999 2023-10-21 23:59:59.999
2023-10-21 23:59:59.999999 2023-10-21 23:59:59.999999
2023-10-21 23:59:59.999999999 2023-10-21 23:59:59.999999999
-- plus operator
-- add
2024-10-23 00:00:00.000
2024-01-23 00:00:00.000
2023-11-23 00:00:00.000
2023-10-30 00:00:00.000
2023-10-24 00:00:00.000
2023-10-23 01:00:00.000
2023-10-23 00:01:00.000
2023-10-23 00:00:01.000
2023-10-23 00:00:00.001
2023-10-23 00:00:00.000001
2023-10-23 00:00:00.000000001
-- subtract
2022-10-23 00:00:00.000
2023-07-23 00:00:00.000
2023-09-23 00:00:00.000
2023-10-16 00:00:00.000
2023-10-22 00:00:00.000
2023-10-22 23:00:00.000
2023-10-22 23:59:00.000
2023-10-22 23:59:59.000
2023-10-22 23:59:59.999
2023-10-22 23:59:59.999999
2023-10-22 23:59:59.999999999

View File

@ -0,0 +1,139 @@
SELECT '-- const date, const delta';
SELECT ' -- add';
SELECT addYears('2023-10-22', 1), addYears('2023-10-22 00:00:00.000', 1);
SELECT addQuarters('2023-10-22', 1), addQuarters('2023-10-22 00:00:00.000', 1);
SELECT addMonths('2023-10-22', 1), addMonths('2023-10-22 00:00:00.000', 1);
SELECT addWeeks('2023-10-22', 1), addWeeks('2023-10-22 00:00:00.000', 1);
SELECT addDays('2023-10-22', 1), addDays('2023-10-22 00:00:00.000', 1);
SELECT addHours('2023-10-22', 1), addHours('2023-10-22 00:00:00.000', 1);
SELECT addMinutes('2023-10-22', 1), addMinutes('2023-10-22 00:00:00.000', 1);
SELECT addSeconds('2023-10-22', 1), addSeconds('2023-10-22 00:00:00.000', 1);
SELECT addMilliseconds('2023-10-22', 1), addMilliseconds('2023-10-22 00:00:00.000', 1);
SELECT addMicroseconds('2023-10-22', 1), addMicroseconds('2023-10-22 00:00:00.000', 1);
SELECT addNanoseconds('2023-10-22', 1), addNanoseconds('2023-10-22 00:00:00.000', 1);
SELECT ' -- subtract';
SELECT subtractYears('2023-10-22', 1), subtractYears('2023-10-22 00:00:00.000', 1);
SELECT subtractQuarters('2023-10-22', 1), subtractQuarters('2023-10-22 00:00:00.000', 1);
SELECT subtractMonths('2023-10-22', 1), subtractMonths('2023-10-22 00:00:00.000', 1);
SELECT subtractWeeks('2023-10-22', 1), subtractWeeks('2023-10-22 00:00:00.000', 1);
SELECT subtractDays('2023-10-22', 1), subtractDays('2023-10-22 00:00:00.000', 1);
SELECT subtractHours('2023-10-22', 1), subtractHours('2023-10-22 00:00:00.000', 1);
SELECT subtractMinutes('2023-10-22', 1), subtractMinutes('2023-10-22 00:00:00.000', 1);
SELECT subtractSeconds('2023-10-22', 1), subtractSeconds('2023-10-22 00:00:00.000', 1);
SELECT subtractMilliseconds('2023-10-22', 1), subtractMilliseconds('2023-10-22 00:00:00.000', 1);
SELECT subtractMicroseconds('2023-10-22', 1), subtractMicroseconds('2023-10-22 00:00:00.000', 1);
SELECT subtractNanoseconds('2023-10-22', 1), subtractNanoseconds('2023-10-22 00:00:00.000', 1);
SELECT '-- non-const date, const delta';
SELECT ' -- add';
SELECT addYears(materialize('2023-10-22'), 1), addYears(materialize('2023-10-22 00:00:00.000'), 1);
SELECT addQuarters(materialize('2023-10-22'), 1), addQuarters(materialize('2023-10-22 00:00:00.000'), 1);
SELECT addMonths(materialize('2023-10-22'), 1), addMonths(materialize('2023-10-22 00:00:00.000'), 1);
SELECT addWeeks(materialize('2023-10-22'), 1), addWeeks(materialize('2023-10-22 00:00:00.000'), 1);
SELECT addDays(materialize('2023-10-22'), 1), addDays(materialize('2023-10-22 00:00:00.000'), 1);
SELECT addHours(materialize('2023-10-22'), 1), addHours(materialize('2023-10-22 00:00:00.000'), 1);
SELECT addMinutes(materialize('2023-10-22'), 1), addMinutes(materialize('2023-10-22 00:00:00.000'), 1);
SELECT addSeconds(materialize('2023-10-22'), 1), addSeconds(materialize('2023-10-22 00:00:00.000'), 1);
SELECT addMilliseconds(materialize('2023-10-22'), 1), addMilliseconds(materialize('2023-10-22 00:00:00.000'), 1);
SELECT addMicroseconds(materialize('2023-10-22'), 1), addMicroseconds(materialize('2023-10-22 00:00:00.000'), 1);
SELECT addNanoseconds(materialize('2023-10-22'), 1), addNanoseconds(materialize('2023-10-22 00:00:00.000'), 1);
SELECT ' -- subtract';
SELECT subtractYears(materialize('2023-10-22'), 1), subtractYears(materialize('2023-10-22 00:00:00.000'), 1);
SELECT subtractQuarters(materialize('2023-10-22'), 1), subtractQuarters(materialize('2023-10-22 00:00:00.000'), 1);
SELECT subtractMonths(materialize('2023-10-22'), 1), subtractMonths(materialize('2023-10-22 00:00:00.000'), 1);
SELECT subtractWeeks(materialize('2023-10-22'), 1), subtractWeeks(materialize('2023-10-22 00:00:00.000'), 1);
SELECT subtractDays(materialize('2023-10-22'), 1), subtractDays(materialize('2023-10-22 00:00:00.000'), 1);
SELECT subtractHours(materialize('2023-10-22'), 1), subtractHours(materialize('2023-10-22 00:00:00.000'), 1);
SELECT subtractMinutes(materialize('2023-10-22'), 1), subtractMinutes(materialize('2023-10-22 00:00:00.000'), 1);
SELECT subtractSeconds(materialize('2023-10-22'), 1), subtractSeconds(materialize('2023-10-22 00:00:00.000'), 1);
SELECT subtractMilliseconds(materialize('2023-10-22'), 1), subtractMilliseconds(materialize('2023-10-22 00:00:00.000'), 1);
SELECT subtractMicroseconds(materialize('2023-10-22'), 1), subtractMicroseconds(materialize('2023-10-22 00:00:00.000'), 1);
SELECT subtractNanoseconds(materialize('2023-10-22'), 1), subtractNanoseconds(materialize('2023-10-22 00:00:00.000'), 1);
SELECT '-- const date, non-const delta';
SELECT ' -- add';
SELECT addYears('2023-10-22', materialize(1)), addYears('2023-10-22 00:00:00.000', materialize(1));
SELECT addQuarters('2023-10-22', materialize(1)), addQuarters('2023-10-22 00:00:00.000', materialize(1));
SELECT addMonths('2023-10-22', materialize(1)), addMonths('2023-10-22 00:00:00.000', materialize(1));
SELECT addWeeks('2023-10-22', materialize(1)), addWeeks('2023-10-22 00:00:00.000', materialize(1));
SELECT addDays('2023-10-22', materialize(1)), addDays('2023-10-22 00:00:00.000', materialize(1));
SELECT addHours('2023-10-22', materialize(1)), addHours('2023-10-22 00:00:00.000', materialize(1));
SELECT addMinutes('2023-10-22', materialize(1)), addMinutes('2023-10-22 00:00:00.000', materialize(1));
SELECT addSeconds('2023-10-22', materialize(1)), addSeconds('2023-10-22 00:00:00.000', materialize(1));
SELECT addMilliseconds('2023-10-22', materialize(1)), addMilliseconds('2023-10-22 00:00:00.000', materialize(1));
SELECT addMicroseconds('2023-10-22', materialize(1)), addMicroseconds('2023-10-22 00:00:00.000', materialize(1));
SELECT addNanoseconds('2023-10-22', materialize(1)), addNanoseconds('2023-10-22 00:00:00.000', materialize(1));
SELECT ' -- subtract';
SELECT subtractYears('2023-10-22', materialize(1)), subtractYears('2023-10-22 00:00:00.000', materialize(1));
SELECT subtractQuarters('2023-10-22', materialize(1)), subtractQuarters('2023-10-22 00:00:00.000', materialize(1));
SELECT subtractMonths('2023-10-22', materialize(1)), subtractMonths('2023-10-22 00:00:00.000', materialize(1));
SELECT subtractWeeks('2023-10-22', materialize(1)), subtractWeeks('2023-10-22 00:00:00.000', materialize(1));
SELECT subtractDays('2023-10-22', materialize(1)), subtractDays('2023-10-22 00:00:00.000', materialize(1));
SELECT subtractHours('2023-10-22', materialize(1)), subtractHours('2023-10-22 00:00:00.000', materialize(1));
SELECT subtractMinutes('2023-10-22', materialize(1)), subtractMinutes('2023-10-22 00:00:00.000', materialize(1));
SELECT subtractSeconds('2023-10-22', materialize(1)), subtractSeconds('2023-10-22 00:00:00.000', materialize(1));
SELECT subtractMilliseconds('2023-10-22', materialize(1)), subtractMilliseconds('2023-10-22 00:00:00.000', materialize(1));
SELECT subtractMicroseconds('2023-10-22', materialize(1)), subtractMicroseconds('2023-10-22 00:00:00.000', materialize(1));
SELECT subtractNanoseconds('2023-10-22', materialize(1)), subtractNanoseconds('2023-10-22 00:00:00.000', materialize(1));
SELECT '-- non-const date, non-const delta';
SELECT ' -- add';
SELECT addYears(materialize('2023-10-22'), materialize(1)), addYears(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT addQuarters(materialize('2023-10-22'), materialize(1)), addQuarters(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT addMonths(materialize('2023-10-22'), materialize(1)), addMonths(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT addWeeks(materialize('2023-10-22'), materialize(1)), addWeeks(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT addDays(materialize('2023-10-22'), materialize(1)), addDays(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT addHours(materialize('2023-10-22'), materialize(1)), addHours(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT addMinutes(materialize('2023-10-22'), materialize(1)), addMinutes(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT addSeconds(materialize('2023-10-22'), materialize(1)), addSeconds(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT addMilliseconds(materialize('2023-10-22'), materialize(1)), addMilliseconds(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT addMicroseconds(materialize('2023-10-22'), materialize(1)), addMicroseconds(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT addNanoseconds(materialize('2023-10-22'), materialize(1)), addNanoseconds(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT ' -- subtract';
SELECT subtractYears(materialize('2023-10-22'), materialize(1)), subtractYears(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT subtractQuarters(materialize('2023-10-22'), materialize(1)), subtractQuarters(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT subtractMonths(materialize('2023-10-22'), materialize(1)), subtractMonths(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT subtractWeeks(materialize('2023-10-22'), materialize(1)), subtractWeeks(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT subtractDays(materialize('2023-10-22'), materialize(1)), subtractDays(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT subtractHours(materialize('2023-10-22'), materialize(1)), subtractHours(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT subtractMinutes(materialize('2023-10-22'), materialize(1)), subtractMinutes(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT subtractSeconds(materialize('2023-10-22'), materialize(1)), subtractSeconds(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT subtractMilliseconds(materialize('2023-10-22'), materialize(1)), subtractMilliseconds(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT subtractMicroseconds(materialize('2023-10-22'), materialize(1)), subtractMicroseconds(materialize('2023-10-22 00:00:00.000'), materialize(1));
SELECT subtractNanoseconds(materialize('2023-10-22'), materialize(1)), subtractNanoseconds(materialize('2023-10-22 00:00:00.000'), 1);
SELECT '-- plus operator';
SELECT '2023-10-23' + INTERVAL 1 YEAR;
SELECT '2023-10-23' + INTERVAL 1 QUARTER;
SELECT '2023-10-23' + INTERVAL 1 MONTH;
SELECT '2023-10-23' + INTERVAL 1 WEEK;
SELECT '2023-10-23' + INTERVAL 1 DAY;
SELECT '2023-10-23' + INTERVAL 1 HOUR;
SELECT '2023-10-23' + INTERVAL 1 MINUTE;
SELECT '2023-10-23' + INTERVAL 1 SECOND;
SELECT '2023-10-23' + INTERVAL 1 MILLISECOND;
SELECT '2023-10-23' + INTERVAL 1 MICROSECOND;
SELECT '2023-10-23' + INTERVAL 1 NANOSECOND;
SELECT '-- minus operator';
SELECT '2023-10-23' - INTERVAL 1 YEAR;
SELECT '2023-10-23' - INTERVAL 1 QUARTER;
SELECT '2023-10-23' - INTERVAL 1 MONTH;
SELECT '2023-10-23' - INTERVAL 1 WEEK;
SELECT '2023-10-23' - INTERVAL 1 DAY;
SELECT '2023-10-23' - INTERVAL 1 HOUR;
SELECT '2023-10-23' - INTERVAL 1 MINUTE;
SELECT '2023-10-23' - INTERVAL 1 SECOND;
SELECT '2023-10-23' - INTERVAL 1 MILLISECOND;
SELECT '2023-10-23' - INTERVAL 1 MICROSECOND;
SELECT '2023-10-23' - INTERVAL 1 NANOSECOND;