Merge pull request #56738 from yariks5s/time_buckets_impl

Add time-bucket overload to function `toStartOfInterval()`
This commit is contained in:
Yarik Briukhovetskyi 2024-09-09 09:18:35 +00:00 committed by GitHub
commit 2c35d53bf6
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
6 changed files with 424 additions and 74 deletions

View File

@ -1617,8 +1617,19 @@ The calculation is performed relative to specific points in time:
If unit `WEEK` was specified, `toStartOfInterval` assumes that weeks start on Monday. Note that this behavior is different from that of function `toStartOfWeek` in which weeks start by default on Sunday.
**See Also**
**Syntax**
```sql
toStartOfInterval(value, INTERVAL x unit[, time_zone])
toStartOfInterval(value, INTERVAL x unit[, origin[, time_zone]])
```
The second overload emulates TimescaleDB's `time_bucket()` function, respectively PostgreSQL's `date_bin()` function, e.g.
``` SQL
SELECT toStartOfInterval(toDateTime('2023-01-01 14:45:00'), INTERVAL 1 MINUTE, toDateTime('2023-01-01 14:35:30'));
```
**See Also**
- [date_trunc](#date_trunc)
## toTime
@ -3884,19 +3895,29 @@ Result:
└───────────────────────────────────────────────────────────────────────┘
```
## timeSlots(StartTime, Duration,\[, Size\])
## timeSlots
For a time interval starting at StartTime and continuing for Duration seconds, it returns an array of moments in time, consisting of points from this interval rounded down to the Size in seconds. Size is an optional parameter set to 1800 (30 minutes) by default.
This is necessary, for example, when searching for pageviews in the corresponding session.
Accepts DateTime and DateTime64 as StartTime argument. For DateTime, Duration and Size arguments must be `UInt32`. For DateTime64 they must be `Decimal64`.
Returns an array of DateTime/DateTime64 (return type matches the type of StartTime). For DateTime64, the return value's scale can differ from the scale of StartTime --- the highest scale among all given arguments is taken.
Example:
**Syntax**
```sql
timeSlots(StartTime, Duration,\[, Size\])
```
**Example**
```sql
SELECT timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600));
SELECT timeSlots(toDateTime('1980-12-12 21:01:02', 'UTC'), toUInt32(600), 299);
SELECT timeSlots(toDateTime64('1980-12-12 21:01:02.1234', 4, 'UTC'), toDecimal64(600.1, 1), toDecimal64(299, 0));
```
Result:
``` text
┌─timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600))─┐
│ ['2012-01-01 12:00:00','2012-01-01 12:30:00'] │

View File

@ -492,7 +492,7 @@ struct ToStartOfInterval<IntervalKind::Kind::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)
{
@ -527,7 +527,7 @@ struct ToStartOfInterval<IntervalKind::Kind::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)
{
@ -570,7 +570,7 @@ struct ToStartOfInterval<IntervalKind::Kind::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)
{
@ -613,7 +613,7 @@ struct ToStartOfInterval<IntervalKind::Kind::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);
}
@ -634,7 +634,7 @@ struct ToStartOfInterval<IntervalKind::Kind::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);
}
@ -655,7 +655,7 @@ struct ToStartOfInterval<IntervalKind::Kind::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);
}
@ -676,7 +676,7 @@ struct ToStartOfInterval<IntervalKind::Kind::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);
}
@ -697,9 +697,13 @@ struct ToStartOfInterval<IntervalKind::Kind::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::Kind::Day>::execute(t, weeks * 7, time_zone, scale_multiplier, origin);
}
};
@ -718,9 +722,23 @@ struct ToStartOfInterval<IntervalKind::Kind::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);
const Int64 scaled_time = t / scale_multiplier;
if (origin == 0)
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(scaled_time), months);
else
{
const Int64 scaled_origin = origin / scale_multiplier;
const Int64 days = time_zone.toDayOfMonth(scaled_time + scaled_origin) - time_zone.toDayOfMonth(scaled_origin);
Int64 months_to_add = time_zone.toMonth(scaled_time + scaled_origin) - time_zone.toMonth(scaled_origin);
const Int64 years = time_zone.toYear(scaled_time + scaled_origin) - time_zone.toYear(scaled_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;
return (time_zone.addMonths(time_zone.toDate(scaled_origin), month_multiplier) - time_zone.toDate(scaled_origin));
}
}
};
@ -739,9 +757,12 @@ struct ToStartOfInterval<IntervalKind::Kind::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::Kind::Month>::execute(t, quarters * 3, time_zone, scale_multiplier, origin);
}
};
@ -760,9 +781,12 @@ struct ToStartOfInterval<IntervalKind::Kind::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
return ToStartOfInterval<IntervalKind::Kind::Month>::execute(t, years * 12, time_zone, scale_multiplier, origin);
}
};

