diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index 3d7be1e9159..5e5ec9db825 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -21,25 +21,25 @@ namespace DB /** Window functions: * * TUMBLE(time_attr, interval [, timezone]) - * + * * TUMBLE_START(window_id) - * + * * TUMBLE_START(time_attr, interval [, timezone]) - * + * * TUMBLE_END(window_id) - * + * * TUMBLE_END(time_attr, interval [, timezone]) - * + * * HOP(time_attr, hop_interval, window_interval [, timezone]) - * + * * HOP_START(window_id) - * + * * HOP_START(time_attr, hop_interval, window_interval [, timezone]) - * + * * HOP_END(window_id) - * + * * HOP_END(time_attr, hop_interval, window_interval [, timezone]) - * + * */ enum WindowFunctionName { @@ -137,7 +137,7 @@ struct ToStartOfTransform; namespace { - static std::tuple + std::tuple dispatchForIntervalColumns(const ColumnWithTypeAndName & interval_column, const String & function_name) { const auto * interval_type = checkAndGetDataType(interval_column.type.get()); @@ -157,7 +157,7 @@ namespace return {interval_type->getKind(), num_units}; } - static ColumnPtr executeWindowBound(const ColumnPtr & column, int index, const String & function_name) + ColumnPtr executeWindowBound(const ColumnPtr & column, int index, const String & function_name) { if (const ColumnTuple * col_tuple = checkAndGetColumn(column.get()); col_tuple) { @@ -224,523 +224,521 @@ namespace checkIntervalArgument(argument, function_name, interval_kind, result_type_is_date); return true; } +} +template +struct WindowImpl +{ + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name); +}; - template - struct WindowImpl +template <> +struct WindowImpl +{ + static constexpr auto name = "TUMBLE"; + + [[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; - template <> - struct WindowImpl - { - static constexpr auto name = "TUMBLE"; - - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + if (arguments.size() == 2) { - bool result_type_is_date; + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); + } + else if (arguments.size() == 3) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); + checkTimeZoneArgument(arguments.at(2), function_name); + } + else + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 2 or 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } - if (arguments.size() == 2) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); - } - else if (arguments.size() == 3) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); - checkTimeZoneArgument(arguments.at(2), function_name); - } - else - { + DataTypePtr dataType = nullptr; + if (result_type_is_date) + dataType = std::make_shared(); + else + dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); + + return std::make_shared(DataTypes{dataType, dataType}); + } + + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + const auto & time_column = arguments[0]; + const auto & interval_column = arguments[1]; + const auto & from_datatype = *time_column.type.get(); + const auto which_type = WhichDataType(from_datatype); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); + if (!which_type.isDateTime() || !time_column_vec) + throw Exception( + "Illegal column " + time_column.name + " of function " + function_name + ". Must contain dates or dates with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto interval = dispatchForIntervalColumns(interval_column, function_name); + + switch (std::get<0>(interval)) + { + case IntervalKind::Second: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Minute: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Hour: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Day: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Week: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Month: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Quarter: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Year: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + } + __builtin_unreachable(); + } + + template + static ColumnPtr execute_tumble(const ColumnUInt32 & time_column, UInt64 num_units, const DateLUTImpl & time_zone) + { + const auto & time_data = time_column.getData(); + size_t size = time_column.size(); + auto start = ColumnVector::create(); + auto end = ColumnVector::create(); + auto & start_data = start->getData(); + auto & end_data = end->getData(); + start_data.resize(size); + end_data.resize(size); + for (size_t i = 0; i != size; ++i) + { + start_data[i] = ToStartOfTransform::execute(time_data[i], num_units, time_zone); + end_data[i] = AddTime::execute(start_data[i], num_units, time_zone); + } + MutableColumns result; + result.emplace_back(std::move(start)); + result.emplace_back(std::move(end)); + return ColumnTuple::create(std::move(result)); + } +}; + +template <> +struct WindowImpl +{ + static constexpr auto name = "TUMBLE_START"; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + if (arguments.size() == 1) + { + auto type_ = WhichDataType(arguments[0].type); + if (!type_.isTuple() && !type_.isUInt32()) throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 2 or 3", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - - DataTypePtr dataType = nullptr; - if (result_type_is_date) - dataType = std::make_shared(); - else - dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); - - return std::make_shared(DataTypes{dataType, dataType}); + "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", ErrorCodes::ILLEGAL_COLUMN); } - - static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + else { - const auto & time_column = arguments[0]; - const auto & interval_column = arguments[1]; - const auto & from_datatype = *time_column.type.get(); - const auto which_type = WhichDataType(from_datatype); - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); - const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); - if (!which_type.isDateTime() || !time_column_vec) - throw Exception( - "Illegal column " + time_column.name + " of function " + function_name + ". Must contain dates or dates with time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - auto interval = dispatchForIntervalColumns(interval_column, function_name); - - switch (std::get<0>(interval)) - { - case IntervalKind::Second: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Minute: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Hour: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Day: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Week: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Month: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Quarter: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Year: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - } - __builtin_unreachable(); + WindowImpl::getReturnType(arguments, function_name); } + return std::make_shared(); + } - template - static ColumnPtr execute_tumble(const ColumnUInt32 & time_column, UInt64 num_units, const DateLUTImpl & time_zone) - { - const auto & time_data = time_column.getData(); - size_t size = time_column.size(); - auto start = ColumnVector::create(); - auto end = ColumnVector::create(); - auto & start_data = start->getData(); - auto & end_data = end->getData(); - start_data.resize(size); - end_data.resize(size); - for (size_t i = 0; i != size; ++i) - { - start_data[i] = ToStartOfTransform::execute(time_data[i], num_units, time_zone); - end_data[i] = AddTime::execute(start_data[i], num_units, time_zone); - } - MutableColumns result; - result.emplace_back(std::move(start)); - result.emplace_back(std::move(end)); - return ColumnTuple::create(std::move(result)); - } - }; - - template <> - struct WindowImpl + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { - static constexpr auto name = "TUMBLE_START"; + const auto which_type = WhichDataType(arguments[0].type); + ColumnPtr result_column_; + if (which_type.isDateTime()) + result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); + else + result_column_ = arguments[0].column; + return executeWindowBound(result_column_, 0, function_name); + } +}; - static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) +template <> +struct WindowImpl +{ + static constexpr auto name = "TUMBLE_END"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + return WindowImpl::getReturnType(arguments, function_name); + } + + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) + { + const auto which_type = WhichDataType(arguments[0].type); + ColumnPtr result_column_; + if (which_type.isDateTime()) + result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); + else + result_column_ = arguments[0].column; + return executeWindowBound(result_column_, 1, function_name); + } +}; + +template <> +struct WindowImpl +{ + static constexpr auto name = "HOP"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + bool result_type_is_date; + IntervalKind interval_kind_1; + IntervalKind interval_kind_2; + + if (arguments.size() == 3) { - if (arguments.size() == 1) + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); + } + else if (arguments.size() == 4) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); + checkTimeZoneArgument(arguments.at(3), function_name); + } + else + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 3 or 4", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + if (interval_kind_1 != interval_kind_2) + throw Exception( + "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); + + DataTypePtr dataType = nullptr; + if (result_type_is_date) + dataType = std::make_shared(); + else + dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0)); + return std::make_shared(DataTypes{dataType, dataType}); + } + + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + const auto & time_column = arguments[0]; + const auto & hop_interval_column = arguments[1]; + const auto & window_interval_column = arguments[2]; + const auto & from_datatype = *time_column.type.get(); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); + if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) + throw Exception( + "Illegal column " + time_column.name + " argument of function " + function_name + + ". Must contain dates or dates with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); + auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); + + if (std::get<1>(hop_interval) > std::get<1>(window_interval)) + throw Exception( + "Value for hop interval of function " + function_name + " must not larger than window interval", + ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + switch (std::get<0>(window_interval)) + { + case IntervalKind::Second: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Minute: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Hour: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Day: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Week: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Month: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Quarter: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Year: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + } + __builtin_unreachable(); + } + + template + static ColumnPtr + execute_hop(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) + { + const auto & time_data = time_column.getData(); + size_t size = time_column.size(); + auto start = ColumnVector::create(); + auto end = ColumnVector::create(); + auto & start_data = start->getData(); + auto & end_data = end->getData(); + start_data.resize(size); + end_data.resize(size); + + for (size_t i = 0; i < size; ++i) + { + ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); + ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); + wstart = AddTime::execute(wend, -1 * window_num_units, time_zone); + + ToType wend_ = wend; + ToType wend_latest; + + do { - auto type_ = WhichDataType(arguments[0].type); - if (!type_.isTuple() && !type_.isUInt32()) + wend_latest = wend_; + wend_ = AddTime::execute(wend_, -1 * hop_num_units, time_zone); + } while (wend_ > time_data[i]); + + end_data[i] = wend_latest; + start_data[i] = AddTime::execute(wend_latest, -1 * window_num_units, time_zone); + } + MutableColumns result; + result.emplace_back(std::move(start)); + result.emplace_back(std::move(end)); + return ColumnTuple::create(std::move(result)); + } +}; + +template <> +struct WindowImpl +{ + static constexpr auto name = "WINDOW_ID"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + bool result_type_is_date; + IntervalKind interval_kind_1; + IntervalKind interval_kind_2; + + if (arguments.size() == 2) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + } + else if (arguments.size() == 3) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + if (checkIntervalOrTimeZoneArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date)) + { + if (interval_kind_1 != interval_kind_2) throw Exception( - "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", ErrorCodes::ILLEGAL_COLUMN); + "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); } - else - { - WindowImpl::getReturnType(arguments, function_name); - } - return std::make_shared(); } - - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + else if (arguments.size() == 4) { - const auto which_type = WhichDataType(arguments[0].type); - ColumnPtr result_column_; - if (which_type.isDateTime()) - result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); - else - result_column_ = arguments[0].column; - return executeWindowBound(result_column_, 0, function_name); + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); + checkTimeZoneArgument(arguments.at(3), function_name); + } + else + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 2, 3 or 4", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } - }; + if (result_type_is_date) + return std::make_shared(); + else + return std::make_shared(); + } - template <> - struct WindowImpl + [[maybe_unused]] static ColumnPtr + dispatchForHopColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { - static constexpr auto name = "TUMBLE_END"; + const auto & time_column = arguments[0]; + const auto & hop_interval_column = arguments[1]; + const auto & window_interval_column = arguments[2]; + const auto & from_datatype = *time_column.type.get(); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); + if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) + throw Exception( + "Illegal column " + time_column.name + " argument of function " + function_name + + ". Must contain dates or dates with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); + auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); + + if (std::get<1>(hop_interval) > std::get<1>(window_interval)) + throw Exception( + "Value for hop interval of function " + function_name + " must not larger than window interval", + ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + switch (std::get<0>(window_interval)) { - return WindowImpl::getReturnType(arguments, function_name); + case IntervalKind::Second: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Minute: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Hour: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Day: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Week: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Month: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Quarter: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Year: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); } + __builtin_unreachable(); + } - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) - { - const auto which_type = WhichDataType(arguments[0].type); - ColumnPtr result_column_; - if (which_type.isDateTime()) - result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); - else - result_column_ = arguments[0].column; - return executeWindowBound(result_column_, 1, function_name); - } - }; - - template <> - struct WindowImpl + template + static ColumnPtr + execute_hop_slice(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) { - static constexpr auto name = "HOP"; + Int64 gcd_num_units = std::gcd(hop_num_units, window_num_units); - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + const auto & time_data = time_column.getData(); + size_t size = time_column.size(); + + auto end = ColumnVector::create(); + auto & end_data = end->getData(); + end_data.resize(size); + for (size_t i = 0; i < size; ++i) { - bool result_type_is_date; - IntervalKind interval_kind_1; - IntervalKind interval_kind_2; + ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); + ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); - if (arguments.size() == 3) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); - } - else if (arguments.size() == 4) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); - checkTimeZoneArgument(arguments.at(3), function_name); - } - else - { - throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 3 or 4", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } + ToType wend_ = wend; + ToType wend_latest; - if (interval_kind_1 != interval_kind_2) - throw Exception( - "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); + do + { + wend_latest = wend_; + wend_ = AddTime::execute(wend_, -1 * gcd_num_units, time_zone); + } while (wend_ > time_data[i]); - DataTypePtr dataType = nullptr; - if (result_type_is_date) - dataType = std::make_shared(); - else - dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0)); - return std::make_shared(DataTypes{dataType, dataType}); + end_data[i] = wend_latest; } + return end; + } - static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - const auto & time_column = arguments[0]; - const auto & hop_interval_column = arguments[1]; - const auto & window_interval_column = arguments[2]; - const auto & from_datatype = *time_column.type.get(); - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); - const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); - if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) - throw Exception( - "Illegal column " + time_column.name + " argument of function " + function_name - + ". Must contain dates or dates with time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); - auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); - - if (std::get<1>(hop_interval) > std::get<1>(window_interval)) - throw Exception( - "Value for hop interval of function " + function_name + " must not larger than window interval", - ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - switch (std::get<0>(window_interval)) - { - case IntervalKind::Second: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Minute: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Hour: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Day: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Week: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Month: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Quarter: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Year: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - } - __builtin_unreachable(); - } - - template - static ColumnPtr - execute_hop(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) - { - const auto & time_data = time_column.getData(); - size_t size = time_column.size(); - auto start = ColumnVector::create(); - auto end = ColumnVector::create(); - auto & start_data = start->getData(); - auto & end_data = end->getData(); - start_data.resize(size); - end_data.resize(size); - - for (size_t i = 0; i < size; ++i) - { - ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); - ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); - wstart = AddTime::execute(wend, -1 * window_num_units, time_zone); - - ToType wend_ = wend; - ToType wend_latest; - - do - { - wend_latest = wend_; - wend_ = AddTime::execute(wend_, -1 * hop_num_units, time_zone); - } while (wend_ > time_data[i]); - - end_data[i] = wend_latest; - start_data[i] = AddTime::execute(wend_latest, -1 * window_num_units, time_zone); - } - MutableColumns result; - result.emplace_back(std::move(start)); - result.emplace_back(std::move(end)); - return ColumnTuple::create(std::move(result)); - } - }; - - template <> - struct WindowImpl + [[maybe_unused]] static ColumnPtr + dispatchForTumbleColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { - static constexpr auto name = "WINDOW_ID"; + ColumnPtr column = WindowImpl::dispatchForColumns(arguments, function_name); + return executeWindowBound(column, 1, function_name); + } - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + if (arguments.size() == 2) + return dispatchForTumbleColumns(arguments, function_name); + else { - bool result_type_is_date; - IntervalKind interval_kind_1; - IntervalKind interval_kind_2; - - if (arguments.size() == 2) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - } - else if (arguments.size() == 3) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - if (checkIntervalOrTimeZoneArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date)) - { - if (interval_kind_1 != interval_kind_2) - throw Exception( - "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); - } - } - else if (arguments.size() == 4) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); - checkTimeZoneArgument(arguments.at(3), function_name); - } - else - { - throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 2, 3 or 4", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - - if (result_type_is_date) - return std::make_shared(); - else - return std::make_shared(); - } - - [[maybe_unused]] static ColumnPtr - dispatchForHopColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - const auto & time_column = arguments[0]; - const auto & hop_interval_column = arguments[1]; - const auto & window_interval_column = arguments[2]; - const auto & from_datatype = *time_column.type.get(); - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); - const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); - if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) - throw Exception( - "Illegal column " + time_column.name + " argument of function " + function_name - + ". Must contain dates or dates with time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); - auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); - - if (std::get<1>(hop_interval) > std::get<1>(window_interval)) - throw Exception( - "Value for hop interval of function " + function_name + " must not larger than window interval", - ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - switch (std::get<0>(window_interval)) - { - case IntervalKind::Second: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Minute: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Hour: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Day: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Week: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Month: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Quarter: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Year: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - } - __builtin_unreachable(); - } - - template - static ColumnPtr - execute_hop_slice(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) - { - Int64 gcd_num_units = std::gcd(hop_num_units, window_num_units); - - const auto & time_data = time_column.getData(); - size_t size = time_column.size(); - - auto end = ColumnVector::create(); - auto & end_data = end->getData(); - end_data.resize(size); - for (size_t i = 0; i < size; ++i) - { - ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); - ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); - - ToType wend_ = wend; - ToType wend_latest; - - do - { - wend_latest = wend_; - wend_ = AddTime::execute(wend_, -1 * gcd_num_units, time_zone); - } while (wend_ > time_data[i]); - - end_data[i] = wend_latest; - } - return end; - } - - [[maybe_unused]] static ColumnPtr - dispatchForTumbleColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - ColumnPtr column = WindowImpl::dispatchForColumns(arguments, function_name); - return executeWindowBound(column, 1, function_name); - } - - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - if (arguments.size() == 2) + const auto & third_column = arguments[2]; + if (arguments.size() == 3 && WhichDataType(third_column.type).isString()) return dispatchForTumbleColumns(arguments, function_name); else - { - const auto & third_column = arguments[2]; - if (arguments.size() == 3 && WhichDataType(third_column.type).isString()) - return dispatchForTumbleColumns(arguments, function_name); - else - return dispatchForHopColumns(arguments, function_name); - } + return dispatchForHopColumns(arguments, function_name); } - }; + } +}; - template <> - struct WindowImpl +template <> +struct WindowImpl +{ + static constexpr auto name = "HOP_START"; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { - static constexpr auto name = "HOP_START"; - - static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + if (arguments.size() == 1) { - if (arguments.size() == 1) - { - auto type_ = WhichDataType(arguments[0].type); - if (!type_.isTuple() && !type_.isUInt32()) - throw Exception( - "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", - ErrorCodes::ILLEGAL_COLUMN); - } - else - { - WindowImpl::getReturnType(arguments, function_name); - } - return std::make_shared(); + auto type_ = WhichDataType(arguments[0].type); + if (!type_.isTuple() && !type_.isUInt32()) + throw Exception( + "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", + ErrorCodes::ILLEGAL_COLUMN); } - - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + else { - const auto & time_column = arguments[0]; - const auto which_type = WhichDataType(time_column.type); - ColumnPtr result_column_; - if (arguments.size() == 1) - { - if (which_type.isUInt32()) - return time_column.column; - else //isTuple - result_column_ = time_column.column; - } - else - result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); - return executeWindowBound(result_column_, 0, function_name); + WindowImpl::getReturnType(arguments, function_name); } - }; + return std::make_shared(); + } - template <> - struct WindowImpl + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { - static constexpr auto name = "HOP_END"; - - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + const auto & time_column = arguments[0]; + const auto which_type = WhichDataType(time_column.type); + ColumnPtr result_column_; + if (arguments.size() == 1) { - return WindowImpl::getReturnType(arguments, function_name); + if (which_type.isUInt32()) + return time_column.column; + else //isTuple + result_column_ = time_column.column; } + else + result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); + return executeWindowBound(result_column_, 0, function_name); + } +}; - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) +template <> +struct WindowImpl +{ + static constexpr auto name = "HOP_END"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + return WindowImpl::getReturnType(arguments, function_name); + } + + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + const auto & time_column = arguments[0]; + const auto which_type = WhichDataType(time_column.type); + ColumnPtr result_column_; + if (arguments.size() == 1) { - const auto & time_column = arguments[0]; - const auto which_type = WhichDataType(time_column.type); - ColumnPtr result_column_; - if (arguments.size() == 1) - { - if (which_type.isUInt32()) - return time_column.column; - else //isTuple - result_column_ = time_column.column; - } - else - result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); - - return executeWindowBound(result_column_, 1, function_name); + if (which_type.isUInt32()) + return time_column.column; + else //isTuple + result_column_ = time_column.column; } - }; + else + result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); + + return executeWindowBound(result_column_, 1, function_name); + } }; template diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index e88150ea515..d7fdf759b3b 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -165,8 +165,8 @@ namespace { if (auto * t = ast->as()) { - if (t->name == "HOP" || t->name == "TUMBLE") - t->name = "WINDOW_ID"; + if (t->name == "HOP" || t->name == "TUMBLE") + t->name = "WINDOW_ID"; } } }; @@ -205,7 +205,10 @@ namespace static void visit(const ASTIdentifier & node, ASTPtr & node_ptr, Data & data) { if (node.getColumnName() == data.window_id_alias) - dynamic_cast(node_ptr.get())->setShortName(data.window_id_name); + { + if (auto identifier = std::dynamic_pointer_cast(node_ptr)) + identifier->setShortName(data.window_id_name); + } } }; diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql old mode 100755 new mode 100644 diff --git a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql old mode 100755 new mode 100644 diff --git a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql old mode 100755 new mode 100644 diff --git a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql old mode 100755 new mode 100644