diff --git a/src/Functions/FunctionsTimeWindow.cpp b/src/Functions/FunctionsTimeWindow.cpp index 017661231f5..91eb70fdd66 100644 --- a/src/Functions/FunctionsTimeWindow.cpp +++ b/src/Functions/FunctionsTimeWindow.cpp @@ -27,87 +27,124 @@ namespace ErrorCodes namespace { - std::tuple - dispatchForIntervalColumns(const ColumnWithTypeAndName & interval_column, const String & function_name) - { - const auto * interval_type = checkAndGetDataType(interval_column.type.get()); - if (!interval_type) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", - interval_column.name, function_name); - const auto * interval_column_const_int64 = checkAndGetColumnConst(interval_column.column.get()); - if (!interval_column_const_int64) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", - interval_column.name, function_name); - Int64 num_units = interval_column_const_int64->getValue(); - if (num_units <= 0) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Value for column {} of function {} must be positive", - interval_column.name, function_name); - return {interval_type->getKind(), num_units}; - } +std::tuple +dispatchForIntervalColumns(const ColumnWithTypeAndName & interval_column, const String & function_name) +{ + const auto * interval_type = checkAndGetDataType(interval_column.type.get()); + if (!interval_type) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", + interval_column.name, function_name); + const auto * interval_column_const_int64 = checkAndGetColumnConst(interval_column.column.get()); + if (!interval_column_const_int64) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", + interval_column.name, function_name); + Int64 num_units = interval_column_const_int64->getValue(); + if (num_units <= 0) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Value for column {} of function {} must be positive", + interval_column.name, function_name); - ColumnPtr executeWindowBound(const ColumnPtr & column, int index, const String & function_name) + return {interval_type->getKind(), num_units}; +} + +ColumnPtr executeWindowBound(const ColumnPtr & column, int index, const String & function_name) +{ + if (const ColumnTuple * col_tuple = checkAndGetColumn(column.get()); col_tuple) { - if (const ColumnTuple * col_tuple = checkAndGetColumn(column.get()); col_tuple) - { - if (!checkColumn>(*col_tuple->getColumnPtr(index))) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. " - "Must be a Tuple(DataTime, DataTime)", function_name); - return col_tuple->getColumnPtr(index); - } - else - { + if (!checkColumn>(*col_tuple->getColumnPtr(index))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. " - "Must be Tuple", function_name); - } + "Must be a Tuple(DataTime, DataTime)", function_name); + return col_tuple->getColumnPtr(index); } - - void checkFirstArgument(const ColumnWithTypeAndName & argument, const String & function_name) + else { - if (!isDateTime(argument.type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " - "Should be a date with time", argument.type->getName(), function_name); - } - - void checkIntervalArgument(const ColumnWithTypeAndName & argument, const String & function_name, IntervalKind & interval_kind, bool & result_type_is_date) - { - const auto * interval_type = checkAndGetDataType(argument.type.get()); - if (!interval_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " - "Should be an interval of time", argument.type->getName(), function_name); - interval_kind = interval_type->getKind(); - result_type_is_date = (interval_type->getKind() == IntervalKind::Kind::Year) || (interval_type->getKind() == IntervalKind::Kind::Quarter) - || (interval_type->getKind() == IntervalKind::Kind::Month) || (interval_type->getKind() == IntervalKind::Kind::Week); - } - - void checkIntervalArgument(const ColumnWithTypeAndName & argument, const String & function_name, bool & result_type_is_date) - { - IntervalKind interval_kind; - checkIntervalArgument(argument, function_name, interval_kind, result_type_is_date); - } - - void checkTimeZoneArgument( - const ColumnWithTypeAndName & argument, - const String & function_name) - { - if (!WhichDataType(argument.type).isString()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " - "This argument is optional and must be a constant string with timezone name", - argument.type->getName(), function_name); - } - - bool checkIntervalOrTimeZoneArgument(const ColumnWithTypeAndName & argument, const String & function_name, IntervalKind & interval_kind, bool & result_type_is_date) - { - if (WhichDataType(argument.type).isString()) - { - checkTimeZoneArgument(argument, function_name); - return false; - } - checkIntervalArgument(argument, function_name, interval_kind, result_type_is_date); - return true; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. " + "Must be Tuple", function_name); } } +void checkFirstArgument(const ColumnWithTypeAndName & argument, const String & function_name) +{ + if (!isDateTime(argument.type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " + "Should be a date with time", argument.type->getName(), function_name); +} + +void checkIntervalArgument(const ColumnWithTypeAndName & argument, const String & function_name, IntervalKind & interval_kind, bool & result_type_is_date) +{ + const auto * interval_type = checkAndGetDataType(argument.type.get()); + if (!interval_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " + "Should be an interval of time", argument.type->getName(), function_name); + interval_kind = interval_type->getKind(); + result_type_is_date = (interval_type->getKind() == IntervalKind::Kind::Year) || (interval_type->getKind() == IntervalKind::Kind::Quarter) + || (interval_type->getKind() == IntervalKind::Kind::Month) || (interval_type->getKind() == IntervalKind::Kind::Week); +} + +void checkIntervalArgument(const ColumnWithTypeAndName & argument, const String & function_name, bool & result_type_is_date) +{ + IntervalKind interval_kind; + checkIntervalArgument(argument, function_name, interval_kind, result_type_is_date); +} + +void checkTimeZoneArgument( + const ColumnWithTypeAndName & argument, + const String & function_name) +{ + if (!WhichDataType(argument.type).isString()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " + "This argument is optional and must be a constant string with timezone name", + argument.type->getName(), function_name); +} + +bool checkIntervalOrTimeZoneArgument(const ColumnWithTypeAndName & argument, const String & function_name, IntervalKind & interval_kind, bool & result_type_is_date) +{ + if (WhichDataType(argument.type).isString()) + { + checkTimeZoneArgument(argument, function_name); + return false; + } + checkIntervalArgument(argument, function_name, interval_kind, result_type_is_date); + return true; +} + +template +struct TimeWindowImpl +{ + static constexpr auto name = "UNKNOWN"; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name); + + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name); +}; + +template +class FunctionTimeWindow : public IFunction +{ +public: + static constexpr auto name = TimeWindowImpl::name; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override; +}; + +using FunctionTumble = FunctionTimeWindow; +using FunctionTumbleStart = FunctionTimeWindow; +using FunctionTumbleEnd = FunctionTimeWindow; +using FunctionHop = FunctionTimeWindow; +using FunctionWindowId = FunctionTimeWindow; +using FunctionHopStart = FunctionTimeWindow; +using FunctionHopEnd = FunctionTimeWindow; + + template <> struct TimeWindowImpl { @@ -393,6 +430,9 @@ struct TimeWindowImpl wstart = AddTime::execute(wend, -window_num_units, time_zone); ToType wend_latest; + if (wstart > wend) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time overflow in function {}", name); + do { wend_latest = wend; @@ -414,7 +454,7 @@ struct TimeWindowImpl { static constexpr auto name = "windowID"; - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { bool result_type_is_date; IntervalKind interval_kind_1; @@ -456,8 +496,7 @@ struct TimeWindowImpl return std::make_shared(); } - [[maybe_unused]] static ColumnPtr - dispatchForHopColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + static ColumnPtr dispatchForHopColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { const auto & time_column = arguments[0]; const auto & hop_interval_column = arguments[1]; @@ -527,6 +566,9 @@ struct TimeWindowImpl ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); ToType wend_latest; + if (wstart > wend) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Time overflow in function {}", name); + do { wend_latest = wend; @@ -538,14 +580,13 @@ struct TimeWindowImpl return end; } - [[maybe_unused]] static ColumnPtr - dispatchForTumbleColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + static ColumnPtr dispatchForTumbleColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { ColumnPtr column = TimeWindowImpl::dispatchForColumns(arguments, function_name); return executeWindowBound(column, 1, function_name); } - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { if (arguments.size() == 2) return dispatchForTumbleColumns(arguments, function_name); @@ -585,7 +626,7 @@ struct TimeWindowImpl } } - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { const auto & time_column = arguments[0]; const auto which_type = WhichDataType(time_column.type); @@ -608,12 +649,12 @@ struct TimeWindowImpl { static constexpr auto name = "hopEnd"; - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { return TimeWindowImpl::getReturnType(arguments, function_name); } - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { const auto & time_column = arguments[0]; const auto which_type = WhichDataType(time_column.type); @@ -644,6 +685,8 @@ ColumnPtr FunctionTimeWindow::executeImpl(const ColumnsWithTypeAndName & a return TimeWindowImpl::dispatchForColumns(arguments, name); } +} + REGISTER_FUNCTION(TimeWindow) { factory.registerFunction(); diff --git a/src/Functions/FunctionsTimeWindow.h b/src/Functions/FunctionsTimeWindow.h index 658963cb69c..ab8deef08c0 100644 --- a/src/Functions/FunctionsTimeWindow.h +++ b/src/Functions/FunctionsTimeWindow.h @@ -156,39 +156,4 @@ template <> \ ADD_SUBSECONDS(Nanosecond, 9) #undef ADD_SUBSECONDS -template -struct TimeWindowImpl -{ - static constexpr auto name = "UNKNOWN"; - - static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name); - - static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name); -}; - -template -class FunctionTimeWindow : public IFunction -{ -public: - static constexpr auto name = TimeWindowImpl::name; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override { return name; } - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override; -}; - -using FunctionTumble = FunctionTimeWindow; -using FunctionTumbleStart = FunctionTimeWindow; -using FunctionTumbleEnd = FunctionTimeWindow; -using FunctionHop = FunctionTimeWindow; -using FunctionWindowId = FunctionTimeWindow; -using FunctionHopStart = FunctionTimeWindow; -using FunctionHopEnd = FunctionTimeWindow; } diff --git a/tests/queries/0_stateless/03013_hop_infinite_loop.reference b/tests/queries/0_stateless/03013_hop_infinite_loop.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03013_hop_infinite_loop.sql b/tests/queries/0_stateless/03013_hop_infinite_loop.sql new file mode 100644 index 00000000000..bdac87f5af1 --- /dev/null +++ b/tests/queries/0_stateless/03013_hop_infinite_loop.sql @@ -0,0 +1,2 @@ +SET allow_experimental_window_view = 1; +SELECT hop(toDateTime(0), INTERVAL 1 DAY, INTERVAL 3 DAY); -- { serverError BAD_ARGUMENTS }