ClickHouse/src/AggregateFunctions/AggregateFunctionMap.h

258 lines
8.8 KiB
C++
Raw Normal View History

2021-05-05 15:11:56 +00:00
#pragma once
#include <unordered_map>
2022-01-30 19:49:48 +00:00
#include <base/sort.h>
2021-05-05 15:11:56 +00:00
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Columns/ColumnFixedString.h>
2021-05-05 15:11:56 +00:00
#include <Columns/ColumnMap.h>
#include <Columns/ColumnString.h>
2021-05-05 15:11:56 +00:00
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnVector.h>
#include <Core/ColumnWithTypeAndName.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
2021-11-01 08:58:49 +00:00
#include "base/types.h"
2021-05-05 15:11:56 +00:00
#include <Common/Arena.h>
#include "AggregateFunctions/AggregateFunctionFactory.h"
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
template <typename KeyType>
struct AggregateFunctionMapCombinatorData
{
using SearchType = KeyType;
2021-05-05 15:11:56 +00:00
std::unordered_map<KeyType, AggregateDataPtr> merged_maps;
static void writeKey(KeyType key, WriteBuffer & buf) { writeBinary(key, buf); }
static void readKey(KeyType & key, ReadBuffer & buf) { readBinary(key, buf); }
};
template <>
struct AggregateFunctionMapCombinatorData<String>
{
struct StringHash
{
using hash_type = std::hash<std::string_view>;
using is_transparent = void;
size_t operator()(std::string_view str) const { return hash_type{}(str); }
};
2021-05-05 15:11:56 +00:00
#ifdef __cpp_lib_generic_unordered_lookup
using SearchType = std::string_view;
#else
using SearchType = std::string;
#endif
std::unordered_map<String, AggregateDataPtr, StringHash, std::equal_to<>> merged_maps;
static void writeKey(String key, WriteBuffer & buf)
{
writeVarUInt(key.size(), buf);
writeString(key, buf);
}
static void readKey(String & key, ReadBuffer & buf)
{
UInt64 size;
readVarUInt(size, buf);
key.resize(size);
buf.readStrict(key.data(), size);
}
2021-05-05 15:11:56 +00:00
};
template <typename KeyType>
class AggregateFunctionMap final
: public IAggregateFunctionDataHelper<AggregateFunctionMapCombinatorData<KeyType>, AggregateFunctionMap<KeyType>>
{
private:
DataTypePtr key_type;
AggregateFunctionPtr nested_func;
using Data = AggregateFunctionMapCombinatorData<KeyType>;
using Base = IAggregateFunctionDataHelper<Data, AggregateFunctionMap<KeyType>>;
2021-05-05 15:11:56 +00:00
public:
AggregateFunctionMap(AggregateFunctionPtr nested, const DataTypes & types) : Base(types, nested->getParameters()), nested_func(nested)
{
if (types.empty())
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function " + getName() + " requires at least one argument");
if (types.size() > 1)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function " + getName() + " requires only one map argument");
2021-05-05 15:11:56 +00:00
const auto * map_type = checkAndGetDataType<DataTypeMap>(types[0].get());
if (!map_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function " + getName() + " requires map as argument");
key_type = map_type->getKeyType();
}
String getName() const override { return nested_func->getName() + "Map"; }
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeMap>(DataTypes{key_type, nested_func->getReturnType()}); }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
2021-05-05 15:11:56 +00:00
{
const auto & map_column = assert_cast<const ColumnMap &>(*columns[0]);
const auto & map_nested_tuple = map_column.getNestedData();
const IColumn::Offsets & map_array_offsets = map_column.getNestedColumn().getOffsets();
const size_t offset = map_array_offsets[row_num - 1];
const size_t size = (map_array_offsets[row_num] - offset);
const auto & key_column = map_nested_tuple.getColumn(0);
const auto & val_column = map_nested_tuple.getColumn(1);
auto & merged_maps = this->data(place).merged_maps;
for (size_t i = 0; i < size; ++i)
{
typename Data::SearchType key;
2022-03-11 13:34:58 +00:00
if constexpr (std::is_same_v<KeyType, String>)
2021-05-05 15:11:56 +00:00
{
StringRef key_ref;
if (key_type->getTypeId() == TypeIndex::FixedString)
key_ref = assert_cast<const ColumnFixedString &>(key_column).getDataAt(offset + i);
else
key_ref = assert_cast<const ColumnString &>(key_column).getDataAt(offset + i);
#ifdef __cpp_lib_generic_unordered_lookup
key = key_ref.toView();
#else
key = key_ref.toString();
#endif
2021-05-05 15:11:56 +00:00
}
else
{
key = assert_cast<const ColumnVector<KeyType> &>(key_column).getData()[offset + i];
}
AggregateDataPtr nested_place;
auto it = merged_maps.find(key);
if (it == merged_maps.end())
{
// create a new place for each key
nested_place = arena->alignedAlloc(nested_func->sizeOfData(), nested_func->alignOfData());
2021-05-05 15:11:56 +00:00
nested_func->create(nested_place);
merged_maps.emplace(key, nested_place);
}
else
nested_place = it->second;
const IColumn * nested_columns[1] = {&val_column};
nested_func->add(nested_place, nested_columns, offset + i, arena);
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
2021-05-05 15:11:56 +00:00
{
auto & merged_maps = this->data(place).merged_maps;
const auto & rhs_maps = this->data(rhs).merged_maps;
for (const auto & elem : rhs_maps)
{
const auto & it = merged_maps.find(elem.first);
Fix use-after-free for Map combinator that leads to incorrect result This use-after-free can be reproduced with distributed queries. Also note, that this is not sumMappedArray() and friends (that previously called sumMap()) but Map combinator. You will find ASan report in details. <details> READ of size 8 at 0x62d00012d218 thread T186 (QueryPipelineEx) 2022.07.03 05:09:40.000234 [ 31956 ] {} <Trace> AsynchronousMetrics: MemoryTracking: was 1.23 GiB, peak 1.23 GiB, will set to 1.25 GiB (RSS), difference: 19.51 MiB 2022.07.03 05:09:41.000137 [ 31956 ] {} <Trace> AsynchronousMetrics: MemoryTracking: was 1.25 GiB, peak 1.25 GiB, will set to 1.26 GiB (RSS), difference: 3.76 MiB #0 0x1233a0d8 in DB::AggregateFunctionSumData<>::get() const build_docker/../src/AggregateFunctions/AggregateFunctionSum.h:245:16 #1 0x1233a0d8 in DB::AggregateFunctionSum<>::insertResultInto(char*, DB::IColumn&, DB::Arena*) const build_docker/../src/AggregateFunctions/AggregateFunctionSum.h:536:70 #2 0x1470f910 in DB::AggregateFunctionMap<char8_t>::insertResultInto() const build_docker/../src/AggregateFunctions/AggregateFunctionMap.h:236:26 #3 0x147110ce in DB::IAggregateFunctionHelper<>::insertResultIntoBatch() const build_docker/../src/AggregateFunctions/IAggregateFunction.h:618:53 #4 0x2c4269d7 in void DB::Aggregator::convertToBlockImplFinal<>() const build_docker/../src/Interpreters/Aggregator.cpp:1878:49 #5 0x2c403b9f in void DB::Aggregator::convertToBlockImpl<>() const build_docker/../src/Interpreters/Aggregator.cpp:1714:13 #6 0x2be09b53 in DB::Aggregator::prepareBlockAndFillSingleLevel() const::$_2::operator()() const build_docker/../src/Interpreters/Aggregator.cpp:2144:9 #7 0x2be09b53 in DB::Block DB::Aggregator::prepareBlockAndFill<>() const build_docker/../src/Interpreters/Aggregator.cpp:2000:5 #8 0x2be09b53 in DB::Aggregator::prepareBlockAndFillSingleLevel() const build_docker/../src/Interpreters/Aggregator.cpp:2150:12 #9 0x2be37de3 in DB::Aggregator::mergeBlocks() build_docker/../src/Interpreters/Aggregator.cpp:3032:17 #10 0x308c27f8 in DB::MergingAggregatedBucketTransform::transform() build_docker/../src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp:360:37 0x62d00012d218 is located 3608 bytes inside of 32768-byte region [0x62d00012c400,0x62d000134400) freed by thread T186 (QueryPipelineEx) here: #0 0xd701312 in free (/work1/azat/tmp/upstream/clickhouse-asan+0xd701312) (BuildId: b7977aef37e9f720) ... #8 0x2e3c22eb in DB::ColumnAggregateFunction::~ColumnAggregateFunction() build_docker/../src/Columns/ColumnAggregateFunction.cpp:89:1 ... #18 0xd9fcdd4 in std::__1::vector<DB::ColumnWithTypeAndName, std::__1::allocator<DB::ColumnWithTypeAndName> >::~vector() build_docker/../contrib/libcxx/include/vector:401:9 #19 0x2be373f4 in DB::Aggregator::mergeBlocks() build_docker/../contrib/libcxx/include/__memory/unique_ptr.h #20 0x308c27f8 in DB::MergingAggregatedBucketTransform::transform() build_docker/../src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp:360:37 previously allocated by thread T186 (QueryPipelineEx) here: #0 0xd7015be in malloc (/work1/azat/tmp/upstream/clickhouse-asan+0xd7015be) (BuildId: b7977aef37e9f720) #1 0xd85190a in Allocator<false, false>::allocNoTrack(unsigned long, unsigned long) build_docker/../src/Common/Allocator.h:227:27 #2 0xd988d45 in Allocator<false, false>::alloc(unsigned long, unsigned long) build_docker/../src/Common/Allocator.h:96:16 #3 0xd988d45 in DB::Arena::MemoryChunk::MemoryChunk(unsigned long, DB::Arena::MemoryChunk*) build_docker/../src/Common/Arena.h:54:64 #4 0xd98904b in DB::Arena::addMemoryChunk(unsigned long) build_docker/../src/Common/Arena.h:122:20 #5 0xec9542c in DB::Arena::alignedAlloc(unsigned long, unsigned long) build_docker/../src/Common/Arena.h:171:13 #6 0x1470f123 in DB::AggregateFunctionMap<char8_t>::deserialize() const build_docker/../src/AggregateFunctions/AggregateFunctionMap.h:205:35 </details> P.S. Thanks to @den-crane for the reproducer. Fixes: #35359 (cc @den-crane @dongxiao-yang) Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-07-03 08:15:10 +00:00
AggregateDataPtr nested_place;
if (it == merged_maps.end())
2021-05-05 15:11:56 +00:00
{
Fix use-after-free for Map combinator that leads to incorrect result This use-after-free can be reproduced with distributed queries. Also note, that this is not sumMappedArray() and friends (that previously called sumMap()) but Map combinator. You will find ASan report in details. <details> READ of size 8 at 0x62d00012d218 thread T186 (QueryPipelineEx) 2022.07.03 05:09:40.000234 [ 31956 ] {} <Trace> AsynchronousMetrics: MemoryTracking: was 1.23 GiB, peak 1.23 GiB, will set to 1.25 GiB (RSS), difference: 19.51 MiB 2022.07.03 05:09:41.000137 [ 31956 ] {} <Trace> AsynchronousMetrics: MemoryTracking: was 1.25 GiB, peak 1.25 GiB, will set to 1.26 GiB (RSS), difference: 3.76 MiB #0 0x1233a0d8 in DB::AggregateFunctionSumData<>::get() const build_docker/../src/AggregateFunctions/AggregateFunctionSum.h:245:16 #1 0x1233a0d8 in DB::AggregateFunctionSum<>::insertResultInto(char*, DB::IColumn&, DB::Arena*) const build_docker/../src/AggregateFunctions/AggregateFunctionSum.h:536:70 #2 0x1470f910 in DB::AggregateFunctionMap<char8_t>::insertResultInto() const build_docker/../src/AggregateFunctions/AggregateFunctionMap.h:236:26 #3 0x147110ce in DB::IAggregateFunctionHelper<>::insertResultIntoBatch() const build_docker/../src/AggregateFunctions/IAggregateFunction.h:618:53 #4 0x2c4269d7 in void DB::Aggregator::convertToBlockImplFinal<>() const build_docker/../src/Interpreters/Aggregator.cpp:1878:49 #5 0x2c403b9f in void DB::Aggregator::convertToBlockImpl<>() const build_docker/../src/Interpreters/Aggregator.cpp:1714:13 #6 0x2be09b53 in DB::Aggregator::prepareBlockAndFillSingleLevel() const::$_2::operator()() const build_docker/../src/Interpreters/Aggregator.cpp:2144:9 #7 0x2be09b53 in DB::Block DB::Aggregator::prepareBlockAndFill<>() const build_docker/../src/Interpreters/Aggregator.cpp:2000:5 #8 0x2be09b53 in DB::Aggregator::prepareBlockAndFillSingleLevel() const build_docker/../src/Interpreters/Aggregator.cpp:2150:12 #9 0x2be37de3 in DB::Aggregator::mergeBlocks() build_docker/../src/Interpreters/Aggregator.cpp:3032:17 #10 0x308c27f8 in DB::MergingAggregatedBucketTransform::transform() build_docker/../src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp:360:37 0x62d00012d218 is located 3608 bytes inside of 32768-byte region [0x62d00012c400,0x62d000134400) freed by thread T186 (QueryPipelineEx) here: #0 0xd701312 in free (/work1/azat/tmp/upstream/clickhouse-asan+0xd701312) (BuildId: b7977aef37e9f720) ... #8 0x2e3c22eb in DB::ColumnAggregateFunction::~ColumnAggregateFunction() build_docker/../src/Columns/ColumnAggregateFunction.cpp:89:1 ... #18 0xd9fcdd4 in std::__1::vector<DB::ColumnWithTypeAndName, std::__1::allocator<DB::ColumnWithTypeAndName> >::~vector() build_docker/../contrib/libcxx/include/vector:401:9 #19 0x2be373f4 in DB::Aggregator::mergeBlocks() build_docker/../contrib/libcxx/include/__memory/unique_ptr.h #20 0x308c27f8 in DB::MergingAggregatedBucketTransform::transform() build_docker/../src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp:360:37 previously allocated by thread T186 (QueryPipelineEx) here: #0 0xd7015be in malloc (/work1/azat/tmp/upstream/clickhouse-asan+0xd7015be) (BuildId: b7977aef37e9f720) #1 0xd85190a in Allocator<false, false>::allocNoTrack(unsigned long, unsigned long) build_docker/../src/Common/Allocator.h:227:27 #2 0xd988d45 in Allocator<false, false>::alloc(unsigned long, unsigned long) build_docker/../src/Common/Allocator.h:96:16 #3 0xd988d45 in DB::Arena::MemoryChunk::MemoryChunk(unsigned long, DB::Arena::MemoryChunk*) build_docker/../src/Common/Arena.h:54:64 #4 0xd98904b in DB::Arena::addMemoryChunk(unsigned long) build_docker/../src/Common/Arena.h:122:20 #5 0xec9542c in DB::Arena::alignedAlloc(unsigned long, unsigned long) build_docker/../src/Common/Arena.h:171:13 #6 0x1470f123 in DB::AggregateFunctionMap<char8_t>::deserialize() const build_docker/../src/AggregateFunctions/AggregateFunctionMap.h:205:35 </details> P.S. Thanks to @den-crane for the reproducer. Fixes: #35359 (cc @den-crane @dongxiao-yang) Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-07-03 08:15:10 +00:00
// elem.second cannot be copied since this it will be destroyed after merging,
// and lead to use-after-free.
nested_place = arena->alignedAlloc(nested_func->sizeOfData(), nested_func->alignOfData());
nested_func->create(nested_place);
merged_maps.emplace(elem.first, nested_place);
2021-05-05 15:11:56 +00:00
}
else
Fix use-after-free for Map combinator that leads to incorrect result This use-after-free can be reproduced with distributed queries. Also note, that this is not sumMappedArray() and friends (that previously called sumMap()) but Map combinator. You will find ASan report in details. <details> READ of size 8 at 0x62d00012d218 thread T186 (QueryPipelineEx) 2022.07.03 05:09:40.000234 [ 31956 ] {} <Trace> AsynchronousMetrics: MemoryTracking: was 1.23 GiB, peak 1.23 GiB, will set to 1.25 GiB (RSS), difference: 19.51 MiB 2022.07.03 05:09:41.000137 [ 31956 ] {} <Trace> AsynchronousMetrics: MemoryTracking: was 1.25 GiB, peak 1.25 GiB, will set to 1.26 GiB (RSS), difference: 3.76 MiB #0 0x1233a0d8 in DB::AggregateFunctionSumData<>::get() const build_docker/../src/AggregateFunctions/AggregateFunctionSum.h:245:16 #1 0x1233a0d8 in DB::AggregateFunctionSum<>::insertResultInto(char*, DB::IColumn&, DB::Arena*) const build_docker/../src/AggregateFunctions/AggregateFunctionSum.h:536:70 #2 0x1470f910 in DB::AggregateFunctionMap<char8_t>::insertResultInto() const build_docker/../src/AggregateFunctions/AggregateFunctionMap.h:236:26 #3 0x147110ce in DB::IAggregateFunctionHelper<>::insertResultIntoBatch() const build_docker/../src/AggregateFunctions/IAggregateFunction.h:618:53 #4 0x2c4269d7 in void DB::Aggregator::convertToBlockImplFinal<>() const build_docker/../src/Interpreters/Aggregator.cpp:1878:49 #5 0x2c403b9f in void DB::Aggregator::convertToBlockImpl<>() const build_docker/../src/Interpreters/Aggregator.cpp:1714:13 #6 0x2be09b53 in DB::Aggregator::prepareBlockAndFillSingleLevel() const::$_2::operator()() const build_docker/../src/Interpreters/Aggregator.cpp:2144:9 #7 0x2be09b53 in DB::Block DB::Aggregator::prepareBlockAndFill<>() const build_docker/../src/Interpreters/Aggregator.cpp:2000:5 #8 0x2be09b53 in DB::Aggregator::prepareBlockAndFillSingleLevel() const build_docker/../src/Interpreters/Aggregator.cpp:2150:12 #9 0x2be37de3 in DB::Aggregator::mergeBlocks() build_docker/../src/Interpreters/Aggregator.cpp:3032:17 #10 0x308c27f8 in DB::MergingAggregatedBucketTransform::transform() build_docker/../src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp:360:37 0x62d00012d218 is located 3608 bytes inside of 32768-byte region [0x62d00012c400,0x62d000134400) freed by thread T186 (QueryPipelineEx) here: #0 0xd701312 in free (/work1/azat/tmp/upstream/clickhouse-asan+0xd701312) (BuildId: b7977aef37e9f720) ... #8 0x2e3c22eb in DB::ColumnAggregateFunction::~ColumnAggregateFunction() build_docker/../src/Columns/ColumnAggregateFunction.cpp:89:1 ... #18 0xd9fcdd4 in std::__1::vector<DB::ColumnWithTypeAndName, std::__1::allocator<DB::ColumnWithTypeAndName> >::~vector() build_docker/../contrib/libcxx/include/vector:401:9 #19 0x2be373f4 in DB::Aggregator::mergeBlocks() build_docker/../contrib/libcxx/include/__memory/unique_ptr.h #20 0x308c27f8 in DB::MergingAggregatedBucketTransform::transform() build_docker/../src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp:360:37 previously allocated by thread T186 (QueryPipelineEx) here: #0 0xd7015be in malloc (/work1/azat/tmp/upstream/clickhouse-asan+0xd7015be) (BuildId: b7977aef37e9f720) #1 0xd85190a in Allocator<false, false>::allocNoTrack(unsigned long, unsigned long) build_docker/../src/Common/Allocator.h:227:27 #2 0xd988d45 in Allocator<false, false>::alloc(unsigned long, unsigned long) build_docker/../src/Common/Allocator.h:96:16 #3 0xd988d45 in DB::Arena::MemoryChunk::MemoryChunk(unsigned long, DB::Arena::MemoryChunk*) build_docker/../src/Common/Arena.h:54:64 #4 0xd98904b in DB::Arena::addMemoryChunk(unsigned long) build_docker/../src/Common/Arena.h:122:20 #5 0xec9542c in DB::Arena::alignedAlloc(unsigned long, unsigned long) build_docker/../src/Common/Arena.h:171:13 #6 0x1470f123 in DB::AggregateFunctionMap<char8_t>::deserialize() const build_docker/../src/AggregateFunctions/AggregateFunctionMap.h:205:35 </details> P.S. Thanks to @den-crane for the reproducer. Fixes: #35359 (cc @den-crane @dongxiao-yang) Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-07-03 08:15:10 +00:00
{
nested_place = it->second;
}
nested_func->merge(nested_place, elem.second, arena);
2021-05-05 15:11:56 +00:00
}
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
2021-05-05 15:11:56 +00:00
{
auto & merged_maps = this->data(place).merged_maps;
writeVarUInt(merged_maps.size(), buf);
for (const auto & elem : merged_maps)
{
this->data(place).writeKey(elem.first, buf);
nested_func->serialize(elem.second, buf);
}
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
2021-05-05 15:11:56 +00:00
{
auto & merged_maps = this->data(place).merged_maps;
UInt64 size;
2021-05-05 15:11:56 +00:00
readVarUInt(size, buf);
for (UInt64 i = 0; i < size; ++i)
2021-05-05 15:11:56 +00:00
{
KeyType key;
AggregateDataPtr nested_place;
this->data(place).readKey(key, buf);
nested_place = arena->alignedAlloc(nested_func->sizeOfData(), nested_func->alignOfData());
2021-05-05 15:11:56 +00:00
nested_func->create(nested_place);
merged_maps.emplace(key, nested_place);
2021-11-18 07:51:19 +00:00
nested_func->deserialize(nested_place, buf, std::nullopt, arena);
2021-05-05 15:11:56 +00:00
}
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * arena) const override
2021-05-05 15:11:56 +00:00
{
auto & map_column = assert_cast<ColumnMap &>(to);
auto & nested_column = map_column.getNestedColumn();
auto & nested_data_column = map_column.getNestedData();
auto & key_column = nested_data_column.getColumn(0);
auto & val_column = nested_data_column.getColumn(1);
auto & merged_maps = this->data(place).merged_maps;
// sort the keys
std::vector<KeyType> keys;
keys.reserve(merged_maps.size());
for (auto & it : merged_maps)
{
keys.push_back(it.first);
}
2022-01-30 19:49:48 +00:00
::sort(keys.begin(), keys.end());
2021-05-05 15:11:56 +00:00
// insert using sorted keys to result column
for (auto & key : keys)
{
key_column.insert(key);
nested_func->insertResultInto(merged_maps[key], val_column, arena);
}
IColumn::Offsets & res_offsets = nested_column.getOffsets();
res_offsets.push_back(val_column.size());
2021-05-05 15:11:56 +00:00
}
bool allocatesMemoryInArena() const override { return true; }
AggregateFunctionPtr getNestedFunction() const override { return nested_func; }
};
}