Skip to content

Commit

Permalink
Merge pull request #60814 from ClickHouse/backport/23.3/60764
Browse files Browse the repository at this point in the history
Backport #60764 to 23.3: Fix crash in arrayEnumerateRanked
  • Loading branch information
Algunenano committed Mar 5, 2024
2 parents f6c8a7c + f2f6120 commit 420d125
Show file tree
Hide file tree
Showing 5 changed files with 102 additions and 77 deletions.
137 changes: 77 additions & 60 deletions src/Functions/array/arrayEnumerateRanked.cpp
Original file line number Diff line number Diff line change
@@ -1,8 +1,8 @@
#include <algorithm>
#include <Columns/ColumnConst.h>
#include <Functions/array/arrayEnumerateRanked.h>
#include <Common/assert_cast.h>
#include "arrayEnumerateRanked.h"

#include <algorithm>

namespace DB
{
Expand All @@ -12,88 +12,105 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}

ArraysDepths getArraysDepths(const ColumnsWithTypeAndName & arguments)
ArraysDepths getArraysDepths(const ColumnsWithTypeAndName & arguments, const char * function_name)
{
const size_t num_arguments = arguments.size();
if (!num_arguments)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Missing arguments for function arrayEnumerateUniqRanked");

DepthType clear_depth = 1;
DepthTypes depths;
size_t i = 0;
if (const DataTypeArray * type_array = typeid_cast<const DataTypeArray *>(arguments[0].type.get()); !type_array)
{
/// If the first argument is not an array, it must be a const positive and non zero number
const auto & depth_column = arguments[i].column;
if (!depth_column || !isColumnConst(*depth_column))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument of {} must be Const(UInt64)", function_name);
Field f = assert_cast<const ColumnConst &>(*depth_column).getField();
if (f.getType() != Field::Types::UInt64 || f.safeGet<UInt64>() == 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument of {} must be a positive integer", function_name);

clear_depth = static_cast<DepthType>(f.safeGet<UInt64>());
i++;
}

/// function signature is the following:
/// f(c0, arr1, c1, arr2, c2, ...)
///
/// c0 is something called "clear_depth" here.
/// cN... - how deep to look into the corresponding arrN, (called "depths" here)
/// may be omitted - then it means "look at the full depth".

size_t array_num = 0;
DepthType prev_array_depth = 0;
for (size_t i = 0; i < num_arguments; ++i)
/// The rest of the arguments must be in the shape: arr1, c1, arr2, c2, ...
/// cN... - how deep to look into the corresponding arrN, (called "depths" here)
/// may be omitted - then it means "look at the full depth"
DepthTypes depths;
for (; i < num_arguments; i++)
{
const DataTypePtr & type = arguments[i].type;
const DataTypeArray * type_array = typeid_cast<const DataTypeArray *>(type.get());

if (type_array)
const DataTypeArray * current_type_array = typeid_cast<const DataTypeArray *>(type.get());
if (!current_type_array)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Incorrect argument {} type of function {}. Expected an Array, got {}",
i + 1,
function_name,
type->getName());

if (i == num_arguments - 1)
{
if (depths.size() < array_num && prev_array_depth)
depths.emplace_back(prev_array_depth);

prev_array_depth = static_cast<DepthType>(type_array->getNumberOfDimensions());
++array_num;
depths.emplace_back(current_type_array->getNumberOfDimensions());
}
else
{
const auto & depth_column = arguments[i].column;

if (depth_column && isColumnConst(*depth_column))
const DataTypeArray * next_argument_array = typeid_cast<const DataTypeArray *>(arguments[i + 1].type.get());
if (next_argument_array)
{
UInt64 value = assert_cast<const ColumnConst &>(*depth_column).getValue<UInt64>();
if (!value)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Incorrect arguments for function arrayEnumerateUniqRanked "
"or arrayEnumerateDenseRanked: depth ({}) cannot be less or equal 0.",
std::to_string(value));

if (i == 0)
{
clear_depth = static_cast<DepthType>(value);
}
else
{
if (depths.size() >= array_num)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Incorrect arguments for function arrayEnumerateUniqRanked "
"or arrayEnumerateDenseRanked: depth ({}) for missing array.",
std::to_string(value));
if (value > prev_array_depth)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Arguments for function arrayEnumerateUniqRanked/arrayEnumerateDenseRanked incorrect: depth={}"
" for array with depth={}.",
std::to_string(value), std::to_string(prev_array_depth));

depths.emplace_back(value);
}
depths.emplace_back(current_type_array->getNumberOfDimensions());
}
else
{
i++;
/// The following argument is not array, so it must be a const positive integer with the depth
const auto & depth_column = arguments[i].column;
if (!depth_column || !isColumnConst(*depth_column))
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Incorrect argument {} type of function {}. Expected an Array or Const(UInt64), got {}",
i + 1,
function_name,
arguments[i].type->getName());
Field f = assert_cast<const ColumnConst &>(*depth_column).getField();
if (f.getType() != Field::Types::UInt64 || f.safeGet<UInt64>() == 0)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Incorrect argument {} of function {}. Expected a positive integer",
i + 1,
function_name);
UInt64 value = f.safeGet<UInt64>();
UInt64 prev_array_depth = current_type_array->getNumberOfDimensions();
if (value > prev_array_depth)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Incorrect argument {} of function {}. Required depth '{}' is larger than the array depth ({})",
i + 1,
function_name,
value,
prev_array_depth);
depths.emplace_back(value);
}
}
}

