This commit is contained in:
Yarik Briukhovetskyi 2024-01-05 16:03:22 +00:00
parent 174309821a
commit 861421d27a
4 changed files with 139 additions and 75 deletions

View File

@ -487,7 +487,7 @@ struct ToStartOfInterval<IntervalKind::Nanosecond>
{
throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static Int64 execute(Int64 t, Int64 nanoseconds, const DateLUTImpl &, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 nanoseconds, const DateLUTImpl &, Int64 scale_multiplier, Int64 /*origin*/ = 0)
{
if (scale_multiplier < 1000000000)
{
@ -522,7 +522,7 @@ struct ToStartOfInterval<IntervalKind::Microsecond>
{
throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static Int64 execute(Int64 t, Int64 microseconds, const DateLUTImpl &, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 microseconds, const DateLUTImpl &, Int64 scale_multiplier, Int64 /*origin*/ = 0)
{
if (scale_multiplier < 1000000)
{
@ -565,7 +565,7 @@ struct ToStartOfInterval<IntervalKind::Millisecond>
{
throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME);
}
static Int64 execute(Int64 t, Int64 milliseconds, const DateLUTImpl &, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 milliseconds, const DateLUTImpl &, Int64 scale_multiplier, Int64 /*origin*/ = 0)
{
if (scale_multiplier < 1000)
{
@ -608,7 +608,7 @@ struct ToStartOfInterval<IntervalKind::Second>
{
return time_zone.toStartOfSecondInterval(t, seconds);
}
static Int64 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 /*origin*/ = 0)
{
return time_zone.toStartOfSecondInterval(t / scale_multiplier, seconds);
}
@ -629,7 +629,7 @@ struct ToStartOfInterval<IntervalKind::Minute>
{
return time_zone.toStartOfMinuteInterval(t, minutes);
}
static Int64 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 /*origin*/ = 0)
{
return time_zone.toStartOfMinuteInterval(t / scale_multiplier, minutes);
}
@ -650,7 +650,7 @@ struct ToStartOfInterval<IntervalKind::Hour>
{
return time_zone.toStartOfHourInterval(t, hours);
}
static Int64 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 /*origin*/ = 0)
{
return time_zone.toStartOfHourInterval(t / scale_multiplier, hours);
}
@ -671,7 +671,7 @@ struct ToStartOfInterval<IntervalKind::Day>
{
return static_cast<UInt32>(time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days));
}
static Int64 execute(Int64 t, Int64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 /*origin*/ = 0)
{
return time_zone.toStartOfDayInterval(time_zone.toDayNum(t / scale_multiplier), days);
}
@ -692,9 +692,12 @@ struct ToStartOfInterval<IntervalKind::Week>
{
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks);
}
static UInt16 execute(Int64 t, Int64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 origin = 0)
{
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks);
if (origin == 0)
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks);
else
return ToStartOfInterval<IntervalKind::Day>::execute(t, weeks * 7, time_zone, scale_multiplier, origin);
}
};
@ -713,9 +716,24 @@ struct ToStartOfInterval<IntervalKind::Month>
{
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months);
}
static UInt16 execute(Int64 t, Int64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 origin = 0)
{
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months);
if (origin == 0)
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months);
else
{
Int64 days = time_zone.toDayOfMonth(t / scale_multiplier + origin) - time_zone.toDayOfMonth(origin);
Int64 months_to_add = time_zone.toMonth(t / scale_multiplier + origin) - time_zone.toMonth(origin);
Int64 years = time_zone.toYear(t / scale_multiplier + origin) - time_zone.toYear(origin);
months_to_add = days < 0 ? months_to_add - 1 : months_to_add;
months_to_add += years * 12;
Int64 month_multiplier = (months_to_add / months) * months;
Int64 a = 0;
a = time_zone.addMonths(time_zone.toDate(origin), month_multiplier);
// a += time_zone.toTime(origin);
return a - time_zone.toDate(origin);
}
}
};
@ -734,9 +752,12 @@ struct ToStartOfInterval<IntervalKind::Quarter>
{
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters);
}
static UInt16 execute(Int64 t, Int64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 origin = 0)
{
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters);
if (origin == 0)
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters);
else
return ToStartOfInterval<IntervalKind::Month>::execute(t, quarters * 3, time_zone, scale_multiplier, origin);
}
};
@ -755,9 +776,15 @@ struct ToStartOfInterval<IntervalKind::Year>
{
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years);
}
static UInt16 execute(Int64 t, Int64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 origin = 0)
{
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years);
if (origin == 0)
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years);
else
{
auto a = ToStartOfInterval<IntervalKind::Month>::execute(t, years * 12, time_zone, scale_multiplier, origin);
return a;
}
}
};

