From 4375a336fd773f5f0161ef3acacf6a3bead55ebf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 3 Jul 2022 11:15:10 +0300 Subject: [PATCH] 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.
READ of size 8 at 0x62d00012d218 thread T186 (QueryPipelineEx) 2022.07.03 05:09:40.000234 [ 31956 ] {} 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 ] {} 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::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 >::~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::allocNoTrack(unsigned long, unsigned long) build_docker/../src/Common/Allocator.h:227:27 #2 0xd988d45 in Allocator::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::deserialize() const build_docker/../src/AggregateFunctions/AggregateFunctionMap.h:205:35
P.S. Thanks to @den-crane for the reproducer. Fixes: #35359 (cc @den-crane @dongxiao-yang) Signed-off-by: Azat Khuzhin --- src/AggregateFunctions/AggregateFunctionMap.h | 15 +++- .../02351_Map_combinator_dist.reference | 4 + .../0_stateless/02351_Map_combinator_dist.sql | 81 +++++++++++++++++++ 3 files changed, 97 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02351_Map_combinator_dist.reference create mode 100644 tests/queries/0_stateless/02351_Map_combinator_dist.sql diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 8d77e22300b..5ccc9041c36 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -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); } } diff --git a/tests/queries/0_stateless/02351_Map_combinator_dist.reference b/tests/queries/0_stateless/02351_Map_combinator_dist.reference new file mode 100644 index 00000000000..98fb6a68656 --- /dev/null +++ b/tests/queries/0_stateless/02351_Map_combinator_dist.reference @@ -0,0 +1,4 @@ +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02351_Map_combinator_dist.sql b/tests/queries/0_stateless/02351_Map_combinator_dist.sql new file mode 100644 index 00000000000..937afa5480e --- /dev/null +++ b/tests/queries/0_stateless/02351_Map_combinator_dist.sql @@ -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;