mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #60188 from Algunenano/nullable_arrayReduce
Fix arrayReduce with nullable aggregate function name
This commit is contained in:
commit
68a12a5fe3
@ -1,14 +1,15 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/Combinators/AggregateFunctionState.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/Arena.h>
|
||||
|
||||
#include <Common/scope_guard_safe.h>
|
||||
@ -48,6 +49,11 @@ public:
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
/// As we parse the function name and deal with arrays we don't want to default NULL handler, which will hide
|
||||
/// nullability from us (which also means hidden from the aggregate functions)
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
/// Same for low cardinality. We want to return exactly what the aggregate function returns, no meddling
|
||||
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
|
||||
@ -115,7 +121,8 @@ ColumnPtr FunctionArrayReduce::executeImpl(const ColumnsWithTypeAndName & argume
|
||||
const IAggregateFunction & agg_func = *aggregate_function;
|
||||
std::unique_ptr<Arena> arena = std::make_unique<Arena>();
|
||||
|
||||
/// Aggregate functions do not support constant columns. Therefore, we materialize them.
|
||||
/// Aggregate functions do not support constant or lowcardinality columns. Therefore, we materialize them and
|
||||
/// keep a reference so they are alive until we finish using their nested columns (array data/offset)
|
||||
std::vector<ColumnPtr> materialized_columns;
|
||||
|
||||
const size_t num_arguments_columns = arguments.size() - 1;
|
||||
@ -126,6 +133,12 @@ ColumnPtr FunctionArrayReduce::executeImpl(const ColumnsWithTypeAndName & argume
|
||||
for (size_t i = 0; i < num_arguments_columns; ++i)
|
||||
{
|
||||
const IColumn * col = arguments[i + 1].column.get();
|
||||
auto col_no_lowcardinality = recursiveRemoveLowCardinality(arguments[i + 1].column);
|
||||
if (col_no_lowcardinality != arguments[i + 1].column)
|
||||
{
|
||||
materialized_columns.emplace_back(col_no_lowcardinality);
|
||||
col = col_no_lowcardinality.get();
|
||||
}
|
||||
|
||||
const ColumnArray::Offsets * offsets_i = nullptr;
|
||||
if (const ColumnArray * arr = checkAndGetColumn<ColumnArray>(col))
|
||||
|
@ -1 +1 @@
|
||||
SELECT caseWithExpr(arrayReduce(NULL, []), []); -- { serverError BAD_ARGUMENTS }
|
||||
SELECT caseWithExpr(arrayReduce(NULL, []), []); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
|
@ -0,0 +1,15 @@
|
||||
-- { echoOn }
|
||||
SELECT arrayReduce('sum', []::Array(UInt8)) as a, toTypeName(a);
|
||||
0 UInt64
|
||||
SELECT arrayReduce('sumOrNull', []::Array(UInt8)) as a, toTypeName(a);
|
||||
\N Nullable(UInt64)
|
||||
SELECT arrayReduce('sum', [NULL]::Array(Nullable(UInt8))) as a, toTypeName(a);
|
||||
\N Nullable(UInt64)
|
||||
SELECT arrayReduce('sum', [NULL, 10]::Array(Nullable(UInt8))) as a, toTypeName(a);
|
||||
10 Nullable(UInt64)
|
||||
SELECT arrayReduce('any_respect_nulls', [NULL, 10]::Array(Nullable(UInt8))) as a, toTypeName(a);
|
||||
\N Nullable(UInt8)
|
||||
SELECT arrayReduce('any_respect_nulls', [10, NULL]::Array(Nullable(UInt8))) as a, toTypeName(a);
|
||||
10 Nullable(UInt8)
|
||||
SELECT arrayReduce('median', [toLowCardinality(toNullable(8))]) as t, toTypeName(t);
|
||||
8 Nullable(Float64)
|
17
tests/queries/0_stateless/02996_nullable_arrayReduce.sql
Normal file
17
tests/queries/0_stateless/02996_nullable_arrayReduce.sql
Normal file
@ -0,0 +1,17 @@
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/59600
|
||||
SELECT arrayReduce(toNullable('stddevSampOrNull'), [1]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT arrayReduce(toNullable('median'), [toDecimal32OrNull(toFixedString('1', 1), 2), 8]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT toFixedString('--- Int Empty ---', toLowCardinality(17)), arrayReduce(toNullable('avgOrNull'), [1]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT arrayReduce('any', toNullable(3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT arrayReduce(toLowCardinality('median'), [toLowCardinality(toNullable(8))]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
|
||||
-- { echoOn }
|
||||
SELECT arrayReduce('sum', []::Array(UInt8)) as a, toTypeName(a);
|
||||
SELECT arrayReduce('sumOrNull', []::Array(UInt8)) as a, toTypeName(a);
|
||||
SELECT arrayReduce('sum', [NULL]::Array(Nullable(UInt8))) as a, toTypeName(a);
|
||||
SELECT arrayReduce('sum', [NULL, 10]::Array(Nullable(UInt8))) as a, toTypeName(a);
|
||||
SELECT arrayReduce('any_respect_nulls', [NULL, 10]::Array(Nullable(UInt8))) as a, toTypeName(a);
|
||||
SELECT arrayReduce('any_respect_nulls', [10, NULL]::Array(Nullable(UInt8))) as a, toTypeName(a);
|
||||
|
||||
SELECT arrayReduce('median', [toLowCardinality(toNullable(8))]) as t, toTypeName(t);
|
||||
-- { echoOff }
|
Loading…
Reference in New Issue
Block a user