mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #61523 from ClickHouse/fix-infinite-loop-hop
Fix infinite loop in function `hop`
This commit is contained in:
commit
ccd34a269b
@ -23,91 +23,129 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
std::tuple<IntervalKind::Kind, Int64>
|
||||
dispatchForIntervalColumns(const ColumnWithTypeAndName & interval_column, const String & function_name)
|
||||
{
|
||||
const auto * interval_type = checkAndGetDataType<DataTypeInterval>(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<ColumnInt64>(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<Int64>();
|
||||
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<IntervalKind::Kind, Int64>
|
||||
dispatchForIntervalColumns(const ColumnWithTypeAndName & interval_column, const String & function_name)
|
||||
{
|
||||
const auto * interval_type = checkAndGetDataType<DataTypeInterval>(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<ColumnInt64>(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<Int64>();
|
||||
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<ColumnTuple>(column.get()); col_tuple)
|
||||
{
|
||||
if (const ColumnTuple * col_tuple = checkAndGetColumn<ColumnTuple>(column.get()); col_tuple)
|
||||
{
|
||||
if (!checkColumn<ColumnVector<UInt32>>(*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<ColumnVector<UInt32>>(*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<DataTypeInterval>(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<DataTypeInterval>(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 <TimeWindowFunctionName type>
|
||||
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 <TimeWindowFunctionName type>
|
||||
class FunctionTimeWindow : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = TimeWindowImpl<type>::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionTimeWindow>(); }
|
||||
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<TUMBLE>;
|
||||
using FunctionTumbleStart = FunctionTimeWindow<TUMBLE_START>;
|
||||
using FunctionTumbleEnd = FunctionTimeWindow<TUMBLE_END>;
|
||||
using FunctionHop = FunctionTimeWindow<HOP>;
|
||||
using FunctionWindowId = FunctionTimeWindow<WINDOW_ID>;
|
||||
using FunctionHopStart = FunctionTimeWindow<HOP_START>;
|
||||
using FunctionHopEnd = FunctionTimeWindow<HOP_END>;
|
||||
|
||||
|
||||
template <>
|
||||
struct TimeWindowImpl<TUMBLE>
|
||||
{
|
||||
@ -393,6 +431,9 @@ struct TimeWindowImpl<HOP>
|
||||
wstart = AddTime<kind>::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 +455,7 @@ struct TimeWindowImpl<WINDOW_ID>
|
||||
{
|
||||
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 +497,7 @@ struct TimeWindowImpl<WINDOW_ID>
|
||||
return std::make_shared<DataTypeUInt32>();
|
||||
}
|
||||
|
||||
[[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 +567,9 @@ struct TimeWindowImpl<WINDOW_ID>
|
||||
ToType wend = AddTime<kind>::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 +581,13 @@ struct TimeWindowImpl<WINDOW_ID>
|
||||
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<TUMBLE>::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 +627,7 @@ struct TimeWindowImpl<HOP_START>
|
||||
}
|
||||
}
|
||||
|
||||
[[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 +650,12 @@ struct TimeWindowImpl<HOP_END>
|
||||
{
|
||||
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<HOP_START>::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 +686,8 @@ ColumnPtr FunctionTimeWindow<type>::executeImpl(const ColumnsWithTypeAndName & a
|
||||
return TimeWindowImpl<type>::dispatchForColumns(arguments, name);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
REGISTER_FUNCTION(TimeWindow)
|
||||
{
|
||||
factory.registerFunction<FunctionTumble>();
|
||||
|
@ -160,39 +160,4 @@ template <> \
|
||||
|
||||
/// NOLINTEND(bugprone-macro-parentheses)
|
||||
|
||||
template <TimeWindowFunctionName type>
|
||||
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 <TimeWindowFunctionName type>
|
||||
class FunctionTimeWindow : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = TimeWindowImpl<type>::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionTimeWindow>(); }
|
||||
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<TUMBLE>;
|
||||
using FunctionTumbleStart = FunctionTimeWindow<TUMBLE_START>;
|
||||
using FunctionTumbleEnd = FunctionTimeWindow<TUMBLE_END>;
|
||||
using FunctionHop = FunctionTimeWindow<HOP>;
|
||||
using FunctionWindowId = FunctionTimeWindow<WINDOW_ID>;
|
||||
using FunctionHopStart = FunctionTimeWindow<HOP_START>;
|
||||
using FunctionHopEnd = FunctionTimeWindow<HOP_END>;
|
||||
}
|
||||
|
2
tests/queries/0_stateless/03013_hop_infinite_loop.sql
Normal file
2
tests/queries/0_stateless/03013_hop_infinite_loop.sql
Normal file
@ -0,0 +1,2 @@
|
||||
SET allow_experimental_window_view = 1;
|
||||
SELECT hop(toDateTime(0), INTERVAL 1 DAY, INTERVAL 3 DAY); -- { serverError BAD_ARGUMENTS }
|
Loading…
Reference in New Issue
Block a user