View File

@ -10,21 +10,31 @@
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <IO/WriteHelpers.h>
#include <algorithm>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
class FunctionToStartOfInterval : public IFunction
{
private:
enum class Overload
{
Default, /// toStartOfInterval(time, interval) or toStartOfInterval(time, interval, timezone)
Origin /// toStartOfInterval(time, interval, origin) or toStartOfInterval(time, interval, origin, timezone)
};
mutable Overload overload;
public:
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionToStartOfInterval>(); }
@ -34,7 +44,7 @@ public:
size_t getNumberOfArguments() const override { return 0; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; }
bool hasInformationAboutMonotonicity() const override { return true; }
Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override { return { .is_monotonic = true, .is_always_monotonic = true }; }
@ -72,6 +82,9 @@ public:
"Illegal type {} of 2nd argument of function {}, expected a time interval",
type_arg2->getName(), getName());
overload = Overload::Default;
/// Determine result type for default overload (no origin)
switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case)
{
case IntervalKind::Kind::Nanosecond:
@ -97,13 +110,49 @@ public:
auto check_third_argument = [&]
{
const DataTypePtr & type_arg3 = arguments[2].type;
if (!isString(type_arg3))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of 3rd argument of function {}, expected a constant timezone string",
if (isString(type_arg3))
{
if (value_is_date && result_type == ResultType::Date)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"A timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64",
getName(), interval_type->getKind().toString());
}
else if (isDateOrDate32OrDateTimeOrDateTime64(type_arg3))
{
overload = Overload::Origin;
const DataTypePtr & type_arg1 = arguments[0].type;
if (isDate(type_arg1) && isDate(type_arg3))
result_type = ResultType::Date;
else if (isDate32(type_arg1) && isDate32(type_arg3))
result_type = ResultType::Date32;
else if (isDateTime(type_arg1) && isDateTime(type_arg3))
result_type = ResultType::DateTime;
else if (isDateTime64(type_arg1) && isDateTime64(type_arg3))
result_type = ResultType::DateTime64;
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName());
}
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 3rd argument of function {}. "
"This argument is optional and must be a constant String with timezone name or a Date/Date32/DateTime/DateTime64 with a constant origin",
type_arg3->getName(), getName());
if (value_is_date && result_type == ResultType::Date) /// weird why this is && instead of || but too afraid to change it
};
auto check_fourth_argument = [&]
{
if (overload != Overload::Origin) /// sanity check
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 3rd argument of function {}. "
"The third argument must a Date/Date32/DateTime/DateTime64 with a constant origin",
arguments[2].type->getName(), getName());
const DataTypePtr & type_arg4 = arguments[3].type;
if (!isString(type_arg4))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 4th argument of function {}. "
"This argument is optional and must be a constant String with timezone name",
type_arg4->getName(), getName());
if (value_is_date && result_type == ResultType::Date)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The timezone argument of function {} with interval type {} is allowed only when the 1st argument has type DateTime or DateTimt64",
"A timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64",
getName(), interval_type->getKind().toString());
};
@ -118,10 +167,17 @@ public:
check_second_argument();
check_third_argument();
}
else if (arguments.size() == 4)
{
check_first_argument();
check_second_argument();
check_third_argument();
check_fourth_argument();
}
else
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, should be 2 or 3",
"Number of arguments for function {} doesn't match: passed {}, must be 2, 3 or 4",
getName(), arguments.size());
}
@ -132,10 +188,19 @@ public:
case ResultType::Date32:
return std::make_shared<DataTypeDate32>();
case ResultType::DateTime:
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false));
{
const size_t time_zone_arg_num = (overload == Overload::Default) ? 2 : 3;
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false));
}
case ResultType::DateTime64:
{
UInt32 scale = 0;
if (isDateTime64(arguments[0].type) && overload == Overload::Origin)
{
scale = assert_cast<const DataTypeDateTime64 &>(*arguments[0].type.get()).getScale();
if (assert_cast<const DataTypeDateTime64 &>(*arguments[2].type.get()).getScale() != scale)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same scale", getName());
}
if (interval_type->getKind() == IntervalKind::Kind::Nanosecond)
scale = 9;
else if (interval_type->getKind() == IntervalKind::Kind::Microsecond)
@ -143,69 +208,103 @@ public:
else if (interval_type->getKind() == IntervalKind::Kind::Millisecond)
scale = 3;
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false));
const size_t time_zone_arg_num = (overload == Overload::Default) ? 2 : 3;
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false));
}
}
std::unreachable();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override
{
const auto & time_column = arguments[0];
const auto & interval_column = arguments[1];
const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0);
auto result_column = dispatchForTimeColumn(time_column, interval_column, result_type, time_zone, input_rows_count);
ColumnWithTypeAndName origin_column;
if (overload == Overload::Origin)
origin_column = arguments[2];
const size_t time_zone_arg_num = (overload == Overload::Default) ? 2 : 3;
const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, time_zone_arg_num, 0);
ColumnPtr result_column;
if (isDate(result_type))
result_column = dispatchForTimeColumn<DataTypeDate>(time_column, interval_column, origin_column, result_type, time_zone);
else if (isDate32(result_type))
result_column = dispatchForTimeColumn<DataTypeDate32>(time_column, interval_column, origin_column, result_type, time_zone);
else if (isDateTime(result_type))
result_column = dispatchForTimeColumn<DataTypeDateTime>(time_column, interval_column, origin_column, result_type, time_zone);
else if (isDateTime64(result_type))
result_column = dispatchForTimeColumn<DataTypeDateTime64>(time_column, interval_column, origin_column, result_type, time_zone);
return result_column;
}
private:
template <typename ReturnType>
ColumnPtr dispatchForTimeColumn(
const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column,
const DataTypePtr & result_type, const DateLUTImpl & time_zone,
size_t input_rows_count) const
const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const
{
const auto & time_column_type = *time_column.type.get();
const auto & time_column_col = *time_column.column.get();
if (isDateTime64(time_column_type))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime64>(&time_column_col);
auto scale = assert_cast<const DataTypeDateTime64 &>(time_column_type).getScale();
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime64 &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count, scale);
}
else if (isDateTime(time_column_type))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime>(&time_column_col);
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count);
}
else if (isDate(time_column_type))
if (isDate(time_column_type))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDate>(&time_column_col);
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count);
return dispatchForIntervalColumn<ReturnType, DataTypeDate, ColumnDate>(assert_cast<const DataTypeDate &>(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone);
}
else if (isDate32(time_column_type))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDate32>(&time_column_col);
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate32 &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count);
return dispatchForIntervalColumn<ReturnType, DataTypeDate32, ColumnDate32>(assert_cast<const DataTypeDate32 &>(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone);
}
else if (isDateTime(time_column_type))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime>(&time_column_col);
if (time_column_vec)
return dispatchForIntervalColumn<ReturnType, DataTypeDateTime, ColumnDateTime>(assert_cast<const DataTypeDateTime &>(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone);
}
else if (isDateTime64(time_column_type))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime64>(&time_column_col);
auto scale = assert_cast<const DataTypeDateTime64 &>(time_column_type).getScale();
if (time_column_vec)
return dispatchForIntervalColumn<ReturnType, DataTypeDateTime64, ColumnDateTime64>(assert_cast<const DataTypeDateTime64 &>(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone, scale);
}
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, Date32, DateTime or DateTime64", getName());
}
template <typename TimeDataType, typename TimeColumnType>
template <typename ReturnType, typename TimeDataType, typename TimeColumnType>
ColumnPtr dispatchForIntervalColumn(
const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column,
const DataTypePtr & result_type, const DateLUTImpl & time_zone, size_t input_rows_count, UInt16 scale = 1) const
const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column,
const DataTypePtr & result_type, const DateLUTImpl & time_zone, UInt16 scale = 1) const
{
const auto * interval_type = checkAndGetDataType<DataTypeInterval>(interval_column.type.get());
if (!interval_type)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for 2nd argument of function {}, must be a time interval", getName());
switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case)
{
case IntervalKind::Kind::Nanosecond:
case IntervalKind::Kind::Microsecond:
case IntervalKind::Kind::Millisecond:
if (isDateOrDate32(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::Kind::Second:
case IntervalKind::Kind::Minute:
case IntervalKind::Kind::Hour:
if (isDateOrDate32(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());
@ -217,51 +316,102 @@ private:
switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case)
{
case IntervalKind::Kind::Nanosecond:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime64, IntervalKind::Kind::Nanosecond>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
return execute<ReturnType, TimeDataType, TimeColumnType, IntervalKind::Kind::Nanosecond>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
case IntervalKind::Kind::Microsecond:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime64, IntervalKind::Kind::Microsecond>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
return execute<ReturnType, TimeDataType, TimeColumnType, IntervalKind::Kind::Microsecond>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
case IntervalKind::Kind::Millisecond:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime64, IntervalKind::Kind::Millisecond>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
return execute<ReturnType, TimeDataType, TimeColumnType, IntervalKind::Kind::Millisecond>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
case IntervalKind::Kind::Second:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime, IntervalKind::Kind::Second>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
return execute<ReturnType, TimeDataType, TimeColumnType, IntervalKind::Kind::Second>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
case IntervalKind::Kind::Minute:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime, IntervalKind::Kind::Minute>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
return execute<ReturnType, TimeDataType, TimeColumnType, IntervalKind::Kind::Minute>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
case IntervalKind::Kind::Hour:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime, IntervalKind::Kind::Hour>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
return execute<ReturnType, TimeDataType, TimeColumnType, IntervalKind::Kind::Hour>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
case IntervalKind::Kind::Day:
return execute<TimeDataType, TimeColumnType, DataTypeDateTime, IntervalKind::Kind::Day>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
return execute<ReturnType, TimeDataType, TimeColumnType, IntervalKind::Kind::Day>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
case IntervalKind::Kind::Week:
return execute<TimeDataType, TimeColumnType, DataTypeDate, IntervalKind::Kind::Week>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
return execute<ReturnType, TimeDataType, TimeColumnType, IntervalKind::Kind::Week>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
case IntervalKind::Kind::Month:
return execute<TimeDataType, TimeColumnType, DataTypeDate, IntervalKind::Kind::Month>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
return execute<ReturnType, TimeDataType, TimeColumnType, IntervalKind::Kind::Month>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
case IntervalKind::Kind::Quarter:
return execute<TimeDataType, TimeColumnType, DataTypeDate, IntervalKind::Kind::Quarter>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
return execute<ReturnType, TimeDataType, TimeColumnType, IntervalKind::Kind::Quarter>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
case IntervalKind::Kind::Year:
return execute<TimeDataType, TimeColumnType, DataTypeDate, IntervalKind::Kind::Year>(time_data_type, time_column, num_units, result_type, time_zone, input_rows_count, scale);
return execute<ReturnType, TimeDataType, TimeColumnType, IntervalKind::Kind::Year>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
}
std::unreachable();
}
template <typename TimeDataType, typename TimeColumnType, typename ResultDataType, IntervalKind::Kind unit>
ColumnPtr execute(
const TimeDataType &, const TimeColumnType & time_column_type, Int64 num_units,
const DataTypePtr & result_type, const DateLUTImpl & time_zone, size_t input_rows_count, UInt16 scale) const
template <typename ResultDataType, typename TimeDataType, typename TimeColumnType, IntervalKind::Kind unit>
ColumnPtr execute(const TimeDataType &, const TimeColumnType & time_column_type, Int64 num_units, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, UInt16 scale) const
{
using ResultColumnType = typename ResultDataType::ColumnType;
using ResultFieldType = typename ResultDataType::FieldType;
const auto & time_data = time_column_type.getData();
size_t size = time_data.size();
auto result_col = result_type->createColumn();
auto * col_to = assert_cast<ResultColumnType *>(result_col.get());
auto & result_data = col_to->getData();
result_data.resize(input_rows_count);
result_data.resize(size);
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
for (size_t i = 0; i != input_rows_count; ++i)
result_data[i] = static_cast<ResultFieldType>(ToStartOfInterval<unit>::execute(time_data[i], num_units, time_zone, scale_multiplier));
if (origin_column.column) // Overload: Origin
{
const bool is_small_interval = (unit == IntervalKind::Kind::Nanosecond || unit == IntervalKind::Kind::Microsecond || unit == IntervalKind::Kind::Millisecond);
const bool is_result_date = isDateOrDate32(result_type);
Int64 result_scale = scale_multiplier;
Int64 origin_scale = 1;
if (isDateTime64(result_type)) /// We have origin scale only in case if arguments are DateTime64.
origin_scale = assert_cast<const DataTypeDateTime64 &>(*origin_column.type).getScaleMultiplier();
else if (!is_small_interval) /// In case of large interval and arguments are not DateTime64, we should not have scale in result.
result_scale = 1;
if (is_small_interval)
result_scale = assert_cast<const DataTypeDateTime64 &>(*result_type).getScaleMultiplier();
/// 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. In case of large intervals, we should not have scale difference.
Int64 scale_diff = is_small_interval ? std::max(result_scale / origin_scale, origin_scale / result_scale) : 1;
static constexpr Int64 SECONDS_PER_DAY = 86'400;
UInt64 origin = origin_column.column->get64(0);
for (size_t i = 0; i != size; ++i)
{
UInt64 time_arg = time_data[i];
if (origin > static_cast<size_t>(time_arg))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date / date with time");
if (is_result_date) /// All internal calculations of ToStartOfInterval<...> expect arguments to be seconds or milli-, micro-, nanoseconds.
{
time_arg *= SECONDS_PER_DAY;
origin *= SECONDS_PER_DAY;
}
Int64 offset = ToStartOfInterval<unit>::execute(time_arg - origin, num_units, time_zone, result_scale, origin);
/// In case if arguments are DateTime64 with large interval, we should apply scale on it.
offset *= (!is_small_interval) ? result_scale : 1;
if (is_result_date) /// Convert back to date after calculations.
{
offset /= SECONDS_PER_DAY;
origin /= SECONDS_PER_DAY;
}
result_data[i] = 0;
result_data[i] += (result_scale < origin_scale) ? (origin + offset) / scale_diff : (origin + offset) * scale_diff;
}
}
else // Overload: Default
{
for (size_t i = 0; i != size; ++i)
result_data[i] = static_cast<typename ResultDataType::FieldType>(ToStartOfInterval<unit>::execute(time_data[i], num_units, time_zone, scale_multiplier));
}
return result_col;
}

View File

@ -0,0 +1,59 @@
-- Negative tests
Time and origin as Date
2023-02-01
2023-08-01
2023-10-08
2023-10-08
2023-10-09
Time and origin as Date32
2023-02-01
2023-08-01
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-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: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-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.987
2023-10-09 10:11:12.987654
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-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
Non-const arguments
2023-03-01 16:55:00.00
2023-02-01 16:55:00.00
2023-03-01 16:55:00.00
2023-02-01 16:55:00.00
2023-03-01 16:55:00.00
2023-03-01 16:55:00
2023-02-01 16:55:00
2023-03-01 16:55:00
2023-02-01 16:55:00
2023-03-01 16:55:00
2023-01-02 15:44:30
2023-02-01 16:44:30.00

View File

@ -0,0 +1,95 @@
set session_timezone = 'UTC';
SELECT '-- Negative tests';
-- time and origin arguments must have the same type
SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalSecond(5), toDate32('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalMillisecond(12), toDateTime('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDate32('2023-01-02 14:45:50'), toIntervalHour(5), toDate('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime64('2023-01-02 14:44:30', 2)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalMinute(1), toDate('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- the origin must be before the time
SELECT toStartOfInterval(toDateTime('2023-01-02 14:42:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30')); -- { serverError BAD_ARGUMENTS }
-- the origin must be constant
SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), number % 2 == 0 ? toDateTime('2023-02-01 15:55:00') : toDateTime('2023-01-01 15:55:00')) from numbers(1); -- { serverError ILLEGAL_COLUMN }
SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalHour(1), materialize(toDateTime('2023-01-02 14:44:30')), 'Europe/Amsterdam'); -- { serverError ILLEGAL_COLUMN }
-- with 4 arguments, the 3rd one must not be a string or an integer
SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 'Europe/Amsterdam', 'Europe/Amsterdam'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 5, 'Europe/Amsterdam'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- 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 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'));
SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalWeek(1), toDate('2023-10-01'));
SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalDay(1), toDate('2023-10-08'));
SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalHour(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMinute(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalSecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMillisecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMicrosecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalNanosecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT 'Time and origin as Date32';
SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalYear(1), toDate32('2022-02-01'));
SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalQuarter(1), toDate32('2022-02-01'));
SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalMonth(1), toDate32('2023-09-08'));
SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalWeek(1), toDate32('2023-10-01'));
SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalDay(1), toDate32('2023-10-08'));
SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalHour(1), toDate32('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalMinute(1), toDate32('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalSecond(1), toDate32('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalMillisecond(1), toDate32('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalMicrosecond(1), toDate32('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT toStartOfInterval(toDate32('2023-10-09'), toIntervalNanosecond(1), toDate32('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT 'Time and origin as DateTime';
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalYear(1), toDateTime('2022-02-01 09:08:07'));
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalQuarter(1), toDateTime('2022-02-01 09:08:07'));
SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMonth(1), toDateTime('2023-09-08 09:08:07'));
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 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 }
SELECT 'Time and origin as DateTime64(9)';
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalYear(1), toDateTime64('2022-02-01 09:08:07.123456789', 9));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalQuarter(1), toDateTime64('2022-02-01 09:08:07.123456789', 9));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMonth(1), toDateTime64('2023-09-10 09:08:07.123456789', 9));
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 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));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalNanosecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9));
SELECT 'Time and origin as DateTime64(3)';
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalYear(1), toDateTime64('2022-02-01 09:08:07.123', 3));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalQuarter(1), toDateTime64('2022-02-01 09:08:07.123', 3));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMonth(1), toDateTime64('2023-09-08 09:08:07.123', 3));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalWeek(1), toDateTime64('2023-10-01 09:08:07.123', 3));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalDay(1), toDateTime64('2023-10-08 09:08:07.123', 3));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalHour(1), toDateTime64('2023-10-09 09:10:07.123', 3));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMinute(1), toDateTime64('2023-10-09 10:10:11.123', 3));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalSecond(1), toDateTime64('2023-10-09 10:11:10.123', 3));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMillisecond(1), toDateTime64('2023-10-09 10:11:12.123', 3));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMicrosecond(1), toDateTime64('2023-10-09 10:11:12.123', 3));
SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalNanosecond(1), toDateTime64('2023-10-09 10:11:12.123', 3));
SELECT 'Non-const arguments';
SELECT toStartOfInterval(number % 2 == 0 ? toDateTime64('2023-03-01 15:55:00', 2) : toDateTime64('2023-02-01 15:55:00', 2), toIntervalMinute(1), toDateTime64('2023-01-01 13:55:00', 2), 'Europe/Amsterdam') from numbers(5);
SELECT toStartOfInterval(number % 2 == 0 ? toDateTime('2023-03-01 15:55:00') : toDateTime('2023-02-01 15:55:00'), toIntervalHour(1), toDateTime('2023-01-01 13:55:00'), 'Europe/Amsterdam') from numbers(5);
SELECT toStartOfInterval(materialize(toDateTime('2023-01-02 14:45:50')), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam');
SELECT toStartOfInterval(materialize(toDateTime64('2023-02-01 15:45:50', 2)), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam');

View File

@ -975,6 +975,7 @@ ThreadPoolRemoteFSReaderThreads
ThreadPoolRemoteFSReaderThreadsActive
ThreadsActive
ThreadsInOvercommitTracker
TimescaleDB's
TimeSeries
Timeunit
TinyLog