mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Fix concurrent access to LowCardinality during GROUP BY (leads to SIGSEGV)
The problem is that GROUP BY can update saved_hash, which can be also
updated by subsequent update of a dictionary, and this will lead to
use-after-free.
You will find ASan report in `details`.
<details>
==24679==ERROR: AddressSanitizer: heap-use-after-free on address 0x604000615d20 at pc 0x000022cc8684 bp 0x7ffea6b5f850 sp 0x7ffea6b5f848
READ of size 8 at 0x604000615d20 thread T223 (QueryPipelineEx)
0 0x22cc8683 in DB::ReverseIndex<>::insert(StringRef const&) obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h
1 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>()::'lambda'()::operator()() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:540:39
2 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:570:23
3 0x22cb9c66 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom()::'lambda'(auto)::operator()<char8_t>(auto) const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:592:26
4 0x22cb9c66 in DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:600:28
5 0x2500b897 in DB::ColumnLowCardinality::insertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnLowCardinality.cpp:205:62
6 0x25a182f4 in DB::appendBlock(DB::Block const&, DB::Block&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:470:23
7 0x25a182f4 in DB::BufferSink::insertIntoBuffer(DB::Block const&, DB::StorageBuffer::Buffer&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:634:9
8 0x25a173cc in DB::BufferSink::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:595:9
9 0x26d1c997 in DB::SinkToStorage::transform(DB::Chunk&) obj-x86_64-linux-gnu/../src/Processors/Sinks/SinkToStorage.cpp:18:5
0x604000615d20 is located 16 bytes inside of 40-byte region [0x604000615d10,0x604000615d38)
freed by thread T37 (QueryPipelineEx) here:
2 0x22cb9392 in boost::intrusive_ptr<DB::ColumnVector<unsigned long> >::~intrusive_ptr() obj-x86_64-linux-gnu/../contrib/boost/boost/smart_ptr/intrusive_ptr.hpp:98:23
4 0x22cb9392 in COW<DB::IColumn>::mutable_ptr<DB::ColumnVector<unsigned long> >::operator=() obj-x86_64-linux-gnu/../src/Common/COW.h💯57
5 0x22cb9392 in DB::ReverseIndex<>::tryGetSavedHash() const obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:362:28
6 0x22cb9392 in DB::ColumnUnique<DB::ColumnString>::tryGetSavedHash() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:125:76
7 0x242eaed3 in DB::ColumnsHashing::HashMethodSingleLowCardinalityColumn<>::HashMethodSingleLowCardinalityColumn() obj-x86_64-linux-gnu/../src/Common/ColumnsHashing.h:287:50
8 0x242206c6 in void DB::Aggregator::executeImpl<>() const obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:596:28
9 0x24148e99 in DB::Aggregator::executeOnBlock() const obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:1004:9
10 0x26c24f3a in DB::AggregatingTransform::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingTransform.cpp:539:33
11 0x26c2054e in DB::AggregatingTransform::work() obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingTransform.cpp:500:9
previously allocated by thread T37 (QueryPipelineEx) here:
0 0xb6d44fd in operator new(unsigned long) (/src/ch/tmp/upstream/clickhouse-asan+0xb6d44fd)
1 0x11b78580 in COW<DB::IColumn>::mutable_ptr<> COWHelper<>::create<unsigned long&>(unsigned long&) (/src/ch/tmp/upstream/clickhouse-asan+0x11b78580)
2 0x22cbf7b1 in DB::ReverseIndex<>::calcHashes() const obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:472:17
3 0x22cc2307 in DB::ReverseIndex<>::buildIndex() obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:438:22
4 0x22cc658c in DB::ReverseIndex<>::insert(StringRef const&) obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:484:9
5 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>()::'lambda'()::operator()() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:540:39
6 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:570:23
7 0x22cb9c66 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom()::'lambda'(auto)::operator()<char8_t>(auto) const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:592:26
8 0x22cb9c66 in DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:600:28
9 0x2500b897 in DB::ColumnLowCardinality::insertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnLowCardinality.cpp:205:62
10 0x25a182f4 in DB::appendBlock(DB::Block const&, DB::Block&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:470:23
11 0x25a182f4 in DB::BufferSink::insertIntoBuffer(DB::Block const&, DB::StorageBuffer::Buffer&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:634:9
12 0x25a173cc in DB::BufferSink::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:595:9
13 0x26d1c997 in DB::SinkToStorage::transform(DB::Chunk&) obj-x86_64-linux-gnu/../src/Processors/Sinks/SinkToStorage.cpp:18:5
SUMMARY: AddressSanitizer: heap-use-after-free obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h in DB::ReverseIndex<unsigned long, DB::ColumnString>::insert(StringRef const&)
Shadow bytes around the buggy address:
0x0c08800bab50: fa fa fd fd fd fd fd fa fa fa 00 00 00 00 00 00
0x0c08800bab60: fa fa fd fd fd fd fd fa fa fa 00 00 00 00 00 fa
0x0c08800bab70: fa fa 00 00 00 00 00 00 fa fa 00 00 00 00 00 00
0x0c08800bab80: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fa
0x0c08800bab90: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd
=>0x0c08800baba0: fa fa fd fd[fd]fd fd fa fa fa fd fd fd fd fd fa
0x0c08800babb0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fa
0x0c08800babc0: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd
0x0c08800babd0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fa
0x0c08800babe0: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd
0x0c08800babf0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fd
Shadow byte legend (one shadow byte represents 8 application bytes):
Addressable: 00
Partially addressable: 01 02 03 04 05 06 07
Heap left redzone: fa
Freed heap region: fd
Stack left redzone: f1
Stack mid redzone: f2
Stack right redzone: f3
Stack after return: f5
Stack use after scope: f8
Global redzone: f9
Global init order: f6
Poisoned by user: f7
Container overflow: fc
Array cookie: ac
Intra object redzone: bb
ASan internal: fe
Left alloca redzone: ca
Right alloca redzone: cb
==24679==ABORTING
</details>
This commit is contained in:
parent
cc150b7316
commit
342ec02664
@ -317,7 +317,7 @@ class ReverseIndex
|
||||
{
|
||||
public:
|
||||
ReverseIndex(UInt64 num_prefix_rows_to_skip_, UInt64 base_index_)
|
||||
: num_prefix_rows_to_skip(num_prefix_rows_to_skip_), base_index(base_index_), saved_hash_ptr(nullptr) {}
|
||||
: num_prefix_rows_to_skip(num_prefix_rows_to_skip_), base_index(base_index_), external_saved_hash_ptr(nullptr) {}
|
||||
|
||||
void setColumn(ColumnType * column_);
|
||||
|
||||
@ -352,14 +352,14 @@ public:
|
||||
if (!use_saved_hash)
|
||||
return nullptr;
|
||||
|
||||
UInt64 * ptr = saved_hash_ptr.load();
|
||||
UInt64 * ptr = external_saved_hash_ptr.load();
|
||||
if (!ptr)
|
||||
{
|
||||
auto hash = calcHashes();
|
||||
ptr = &hash->getData()[0];
|
||||
UInt64 * expected = nullptr;
|
||||
if (saved_hash_ptr.compare_exchange_strong(expected, ptr))
|
||||
saved_hash = std::move(hash);
|
||||
if (external_saved_hash_ptr.compare_exchange_strong(expected, ptr))
|
||||
external_saved_hash = std::move(hash);
|
||||
else
|
||||
ptr = expected;
|
||||
}
|
||||
@ -379,7 +379,9 @@ private:
|
||||
/// Lazy initialized.
|
||||
std::unique_ptr<IndexMapType> index;
|
||||
mutable ColumnUInt64::MutablePtr saved_hash;
|
||||
mutable std::atomic<UInt64 *> saved_hash_ptr;
|
||||
/// For usage during GROUP BY
|
||||
mutable ColumnUInt64::MutablePtr external_saved_hash;
|
||||
mutable std::atomic<UInt64 *> external_saved_hash_ptr;
|
||||
|
||||
void buildIndex();
|
||||
|
||||
|
22
tests/queries/0_stateless/02046_low_cardinality_parallel_group_by.sh
Executable file
22
tests/queries/0_stateless/02046_low_cardinality_parallel_group_by.sh
Executable file
@ -0,0 +1,22 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
# This is the regression test for parallel usage of LowCardinality column
|
||||
# via Buffer engine.
|
||||
#
|
||||
# See also:
|
||||
# - https://github.com/ClickHouse/ClickHouse/issues/24158
|
||||
# - https://github.com/ClickHouse/ClickHouse/pull/3138
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS low_card_buffer_test"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE low_card_buffer_test (test_text LowCardinality(String)) ENGINE=Buffer('', '', 16, 60, 360, 100, 1000, 10000, 100000)"
|
||||
|
||||
$CLICKHOUSE_BENCHMARK -d 0 -i 1000 -c 5 <<<"SELECT count() FROM low_card_buffer_test GROUP BY test_text format Null" 2>/dev/null &
|
||||
$CLICKHOUSE_BENCHMARK -d 0 -i 1000 -c 2 <<<"INSERT INTO low_card_buffer_test values('TEST1')" 2>/dev/null &
|
||||
wait
|
||||
|
||||
# server is alive
|
||||
$CLICKHOUSE_CLIENT -q "SELECT 1 FORMAT Null"
|
Loading…
Reference in New Issue
Block a user