if (depths.size() < array_num)
depths.emplace_back(prev_array_depth);

if (depths.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Incorrect arguments for function arrayEnumerateUniqRanked or arrayEnumerateDenseRanked: "
"at least one array should be passed.");
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Incorrect arguments for function {}: At least one array should be passed", function_name);

DepthType max_array_depth = 0;
for (auto depth : depths)
max_array_depth = std::max(depth, max_array_depth);

if (clear_depth > max_array_depth)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Incorrect arguments for function arrayEnumerateUniqRanked or arrayEnumerateDenseRanked: "
"clear_depth ({}) can't be larger than max_array_depth ({}).",
std::to_string(clear_depth), std::to_string(max_array_depth));
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Incorrect arguments for function {}: clear_depth ({}) can't be larger than max_array_depth ({})",
function_name,
clear_depth,
max_array_depth);

return {clear_depth, depths, max_array_depth};
}
Expand Down
22 changes: 14 additions & 8 deletions src/Functions/array/arrayEnumerateRanked.h
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ struct ArraysDepths
};

/// Return depth info about passed arrays
ArraysDepths getArraysDepths(const ColumnsWithTypeAndName & arguments);
ArraysDepths getArraysDepths(const ColumnsWithTypeAndName & arguments, const char * function_name);

template <typename Derived>
class FunctionArrayEnumerateRankedExtended : public IFunction
Expand All @@ -105,7 +105,7 @@ class FunctionArrayEnumerateRankedExtended : public IFunction
"Number of arguments for function {} doesn't match: passed {}, should be at least 1.",
getName(), arguments.size());

const ArraysDepths arrays_depths = getArraysDepths(arguments);
const ArraysDepths arrays_depths = getArraysDepths(arguments, Derived::name);

/// Return type is the array of the depth as the maximum effective depth of arguments, containing UInt32.

Expand Down Expand Up @@ -158,7 +158,7 @@ ColumnPtr FunctionArrayEnumerateRankedExtended<Derived>::executeImpl(
Columns array_holders;
ColumnPtr offsets_column;

const ArraysDepths arrays_depths = getArraysDepths(arguments);
const ArraysDepths arrays_depths = getArraysDepths(arguments, Derived::name);

/// If the column is Array - return it. If the const Array - materialize it, keep ownership and return.
auto get_array_column = [&](const auto & column) -> const DB::ColumnArray *
Expand Down Expand Up @@ -217,17 +217,23 @@ ColumnPtr FunctionArrayEnumerateRankedExtended<Derived>::executeImpl(
{
if (*offsets_by_depth[col_depth] != array->getOffsets())
{
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH,
"Lengths and effective depths of all arrays passed to {} must be equal.", getName());
throw Exception(
ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH,
"Lengths and effective depths of all arrays passed to {} must be equal",
getName());
}
}
}

