mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +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>
This commit is contained in:
parent
6384fe23c3
commit
4375a336fd
@ -169,12 +169,21 @@ public:
|
||||
{
|
||||
const auto & it = merged_maps.find(elem.first);
|
||||
|
||||
if (it != merged_maps.end())
|
||||
AggregateDataPtr nested_place;
|
||||
if (it == merged_maps.end())
|
||||
{
|
||||
nested_func->merge(it->second, elem.second, arena);
|
||||
// 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);
|
||||
}
|
||||
else
|
||||
merged_maps[elem.first] = elem.second;
|
||||
{
|
||||
nested_place = it->second;
|
||||
}
|
||||
|
||||
nested_func->merge(nested_place, elem.second, arena);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -0,0 +1,4 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
81
tests/queries/0_stateless/02351_Map_combinator_dist.sql
Normal file
81
tests/queries/0_stateless/02351_Map_combinator_dist.sql
Normal file
@ -0,0 +1,81 @@
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/35359
|
||||
|
||||
-- sumMap
|
||||
SELECT x[67]
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
A,
|
||||
sumMap(CAST(arrayMap(x -> (x, 1), r), 'Map(UInt8,Int64)')) AS x
|
||||
FROM remote('127.{1,1}', view(
|
||||
SELECT
|
||||
number AS A,
|
||||
range(150) AS r
|
||||
FROM numbers(60)
|
||||
WHERE (A % 2) = shardNum()
|
||||
))
|
||||
GROUP BY A
|
||||
LIMIT 100000000
|
||||
)
|
||||
WHERE A = 53
|
||||
SETTINGS prefer_localhost_replica = 0, distributed_aggregation_memory_efficient = 1, group_by_two_level_threshold = 0, group_by_two_level_threshold_bytes = 0;
|
||||
|
||||
-- minMap
|
||||
SELECT x[0]
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
A,
|
||||
minMap(CAST(arrayMap(x -> (x, 1), r), 'Map(UInt8,Int64)')) AS x
|
||||
FROM remote('127.{1,1}', view(
|
||||
SELECT
|
||||
number AS A,
|
||||
range(150) AS r
|
||||
FROM numbers(60)
|
||||
WHERE (A % 2) = shardNum()
|
||||
))
|
||||
GROUP BY A
|
||||
LIMIT 100000000
|
||||
)
|
||||
WHERE A = 41
|
||||
SETTINGS prefer_localhost_replica = 0, distributed_aggregation_memory_efficient = 1, group_by_two_level_threshold = 0, group_by_two_level_threshold_bytes = 0;
|
||||
|
||||
-- maxMap
|
||||
SELECT x[0]
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
A,
|
||||
maxMap(CAST(arrayMap(x -> (x, 1), r), 'Map(UInt8,Int64)')) AS x
|
||||
FROM remote('127.{1,1}', view(
|
||||
SELECT
|
||||
number AS A,
|
||||
range(150) AS r
|
||||
FROM numbers(60)
|
||||
WHERE (A % 2) = shardNum()
|
||||
))
|
||||
GROUP BY A
|
||||
LIMIT 100000000
|
||||
)
|
||||
WHERE A = 41
|
||||
SETTINGS prefer_localhost_replica = 0, distributed_aggregation_memory_efficient = 1, group_by_two_level_threshold = 0, group_by_two_level_threshold_bytes = 0;
|
||||
|
||||
-- avgMap
|
||||
SELECT x[0]
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
A,
|
||||
avgMap(CAST(arrayMap(x -> (x, 1), r), 'Map(UInt8,Int64)')) AS x
|
||||
FROM remote('127.{1,1}', view(
|
||||
SELECT
|
||||
number AS A,
|
||||
range(150) AS r
|
||||
FROM numbers(60)
|
||||
WHERE (A % 2) = shardNum()
|
||||
))
|
||||
GROUP BY A
|
||||
LIMIT 100000000
|
||||
)
|
||||
WHERE A = 41
|
||||
SETTINGS prefer_localhost_replica = 0, distributed_aggregation_memory_efficient = 1, group_by_two_level_threshold = 0, group_by_two_level_threshold_bytes = 0;
|
Loading…
Reference in New Issue
Block a user