mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-02 20:42:04 +00:00
Fix possible use-heap-after-free in -Map combinator
This commit is contained in:
parent
7776512b04
commit
9da22f7b7c
@ -84,6 +84,11 @@ private:
|
|||||||
using Base = IAggregateFunctionDataHelper<Data, AggregateFunctionMap<KeyType>>;
|
using Base = IAggregateFunctionDataHelper<Data, AggregateFunctionMap<KeyType>>;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
|
bool isState() const override
|
||||||
|
{
|
||||||
|
return nested_func->isState();
|
||||||
|
}
|
||||||
|
|
||||||
AggregateFunctionMap(AggregateFunctionPtr nested, const DataTypes & types) : Base(types, nested->getParameters()), nested_func(nested)
|
AggregateFunctionMap(AggregateFunctionPtr nested, const DataTypes & types) : Base(types, nested->getParameters()), nested_func(nested)
|
||||||
{
|
{
|
||||||
if (types.empty())
|
if (types.empty())
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
#include <Interpreters/AggregationUtils.h>
|
#include <Interpreters/AggregationUtils.h>
|
||||||
|
#include <Columns/ColumnMap.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -6,6 +7,7 @@ namespace DB
|
|||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
|
extern const int ILLEGAL_COLUMN;
|
||||||
}
|
}
|
||||||
|
|
||||||
OutputBlockColumns prepareOutputBlockColumns(
|
OutputBlockColumns prepareOutputBlockColumns(
|
||||||
@ -50,19 +52,27 @@ OutputBlockColumns prepareOutputBlockColumns(
|
|||||||
|
|
||||||
if (aggregate_functions[i]->isState())
|
if (aggregate_functions[i]->isState())
|
||||||
{
|
{
|
||||||
|
IColumn * column_to_check = final_aggregate_columns[i].get();
|
||||||
|
/// Aggregate state can be wrapped into array/map if aggregate function ends with -Resample/Map combinator
|
||||||
|
if (auto * column_map = typeid_cast<ColumnMap *>(final_aggregate_columns[i].get()))
|
||||||
|
column_to_check = &column_map->getNestedData().getColumn(1);
|
||||||
|
else if (auto * column_array = typeid_cast<ColumnArray *>(final_aggregate_columns[i].get()))
|
||||||
|
column_to_check = &column_array->getData();
|
||||||
|
|
||||||
/// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states.
|
/// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states.
|
||||||
if (auto * column_aggregate_func = typeid_cast<ColumnAggregateFunction *>(final_aggregate_columns[i].get()))
|
if (auto * column_aggregate_func = typeid_cast<ColumnAggregateFunction *>(column_to_check))
|
||||||
|
{
|
||||||
for (auto & pool : aggregates_pools)
|
for (auto & pool : aggregates_pools)
|
||||||
column_aggregate_func->addArena(pool);
|
column_aggregate_func->addArena(pool);
|
||||||
|
}
|
||||||
/// Aggregate state can be wrapped into array if aggregate function ends with -Resample combinator.
|
else
|
||||||
final_aggregate_columns[i]->forEachSubcolumn(
|
{
|
||||||
[&aggregates_pools](auto & subcolumn)
|
throw Exception(
|
||||||
{
|
ErrorCodes::ILLEGAL_COLUMN,
|
||||||
if (auto * column_aggregate_func = typeid_cast<ColumnAggregateFunction *>(subcolumn.get()))
|
"Aggregate function {} was marked as State, but result column {} doesn't contain AggregateFunction column",
|
||||||
for (auto & pool : aggregates_pools)
|
aggregate_functions[i]->getName(),
|
||||||
column_aggregate_func->addArena(pool);
|
final_aggregate_columns[i]->getName());
|
||||||
});
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
11
tests/queries/0_stateless/02418_map_combinator_bug.sql
Normal file
11
tests/queries/0_stateless/02418_map_combinator_bug.sql
Normal file
@ -0,0 +1,11 @@
|
|||||||
|
drop table if exists test;
|
||||||
|
create table test (x Map(UInt8, AggregateFunction(uniq, UInt64))) engine=Memory;
|
||||||
|
insert into test select uniqStateMap(map(1, number)) from numbers(10);
|
||||||
|
select * from test format Null;
|
||||||
|
drop table test;
|
||||||
|
|
||||||
|
create table test (x AggregateFunction(uniq, UInt64), y Int64) engine=Memory;
|
||||||
|
insert into test select uniqState(number) as x, number as y from numbers(10) group by number;
|
||||||
|
select uniqStateMap(map(1, x)) OVER (PARTITION BY y) from test; -- {serverError ILLEGAL_COLUMN}
|
||||||
|
drop table test;
|
||||||
|
|
Loading…
Reference in New Issue
Block a user