if (col_depth < arrays_depths.depths[array_num])
{
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH,
"{}: Passed array number {} depth ({}) is more than the actual array depth ({}).",
getName(), array_num, std::to_string(arrays_depths.depths[array_num]), col_depth);
throw Exception(
ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH,
"{}: Passed array number {} depth ({}) is more than the actual array depth ({})",
getName(),
array_num,
std::to_string(arrays_depths.depths[array_num]),
col_depth);
}

auto * array_data = &array->getData();
Expand Down
18 changes: 9 additions & 9 deletions tests/queries/0_stateless/00909_arrayEnumerateUniq.sql
Original file line number Diff line number Diff line change
Expand Up @@ -181,15 +181,15 @@ SELECT arrayEnumerateUniqRanked([1,2], 1, 2); -- { serverError 36 }
SELECT arrayEnumerateUniqRanked([1,2], 1, 3, 4, 5); -- { serverError 36 }
SELECT arrayEnumerateUniqRanked([1,2], 1, 3, [4], 5); -- { serverError 36 }
SELECT arrayEnumerateDenseRanked([[[[[[[[[[42]]]]]]]]]]);
SELECT arrayEnumerateUniqRanked('wat', [1,2]); -- { serverError 170 }
SELECT arrayEnumerateUniqRanked(1, [1,2], 'boom'); -- { serverError 170 }
SELECT arrayEnumerateDenseRanked(['\0'], -8363126); -- { serverError 170 }
SELECT arrayEnumerateDenseRanked(-10, ['\0'], -8363126); -- { serverError 170 }
SELECT arrayEnumerateDenseRanked(1, ['\0'], -8363126); -- { serverError 170 }
SELECT arrayEnumerateDenseRanked(-101, ['\0']); -- { serverError 170 }
SELECT arrayEnumerateDenseRanked(1.1, [10,20,10,30]); -- { serverError 170 }
SELECT arrayEnumerateDenseRanked([10,20,10,30], 0.4); -- { serverError 170 }
SELECT arrayEnumerateDenseRanked([10,20,10,30], 1.8); -- { serverError 170 }
SELECT arrayEnumerateUniqRanked('wat', [1,2]); -- { serverError BAD_ARGUMENTS }
SELECT arrayEnumerateUniqRanked(1, [1,2], 'boom'); -- { serverError BAD_ARGUMENTS }
SELECT arrayEnumerateDenseRanked(['\0'], -8363126); -- { serverError BAD_ARGUMENTS }
SELECT arrayEnumerateDenseRanked(-10, ['\0'], -8363126); -- { serverError BAD_ARGUMENTS }
SELECT arrayEnumerateDenseRanked(1, ['\0'], -8363126); -- { serverError BAD_ARGUMENTS }
SELECT arrayEnumerateDenseRanked(-101, ['\0']); -- { serverError BAD_ARGUMENTS }
SELECT arrayEnumerateDenseRanked(1.1, [10,20,10,30]); -- { serverError BAD_ARGUMENTS }
SELECT arrayEnumerateDenseRanked([10,20,10,30], 0.4); -- { serverError BAD_ARGUMENTS }
SELECT arrayEnumerateDenseRanked([10,20,10,30], 1.8); -- { serverError BAD_ARGUMENTS }
SELECT arrayEnumerateUniqRanked(1, [], 1000000000); -- { serverError 36 }


Expand Down
Empty file.
2 changes: 2 additions & 0 deletions tests/queries/0_stateless/03003_arrayEnumerate_crash.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
SELECT arrayEnumerateUniqRanked(arrayEnumerateUniqRanked([[1, 2, 3], [2, 2, 1], [3]]), materialize(1 AS x) OR toLowCardinality(-9223372036854775808)); -- { serverError BAD_ARGUMENTS }
SELECT arrayEnumerateUniqRanked([[1, 2, 3], [2, 2, 1], [3]], number) FROM numbers(10); -- { serverError BAD_ARGUMENTS }

0 comments on commit 420d125

Please sign in to comment.