未验证 提交 2d9e0ec0 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #12376 from ClickHouse/fix-totals-state-2

Fix TOTALS/ROLLUP/CUBE for aggregate functions with -State and Nullable arguments
......@@ -247,6 +247,11 @@ public:
{
return true;
}
bool isState() const override
{
return nested_func->isState();
}
};
......
......@@ -2,6 +2,7 @@
#include <AggregateFunctions/AggregateFunctionNull.h>
#include <AggregateFunctions/AggregateFunctionNothing.h>
#include <AggregateFunctions/AggregateFunctionCount.h>
#include <AggregateFunctions/AggregateFunctionState.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
......@@ -71,6 +72,19 @@ public:
if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params))
return adapter;
/// If applied to aggregate function with -State combinator, we apply -Null combinator to it's nested_function instead of itself.
/// Because Nullable AggregateFunctionState does not make sense and ruins the logic of managing aggregate function states.
if (const AggregateFunctionState * function_state = typeid_cast<const AggregateFunctionState *>(nested_function.get()))
{
auto transformed_nested_function = transformAggregateFunction(function_state->getNestedFunction(), properties, arguments, params);
return std::make_shared<AggregateFunctionState>(
transformed_nested_function,
transformed_nested_function->getArgumentTypes(),
transformed_nested_function->getParameters());
}
bool return_type_is_nullable = !properties.returns_default_when_only_null && nested_function->getReturnType()->canBeInsideNullable();
bool serialize_flag = return_type_is_nullable || properties.returns_default_when_only_null;
......
......@@ -122,8 +122,9 @@ public:
throw Exception("Method predictValues is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/** Returns true for aggregate functions of type -State.
/** Returns true for aggregate functions of type -State
* They are executed as other aggregate functions, but not finalized (return an aggregation state that can be combined with another).
* Also returns true when the final value of this aggregate function contains State of other aggregate function inside.
*/
virtual bool isState() const { return false; }
......
......@@ -85,6 +85,20 @@ void ColumnAggregateFunction::addArena(ConstArenaPtr arena_)
foreign_arenas.push_back(arena_);
}
namespace
{
ConstArenas concatArenas(const ConstArenas & array, ConstArenaPtr arena)
{
ConstArenas result = array;
if (arena)
result.push_back(std::move(arena));
return result;
}
}
MutableColumnPtr ColumnAggregateFunction::convertToValues(MutableColumnPtr column)
{
/** If the aggregate function returns an unfinalized/unfinished state,
......@@ -121,19 +135,27 @@ MutableColumnPtr ColumnAggregateFunction::convertToValues(MutableColumnPtr colum
auto & func = column_aggregate_func.func;
auto & data = column_aggregate_func.data;
if (const AggregateFunctionState *function_state = typeid_cast<const AggregateFunctionState *>(func.get()))
{
auto res = column_aggregate_func.createView();
res->set(function_state->getNestedFunction());
res->data.assign(data.begin(), data.end());
return res;
}
/// insertResultInto may invalidate states, so we must unshare ownership of them
column_aggregate_func.ensureOwnership();
MutableColumnPtr res = func->getReturnType()->createColumn();
res->reserve(data.size());
/// If there are references to states in final column, we must hold their ownership
/// by holding arenas and source.
auto callback = [&](auto & subcolumn)
{
if (auto * aggregate_subcolumn = typeid_cast<ColumnAggregateFunction *>(subcolumn.get()))
{
aggregate_subcolumn->foreign_arenas = concatArenas(column_aggregate_func.foreign_arenas, column_aggregate_func.my_arena);
aggregate_subcolumn->src = column_aggregate_func.getPtr();
}
};
callback(res);
res->forEachSubcolumn(callback);
for (auto * val : data)
func->insertResultInto(val, *res, &column_aggregate_func.createOrGetArena());
......@@ -629,20 +651,6 @@ void ColumnAggregateFunction::getExtremes(Field & min, Field & max) const
max = serialized;
}
namespace
{
ConstArenas concatArenas(const ConstArenas & array, ConstArenaPtr arena)
{
ConstArenas result = array;
if (arena)
result.push_back(std::move(arena));
return result;
}
}
ColumnAggregateFunction::MutablePtr ColumnAggregateFunction::createView() const
{
auto res = create(func, concatArenas(foreign_arenas, my_arena));
......
#include <Functions/FunctionFactory.h>
// TODO include this last because of a broken roaring header. See the comment
// inside.
// TODO include this last because of a broken roaring header. See the comment inside.
#include <Functions/FunctionsBitmap.h>
......
0100012CCBC234
0100012CCBC234
---
0100012CCBC234
0100012CCBC234
---
0100012CCBC234
0100012CCBC234
---
0100012CCBC234
0100012CCBC234
---
0100012CCBC234
0100012CCBC234
---
0100012CCBC234
0100012CCBC234
---
1
1
---
0 1
1 1
2 1
3 1
4 1
0 1
---
0 1
1 1
2 1
3 1
4 1
0 1
---
0 [0]
1 [0]
2 [0]
3 [0]
4 [0]
0 [0]
---
0 [0]
1 [0]
2 [0]
3 [0]
4 [0]
\N [0]
---
0100012CCBC234
---
0100012CCBC234
---
SELECT hex(toString(uniqState(toNullable(1)))) WITH TOTALS;
SELECT '---';
SELECT hex(toString(uniqState(x))) FROM (SELECT toNullable(1) AS x) WITH TOTALS;
SELECT '---';
SELECT DISTINCT hex(toString(uniqState(x))) FROM (SELECT materialize(1) AS k, toNullable(1) AS x FROM numbers(1)) GROUP BY k WITH TOTALS ORDER BY k;
SELECT '---';
SELECT DISTINCT hex(toString(uniqState(x))) FROM (SELECT materialize(1) AS k, toNullable(1) AS x FROM numbers(10)) GROUP BY k WITH TOTALS ORDER BY k;
SELECT '---';
SELECT DISTINCT hex(toString(uniqState(x))) FROM (SELECT intDiv(number, 3) AS k, toNullable(1) AS x FROM numbers(10)) GROUP BY k WITH TOTALS ORDER BY k;
SELECT '---';
SELECT DISTINCT hex(toString(uniqState(x))) FROM (SELECT intDiv(number, 3) AS k, toNullable(1) AS x FROM system.numbers LIMIT 100000) GROUP BY k WITH TOTALS ORDER BY k;
SELECT '---';
SELECT DISTINCT arrayUniq(finalizeAggregation(groupArrayState(x))) FROM (SELECT intDiv(number, 3) AS k, toNullable(1) AS x FROM system.numbers LIMIT 100000) GROUP BY k WITH TOTALS ORDER BY k;
SELECT '---';
SELECT k, finalizeAggregation(uniqState(x)) FROM (SELECT intDiv(number, 3) AS k, toNullable(1) AS x FROM system.numbers LIMIT 100000) GROUP BY k WITH TOTALS ORDER BY k LIMIT 5;
SELECT '---';
SELECT k, finalizeAggregation(uniqState(x)) FROM (WITH toNullable(number = 3 ? 3 : 1) AS d SELECT intDiv(number, 3) AS k, number % d AS x FROM system.numbers LIMIT 100000) GROUP BY k WITH TOTALS ORDER BY k LIMIT 5;
SELECT '---';
SELECT k, finalizeAggregation(quantilesTimingState(0.5)(x)) FROM (WITH toNullable(number = 3 ? 3 : 1) AS d SELECT intDiv(number, 3) AS k, number % d AS x FROM system.numbers LIMIT 100000) GROUP BY k WITH TOTALS ORDER BY k LIMIT 5;
SELECT '---';
SELECT k, finalizeAggregation(quantilesTimingState(0.5)(x)) FROM (SELECT intDiv(number, if(number = 9223372036854775807, -2, if(number = 3, number = if(number = 1, NULL, 3), 1)) AS d) AS k, number % d AS x FROM system.numbers LIMIT 100000) GROUP BY k WITH TOTALS ORDER BY k ASC LIMIT 5;
SELECT '---';
SELECT DISTINCT hex(toString(uniqState(x))) FROM (SELECT materialize(1) AS k, toNullable(1) AS x FROM numbers(1)) GROUP BY k WITH ROLLUP ORDER BY k;
SELECT '---';
SELECT DISTINCT hex(toString(uniqState(x))) FROM (SELECT materialize(1) AS k, toNullable(1) AS x FROM numbers(1)) GROUP BY k WITH CUBE ORDER BY k;
SELECT '---';
5B27015C30012CCBC234272C27015C305C30275D
02000000000000000100012CCBC234010000
['0100012CCBC234','010000']
[1,0]
5B27015C30012CCBC234272C27015C305C30275D
5B27015C30012CCBC234272C27015C305C30275D
02000000000000000100012CCBC234010000
02000000000000000100012CCBC234010000
['0100012CCBC234','010000']
['0100012CCBC234','010000']
[1,0]
[1,0]
SELECT hex(toString(uniqStateForEach([1, NULL])));
SELECT hex(toString(uniqStateForEachState([1, NULL])));
SELECT arrayMap(x -> hex(toString(x)), finalizeAggregation(uniqStateForEachState([1, NULL])));
SELECT arrayMap(x -> finalizeAggregation(x), finalizeAggregation(uniqStateForEachState([1, NULL])));
SELECT hex(toString(uniqStateForEach([1, NULL]))) WITH TOTALS;
SELECT hex(toString(uniqStateForEachState([1, NULL]))) WITH TOTALS;
SELECT arrayMap(x -> hex(toString(x)), finalizeAggregation(uniqStateForEachState([1, NULL]))) WITH TOTALS;
SELECT arrayMap(x -> finalizeAggregation(x), finalizeAggregation(uniqStateForEachState([1, NULL]))) WITH TOTALS;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册