From ed015e19601048cb644c0cfd02b5b8aefc6c7ea7 Mon Sep 17 00:00:00 2001 From: yokofly Date: Sun, 7 Jan 2024 22:23:28 -0800 Subject: [PATCH 1/8] Inline threshold optimizations porting https://github.com/ClickHouse/ClickHouse/pull/34544 --- src/Common/SipHash.h | 6 +++--- src/IO/BitHelpers.h | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 8d18dcf00b5..e66b4ae3c8c 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -83,7 +83,7 @@ class SipHash current_word = 0; } - void update(const char * data, UInt64 size) + ALWAYS_INLINE void update(const char * data, UInt64 size) { const char * end = data + size; @@ -137,12 +137,12 @@ class SipHash } template - void update(const T & x) + ALWAYS_INLINE void update(const T & x) { update(reinterpret_cast(&x), sizeof(x)); } - void update(const std::string & x) + ALWAYS_INLINE void update(const std::string & x) { update(x.data(), x.length()); } diff --git a/src/IO/BitHelpers.h b/src/IO/BitHelpers.h index d15297637a3..9484040b539 100644 --- a/src/IO/BitHelpers.h +++ b/src/IO/BitHelpers.h @@ -56,7 +56,7 @@ class BitReader {} // reads bits_to_read high-bits from bits_buffer - inline UInt64 readBits(UInt8 bits_to_read) + ALWAYS_INLINE inline UInt64 readBits(UInt8 bits_to_read) { if (bits_to_read > bits_count) fillBitBuffer(); @@ -72,7 +72,7 @@ class BitReader return getBitsFromBitBuffer(8); } - inline UInt8 readBit() + ALWAYS_INLINE inline UInt8 readBit() { return static_cast(readBits(1)); } @@ -123,7 +123,7 @@ class BitReader // Fills internal bits_buffer with data from source, reads at most 64 bits - size_t fillBitBuffer() + ALWAYS_INLINE size_t fillBitBuffer() { const size_t available = source_end - source_current; const auto bytes_to_read = std::min(64 / 8, available); From b7732b24cd918c23a84eee3e647136a46d659d75 Mon Sep 17 00:00:00 2001 From: yokofly Date: Sun, 7 Jan 2024 22:26:14 -0800 Subject: [PATCH 2/8] Move memory deallocation out of critical section of ThreadGroupStatus::mutex porting https://github.com/ClickHouse/ClickHouse/pull/43586 --- src/Interpreters/ThreadStatusExt.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 7249cb07724..2ceadd191e9 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -405,11 +405,15 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) query_context.reset(); thread_trace_context.trace_id = 0; thread_trace_context.span_id = 0; + + /// The memory of thread_group->finished_threads_counters_memory is temporarily moved to this vector, which is deallocated out of critical section. + std::vector move_to_temp; + /// Avoid leaking of ThreadGroupStatus::finished_threads_counters_memory /// (this is in case someone uses system thread but did not call getProfileEventsCountersAndMemoryForThreads()) { std::lock_guard guard(thread_group->mutex); - auto stats = std::move(thread_group->finished_threads_counters_memory); + move_to_temp = std::move(thread_group->finished_threads_counters_memory); } thread_group.reset(); From dab6c2e7b7390c6869c680fcf53aa26bd3b0bcef Mon Sep 17 00:00:00 2001 From: yokofly Date: Sun, 7 Jan 2024 22:28:10 -0800 Subject: [PATCH 3/8] Vectorize filter generation of ColumnNullable in FilterDescription porting https://github.com/ClickHouse/ClickHouse/pull/45962/ --- src/Columns/FilterDescription.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Columns/FilterDescription.cpp b/src/Columns/FilterDescription.cpp index ef661675e34..aa90fff2c00 100644 --- a/src/Columns/FilterDescription.cpp +++ b/src/Columns/FilterDescription.cpp @@ -93,7 +93,12 @@ FilterDescription::FilterDescription(const IColumn & column_) size_t size = res.size(); for (size_t i = 0; i < size; ++i) - res[i] = res[i] && !null_map[i]; + { + auto has_val = static_cast(!!res[i]); + auto not_null = static_cast(!null_map[i]); + /// Instead of the logical AND operator(&&), the bitwise one(&) is utilized for the auto vectorization. + res[i] = has_val & not_null; + } data = &res; data_holder = std::move(mutable_holder); From f9a88d1f1c60cc0160a714afaf198a306d48871b Mon Sep 17 00:00:00 2001 From: yokofly Date: Sun, 7 Jan 2024 22:37:31 -0800 Subject: [PATCH 4/8] Check overflow in HashTable::alloc porting https://github.com/ClickHouse/ClickHouse/pull/46349 --- src/Common/HashTable/HashTable.h | 19 ++++++++++++++++--- ...deserialization_hash_table_crash.reference | 0 ...state_deserialization_hash_table_crash.sql | 4 ++++ 3 files changed, 20 insertions(+), 3 deletions(-) create mode 100644 tests/queries_not_supported/0_stateless/allocate_huge_memory/02560_agg_state_deserialization_hash_table_crash.reference create mode 100644 tests/queries_not_supported/0_stateless/allocate_huge_memory/02560_agg_state_deserialization_hash_table_crash.sql diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index a1c66f5c0f7..f146046e87b 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -41,6 +41,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NO_AVAILABLE_DATA; + extern const int CANNOT_ALLOCATE_MEMORY; } } @@ -501,9 +502,21 @@ class HashTable : private boost::noncopyable, return place_value; } + static size_t allocCheckOverflow(size_t buffer_size) + { + size_t size = 0; + if (common::mulOverflow(buffer_size, sizeof(Cell), size)) + throw DB::Exception( + DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, + "Integer overflow trying to allocate memory for HashTable. Trying to allocate {} cells of {} bytes each", + buffer_size, sizeof(Cell)); + + return size; + } + void alloc(const Grower & new_grower) { - buf = reinterpret_cast(Allocator::alloc(new_grower.bufSize() * sizeof(Cell))); + buf = reinterpret_cast(Allocator::alloc(allocCheckOverflow(new_grower.bufSize()))); grower = new_grower; } @@ -560,11 +573,11 @@ class HashTable : private boost::noncopyable, if constexpr (Cell::need_to_notify_cell_during_move) { - buf = reinterpret_cast(Allocator::alloc(new_grower.bufSize() * sizeof(Cell))); + buf = reinterpret_cast(Allocator::alloc(allocCheckOverflow(new_grower.bufSize()))); memcpy(reinterpret_cast(buf), reinterpret_cast(old_buffer.get()), old_buffer_size); } else - buf = reinterpret_cast(Allocator::realloc(buf, old_buffer_size, new_grower.bufSize() * sizeof(Cell))); + buf = reinterpret_cast(Allocator::realloc(buf, old_buffer_size, allocCheckOverflow(new_grower.bufSize()))); grower = new_grower; diff --git a/tests/queries_not_supported/0_stateless/allocate_huge_memory/02560_agg_state_deserialization_hash_table_crash.reference b/tests/queries_not_supported/0_stateless/allocate_huge_memory/02560_agg_state_deserialization_hash_table_crash.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries_not_supported/0_stateless/allocate_huge_memory/02560_agg_state_deserialization_hash_table_crash.sql b/tests/queries_not_supported/0_stateless/allocate_huge_memory/02560_agg_state_deserialization_hash_table_crash.sql new file mode 100644 index 00000000000..5b6662faeb3 --- /dev/null +++ b/tests/queries_not_supported/0_stateless/allocate_huge_memory/02560_agg_state_deserialization_hash_table_crash.sql @@ -0,0 +1,4 @@ +DROP TABLE IF EXISTS tab; +create table tab (d Int64, s AggregateFunction(groupUniqArrayArray, Array(UInt64)), c SimpleAggregateFunction(groupUniqArrayArray, Array(UInt64))) engine = SummingMergeTree() order by d; +INSERT INTO tab VALUES (1, 'このコー'); -- { clientError CANNOT_ALLOCATE_MEMORY } +DROP TABLE tab; From 01c8ec6496e34cf5ba0d4fd382698925afd815a9 Mon Sep 17 00:00:00 2001 From: yokofly Date: Sun, 7 Jan 2024 22:39:52 -0800 Subject: [PATCH 5/8] SerializationString improve performance porting https://github.com/ClickHouse/ClickHouse/pull/57717 --- src/DataTypes/Serializations/SerializationString.cpp | 8 +++++++- .../currentDatabase/01926_order_by_desc_limit.sql | 6 ++---- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 2720049331d..c4a67675a4a 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -150,6 +150,9 @@ template static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnString::Offsets & offsets, ReadBuffer & istr, size_t limit) { size_t offset = data.size(); + /// Avoiding calling resize in a loop improves the performance. + data.resize(std::max(data.capacity(), static_cast(4096))); + for (size_t i = 0; i < limit; ++i) { if (istr.eof()) @@ -161,7 +164,8 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt offset += size + 1; offsets.push_back(offset); - data.resize(offset); + if (unlikely(offset > data.size())) + data.resize(roundUpToPowerOfTwoOrZero(std::max(offset, data.size() * 2))); if (size) { @@ -193,6 +197,8 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt data[offset - 1] = 0; } + + data.resize(offset); } diff --git a/tests/queries_not_supported/0_stateless/currentDatabase/01926_order_by_desc_limit.sql b/tests/queries_not_supported/0_stateless/currentDatabase/01926_order_by_desc_limit.sql index e9c47e02d4c..eafb57ab755 100644 --- a/tests/queries_not_supported/0_stateless/currentDatabase/01926_order_by_desc_limit.sql +++ b/tests/queries_not_supported/0_stateless/currentDatabase/01926_order_by_desc_limit.sql @@ -7,11 +7,9 @@ SETTINGS index_granularity = 1024; INSERT INTO order_by_desc SELECT number, repeat('a', 1024) FROM numbers(1024 * 300); OPTIMIZE TABLE order_by_desc FINAL; -SELECT s FROM order_by_desc ORDER BY u DESC LIMIT 10 FORMAT Null -SETTINGS max_memory_usage = '400M'; +SELECT s FROM order_by_desc ORDER BY u DESC LIMIT 10 FORMAT Null; -SELECT s FROM order_by_desc ORDER BY u LIMIT 10 FORMAT Null -SETTINGS max_memory_usage = '400M'; +SELECT s FROM order_by_desc ORDER BY u LIMIT 10 FORMAT Null; SYSTEM FLUSH LOGS; From 4feecd387bc54e99d2cbed66dda58bcd4e7c0a9b Mon Sep 17 00:00:00 2001 From: yokofly Date: Sun, 7 Jan 2024 22:40:49 -0800 Subject: [PATCH 6/8] SerializationString reduce memory usage porting https://github.com/ClickHouse/ClickHouse/pull/57787 --- src/DataTypes/Serializations/SerializationString.cpp | 2 +- .../currentDatabase/01926_order_by_desc_limit.sql | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index c4a67675a4a..711931a8b4a 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -165,7 +165,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt offsets.push_back(offset); if (unlikely(offset > data.size())) - data.resize(roundUpToPowerOfTwoOrZero(std::max(offset, data.size() * 2))); + data.resize_exact(roundUpToPowerOfTwoOrZero(std::max(offset, data.size() * 2))); if (size) { diff --git a/tests/queries_not_supported/0_stateless/currentDatabase/01926_order_by_desc_limit.sql b/tests/queries_not_supported/0_stateless/currentDatabase/01926_order_by_desc_limit.sql index eafb57ab755..e9c47e02d4c 100644 --- a/tests/queries_not_supported/0_stateless/currentDatabase/01926_order_by_desc_limit.sql +++ b/tests/queries_not_supported/0_stateless/currentDatabase/01926_order_by_desc_limit.sql @@ -7,9 +7,11 @@ SETTINGS index_granularity = 1024; INSERT INTO order_by_desc SELECT number, repeat('a', 1024) FROM numbers(1024 * 300); OPTIMIZE TABLE order_by_desc FINAL; -SELECT s FROM order_by_desc ORDER BY u DESC LIMIT 10 FORMAT Null; +SELECT s FROM order_by_desc ORDER BY u DESC LIMIT 10 FORMAT Null +SETTINGS max_memory_usage = '400M'; -SELECT s FROM order_by_desc ORDER BY u LIMIT 10 FORMAT Null; +SELECT s FROM order_by_desc ORDER BY u LIMIT 10 FORMAT Null +SETTINGS max_memory_usage = '400M'; SYSTEM FLUSH LOGS; From 57af18066a513d1650d7c48aa2b0c1120578cfaf Mon Sep 17 00:00:00 2001 From: yokofly Date: Sun, 7 Jan 2024 22:45:00 -0800 Subject: [PATCH 7/8] optimize ColumnVector::insertMany and ColumnVector::insertManyFrom porting https://github.com/ClickHouse/ClickHouse/pull/55714/ --- src/Columns/ColumnVector.h | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index f14e71b8854..af21ac36fa0 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -154,6 +154,17 @@ class ColumnVector final : public COWHelper> data.push_back(assert_cast(src).getData()[n]); } + void insertManyFrom(const IColumn & src, size_t position, size_t length) override + { + ValueType v = assert_cast(src).getData()[position]; + data.resize_fill(data.size() + length, v); + } + + void insertMany(const Field & field, size_t length) override + { + data.resize_fill(data.size() + length, static_cast(field.get())); + } + void insertData(const char * pos, size_t) override { data.emplace_back(unalignedLoad(pos)); From 3d0b2ade8fbd277abac64899315ffc630dbb2a9c Mon Sep 17 00:00:00 2001 From: yokofly Date: Tue, 23 Jan 2024 23:03:04 -0800 Subject: [PATCH 8/8] Fix Integer overflow in Poco::UTF32Encoding porting clickhouse/clickhouse#58073 --- .../Foundation/include/Poco/UTF32Encoding.h | 11 ++++- base/poco/Foundation/src/UTF32Encoding.cpp | 42 +++++++++---------- 2 files changed, 31 insertions(+), 22 deletions(-) diff --git a/base/poco/Foundation/include/Poco/UTF32Encoding.h b/base/poco/Foundation/include/Poco/UTF32Encoding.h index 6f8af525a66..2610d5fa09c 100644 --- a/base/poco/Foundation/include/Poco/UTF32Encoding.h +++ b/base/poco/Foundation/include/Poco/UTF32Encoding.h @@ -68,7 +68,16 @@ class Foundation_API UTF32Encoding: public TextEncoding int convert(int ch, unsigned char* bytes, int length) const; int queryConvert(const unsigned char* bytes, int length) const; int sequenceLength(const unsigned char* bytes, int length) const; - + +protected: + static int safeToInt(Poco::UInt32 value) + { + if (value <= 0x10FFFF) + return static_cast(value); + else + return -1; + } + private: bool _flipBytes; static const char* _names[]; diff --git a/base/poco/Foundation/src/UTF32Encoding.cpp b/base/poco/Foundation/src/UTF32Encoding.cpp index ff07006a4fb..e600c5d9445 100644 --- a/base/poco/Foundation/src/UTF32Encoding.cpp +++ b/base/poco/Foundation/src/UTF32Encoding.cpp @@ -30,22 +30,22 @@ const char* UTF32Encoding::_names[] = const TextEncoding::CharacterMap UTF32Encoding::_charMap = { - /* 00 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* 10 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* 20 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* 30 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* 40 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* 50 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* 60 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* 70 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* 80 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* 90 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* a0 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* b0 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* c0 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* d0 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* e0 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - /* f0 */ -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, + /* 00 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* 10 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* 20 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* 30 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* 40 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* 50 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* 60 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* 70 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* 80 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* 90 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* a0 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* b0 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* c0 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* d0 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* e0 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, + /* f0 */ -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, -4, }; @@ -118,7 +118,7 @@ const TextEncoding::CharacterMap& UTF32Encoding::characterMap() const int UTF32Encoding::convert(const unsigned char* bytes) const { UInt32 uc; - unsigned char* p = (unsigned char*) &uc; + unsigned char* p = reinterpret_cast(&uc); *p++ = *bytes++; *p++ = *bytes++; *p++ = *bytes++; @@ -129,7 +129,7 @@ int UTF32Encoding::convert(const unsigned char* bytes) const ByteOrder::flipBytes(uc); } - return uc; + return safeToInt(uc); } @@ -138,7 +138,7 @@ int UTF32Encoding::convert(int ch, unsigned char* bytes, int length) const if (bytes && length >= 4) { UInt32 ch1 = _flipBytes ? ByteOrder::flipBytes((UInt32) ch) : (UInt32) ch; - unsigned char* p = (unsigned char*) &ch1; + unsigned char* p = reinterpret_cast(&ch1); *bytes++ = *p++; *bytes++ = *p++; *bytes++ = *p++; @@ -155,14 +155,14 @@ int UTF32Encoding::queryConvert(const unsigned char* bytes, int length) const if (length >= 4) { UInt32 uc; - unsigned char* p = (unsigned char*) &uc; + unsigned char* p = reinterpret_cast(&uc); *p++ = *bytes++; *p++ = *bytes++; *p++ = *bytes++; *p++ = *bytes++; if (_flipBytes) ByteOrder::flipBytes(uc); - return uc; + ret = safeToInt(uc); } return ret;