Merge pull request #70307 from ClickHouse/backport/24.8/70207

Backport #70207 to 24.8: Fix multiple issues with arrayMin and arrayMax
This commit is contained in:
robot-ch-test-poll2 2024-10-03 18:19:07 +04:00 committed by GitHub
commit a5332294f1
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
3 changed files with 145 additions and 81 deletions

View File

@ -104,7 +104,7 @@ struct ArrayAggregateImpl
static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypePtr & /*array_element*/)
{
if (aggregate_operation == AggregateOperation::max || aggregate_operation == AggregateOperation::min)
if constexpr (aggregate_operation == AggregateOperation::max || aggregate_operation == AggregateOperation::min)
{
return expression_return;
}
@ -152,9 +152,62 @@ struct ArrayAggregateImpl
return result;
}
template <AggregateOperation op = aggregate_operation>
requires(op == AggregateOperation::min || op == AggregateOperation::max)
static void executeMinOrMax(const ColumnPtr & mapped, const ColumnArray::Offsets & offsets, ColumnPtr & res_ptr)
{
const ColumnConst * const_column = checkAndGetColumn<ColumnConst>(&*mapped);
if (const_column)
{
MutableColumnPtr res_column = const_column->getDataColumn().cloneEmpty();
res_column->insertMany(const_column->getField(), offsets.size());
res_ptr = std::move(res_column);
return;
}
MutableColumnPtr res_column = mapped->cloneEmpty();
static constexpr int nan_null_direction_hint = aggregate_operation == AggregateOperation::min ? 1 : -1;
/// TODO: Introduce row_begin and row_end to getPermutation or an equivalent function to use that instead
/// (same use case as SingleValueDataBase::getSmallestIndex)
UInt64 start_of_array = 0;
for (auto end_of_array : offsets)
{
/// Array is empty
if (start_of_array == end_of_array)
{
res_column->insertDefault();
continue;
}
UInt64 index = start_of_array;
for (UInt64 i = index + 1; i < end_of_array; i++)
{
if constexpr (aggregate_operation == AggregateOperation::min)
{
if ((mapped->compareAt(i, index, *mapped, nan_null_direction_hint) < 0))
index = i;
}
else
{
if ((mapped->compareAt(i, index, *mapped, nan_null_direction_hint) > 0))
index = i;
}
}
res_column->insertFrom(*mapped, index);
start_of_array = end_of_array;
}
chassert(res_column->size() == offsets.size());
res_ptr = std::move(res_column);
}
template <typename Element>
static NO_SANITIZE_UNDEFINED bool executeType(const ColumnPtr & mapped, const ColumnArray::Offsets & offsets, ColumnPtr & res_ptr)
{
/// Min and Max are implemented in a different function
static_assert(aggregate_operation != AggregateOperation::min && aggregate_operation != AggregateOperation::max);
using ResultType = ArrayAggregateResult<Element, aggregate_operation>;
using ColVecType = ColumnVectorOrDecimal<Element>;
using ColVecResultType = ColumnVectorOrDecimal<ResultType>;
@ -197,11 +250,6 @@ struct ArrayAggregateImpl
/// Just multiply the value by array size.
res[i] = x * static_cast<ResultType>(array_size);
}
else if constexpr (aggregate_operation == AggregateOperation::min ||
aggregate_operation == AggregateOperation::max)
{
res[i] = x;
}
else if constexpr (aggregate_operation == AggregateOperation::average)
{
if constexpr (is_decimal<Element>)
@ -292,20 +340,6 @@ struct ArrayAggregateImpl
{
aggregate_value += element;
}
else if constexpr (aggregate_operation == AggregateOperation::min)
{
if (element < aggregate_value)
{
aggregate_value = element;
}
}
else if constexpr (aggregate_operation == AggregateOperation::max)
{
if (element > aggregate_value)
{
aggregate_value = element;
}
}
else if constexpr (aggregate_operation == AggregateOperation::product)
{
if constexpr (is_decimal<Element>)
@ -360,74 +394,41 @@ struct ArrayAggregateImpl
static ColumnPtr execute(const ColumnArray & array, ColumnPtr mapped)
{
if constexpr (aggregate_operation == AggregateOperation::max || aggregate_operation == AggregateOperation::min)
{
MutableColumnPtr res;
const auto & column = array.getDataPtr();
const ColumnConst * const_column = checkAndGetColumn<ColumnConst>(&*column);
if (const_column)
{
res = const_column->getDataColumn().cloneEmpty();
}
else
{
res = column->cloneEmpty();
}
const IColumn::Offsets & offsets = array.getOffsets();
size_t pos = 0;
for (const auto & offset : offsets)
{
if (offset == pos)
{
res->insertDefault();
continue;
}
size_t current_max_or_min_index = pos;
++pos;
for (; pos < offset; ++pos)
{
int compare_result = column->compareAt(pos, current_max_or_min_index, *column, 1);
if (aggregate_operation == AggregateOperation::max && compare_result > 0)
{
current_max_or_min_index = pos;
}
else if (aggregate_operation == AggregateOperation::min && compare_result < 0)
{
current_max_or_min_index = pos;
}
}
res->insert((*column)[current_max_or_min_index]);
}
return res;
}
const IColumn::Offsets & offsets = array.getOffsets();
ColumnPtr res;
if (executeType<UInt8>(mapped, offsets, res) ||
executeType<UInt16>(mapped, offsets, res) ||
executeType<UInt32>(mapped, offsets, res) ||
executeType<UInt64>(mapped, offsets, res) ||
executeType<UInt128>(mapped, offsets, res) ||
executeType<UInt256>(mapped, offsets, res) ||
executeType<Int8>(mapped, offsets, res) ||
executeType<Int16>(mapped, offsets, res) ||
executeType<Int32>(mapped, offsets, res) ||
executeType<Int64>(mapped, offsets, res) ||
executeType<Int128>(mapped, offsets, res) ||
executeType<Int256>(mapped, offsets, res) ||
executeType<Float32>(mapped, offsets, res) ||
executeType<Float64>(mapped, offsets, res) ||
executeType<Decimal32>(mapped, offsets, res) ||
executeType<Decimal64>(mapped, offsets, res) ||
executeType<Decimal128>(mapped, offsets, res) ||
executeType<Decimal256>(mapped, offsets, res) ||
executeType<DateTime64>(mapped, offsets, res))
if constexpr (aggregate_operation == AggregateOperation::min || aggregate_operation == AggregateOperation::max)
{
executeMinOrMax(mapped, offsets, res);
return res;
}
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Unexpected column for arraySum: {}", mapped->getName());
{
if (executeType<UInt8>(mapped, offsets, res) ||
executeType<UInt16>(mapped, offsets, res) ||
executeType<UInt32>(mapped, offsets, res) ||
executeType<UInt64>(mapped, offsets, res) ||
executeType<UInt128>(mapped, offsets, res) ||
executeType<UInt256>(mapped, offsets, res) ||
executeType<Int8>(mapped, offsets, res) ||
executeType<Int16>(mapped, offsets, res) ||
executeType<Int32>(mapped, offsets, res) ||
executeType<Int64>(mapped, offsets, res) ||
executeType<Int128>(mapped, offsets, res) ||
executeType<Int256>(mapped, offsets, res) ||
executeType<Float32>(mapped, offsets, res) ||
executeType<Float64>(mapped, offsets, res) ||
executeType<Decimal32>(mapped, offsets, res) ||
executeType<Decimal64>(mapped, offsets, res) ||
executeType<Decimal128>(mapped, offsets, res) ||
executeType<Decimal256>(mapped, offsets, res) ||
executeType<DateTime64>(mapped, offsets, res))
{
return res;
}
}
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Unexpected column for arraySum: {}", mapped->getName());
}
};

View File

@ -0,0 +1,37 @@
-- { echoOn }
-- https://github.com/ClickHouse/ClickHouse/issues/68895
SELECT arrayMax(x -> toFixedString('.', 1), []);
.
-- https://github.com/ClickHouse/ClickHouse/issues/69600
SELECT arrayMax(x -> (-x), [1, 2, 4]) AS res;
-1
SELECT arrayMax(x -> toUInt16(-x), [1, 2, 4]) AS res;
65535
-- https://github.com/ClickHouse/ClickHouse/pull/69640
SELECT arrayMin(x1 -> (x1 * toNullable(-1)), materialize([1, 2, 3]));
-3
SELECT arrayMin(x1 -> x1 * -1, [1,2,3]);
-3
DROP TABLE IF EXISTS test_aggregation_array;
CREATE TABLE test_aggregation_array (x Array(Int)) ENGINE=MergeTree() ORDER by tuple();
INSERT INTO test_aggregation_array VALUES ([1,2,3,4,5,6]), ([]), ([1,2,3]);
SELECT [arrayMin(x1 -> (x1 * materialize(-1)), [toNullable(toUInt256(0)), materialize(4)])], arrayMin([arrayMin([0])]) FROM test_aggregation_array GROUP BY arrayAvg([1]), [0, toUInt256(8)] WITH CUBE SETTINGS allow_experimental_analyzer = 1;
[-4] 0
[-4] 0
[-4] 0
[-4] 0
SELECT [arrayMin([3, arrayMin([toUInt128(8)]), 4, 5]), arrayMax([materialize(1)]), arrayMin([arrayMax([1]), 2]), 2], arrayMin([0, toLowCardinality(8)]), 2, arrayMax(x1 -> (x1 * -1), x) FROM test_aggregation_array;
[3,1,1,2] 0 2 -1
[3,1,1,2] 0 2 0
[3,1,1,2] 0 2 -1
select arrayMax(x -> x.1, [(1, 'a'), (0, 'b')]);
1
select arrayMin(x -> x.2, [(1, 'a'), (0, 'b')]);
a
-- Extra validation of generic arrayMin/arrayMax
WITH [(1,2),(1,3)] AS t SELECT arrayMin(t), arrayMax(t);
(1,2) (1,3)
WITH [map('a', 1, 'b', 2), map('a',1,'b',3)] AS t SELECT arrayMin(t), arrayMax(t);
{'a':1,'b':2} {'a':1,'b':3}
WITH [map('a', 1, 'b', 2, 'c', 10), map('a',1,'b',3, 'c', 0)] AS t SELECT arrayMin(x -> x['c'], t), arrayMax(x -> x['c'], t);
0 10

View File

@ -0,0 +1,26 @@
-- { echoOn }
-- https://github.com/ClickHouse/ClickHouse/issues/68895
SELECT arrayMax(x -> toFixedString('.', 1), []);
-- https://github.com/ClickHouse/ClickHouse/issues/69600
SELECT arrayMax(x -> (-x), [1, 2, 4]) AS res;
SELECT arrayMax(x -> toUInt16(-x), [1, 2, 4]) AS res;
-- https://github.com/ClickHouse/ClickHouse/pull/69640
SELECT arrayMin(x1 -> (x1 * toNullable(-1)), materialize([1, 2, 3]));
SELECT arrayMin(x1 -> x1 * -1, [1,2,3]);
DROP TABLE IF EXISTS test_aggregation_array;
CREATE TABLE test_aggregation_array (x Array(Int)) ENGINE=MergeTree() ORDER by tuple();
INSERT INTO test_aggregation_array VALUES ([1,2,3,4,5,6]), ([]), ([1,2,3]);
SELECT [arrayMin(x1 -> (x1 * materialize(-1)), [toNullable(toUInt256(0)), materialize(4)])], arrayMin([arrayMin([0])]) FROM test_aggregation_array GROUP BY arrayAvg([1]), [0, toUInt256(8)] WITH CUBE SETTINGS allow_experimental_analyzer = 1;
SELECT [arrayMin([3, arrayMin([toUInt128(8)]), 4, 5]), arrayMax([materialize(1)]), arrayMin([arrayMax([1]), 2]), 2], arrayMin([0, toLowCardinality(8)]), 2, arrayMax(x1 -> (x1 * -1), x) FROM test_aggregation_array;
select arrayMax(x -> x.1, [(1, 'a'), (0, 'b')]);
select arrayMin(x -> x.2, [(1, 'a'), (0, 'b')]);
-- Extra validation of generic arrayMin/arrayMax
WITH [(1,2),(1,3)] AS t SELECT arrayMin(t), arrayMax(t);
WITH [map('a', 1, 'b', 2), map('a',1,'b',3)] AS t SELECT arrayMin(t), arrayMax(t);
WITH [map('a', 1, 'b', 2, 'c', 10), map('a',1,'b',3, 'c', 0)] AS t SELECT arrayMin(x -> x['c'], t), arrayMax(x -> x['c'], t);