View File

@ -1,17 +1,17 @@
#include <cmath>
#include <string>
#include <type_traits>
#include <Columns/ColumnsDateTime.h>
#include <Columns/ColumnsNumber.h>
#include <Common/DateLUTImpl.h>
#include <Common/IntervalKind.h>
#include "DataTypes/IDataType.h"
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeInterval.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <IO/WriteHelpers.h>
#include <base/arithmeticOverflow.h>
@ -270,6 +270,27 @@ private:
if (!interval_type)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for 2nd argument of function {}, must be a time interval", getName());
if (isDate(time_data_type) || isDateTime(time_data_type))
{
switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case)
{
case IntervalKind::Nanosecond:
case IntervalKind::Microsecond:
case IntervalKind::Millisecond:
if (isDate(time_data_type) || isDateTime(time_data_type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal interval kind for argument data type {}", isDate(time_data_type) ? "Date" : "DateTime");
break;
case IntervalKind::Second:
case IntervalKind::Minute:
case IntervalKind::Hour:
if (isDate(time_data_type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal interval kind for argument data type Date");
break;
default:
break;
}
}
const auto * interval_column_const_int64 = checkAndGetColumnConst<ColumnInt64>(interval_column.column.get());
if (!interval_column_const_int64)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for 2nd argument of function {}, must be a const time interval", getName());
@ -337,94 +358,110 @@ private:
auto & result_data = col_to->getData();
result_data.resize(size);
const Int64 scale_time = DecimalUtils::scaleMultiplier<DateTime64>(scale);
const Int64 scale_endtime = DecimalUtils::scaleMultiplier<DateTime64>(scale);
const Int64 scale_interval = scaleFromInterval<unit>();
/// In case if we have a difference between time arguments and Interval, we need to calculate the difference between them
/// to get the right precision for the result.
const Int64 scale_diff = (scale_interval > scale_time) ? (scale_interval / scale_time) : (scale_time / scale_interval);
const Int64 scale_diff = (scale_interval > scale_endtime) ? (scale_interval / scale_endtime) : (scale_endtime / scale_interval);
if (origin_column.column == nullptr)
{
if (scale_time > scale_interval && scale_interval != 1)
if (scale_endtime > scale_interval && scale_interval != 1)
{
for (size_t i = 0; i != size; ++i)
{
/// If we have a time argument that has bigger scale than the interval can contain and interval is not default, we need
/// to return a value with bigger precision and thus we should multiply result on the scale difference.
result_data[i] = 0;
result_data[i] += static_cast<ResultFieldType>(ToStartOfInterval<unit>::execute(time_data[i], num_units, time_zone, scale_time)) * scale_diff;
result_data[i] += static_cast<ResultFieldType>(ToStartOfInterval<unit>::execute(time_data[i], num_units, time_zone, scale_endtime)) * scale_diff;
}
}
else
{
for (size_t i = 0; i != size; ++i)
result_data[i] = static_cast<ResultFieldType>(ToStartOfInterval<unit>::execute(time_data[i], num_units, time_zone, scale_time));
result_data[i] = static_cast<ResultFieldType>(ToStartOfInterval<unit>::execute(time_data[i], num_units, time_zone, scale_endtime));
}
}
else
{
UInt64 origin = origin_column.column->get64(0);
Int64 origin_scale = 1;
if (isDateTime64(origin_column.type.get()))
origin_scale = assert_cast<const DataTypeDateTime64 &>(*origin_column.type.get()).getScale();
for (size_t i = 0; i != size; ++i)
{
auto t = time_data[i];
if (origin > static_cast<size_t>(t))
UInt64 end_time = time_data[i];
if (origin > static_cast<size_t>(end_time) && origin_scale == scale)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date / date with time");
else if (origin_scale > scale)
origin /= static_cast<UInt64>(std::pow(10, origin_scale - scale)); /// If aguments have different scales, we make
else if (origin_scale < scale) /// origin argument to have the same scale as the first argument.
origin *= static_cast<UInt64>(std::pow(10, scale - origin_scale));
/// The trick to calculate the interval starting from an offset is to
/// 1. subtract the offset,
/// 2. perform the calculation, and
/// 3. add the offset to the result.
t -= origin;
auto res = static_cast<ResultFieldType>(ToStartOfInterval<unit>::execute(t, num_units, time_zone, scale_time));
static constexpr size_t SECONDS_PER_DAY = 86'400;
result_data[i] = 0;
if (unit == IntervalKind::Week || unit == IntervalKind::Month || unit == IntervalKind::Quarter || unit == IntervalKind::Year)
if (isDate(origin_column.type.get())) /// We need to perform calculations on dateTime (dateTime64) values only.
{
/// For such intervals, ToStartOfInterval<unit>::execute() returns days
if (isDate(result_type))
result_data[i] += origin + res;
else if (isDateTime(result_type))
result_data[i] += origin + res * SECONDS_PER_DAY;
else if (isDateTime64(result_type))
result_data[i] += origin + (res * SECONDS_PER_DAY * scale_time);
end_time *= SECONDS_PER_DAY;
origin *= SECONDS_PER_DAY;
}
Int64 delta = (end_time - origin) * (isDateTime64(origin_column.type.get()) ? 1 : scale_endtime); /// No need to multiply on scale endtime if we have dateTime64 argument.
Int64 offset = 0;
{
auto origin_data = isDateTime64(result_type) ? origin / scale_endtime : origin;
offset = static_cast<DataTypeDateTime::FieldType>(ToStartOfInterval<unit>::execute(delta, num_units, time_zone, scale_endtime, origin_data));
}
if (isDate(result_type)) /// The result should be a date and the calculations were as datetime.
result_data[i] += (origin + offset) / SECONDS_PER_DAY;
else if (unit == IntervalKind::Week || unit == IntervalKind::Month || unit == IntervalKind::Quarter || unit == IntervalKind::Year)
{
if (isDateTime64(result_type)) /// We need to have the right scale for offset, origin already has the right scale.
offset *= scale_endtime;
result_data[i] += origin + offset;
}
else
{
/// ToStartOfInterval<unit>::execute() returns seconds
if (isDate(result_type))
res = res / SECONDS_PER_DAY;
/// ToStartOfInterval<unit>::execute() returns seconds.
if (scale_interval == 1)
{
/// Interval has default scale, i.e. Year - Second
if (isDateTime64(result_type)) /// We need to have the right scale for offset, origin already has the right scale.
offset *= scale_endtime;
if (scale_time % 1000 != 0 && scale_time >= 1000)
/// The arguments are DateTime64 with precision like 4,5,7,8. Here res has right precision and origin doesn't.
result_data[i] += (origin + res / scale_time) * scale_time;
else if (scale_time == 100)
/// The arguments are DateTime64 with precision 2. Here origin has right precision and res doesn't
result_data[i] += (origin + res * scale_time);
/// Interval has default scale, i.e. Year - Second.
if (scale_endtime % 1000 != 0 && scale_endtime >= 1000)
/// The arguments are DateTime64 with precision like 4,5,7,8. Here offset has right precision and origin doesn't.
result_data[i] += (origin + offset / scale_endtime) * scale_endtime;
else
/// Precision of DateTime64 is 1, 3, 6, 9, e.g. has right precision in res and origin.
result_data[i] += (origin + res);
/// Precision of DateTime64 is 1, 2, 3, 6, 9, e.g. has right precision in offset and origin.
result_data[i] += (origin + offset);
}
else
{
/// Interval has some specific scale (3,6,9), i.e. Millisecond - Nanosecond
/// Interval has some specific scale (3,6,9), i.e. Millisecond - Nanosecond.
if (scale_interval < scale_time)
if (scale_interval < scale_endtime)
/// If we have a time argument that has bigger scale than the interval can contain, we need
/// to return a value with bigger precision and thus we should multiply result on the scale difference.
result_data[i] += origin + res * scale_diff;
result_data[i] += origin + offset * scale_diff;
else
/// The other case: interval has bigger scale than the interval or they have the same scale, so res has the right precision and origin doesn't
result_data[i] += (origin + res / scale_diff) * scale_diff;
/// The other case: interval has bigger scale than the interval or they have the same scale, so offset has the right precision and origin doesn't.
result_data[i] += (origin + offset / scale_diff) * scale_diff;
}
}
}

View File

@ -1,40 +1,40 @@
-- Negative tests
Time and origin as Time
Time and origin as Date
2023-02-01
2023-08-01
2023-10-09
2023-10-05
2023-10-08
2023-10-08
2023-10-09
Time and origin as DateTime
2023-02-01 09:08:07
2023-08-01 09:08:07
2023-10-09 09:08:07
2023-10-05 09:08:07
2023-10-08 09:08:07
2023-10-08 09:08:07
2023-10-09 09:08:07
2023-10-09 10:10:07
2023-10-09 10:11:11
2023-10-09 10:11:07
2023-10-09 10:11:12
Time and origin as DateTime64(9)
2023-02-01 09:08:07.123456789
2023-08-01 09:08:07.123456789
2023-09-10 09:08:07.123456789
2023-10-05 09:08:07.123456789
2023-10-08 09:08:07.123543189
2023-10-09 09:10:07.123460389
2023-10-09 10:10:11.123456849
2023-10-09 10:11:10.123456791
2023-10-08 09:08:07.123456789
2023-10-09 09:08:07.123456789
2023-10-09 10:10:07.123456789
2023-10-09 10:11:11.123456789
2023-10-09 10:11:12.123456789
2023-10-09 10:11:12.987456789
2023-10-09 10:11:12.987653789
2023-10-09 10:11:12.987654321
Time and origin as DateTime64(3)
2023-02-01 09:08:07.123
2023-08-01 09:08:07.123
2023-10-08 09:08:07.123
2023-10-08 09:08:07.123
2023-10-09 09:08:07.123
2023-10-05 09:08:07.123
2023-10-08 09:09:33.523
2023-10-09 09:10:10.723
2023-10-09 10:10:11.183
2023-10-09 10:11:10.125
2023-10-09 10:10:07.123
2023-10-09 10:11:11.123
2023-10-09 10:11:12.123
2023-10-09 10:11:12.987
2023-10-09 10:11:12.987000
2023-10-09 10:11:12.987000000

View File

@ -23,7 +23,7 @@ SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 5
-- too many arguments
SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-02 14:44:30'), 'Europe/Amsterdam', 5); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT 'Time and origin as Time';
SELECT 'Time and origin as Date';
SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalYear(1), toDate('2022-02-01'));
SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalQuarter(1), toDate('2022-02-01'));
SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMonth(1), toDate('2023-09-08'));
@ -43,8 +43,8 @@ SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMonth(1),
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalWeek(1), toDateTime('2023-10-01 09:08:07'));
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalDay(1), toDateTime('2023-10-08 09:08:07'));
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalHour(1), toDateTime('2023-10-09 09:10:07'));
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMinute(1), toDateTime('2023-10-09 10:10:11'));
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalSecond(1), toDateTime('2023-10-09 10:11:10'));
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMinute(1), toDateTime('2023-10-09 09:10:07'));
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalSecond(1), toDateTime('2023-10-09 09:10:07'));
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMillisecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMicrosecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalNanosecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
@ -56,7 +56,7 @@ SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toInt
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalWeek(1), toDateTime64('2023-10-01 09:08:07.123456789', 9));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalDay(1), toDateTime64('2023-10-08 09:08:07.123456789', 9));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalHour(1), toDateTime64('2023-10-09 09:10:07.123456789', 9));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMinute(1), toDateTime64('2023-10-09 10:10:11.123456789', 9));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMinute(1), toDateTime64('2023-10-09 09:10:11.123456789', 9));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalSecond(1), toDateTime64('2023-10-09 10:11:10.123456789', 9));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMillisecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMicrosecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9));