diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 4afd2745526..3fce357cc19 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -146,6 +146,12 @@ def prepare_for_hung_check(drop_databases): "KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'" ) ) + # Long query from 02136_kill_scalar_queries + call_with_retry( + make_query_command( + "KILL QUERY WHERE query LIKE 'SELECT (SELECT number FROM system.numbers WHERE number = 1000000000000)%'" + ) + ) if drop_databases: for i in range(5): diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index f6b8e3a1da6..7c04a6594a6 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -21,6 +21,13 @@ ENGINE = HDFS(URI, format) `SELECT` queries, the format must be supported for input, and to perform `INSERT` queries – for output. The available formats are listed in the [Formats](../../../interfaces/formats.md#formats) section. +- [PARTITION BY expr] + +### PARTITION BY + +`PARTITION BY` — Optional. In most cases you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression). + +For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. **Example:** diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 245bd6f4468..723425429a5 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -13,6 +13,7 @@ This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ec ``` sql CREATE TABLE s3_engine_table (name String, value UInt32) ENGINE = S3(path, [aws_access_key_id, aws_secret_access_key,] format, [compression]) + [PARTITION BY expr] [SETTINGS ...] ``` @@ -23,6 +24,12 @@ CREATE TABLE s3_engine_table (name String, value UInt32) - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). - `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension. +### PARTITION BY + +`PARTITION BY` — Optional. In most cases you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression). + +For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. + **Example** ``` sql diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 80427f301f3..c24e56da7f4 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -77,7 +77,7 @@ Use the `ORDER BY tuple()` syntax, if you do not need sorting. See [Selecting th #### PARTITION BY -`PARTITION BY` — The [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md). Optional. In most cases you don't need partition key, and in most other cases you don't need partition key more granular than by months. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead make client identifier or name the first column in the ORDER BY expression). +`PARTITION BY` — The [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md). Optional. In most cases, you don't need a partition key, and if you do need to partition, generally you do not need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression). For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. diff --git a/docs/en/engines/table-engines/special/file.md b/docs/en/engines/table-engines/special/file.md index 5a9113f3a18..8314c511236 100644 --- a/docs/en/engines/table-engines/special/file.md +++ b/docs/en/engines/table-engines/special/file.md @@ -86,3 +86,9 @@ $ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64 - `SELECT ... SAMPLE` - Indices - Replication + +## PARTITION BY + +`PARTITION BY` — Optional. It is possible to create separate files by partitioning the data on a partition key. In most cases, you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression). + +For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index 095ffbbb827..af8a80c75b0 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -96,3 +96,9 @@ SELECT * FROM url_engine_table - `ALTER` and `SELECT...SAMPLE` operations. - Indexes. - Replication. + +## PARTITION BY + +`PARTITION BY` — Optional. It is possible to create separate files by partitioning the data on a partition key. In most cases, you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression). + +For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. diff --git a/docs/en/sql-reference/statements/explain.md b/docs/en/sql-reference/statements/explain.md index 59af48b79ab..5081abf2fb8 100644 --- a/docs/en/sql-reference/statements/explain.md +++ b/docs/en/sql-reference/statements/explain.md @@ -276,14 +276,12 @@ EXPLAIN json = 1, description = 0, header = 1 SELECT 1, 2 + dummy; With `indexes` = 1, the `Indexes` key is added. It contains an array of used indexes. Each index is described as JSON with `Type` key (a string `MinMax`, `Partition`, `PrimaryKey` or `Skip`) and optional keys: -- `Name` — An index name (for now, is used only for `Skip` index). -- `Keys` — An array of columns used by the index. -- `Condition` — A string with condition used. -- `Description` — An index (for now, is used only for `Skip` index). -- `Initial Parts` — A number of parts before the index is applied. -- `Selected Parts` — A number of parts after the index is applied. -- `Initial Granules` — A number of granules before the index is applied. -- `Selected Granulesis` — A number of granules after the index is applied. +- `Name` — The index name (currently only used for `Skip` indexes). +- `Keys` — The array of columns used by the index. +- `Condition` — The used condition. +- `Description` — The index description (currently only used for `Skip` indexes). +- `Parts` — The number of parts before/after the index is applied. +- `Granules` — The number of granules before/after the index is applied. Example: @@ -294,46 +292,36 @@ Example: "Type": "MinMax", "Keys": ["y"], "Condition": "(y in [1, +inf))", - "Initial Parts": 5, - "Selected Parts": 4, - "Initial Granules": 12, - "Selected Granules": 11 + "Parts": 5/4, + "Granules": 12/11 }, { "Type": "Partition", "Keys": ["y", "bitAnd(z, 3)"], "Condition": "and((bitAnd(z, 3) not in [1, 1]), and((y in [1, +inf)), (bitAnd(z, 3) not in [1, 1])))", - "Initial Parts": 4, - "Selected Parts": 3, - "Initial Granules": 11, - "Selected Granules": 10 + "Parts": 4/3, + "Granules": 11/10 }, { "Type": "PrimaryKey", "Keys": ["x", "y"], "Condition": "and((x in [11, +inf)), (y in [1, +inf)))", - "Initial Parts": 3, - "Selected Parts": 2, - "Initial Granules": 10, - "Selected Granules": 6 + "Parts": 3/2, + "Granules": 10/6 }, { "Type": "Skip", "Name": "t_minmax", "Description": "minmax GRANULARITY 2", - "Initial Parts": 2, - "Selected Parts": 1, - "Initial Granules": 6, - "Selected Granules": 2 + "Parts": 2/1, + "Granules": 6/2 }, { "Type": "Skip", "Name": "t_set", "Description": "set GRANULARITY 2", - "Initial Parts": 1, - "Selected Parts": 1, - "Initial Granules": 2, - "Selected Granules": 1 + "": 1/1, + "Granules": 2/1 } ] ``` diff --git a/docs/ru/sql-reference/statements/explain.md b/docs/ru/sql-reference/statements/explain.md index 4d5fa70e098..0179c840df6 100644 --- a/docs/ru/sql-reference/statements/explain.md +++ b/docs/ru/sql-reference/statements/explain.md @@ -248,10 +248,8 @@ EXPLAIN json = 1, description = 0, header = 1 SELECT 1, 2 + dummy; - `Keys` — массив столбцов, используемых индексом. - `Condition` — строка с используемым условием. - `Description` — индекс (на данный момент используется только для индекса `Skip`). -- `Initial Parts` — количество кусков до применения индекса. -- `Selected Parts` — количество кусков после применения индекса. -- `Initial Granules` — количество гранул до применения индекса. -- `Selected Granulesis` — количество гранул после применения индекса. +- `Parts` — количество кусков до/после применения индекса. +- `Granules` — количество гранул до/после применения индекса. Пример: @@ -262,46 +260,36 @@ EXPLAIN json = 1, description = 0, header = 1 SELECT 1, 2 + dummy; "Type": "MinMax", "Keys": ["y"], "Condition": "(y in [1, +inf))", - "Initial Parts": 5, - "Selected Parts": 4, - "Initial Granules": 12, - "Selected Granules": 11 + "Parts": 5/4, + "Granules": 12/11 }, { "Type": "Partition", "Keys": ["y", "bitAnd(z, 3)"], "Condition": "and((bitAnd(z, 3) not in [1, 1]), and((y in [1, +inf)), (bitAnd(z, 3) not in [1, 1])))", - "Initial Parts": 4, - "Selected Parts": 3, - "Initial Granules": 11, - "Selected Granules": 10 + "Parts": 4/3, + "Granules": 11/10 }, { "Type": "PrimaryKey", "Keys": ["x", "y"], "Condition": "and((x in [11, +inf)), (y in [1, +inf)))", - "Initial Parts": 3, - "Selected Parts": 2, - "Initial Granules": 10, - "Selected Granules": 6 + "Parts": 3/2, + "Granules": 10/6 }, { "Type": "Skip", "Name": "t_minmax", "Description": "minmax GRANULARITY 2", - "Initial Parts": 2, - "Selected Parts": 1, - "Initial Granules": 6, - "Selected Granules": 2 + "Parts": 2/1, + "Granules": 6/2 }, { "Type": "Skip", "Name": "t_set", "Description": "set GRANULARITY 2", - "Initial Parts": 1, - "Selected Parts": 1, - "Initial Granules": 2, - "Selected Granules": 1 + "": 1/1, + "Granules": 2/1 } ] ``` diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index db90a3504c5..ce234bb0100 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include @@ -167,16 +167,16 @@ void BackupWriterS3::copyFileNative(DiskPtr src_disk, const String & src_file_na auto object_storage = src_disk->getObjectStorage(); std::string src_bucket = object_storage->getObjectsNamespace(); auto file_path = fs::path(s3_uri.key) / dest_file_name; - copyFileS3ToS3(client, src_bucket, objects[0].absolute_path, src_offset, src_size, s3_uri.bucket, file_path, request_settings, {}, - threadPoolCallbackRunner(IOThreadPool::get(), "BackupWriterS3")); + copyS3File(client, src_bucket, objects[0].absolute_path, src_offset, src_size, s3_uri.bucket, file_path, request_settings, {}, + threadPoolCallbackRunner(IOThreadPool::get(), "BackupWriterS3")); } } void BackupWriterS3::copyDataToFile( const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name) { - copyDataToS3(create_read_buffer, offset, size, client, s3_uri.bucket, fs::path(s3_uri.key) / dest_file_name, request_settings, {}, - threadPoolCallbackRunner(IOThreadPool::get(), "BackupWriterS3")); + copyDataToS3File(create_read_buffer, offset, size, client, s3_uri.bucket, fs::path(s3_uri.key) / dest_file_name, request_settings, {}, + threadPoolCallbackRunner(IOThreadPool::get(), "BackupWriterS3")); } BackupWriterS3::~BackupWriterS3() = default; diff --git a/src/Common/CancelableSharedMutex.cpp b/src/Common/CancelableSharedMutex.cpp index c8ca93309ee..d9f2c71e8db 100644 --- a/src/Common/CancelableSharedMutex.cpp +++ b/src/Common/CancelableSharedMutex.cpp @@ -98,9 +98,15 @@ void CancelableSharedMutex::lock_shared() bool CancelableSharedMutex::try_lock_shared() { UInt64 value = state.load(); - if (!(value & writers) && state.compare_exchange_strong(value, value + 1)) // overflow is not realistic - return true; - return false; + while (true) + { + if (value & writers) + return false; + if (state.compare_exchange_strong(value, value + 1)) // overflow is not realistic + break; + // Concurrent try_lock_shared() should not fail, so we have to retry CAS, but avoid blocking wait + } + return true; } void CancelableSharedMutex::unlock_shared() diff --git a/src/Common/FST.cpp b/src/Common/FST.cpp index b409d37919f..30e10610eab 100644 --- a/src/Common/FST.cpp +++ b/src/Common/FST.cpp @@ -12,6 +12,7 @@ namespace DB { + namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -20,7 +21,12 @@ namespace ErrorCodes namespace FST { -UInt64 Arc::serialize(WriteBuffer& write_buffer) const +Arc::Arc(Output output_, const StatePtr & target_) + : output(output_) + , target(target_) +{} + +UInt64 Arc::serialize(WriteBuffer & write_buffer) const { UInt64 written_bytes = 0; bool has_output = output != 0; @@ -55,6 +61,14 @@ void LabelsAsBitmap::addLabel(char label) data |= bit_label; } +bool LabelsAsBitmap::hasLabel(char label) const +{ + UInt8 index = label; + UInt256 bit_label = 1; + bit_label <<= index; + return ((data & bit_label) != 0); +} + UInt64 LabelsAsBitmap::getIndex(char label) const { UInt64 bit_count = 0; @@ -78,7 +92,7 @@ UInt64 LabelsAsBitmap::getIndex(char label) const return bit_count; } -UInt64 LabelsAsBitmap::serialize(WriteBuffer& write_buffer) +UInt64 LabelsAsBitmap::serialize(WriteBuffer & write_buffer) { writeVarUInt(data.items[0], write_buffer); writeVarUInt(data.items[1], write_buffer); @@ -91,19 +105,28 @@ UInt64 LabelsAsBitmap::serialize(WriteBuffer& write_buffer) + getLengthOfVarUInt(data.items[3]); } -bool LabelsAsBitmap::hasLabel(char label) const +UInt64 State::hash() const { - UInt8 index = label; - UInt256 bit_label = 1; - bit_label <<= index; + std::vector values; + values.reserve(arcs.size() * (sizeof(Output) + sizeof(UInt64) + 1)); - return ((data & bit_label) != 0); + for (const auto & [label, arc] : arcs) + { + values.push_back(label); + const auto * ptr = reinterpret_cast(&arc.output); + std::copy(ptr, ptr + sizeof(Output), std::back_inserter(values)); + + ptr = reinterpret_cast(&arc.target->id); + std::copy(ptr, ptr + sizeof(UInt64), std::back_inserter(values)); + } + + return CityHash_v1_0_2::CityHash64(values.data(), values.size()); } -Arc* State::getArc(char label) const +Arc * State::getArc(char label) const { auto it = arcs.find(label); - if (it == arcs.cend()) + if (it == arcs.end()) return nullptr; return const_cast(&it->second); @@ -118,46 +141,11 @@ void State::clear() { id = 0; state_index = 0; - flag = 0; - arcs.clear(); + flag = 0; } -UInt64 State::hash() const -{ - std::vector values; - values.reserve(arcs.size() * (sizeof(Output) + sizeof(UInt64) + 1)); - for (const auto & [label, arc] : arcs) - { - values.push_back(label); - const auto * ptr = reinterpret_cast(&arc.output); - std::copy(ptr, ptr + sizeof(Output), std::back_inserter(values)); - - ptr = reinterpret_cast(&arc.target->id); - std::copy(ptr, ptr + sizeof(UInt64), std::back_inserter(values)); - } - - return CityHash_v1_0_2::CityHash64(values.data(), values.size()); -} - -bool operator== (const State & state1, const State & state2) -{ - if (state1.arcs.size() != state2.arcs.size()) - return false; - - for (const auto & [label, arc] : state1.arcs) - { - const auto it = state2.arcs.find(label); - if (it == state2.arcs.cend()) - return false; - - if (it->second != arc) - return false; - } - return true; -} - -UInt64 State::serialize(WriteBuffer& write_buffer) +UInt64 State::serialize(WriteBuffer & write_buffer) { UInt64 written_bytes = 0; @@ -171,10 +159,8 @@ UInt64 State::serialize(WriteBuffer& write_buffer) std::vector labels; labels.reserve(arcs.size()); - for (auto& [label, state] : arcs) - { + for (auto & [label, state] : arcs) labels.push_back(label); - } UInt8 label_size = labels.size(); write_buffer.write(label_size); @@ -186,7 +172,7 @@ UInt64 State::serialize(WriteBuffer& write_buffer) /// Serialize all arcs for (char label : labels) { - Arc* arc = getArc(label); + Arc * arc = getArc(label); assert(arc != nullptr); written_bytes += arc->serialize(write_buffer); } @@ -196,15 +182,13 @@ UInt64 State::serialize(WriteBuffer& write_buffer) /// Serialize bitmap LabelsAsBitmap bmp; for (auto & [label, state] : arcs) - { bmp.addLabel(label); - } written_bytes += bmp.serialize(write_buffer); /// Serialize all arcs for (auto & [label, state] : arcs) { - Arc* arc = getArc(label); + Arc * arc = getArc(label); assert(arc != nullptr); written_bytes += arc->serialize(write_buffer); } @@ -213,16 +197,36 @@ UInt64 State::serialize(WriteBuffer& write_buffer) return written_bytes; } -FSTBuilder::FSTBuilder(WriteBuffer& write_buffer_) : write_buffer(write_buffer_) +bool operator==(const State & state1, const State & state2) +{ + if (state1.arcs.size() != state2.arcs.size()) + return false; + + for (const auto & [label, arc] : state1.arcs) + { + const auto it = state2.arcs.find(label); + if (it == state2.arcs.end()) + return false; + + if (it->second != arc) + return false; + } + return true; +} + +void State::readFlag(ReadBuffer & read_buffer) +{ + read_buffer.readStrict(reinterpret_cast(flag)); +} + +FstBuilder::FstBuilder(WriteBuffer & write_buffer_) : write_buffer(write_buffer_) { for (auto & temp_state : temp_states) - { temp_state = std::make_shared(); - } } /// See FindMinimized in the paper pseudo code l11-l21. -StatePtr FSTBuilder::findMinimized(const State & state, bool & found) +StatePtr FstBuilder::findMinimized(const State & state, bool & found) { found = false; auto hash = state.hash(); @@ -230,7 +234,7 @@ StatePtr FSTBuilder::findMinimized(const State & state, bool & found) /// MEMBER: in the paper pseudo code l15 auto it = minimized_states.find(hash); - if (it != minimized_states.cend() && *it->second == state) + if (it != minimized_states.end() && *it->second == state) { found = true; return it->second; @@ -244,8 +248,11 @@ StatePtr FSTBuilder::findMinimized(const State & state, bool & found) return p; } +namespace +{ + /// See the paper pseudo code l33-34. -size_t FSTBuilder::getCommonPrefixLength(const String & word1, const String & word2) +size_t getCommonPrefixLength(std::string_view word1, std::string_view word2) { size_t i = 0; while (i < word1.size() && i < word2.size() && word1[i] == word2[i]) @@ -253,8 +260,10 @@ size_t FSTBuilder::getCommonPrefixLength(const String & word1, const String & wo return i; } +} + /// See the paper pseudo code l33-39 and l70-72(when down_to is 0). -void FSTBuilder::minimizePreviousWordSuffix(Int64 down_to) +void FstBuilder::minimizePreviousWordSuffix(Int64 down_to) { for (Int64 i = static_cast(previous_word.size()); i >= down_to; --i) { @@ -264,7 +273,7 @@ void FSTBuilder::minimizePreviousWordSuffix(Int64 down_to) if (i != 0) { Output output = 0; - Arc* arc = temp_states[i - 1]->getArc(previous_word[i - 1]); + Arc * arc = temp_states[i - 1]->getArc(previous_word[i - 1]); if (arc) output = arc->output; @@ -287,7 +296,7 @@ void FSTBuilder::minimizePreviousWordSuffix(Int64 down_to) } } -void FSTBuilder::add(const std::string & current_word, Output current_output) +void FstBuilder::add(std::string_view current_word, Output current_output) { /// We assume word size is no greater than MAX_TERM_LENGTH(256). /// FSTs without word size limitation would be inefficient and easy to cause memory bloat @@ -295,10 +304,10 @@ void FSTBuilder::add(const std::string & current_word, Output current_output) /// MAX_TERM_LENGTH, the granule cannot be dropped and will be fully-scanned. It doesn't affect "ngram" tokenizers. /// Another limitation is that if the query string has tokens which exceed this length /// it will fallback to default searching when using "split" tokenizers. - auto current_word_len = current_word.size(); + size_t current_word_len = current_word.size(); if (current_word_len > MAX_TERM_LENGTH) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Too long term ({}) passed to FST builder.", current_word_len); + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot build inverted index: The maximum term length is {}, this is exceeded by term {}", MAX_TERM_LENGTH, current_word_len); size_t prefix_length_plus1 = getCommonPrefixLength(current_word, previous_word) + 1; @@ -333,9 +342,7 @@ void FSTBuilder::add(const std::string & current_word, Output current_output) if (word_suffix != 0) { for (auto & [label, arc] : temp_states[i]->arcs) - { arc.output += word_suffix; - } } /// Reduce current_output current_output -= common_prefix; @@ -350,7 +357,7 @@ void FSTBuilder::add(const std::string & current_word, Output current_output) previous_word = current_word; } -UInt64 FSTBuilder::build() +UInt64 FstBuilder::build() { minimizePreviousWordSuffix(0); @@ -364,7 +371,8 @@ UInt64 FSTBuilder::build() return previous_state_index + previous_written_bytes + length + 1; } -FiniteStateTransducer::FiniteStateTransducer(std::vector data_) : data(std::move(data_)) +FiniteStateTransducer::FiniteStateTransducer(std::vector data_) + : data(std::move(data_)) { } @@ -373,28 +381,28 @@ void FiniteStateTransducer::clear() data.clear(); } -std::pair FiniteStateTransducer::getOutput(const String & term) +std::pair FiniteStateTransducer::getOutput(std::string_view term) { - std::pair result{ 0, false }; + std::pair result(0, false); /// Read index of initial state ReadBufferFromMemory read_buffer(data.data(), data.size()); - read_buffer.seek(data.size()-1, SEEK_SET); + read_buffer.seek(data.size() - 1, SEEK_SET); - UInt8 length{ 0 }; - read_buffer.readStrict(reinterpret_cast(length)); + UInt8 length = 0; + read_buffer.readStrict(reinterpret_cast(length)); /// FST contains no terms if (length == 0) - return { 0, false }; + return {0, false}; read_buffer.seek(data.size() - 1 - length, SEEK_SET); - UInt64 state_index{ 0 }; + UInt64 state_index = 0; readVarUInt(state_index, read_buffer); for (size_t i = 0; i <= term.size(); ++i) { - UInt64 arc_output{ 0 }; + UInt64 arc_output = 0; /// Read flag State temp_state; @@ -411,22 +419,22 @@ std::pair FiniteStateTransducer::getOutput(const String & term) if (temp_state.getEncodingMethod() == State::EncodingMethod::Sequential) { /// Read number of labels - UInt8 label_num{ 0 }; - read_buffer.readStrict(reinterpret_cast(label_num)); + UInt8 label_num = 0; + read_buffer.readStrict(reinterpret_cast(label_num)); if (label_num == 0) - return { 0, false }; + return {0, false}; auto labels_position = read_buffer.getPosition(); /// Find the index of the label from "labels" bytes - auto begin_it{ data.begin() + labels_position }; - auto end_it{ data.begin() + labels_position + label_num }; + auto begin_it = data.begin() + labels_position; + auto end_it = data.begin() + labels_position + label_num; auto pos = std::find(begin_it, end_it, label); if (pos == end_it) - return { 0, false }; + return {0, false}; /// Read the arc for the label UInt64 arc_index = (pos - begin_it); @@ -439,9 +447,7 @@ std::pair FiniteStateTransducer::getOutput(const String & term) arc_output = 0; readVarUInt(state_index, read_buffer); if (state_index & 0x1) // output is followed - { readVarUInt(arc_output, read_buffer); - } state_index >>= 1; } } @@ -455,7 +461,7 @@ std::pair FiniteStateTransducer::getOutput(const String & term) readVarUInt(bmp.data.items[3], read_buffer); if (!bmp.hasLabel(label)) - return { 0, false }; + return {0, false}; /// Read the arc for the label size_t arc_index = bmp.getIndex(label); @@ -465,9 +471,7 @@ std::pair FiniteStateTransducer::getOutput(const String & term) arc_output = 0; readVarUInt(state_index, read_buffer); if (state_index & 0x1) // output is followed - { readVarUInt(arc_output, read_buffer); - } state_index >>= 1; } } @@ -476,5 +480,7 @@ std::pair FiniteStateTransducer::getOutput(const String & term) } return result; } + } + } diff --git a/src/Common/FST.h b/src/Common/FST.h index 6bb4fdba7e2..64c3c8853f6 100644 --- a/src/Common/FST.h +++ b/src/Common/FST.h @@ -19,18 +19,18 @@ namespace DB /// [Direct Construction of Minimal Acyclic Subsequential Transduers] by Stoyan Mihov and Denis Maurel, University of Tours, France namespace FST { + using Output = UInt64; class State; using StatePtr = std::shared_ptr; -/// Arc represents a transition from one state to another +/// Arc represents a transition from one state to another. /// It includes the target state to which the arc points and the arc's output. struct Arc { Arc() = default; - - explicit Arc(Output output_, const StatePtr & target_) : output{output_}, target{target_} { } + Arc(Output output_, const StatePtr & target_); /// 0 means the arc has no output Output output = 0; @@ -53,13 +53,15 @@ public: /// computes the rank UInt64 getIndex(char label) const; - UInt64 serialize(WriteBuffer& write_buffer); + UInt64 serialize(WriteBuffer & write_buffer); + private: - friend class State; - friend class FiniteStateTransducer; /// data holds a 256-bit bitmap for all labels of a state. Its 256 bits correspond to 256 /// possible label values. - UInt256 data{ 0 }; + UInt256 data = 0; + + friend class State; + friend class FiniteStateTransducer; }; /// State implements the State in Finite State Transducer @@ -77,9 +79,9 @@ public: /// Note this is NOT enabled for now since it is experimental Bitmap, }; - State() = default; - State(const State & state) = default; + State() = default; + State(const State & State) = default; UInt64 hash() const; @@ -91,22 +93,12 @@ public: UInt64 serialize(WriteBuffer & write_buffer); - bool isFinal() const - { - return flag_values.is_final == 1; - } - void setFinal(bool value) - { - flag_values.is_final = value; - } - EncodingMethod getEncodingMethod() const - { - return flag_values.encoding_method; - } - void readFlag(ReadBuffer & read_buffer) - { - read_buffer.readStrict(reinterpret_cast(flag)); - } + bool isFinal() const { return flag_values.is_final == 1; } + void setFinal(bool value) { flag_values.is_final = value; } + + EncodingMethod getEncodingMethod() const { return flag_values.encoding_method; } + + void readFlag(ReadBuffer & read_buffer); /// Transient ID of the state which is used for building FST. It won't be serialized UInt64 id = 0; @@ -116,6 +108,7 @@ public: /// Arcs which are started from state, the 'char' is the label on the arc std::unordered_map arcs; + private: struct FlagValues { @@ -132,22 +125,21 @@ private: bool operator==(const State & state1, const State & state2); -inline constexpr size_t MAX_TERM_LENGTH = 256; +static constexpr size_t MAX_TERM_LENGTH = 256; -/// FSTBuilder is used to build Finite State Transducer by adding words incrementally. +/// FstBuilder is used to build Finite State Transducer by adding words incrementally. /// Note that all the words have to be added in sorted order in order to achieve minimized result. -/// In the end, the caller should call build() to serialize minimized FST to WriteBuffer -class FSTBuilder +/// In the end, the caller should call build() to serialize minimized FST to WriteBuffer. +class FstBuilder { public: - explicit FSTBuilder(WriteBuffer & write_buffer_); + explicit FstBuilder(WriteBuffer & write_buffer_); - void add(const std::string & word, Output output); + void add(std::string_view word, Output output); UInt64 build(); private: StatePtr findMinimized(const State & s, bool & found); void minimizePreviousWordSuffix(Int64 down_to); - static size_t getCommonPrefixLength(const String & word1, const String & word2); std::array temp_states; String previous_word; @@ -171,8 +163,8 @@ class FiniteStateTransducer public: FiniteStateTransducer() = default; explicit FiniteStateTransducer(std::vector data_); - std::pair getOutput(const String & term); void clear(); + std::pair getOutput(std::string_view term); std::vector & getData() { return data; } private: diff --git a/src/Common/ProfilingScopedRWLock.h b/src/Common/ProfilingScopedRWLock.h index f5753f9ce46..50e52e66e2d 100644 --- a/src/Common/ProfilingScopedRWLock.h +++ b/src/Common/ProfilingScopedRWLock.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -12,7 +12,7 @@ class ProfilingScopedWriteRWLock { public: - ProfilingScopedWriteRWLock(std::shared_mutex & rwl_, ProfileEvents::Event event) : + ProfilingScopedWriteRWLock(SharedMutex & rwl_, ProfileEvents::Event event) : scoped_write_lock(rwl_) { ProfileEvents::increment(event, watch.elapsed()); @@ -20,14 +20,14 @@ public: private: Stopwatch watch; - std::unique_lock scoped_write_lock; + std::unique_lock scoped_write_lock; }; class ProfilingScopedReadRWLock { public: - ProfilingScopedReadRWLock(std::shared_mutex & rwl, ProfileEvents::Event event) : + ProfilingScopedReadRWLock(SharedMutex & rwl, ProfileEvents::Event event) : scoped_read_lock(rwl) { ProfileEvents::increment(event, watch.elapsed()); @@ -35,7 +35,7 @@ public: private: Stopwatch watch; - std::shared_lock scoped_read_lock; + std::shared_lock scoped_read_lock; }; } diff --git a/src/Common/SharedMutex.cpp b/src/Common/SharedMutex.cpp index 31525dbd668..1df09ca998a 100644 --- a/src/Common/SharedMutex.cpp +++ b/src/Common/SharedMutex.cpp @@ -37,9 +37,7 @@ void SharedMutex::lock() bool SharedMutex::try_lock() { UInt64 value = 0; - if (state.compare_exchange_strong(value, writers)) - return true; - return false; + return state.compare_exchange_strong(value, writers); } void SharedMutex::unlock() @@ -68,9 +66,15 @@ void SharedMutex::lock_shared() bool SharedMutex::try_lock_shared() { UInt64 value = state.load(); - if (!(value & writers) && state.compare_exchange_strong(value, value + 1)) - return true; - return false; + while (true) + { + if (value & writers) + return false; + if (state.compare_exchange_strong(value, value + 1)) + break; + // Concurrent try_lock_shared() should not fail, so we have to retry CAS, but avoid blocking wait + } + return true; } void SharedMutex::unlock_shared() diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 7b6fb433ec5..69c5732ddb6 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -13,7 +13,6 @@ #include #include #include -#include #include diff --git a/src/Common/tests/gtest_fst.cpp b/src/Common/tests/gtest_fst.cpp index 211f98cab74..1f4595cdea7 100644 --- a/src/Common/tests/gtest_fst.cpp +++ b/src/Common/tests/gtest_fst.cpp @@ -7,7 +7,7 @@ TEST(FST, SimpleTest) { - std::vector> indexed_data + std::vector> indexed_data { {"mop", 100}, {"moth", 91}, @@ -17,7 +17,7 @@ TEST(FST, SimpleTest) {"top", 55}, }; - std::vector> not_indexed_data + std::vector> not_indexed_data { {"mo", 100}, {"moth1", 91}, @@ -29,42 +29,40 @@ TEST(FST, SimpleTest) std::vector buffer; DB::WriteBufferFromVector> wbuf(buffer); - DB::FST::FSTBuilder builder(wbuf); + DB::FST::FstBuilder builder(wbuf); - for (auto& [term, output] : indexed_data) - { + for (auto & [term, output] : indexed_data) builder.add(term, output); - } builder.build(); wbuf.finalize(); DB::FST::FiniteStateTransducer fst(buffer); - for (auto& [term, output] : indexed_data) + for (auto & [term, output] : indexed_data) { auto [result, found] = fst.getOutput(term); - ASSERT_EQ(found, true); + ASSERT_TRUE(found); ASSERT_EQ(result, output); } - for (auto& [term, output] : not_indexed_data) + for (auto & [term, output] : not_indexed_data) { auto [result, found] = fst.getOutput(term); - ASSERT_EQ(found, false); + ASSERT_FALSE(found); } } TEST(FST, TestForLongTerms) { /// Test long terms within limitation - std::string term1(DB::FST::MAX_TERM_LENGTH - 1, 'A'); - std::string term2(DB::FST::MAX_TERM_LENGTH, 'B'); + String term1(DB::FST::MAX_TERM_LENGTH - 1, 'A'); + String term2(DB::FST::MAX_TERM_LENGTH, 'B'); DB::FST::Output output1 = 100; DB::FST::Output output2 = 200; std::vector buffer; DB::WriteBufferFromVector> wbuf(buffer); - DB::FST::FSTBuilder builder(wbuf); + DB::FST::FstBuilder builder(wbuf); builder.add(term1, output1); builder.add(term2, output2); @@ -75,20 +73,20 @@ TEST(FST, TestForLongTerms) DB::FST::FiniteStateTransducer fst(buffer); auto [result1, found1] = fst.getOutput(term1); - ASSERT_EQ(found1, true); + ASSERT_TRUE(found1); ASSERT_EQ(result1, output1); auto [result2, found2] = fst.getOutput(term2); - ASSERT_EQ(found2, true); + ASSERT_TRUE(found2); ASSERT_EQ(result2, output2); /// Test exception case when term length exceeds limitation - std::string term3(DB::FST::MAX_TERM_LENGTH + 1, 'C'); + String term3(DB::FST::MAX_TERM_LENGTH + 1, 'C'); DB::FST::Output output3 = 300; std::vector buffer3; DB::WriteBufferFromVector> wbuf3(buffer3); - DB::FST::FSTBuilder builder3(wbuf3); + DB::FST::FstBuilder builder3(wbuf3); EXPECT_THROW(builder3.add(term3, output3), DB::Exception); } diff --git a/src/Common/tests/gtest_threading.cpp b/src/Common/tests/gtest_threading.cpp index 8662e93e81b..8329045cd70 100644 --- a/src/Common/tests/gtest_threading.cpp +++ b/src/Common/tests/gtest_threading.cpp @@ -27,7 +27,7 @@ namespace DB struct NoCancel {}; // for all PerfTests -static constexpr int requests = 512 * 1024; +static constexpr int requests = 128 * 1024; static constexpr int max_threads = 16; template @@ -91,6 +91,49 @@ void TestSharedMutex() ASSERT_EQ(test, writers); } + + // Test multiple readers can acquire lock simultaneously using try_shared_lock + for (int readers = 1; readers <= 128; readers *= 2) + { + T sm; + std::atomic test(0); + std::barrier sync(readers + 1); + + std::vector threads; + threads.reserve(readers); + auto reader = [&] + { + [[maybe_unused]] Status status; + bool acquired = sm.try_lock_shared(); + ASSERT_TRUE(acquired); + if (!acquired) return; // Just to make TSA happy + sync.arrive_and_wait(); // (A) sync with writer + test++; + sync.arrive_and_wait(); // (B) wait for writer to call try_lock() while shared_lock is held + sm.unlock_shared(); + sync.arrive_and_wait(); // (C) wait for writer to release lock, to ensure try_lock_shared() will see no writer + }; + + for (int i = 0; i < readers; i++) + threads.emplace_back(reader); + + { // writer + [[maybe_unused]] Status status; + sync.arrive_and_wait(); // (A) wait for all reader to acquire lock to avoid blocking them + ASSERT_FALSE(sm.try_lock()); + sync.arrive_and_wait(); // (B) sync with readers + { + std::unique_lock lock(sm); + test++; + } + sync.arrive_and_wait(); // (C) sync with readers + } + + for (auto & thread : threads) + thread.join(); + + ASSERT_EQ(test, readers + 1); + } } template diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4f691df5bee..1948a6da012 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -528,8 +528,7 @@ class IColumn; M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ M(Bool, enable_global_with_statement, true, "Propagate WITH statements to UNION queries and all subqueries", 0) \ M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \ - M(Bool, optimize_syntax_fuse_functions, false, "Not ready for production, do not use. Allow apply syntax optimisation: fuse aggregate functions", 0) \ - M(Bool, optimize_fuse_sum_count_avg, false, "Replace calls of functions `sum`, `avg`, `count` with identical arguments into one `sumCount`", 0) \ + M(Bool, optimize_syntax_fuse_functions, false, "Allow apply fuse aggregating function. Available only with `allow_experimental_analyzer`", 0) \ M(Bool, flatten_nested, true, "If true, columns of type Nested will be flatten to separate array columns instead of one array of tuples", 0) \ M(Bool, asterisk_include_materialized_columns, false, "Include MATERIALIZED columns for wildcard query", 0) \ M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \ @@ -727,6 +726,8 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, max_pipeline_depth, 0) \ MAKE_OBSOLETE(M, Seconds, temporary_live_view_timeout, 1) \ MAKE_OBSOLETE(M, Milliseconds, async_insert_cleanup_timeout_ms, 1000) \ + MAKE_OBSOLETE(M, Bool, optimize_fuse_sum_count_avg, 0) \ + /** The section above is for obsolete settings. Do not add anything there. */ diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index d05ab5b4d4b..c35683127c3 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -379,14 +379,18 @@ void DatabaseOnDisk::renameTable( if (dictionary && table && !table->isDictionary()) throw Exception(ErrorCodes::INCORRECT_QUERY, "Use RENAME/EXCHANGE TABLE (instead of RENAME/EXCHANGE DICTIONARY) for tables"); + /// We have to lock the table before detaching, because otherwise lockExclusively will throw. But the table may not exist. + bool need_lock = table != nullptr; + if (need_lock) + table_lock = table->lockExclusively(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); + detachTable(local_context, table_name); + if (!need_lock) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table was detached without locking, it's a bug"); UUID prev_uuid = UUIDHelpers::Nil; try { - table_lock = table->lockExclusively( - local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); - table_metadata_path = getObjectMetadataPath(table_name); attach_query = parseQueryFromMetadata(log, local_context, table_metadata_path); auto & create = attach_query->as(); diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 2925c2847f2..bb98e2bd3bb 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -236,6 +236,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n backQuote(database_name), backQuote(table_name)); res = it->second; tables.erase(it); + res->is_detached = true; auto table_id = res->getStorageID(); if (table_id.hasUUID()) @@ -272,6 +273,10 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid); throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", table_id.getFullTableName()); } + + /// It is important to reset is_detached here since in case of RENAME in + /// non-Atomic database the is_detached is set to true before RENAME. + table->is_detached = false; } void DatabaseWithOwnTablesBase::shutdown() diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index b5223e495ce..6c1c8b35f43 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -448,7 +448,7 @@ void DatabaseMySQL::detachTablePermanently(ContextPtr, const String & table_name remove_or_detach_tables.erase(table_name); throw; } - table_iter->second.second->is_dropped = true; + table_iter->second.second->is_detached = true; } void DatabaseMySQL::dropTable(ContextPtr local_context, const String & table_name, bool /*sync*/) diff --git a/src/Databases/SQLite/SQLiteUtils.cpp b/src/Databases/SQLite/SQLiteUtils.cpp index 21b16f8d985..411c57d41fc 100644 --- a/src/Databases/SQLite/SQLiteUtils.cpp +++ b/src/Databases/SQLite/SQLiteUtils.cpp @@ -14,6 +14,7 @@ namespace ErrorCodes extern const int PATH_ACCESS_DENIED; } +static std::mutex init_sqlite_db_mutex; void processSQLiteError(const String & message, bool throw_on_error) { @@ -52,7 +53,11 @@ SQLitePtr openSQLiteDB(const String & path, ContextPtr context, bool throw_on_er LOG_DEBUG(&Poco::Logger::get("SQLite"), "SQLite database path {} does not exist, will create an empty SQLite database", database_path); sqlite3 * tmp_sqlite_db = nullptr; - int status = sqlite3_open(database_path.c_str(), &tmp_sqlite_db); + int status; + { + std::lock_guard lock(init_sqlite_db_mutex); + status = sqlite3_open(database_path.c_str(), &tmp_sqlite_db); + } if (status != SQLITE_OK) { diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index d4716999b47..e19c4a66b1f 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -4,7 +4,6 @@ #include #include #include -#include #include #include @@ -14,6 +13,7 @@ #include #include +#include #include #include @@ -206,7 +206,7 @@ private: /// This lock is used for the inner cache state update function lock it for /// write, when it need to update cache state all other functions just /// readers. Surprisingly this lock is also used for last_exception pointer. - mutable std::shared_mutex rw_lock; + mutable SharedMutex rw_lock; mutable std::exception_ptr last_exception; mutable std::atomic error_count {0}; diff --git a/src/Dictionaries/CacheDictionaryUpdateQueue.h b/src/Dictionaries/CacheDictionaryUpdateQueue.h index 8db5c4a59df..8d0581d2052 100644 --- a/src/Dictionaries/CacheDictionaryUpdateQueue.h +++ b/src/Dictionaries/CacheDictionaryUpdateQueue.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp deleted file mode 100644 index fd47606bb7e..00000000000 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ /dev/null @@ -1,1025 +0,0 @@ -#include - -#include - -#include -#include -#include -#include -#include -#include -#include - -#include - -#include -#include - -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; - extern const int DICTIONARY_IS_EMPTY; - extern const int UNSUPPORTED_METHOD; - extern const int TYPE_MISMATCH; -} - -namespace -{ - template - void callOnRangeType(const DataTypePtr & range_type, F && func) - { - auto call = [&](const auto & types) - { - using Types = std::decay_t; - using DataType = typename Types::LeftType; - - if constexpr (IsDataTypeDecimalOrNumber || IsDataTypeDateOrDateTime || IsDataTypeEnum) - { - using ColumnType = typename DataType::ColumnType; - func(TypePair()); - return true; - } - - return false; - }; - - auto type_index = range_type->getTypeId(); - if (!callOnIndexAndDataType(type_index, call)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Dictionary structure type of 'range_min' and 'range_max' should " - "be an Integer, Float, Decimal, Date, Date32, DateTime DateTime64, or Enum." - " Actual 'range_min' and 'range_max' type is {}", - range_type->getName()); - } - } -} - -template -RangeHashedDictionary::RangeHashedDictionary( - const StorageID & dict_id_, - const DictionaryStructure & dict_struct_, - DictionarySourcePtr source_ptr_, - DictionaryLifetime dict_lifetime_, - RangeHashedDictionaryConfiguration configuration_, - BlockPtr update_field_loaded_block_) - : IDictionary(dict_id_) - , dict_struct(dict_struct_) - , source_ptr(std::move(source_ptr_)) - , dict_lifetime(dict_lifetime_) - , configuration(configuration_) - , update_field_loaded_block(std::move(update_field_loaded_block_)) -{ - createAttributes(); - loadData(); - calculateBytesAllocated(); -} - -template -ColumnPtr RangeHashedDictionary::getColumn( - const std::string & attribute_name, - const DataTypePtr & result_type, - const Columns & key_columns, - const DataTypes & key_types, - const ColumnPtr & default_values_column) const -{ - if (dictionary_key_type == DictionaryKeyType::Complex) - { - auto key_types_copy = key_types; - key_types_copy.pop_back(); - dict_struct.validateKeyTypes(key_types_copy); - } - - ColumnPtr result; - - const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); - const size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; - const auto & attribute = attributes[attribute_index]; - - /// Cast range column to storage type - Columns modified_key_columns = key_columns; - const ColumnPtr & range_storage_column = key_columns.back(); - ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types.back(), ""}; - modified_key_columns.back() = castColumnAccurate(column_to_cast, dict_struct.range_min->type); - - size_t keys_size = key_columns.front()->size(); - bool is_attribute_nullable = attribute.is_value_nullable.has_value(); - - ColumnUInt8::MutablePtr col_null_map_to; - ColumnUInt8::Container * vec_null_map_to = nullptr; - if (is_attribute_nullable) - { - col_null_map_to = ColumnUInt8::create(keys_size, false); - vec_null_map_to = &col_null_map_to->getData(); - } - - auto type_call = [&](const auto & dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - using ValueType = DictionaryValueType; - using ColumnProvider = DictionaryAttributeColumnProvider; - - DictionaryDefaultValueExtractor default_value_extractor(dictionary_attribute.null_value, default_values_column); - - auto column = ColumnProvider::getColumn(dictionary_attribute, keys_size); - - if constexpr (std::is_same_v) - { - auto * out = column.get(); - - getItemsImpl( - attribute, - modified_key_columns, - [&](size_t, const Array & value, bool) - { - out->insert(value); - }, - default_value_extractor); - } - else if constexpr (std::is_same_v) - { - auto * out = column.get(); - - if (is_attribute_nullable) - getItemsImpl( - attribute, - modified_key_columns, - [&](size_t row, StringRef value, bool is_null) - { - (*vec_null_map_to)[row] = is_null; - out->insertData(value.data, value.size); - }, - default_value_extractor); - else - getItemsImpl( - attribute, - modified_key_columns, - [&](size_t, StringRef value, bool) - { - out->insertData(value.data, value.size); - }, - default_value_extractor); - } - else - { - auto & out = column->getData(); - - if (is_attribute_nullable) - getItemsImpl( - attribute, - modified_key_columns, - [&](size_t row, const auto value, bool is_null) - { - (*vec_null_map_to)[row] = is_null; - out[row] = value; - }, - default_value_extractor); - else - getItemsImpl( - attribute, - modified_key_columns, - [&](size_t row, const auto value, bool) - { - out[row] = value; - }, - default_value_extractor); - } - - result = std::move(column); - }; - - callOnDictionaryAttributeType(attribute.type, type_call); - - if (is_attribute_nullable) - result = ColumnNullable::create(result, std::move(col_null_map_to)); - - return result; -} - -template -ColumnPtr RangeHashedDictionary::getColumnInternal( - const std::string & attribute_name, - const DataTypePtr & result_type, - const PaddedPODArray & key_to_index) const -{ - ColumnPtr result; - - const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); - const size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; - const auto & attribute = attributes[attribute_index]; - - size_t keys_size = key_to_index.size(); - bool is_attribute_nullable = attribute.is_value_nullable.has_value(); - - ColumnUInt8::MutablePtr col_null_map_to; - ColumnUInt8::Container * vec_null_map_to = nullptr; - if (is_attribute_nullable) - { - col_null_map_to = ColumnUInt8::create(keys_size, false); - vec_null_map_to = &col_null_map_to->getData(); - } - - auto type_call = [&](const auto & dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - using ValueType = DictionaryValueType; - using ColumnProvider = DictionaryAttributeColumnProvider; - - auto column = ColumnProvider::getColumn(dictionary_attribute, keys_size); - - if constexpr (std::is_same_v) - { - auto * out = column.get(); - - getItemsInternalImpl( - attribute, - key_to_index, - [&](size_t, const Array & value, bool) - { - out->insert(value); - }); - } - else if constexpr (std::is_same_v) - { - auto * out = column.get(); - - if (is_attribute_nullable) - getItemsInternalImpl( - attribute, - key_to_index, - [&](size_t row, StringRef value, bool is_null) - { - (*vec_null_map_to)[row] = is_null; - out->insertData(value.data, value.size); - }); - else - getItemsInternalImpl( - attribute, - key_to_index, - [&](size_t, StringRef value, bool) - { - out->insertData(value.data, value.size); - }); - } - else - { - auto & out = column->getData(); - - if (is_attribute_nullable) - getItemsInternalImpl( - attribute, - key_to_index, - [&](size_t row, const auto value, bool is_null) - { - (*vec_null_map_to)[row] = is_null; - out[row] = value; - }); - else - getItemsInternalImpl( - attribute, - key_to_index, - [&](size_t row, const auto value, bool) - { - out[row] = value; - }); - } - - result = std::move(column); - }; - - callOnDictionaryAttributeType(attribute.type, type_call); - - if (is_attribute_nullable) - result = ColumnNullable::create(result, std::move(col_null_map_to)); - - return result; -} - -template -ColumnUInt8::Ptr RangeHashedDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types) const -{ - if (dictionary_key_type == DictionaryKeyType::Complex) - { - auto key_types_copy = key_types; - key_types_copy.pop_back(); - dict_struct.validateKeyTypes(key_types_copy); - } - - /// Cast range column to storage type - const ColumnPtr & range_storage_column = key_columns.back(); - ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types.back(), ""}; - auto range_column_updated = castColumnAccurate(column_to_cast, dict_struct.range_min->type); - auto key_columns_copy = key_columns; - key_columns_copy.pop_back(); - - DictionaryKeysArenaHolder arena_holder; - DictionaryKeysExtractor keys_extractor(key_columns_copy, arena_holder.getComplexKeyArena()); - const size_t keys_size = keys_extractor.getKeysSize(); - - auto result = ColumnUInt8::create(keys_size); - auto & out = result->getData(); - size_t keys_found = 0; - - callOnRangeType(dict_struct.range_min->type, [&](const auto & types) - { - using Types = std::decay_t; - using RangeColumnType = typename Types::LeftType; - using RangeStorageType = typename RangeColumnType::ValueType; - - const auto * range_column_typed = typeid_cast(range_column_updated.get()); - if (!range_column_typed) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Dictionary {} range column type should be equal to {}", - getFullName(), - dict_struct.range_min->type->getName()); - const auto & range_column_data = range_column_typed->getData(); - - const auto & key_attribute_container = std::get>(key_attribute.container); - - for (size_t key_index = 0; key_index < keys_size; ++key_index) - { - const auto key = keys_extractor.extractCurrentKey(); - const auto it = key_attribute_container.find(key); - - if (it) - { - const auto date = range_column_data[key_index]; - const auto & interval_tree = it->getMapped(); - out[key_index] = interval_tree.has(date); - keys_found += out[key_index]; - } - else - { - out[key_index] = false; - } - - keys_extractor.rollbackCurrentKey(); - } - }); - - query_count.fetch_add(keys_size, std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); - - return result; -} - -template -void RangeHashedDictionary::createAttributes() -{ - const auto size = dict_struct.attributes.size(); - attributes.reserve(size); - - for (const auto & attribute : dict_struct.attributes) - { - attributes.push_back(createAttribute(attribute)); - - if (attribute.hierarchical) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Hierarchical attributes not supported by {} dictionary.", - getDictionaryID().getNameForLogs()); - } - - callOnRangeType(dict_struct.range_min->type, [&](const auto & types) - { - using Types = std::decay_t; - using RangeColumnType = typename Types::LeftType; - using RangeStorageType = typename RangeColumnType::ValueType; - - key_attribute.container = KeyAttributeContainerType(); - key_attribute.invalid_intervals_container = InvalidIntervalsContainerType(); - }); -} - -template -void RangeHashedDictionary::loadData() -{ - if (!source_ptr->hasUpdateField()) - { - QueryPipeline pipeline(source_ptr->loadAll()); - PullingPipelineExecutor executor(pipeline); - Block block; - - while (executor.pull(block)) - { - blockToAttributes(block); - } - } - else - { - updateData(); - } - - callOnRangeType(dict_struct.range_min->type, [&](const auto & types) - { - using Types = std::decay_t; - using RangeColumnType = typename Types::LeftType; - using RangeStorageType = typename RangeColumnType::ValueType; - - auto & key_attribute_container = std::get>(key_attribute.container); - - for (auto & [_, intervals] : key_attribute_container) - intervals.build(); - }); - - if (configuration.require_nonempty && 0 == element_count) - throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, - "{}: dictionary source is empty and 'require_nonempty' property is set."); -} - -template -void RangeHashedDictionary::calculateBytesAllocated() -{ - callOnRangeType(dict_struct.range_min->type, [&](const auto & types) - { - using Types = std::decay_t; - using RangeColumnType = typename Types::LeftType; - using RangeStorageType = typename RangeColumnType::ValueType; - - auto & key_attribute_container = std::get>(key_attribute.container); - - bucket_count = key_attribute_container.getBufferSizeInCells(); - bytes_allocated += key_attribute_container.getBufferSizeInBytes(); - - for (auto & [_, intervals] : key_attribute_container) - bytes_allocated += intervals.getSizeInBytes(); - }); - - bytes_allocated += attributes.size() * sizeof(attributes.front()); - for (const auto & attribute : attributes) - { - auto type_call = [&](const auto & dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - using ValueType = DictionaryValueType; - - const auto & container = std::get>(attribute.container); - - bytes_allocated += container.size() * sizeof(ValueType); - - if (attribute.is_value_nullable) - bytes_allocated += (*attribute.is_value_nullable).size() * sizeof(bool); - }; - - callOnDictionaryAttributeType(attribute.type, type_call); - } - - if (update_field_loaded_block) - bytes_allocated += update_field_loaded_block->allocatedBytes(); - - bytes_allocated += string_arena.size(); -} - -template -typename RangeHashedDictionary::Attribute RangeHashedDictionary::createAttribute(const DictionaryAttribute & dictionary_attribute) -{ - std::optional> is_value_nullable; - - if (dictionary_attribute.is_nullable) - is_value_nullable.emplace(std::vector()); - - Attribute attribute{dictionary_attribute.underlying_type, {}, std::move(is_value_nullable)}; - - auto type_call = [&](const auto & dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - using ValueType = DictionaryValueType; - - attribute.container = AttributeContainerType(); - }; - - callOnDictionaryAttributeType(dictionary_attribute.underlying_type, type_call); - - return attribute; -} - -template -template -void RangeHashedDictionary::getItemsImpl( - const Attribute & attribute, - const Columns & key_columns, - ValueSetter && set_value, - DefaultValueExtractor & default_value_extractor) const -{ - const auto & attribute_container = std::get>(attribute.container); - - size_t keys_found = 0; - - const ColumnPtr & range_column = key_columns.back(); - auto key_columns_copy = key_columns; - key_columns_copy.pop_back(); - - DictionaryKeysArenaHolder arena_holder; - DictionaryKeysExtractor keys_extractor(key_columns_copy, arena_holder.getComplexKeyArena()); - const size_t keys_size = keys_extractor.getKeysSize(); - - callOnRangeType(dict_struct.range_min->type, [&](const auto & types) - { - using Types = std::decay_t; - using RangeColumnType = typename Types::LeftType; - using RangeStorageType = typename RangeColumnType::ValueType; - using RangeInterval = Interval; - - const auto * range_column_typed = typeid_cast(range_column.get()); - if (!range_column_typed) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Dictionary {} range column type should be equal to {}", - getFullName(), - dict_struct.range_min->type->getName()); - - const auto & range_column_data = range_column_typed->getData(); - - const auto & key_attribute_container = std::get>(key_attribute.container); - - for (size_t key_index = 0; key_index < keys_size; ++key_index) - { - auto key = keys_extractor.extractCurrentKey(); - const auto it = key_attribute_container.find(key); - - if (it) - { - const auto date = range_column_data[key_index]; - const auto & interval_tree = it->getMapped(); - - size_t value_index = 0; - std::optional range; - - interval_tree.find(date, [&](auto & interval, auto & interval_value_index) - { - if (range) - { - if (likely(configuration.lookup_strategy == RangeHashedDictionaryLookupStrategy::min) && interval < *range) - { - range = interval; - value_index = interval_value_index; - } - else if (configuration.lookup_strategy == RangeHashedDictionaryLookupStrategy::max && interval > * range) - { - range = interval; - value_index = interval_value_index; - } - } - else - { - range = interval; - value_index = interval_value_index; - } - - return true; - }); - - if (range.has_value()) - { - ++keys_found; - - AttributeType value = attribute_container[value_index]; - - if constexpr (is_nullable) - { - bool is_null = (*attribute.is_value_nullable)[value_index]; - - if (!is_null) - set_value(key_index, value, false); - else - set_value(key_index, default_value_extractor[key_index], true); - } - else - { - set_value(key_index, value, false); - } - - keys_extractor.rollbackCurrentKey(); - continue; - } - } - - if constexpr (is_nullable) - set_value(key_index, default_value_extractor[key_index], default_value_extractor.isNullAt(key_index)); - else - set_value(key_index, default_value_extractor[key_index], false); - - keys_extractor.rollbackCurrentKey(); - } - }); - - query_count.fetch_add(keys_size, std::memory_order_relaxed); - found_count.fetch_add(keys_found, std::memory_order_relaxed); -} - -template -template -void RangeHashedDictionary::getItemsInternalImpl( - const Attribute & attribute, - const PaddedPODArray & key_to_index, - ValueSetter && set_value) const -{ - size_t keys_size = key_to_index.size(); - - const auto & container = std::get>(attribute.container); - size_t container_size = container.size(); - - for (size_t key_index = 0; key_index < keys_size; ++key_index) - { - UInt64 container_index = key_to_index[key_index]; - - if (unlikely(container_index >= container_size)) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Dictionary {} expected attribute container index {} must be less than attribute container size {}", - getFullName(), - container_index, - container_size - ); - } - - AttributeType value = container[container_index]; - - if constexpr (is_nullable) - { - bool is_null = (*attribute.is_value_nullable)[container_index]; - - if (!is_null) - set_value(key_index, value, false); - else - set_value(key_index, value, true); - } - else - { - set_value(key_index, value, false); - } - } - - query_count.fetch_add(keys_size, std::memory_order_relaxed); - found_count.fetch_add(keys_size, std::memory_order_relaxed); -} - -template -void RangeHashedDictionary::updateData() -{ - if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) - { - QueryPipeline pipeline(source_ptr->loadUpdatedAll()); - - PullingPipelineExecutor executor(pipeline); - Block block; - while (executor.pull(block)) - { - /// We are using this to keep saved data if input stream consists of multiple blocks - if (!update_field_loaded_block) - update_field_loaded_block = std::make_shared(block.cloneEmpty()); - - for (size_t attribute_index = 0; attribute_index < block.columns(); ++attribute_index) - { - const IColumn & update_column = *block.getByPosition(attribute_index).column.get(); - MutableColumnPtr saved_column = update_field_loaded_block->getByPosition(attribute_index).column->assumeMutable(); - saved_column->insertRangeFrom(update_column, 0, update_column.size()); - } - } - } - else - { - static constexpr size_t range_columns_size = 2; - - auto pipe = source_ptr->loadUpdatedAll(); - - /// Use complex dictionary key type to count range columns as part of complex primary key during update - mergeBlockWithPipe( - dict_struct.getKeysSize() + range_columns_size, - *update_field_loaded_block, - std::move(pipe)); - } - - if (update_field_loaded_block) - { - blockToAttributes(*update_field_loaded_block.get()); - } -} - -template -void RangeHashedDictionary::blockToAttributes(const Block & block) -{ - size_t attributes_size = attributes.size(); - size_t dictionary_keys_size = dict_struct.getKeysSize(); - - static constexpr size_t ranges_size = 2; - - size_t block_columns = block.columns(); - size_t range_dictionary_attributes_size = attributes_size + dictionary_keys_size + ranges_size; - - if (range_dictionary_attributes_size != block.columns()) - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Block size mismatch. Actual {}. Expected {}", - block_columns, - range_dictionary_attributes_size); - } - - Columns key_columns; - key_columns.reserve(dictionary_keys_size); - - /// Split into keys columns and attribute columns - for (size_t i = 0; i < dictionary_keys_size; ++i) - key_columns.emplace_back(block.getByPosition(i).column); - - DictionaryKeysArenaHolder arena_holder; - DictionaryKeysExtractor keys_extractor(key_columns, arena_holder.getComplexKeyArena()); - const size_t keys_size = keys_extractor.getKeysSize(); - - size_t block_attributes_skip_offset = dictionary_keys_size; - - const auto * min_range_column = block.getByPosition(block_attributes_skip_offset).column.get(); - const auto * max_range_column = block.getByPosition(block_attributes_skip_offset + 1).column.get(); - - const NullMap * min_range_null_map = nullptr; - const NullMap * max_range_null_map = nullptr; - - if (const auto * min_range_column_nullable = checkAndGetColumn(min_range_column)) - { - min_range_column = &min_range_column_nullable->getNestedColumn(); - min_range_null_map = &min_range_column_nullable->getNullMapColumn().getData(); - } - - if (const auto * max_range_column_nullable = checkAndGetColumn(max_range_column)) - { - max_range_column = &max_range_column_nullable->getNestedColumn(); - max_range_null_map = &max_range_column_nullable->getNullMapColumn().getData(); - } - - callOnRangeType(dict_struct.range_min->type, [&](const auto & types) - { - using Types = std::decay_t; - using RangeColumnType = typename Types::LeftType; - using RangeStorageType = typename RangeColumnType::ValueType; - - const auto * min_range_column_typed = typeid_cast(min_range_column); - if (!min_range_column_typed) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Dictionary {} range min column type should be equal to {}", - getFullName(), - dict_struct.range_min->type->getName()); - - const auto * max_range_column_typed = typeid_cast(max_range_column); - if (!max_range_column_typed) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Dictionary {} range max column type should be equal to {}", - getFullName(), - dict_struct.range_max->type->getName()); - - const auto & min_range_column_data = min_range_column_typed->getData(); - const auto & max_range_column_data = max_range_column_typed->getData(); - - auto & key_attribute_container = std::get>(key_attribute.container); - auto & invalid_intervals_container = std::get>(key_attribute.invalid_intervals_container); - - block_attributes_skip_offset += 2; - - Field column_value; - - for (size_t key_index = 0; key_index < keys_size; ++key_index) - { - auto key = keys_extractor.extractCurrentKey(); - - RangeStorageType lower_bound = min_range_column_data[key_index]; - RangeStorageType upper_bound = max_range_column_data[key_index]; - - bool invalid_range = false; - - if (unlikely(min_range_null_map && (*min_range_null_map)[key_index])) - { - lower_bound = std::numeric_limits::min(); - invalid_range = true; - } - - if (unlikely(max_range_null_map && (*max_range_null_map)[key_index])) - { - upper_bound = std::numeric_limits::max(); - invalid_range = true; - } - - if (unlikely(!configuration.convert_null_range_bound_to_open && invalid_range)) - { - keys_extractor.rollbackCurrentKey(); - continue; - } - - if constexpr (std::is_same_v) - key = copyStringInArena(string_arena, key); - - for (size_t attribute_index = 0; attribute_index < attributes.size(); ++attribute_index) - { - const auto & attribute_column = *block.getByPosition(attribute_index + block_attributes_skip_offset).column; - auto & attribute = attributes[attribute_index]; - attribute_column.get(key_index, column_value); - - setAttributeValue(attribute, column_value); - } - - auto interval = Interval(lower_bound, upper_bound); - auto it = key_attribute_container.find(key); - - bool emplaced_in_interval_tree = false; - - if (it) - { - auto & intervals = it->getMapped(); - emplaced_in_interval_tree = intervals.emplace(interval, element_count); - } - else - { - IntervalMap intervals; - emplaced_in_interval_tree = intervals.emplace(interval, element_count); - key_attribute_container.insert({key, std::move(intervals)}); - } - - if (unlikely(!emplaced_in_interval_tree)) - { - InvalidIntervalWithKey invalid_interval{key, interval, element_count}; - invalid_intervals_container.emplace_back(invalid_interval); - } - - ++element_count; - keys_extractor.rollbackCurrentKey(); - } - }); -} - -template -void RangeHashedDictionary::setAttributeValue(Attribute & attribute, const Field & value) -{ - auto type_call = [&](const auto & dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - using ValueType = DictionaryValueType; - - auto & container = std::get>(attribute.container); - container.emplace_back(); - - if (unlikely(attribute.is_value_nullable.has_value())) - { - bool value_is_null = value.isNull(); - attribute.is_value_nullable->emplace_back(value_is_null); - - if (unlikely(value_is_null)) - return; - } - - ValueType value_to_insert; - - if constexpr (std::is_same_v) - { - const auto & string = value.get(); - StringRef string_ref = copyStringInArena(string_arena, string); - value_to_insert = string_ref; - } - else - { - value_to_insert = static_cast(value.get()); - } - - container.back() = value_to_insert; - }; - - callOnDictionaryAttributeType(attribute.type, type_call); -} - -template -Pipe RangeHashedDictionary::read(const Names & column_names, size_t max_block_size, size_t num_streams) const -{ - auto key_to_index_column = ColumnUInt64::create(); - auto range_min_column = dict_struct.range_min->type->createColumn(); - auto range_max_column = dict_struct.range_max->type->createColumn(); - - PaddedPODArray keys; - - callOnRangeType(dict_struct.range_min->type, [&](const auto & types) - { - using Types = std::decay_t; - using RangeColumnType = typename Types::LeftType; - using RangeStorageType = typename RangeColumnType::ValueType; - - auto * range_min_column_typed = typeid_cast(range_min_column.get()); - if (!range_min_column_typed) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Dictionary {} range min column type should be equal to {}", - getFullName(), - dict_struct.range_min->type->getName()); - - auto * range_max_column_typed = typeid_cast(range_max_column.get()); - if (!range_max_column_typed) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Dictionary {} range max column type should be equal to {}", - getFullName(), - dict_struct.range_max->type->getName()); - - auto & key_to_index_column_data = key_to_index_column->getData(); - auto & range_min_column_data = range_min_column_typed->getData(); - auto & range_max_column_data = range_max_column_typed->getData(); - - const auto & container = std::get>(key_attribute.container); - const auto & invalid_intervals_container = std::get>(key_attribute.invalid_intervals_container); - - keys.reserve(element_count); - key_to_index_column_data.reserve(element_count); - range_min_column_data.reserve(element_count); - range_max_column_data.reserve(element_count); - - for (const auto & key : container) - { - for (const auto & [interval, index] : key.getMapped()) - { - keys.emplace_back(key.getKey()); - key_to_index_column_data.emplace_back(index); - range_min_column_data.push_back(interval.left); - range_max_column_data.push_back(interval.right); - } - } - - for (const auto & invalid_interval_with_key : invalid_intervals_container) - { - keys.emplace_back(invalid_interval_with_key.key); - key_to_index_column_data.emplace_back(invalid_interval_with_key.attribute_value_index); - range_min_column_data.push_back(invalid_interval_with_key.interval.left); - range_max_column_data.push_back(invalid_interval_with_key.interval.right); - } - }); - - auto range_min_column_with_type = ColumnWithTypeAndName{std::move(range_min_column), dict_struct.range_min->type, dict_struct.range_min->name}; - auto range_max_column_with_type = ColumnWithTypeAndName{std::move(range_max_column), dict_struct.range_max->type, dict_struct.range_max->name}; - - ColumnsWithTypeAndName key_columns; - if constexpr (dictionary_key_type == DictionaryKeyType::Simple) - { - auto keys_column = getColumnFromPODArray(std::move(keys)); - key_columns = {ColumnWithTypeAndName(std::move(keys_column), std::make_shared(), dict_struct.id->name)}; - } - else - { - key_columns = deserializeColumnsWithTypeAndNameFromKeys(dict_struct, keys, 0, keys.size()); - } - - key_columns.emplace_back(ColumnWithTypeAndName{std::move(key_to_index_column), std::make_shared(), ""}); - - ColumnsWithTypeAndName data_columns = {std::move(range_min_column_with_type), std::move(range_max_column_with_type)}; - - std::shared_ptr dictionary = shared_from_this(); - - DictionarySourceCoordinator::ReadColumnsFunc read_keys_func = [dictionary_copy = dictionary]( - const Strings & attribute_names, - const DataTypes & result_types, - const Columns & key_columns, - const DataTypes, - const Columns &) - { - auto range_dictionary_ptr = std::static_pointer_cast>(dictionary_copy); - - size_t attribute_names_size = attribute_names.size(); - - Columns result; - result.reserve(attribute_names_size); - - const ColumnPtr & key_column = key_columns.back(); - - const auto * key_to_index_column = typeid_cast(key_column.get()); - if (!key_to_index_column) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Dictionary {} read expect indexes column with type UInt64", - range_dictionary_ptr->getFullName()); - - const auto & data = key_to_index_column->getData(); - - for (size_t i = 0; i < attribute_names_size; ++i) - { - const auto & attribute_name = attribute_names[i]; - const auto & result_type = result_types[i]; - - result.emplace_back(range_dictionary_ptr->getColumnInternal(attribute_name, result_type, data)); - } - - return result; - }; - - auto coordinator = std::make_shared( - dictionary, - column_names, - std::move(key_columns), - std::move(data_columns), - max_block_size, - std::move(read_keys_func)); - auto result = coordinator->read(num_streams); - - return result; -} - -template class RangeHashedDictionary; -template class RangeHashedDictionary; - -} diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index abff492471e..d6bb510542e 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -9,16 +9,42 @@ #include #include #include +#include #include #include #include #include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include + +#include + namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int DICTIONARY_IS_EMPTY; + extern const int UNSUPPORTED_METHOD; + extern const int TYPE_MISMATCH; +} + + enum class RangeHashedDictionaryLookupStrategy : uint8_t { min, @@ -255,4 +281,994 @@ private: extern template class RangeHashedDictionary; extern template class RangeHashedDictionary; +namespace +{ + template + void callOnRangeType(const DataTypePtr & range_type, F && func) + { + auto call = [&](const auto & types) + { + using Types = std::decay_t; + using DataType = typename Types::LeftType; + + if constexpr (IsDataTypeDecimalOrNumber || IsDataTypeDateOrDateTime || IsDataTypeEnum) + { + using ColumnType = typename DataType::ColumnType; + func(TypePair()); + return true; + } + + return false; + }; + + auto type_index = range_type->getTypeId(); + if (!callOnIndexAndDataType(type_index, call)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Dictionary structure type of 'range_min' and 'range_max' should " + "be an Integer, Float, Decimal, Date, Date32, DateTime DateTime64, or Enum." + " Actual 'range_min' and 'range_max' type is {}", + range_type->getName()); + } + } +} + +template +RangeHashedDictionary::RangeHashedDictionary( + const StorageID & dict_id_, + const DictionaryStructure & dict_struct_, + DictionarySourcePtr source_ptr_, + DictionaryLifetime dict_lifetime_, + RangeHashedDictionaryConfiguration configuration_, + BlockPtr update_field_loaded_block_) + : IDictionary(dict_id_) + , dict_struct(dict_struct_) + , source_ptr(std::move(source_ptr_)) + , dict_lifetime(dict_lifetime_) + , configuration(configuration_) + , update_field_loaded_block(std::move(update_field_loaded_block_)) +{ + createAttributes(); + loadData(); + calculateBytesAllocated(); +} + +template +ColumnPtr RangeHashedDictionary::getColumn( + const std::string & attribute_name, + const DataTypePtr & result_type, + const Columns & key_columns, + const DataTypes & key_types, + const ColumnPtr & default_values_column) const +{ + if (dictionary_key_type == DictionaryKeyType::Complex) + { + auto key_types_copy = key_types; + key_types_copy.pop_back(); + dict_struct.validateKeyTypes(key_types_copy); + } + + ColumnPtr result; + + const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); + const size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; + const auto & attribute = attributes[attribute_index]; + + /// Cast range column to storage type + Columns modified_key_columns = key_columns; + const ColumnPtr & range_storage_column = key_columns.back(); + ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types.back(), ""}; + modified_key_columns.back() = castColumnAccurate(column_to_cast, dict_struct.range_min->type); + + size_t keys_size = key_columns.front()->size(); + bool is_attribute_nullable = attribute.is_value_nullable.has_value(); + + ColumnUInt8::MutablePtr col_null_map_to; + ColumnUInt8::Container * vec_null_map_to = nullptr; + if (is_attribute_nullable) + { + col_null_map_to = ColumnUInt8::create(keys_size, false); + vec_null_map_to = &col_null_map_to->getData(); + } + + auto type_call = [&](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + using ColumnProvider = DictionaryAttributeColumnProvider; + + DictionaryDefaultValueExtractor default_value_extractor(dictionary_attribute.null_value, default_values_column); + + auto column = ColumnProvider::getColumn(dictionary_attribute, keys_size); + + if constexpr (std::is_same_v) + { + auto * out = column.get(); + + getItemsImpl( + attribute, + modified_key_columns, + [&](size_t, const Array & value, bool) + { + out->insert(value); + }, + default_value_extractor); + } + else if constexpr (std::is_same_v) + { + auto * out = column.get(); + + if (is_attribute_nullable) + getItemsImpl( + attribute, + modified_key_columns, + [&](size_t row, StringRef value, bool is_null) + { + (*vec_null_map_to)[row] = is_null; + out->insertData(value.data, value.size); + }, + default_value_extractor); + else + getItemsImpl( + attribute, + modified_key_columns, + [&](size_t, StringRef value, bool) + { + out->insertData(value.data, value.size); + }, + default_value_extractor); + } + else + { + auto & out = column->getData(); + + if (is_attribute_nullable) + getItemsImpl( + attribute, + modified_key_columns, + [&](size_t row, const auto value, bool is_null) + { + (*vec_null_map_to)[row] = is_null; + out[row] = value; + }, + default_value_extractor); + else + getItemsImpl( + attribute, + modified_key_columns, + [&](size_t row, const auto value, bool) + { + out[row] = value; + }, + default_value_extractor); + } + + result = std::move(column); + }; + + callOnDictionaryAttributeType(attribute.type, type_call); + + if (is_attribute_nullable) + result = ColumnNullable::create(result, std::move(col_null_map_to)); + + return result; +} + +template +ColumnPtr RangeHashedDictionary::getColumnInternal( + const std::string & attribute_name, + const DataTypePtr & result_type, + const PaddedPODArray & key_to_index) const +{ + ColumnPtr result; + + const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); + const size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; + const auto & attribute = attributes[attribute_index]; + + size_t keys_size = key_to_index.size(); + bool is_attribute_nullable = attribute.is_value_nullable.has_value(); + + ColumnUInt8::MutablePtr col_null_map_to; + ColumnUInt8::Container * vec_null_map_to = nullptr; + if (is_attribute_nullable) + { + col_null_map_to = ColumnUInt8::create(keys_size, false); + vec_null_map_to = &col_null_map_to->getData(); + } + + auto type_call = [&](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + using ColumnProvider = DictionaryAttributeColumnProvider; + + auto column = ColumnProvider::getColumn(dictionary_attribute, keys_size); + + if constexpr (std::is_same_v) + { + auto * out = column.get(); + + getItemsInternalImpl( + attribute, + key_to_index, + [&](size_t, const Array & value, bool) + { + out->insert(value); + }); + } + else if constexpr (std::is_same_v) + { + auto * out = column.get(); + + if (is_attribute_nullable) + getItemsInternalImpl( + attribute, + key_to_index, + [&](size_t row, StringRef value, bool is_null) + { + (*vec_null_map_to)[row] = is_null; + out->insertData(value.data, value.size); + }); + else + getItemsInternalImpl( + attribute, + key_to_index, + [&](size_t, StringRef value, bool) + { + out->insertData(value.data, value.size); + }); + } + else + { + auto & out = column->getData(); + + if (is_attribute_nullable) + getItemsInternalImpl( + attribute, + key_to_index, + [&](size_t row, const auto value, bool is_null) + { + (*vec_null_map_to)[row] = is_null; + out[row] = value; + }); + else + getItemsInternalImpl( + attribute, + key_to_index, + [&](size_t row, const auto value, bool) + { + out[row] = value; + }); + } + + result = std::move(column); + }; + + callOnDictionaryAttributeType(attribute.type, type_call); + + if (is_attribute_nullable) + result = ColumnNullable::create(result, std::move(col_null_map_to)); + + return result; +} + +template +ColumnUInt8::Ptr RangeHashedDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types) const +{ + if (dictionary_key_type == DictionaryKeyType::Complex) + { + auto key_types_copy = key_types; + key_types_copy.pop_back(); + dict_struct.validateKeyTypes(key_types_copy); + } + + /// Cast range column to storage type + const ColumnPtr & range_storage_column = key_columns.back(); + ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types.back(), ""}; + auto range_column_updated = castColumnAccurate(column_to_cast, dict_struct.range_min->type); + auto key_columns_copy = key_columns; + key_columns_copy.pop_back(); + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor keys_extractor(key_columns_copy, arena_holder.getComplexKeyArena()); + const size_t keys_size = keys_extractor.getKeysSize(); + + auto result = ColumnUInt8::create(keys_size); + auto & out = result->getData(); + size_t keys_found = 0; + + callOnRangeType(dict_struct.range_min->type, [&](const auto & types) + { + using Types = std::decay_t; + using RangeColumnType = typename Types::LeftType; + using RangeStorageType = typename RangeColumnType::ValueType; + + const auto * range_column_typed = typeid_cast(range_column_updated.get()); + if (!range_column_typed) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Dictionary {} range column type should be equal to {}", + getFullName(), + dict_struct.range_min->type->getName()); + const auto & range_column_data = range_column_typed->getData(); + + const auto & key_attribute_container = std::get>(key_attribute.container); + + for (size_t key_index = 0; key_index < keys_size; ++key_index) + { + const auto key = keys_extractor.extractCurrentKey(); + const auto it = key_attribute_container.find(key); + + if (it) + { + const auto date = range_column_data[key_index]; + const auto & interval_tree = it->getMapped(); + out[key_index] = interval_tree.has(date); + keys_found += out[key_index]; + } + else + { + out[key_index] = false; + } + + keys_extractor.rollbackCurrentKey(); + } + }); + + query_count.fetch_add(keys_size, std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); + + return result; +} + +template +void RangeHashedDictionary::createAttributes() +{ + const auto size = dict_struct.attributes.size(); + attributes.reserve(size); + + for (const auto & attribute : dict_struct.attributes) + { + attributes.push_back(createAttribute(attribute)); + + if (attribute.hierarchical) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Hierarchical attributes not supported by {} dictionary.", + getDictionaryID().getNameForLogs()); + } + + callOnRangeType(dict_struct.range_min->type, [&](const auto & types) + { + using Types = std::decay_t; + using RangeColumnType = typename Types::LeftType; + using RangeStorageType = typename RangeColumnType::ValueType; + + key_attribute.container = KeyAttributeContainerType(); + key_attribute.invalid_intervals_container = InvalidIntervalsContainerType(); + }); +} + +template +void RangeHashedDictionary::loadData() +{ + if (!source_ptr->hasUpdateField()) + { + QueryPipeline pipeline(source_ptr->loadAll()); + PullingPipelineExecutor executor(pipeline); + Block block; + + while (executor.pull(block)) + { + blockToAttributes(block); + } + } + else + { + updateData(); + } + + callOnRangeType(dict_struct.range_min->type, [&](const auto & types) + { + using Types = std::decay_t; + using RangeColumnType = typename Types::LeftType; + using RangeStorageType = typename RangeColumnType::ValueType; + + auto & key_attribute_container = std::get>(key_attribute.container); + + for (auto & [_, intervals] : key_attribute_container) + intervals.build(); + }); + + if (configuration.require_nonempty && 0 == element_count) + throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, + "{}: dictionary source is empty and 'require_nonempty' property is set."); +} + +template +void RangeHashedDictionary::calculateBytesAllocated() +{ + callOnRangeType(dict_struct.range_min->type, [&](const auto & types) + { + using Types = std::decay_t; + using RangeColumnType = typename Types::LeftType; + using RangeStorageType = typename RangeColumnType::ValueType; + + auto & key_attribute_container = std::get>(key_attribute.container); + + bucket_count = key_attribute_container.getBufferSizeInCells(); + bytes_allocated += key_attribute_container.getBufferSizeInBytes(); + + for (auto & [_, intervals] : key_attribute_container) + bytes_allocated += intervals.getSizeInBytes(); + }); + + bytes_allocated += attributes.size() * sizeof(attributes.front()); + for (const auto & attribute : attributes) + { + auto type_call = [&](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + + const auto & container = std::get>(attribute.container); + + bytes_allocated += container.size() * sizeof(ValueType); + + if (attribute.is_value_nullable) + bytes_allocated += (*attribute.is_value_nullable).size() * sizeof(bool); + }; + + callOnDictionaryAttributeType(attribute.type, type_call); + } + + if (update_field_loaded_block) + bytes_allocated += update_field_loaded_block->allocatedBytes(); + + bytes_allocated += string_arena.size(); +} + +template +typename RangeHashedDictionary::Attribute RangeHashedDictionary::createAttribute(const DictionaryAttribute & dictionary_attribute) +{ + std::optional> is_value_nullable; + + if (dictionary_attribute.is_nullable) + is_value_nullable.emplace(std::vector()); + + Attribute attribute{dictionary_attribute.underlying_type, {}, std::move(is_value_nullable)}; + + auto type_call = [&](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + + attribute.container = AttributeContainerType(); + }; + + callOnDictionaryAttributeType(dictionary_attribute.underlying_type, type_call); + + return attribute; +} + +template +template +void RangeHashedDictionary::getItemsImpl( + const Attribute & attribute, + const Columns & key_columns, + ValueSetter && set_value, + DefaultValueExtractor & default_value_extractor) const +{ + const auto & attribute_container = std::get>(attribute.container); + + size_t keys_found = 0; + + const ColumnPtr & range_column = key_columns.back(); + auto key_columns_copy = key_columns; + key_columns_copy.pop_back(); + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor keys_extractor(key_columns_copy, arena_holder.getComplexKeyArena()); + const size_t keys_size = keys_extractor.getKeysSize(); + + callOnRangeType(dict_struct.range_min->type, [&](const auto & types) + { + using Types = std::decay_t; + using RangeColumnType = typename Types::LeftType; + using RangeStorageType = typename RangeColumnType::ValueType; + using RangeInterval = Interval; + + const auto * range_column_typed = typeid_cast(range_column.get()); + if (!range_column_typed) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Dictionary {} range column type should be equal to {}", + getFullName(), + dict_struct.range_min->type->getName()); + + const auto & range_column_data = range_column_typed->getData(); + + const auto & key_attribute_container = std::get>(key_attribute.container); + + for (size_t key_index = 0; key_index < keys_size; ++key_index) + { + auto key = keys_extractor.extractCurrentKey(); + const auto it = key_attribute_container.find(key); + + if (it) + { + const auto date = range_column_data[key_index]; + const auto & interval_tree = it->getMapped(); + + size_t value_index = 0; + std::optional range; + + interval_tree.find(date, [&](auto & interval, auto & interval_value_index) + { + if (range) + { + if (likely(configuration.lookup_strategy == RangeHashedDictionaryLookupStrategy::min) && interval < *range) + { + range = interval; + value_index = interval_value_index; + } + else if (configuration.lookup_strategy == RangeHashedDictionaryLookupStrategy::max && interval > * range) + { + range = interval; + value_index = interval_value_index; + } + } + else + { + range = interval; + value_index = interval_value_index; + } + + return true; + }); + + if (range.has_value()) + { + ++keys_found; + + AttributeType value = attribute_container[value_index]; + + if constexpr (is_nullable) + { + bool is_null = (*attribute.is_value_nullable)[value_index]; + + if (!is_null) + set_value(key_index, value, false); + else + set_value(key_index, default_value_extractor[key_index], true); + } + else + { + set_value(key_index, value, false); + } + + keys_extractor.rollbackCurrentKey(); + continue; + } + } + + if constexpr (is_nullable) + set_value(key_index, default_value_extractor[key_index], default_value_extractor.isNullAt(key_index)); + else + set_value(key_index, default_value_extractor[key_index], false); + + keys_extractor.rollbackCurrentKey(); + } + }); + + query_count.fetch_add(keys_size, std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); +} + +template +template +void RangeHashedDictionary::getItemsInternalImpl( + const Attribute & attribute, + const PaddedPODArray & key_to_index, + ValueSetter && set_value) const +{ + size_t keys_size = key_to_index.size(); + + const auto & container = std::get>(attribute.container); + size_t container_size = container.size(); + + for (size_t key_index = 0; key_index < keys_size; ++key_index) + { + UInt64 container_index = key_to_index[key_index]; + + if (unlikely(container_index >= container_size)) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Dictionary {} expected attribute container index {} must be less than attribute container size {}", + getFullName(), + container_index, + container_size + ); + } + + AttributeType value = container[container_index]; + + if constexpr (is_nullable) + { + bool is_null = (*attribute.is_value_nullable)[container_index]; + + if (!is_null) + set_value(key_index, value, false); + else + set_value(key_index, value, true); + } + else + { + set_value(key_index, value, false); + } + } + + query_count.fetch_add(keys_size, std::memory_order_relaxed); + found_count.fetch_add(keys_size, std::memory_order_relaxed); +} + +template +void RangeHashedDictionary::updateData() +{ + if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) + { + QueryPipeline pipeline(source_ptr->loadUpdatedAll()); + + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) + { + /// We are using this to keep saved data if input stream consists of multiple blocks + if (!update_field_loaded_block) + update_field_loaded_block = std::make_shared(block.cloneEmpty()); + + for (size_t attribute_index = 0; attribute_index < block.columns(); ++attribute_index) + { + const IColumn & update_column = *block.getByPosition(attribute_index).column.get(); + MutableColumnPtr saved_column = update_field_loaded_block->getByPosition(attribute_index).column->assumeMutable(); + saved_column->insertRangeFrom(update_column, 0, update_column.size()); + } + } + } + else + { + static constexpr size_t range_columns_size = 2; + + auto pipe = source_ptr->loadUpdatedAll(); + + /// Use complex dictionary key type to count range columns as part of complex primary key during update + mergeBlockWithPipe( + dict_struct.getKeysSize() + range_columns_size, + *update_field_loaded_block, + std::move(pipe)); + } + + if (update_field_loaded_block) + { + blockToAttributes(*update_field_loaded_block.get()); + } +} + +template +void RangeHashedDictionary::blockToAttributes(const Block & block) +{ + size_t attributes_size = attributes.size(); + size_t dictionary_keys_size = dict_struct.getKeysSize(); + + static constexpr size_t ranges_size = 2; + + size_t block_columns = block.columns(); + size_t range_dictionary_attributes_size = attributes_size + dictionary_keys_size + ranges_size; + + if (range_dictionary_attributes_size != block.columns()) + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Block size mismatch. Actual {}. Expected {}", + block_columns, + range_dictionary_attributes_size); + } + + Columns key_columns; + key_columns.reserve(dictionary_keys_size); + + /// Split into keys columns and attribute columns + for (size_t i = 0; i < dictionary_keys_size; ++i) + key_columns.emplace_back(block.getByPosition(i).column); + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor keys_extractor(key_columns, arena_holder.getComplexKeyArena()); + const size_t keys_size = keys_extractor.getKeysSize(); + + size_t block_attributes_skip_offset = dictionary_keys_size; + + const auto * min_range_column = block.getByPosition(block_attributes_skip_offset).column.get(); + const auto * max_range_column = block.getByPosition(block_attributes_skip_offset + 1).column.get(); + + const NullMap * min_range_null_map = nullptr; + const NullMap * max_range_null_map = nullptr; + + if (const auto * min_range_column_nullable = checkAndGetColumn(min_range_column)) + { + min_range_column = &min_range_column_nullable->getNestedColumn(); + min_range_null_map = &min_range_column_nullable->getNullMapColumn().getData(); + } + + if (const auto * max_range_column_nullable = checkAndGetColumn(max_range_column)) + { + max_range_column = &max_range_column_nullable->getNestedColumn(); + max_range_null_map = &max_range_column_nullable->getNullMapColumn().getData(); + } + + callOnRangeType(dict_struct.range_min->type, [&](const auto & types) + { + using Types = std::decay_t; + using RangeColumnType = typename Types::LeftType; + using RangeStorageType = typename RangeColumnType::ValueType; + + const auto * min_range_column_typed = typeid_cast(min_range_column); + if (!min_range_column_typed) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Dictionary {} range min column type should be equal to {}", + getFullName(), + dict_struct.range_min->type->getName()); + + const auto * max_range_column_typed = typeid_cast(max_range_column); + if (!max_range_column_typed) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Dictionary {} range max column type should be equal to {}", + getFullName(), + dict_struct.range_max->type->getName()); + + const auto & min_range_column_data = min_range_column_typed->getData(); + const auto & max_range_column_data = max_range_column_typed->getData(); + + auto & key_attribute_container = std::get>(key_attribute.container); + auto & invalid_intervals_container = std::get>(key_attribute.invalid_intervals_container); + + block_attributes_skip_offset += 2; + + Field column_value; + + for (size_t key_index = 0; key_index < keys_size; ++key_index) + { + auto key = keys_extractor.extractCurrentKey(); + + RangeStorageType lower_bound = min_range_column_data[key_index]; + RangeStorageType upper_bound = max_range_column_data[key_index]; + + bool invalid_range = false; + + if (unlikely(min_range_null_map && (*min_range_null_map)[key_index])) + { + lower_bound = std::numeric_limits::min(); + invalid_range = true; + } + + if (unlikely(max_range_null_map && (*max_range_null_map)[key_index])) + { + upper_bound = std::numeric_limits::max(); + invalid_range = true; + } + + if (unlikely(!configuration.convert_null_range_bound_to_open && invalid_range)) + { + keys_extractor.rollbackCurrentKey(); + continue; + } + + if constexpr (std::is_same_v) + key = copyStringInArena(string_arena, key); + + for (size_t attribute_index = 0; attribute_index < attributes.size(); ++attribute_index) + { + const auto & attribute_column = *block.getByPosition(attribute_index + block_attributes_skip_offset).column; + auto & attribute = attributes[attribute_index]; + attribute_column.get(key_index, column_value); + + setAttributeValue(attribute, column_value); + } + + auto interval = Interval(lower_bound, upper_bound); + auto it = key_attribute_container.find(key); + + bool emplaced_in_interval_tree = false; + + if (it) + { + auto & intervals = it->getMapped(); + emplaced_in_interval_tree = intervals.emplace(interval, element_count); + } + else + { + IntervalMap intervals; + emplaced_in_interval_tree = intervals.emplace(interval, element_count); + key_attribute_container.insert({key, std::move(intervals)}); + } + + if (unlikely(!emplaced_in_interval_tree)) + { + InvalidIntervalWithKey invalid_interval{key, interval, element_count}; + invalid_intervals_container.emplace_back(invalid_interval); + } + + ++element_count; + keys_extractor.rollbackCurrentKey(); + } + }); +} + +template +void RangeHashedDictionary::setAttributeValue(Attribute & attribute, const Field & value) +{ + auto type_call = [&](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + + auto & container = std::get>(attribute.container); + container.emplace_back(); + + if (unlikely(attribute.is_value_nullable.has_value())) + { + bool value_is_null = value.isNull(); + attribute.is_value_nullable->emplace_back(value_is_null); + + if (unlikely(value_is_null)) + return; + } + + ValueType value_to_insert; + + if constexpr (std::is_same_v) + { + const auto & string = value.get(); + StringRef string_ref = copyStringInArena(string_arena, string); + value_to_insert = string_ref; + } + else + { + value_to_insert = static_cast(value.get()); + } + + container.back() = value_to_insert; + }; + + callOnDictionaryAttributeType(attribute.type, type_call); +} + +template +Pipe RangeHashedDictionary::read(const Names & column_names, size_t max_block_size, size_t num_streams) const +{ + auto key_to_index_column = ColumnUInt64::create(); + auto range_min_column = dict_struct.range_min->type->createColumn(); + auto range_max_column = dict_struct.range_max->type->createColumn(); + + PaddedPODArray keys; + + callOnRangeType(dict_struct.range_min->type, [&](const auto & types) + { + using Types = std::decay_t; + using RangeColumnType = typename Types::LeftType; + using RangeStorageType = typename RangeColumnType::ValueType; + + auto * range_min_column_typed = typeid_cast(range_min_column.get()); + if (!range_min_column_typed) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Dictionary {} range min column type should be equal to {}", + getFullName(), + dict_struct.range_min->type->getName()); + + auto * range_max_column_typed = typeid_cast(range_max_column.get()); + if (!range_max_column_typed) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Dictionary {} range max column type should be equal to {}", + getFullName(), + dict_struct.range_max->type->getName()); + + auto & key_to_index_column_data = key_to_index_column->getData(); + auto & range_min_column_data = range_min_column_typed->getData(); + auto & range_max_column_data = range_max_column_typed->getData(); + + const auto & container = std::get>(key_attribute.container); + const auto & invalid_intervals_container = std::get>(key_attribute.invalid_intervals_container); + + keys.reserve(element_count); + key_to_index_column_data.reserve(element_count); + range_min_column_data.reserve(element_count); + range_max_column_data.reserve(element_count); + + for (const auto & key : container) + { + for (const auto & [interval, index] : key.getMapped()) + { + keys.emplace_back(key.getKey()); + key_to_index_column_data.emplace_back(index); + range_min_column_data.push_back(interval.left); + range_max_column_data.push_back(interval.right); + } + } + + for (const auto & invalid_interval_with_key : invalid_intervals_container) + { + keys.emplace_back(invalid_interval_with_key.key); + key_to_index_column_data.emplace_back(invalid_interval_with_key.attribute_value_index); + range_min_column_data.push_back(invalid_interval_with_key.interval.left); + range_max_column_data.push_back(invalid_interval_with_key.interval.right); + } + }); + + auto range_min_column_with_type = ColumnWithTypeAndName{std::move(range_min_column), dict_struct.range_min->type, dict_struct.range_min->name}; + auto range_max_column_with_type = ColumnWithTypeAndName{std::move(range_max_column), dict_struct.range_max->type, dict_struct.range_max->name}; + + ColumnsWithTypeAndName key_columns; + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + { + auto keys_column = getColumnFromPODArray(std::move(keys)); + key_columns = {ColumnWithTypeAndName(std::move(keys_column), std::make_shared(), dict_struct.id->name)}; + } + else + { + key_columns = deserializeColumnsWithTypeAndNameFromKeys(dict_struct, keys, 0, keys.size()); + } + + key_columns.emplace_back(ColumnWithTypeAndName{std::move(key_to_index_column), std::make_shared(), ""}); + + ColumnsWithTypeAndName data_columns = {std::move(range_min_column_with_type), std::move(range_max_column_with_type)}; + + std::shared_ptr dictionary = shared_from_this(); + + DictionarySourceCoordinator::ReadColumnsFunc read_keys_func = [dictionary_copy = dictionary]( + const Strings & attribute_names, + const DataTypes & result_types, + const Columns & key_columns, + const DataTypes, + const Columns &) + { + auto range_dictionary_ptr = std::static_pointer_cast>(dictionary_copy); + + size_t attribute_names_size = attribute_names.size(); + + Columns result; + result.reserve(attribute_names_size); + + const ColumnPtr & key_column = key_columns.back(); + + const auto * key_to_index_column = typeid_cast(key_column.get()); + if (!key_to_index_column) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Dictionary {} read expect indexes column with type UInt64", + range_dictionary_ptr->getFullName()); + + const auto & data = key_to_index_column->getData(); + + for (size_t i = 0; i < attribute_names_size; ++i) + { + const auto & attribute_name = attribute_names[i]; + const auto & result_type = result_types[i]; + + result.emplace_back(range_dictionary_ptr->getColumnInternal(attribute_name, result_type, data)); + } + + return result; + }; + + auto coordinator = std::make_shared( + dictionary, + column_names, + std::move(key_columns), + std::move(data_columns), + max_block_size, + std::move(read_keys_func)); + auto result = coordinator->read(num_streams); + + return result; +} + } diff --git a/src/Dictionaries/RangeHashedDictionaryComplex.cpp b/src/Dictionaries/RangeHashedDictionaryComplex.cpp new file mode 100644 index 00000000000..76b3920627e --- /dev/null +++ b/src/Dictionaries/RangeHashedDictionaryComplex.cpp @@ -0,0 +1,13 @@ +#include + +/// RangeHashedDictionary is instantiated from two files +/// RangeHashedDictionarySimple.cpp and RangeHashedDictionaryComplex.cpp +/// to better parallelize the build procedure and avoid MSan build failure +/// due to excessive resource consumption. + +namespace DB +{ + +template class RangeHashedDictionary; + +} diff --git a/src/Dictionaries/RangeHashedDictionarySimple.cpp b/src/Dictionaries/RangeHashedDictionarySimple.cpp new file mode 100644 index 00000000000..6f0369607d8 --- /dev/null +++ b/src/Dictionaries/RangeHashedDictionarySimple.cpp @@ -0,0 +1,13 @@ +#include + +/// RangeHashedDictionary is instantiated from two files +/// RangeHashedDictionarySimple.cpp and RangeHashedDictionaryComplex.cpp +/// to better parallelize the build procedure and avoid MSan build failure +/// due to excessive resource consumption. + +namespace DB +{ + +template class RangeHashedDictionary; + +} diff --git a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h index 849e7235c0a..246d2aebfaa 100644 --- a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -15,7 +16,7 @@ class FakeMetadataStorageFromDisk final : public IMetadataStorage private: friend class FakeMetadataStorageFromDiskTransaction; - mutable std::shared_mutex metadata_mutex; + mutable SharedMutex metadata_mutex; DiskPtr disk; ObjectStoragePtr object_storage; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp index 508982ac9c4..96c8b3daf04 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp @@ -83,7 +83,7 @@ std::string MetadataStorageFromDisk::readInlineDataToString(const std::string & return readMetadata(path)->getInlineData(); } -DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const std::string & path, std::shared_lock &) const +DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const std::string & path, std::shared_lock &) const { auto metadata = std::make_unique(disk->getPath(), object_storage_root_path, path); auto str = readFileToString(path); @@ -91,7 +91,7 @@ DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const return metadata; } -DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const std::string & path, std::unique_lock &) const +DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const std::string & path, std::unique_lock &) const { auto metadata = std::make_unique(disk->getPath(), object_storage_root_path, path); auto str = readFileToString(path); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h index 2c80572e7b4..5273f0b041e 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -17,7 +18,7 @@ class MetadataStorageFromDisk final : public IMetadataStorage private: friend class MetadataStorageFromDiskTransaction; - mutable std::shared_mutex metadata_mutex; + mutable SharedMutex metadata_mutex; DiskPtr disk; std::string object_storage_root_path; @@ -67,8 +68,8 @@ public: DiskObjectStorageMetadataPtr readMetadata(const std::string & path) const; - DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::unique_lock & lock) const; - DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::shared_lock & lock) const; + DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::unique_lock & lock) const; + DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::shared_lock & lock) const; }; class MetadataStorageFromDiskTransaction final : public IMetadataTransaction diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp index ce5171fedee..7463622cb06 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp @@ -26,7 +26,7 @@ SetLastModifiedOperation::SetLastModifiedOperation(const std::string & path_, Po { } -void SetLastModifiedOperation::execute(std::unique_lock &) +void SetLastModifiedOperation::execute(std::unique_lock &) { old_timestamp = disk.getLastModified(path); disk.setLastModified(path, new_timestamp); @@ -44,7 +44,7 @@ ChmodOperation::ChmodOperation(const std::string & path_, mode_t mode_, IDisk & { } -void ChmodOperation::execute(std::unique_lock &) +void ChmodOperation::execute(std::unique_lock &) { old_mode = disk.stat(path).st_mode; disk.chmod(path, mode); @@ -61,7 +61,7 @@ UnlinkFileOperation::UnlinkFileOperation(const std::string & path_, IDisk & disk { } -void UnlinkFileOperation::execute(std::unique_lock &) +void UnlinkFileOperation::execute(std::unique_lock &) { auto buf = disk.readFile(path, ReadSettings{}, std::nullopt, disk.getFileSize(path)); readStringUntilEOF(prev_data, *buf); @@ -81,7 +81,7 @@ CreateDirectoryOperation::CreateDirectoryOperation(const std::string & path_, ID { } -void CreateDirectoryOperation::execute(std::unique_lock &) +void CreateDirectoryOperation::execute(std::unique_lock &) { disk.createDirectory(path); } @@ -97,7 +97,7 @@ CreateDirectoryRecursiveOperation::CreateDirectoryRecursiveOperation(const std:: { } -void CreateDirectoryRecursiveOperation::execute(std::unique_lock &) +void CreateDirectoryRecursiveOperation::execute(std::unique_lock &) { namespace fs = std::filesystem; fs::path p(path); @@ -124,7 +124,7 @@ RemoveDirectoryOperation::RemoveDirectoryOperation(const std::string & path_, ID { } -void RemoveDirectoryOperation::execute(std::unique_lock &) +void RemoveDirectoryOperation::execute(std::unique_lock &) { disk.removeDirectory(path); } @@ -141,7 +141,7 @@ RemoveRecursiveOperation::RemoveRecursiveOperation(const std::string & path_, ID { } -void RemoveRecursiveOperation::execute(std::unique_lock &) +void RemoveRecursiveOperation::execute(std::unique_lock &) { if (disk.isFile(path)) disk.moveFile(path, temp_path); @@ -174,7 +174,7 @@ CreateHardlinkOperation::CreateHardlinkOperation(const std::string & path_from_, { } -void CreateHardlinkOperation::execute(std::unique_lock & lock) +void CreateHardlinkOperation::execute(std::unique_lock & lock) { auto metadata = metadata_storage.readMetadataUnlocked(path_from, lock); @@ -201,7 +201,7 @@ MoveFileOperation::MoveFileOperation(const std::string & path_from_, const std:: { } -void MoveFileOperation::execute(std::unique_lock &) +void MoveFileOperation::execute(std::unique_lock &) { disk.moveFile(path_from, path_to); } @@ -218,7 +218,7 @@ MoveDirectoryOperation::MoveDirectoryOperation(const std::string & path_from_, c { } -void MoveDirectoryOperation::execute(std::unique_lock &) +void MoveDirectoryOperation::execute(std::unique_lock &) { disk.moveDirectory(path_from, path_to); } @@ -236,7 +236,7 @@ ReplaceFileOperation::ReplaceFileOperation(const std::string & path_from_, const { } -void ReplaceFileOperation::execute(std::unique_lock &) +void ReplaceFileOperation::execute(std::unique_lock &) { if (disk.exists(path_to)) disk.moveFile(path_to, temp_path_to); @@ -262,7 +262,7 @@ WriteFileOperation::WriteFileOperation(const std::string & path_, IDisk & disk_, { } -void WriteFileOperation::execute(std::unique_lock &) +void WriteFileOperation::execute(std::unique_lock &) { if (disk.exists(path)) { @@ -288,7 +288,7 @@ void WriteFileOperation::undo() } } -void AddBlobOperation::execute(std::unique_lock & metadata_lock) +void AddBlobOperation::execute(std::unique_lock & metadata_lock) { DiskObjectStorageMetadataPtr metadata; if (metadata_storage.exists(path)) @@ -309,7 +309,7 @@ void AddBlobOperation::undo() write_operation->undo(); } -void UnlinkMetadataFileOperation::execute(std::unique_lock & metadata_lock) +void UnlinkMetadataFileOperation::execute(std::unique_lock & metadata_lock) { auto metadata = metadata_storage.readMetadataUnlocked(path, metadata_lock); uint32_t ref_count = metadata->getRefCount(); @@ -336,7 +336,7 @@ void UnlinkMetadataFileOperation::undo() write_operation->undo(); } -void SetReadonlyFileOperation::execute(std::unique_lock & metadata_lock) +void SetReadonlyFileOperation::execute(std::unique_lock & metadata_lock) { auto metadata = metadata_storage.readMetadataUnlocked(path, metadata_lock); metadata->setReadOnly(); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h index 0bce6141301..d8e4892a0a5 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h @@ -1,5 +1,6 @@ #pragma once +#include #include namespace DB @@ -13,7 +14,7 @@ class IDisk; struct IMetadataOperation { - virtual void execute(std::unique_lock & metadata_lock) = 0; + virtual void execute(std::unique_lock & metadata_lock) = 0; virtual void undo() = 0; virtual void finalize() {} virtual ~IMetadataOperation() = default; @@ -26,7 +27,7 @@ struct SetLastModifiedOperation final : public IMetadataOperation { SetLastModifiedOperation(const std::string & path_, Poco::Timestamp new_timestamp_, IDisk & disk_); - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -41,7 +42,7 @@ struct ChmodOperation final : public IMetadataOperation { ChmodOperation(const std::string & path_, mode_t mode_, IDisk & disk_); - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -57,7 +58,7 @@ struct UnlinkFileOperation final : public IMetadataOperation { UnlinkFileOperation(const std::string & path_, IDisk & disk_); - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -72,7 +73,7 @@ struct CreateDirectoryOperation final : public IMetadataOperation { CreateDirectoryOperation(const std::string & path_, IDisk & disk_); - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -86,7 +87,7 @@ struct CreateDirectoryRecursiveOperation final : public IMetadataOperation { CreateDirectoryRecursiveOperation(const std::string & path_, IDisk & disk_); - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -101,7 +102,7 @@ struct RemoveDirectoryOperation final : public IMetadataOperation { RemoveDirectoryOperation(const std::string & path_, IDisk & disk_); - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -114,7 +115,7 @@ struct RemoveRecursiveOperation final : public IMetadataOperation { RemoveRecursiveOperation(const std::string & path_, IDisk & disk_); - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -130,7 +131,7 @@ struct WriteFileOperation final : public IMetadataOperation { WriteFileOperation(const std::string & path_, IDisk & disk_, const std::string & data_); - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; private: @@ -149,7 +150,7 @@ struct CreateHardlinkOperation final : public IMetadataOperation IDisk & disk_, const MetadataStorageFromDisk & metadata_storage_); - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -166,7 +167,7 @@ struct MoveFileOperation final : public IMetadataOperation { MoveFileOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_); - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -181,7 +182,7 @@ struct MoveDirectoryOperation final : public IMetadataOperation { MoveDirectoryOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_); - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -196,7 +197,7 @@ struct ReplaceFileOperation final : public IMetadataOperation { ReplaceFileOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_); - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -226,7 +227,7 @@ struct AddBlobOperation final : public IMetadataOperation , metadata_storage(metadata_storage_) {} - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -254,7 +255,7 @@ struct UnlinkMetadataFileOperation final : public IMetadataOperation { } - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; @@ -279,7 +280,7 @@ struct SetReadonlyFileOperation final : public IMetadataOperation { } - void execute(std::unique_lock & metadata_lock) override; + void execute(std::unique_lock & metadata_lock) override; void undo() override; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index a56a78d6722..f0433762150 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -16,18 +16,13 @@ #include #include #include +#include #include #include -#include #include -#include #include #include -#include -#include -#include -#include #include #include @@ -39,22 +34,9 @@ namespace ProfileEvents { extern const Event S3DeleteObjects; - extern const Event S3HeadObject; extern const Event S3ListObjects; - extern const Event S3CopyObject; - extern const Event S3CreateMultipartUpload; - extern const Event S3UploadPartCopy; - extern const Event S3AbortMultipartUpload; - extern const Event S3CompleteMultipartUpload; - extern const Event DiskS3DeleteObjects; - extern const Event DiskS3HeadObject; extern const Event DiskS3ListObjects; - extern const Event DiskS3CopyObject; - extern const Event DiskS3CreateMultipartUpload; - extern const Event DiskS3UploadPartCopy; - extern const Event DiskS3AbortMultipartUpload; - extern const Event DiskS3CompleteMultipartUpload; } namespace DB @@ -125,21 +107,11 @@ std::string S3ObjectStorage::generateBlobNameForPath(const std::string & /* path getRandomASCIIString(key_name_total_size - key_name_prefix_size)); } -size_t S3ObjectStorage::getObjectSize(const std::string & bucket_from, const std::string & key) const -{ - return S3::getObjectSize(*client.get(), bucket_from, key, {}, /* for_disk_s3= */ true); -} - bool S3ObjectStorage::exists(const StoredObject & object) const { return S3::objectExists(*client.get(), bucket, object.absolute_path, {}, /* for_disk_s3= */ true); } -void S3ObjectStorage::checkObjectExists(const std::string & bucket_from, const std::string & key, std::string_view description) const -{ - return S3::checkObjectExists(*client.get(), bucket_from, key, {}, /* for_disk_s3= */ true, description); -} - std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT const StoredObjects & objects, const ReadSettings & read_settings, @@ -431,7 +403,12 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT /// Shortcut for S3 if (auto * dest_s3 = dynamic_cast(&object_storage_to); dest_s3 != nullptr) { - copyObjectImpl(bucket, object_from.absolute_path, dest_s3->bucket, object_to.absolute_path, {}, object_to_attributes); + auto client_ptr = client.get(); + auto size = S3::getObjectSize(*client_ptr, bucket, object_from.absolute_path, {}, /* for_disk_s3= */ true); + auto settings_ptr = s3_settings.get(); + auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); + copyS3File(client_ptr, bucket, object_from.absolute_path, 0, size, dest_s3->bucket, object_to.absolute_path, + settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } else { @@ -439,148 +416,15 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT } } -void S3ObjectStorage::copyObjectImpl( - const String & src_bucket, - const String & src_key, - const String & dst_bucket, - const String & dst_key, - size_t size, - std::optional metadata) const -{ - auto client_ptr = client.get(); - - ProfileEvents::increment(ProfileEvents::S3CopyObject); - ProfileEvents::increment(ProfileEvents::DiskS3CopyObject); - Aws::S3::Model::CopyObjectRequest request; - request.SetCopySource(src_bucket + "/" + src_key); - request.SetBucket(dst_bucket); - request.SetKey(dst_key); - if (metadata) - { - request.SetMetadata(*metadata); - request.SetMetadataDirective(Aws::S3::Model::MetadataDirective::REPLACE); - } - - auto outcome = client_ptr->CopyObject(request); - - if (!outcome.IsSuccess() && (outcome.GetError().GetExceptionName() == "EntityTooLarge" - || outcome.GetError().GetExceptionName() == "InvalidRequest")) - { // Can't come here with MinIO, MinIO allows single part upload for large objects. - copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, size, metadata); - return; - } - - throwIfError(outcome); - - auto settings_ptr = s3_settings.get(); - if (settings_ptr->request_settings.check_objects_after_upload) - checkObjectExists(dst_bucket, dst_key, "Immediately after upload"); -} - -void S3ObjectStorage::copyObjectMultipartImpl( - const String & src_bucket, - const String & src_key, - const String & dst_bucket, - const String & dst_key, - size_t size, - std::optional metadata) const -{ - auto settings_ptr = s3_settings.get(); - auto client_ptr = client.get(); - - String multipart_upload_id; - - { - ProfileEvents::increment(ProfileEvents::S3CreateMultipartUpload); - ProfileEvents::increment(ProfileEvents::DiskS3CreateMultipartUpload); - Aws::S3::Model::CreateMultipartUploadRequest request; - request.SetBucket(dst_bucket); - request.SetKey(dst_key); - if (metadata) - request.SetMetadata(*metadata); - - auto outcome = client_ptr->CreateMultipartUpload(request); - - throwIfError(outcome); - - multipart_upload_id = outcome.GetResult().GetUploadId(); - } - - std::vector part_tags; - - size_t upload_part_size = settings_ptr->request_settings.getUploadSettings().min_upload_part_size; - for (size_t position = 0, part_number = 1; position < size; ++part_number, position += upload_part_size) - { - ProfileEvents::increment(ProfileEvents::S3UploadPartCopy); - ProfileEvents::increment(ProfileEvents::DiskS3UploadPartCopy); - Aws::S3::Model::UploadPartCopyRequest part_request; - part_request.SetCopySource(src_bucket + "/" + src_key); - part_request.SetBucket(dst_bucket); - part_request.SetKey(dst_key); - part_request.SetUploadId(multipart_upload_id); - part_request.SetPartNumber(static_cast(part_number)); - part_request.SetCopySourceRange(fmt::format("bytes={}-{}", position, std::min(size, position + upload_part_size) - 1)); - - auto outcome = client_ptr->UploadPartCopy(part_request); - if (!outcome.IsSuccess()) - { - ProfileEvents::increment(ProfileEvents::S3AbortMultipartUpload); - ProfileEvents::increment(ProfileEvents::DiskS3AbortMultipartUpload); - Aws::S3::Model::AbortMultipartUploadRequest abort_request; - abort_request.SetBucket(dst_bucket); - abort_request.SetKey(dst_key); - abort_request.SetUploadId(multipart_upload_id); - client_ptr->AbortMultipartUpload(abort_request); - // In error case we throw exception later with first error from UploadPartCopy - } - throwIfError(outcome); - - auto etag = outcome.GetResult().GetCopyPartResult().GetETag(); - part_tags.push_back(etag); - } - - { - ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload); - ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); - Aws::S3::Model::CompleteMultipartUploadRequest req; - req.SetBucket(dst_bucket); - req.SetKey(dst_key); - req.SetUploadId(multipart_upload_id); - - Aws::S3::Model::CompletedMultipartUpload multipart_upload; - for (size_t i = 0; i < part_tags.size(); ++i) - { - Aws::S3::Model::CompletedPart part; - multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(static_cast(i) + 1)); - } - - req.SetMultipartUpload(multipart_upload); - - auto outcome = client_ptr->CompleteMultipartUpload(req); - - throwIfError(outcome); - } - - if (settings_ptr->request_settings.check_objects_after_upload) - checkObjectExists(dst_bucket, dst_key, "Immediately after upload"); -} - void S3ObjectStorage::copyObject( // NOLINT const StoredObject & object_from, const StoredObject & object_to, std::optional object_to_attributes) { - auto size = getObjectSize(bucket, object_from.absolute_path); - static constexpr int64_t multipart_upload_threashold = 5UL * 1024 * 1024 * 1024; - - if (size >= multipart_upload_threashold) - { - copyObjectMultipartImpl( - bucket, object_from.absolute_path, bucket, object_to.absolute_path, size, object_to_attributes); - } - else - { - copyObjectImpl( - bucket, object_from.absolute_path, bucket, object_to.absolute_path, size, object_to_attributes); - } + auto client_ptr = client.get(); + auto size = S3::getObjectSize(*client_ptr, bucket, object_from.absolute_path, {}, /* for_disk_s3= */ true); + auto settings_ptr = s3_settings.get(); + auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); + copyS3File(client_ptr, bucket, object_from.absolute_path, 0, size, bucket, object_to.absolute_path, + settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } void S3ObjectStorage::setNewSettings(std::unique_ptr && s3_settings_) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index a6318bf59b8..ba3eeb421c4 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -8,8 +8,6 @@ #include #include #include -#include -#include #include #include #include @@ -167,28 +165,9 @@ private: void setNewClient(std::unique_ptr && client_); - void copyObjectImpl( - const String & src_bucket, - const String & src_key, - const String & dst_bucket, - const String & dst_key, - size_t size, - std::optional metadata = std::nullopt) const; - - void copyObjectMultipartImpl( - const String & src_bucket, - const String & src_key, - const String & dst_bucket, - const String & dst_key, - size_t size, - std::optional metadata = std::nullopt) const; - void removeObjectImpl(const StoredObject & object, bool if_exists); void removeObjectsImpl(const StoredObjects & objects, bool if_exists); - size_t getObjectSize(const std::string & bucket_from, const std::string & key) const; - void checkObjectExists(const std::string & bucket_from, const std::string & key, std::string_view description) const; - std::string bucket; MultiVersion client; diff --git a/src/Functions/FunctionsHashingInt.cpp b/src/Functions/FunctionsHashingInt.cpp new file mode 100644 index 00000000000..8b4eece685b --- /dev/null +++ b/src/Functions/FunctionsHashingInt.cpp @@ -0,0 +1,17 @@ +#include "FunctionsHashing.h" + +#include + +/// FunctionsHashing instantiations are separated into files FunctionsHashing*.cpp +/// to better parallelize the build procedure and avoid MSan build failure +/// due to excessive resource consumption. + +namespace DB +{ + +REGISTER_FUNCTION(HashingInt) +{ + factory.registerFunction(); + factory.registerFunction(); +} +} diff --git a/src/Functions/FunctionsHashing.cpp b/src/Functions/FunctionsHashingMisc.cpp similarity index 80% rename from src/Functions/FunctionsHashing.cpp rename to src/Functions/FunctionsHashingMisc.cpp index c51898b271b..b33d9366094 100644 --- a/src/Functions/FunctionsHashing.cpp +++ b/src/Functions/FunctionsHashingMisc.cpp @@ -2,6 +2,9 @@ #include +/// FunctionsHashing instantiations are separated into files FunctionsHashing*.cpp +/// to better parallelize the build procedure and avoid MSan build failure +/// due to excessive resource consumption. namespace DB { @@ -14,18 +17,10 @@ REGISTER_FUNCTION(Hashing) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsHashingMurmur.cpp b/src/Functions/FunctionsHashingMurmur.cpp new file mode 100644 index 00000000000..9648c21dbf0 --- /dev/null +++ b/src/Functions/FunctionsHashingMurmur.cpp @@ -0,0 +1,21 @@ +#include "FunctionsHashing.h" + +#include + +/// FunctionsHashing instantiations are separated into files FunctionsHashing*.cpp +/// to better parallelize the build procedure and avoid MSan build failure +/// due to excessive resource consumption. + +namespace DB +{ + +REGISTER_FUNCTION(HashingMurmur) +{ + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); +} +} diff --git a/src/Functions/FunctionsHashingSSL.cpp b/src/Functions/FunctionsHashingSSL.cpp index 01a19a3dd6e..2eb0ed88f28 100644 --- a/src/Functions/FunctionsHashingSSL.cpp +++ b/src/Functions/FunctionsHashingSSL.cpp @@ -5,8 +5,9 @@ #include "FunctionsHashing.h" #include -/// SSL functions are located in the separate FunctionsHashingSSL.cpp file -/// to lower the compilation time of FunctionsHashing.cpp +/// FunctionsHashing instantiations are separated into files FunctionsHashing*.cpp +/// to better parallelize the build procedure and avoid MSan build failure +/// due to excessive resource consumption. namespace DB { diff --git a/src/IO/S3/copyDataToS3.cpp b/src/IO/S3/copyS3File.cpp similarity index 85% rename from src/IO/S3/copyDataToS3.cpp rename to src/IO/S3/copyS3File.cpp index f7018c51359..f2aeb36b60e 100644 --- a/src/IO/S3/copyDataToS3.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -1,4 +1,4 @@ -#include +#include #if USE_AWS_S3 @@ -22,8 +22,17 @@ namespace ProfileEvents { extern const Event S3CreateMultipartUpload; extern const Event S3CompleteMultipartUpload; - extern const Event S3UploadPart; extern const Event S3PutObject; + extern const Event S3CopyObject; + extern const Event S3UploadPart; + extern const Event S3UploadPartCopy; + + extern const Event DiskS3CreateMultipartUpload; + extern const Event DiskS3CompleteMultipartUpload; + extern const Event DiskS3PutObject; + extern const Event DiskS3CopyObject; + extern const Event DiskS3UploadPart; + extern const Event DiskS3UploadPartCopy; } @@ -50,6 +59,7 @@ namespace const S3Settings::RequestSettings & request_settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, + bool for_disk_s3_, const Poco::Logger * log_) : client_ptr(client_ptr_) , dest_bucket(dest_bucket_) @@ -59,6 +69,7 @@ namespace , max_unexpected_write_error_retries(request_settings_.max_unexpected_write_error_retries) , object_metadata(object_metadata_) , schedule(schedule_) + , for_disk_s3(for_disk_s3_) , log(log_) { } @@ -74,6 +85,7 @@ namespace size_t max_unexpected_write_error_retries; const std::optional> & object_metadata; ThreadPoolCallbackRunner schedule; + bool for_disk_s3; const Poco::Logger * log; struct UploadPartTask @@ -111,6 +123,8 @@ namespace request.SetStorageClass(Aws::S3::Model::StorageClassMapper::GetStorageClassForName(settings.storage_class_name)); ProfileEvents::increment(ProfileEvents::S3CreateMultipartUpload); + if (for_disk_s3) + ProfileEvents::increment(ProfileEvents::DiskS3CreateMultipartUpload); auto outcome = client_ptr->CreateMultipartUpload(request); @@ -147,10 +161,12 @@ namespace request.SetMultipartUpload(multipart_upload); - size_t max_retry = std::max(max_unexpected_write_error_retries, 1UL); - for (size_t i = 0; i < max_retry; ++i) + size_t max_retries = std::max(max_unexpected_write_error_retries, 1UL); + for (size_t retries = 1;; ++retries) { ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload); + if (for_disk_s3) + ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); auto outcome = client_ptr->CompleteMultipartUpload(request); @@ -159,20 +175,19 @@ namespace LOG_TRACE(log, "Multipart upload has completed. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", dest_bucket, dest_key, multipart_upload_id, part_tags.size()); break; } - else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) + + if ((outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) && (retries < max_retries)) { /// For unknown reason, at least MinIO can respond with NO_SUCH_KEY for put requests /// BTW, NO_SUCH_UPLOAD is expected error and we shouldn't retry it LOG_INFO(log, "Multipart upload failed with NO_SUCH_KEY error for Bucket: {}, Key: {}, Upload_id: {}, Parts: {}, will retry", dest_bucket, dest_key, multipart_upload_id, part_tags.size()); - /// will retry - } - else - { - throw S3Exception( - outcome.GetError().GetErrorType(), - "Message: {}, Key: {}, Bucket: {}, Tags: {}", - outcome.GetError().GetMessage(), dest_key, dest_bucket, fmt::join(part_tags.begin(), part_tags.end(), " ")); + continue; /// will retry } + + throw S3Exception( + outcome.GetError().GetErrorType(), + "Message: {}, Key: {}, Bucket: {}, Tags: {}", + outcome.GetError().GetMessage(), dest_key, dest_bucket, fmt::join(part_tags.begin(), part_tags.end(), " ")); } } @@ -379,11 +394,11 @@ namespace } }; - /// Helper class to help implementing copyDataToS3(). - class CopyDataToS3Helper : public UploadHelper + /// Helper class to help implementing copyDataToS3File(). + class CopyDataToFileHelper : public UploadHelper { public: - CopyDataToS3Helper( + CopyDataToFileHelper( const std::function()> & create_read_buffer_, size_t offset_, size_t size_, @@ -392,8 +407,9 @@ namespace const String & dest_key_, const S3Settings::RequestSettings & request_settings_, const std::optional> & object_metadata_, - ThreadPoolCallbackRunner schedule_) - : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, &Poco::Logger::get("copyDataToS3")) + ThreadPoolCallbackRunner schedule_, + bool for_disk_s3_) + : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyDataToS3File")) , create_read_buffer(create_read_buffer_) , offset(offset_) , size(size_) @@ -444,10 +460,13 @@ namespace void processPutRequest(const Aws::S3::Model::PutObjectRequest & request) { - size_t max_retry = std::max(max_unexpected_write_error_retries, 1UL); - for (size_t i = 0; i < max_retry; ++i) + size_t max_retries = std::max(max_unexpected_write_error_retries, 1UL); + for (size_t retries = 1;; ++retries) { ProfileEvents::increment(ProfileEvents::S3PutObject); + if (for_disk_s3) + ProfileEvents::increment(ProfileEvents::DiskS3PutObject); + auto outcome = client_ptr->PutObject(request); if (outcome.IsSuccess()) @@ -460,7 +479,8 @@ namespace request.GetContentLength()); break; } - else if (outcome.GetError().GetExceptionName() == "EntityTooLarge" || outcome.GetError().GetExceptionName() == "InvalidRequest") + + if (outcome.GetError().GetExceptionName() == "EntityTooLarge" || outcome.GetError().GetExceptionName() == "InvalidRequest") { // Can't come here with MinIO, MinIO allows single part upload for large objects. LOG_INFO( @@ -473,7 +493,8 @@ namespace performMultipartUpload(); break; } - else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) + + if ((outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) && (retries < max_retries)) { /// For unknown reason, at least MinIO can respond with NO_SUCH_KEY for put requests LOG_INFO( @@ -482,18 +503,16 @@ namespace dest_bucket, dest_key, request.GetContentLength()); - /// will retry - } - else - { - throw S3Exception( - outcome.GetError().GetErrorType(), - "Message: {}, Key: {}, Bucket: {}, Object size: {}", - outcome.GetError().GetMessage(), - dest_key, - dest_bucket, - request.GetContentLength()); + continue; /// will retry } + + throw S3Exception( + outcome.GetError().GetErrorType(), + "Message: {}, Key: {}, Bucket: {}, Object size: {}", + outcome.GetError().GetMessage(), + dest_key, + dest_bucket, + request.GetContentLength()); } } @@ -523,6 +542,8 @@ namespace auto & req = typeid_cast(request); ProfileEvents::increment(ProfileEvents::S3UploadPart); + if (for_disk_s3) + ProfileEvents::increment(ProfileEvents::DiskS3UploadPart); auto outcome = client_ptr->UploadPart(req); if (!outcome.IsSuccess()) @@ -535,11 +556,11 @@ namespace } }; - /// Helper class to help implementing copyFileS3ToS3(). - class CopyFileS3ToS3Helper : public UploadHelper + /// Helper class to help implementing copyS3File(). + class CopyFileHelper : public UploadHelper { public: - CopyFileS3ToS3Helper( + CopyFileHelper( const std::shared_ptr & client_ptr_, const String & src_bucket_, const String & src_key_, @@ -549,8 +570,9 @@ namespace const String & dest_key_, const S3Settings::RequestSettings & request_settings_, const std::optional> & object_metadata_, - ThreadPoolCallbackRunner schedule_) - : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, &Poco::Logger::get("copyFileS3ToS3")) + ThreadPoolCallbackRunner schedule_, + bool for_disk_s3_) + : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyS3File")) , src_bucket(src_bucket_) , src_key(src_key_) , offset(src_offset_) @@ -603,9 +625,13 @@ namespace void processCopyRequest(const Aws::S3::Model::CopyObjectRequest & request) { - size_t max_retry = std::max(max_unexpected_write_error_retries, 1UL); - for (size_t i = 0; i < max_retry; ++i) + size_t max_retries = std::max(max_unexpected_write_error_retries, 1UL); + for (size_t retries = 1;; ++retries) { + ProfileEvents::increment(ProfileEvents::S3CopyObject); + if (for_disk_s3) + ProfileEvents::increment(ProfileEvents::DiskS3CopyObject); + auto outcome = client_ptr->CopyObject(request); if (outcome.IsSuccess()) { @@ -617,7 +643,8 @@ namespace size); break; } - else if (outcome.GetError().GetExceptionName() == "EntityTooLarge" || outcome.GetError().GetExceptionName() == "InvalidRequest") + + if (outcome.GetError().GetExceptionName() == "EntityTooLarge" || outcome.GetError().GetExceptionName() == "InvalidRequest") { // Can't come here with MinIO, MinIO allows single part upload for large objects. LOG_INFO( @@ -630,7 +657,8 @@ namespace performMultipartUploadCopy(); break; } - else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) + + if ((outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) && (retries < max_retries)) { /// TODO: Is it true for copy requests? /// For unknown reason, at least MinIO can respond with NO_SUCH_KEY for put requests @@ -640,18 +668,16 @@ namespace dest_bucket, dest_key, size); - /// will retry - } - else - { - throw S3Exception( - outcome.GetError().GetErrorType(), - "Message: {}, Key: {}, Bucket: {}, Object size: {}", - outcome.GetError().GetMessage(), - dest_key, - dest_bucket, - size); + continue; /// will retry } + + throw S3Exception( + outcome.GetError().GetErrorType(), + "Message: {}, Key: {}, Bucket: {}, Object size: {}", + outcome.GetError().GetMessage(), + dest_key, + dest_bucket, + size); } } @@ -676,6 +702,10 @@ namespace { auto & req = typeid_cast(request); + ProfileEvents::increment(ProfileEvents::S3UploadPartCopy); + if (for_disk_s3) + ProfileEvents::increment(ProfileEvents::DiskS3UploadPartCopy); + auto outcome = client_ptr->UploadPartCopy(req); if (!outcome.IsSuccess()) { @@ -689,7 +719,7 @@ namespace } -void copyDataToS3( +void copyDataToS3File( const std::function()> & create_read_buffer, size_t offset, size_t size, @@ -698,14 +728,15 @@ void copyDataToS3( const String & dest_key, const S3Settings::RequestSettings & settings, const std::optional> & object_metadata, - ThreadPoolCallbackRunner schedule) + ThreadPoolCallbackRunner schedule, + bool for_disk_s3) { - CopyDataToS3Helper helper{create_read_buffer, offset, size, dest_s3_client, dest_bucket, dest_key, settings, object_metadata, schedule}; + CopyDataToFileHelper helper{create_read_buffer, offset, size, dest_s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } -void copyFileS3ToS3( +void copyS3File( const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -715,9 +746,10 @@ void copyFileS3ToS3( const String & dest_key, const S3Settings::RequestSettings & settings, const std::optional> & object_metadata, - ThreadPoolCallbackRunner schedule) + ThreadPoolCallbackRunner schedule, + bool for_disk_s3) { - CopyFileS3ToS3Helper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule}; + CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } diff --git a/src/IO/S3/copyDataToS3.h b/src/IO/S3/copyS3File.h similarity index 82% rename from src/IO/S3/copyDataToS3.h rename to src/IO/S3/copyS3File.h index 1ad308fa1d4..808860650b8 100644 --- a/src/IO/S3/copyDataToS3.h +++ b/src/IO/S3/copyS3File.h @@ -16,27 +16,11 @@ namespace DB { class SeekableReadBuffer; -/// Copies data from any seekable source to S3. -/// The same functionality can be done by using the function copyData() and the class WriteBufferFromS3 -/// however copyDataToS3() is faster and spends less memory. -/// The callback `create_read_buffer` can be called from multiple threads in parallel, so that should be thread-safe. -/// The parameters `offset` and `size` specify a part in the source to copy. -void copyDataToS3( - const std::function()> & create_read_buffer, - size_t offset, - size_t size, - const std::shared_ptr & dest_s3_client, - const String & dest_bucket, - const String & dest_key, - const S3Settings::RequestSettings & settings, - const std::optional> & object_metadata = std::nullopt, - ThreadPoolCallbackRunner schedule_ = {}); - /// Copies a file from S3 to S3. /// The same functionality can be done by using the function copyData() and the classes ReadBufferFromS3 and WriteBufferFromS3 -/// however copyFileS3ToS3() is faster and spends less network traffic and memory. +/// however copyS3File() is faster and spends less network traffic and memory. /// The parameters `src_offset` and `src_size` specify a part in the source to copy. -void copyFileS3ToS3( +void copyS3File( const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -46,7 +30,25 @@ void copyFileS3ToS3( const String & dest_key, const S3Settings::RequestSettings & settings, const std::optional> & object_metadata = std::nullopt, - ThreadPoolCallbackRunner schedule_ = {}); + ThreadPoolCallbackRunner schedule_ = {}, + bool for_disk_s3 = false); + +/// Copies data from any seekable source to S3. +/// The same functionality can be done by using the function copyData() and the class WriteBufferFromS3 +/// however copyDataToS3File() is faster and spends less memory. +/// The callback `create_read_buffer` can be called from multiple threads in parallel, so that should be thread-safe. +/// The parameters `offset` and `size` specify a part in the source to copy. +void copyDataToS3File( + const std::function()> & create_read_buffer, + size_t offset, + size_t size, + const std::shared_ptr & dest_s3_client, + const String & dest_bucket, + const String & dest_key, + const S3Settings::RequestSettings & settings, + const std::optional> & object_metadata = std::nullopt, + ThreadPoolCallbackRunner schedule_ = {}, + bool for_disk_s3 = false); } diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 6abc95b6738..10141c78d2b 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -726,7 +726,7 @@ DDLGuardPtr DatabaseCatalog::getDDLGuard(const String & database, const String & return std::make_unique(db_guard.first, db_guard.second, std::move(lock), table, database); } -std::unique_lock DatabaseCatalog::getExclusiveDDLGuardForDatabase(const String & database) +std::unique_lock DatabaseCatalog::getExclusiveDDLGuardForDatabase(const String & database) { DDLGuards::iterator db_guard_iter; { @@ -1279,7 +1279,7 @@ TemporaryLockForUUIDDirectory & TemporaryLockForUUIDDirectory::operator = (Tempo } -DDLGuard::DDLGuard(Map & map_, std::shared_mutex & db_mutex_, std::unique_lock guards_lock_, const String & elem, const String & database_name) +DDLGuard::DDLGuard(Map & map_, SharedMutex & db_mutex_, std::unique_lock guards_lock_, const String & elem, const String & database_name) : map(map_), db_mutex(db_mutex_), guards_lock(std::move(guards_lock_)) { it = map.emplace(elem, Entry{std::make_unique(), 0}).first; diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index a3fa4515a69..5dc3f90b7f4 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -17,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -58,7 +58,7 @@ public: DDLGuard( Map & map_, - std::shared_mutex & db_mutex_, + SharedMutex & db_mutex_, std::unique_lock guards_lock_, const String & elem, const String & database_name); @@ -69,7 +69,7 @@ public: private: Map & map; - std::shared_mutex & db_mutex; + SharedMutex & db_mutex; Map::iterator it; std::unique_lock guards_lock; std::unique_lock table_lock; @@ -142,7 +142,7 @@ public: /// Get an object that protects the table from concurrently executing multiple DDL operations. DDLGuardPtr getDDLGuard(const String & database, const String & table); /// Get an object that protects the database from concurrent DDL queries all tables in the database - std::unique_lock getExclusiveDDLGuardForDatabase(const String & database); + std::unique_lock getExclusiveDDLGuardForDatabase(const String & database); void assertDatabaseExists(const String & database_name) const; @@ -298,7 +298,7 @@ private: /// For the duration of the operation, an element is placed here, and an object is returned, /// which deletes the element in the destructor when counter becomes zero. /// In case the element already exists, waits when query will be executed in other thread. See class DDLGuard below. - using DatabaseGuard = std::pair; + using DatabaseGuard = std::pair; using DDLGuards = std::map; DDLGuards ddl_guards TSA_GUARDED_BY(ddl_guards_mutex); /// If you capture mutex and ddl_guards_mutex, then you need to grab them strictly in this order. diff --git a/src/Interpreters/GinFilter.cpp b/src/Interpreters/GinFilter.cpp index 0810b9a6c6c..4662128e8ab 100644 --- a/src/Interpreters/GinFilter.cpp +++ b/src/Interpreters/GinFilter.cpp @@ -1,31 +1,34 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include namespace DB { + namespace ErrorCodes { extern const int BAD_ARGUMENTS; } + GinFilterParameters::GinFilterParameters(size_t ngrams_, Float64 density_) - : ngrams(ngrams_), density(density_) + : ngrams(ngrams_) + , density(density_) { if (ngrams > 8) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The size of gin filter cannot be greater than 8"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The size of inverted index filter cannot be greater than 8"); if (density <= 0 || density > 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The density of gin filter must be between 0 and 1"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The density inverted index gin filter must be between 0 and 1"); } GinFilter::GinFilter(const GinFilterParameters & params_) @@ -33,23 +36,23 @@ GinFilter::GinFilter(const GinFilterParameters & params_) { } -void GinFilter::add(const char* data, size_t len, UInt32 rowID, GinIndexStorePtr& store, UInt64 limit) const +void GinFilter::add(const char * data, size_t len, UInt32 rowID, GinIndexStorePtr & store, UInt64 limit) const { if (len > FST::MAX_TERM_LENGTH) return; String term(data, len); - auto it = store->getPostings().find(term); + auto it = store->getPostingsListBuilder().find(term); - if (it != store->getPostings().end()) + if (it != store->getPostingsListBuilder().end()) { if (!it->second->contains(rowID)) it->second->add(rowID); } else { - UInt64 threshold = std::lround(limit * params.density); - GinIndexStore::GinIndexPostingsBuilderPtr builder = std::make_shared(threshold); + UInt64 size_limit = std::lround(limit * params.density); + auto builder = std::make_shared(size_limit); builder->add(rowID); store->setPostingsBuilder(term, builder); @@ -66,7 +69,7 @@ void GinFilter::addRowRangeToGinFilter(UInt32 segmentID, UInt32 rowIDStart, UInt if (!rowid_ranges.empty()) { /// Try to merge the rowID range with the last one in the container - GinSegmentWithRowIDRange & last_rowid_range = rowid_ranges.back(); + GinSegmentWithRowIdRange & last_rowid_range = rowid_ranges.back(); if (last_rowid_range.segment_id == segmentID && last_rowid_range.range_end+1 == rowIDStart) @@ -80,93 +83,17 @@ void GinFilter::addRowRangeToGinFilter(UInt32 segmentID, UInt32 rowIDStart, UInt void GinFilter::clear() { + query_string.clear(); terms.clear(); rowid_ranges.clear(); - query_string.clear(); } -bool GinFilter::hasEmptyPostingsList(const PostingsCache& postings_cache) -{ - if (postings_cache.empty()) - return true; - - for (const auto& term_postings : postings_cache) - { - const SegmentedPostingsListContainer& container = term_postings.second; - if (container.empty()) - return true; - } - return false; -} - -bool GinFilter::matchInRange(const PostingsCache& postings_cache, UInt32 segment_id, UInt32 range_start, UInt32 range_end) -{ - /// Check for each terms - GinIndexPostingsList intersection_result; - bool intersection_result_init = false; - - for (const auto& term_postings : postings_cache) - { - /// Check if it is in the same segment by searching for segment_id - const SegmentedPostingsListContainer& container = term_postings.second; - auto container_it = container.find(segment_id); - if (container_it == container.cend()) - { - return false; - } - auto min_in_container = container_it->second->minimum(); - auto max_in_container = container_it->second->maximum(); - - //check if the postings list has always match flag - if (container_it->second->cardinality() == 1 && UINT32_MAX == min_in_container) - { - continue; //always match - } - - if (range_start > max_in_container || min_in_container > range_end) - { - return false; - } - - /// Delay initialization as late as possible - if (!intersection_result_init) - { - intersection_result_init = true; - intersection_result.addRange(range_start, range_end+1); - } - intersection_result &= *container_it->second; - if (intersection_result.cardinality() == 0) - { - return false; - } - } - return true; -} - -bool GinFilter::match(const PostingsCache& postings_cache) const -{ - if (hasEmptyPostingsList(postings_cache)) - { - return false; - } - - /// Check for each row ID ranges - for (const auto &rowid_range: rowid_ranges) - { - if (matchInRange(postings_cache, rowid_range.segment_id, rowid_range.range_start, rowid_range.range_end)) - { - return true; - } - } - return false; -} - -bool GinFilter::contains(const GinFilter & filter, PostingsCacheForStore &cache_store) const +bool GinFilter::contains(const GinFilter & filter, PostingsCacheForStore & cache_store) const { if (filter.getTerms().empty()) return true; - PostingsCachePtr postings_cache = cache_store.getPostings(filter.getQueryString()); + GinPostingsCachePtr postings_cache = cache_store.getPostings(filter.getQueryString()); if (postings_cache == nullptr) { GinIndexStoreDeserializer reader(cache_store.store); @@ -177,9 +104,73 @@ bool GinFilter::contains(const GinFilter & filter, PostingsCacheForStore &cache_ return match(*postings_cache); } -String GinFilter::getName() +namespace { - return FilterName; + +/// Helper method for checking if postings list cache is empty +bool hasEmptyPostingsList(const GinPostingsCache & postings_cache) +{ + if (postings_cache.empty()) + return true; + + for (const auto & term_postings : postings_cache) + { + const GinSegmentedPostingsListContainer & container = term_postings.second; + if (container.empty()) + return true; + } + return false; +} + +/// Helper method to check if the postings list cache has intersection with given row ID range +bool matchInRange(const GinPostingsCache & postings_cache, UInt32 segment_id, UInt32 range_start, UInt32 range_end) +{ + /// Check for each term + GinIndexPostingsList intersection_result; + bool intersection_result_init = false; + + for (const auto & term_postings : postings_cache) + { + /// Check if it is in the same segment by searching for segment_id + const GinSegmentedPostingsListContainer & container = term_postings.second; + auto container_it = container.find(segment_id); + if (container_it == container.cend()) + return false; + auto min_in_container = container_it->second->minimum(); + auto max_in_container = container_it->second->maximum(); + + //check if the postings list has always match flag + if (container_it->second->cardinality() == 1 && UINT32_MAX == min_in_container) + continue; //always match + + if (range_start > max_in_container || min_in_container > range_end) + return false; + + /// Delay initialization as late as possible + if (!intersection_result_init) + { + intersection_result_init = true; + intersection_result.addRange(range_start, range_end+1); + } + intersection_result &= *container_it->second; + if (intersection_result.cardinality() == 0) + return false; + } + return true; +} + +} + +bool GinFilter::match(const GinPostingsCache & postings_cache) const +{ + if (hasEmptyPostingsList(postings_cache)) + return false; + + /// Check for each row ID ranges + for (const auto & rowid_range: rowid_ranges) + if (matchInRange(postings_cache, rowid_range.segment_id, rowid_range.range_start, rowid_range.range_end)) + return true; + return false; } } diff --git a/src/Interpreters/GinFilter.h b/src/Interpreters/GinFilter.h index 0bcd4156f94..8985d84f215 100644 --- a/src/Interpreters/GinFilter.h +++ b/src/Interpreters/GinFilter.h @@ -1,19 +1,23 @@ #pragma once +#include #include #include -#include + namespace DB { + +static inline constexpr auto INVERTED_INDEX_NAME = "inverted"; + struct GinFilterParameters { - explicit GinFilterParameters(size_t ngrams_, Float64 density_); + GinFilterParameters(size_t ngrams_, Float64 density_); size_t ngrams; Float64 density; }; -struct GinSegmentWithRowIDRange +struct GinSegmentWithRowIdRange { /// Segment ID of the row ID range UInt32 segment_id; @@ -25,19 +29,20 @@ struct GinSegmentWithRowIDRange UInt32 range_end; }; +using GinSegmentWithRowIdRangeVector = std::vector; + /// GinFilter provides underlying functionalities for building inverted index and also /// it does filtering the unmatched rows according to its query string. /// It also builds and uses skipping index which stores (segmentID, RowIDStart, RowIDEnd) triples. class GinFilter { public: - using GinSegmentWithRowIDRanges = std::vector; - explicit GinFilter(const GinFilterParameters& params_); + explicit GinFilter(const GinFilterParameters & params_); - /// Add term(which length is 'len' and located at 'data') and its row ID to - /// the postings list builder for building inverted index for the given store. - void add(const char* data, size_t len, UInt32 rowID, GinIndexStorePtr& store, UInt64 limit) const; + /// Add term (located at 'data' with length 'len') and its row ID to the postings list builder + /// for building inverted index for the given store. + void add(const char * data, size_t len, UInt32 rowID, GinIndexStorePtr & store, UInt64 limit) const; /// Accumulate (segmentID, RowIDStart, RowIDEnd) for building skipping index void addRowRangeToGinFilter(UInt32 segmentID, UInt32 rowIDStart, UInt32 rowIDEnd); @@ -45,47 +50,33 @@ public: /// Clear the content void clear(); - /// Check if the filter(built from query string) contains any rows in given filter 'af' by using + /// Check if the filter (built from query string) contains any rows in given filter by using /// given postings list cache - bool contains(const GinFilter & filter, PostingsCacheForStore &cache_store) const; - - /// Const getter for the row ID ranges - const GinSegmentWithRowIDRanges& getFilter() const { return rowid_ranges; } - - /// Mutable getter for the row ID ranges - GinSegmentWithRowIDRanges& getFilter() { return rowid_ranges; } + bool contains(const GinFilter & filter, PostingsCacheForStore & cache_store) const; /// Set the query string of the filter - void setQueryString(const char* data, size_t len) + void setQueryString(const char * data, size_t len) { query_string = String(data, len); } - /// Const getter of the query string - const String &getQueryString() const { return query_string; } - /// Add term which are tokens generated from the query string - void addTerm(const char* data, size_t len) + void addTerm(const char * data, size_t len) { if (len > FST::MAX_TERM_LENGTH) return; terms.push_back(String(data, len)); } - /// Const getter of terms(generated from the query string) - const std::vector& getTerms() const { return terms;} + /// Getter + const String & getQueryString() const { return query_string; } + const std::vector & getTerms() const { return terms; } + const GinSegmentWithRowIdRangeVector & getFilter() const { return rowid_ranges; } + GinSegmentWithRowIdRangeVector & getFilter() { return rowid_ranges; } - /// Check if the given postings list cache has matched rows by using the filter - bool match(const PostingsCache& postings_cache) const; - - /// Get filter name ("inverted") - static String getName(); - - /// Constant of filter name - static constexpr auto FilterName = "inverted"; private: /// Filter parameters - const GinFilterParameters& params; + const GinFilterParameters & params; /// Query string of the filter String query_string; @@ -94,15 +85,12 @@ private: std::vector terms; /// Row ID ranges which are (segmentID, RowIDStart, RowIDEnd) - GinSegmentWithRowIDRanges rowid_ranges; + GinSegmentWithRowIdRangeVector rowid_ranges; - /// Helper method for checking if postings list cache is empty - static bool hasEmptyPostingsList(const PostingsCache& postings_cache); - - /// Helper method to check if the postings list cache has intersection with given row ID range - static bool matchInRange(const PostingsCache& postings_cache, UInt32 segment_id, UInt32 range_start, UInt32 range_end); + /// Check if the given postings list cache has matched rows by using the filter + bool match(const GinPostingsCache & postings_cache) const; }; -using GinFilterPtr = std::shared_ptr; +using GinFilters = std::vector; } diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index be03cee4a35..4f7694e2f07 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -7,6 +7,7 @@ #include #include +#include #include @@ -135,7 +136,7 @@ private: TemporaryDataOnDiskPtr tmp_data; Buckets buckets; - mutable std::shared_mutex rehash_mutex; + mutable SharedMutex rehash_mutex; FileBucket * current_bucket = nullptr; mutable std::mutex current_bucket_mutex; diff --git a/src/Interpreters/ITokenExtractor.h b/src/Interpreters/ITokenExtractor.h index 77de4233b63..fdcc9880bff 100644 --- a/src/Interpreters/ITokenExtractor.h +++ b/src/Interpreters/ITokenExtractor.h @@ -77,12 +77,15 @@ class ITokenExtractorHelper : public ITokenExtractor { size_t cur = 0; String token; + while (cur < length && static_cast(this)->nextInStringLike(data, length, &cur, token)) bloom_filter.add(token.c_str(), token.size()); } + void stringToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const override { gin_filter.setQueryString(data, length); + size_t cur = 0; size_t token_start = 0; size_t token_len = 0; @@ -94,6 +97,7 @@ class ITokenExtractorHelper : public ITokenExtractor void stringPaddedToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const override { gin_filter.setQueryString(data, length); + size_t cur = 0; size_t token_start = 0; size_t token_len = 0; @@ -105,8 +109,10 @@ class ITokenExtractorHelper : public ITokenExtractor void stringLikeToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const override { gin_filter.setQueryString(data, length); + size_t cur = 0; String token; + while (cur < length && static_cast(this)->nextInStringLike(data, length, &cur, token)) gin_filter.addTerm(token.c_str(), token.size()); } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 1f2b81bcc08..611f533d559 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -683,12 +683,13 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti for (const auto & index : create.columns_list->indices->children) { IndexDescription index_desc = IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext()); - if (index_desc.type == GinFilter::FilterName && getContext()->getSettingsRef().allow_experimental_inverted_index == false) + const auto & settings = getContext()->getSettingsRef(); + if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index) { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental Inverted Index feature is not enabled (the setting 'allow_experimental_inverted_index')"); } - if (index_desc.type == "annoy" && !getContext()->getSettingsRef().allow_experimental_annoy_index) + if (index_desc.type == "annoy" && !settings.allow_experimental_annoy_index) throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index is disabled. Turn on allow_experimental_annoy_index"); properties.indices.push_back(index_desc); diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index c9eb3c9a6df..f2f937f6ec0 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -287,6 +287,10 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name, table->drop(); table->is_dropped = true; } + else if (kind == ASTDropQuery::Kind::Detach) + { + table->is_detached = true; + } } } diff --git a/src/Interpreters/InterserverIOHandler.h b/src/Interpreters/InterserverIOHandler.h index 6aa91edcc62..375c6ee9ca5 100644 --- a/src/Interpreters/InterserverIOHandler.h +++ b/src/Interpreters/InterserverIOHandler.h @@ -7,11 +7,11 @@ #include #include #include +#include #include #include #include -#include #include namespace zkutil @@ -43,7 +43,7 @@ public: /// You need to stop the data transfer if blocker is activated. ActionBlocker blocker; - std::shared_mutex rwlock; + SharedMutex rwlock; }; using InterserverIOEndpointPtr = std::shared_ptr; diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 770ca0409bf..8b5d884a0e6 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include #include #include #include @@ -72,7 +71,7 @@ private: using Cache = CacheBase, BlockByteWeight>; - mutable std::shared_mutex rwlock; + mutable SharedMutex rwlock; std::shared_ptr table_join; SizeLimits size_limits; SortDescription left_sort_description; diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 34edfc5a2e2..eae8b15c695 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -25,7 +25,6 @@ #include #include #include -#include #include #include diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 8cdd60b2426..75bb05f8346 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -176,7 +176,7 @@ bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns) bool Set::insertFromBlock(const Columns & columns) { - std::lock_guard lock(rwlock); + std::lock_guard lock(rwlock); if (data.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Method Set::setHeader must be called before Set::insertFromBlock"); diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index bafb0dcea7a..00eff614c7c 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -8,6 +7,7 @@ #include #include +#include #include @@ -131,7 +131,7 @@ private: /** Protects work with the set in the functions `insertFromBlock` and `execute`. * These functions can be called simultaneously from different threads only when using StorageSet, */ - mutable std::shared_mutex rwlock; + mutable SharedMutex rwlock; template void insertFromBlockImpl( diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 72ce13fee4e..3dfc22f20ff 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -208,73 +208,8 @@ struct CustomizeAggregateFunctionsMoveSuffixData } }; -struct FuseSumCountAggregates -{ - std::vector sums {}; - std::vector counts {}; - std::vector avgs {}; - - void addFuncNode(ASTFunction * func) - { - if (func->name == "sum") - sums.push_back(func); - else if (func->name == "count") - counts.push_back(func); - else - { - assert(func->name == "avg"); - avgs.push_back(func); - } - } - - bool canBeFused() const - { - // Need at least two different kinds of functions to fuse. - if (sums.empty() && counts.empty()) - return false; - if (sums.empty() && avgs.empty()) - return false; - if (counts.empty() && avgs.empty()) - return false; - return true; - } -}; - -struct FuseSumCountAggregatesVisitorData -{ - using TypeToVisit = ASTFunction; - - std::unordered_map fuse_map; - - void visit(ASTFunction & func, ASTPtr &) - { - if (func.name == "sum" || func.name == "avg" || func.name == "count") - { - if (func.arguments->children.empty()) - return; - - // Probably we can extend it to match count() for non-nullable argument - // to sum/avg with any other argument. Now we require strict match. - const auto argument = func.arguments->children.at(0)->getColumnName(); - auto it = fuse_map.find(argument); - if (it != fuse_map.end()) - { - it->second.addFuncNode(&func); - } - else - { - FuseSumCountAggregates funcs{}; - funcs.addFuncNode(&func); - fuse_map[argument] = funcs; - } - } - } -}; - using CustomizeAggregateFunctionsOrNullVisitor = InDepthNodeVisitor, true>; using CustomizeAggregateFunctionsMoveOrNullVisitor = InDepthNodeVisitor, true>; -using FuseSumCountAggregatesVisitor = InDepthNodeVisitor, true>; - struct ExistsExpressionData { @@ -376,52 +311,6 @@ void translateQualifiedNames(ASTPtr & query, const ASTSelectQuery & select_query throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED, "Empty list of columns in SELECT query"); } -// Replaces one avg/sum/count function with an appropriate expression with -// sumCount(). -void replaceWithSumCount(String column_name, ASTFunction & func) -{ - auto func_base = makeASTFunction("sumCount", std::make_shared(column_name)); - auto exp_list = std::make_shared(); - if (func.name == "sum" || func.name == "count") - { - /// Rewrite "sum" to sumCount().1, rewrite "count" to sumCount().2 - UInt8 idx = (func.name == "sum" ? 1 : 2); - func.name = "tupleElement"; - exp_list->children.push_back(func_base); - exp_list->children.push_back(std::make_shared(idx)); - } - else - { - /// Rewrite "avg" to sumCount().1 / sumCount().2 - auto new_arg1 = makeASTFunction("tupleElement", func_base, std::make_shared(UInt8(1))); - auto new_arg2 = makeASTFunction("CAST", - makeASTFunction("tupleElement", func_base, std::make_shared(static_cast(2))), - std::make_shared("Float64")); - - func.name = "divide"; - exp_list->children.push_back(new_arg1); - exp_list->children.push_back(new_arg2); - } - func.arguments = exp_list; - func.children.push_back(func.arguments); -} - -void fuseSumCountAggregates(std::unordered_map & fuse_map) -{ - for (auto & it : fuse_map) - { - if (it.second.canBeFused()) - { - for (auto & func: it.second.sums) - replaceWithSumCount(it.first, *func); - for (auto & func: it.second.avgs) - replaceWithSumCount(it.first, *func); - for (auto & func: it.second.counts) - replaceWithSumCount(it.first, *func); - } - } -} - bool hasArrayJoin(const ASTPtr & ast) { if (const ASTFunction * function = ast->as()) @@ -1544,17 +1433,6 @@ void TreeRewriter::normalize( CustomizeGlobalNotInVisitor(data_global_not_null_in).visit(query); } - // Try to fuse sum/avg/count with identical arguments to one sumCount call, - // if we have at least two different functions. E.g. we will replace sum(x) - // and count(x) with sumCount(x).1 and sumCount(x).2, and sumCount() will - // be calculated only once because of CSE. - if (settings.optimize_fuse_sum_count_avg && settings.optimize_syntax_fuse_functions) - { - FuseSumCountAggregatesVisitor::Data data; - FuseSumCountAggregatesVisitor(data).visit(query); - fuseSumCountAggregates(data.fuse_map); - } - /// Rewrite all aggregate functions to add -OrNull suffix to them if (settings.aggregate_functions_null_for_empty) { diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 319451bd39a..7a348f03240 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -915,13 +915,12 @@ bool AlterCommands::hasInvertedIndex(const StorageInMemoryMetadata & metadata, C { continue; } - if (index.type == GinFilter::FilterName) - { + if (index.type == INVERTED_INDEX_NAME) return true; - } } return false; } + void AlterCommands::apply(StorageInMemoryMetadata & metadata, ContextPtr context) const { if (!prepared) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 1dd63858421..9bcfff65c95 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -50,10 +50,10 @@ TableLockHolder IStorage::lockForShare(const String & query_id, const std::chron { TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout); - if (is_dropped) + if (is_dropped || is_detached) { auto table_id = getStorageID(); - throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped", table_id.database_name, table_id.table_name); + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped or detached", table_id.database_name, table_id.table_name); } return result; } @@ -62,7 +62,7 @@ TableLockHolder IStorage::tryLockForShare(const String & query_id, const std::ch { TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout); - if (is_dropped) + if (is_dropped || is_detached) { // Table was dropped while acquiring the lock result = nullptr; @@ -81,7 +81,7 @@ IStorage::AlterLockHolder IStorage::lockForAlter(const std::chrono::milliseconds "Possible deadlock avoided. Client should retry.", getStorageID().getFullTableName(), acquire_timeout.count()); - if (is_dropped) + if (is_dropped || is_detached) throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} is dropped or detached", getStorageID()); return lock; @@ -93,7 +93,7 @@ TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, cons TableExclusiveLockHolder result; result.drop_lock = tryLockTimed(drop_lock, RWLockImpl::Write, query_id, acquire_timeout); - if (is_dropped) + if (is_dropped || is_detached) throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} is dropped or detached", getStorageID()); return result; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index ddc3ef16cbf..a4825358d6e 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -19,7 +19,6 @@ #include #include -#include #include @@ -562,6 +561,7 @@ public: virtual void onActionLockRemove(StorageActionBlockType /* action_type */) {} std::atomic is_dropped{false}; + std::atomic is_detached{false}; /// Does table support index for IN sections virtual bool supportsIndexForIn() const { return false; } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 9206485fb73..8f36ea4d91d 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -470,7 +470,6 @@ void StorageLiveView::drop() DatabaseCatalog::instance().removeViewDependency(select_table_id, table_id); std::lock_guard lock(mutex); - is_dropped = true; condition.notify_all(); } diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index a76503197aa..e4180a8e9f4 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -735,10 +735,7 @@ std::unique_ptr DataPartStorageOnDisk::writeFile( size_t buf_size, const WriteSettings & settings) { - if (transaction) - return transaction->writeFile(fs::path(root_path) / part_dir / name, buf_size, WriteMode::Rewrite, settings, /* autocommit = */ false); - - return volume->getDisk()->writeFile(fs::path(root_path) / part_dir / name, buf_size, WriteMode::Rewrite, settings); + return writeFile(name, buf_size, WriteMode::Rewrite, settings); } std::unique_ptr DataPartStorageOnDisk::writeFile( @@ -749,6 +746,7 @@ std::unique_ptr DataPartStorageOnDisk::writeFile( { if (transaction) return transaction->writeFile(fs::path(root_path) / part_dir / name, buf_size, mode, settings, /* autocommit = */ false); + return volume->getDisk()->writeFile(fs::path(root_path) / part_dir / name, buf_size, mode, settings); } diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index 3e82d44d71e..86b8784e2b1 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -106,17 +106,17 @@ public: const String & name, size_t buf_size, const WriteSettings & settings) override; + std::unique_ptr writeFile( + const String & name, + size_t buf_size, + DB::WriteMode mode, + const WriteSettings & settings) override; std::unique_ptr writeTransactionFile(WriteMode mode) const override; void createFile(const String & name) override; void moveFile(const String & from_name, const String & to_name) override; void replaceFile(const String & from_name, const String & to_name) override; - std::unique_ptr writeFile( - const String & name, - size_t buf_size, - DB::WriteMode mode, - const WriteSettings & settings) override; void removeFile(const String & name) override; void removeFileIfExists(const String & name) override; diff --git a/src/Storages/MergeTree/GinIndexStore.cpp b/src/Storages/MergeTree/GinIndexStore.cpp index 995f4f9f88c..0904855755c 100644 --- a/src/Storages/MergeTree/GinIndexStore.cpp +++ b/src/Storages/MergeTree/GinIndexStore.cpp @@ -1,115 +1,84 @@ #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include namespace DB { -using TokenPostingsBuilderPair = std::pair; -using TokenPostingsBuilderPairs = std::vector; - namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int UNKNOWN_FORMAT_VERSION; }; -GinIndexStore::GinIndexStore(const String & name_, DataPartStoragePtr storage_) - : name(name_) - , storage(storage_) -{ -} -GinIndexStore::GinIndexStore(const String& name_, DataPartStoragePtr storage_, MutableDataPartStoragePtr data_part_storage_builder_, UInt64 max_digestion_size_) - : name(name_) - , storage(storage_) - , data_part_storage_builder(data_part_storage_builder_) - , max_digestion_size(max_digestion_size_) -{ -} - -GinIndexPostingsBuilder::GinIndexPostingsBuilder(UInt64 limit) : rowid_lst{}, size_limit(limit) +GinIndexPostingsBuilder::GinIndexPostingsBuilder(UInt64 limit) + : rowid_lst{} + , size_limit(limit) {} bool GinIndexPostingsBuilder::contains(UInt32 row_id) const { if (useRoaring()) return rowid_bitmap.contains(row_id); - - const auto * const it = std::find(rowid_lst.begin(), rowid_lst.begin()+rowid_lst_length, row_id); - return it != rowid_lst.begin() + rowid_lst_length; + else + { + const auto * const it = std::find(rowid_lst.begin(), rowid_lst.begin()+rowid_lst_length, row_id); + return it != rowid_lst.begin() + rowid_lst_length; + } } void GinIndexPostingsBuilder::add(UInt32 row_id) { if (containsAllRows()) - { return; - } + if (useRoaring()) { if (rowid_bitmap.cardinality() == size_limit) { - //reset the postings list with MATCH ALWAYS; - rowid_lst_length = 1; //makes sure useRoaring() returns false; - rowid_lst[0] = UINT32_MAX; //set CONTAINS ALL flag; + /// reset the postings list with MATCH ALWAYS; + rowid_lst_length = 1; /// makes sure useRoaring() returns false; + rowid_lst[0] = CONTAINS_ALL; /// set CONTAINS_ALL flag; } else - { rowid_bitmap.add(row_id); - } - return; } - assert(rowid_lst_length < MIN_SIZE_FOR_ROARING_ENCODING); - rowid_lst[rowid_lst_length] = row_id; - rowid_lst_length++; - - if (rowid_lst_length == MIN_SIZE_FOR_ROARING_ENCODING) + else { - for (size_t i = 0; i < rowid_lst_length; i++) - rowid_bitmap.add(rowid_lst[i]); + assert(rowid_lst_length < MIN_SIZE_FOR_ROARING_ENCODING); + rowid_lst[rowid_lst_length] = row_id; + rowid_lst_length++; - rowid_lst_length = UsesBitMap; + if (rowid_lst_length == MIN_SIZE_FOR_ROARING_ENCODING) + { + for (size_t i = 0; i < rowid_lst_length; i++) + rowid_bitmap.add(rowid_lst[i]); + + rowid_lst_length = USES_BIT_MAP; + } } } -bool GinIndexPostingsBuilder::useRoaring() const -{ - return rowid_lst_length == UsesBitMap; -} - -bool GinIndexPostingsBuilder::containsAllRows() const -{ - return rowid_lst[0] == UINT32_MAX; -} - -UInt64 GinIndexPostingsBuilder::serialize(WriteBuffer &buffer) const +UInt64 GinIndexPostingsBuilder::serialize(WriteBuffer & buffer) const { UInt64 written_bytes = 0; buffer.write(rowid_lst_length); written_bytes += 1; - if (!useRoaring()) - { - for (size_t i = 0; i < rowid_lst_length; ++i) - { - writeVarUInt(rowid_lst[i], buffer); - written_bytes += getLengthOfVarUInt(rowid_lst[i]); - } - } - else + if (useRoaring()) { auto size = rowid_bitmap.getSizeInBytes(); @@ -121,65 +90,85 @@ UInt64 GinIndexPostingsBuilder::serialize(WriteBuffer &buffer) const buffer.write(buf.get(), size); written_bytes += size; } + else + { + for (size_t i = 0; i < rowid_lst_length; ++i) + { + writeVarUInt(rowid_lst[i], buffer); + written_bytes += getLengthOfVarUInt(rowid_lst[i]); + } + } + return written_bytes; } -GinIndexPostingsListPtr GinIndexPostingsBuilder::deserialize(ReadBuffer &buffer) +GinIndexPostingsListPtr GinIndexPostingsBuilder::deserialize(ReadBuffer & buffer) { UInt8 postings_list_size = 0; - buffer.readStrict(reinterpret_cast(postings_list_size)); + buffer.readStrict(reinterpret_cast(postings_list_size)); - if (postings_list_size != UsesBitMap) + if (postings_list_size == USES_BIT_MAP) + { + size_t size = 0; + readVarUInt(size, buffer); + auto buf = std::make_unique(size); + buffer.readStrict(reinterpret_cast(buf.get()), size); + + GinIndexPostingsListPtr postings_list = std::make_shared(GinIndexPostingsList::read(buf.get())); + + return postings_list; + } + else { assert(postings_list_size < MIN_SIZE_FOR_ROARING_ENCODING); GinIndexPostingsListPtr postings_list = std::make_shared(); UInt32 row_ids[MIN_SIZE_FOR_ROARING_ENCODING]; for (auto i = 0; i < postings_list_size; ++i) - { readVarUInt(row_ids[i], buffer); - } postings_list->addMany(postings_list_size, row_ids); return postings_list; } - else - { - size_t size{0}; - readVarUInt(size, buffer); - auto buf = std::make_unique(size); - buffer.readStrict(reinterpret_cast(buf.get()), size); +} - GinIndexPostingsListPtr postings_list = std::make_shared(GinIndexPostingsList::read(buf.get())); - - return postings_list; - } +GinIndexStore::GinIndexStore(const String & name_, DataPartStoragePtr storage_) + : name(name_) + , storage(storage_) +{ +} +GinIndexStore::GinIndexStore(const String & name_, DataPartStoragePtr storage_, MutableDataPartStoragePtr data_part_storage_builder_, UInt64 max_digestion_size_) + : name(name_) + , storage(storage_) + , data_part_storage_builder(data_part_storage_builder_) + , max_digestion_size(max_digestion_size_) +{ } bool GinIndexStore::exists() const { - String id_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE; - return storage->exists(id_file_name); + String segment_id_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE; + return storage->exists(segment_id_file_name); } -UInt32 GinIndexStore::getNextSegmentIDRange(const String& file_name, size_t n) +UInt32 GinIndexStore::getNextSegmentIDRange(const String & file_name, size_t n) { - std::lock_guard guard(gin_index_store_mutex); + std::lock_guard guard(mutex); - /// When the method is called for the first time, the file doesn't exist yet, need to create it - /// and write segment ID 1. + /// When the method is called for the first time, the file doesn't exist yet, need to create it and write segment ID 1. if (!storage->exists(file_name)) { - /// Create file and write initial segment id = 1 + /// Create file std::unique_ptr ostr = this->data_part_storage_builder->writeFile(file_name, DBMS_DEFAULT_BUFFER_SIZE, {}); /// Write version writeChar(static_cast(CURRENT_GIN_FILE_FORMAT_VERSION), *ostr); + /// Write segment ID 1 writeVarUInt(1, *ostr); ostr->sync(); } - /// read id in file + /// Read id in file UInt32 result = 0; { std::unique_ptr istr = this->storage->readFile(file_name, {}, std::nullopt, std::nullopt); @@ -189,7 +178,8 @@ UInt32 GinIndexStore::getNextSegmentIDRange(const String& file_name, size_t n) readVarUInt(result, *istr); } - //save result+n + + /// Save result + n { std::unique_ptr ostr = this->data_part_storage_builder->writeFile(file_name, DBMS_DEFAULT_BUFFER_SIZE, {}); @@ -204,15 +194,15 @@ UInt32 GinIndexStore::getNextSegmentIDRange(const String& file_name, size_t n) UInt32 GinIndexStore::getNextRowIDRange(size_t numIDs) { - UInt32 result =current_segment.next_row_id; + UInt32 result = current_segment.next_row_id; current_segment.next_row_id += numIDs; return result; } UInt32 GinIndexStore::getNextSegmentID() { - String sid_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE; - return getNextSegmentIDRange(sid_file_name, 1); + String segment_id_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE; + return getNextSegmentIDRange(segment_id_file_name, 1); } UInt32 GinIndexStore::getNumOfSegments() @@ -220,18 +210,18 @@ UInt32 GinIndexStore::getNumOfSegments() if (cached_segment_num) return cached_segment_num; - String sid_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE; - if (!storage->exists(sid_file_name)) + String segment_id_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE; + if (!storage->exists(segment_id_file_name)) return 0; UInt32 result = 0; { - std::unique_ptr istr = this->storage->readFile(sid_file_name, {}, std::nullopt, std::nullopt); + std::unique_ptr istr = this->storage->readFile(segment_id_file_name, {}, std::nullopt, std::nullopt); uint8_t version = 0; readBinary(version, *istr); - if (version > CURRENT_GIN_FILE_FORMAT_VERSION) + if (version > static_cast>(CURRENT_GIN_FILE_FORMAT_VERSION)) throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported inverted index version {}", version); readVarUInt(result, *istr); @@ -250,88 +240,84 @@ bool GinIndexStore::needToWrite() const void GinIndexStore::finalize() { if (!current_postings.empty()) - { writeSegment(); - } } void GinIndexStore::initFileStreams() { - String segment_file_name = getName() + GIN_SEGMENT_FILE_TYPE; - String term_dict_file_name = getName() + GIN_DICTIONARY_FILE_TYPE; + String metadata_file_name = getName() + GIN_SEGMENT_METADATA_FILE_TYPE; + String dict_file_name = getName() + GIN_DICTIONARY_FILE_TYPE; String postings_file_name = getName() + GIN_POSTINGS_FILE_TYPE; - segment_file_stream = data_part_storage_builder->writeFile(segment_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {}); - term_dict_file_stream = data_part_storage_builder->writeFile(term_dict_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {}); + metadata_file_stream = data_part_storage_builder->writeFile(metadata_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {}); + dict_file_stream = data_part_storage_builder->writeFile(dict_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {}); postings_file_stream = data_part_storage_builder->writeFile(postings_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {}); } void GinIndexStore::writeSegment() { - if (segment_file_stream == nullptr) - { + if (metadata_file_stream == nullptr) initFileStreams(); - } + + using TokenPostingsBuilderPair = std::pair; + using TokenPostingsBuilderPairs = std::vector; /// Write segment - segment_file_stream->write(reinterpret_cast(¤t_segment), sizeof(GinIndexSegment)); + metadata_file_stream->write(reinterpret_cast(¤t_segment), sizeof(GinIndexSegment)); TokenPostingsBuilderPairs token_postings_list_pairs; token_postings_list_pairs.reserve(current_postings.size()); - for (const auto& [token, postings_list] : current_postings) - { + for (const auto & [token, postings_list] : current_postings) token_postings_list_pairs.push_back({token, postings_list}); - } /// Sort token-postings list pairs since all tokens have to be added in FST in sorted order std::sort(token_postings_list_pairs.begin(), token_postings_list_pairs.end(), - [](const TokenPostingsBuilderPair& a, const TokenPostingsBuilderPair & b) + [](const TokenPostingsBuilderPair & x, const TokenPostingsBuilderPair & y) { - return a.first < b.first; + return x.first < y.first; }); - ///write postings + /// Write postings std::vector posting_list_byte_sizes(current_postings.size(), 0); - for (size_t current_index = 0; const auto& [token, postings_list] : token_postings_list_pairs) + for (size_t i = 0; const auto & [token, postings_list] : token_postings_list_pairs) { auto posting_list_byte_size = postings_list->serialize(*postings_file_stream); - posting_list_byte_sizes[current_index] = posting_list_byte_size; - current_index++; + posting_list_byte_sizes[i] = posting_list_byte_size; + i++; current_segment.postings_start_offset += posting_list_byte_size; } ///write item dictionary std::vector buffer; WriteBufferFromVector> write_buf(buffer); - FST::FSTBuilder builder(write_buf); + FST::FstBuilder fst_builder(write_buf); UInt64 offset = 0; - for (size_t current_index = 0; const auto& [token, postings_list] : token_postings_list_pairs) + for (size_t i = 0; const auto & [token, postings_list] : token_postings_list_pairs) { - String str_token{token}; - builder.add(str_token, offset); - offset += posting_list_byte_sizes[current_index]; - current_index++; + fst_builder.add(token, offset); + offset += posting_list_byte_sizes[i]; + i++; } - builder.build(); + fst_builder.build(); write_buf.finalize(); /// Write FST size - writeVarUInt(buffer.size(), *term_dict_file_stream); - current_segment.term_dict_start_offset += getLengthOfVarUInt(buffer.size()); + writeVarUInt(buffer.size(), *dict_file_stream); + current_segment.dict_start_offset += getLengthOfVarUInt(buffer.size()); - /// Write FST content - term_dict_file_stream->write(reinterpret_cast(buffer.data()), buffer.size()); - current_segment.term_dict_start_offset += buffer.size(); + /// Write FST blob + dict_file_stream->write(reinterpret_cast(buffer.data()), buffer.size()); + current_segment.dict_start_offset += buffer.size(); current_size = 0; current_postings.clear(); current_segment.segment_id = getNextSegmentID(); - segment_file_stream->sync(); - term_dict_file_stream->sync(); + metadata_file_stream->sync(); + dict_file_stream->sync(); postings_file_stream->sync(); } @@ -343,82 +329,79 @@ GinIndexStoreDeserializer::GinIndexStoreDeserializer(const GinIndexStorePtr & st void GinIndexStoreDeserializer::initFileStreams() { - String segment_file_name = store->getName() + GinIndexStore::GIN_SEGMENT_FILE_TYPE; - String term_dict_file_name = store->getName() + GinIndexStore::GIN_DICTIONARY_FILE_TYPE; + String metadata_file_name = store->getName() + GinIndexStore::GIN_SEGMENT_METADATA_FILE_TYPE; + String dict_file_name = store->getName() + GinIndexStore::GIN_DICTIONARY_FILE_TYPE; String postings_file_name = store->getName() + GinIndexStore::GIN_POSTINGS_FILE_TYPE; - segment_file_stream = store->storage->readFile(segment_file_name, {}, std::nullopt, std::nullopt); - term_dict_file_stream = store->storage->readFile(term_dict_file_name, {}, std::nullopt, std::nullopt); + metadata_file_stream = store->storage->readFile(metadata_file_name, {}, std::nullopt, std::nullopt); + dict_file_stream = store->storage->readFile(dict_file_name, {}, std::nullopt, std::nullopt); postings_file_stream = store->storage->readFile(postings_file_name, {}, std::nullopt, std::nullopt); } void GinIndexStoreDeserializer::readSegments() { - auto num_segments = store->getNumOfSegments(); + UInt32 num_segments = store->getNumOfSegments(); if (num_segments == 0) return; + using GinIndexSegments = std::vector; GinIndexSegments segments (num_segments); - assert(segment_file_stream != nullptr); + assert(metadata_file_stream != nullptr); - segment_file_stream->readStrict(reinterpret_cast(segments.data()), num_segments * sizeof(GinIndexSegment)); - for (size_t i = 0; i < num_segments; ++i) + metadata_file_stream->readStrict(reinterpret_cast(segments.data()), num_segments * sizeof(GinIndexSegment)); + for (UInt32 i = 0; i < num_segments; ++i) { auto seg_id = segments[i].segment_id; - auto term_dict = std::make_shared(); - term_dict->postings_start_offset = segments[i].postings_start_offset; - term_dict->term_dict_start_offset = segments[i].term_dict_start_offset; - store->term_dicts[seg_id] = term_dict; + auto seg_dict = std::make_shared(); + seg_dict->postings_start_offset = segments[i].postings_start_offset; + seg_dict->dict_start_offset = segments[i].dict_start_offset; + store->segment_dictionaries[seg_id] = seg_dict; } } -void GinIndexStoreDeserializer::readSegmentTermDictionaries() +void GinIndexStoreDeserializer::readSegmentDictionaries() { for (UInt32 seg_index = 0; seg_index < store->getNumOfSegments(); ++seg_index) - { - readSegmentTermDictionary(seg_index); - } + readSegmentDictionary(seg_index); } -void GinIndexStoreDeserializer::readSegmentTermDictionary(UInt32 segment_id) +void GinIndexStoreDeserializer::readSegmentDictionary(UInt32 segment_id) { /// Check validity of segment_id - auto it = store->term_dicts.find(segment_id); - if (it == store->term_dicts.end()) - { + auto it = store->segment_dictionaries.find(segment_id); + if (it == store->segment_dictionaries.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid segment id {}", segment_id); - } - assert(term_dict_file_stream != nullptr); + assert(dict_file_stream != nullptr); - /// Set file pointer of term dictionary file - term_dict_file_stream->seek(it->second->term_dict_start_offset, SEEK_SET); + /// Set file pointer of dictionary file + dict_file_stream->seek(it->second->dict_start_offset, SEEK_SET); it->second->offsets.getData().clear(); /// Read FST size - size_t fst_size{0}; - readVarUInt(fst_size, *term_dict_file_stream); + size_t fst_size = 0; + readVarUInt(fst_size, *dict_file_stream); - /// Read FST content + /// Read FST blob it->second->offsets.getData().resize(fst_size); - term_dict_file_stream->readStrict(reinterpret_cast(it->second->offsets.getData().data()), fst_size); + dict_file_stream->readStrict(reinterpret_cast(it->second->offsets.getData().data()), fst_size); } -SegmentedPostingsListContainer GinIndexStoreDeserializer::readSegmentedPostingsLists(const String& term) +GinSegmentedPostingsListContainer GinIndexStoreDeserializer::readSegmentedPostingsLists(const String & term) { assert(postings_file_stream != nullptr); - SegmentedPostingsListContainer container; - for (auto const& seg_term_dict : store->term_dicts) + GinSegmentedPostingsListContainer container; + for (auto const & seg_dict : store->segment_dictionaries) { - auto segment_id = seg_term_dict.first; + auto segment_id = seg_dict.first; - auto [offset, found] = seg_term_dict.second->offsets.getOutput(term); + auto [offset, found] = seg_dict.second->offsets.getOutput(term); if (!found) continue; // Set postings file pointer for reading postings list - postings_file_stream->seek(seg_term_dict.second->postings_start_offset + offset, SEEK_SET); + postings_file_stream->seek(seg_dict.second->postings_start_offset + offset, SEEK_SET); // Read posting list auto postings_list = GinIndexPostingsBuilder::deserialize(*postings_file_stream); @@ -427,10 +410,10 @@ SegmentedPostingsListContainer GinIndexStoreDeserializer::readSegmentedPostingsL return container; } -PostingsCachePtr GinIndexStoreDeserializer::createPostingsCacheFromTerms(const std::vector& terms) +GinPostingsCachePtr GinIndexStoreDeserializer::createPostingsCacheFromTerms(const std::vector & terms) { - auto postings_cache = std::make_shared(); - for (const auto& term : terms) + auto postings_cache = std::make_shared(); + for (const auto & term : terms) { // Make sure don't read for duplicated terms if (postings_cache->find(term) != postings_cache->end()) @@ -442,18 +425,26 @@ PostingsCachePtr GinIndexStoreDeserializer::createPostingsCacheFromTerms(const s return postings_cache; } -GinIndexStoreFactory& GinIndexStoreFactory::instance() +GinPostingsCachePtr PostingsCacheForStore::getPostings(const String & query_string) const +{ + auto it = cache.find(query_string); + if (it == cache.end()) + return nullptr; + return it->second; +} + +GinIndexStoreFactory & GinIndexStoreFactory::instance() { static GinIndexStoreFactory instance; return instance; } -GinIndexStorePtr GinIndexStoreFactory::get(const String& name, DataPartStoragePtr storage) +GinIndexStorePtr GinIndexStoreFactory::get(const String & name, DataPartStoragePtr storage) { - const String& part_path = storage->getRelativePath(); + const String & part_path = storage->getRelativePath(); String key = name + ":" + part_path; - std::lock_guard lock(stores_mutex); + std::lock_guard lock(mutex); GinIndexStores::const_iterator it = stores.find(key); if (it == stores.end()) @@ -464,7 +455,7 @@ GinIndexStorePtr GinIndexStoreFactory::get(const String& name, DataPartStoragePt GinIndexStoreDeserializer deserializer(store); deserializer.readSegments(); - deserializer.readSegmentTermDictionaries(); + deserializer.readSegmentDictionaries(); stores[key] = store; @@ -473,9 +464,9 @@ GinIndexStorePtr GinIndexStoreFactory::get(const String& name, DataPartStoragePt return it->second; } -void GinIndexStoreFactory::remove(const String& part_path) +void GinIndexStoreFactory::remove(const String & part_path) { - std::lock_guard lock(stores_mutex); + std::lock_guard lock(mutex); for (auto it = stores.begin(); it != stores.end();) { if (it->first.find(part_path) != String::npos) @@ -484,4 +475,5 @@ void GinIndexStoreFactory::remove(const String& part_path) ++it; } } + } diff --git a/src/Storages/MergeTree/GinIndexStore.h b/src/Storages/MergeTree/GinIndexStore.h index c326322191f..e075a7e73bf 100644 --- a/src/Storages/MergeTree/GinIndexStore.h +++ b/src/Storages/MergeTree/GinIndexStore.h @@ -1,18 +1,18 @@ #pragma once -#include -#include -#include -#include +#include #include #include #include #include -#include -#include #include +#include +#include +#include +#include +#include -/// GinIndexStore manages the inverted index for a data part, and it is made up of one or more immutable +/// GinIndexStore manages the generalized inverted index ("gin") for a data part, and it is made up of one or more immutable /// index segments. /// /// There are 4 types of index files in a store: @@ -20,40 +20,30 @@ /// 2. Segment Metadata file(.gin_seg): it contains index segment metadata. /// - Its file format is an array of GinIndexSegment as defined in this file. /// - postings_start_offset points to the file(.gin_post) starting position for the segment's postings list. -/// - term_dict_start_offset points to the file(.gin_dict) starting position for the segment's term dictionaries. -/// 3. Term Dictionary file(.gin_dict): it contains term dictionaries. +/// - dict_start_offset points to the file(.gin_dict) starting position for the segment's dictionaries. +/// 3. Dictionary file(.gin_dict): it contains dictionaries. /// - It contains an array of (FST_size, FST_blob) which has size and actual data of FST. /// 4. Postings Lists(.gin_post): it contains postings lists data. /// - It contains an array of serialized postings lists. /// /// During the searching in the segment, the segment's meta data can be found in .gin_seg file. From the meta data, -/// the starting position of its term dictionary is used to locate its FST. Then FST is read into memory. +/// the starting position of its dictionary is used to locate its FST. Then FST is read into memory. /// By using the term and FST, the offset("output" in FST) of the postings list for the term /// in FST is found. The offset plus the postings_start_offset is the file location in .gin_post file /// for its postings list. namespace DB { -enum : uint8_t -{ - GIN_VERSION_0 = 0, - GIN_VERSION_1 = 1, /// Initial version -}; - -static constexpr auto CURRENT_GIN_FILE_FORMAT_VERSION = GIN_VERSION_1; /// GinIndexPostingsList which uses 32-bit Roaring using GinIndexPostingsList = roaring::Roaring; - using GinIndexPostingsListPtr = std::shared_ptr; -/// Gin Index Postings List Builder. +/// Build a postings list for a term class GinIndexPostingsBuilder { public: - constexpr static int MIN_SIZE_FOR_ROARING_ENCODING = 16; - - GinIndexPostingsBuilder(UInt64 limit); + explicit GinIndexPostingsBuilder(UInt64 limit); /// Check whether a row_id is already added bool contains(UInt32 row_id) const; @@ -61,91 +51,88 @@ public: /// Add a row_id into the builder void add(UInt32 row_id); - /// Check whether the builder is using roaring bitmap - bool useRoaring() const; - - /// Check whether the postings list has been flagged to contain all row ids - bool containsAllRows() const; - /// Serialize the content of builder to given WriteBuffer, returns the bytes of serialized data - UInt64 serialize(WriteBuffer &buffer) const; + UInt64 serialize(WriteBuffer & buffer) const; /// Deserialize the postings list data from given ReadBuffer, return a pointer to the GinIndexPostingsList created by deserialization - static GinIndexPostingsListPtr deserialize(ReadBuffer &buffer); + static GinIndexPostingsListPtr deserialize(ReadBuffer & buffer); + private: + constexpr static int MIN_SIZE_FOR_ROARING_ENCODING = 16; + /// When the list length is no greater than MIN_SIZE_FOR_ROARING_ENCODING, array 'rowid_lst' is used + /// As a special case, rowid_lst[0] == CONTAINS_ALL encodes that all rowids are set. std::array rowid_lst; - /// When the list length is greater than MIN_SIZE_FOR_ROARING_ENCODING, Roaring bitmap 'rowid_bitmap' is used + /// When the list length is greater than MIN_SIZE_FOR_ROARING_ENCODING, roaring bitmap 'rowid_bitmap' is used roaring::Roaring rowid_bitmap; /// rowid_lst_length stores the number of row IDs in 'rowid_lst' array, can also be a flag(0xFF) indicating that roaring bitmap is used - UInt8 rowid_lst_length{0}; + UInt8 rowid_lst_length = 0; + + /// Indicates that all rowids are contained, see 'rowid_lst' + static constexpr UInt32 CONTAINS_ALL = std::numeric_limits::max(); + + /// Indicates that roaring bitmap is used, see 'rowid_lst_length'. + static constexpr UInt8 USES_BIT_MAP = 0xFF; - static constexpr UInt8 UsesBitMap = 0xFF; /// Clear the postings list and reset it with MATCHALL flags when the size of the postings list is beyond the limit UInt64 size_limit; + + /// Check whether the builder is using roaring bitmap + bool useRoaring() const { return rowid_lst_length == USES_BIT_MAP; } + + /// Check whether the postings list has been flagged to contain all row ids + bool containsAllRows() const { return rowid_lst[0] == CONTAINS_ALL; } }; -/// Container for postings lists for each segment -using SegmentedPostingsListContainer = std::unordered_map; +using GinIndexPostingsBuilderPtr = std::shared_ptr; -/// Postings lists and terms built from query string -using PostingsCache = std::unordered_map; -using PostingsCachePtr = std::shared_ptr; - -/// Gin Index Segment information, which contains: +/// Gin index segment descriptor, which contains: struct GinIndexSegment { /// Segment ID retrieved from next available ID from file .gin_sid UInt32 segment_id = 0; - /// Next row ID for this segment + /// Start row ID for this segment UInt32 next_row_id = 1; /// .gin_post file offset of this segment's postings lists UInt64 postings_start_offset = 0; - /// .term_dict file offset of this segment's term dictionaries - UInt64 term_dict_start_offset = 0; + /// .gin_dict file offset of this segment's dictionaries + UInt64 dict_start_offset = 0; }; -using GinIndexSegments = std::vector; - -struct SegmentTermDictionary +struct GinSegmentDictionary { /// .gin_post file offset of this segment's postings lists UInt64 postings_start_offset; - /// .gin_dict file offset of this segment's term dictionaries - UInt64 term_dict_start_offset; + /// .gin_dict file offset of this segment's dictionaries + UInt64 dict_start_offset; - /// Finite State Transducer, which can be viewed as a map of , where offset is the + /// (Minimized) Finite State Transducer, which can be viewed as a map of , where offset is the /// offset to the term's posting list in postings list file FST::FiniteStateTransducer offsets; }; -using SegmentTermDictionaryPtr = std::shared_ptr; +using GinSegmentDictionaryPtr = std::shared_ptr; -/// Term dictionaries indexed by segment ID -using SegmentTermDictionaries = std::unordered_map; - -/// Gin Index Store which has Gin Index meta data for the corresponding Data Part +/// Gin index store which has gin index meta data for the corresponding column data part class GinIndexStore { public: - using GinIndexPostingsBuilderPtr = std::shared_ptr; /// Container for all term's Gin Index Postings List Builder using GinIndexPostingsBuilderContainer = std::unordered_map; - explicit GinIndexStore(const String & name_, DataPartStoragePtr storage_); - - GinIndexStore(const String& name_, DataPartStoragePtr storage_, MutableDataPartStoragePtr data_part_storage_builder_, UInt64 max_digestion_size_); + GinIndexStore(const String & name_, DataPartStoragePtr storage_); + GinIndexStore(const String & name_, DataPartStoragePtr storage_, MutableDataPartStoragePtr data_part_storage_builder_, UInt64 max_digestion_size_); /// Check existence by checking the existence of file .gin_sid bool exists() const; - /// Get a range of next 'numIDs' available row IDs + /// Get a range of next 'numIDs'-many available row IDs UInt32 getNextRowIDRange(size_t numIDs); /// Get next available segment ID by updating file .gin_sid @@ -155,25 +142,26 @@ public: UInt32 getNumOfSegments(); /// Get current postings list builder - const GinIndexPostingsBuilderContainer& getPostings() const { return current_postings; } + const GinIndexPostingsBuilderContainer & getPostingsListBuilder() const { return current_postings; } /// Set postings list builder for given term void setPostingsBuilder(const String & term, GinIndexPostingsBuilderPtr builder) { current_postings[term] = builder; } + /// Check if we need to write segment to Gin index files bool needToWrite() const; /// Accumulate the size of text data which has been digested void incrementCurrentSizeBy(UInt64 sz) { current_size += sz; } - UInt32 getCurrentSegmentID() const { return current_segment.segment_id;} + UInt32 getCurrentSegmentID() const { return current_segment.segment_id; } /// Do last segment writing void finalize(); - /// method for writing segment data to Gin index files + /// Method for writing segment data to Gin index files void writeSegment(); - const String & getName() const {return name;} + const String & getName() const { return name; } private: friend class GinIndexStoreDeserializer; @@ -182,7 +170,7 @@ private: void initFileStreams(); /// Get a range of next available segment IDs by updating file .gin_sid - UInt32 getNextSegmentIDRange(const String &file_name, size_t n); + UInt32 getNextSegmentIDRange(const String & file_name, size_t n); String name; DataPartStoragePtr storage; @@ -190,37 +178,89 @@ private: UInt32 cached_segment_num = 0; - std::mutex gin_index_store_mutex; + std::mutex mutex; - /// Terms dictionaries which are loaded from .gin_dict files - SegmentTermDictionaries term_dicts; + /// Dictionaries indexed by segment ID + using GinSegmentDictionaries = std::unordered_map; - /// container for building postings lists during index construction + /// Term's dictionaries which are loaded from .gin_dict files + GinSegmentDictionaries segment_dictionaries; + + /// Container for building postings lists during index construction GinIndexPostingsBuilderContainer current_postings; - /// The following is for segmentation of Gin index - GinIndexSegment current_segment{}; + /// For the segmentation of Gin indexes + GinIndexSegment current_segment; UInt64 current_size = 0; const UInt64 max_digestion_size = 0; - /// File streams for segment, term dictionaries and postings lists - std::unique_ptr segment_file_stream; - std::unique_ptr term_dict_file_stream; + /// File streams for segment, dictionaries and postings lists + std::unique_ptr metadata_file_stream; + std::unique_ptr dict_file_stream; std::unique_ptr postings_file_stream; static constexpr auto GIN_SEGMENT_ID_FILE_TYPE = ".gin_sid"; - static constexpr auto GIN_SEGMENT_FILE_TYPE = ".gin_seg"; + static constexpr auto GIN_SEGMENT_METADATA_FILE_TYPE = ".gin_seg"; static constexpr auto GIN_DICTIONARY_FILE_TYPE = ".gin_dict"; static constexpr auto GIN_POSTINGS_FILE_TYPE = ".gin_post"; + + enum class Format : uint8_t + { + v0 = 0, + v1 = 1, /// Initial version + }; + + static constexpr auto CURRENT_GIN_FILE_FORMAT_VERSION = Format::v0; }; using GinIndexStorePtr = std::shared_ptr; -/// GinIndexStores indexed by part file path -using GinIndexStores = std::unordered_map; +/// Container for postings lists for each segment +using GinSegmentedPostingsListContainer = std::unordered_map; + +/// Postings lists and terms built from query string +using GinPostingsCache = std::unordered_map; +using GinPostingsCachePtr = std::shared_ptr; + +/// Gin index store reader which helps to read segments, dictionaries and postings list +class GinIndexStoreDeserializer : private boost::noncopyable +{ +public: + explicit GinIndexStoreDeserializer(const GinIndexStorePtr & store_); + + /// Read segment information from .gin_seg files + void readSegments(); + + /// Read all dictionaries from .gin_dict files + void readSegmentDictionaries(); + + /// Read dictionary for given segment id + void readSegmentDictionary(UInt32 segment_id); + + /// Read postings lists for the term + GinSegmentedPostingsListContainer readSegmentedPostingsLists(const String & term); + + /// Read postings lists for terms (which are created by tokenzing query string) + GinPostingsCachePtr createPostingsCacheFromTerms(const std::vector & terms); + +private: + /// Initialize gin index files + void initFileStreams(); + + /// The store for the reader + GinIndexStorePtr store; + + /// File streams for reading Gin Index + std::unique_ptr metadata_file_stream; + std::unique_ptr dict_file_stream; + std::unique_ptr postings_file_stream; + + /// Current segment, used in building index + GinIndexSegment current_segment; +}; /// PostingsCacheForStore contains postings lists from 'store' which are retrieved from Gin index files for the terms in query strings -/// PostingsCache is per query string(one query can have multiple query strings): when skipping index(row ID ranges) is used for the part during the +/// GinPostingsCache is per query string (one query can have multiple query strings): when skipping index (row ID ranges) is used for the part during the /// query, the postings cache is created and associated with the store where postings lists are read /// for the tokenized query string. The postings caches are released automatically when the query is done. struct PostingsCacheForStore @@ -229,76 +269,31 @@ struct PostingsCacheForStore GinIndexStorePtr store; /// map of - std::unordered_map cache; + std::unordered_map cache; /// Get postings lists for query string, return nullptr if not found - PostingsCachePtr getPostings(const String &query_string) const - { - auto it {cache.find(query_string)}; - - if (it == cache.cend()) - { - return nullptr; - } - return it->second; - } + GinPostingsCachePtr getPostings(const String & query_string) const; }; -/// GinIndexStore Factory, which is a singleton for storing GinIndexStores +/// A singleton for storing GinIndexStores class GinIndexStoreFactory : private boost::noncopyable { public: /// Get singleton of GinIndexStoreFactory - static GinIndexStoreFactory& instance(); + static GinIndexStoreFactory & instance(); /// Get GinIndexStore by using index name, disk and part_path (which are combined to create key in stores) - GinIndexStorePtr get(const String& name, DataPartStoragePtr storage); + GinIndexStorePtr get(const String & name, DataPartStoragePtr storage); /// Remove all Gin index files which are under the same part_path - void remove(const String& part_path); + void remove(const String & part_path); + + /// GinIndexStores indexed by part file path + using GinIndexStores = std::unordered_map; private: GinIndexStores stores; - std::mutex stores_mutex; -}; - -/// Term dictionary information, which contains: - -/// Gin Index Store Reader which helps to read segments, term dictionaries and postings list -class GinIndexStoreDeserializer : private boost::noncopyable -{ -public: - explicit GinIndexStoreDeserializer(const GinIndexStorePtr & store_); - - /// Read all segment information from .gin_seg files - void readSegments(); - - /// Read all term dictionaries from .gin_dict files - void readSegmentTermDictionaries(); - - /// Read term dictionary for given segment id - void readSegmentTermDictionary(UInt32 segment_id); - - /// Read postings lists for the term - SegmentedPostingsListContainer readSegmentedPostingsLists(const String& term); - - /// Read postings lists for terms(which are created by tokenzing query string) - PostingsCachePtr createPostingsCacheFromTerms(const std::vector& terms); - -private: - /// Initialize Gin index files - void initFileStreams(); - - /// The store for the reader - GinIndexStorePtr store; - - /// File streams for reading Gin Index - std::unique_ptr segment_file_stream; - std::unique_ptr term_dict_file_stream; - std::unique_ptr postings_file_stream; - - /// Current segment, used in building index - GinIndexSegment current_segment; + std::mutex mutex; }; } diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 70cc4d3fe70..84ca3e733c6 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -216,7 +216,11 @@ public: const String & name, size_t buf_size, const WriteSettings & settings) = 0; - virtual std::unique_ptr writeFile(const String & name, size_t buf_size, WriteMode mode, const WriteSettings & settings) = 0; + virtual std::unique_ptr writeFile( + const String & name, + size_t buf_size, + WriteMode mode, + const WriteSettings & settings) = 0; /// A special const method to write transaction file. /// It's const, because file with transaction metadata diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 2c5169a1729..68d5147362b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -1,6 +1,6 @@ #pragma once -#include "IO/WriteSettings.h" +#include #include #include #include @@ -22,8 +22,6 @@ #include #include -#include - namespace zkutil { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bb507263be0..81f5783da4a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2787,10 +2787,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } if (commands.hasInvertedIndex(new_metadata, getContext()) && !settings.allow_experimental_inverted_index) - { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Experimental Inverted Index feature is not enabled (the setting 'allow_experimental_inverted_index')"); - } + "Experimental Inverted Index feature is not enabled (turn on setting 'allow_experimental_inverted_index')"); + commands.apply(new_metadata, getContext()); /// Set of columns that shouldn't be altered. diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 15f3787003c..2e57d58ef41 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -1089,7 +1090,7 @@ protected: MultiVersion storage_settings; /// Used to determine which UUIDs to send to root query executor for deduplication. - mutable std::shared_mutex pinned_part_uuids_mutex; + mutable SharedMutex pinned_part_uuids_mutex; PinnedPartUUIDsPtr pinned_part_uuids; /// True if at least one part was created/removed with transaction. diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index ab280eb095e..1f40177d0fa 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include "IO/WriteBufferFromFileDecorator.h" @@ -215,7 +215,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() settings.query_write_settings)); GinIndexStorePtr store = nullptr; - if (dynamic_cast(&*index_helper) != nullptr) + if (dynamic_cast(&*index_helper) != nullptr) { store = std::make_shared(stream_name, data_part->getDataPartStoragePtr(), data_part->getDataPartStoragePtr(), storage.getSettings()->max_digestion_size_per_segment); gin_index_stores[stream_name] = store; @@ -276,15 +276,13 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block auto & stream = *skip_indices_streams[i]; WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing; - GinIndexStorePtr store = nullptr; - if (dynamic_cast(&*index_helper) != nullptr) + GinIndexStorePtr store; + if (dynamic_cast(&*index_helper) != nullptr) { String stream_name = index_helper->getFileName(); auto it = gin_index_stores.find(stream_name); - if (it == gin_index_stores.cend()) - { + if (it == gin_index_stores.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Index '{}' does not exist", stream_name); - } store = it->second; } @@ -401,9 +399,7 @@ void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(bool sync) stream->sync(); } for (auto & store: gin_index_stores) - { store.second->finalize(); - } gin_index_stores.clear(); skip_indices_streams.clear(); skip_indices_aggregators.clear(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 2377a129ac0..b76b74ab717 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -162,7 +162,7 @@ protected: /// Data is already written up to this mark. size_t current_mark = 0; - GinIndexStores gin_index_stores; + GinIndexStoreFactory::GinIndexStores gin_index_stores; private: void initSkipIndices(); void initPrimaryIndex(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 33ac1b4fdcd..512f194ea53 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include @@ -1689,10 +1689,8 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( PostingsCacheForStore cache_in_store; - if (dynamic_cast(&*index_helper) != nullptr) - { + if (dynamic_cast(&*index_helper) != nullptr) cache_in_store.store = GinIndexStoreFactory::instance().get(index_helper->getFileName(), part->getDataPartStoragePtr()); - } for (size_t i = 0; i < ranges.size(); ++i) { @@ -1707,7 +1705,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( { if (index_mark != index_range.begin || !granule || last_index_mark != index_range.begin) granule = reader.read(); - const auto * gin_filter_condition = dynamic_cast(&*condition); + const auto * gin_filter_condition = dynamic_cast(&*condition); // Cast to Ann condition auto ann_condition = std::dynamic_pointer_cast(condition); if (ann_condition != nullptr) @@ -1734,7 +1732,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( continue; } - bool result{false}; + bool result = false; if (!gin_filter_condition) result = condition->mayBeTrueOnGranule(granule); else diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 1a6b703c1b6..4904eb1e922 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -230,6 +230,7 @@ bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const return rpn_stack[0]; } +/// Keep in-sync with MergeTreeIndexConditionGin::mayBeTrueOnTranuleInPart bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const { std::shared_ptr granule diff --git a/src/Storages/MergeTree/MergeTreeIndexGin.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp similarity index 88% rename from src/Storages/MergeTree/MergeTreeIndexGin.cpp rename to src/Storages/MergeTree/MergeTreeIndexInverted.cpp index 3e2ab34fc05..02222aa530c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGin.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -1,30 +1,28 @@ - -#include +#include #include -#include +#include +#include +#include #include -#include +#include +#include +#include #include -#include +#include #include #include +#include #include #include -#include -#include -#include -#include #include #include #include -#include - #include -#include -#include -#include -#include +#include +#include +#include +#include namespace DB @@ -35,19 +33,18 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; } -MergeTreeIndexGranuleGinFilter::MergeTreeIndexGranuleGinFilter( +MergeTreeIndexGranuleInverted::MergeTreeIndexGranuleInverted( const String & index_name_, size_t columns_number, const GinFilterParameters & params_) : index_name(index_name_) , params(params_) - , gin_filters( - columns_number, GinFilter(params)) + , gin_filters(columns_number, GinFilter(params)) , has_elems(false) { } -void MergeTreeIndexGranuleGinFilter::serializeBinary(WriteBuffer & ostr) const +void MergeTreeIndexGranuleInverted::serializeBinary(WriteBuffer & ostr) const { if (empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty fulltext index {}.", backQuote(index_name)); @@ -59,11 +56,11 @@ void MergeTreeIndexGranuleGinFilter::serializeBinary(WriteBuffer & ostr) const { size_t filter_size = gin_filter.getFilter().size(); size_serialization->serializeBinary(filter_size, ostr, {}); - ostr.write(reinterpret_cast(gin_filter.getFilter().data()), filter_size * sizeof(GinFilter::GinSegmentWithRowIDRanges::value_type)); + ostr.write(reinterpret_cast(gin_filter.getFilter().data()), filter_size * sizeof(GinSegmentWithRowIdRangeVector::value_type)); } } -void MergeTreeIndexGranuleGinFilter::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) +void MergeTreeIndexGranuleInverted::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) { if (version != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version); @@ -81,13 +78,13 @@ void MergeTreeIndexGranuleGinFilter::deserializeBinary(ReadBuffer & istr, MergeT continue; gin_filter.getFilter().assign(filter_size, {}); - istr.readStrict(reinterpret_cast(gin_filter.getFilter().data()), filter_size * sizeof(GinFilter::GinSegmentWithRowIDRanges::value_type)); + istr.readStrict(reinterpret_cast(gin_filter.getFilter().data()), filter_size * sizeof(GinSegmentWithRowIdRangeVector::value_type)); } has_elems = true; } -MergeTreeIndexAggregatorGinFilter::MergeTreeIndexAggregatorGinFilter( +MergeTreeIndexAggregatorInverted::MergeTreeIndexAggregatorInverted( GinIndexStorePtr store_, const Names & index_columns_, const String & index_name_, @@ -99,32 +96,30 @@ MergeTreeIndexAggregatorGinFilter::MergeTreeIndexAggregatorGinFilter( , params(params_) , token_extractor(token_extractor_) , granule( - std::make_shared( + std::make_shared( index_name, index_columns.size(), params)) { } -MergeTreeIndexGranulePtr MergeTreeIndexAggregatorGinFilter::getGranuleAndReset() +MergeTreeIndexGranulePtr MergeTreeIndexAggregatorInverted::getGranuleAndReset() { - auto new_granule = std::make_shared( + auto new_granule = std::make_shared( index_name, index_columns.size(), params); new_granule.swap(granule); return new_granule; } -void MergeTreeIndexAggregatorGinFilter::addToGinFilter(UInt32 rowID, const char* data, size_t length, GinFilter& gin_filter, UInt64 limit) +void MergeTreeIndexAggregatorInverted::addToGinFilter(UInt32 rowID, const char * data, size_t length, GinFilter & gin_filter, UInt64 limit) { size_t cur = 0; size_t token_start = 0; size_t token_len = 0; while (cur < length && token_extractor->nextInStringPadded(data, length, &cur, &token_start, &token_len)) - { gin_filter.add(data + token_start, token_len, rowID, store, limit); - } } -void MergeTreeIndexAggregatorGinFilter::update(const Block & block, size_t * pos, size_t limit) +void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos, size_t limit) { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " @@ -188,7 +183,7 @@ void MergeTreeIndexAggregatorGinFilter::update(const Block & block, size_t * pos *pos += rows_read; } -MergeTreeConditionGinFilter::MergeTreeConditionGinFilter( +MergeTreeConditionInverted::MergeTreeConditionInverted( const SelectQueryInfo & query_info, ContextPtr context_, const Block & index_sample_block, @@ -235,7 +230,7 @@ MergeTreeConditionGinFilter::MergeTreeConditionGinFilter( } /// Keep in-sync with MergeTreeConditionFullText::alwaysUnknownOrTrue -bool MergeTreeConditionGinFilter::alwaysUnknownOrTrue() const +bool MergeTreeConditionInverted::alwaysUnknownOrTrue() const { /// Check like in KeyCondition. std::vector rpn_stack; @@ -282,10 +277,10 @@ bool MergeTreeConditionGinFilter::alwaysUnknownOrTrue() const return rpn_stack[0]; } -bool MergeTreeConditionGinFilter::mayBeTrueOnGranuleInPart(MergeTreeIndexGranulePtr idx_granule,[[maybe_unused]] PostingsCacheForStore &cache_store) const +bool MergeTreeConditionInverted::mayBeTrueOnGranuleInPart(MergeTreeIndexGranulePtr idx_granule,[[maybe_unused]] PostingsCacheForStore & cache_store) const { - std::shared_ptr granule - = std::dynamic_pointer_cast(idx_granule); + std::shared_ptr granule + = std::dynamic_pointer_cast(idx_granule); if (!granule) throw Exception(ErrorCodes::LOGICAL_ERROR, "GinFilter index condition got a granule with the wrong type."); @@ -373,7 +368,7 @@ bool MergeTreeConditionGinFilter::mayBeTrueOnGranuleInPart(MergeTreeIndexGranule return rpn_stack[0].can_be_true; } -bool MergeTreeConditionGinFilter::traverseAtomAST(const RPNBuilderTreeNode & node, RPNElement & out) +bool MergeTreeConditionInverted::traverseAtomAST(const RPNBuilderTreeNode & node, RPNElement & out) { { Field const_value; @@ -453,7 +448,7 @@ bool MergeTreeConditionGinFilter::traverseAtomAST(const RPNBuilderTreeNode & nod return false; } -bool MergeTreeConditionGinFilter::traverseASTEquals( +bool MergeTreeConditionInverted::traverseASTEquals( const String & function_name, const RPNBuilderTreeNode & key_ast, const DataTypePtr & value_type, @@ -606,7 +601,7 @@ bool MergeTreeConditionGinFilter::traverseASTEquals( out.function = RPNElement::FUNCTION_MULTI_SEARCH; /// 2d vector is not needed here but is used because already exists for FUNCTION_IN - std::vector> gin_filters; + std::vector gin_filters; gin_filters.emplace_back(); for (const auto & element : const_value.get()) { @@ -624,7 +619,7 @@ bool MergeTreeConditionGinFilter::traverseASTEquals( return false; } -bool MergeTreeConditionGinFilter::tryPrepareSetGinFilter( +bool MergeTreeConditionInverted::tryPrepareSetGinFilter( const RPNBuilderTreeNode & lhs, const RPNBuilderTreeNode & rhs, RPNElement & out) @@ -667,7 +662,7 @@ bool MergeTreeConditionGinFilter::tryPrepareSetGinFilter( if (data_type->getTypeId() != TypeIndex::String && data_type->getTypeId() != TypeIndex::FixedString) return false; - std::vector> gin_filters; + std::vector gin_filters; std::vector key_position; Columns columns = prepared_set->getSetElements(); @@ -693,55 +688,55 @@ bool MergeTreeConditionGinFilter::tryPrepareSetGinFilter( return true; } -MergeTreeIndexGranulePtr MergeTreeIndexGinFilter::createIndexGranule() const +MergeTreeIndexGranulePtr MergeTreeIndexInverted::createIndexGranule() const { - return std::make_shared(index.name, index.column_names.size(), params); + return std::make_shared(index.name, index.column_names.size(), params); } -MergeTreeIndexAggregatorPtr MergeTreeIndexGinFilter::createIndexAggregator() const +MergeTreeIndexAggregatorPtr MergeTreeIndexInverted::createIndexAggregator() const { /// should not be called: createIndexAggregatorForPart should be used assert(false); return nullptr; } -MergeTreeIndexAggregatorPtr MergeTreeIndexGinFilter::createIndexAggregatorForPart(const GinIndexStorePtr &store) const +MergeTreeIndexAggregatorPtr MergeTreeIndexInverted::createIndexAggregatorForPart(const GinIndexStorePtr & store) const { - return std::make_shared(store, index.column_names, index.name, params, token_extractor.get()); + return std::make_shared(store, index.column_names, index.name, params, token_extractor.get()); } -MergeTreeIndexConditionPtr MergeTreeIndexGinFilter::createIndexCondition( +MergeTreeIndexConditionPtr MergeTreeIndexInverted::createIndexCondition( const SelectQueryInfo & query, ContextPtr context) const { - return std::make_shared(query, context, index.sample_block, params, token_extractor.get()); + return std::make_shared(query, context, index.sample_block, params, token_extractor.get()); }; -bool MergeTreeIndexGinFilter::mayBenefitFromIndexForIn(const ASTPtr & node) const +bool MergeTreeIndexInverted::mayBenefitFromIndexForIn(const ASTPtr & node) const { return std::find(std::cbegin(index.column_names), std::cend(index.column_names), node->getColumnName()) != std::cend(index.column_names); } -MergeTreeIndexPtr ginIndexCreator( +MergeTreeIndexPtr invertedIndexCreator( const IndexDescription & index) { size_t n = index.arguments.empty() ? 0 : index.arguments[0].get(); - Float64 density = index.arguments.size() < 2 ? 1.0f : index.arguments[1].get(); + Float64 density = index.arguments.size() < 2 ? 1.0 : index.arguments[1].get(); GinFilterParameters params(n, density); /// Use SplitTokenExtractor when n is 0, otherwise use NgramTokenExtractor if (n > 0) { auto tokenizer = std::make_unique(n); - return std::make_shared(index, params, std::move(tokenizer)); + return std::make_shared(index, params, std::move(tokenizer)); } else { auto tokenizer = std::make_unique(); - return std::make_shared(index, params, std::move(tokenizer)); + return std::make_shared(index, params, std::move(tokenizer)); } } -void ginIndexValidator(const IndexDescription & index, bool /*attach*/) +void invertedIndexValidator(const IndexDescription & index, bool /*attach*/) { for (const auto & index_data_type : index.data_types) { @@ -764,9 +759,6 @@ void ginIndexValidator(const IndexDescription & index, bool /*attach*/) "column or Array with `String` or `FixedString` values column."); } - if (index.type != GinFilter::getName()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index type: {}", backQuote(index.name)); - if (index.arguments.size() > 2) throw Exception(ErrorCodes::INCORRECT_QUERY, "Inverted index must have less than two arguments."); @@ -776,10 +768,9 @@ void ginIndexValidator(const IndexDescription & index, bool /*attach*/) if (index.arguments.size() == 2 && (index.arguments[1].getType() != Field::Types::Float64 || index.arguments[1].get() <= 0 || index.arguments[1].get() > 1)) throw Exception(ErrorCodes::INCORRECT_QUERY, "The second Inverted index argument must be a float between 0 and 1."); - size_t ngrams = index.arguments.empty() ? 0 : index.arguments[0].get(); - Float64 density = index.arguments.size() < 2 ? 1.0f : index.arguments[1].get(); - /// Just validate + size_t ngrams = index.arguments.empty() ? 0 : index.arguments[0].get(); + Float64 density = index.arguments.size() < 2 ? 1.0 : index.arguments[1].get(); GinFilterParameters params(ngrams, density); } diff --git a/src/Storages/MergeTree/MergeTreeIndexGin.h b/src/Storages/MergeTree/MergeTreeIndexInverted.h similarity index 79% rename from src/Storages/MergeTree/MergeTreeIndexGin.h rename to src/Storages/MergeTree/MergeTreeIndexInverted.h index d915d493810..baabed09905 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGin.h +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.h @@ -1,24 +1,23 @@ #pragma once -#include -#include -#include - -#include -#include -#include #include +#include +#include +#include +#include +#include +#include namespace DB { -struct MergeTreeIndexGranuleGinFilter final : public IMergeTreeIndexGranule +struct MergeTreeIndexGranuleInverted final : public IMergeTreeIndexGranule { - explicit MergeTreeIndexGranuleGinFilter( + explicit MergeTreeIndexGranuleInverted( const String & index_name_, size_t columns_number, const GinFilterParameters & params_); - ~MergeTreeIndexGranuleGinFilter() override = default; + ~MergeTreeIndexGranuleInverted() override = default; void serializeBinary(WriteBuffer & ostr) const override; void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; @@ -27,30 +26,29 @@ struct MergeTreeIndexGranuleGinFilter final : public IMergeTreeIndexGranule String index_name; GinFilterParameters params; - - std::vector gin_filters; + GinFilters gin_filters; bool has_elems; }; -using MergeTreeIndexGranuleGinFilterPtr = std::shared_ptr; +using MergeTreeIndexGranuleInvertedPtr = std::shared_ptr; -struct MergeTreeIndexAggregatorGinFilter final : IMergeTreeIndexAggregator +struct MergeTreeIndexAggregatorInverted final : IMergeTreeIndexAggregator { - explicit MergeTreeIndexAggregatorGinFilter( + explicit MergeTreeIndexAggregatorInverted( GinIndexStorePtr store_, const Names & index_columns_, const String & index_name_, const GinFilterParameters & params_, TokenExtractorPtr token_extractor_); - ~MergeTreeIndexAggregatorGinFilter() override = default; + ~MergeTreeIndexAggregatorInverted() override = default; bool empty() const override { return !granule || granule->empty(); } MergeTreeIndexGranulePtr getGranuleAndReset() override; void update(const Block & block, size_t * pos, size_t limit) override; - void addToGinFilter(UInt32 rowID, const char* data, size_t length, GinFilter& gin_filter, UInt64 limit); + void addToGinFilter(UInt32 rowID, const char * data, size_t length, GinFilter & gin_filter, UInt64 limit); GinIndexStorePtr store; Names index_columns; @@ -58,21 +56,21 @@ struct MergeTreeIndexAggregatorGinFilter final : IMergeTreeIndexAggregator const GinFilterParameters params; TokenExtractorPtr token_extractor; - MergeTreeIndexGranuleGinFilterPtr granule; + MergeTreeIndexGranuleInvertedPtr granule; }; -class MergeTreeConditionGinFilter final : public IMergeTreeIndexCondition, WithContext +class MergeTreeConditionInverted final : public IMergeTreeIndexCondition, WithContext { public: - MergeTreeConditionGinFilter( + MergeTreeConditionInverted( const SelectQueryInfo & query_info, ContextPtr context, const Block & index_sample_block, const GinFilterParameters & params_, TokenExtractorPtr token_extactor_); - ~MergeTreeConditionGinFilter() override = default; + ~MergeTreeConditionInverted() override = default; bool alwaysUnknownOrTrue() const override; bool mayBeTrueOnGranule([[maybe_unused]]MergeTreeIndexGranulePtr idx_granule) const override @@ -81,7 +79,8 @@ public: assert(false); return false; } - bool mayBeTrueOnGranuleInPart(MergeTreeIndexGranulePtr idx_granule, [[maybe_unused]] PostingsCacheForStore& cache_store) const; + bool mayBeTrueOnGranuleInPart(MergeTreeIndexGranulePtr idx_granule, [[maybe_unused]] PostingsCacheForStore & cache_store) const; + private: struct KeyTuplePositionMapping { @@ -124,7 +123,7 @@ private: std::unique_ptr gin_filter; /// For FUNCTION_IN, FUNCTION_NOT_IN and FUNCTION_MULTI_SEARCH - std::vector> set_gin_filters; + std::vector set_gin_filters; /// For FUNCTION_IN and FUNCTION_NOT_IN std::vector set_key_position; @@ -154,10 +153,10 @@ private: PreparedSetsPtr prepared_sets; }; -class MergeTreeIndexGinFilter final : public IMergeTreeIndex +class MergeTreeIndexInverted final : public IMergeTreeIndex { public: - MergeTreeIndexGinFilter( + MergeTreeIndexInverted( const IndexDescription & index_, const GinFilterParameters & params_, std::unique_ptr && token_extractor_) @@ -165,13 +164,12 @@ public: , params(params_) , token_extractor(std::move(token_extractor_)) {} - ~MergeTreeIndexGinFilter() override = default; + ~MergeTreeIndexInverted() override = default; MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator() const override; - MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr &store) const override; - MergeTreeIndexConditionPtr createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const override; + MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr & store) const override; + MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override; bool mayBenefitFromIndexForIn(const ASTPtr & node) const override; diff --git a/src/Storages/MergeTree/MergeTreeIndices.cpp b/src/Storages/MergeTree/MergeTreeIndices.cpp index dc869ba6c01..2be9ecd8de3 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -105,8 +105,9 @@ MergeTreeIndexFactory::MergeTreeIndexFactory() registerCreator("annoy", annoyIndexCreator); registerValidator("annoy", annoyIndexValidator); #endif - registerCreator("inverted", ginIndexCreator); - registerValidator("inverted", ginIndexValidator); + + registerCreator("inverted", invertedIndexCreator); + registerValidator("inverted", invertedIndexValidator); } diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 52cf8c850b3..1ad6b082223 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -16,7 +17,6 @@ #include #include -#include constexpr auto INDEX_FILE_PREFIX = "skp_idx_"; @@ -237,7 +237,8 @@ void hypothesisIndexValidator(const IndexDescription & index, bool attach); MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index); void annoyIndexValidator(const IndexDescription & index, bool attach); #endif -MergeTreeIndexPtr ginIndexCreator(const IndexDescription& index); -void ginIndexValidator(const IndexDescription& index, bool attach); + +MergeTreeIndexPtr invertedIndexCreator(const IndexDescription& index); +void invertedIndexValidator(const IndexDescription& index, bool attach); } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index ae1bace79e3..6eafd8824b8 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -25,7 +26,7 @@ struct Settings; M(UInt64, min_compress_block_size, 0, "When granule is written, compress the data in buffer if the size of pending uncompressed data is larger or equal than the specified threshold. If this setting is not set, the corresponding global setting is used.", 0) \ M(UInt64, max_compress_block_size, 0, "Compress the pending uncompressed data in buffer if its size is larger or equal than the specified threshold. Block of data will be compressed even if the current granule is not finished. If this setting is not set, the corresponding global setting is used.", 0) \ M(UInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ - M(UInt64, max_digestion_size_per_segment, 1024 * 1024 * 256, "Max number of bytes to digest per segment to build GIN index.", 0) \ + M(UInt64, max_digestion_size_per_segment, 256_MiB, "Max number of bytes to digest per segment to build GIN index.", 0) \ \ /** Data storing format settings. */ \ M(UInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index a1c5474ae3a..86c0dffa60d 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -33,7 +33,6 @@ #include #include -#include #include @@ -493,7 +492,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) std::shared_ptr StorageEmbeddedRocksDB::getRocksDBStatistics() const { - std::shared_lock lock(rocksdb_ptr_mx); + std::shared_lock lock(rocksdb_ptr_mx); if (!rocksdb_ptr) return nullptr; return rocksdb_ptr->GetOptions().statistics; @@ -501,7 +500,7 @@ std::shared_ptr StorageEmbeddedRocksDB::getRocksDBStatistic std::vector StorageEmbeddedRocksDB::multiGet(const std::vector & slices_keys, std::vector & values) const { - std::shared_lock lock(rocksdb_ptr_mx); + std::shared_lock lock(rocksdb_ptr_mx); if (!rocksdb_ptr) return {}; return rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 02938fb5f69..7f6fc49fb18 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include @@ -86,7 +86,7 @@ private: const String primary_key; using RocksDBPtr = std::unique_ptr; RocksDBPtr rocksdb_ptr; - mutable std::shared_mutex rocksdb_ptr_mx; + mutable SharedMutex rocksdb_ptr_mx; String rocksdb_dir; Int32 ttl; bool read_only; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7fd4b224905..e2a2f84bc72 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -49,6 +49,7 @@ #include #include #include +#include namespace ProfileEvents diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 652063c780f..8fb2470495f 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1442,11 +1442,11 @@ void StorageWindowView::writeIntoWindowView( }); } - std::shared_lock fire_signal_lock; + std::shared_lock fire_signal_lock; QueryPipelineBuilder builder; if (window_view.is_proctime) { - fire_signal_lock = std::shared_lock(window_view.fire_signal_mutex); + fire_signal_lock = std::shared_lock(window_view.fire_signal_mutex); /// Fill ____timestamp column with current time in case of now() time column. if (window_view.is_time_column_func_now) diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 6da34389e4d..b313e466211 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -213,7 +214,7 @@ private: /// Mutex for the blocks and ready condition std::mutex mutex; - std::shared_mutex fire_signal_mutex; + SharedMutex fire_signal_mutex; mutable std::mutex sample_block_lock; /// Mutex to protect access to sample block IntervalKind::Kind window_kind; diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 3fec6b612b1..513162fb6b7 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -232,19 +232,52 @@ def need_retry(args, stdout, stderr, total_time): ) -def get_processlist(args): +def get_processlist_with_stacktraces(args): try: if args.replicated_database: return clickhouse_execute_json( args, """ - SELECT materialize((hostName(), tcpPort())) as host, * - FROM clusterAllReplicas('test_cluster_database_replicated', system.processes) - WHERE query NOT LIKE '%system.processes%' + SELECT materialize(hostName() || '::' || tcpPort()::String) as host_port, * + -- NOTE: view() here to do JOIN on shards, instead of initiator + FROM clusterAllReplicas('test_cluster_database_replicated', view( + SELECT + groupArray((s.thread_id, arrayStringConcat(arrayMap( + x -> concat(addressToLine(x), '::', demangle(addressToSymbol(x))), + s.trace), '\n') AS stacktrace + )) AS stacktraces, + p.* + FROM system.processes p + JOIN system.stack_trace s USING (query_id) + WHERE query NOT LIKE '%system.processes%' + GROUP BY p.* + )) + ORDER BY elapsed DESC """, + settings={ + "allow_introspection_functions": 1, + }, ) else: - return clickhouse_execute_json(args, "SHOW PROCESSLIST") + return clickhouse_execute_json( + args, + """ + SELECT + groupArray((s.thread_id, arrayStringConcat(arrayMap( + x -> concat(addressToLine(x), '::', demangle(addressToSymbol(x))), + s.trace), '\n') AS stacktrace + )) AS stacktraces, + p.* + FROM system.processes p + JOIN system.stack_trace s USING (query_id) + WHERE query NOT LIKE '%system.processes%' + GROUP BY p.* + ORDER BY elapsed DESC + """, + settings={ + "allow_introspection_functions": 1, + }, + ) except Exception as e: return "Failed to get processlist: " + str(e) @@ -1223,7 +1256,7 @@ class TestSuite: line = line.strip() if line and not is_shebang(line): return line - return '' + return "" def load_tags_from_file(filepath): comment_sign = get_comment_sign(filepath) @@ -1750,7 +1783,7 @@ def removesuffix(text, *suffixes): return text -def reportCoverageFor(args, what, query, permissive = False): +def reportCoverageFor(args, what, query, permissive=False): value = clickhouse_execute(args, query).decode() if value != "": @@ -1763,10 +1796,11 @@ def reportCoverageFor(args, what, query, permissive = False): def reportCoverage(args): - return reportCoverageFor( - args, - "functions", - """ + return ( + reportCoverageFor( + args, + "functions", + """ SELECT name FROM system.functions WHERE NOT is_aggregate AND origin = 'System' AND alias_to = '' @@ -1776,11 +1810,12 @@ def reportCoverage(args): ) ORDER BY name """, - True - ) and reportCoverageFor( - args, - "aggregate functions", - """ + True, + ) + and reportCoverageFor( + args, + "aggregate functions", + """ SELECT name FROM system.functions WHERE is_aggregate AND origin = 'System' AND alias_to = '' @@ -1789,11 +1824,12 @@ def reportCoverage(args): SELECT arrayJoin(used_aggregate_functions) FROM system.query_log WHERE event_date >= yesterday() ) ORDER BY name - """ - ) and reportCoverageFor( - args, - "aggregate function combinators", - """ + """, + ) + and reportCoverageFor( + args, + "aggregate function combinators", + """ SELECT name FROM system.aggregate_function_combinators WHERE NOT is_internal @@ -1802,11 +1838,12 @@ def reportCoverage(args): SELECT arrayJoin(used_aggregate_function_combinators) FROM system.query_log WHERE event_date >= yesterday() ) ORDER BY name - """ - ) and reportCoverageFor( - args, - "data type families", - """ + """, + ) + and reportCoverageFor( + args, + "data type families", + """ SELECT name FROM system.data_type_families WHERE alias_to = '' AND name NOT LIKE 'Interval%' @@ -1815,7 +1852,8 @@ def reportCoverage(args): SELECT arrayJoin(used_data_type_families) FROM system.query_log WHERE event_date >= yesterday() ) ORDER BY name - """ + """, + ) ) def reportLogStats(args): @@ -1844,7 +1882,7 @@ def reportLogStats(args): LIMIT 100 FORMAT TSVWithNamesAndTypes """ - value = clickhouse_execute(args, query).decode() + value = clickhouse_execute(args, query).decode(errors="replace") print("\nTop patterns of log messages:\n") print(value) print("\n") @@ -1856,7 +1894,7 @@ def reportLogStats(args): count() AS count, substr(replaceRegexpAll(message, '[^A-Za-z]+', ''), 1, 32) AS pattern, substr(any(message), 1, 256) as runtime_message, - any((extract(source_file, '\/[a-zA-Z0-9_]+\.[a-z]+'), source_line)) as line + any((extract(source_file, '\/[a-zA-Z0-9_]+\.[a-z]+'), source_line)) as line FROM system.text_log WHERE (now() - toIntervalMinute(mins)) < event_time AND message_format_string = '' GROUP BY pattern @@ -1864,7 +1902,7 @@ def reportLogStats(args): LIMIT 50 FORMAT TSVWithNamesAndTypes """ - value = clickhouse_execute(args, query).decode() + value = clickhouse_execute(args, query).decode(errors="replace") print("\nTop messages without format string (fmt::runtime):\n") print(value) print("\n") @@ -1891,7 +1929,9 @@ def main(args): args, "system", "processes", "is_all_data_sent" ) - if args.s3_storage and (BuildFlags.THREAD in args.build_flags or BuildFlags.DEBUG in args.build_flags): + if args.s3_storage and ( + BuildFlags.THREAD in args.build_flags or BuildFlags.DEBUG in args.build_flags + ): args.no_random_settings = True if args.skip: @@ -1963,10 +2003,9 @@ def main(args): exit_code.value = 1 if args.hung_check: - # Some queries may execute in background for some time after test was finished. This is normal. for _ in range(1, 60): - processlist = get_processlist(args) + processlist = get_processlist_with_stacktraces(args) if not processlist: break sleep(1) @@ -1980,7 +2019,6 @@ def main(args): print(json.dumps(processlist, indent=4)) print(get_transactions_list(args)) - print_stacktraces() exit_code.value = 1 else: print(colored("\nNo queries hung.", args, "green", attrs=["bold"])) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index b76ff2a2479..d5a7579df51 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -127,7 +127,7 @@ def test_backup_to_s3_multipart(): backup_destination = f"S3('http://minio1:9001/root/data/backups/multipart/{backup_name}', 'minio', 'minio123')" check_backup_and_restore(storage_policy, backup_destination, size=1000000) assert node.contains_in_log( - f"copyDataToS3: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}" + f"copyDataToS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}" ) @@ -140,7 +140,7 @@ def test_backup_to_s3_native_copy(): check_backup_and_restore(storage_policy, backup_destination) assert node.contains_in_log("using native copy") assert node.contains_in_log( - f"copyFileS3ToS3: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" + f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" ) @@ -153,7 +153,7 @@ def test_backup_to_s3_native_copy_other_bucket(): check_backup_and_restore(storage_policy, backup_destination) assert node.contains_in_log("using native copy") assert node.contains_in_log( - f"copyFileS3ToS3: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" + f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" ) @@ -164,5 +164,5 @@ def test_backup_to_s3_native_copy_multipart(): check_backup_and_restore(storage_policy, backup_destination, size=1000000) assert node.contains_in_log("using native copy") assert node.contains_in_log( - f"copyFileS3ToS3: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}/" + f"copyS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}/" ) diff --git a/tests/integration/test_replicated_merge_tree_s3_restore/test.py b/tests/integration/test_replicated_merge_tree_s3_restore/test.py index f26b3e7bd35..2181f260f32 100644 --- a/tests/integration/test_replicated_merge_tree_s3_restore/test.py +++ b/tests/integration/test_replicated_merge_tree_s3_restore/test.py @@ -131,24 +131,29 @@ def create_restore_file(node, revision=None, bucket=None, path=None, detached=No ["bash", "-c", "touch /var/lib/clickhouse/disks/s3/restore"], user="root" ) - add_restore_option = 'echo -en "{}={}\n" >> /var/lib/clickhouse/disks/s3/restore' - if revision: + num_restore_options = 0 + + def add_restore_option(key, value): + nonlocal num_restore_options + to = ">>" if num_restore_options else ">" node.exec_in_container( - ["bash", "-c", add_restore_option.format("revision", revision)], user="root" - ) - if bucket: - node.exec_in_container( - ["bash", "-c", add_restore_option.format("source_bucket", bucket)], + [ + "bash", + "-c", + f'echo -en "{key}={value}\n" {to} /var/lib/clickhouse/disks/s3/restore', + ], user="root", ) + num_restore_options += 1 + + if revision: + add_restore_option("revision", revision) + if bucket: + add_restore_option("source_bucket", bucket) if path: - node.exec_in_container( - ["bash", "-c", add_restore_option.format("source_path", path)], user="root" - ) + add_restore_option("source_path", path) if detached: - node.exec_in_container( - ["bash", "-c", add_restore_option.format("detached", "true")], user="root" - ) + add_restore_option("detached", "true") def get_revision_counter(node, backup_number): diff --git a/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.reference b/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.reference deleted file mode 100644 index 70c19fc8ced..00000000000 --- a/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.reference +++ /dev/null @@ -1,12 +0,0 @@ -210 230 20 -SELECT - sum(a), - sumCount(b).1, - sumCount(b).2 -FROM fuse_tbl ----------NOT trigger fuse-------- -210 11.5 -SELECT - sum(a), - avg(b) -FROM fuse_tbl diff --git a/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.sql b/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.sql deleted file mode 100644 index 375662eb405..00000000000 --- a/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.sql +++ /dev/null @@ -1,14 +0,0 @@ -DROP TABLE IF EXISTS fuse_tbl; -CREATE TABLE fuse_tbl(a Int8, b Int8) Engine = Log; -INSERT INTO fuse_tbl SELECT number, number + 1 FROM numbers(1, 20); - -SET optimize_syntax_fuse_functions = 1; -SET optimize_fuse_sum_count_avg = 1; - -SELECT sum(a), sum(b), count(b) from fuse_tbl; -EXPLAIN SYNTAX SELECT sum(a), sum(b), count(b) from fuse_tbl; -SELECT '---------NOT trigger fuse--------'; -SELECT sum(a), avg(b) from fuse_tbl; -EXPLAIN SYNTAX SELECT sum(a), avg(b) from fuse_tbl; - -DROP TABLE fuse_tbl; diff --git a/tests/queries/0_stateless/02244_issue_35598_fuse.reference b/tests/queries/0_stateless/02244_issue_35598_fuse.reference deleted file mode 100644 index 6ce84b402a3..00000000000 --- a/tests/queries/0_stateless/02244_issue_35598_fuse.reference +++ /dev/null @@ -1,2 +0,0 @@ -0 0 nan -0 0 nan diff --git a/tests/queries/0_stateless/02244_issue_35598_fuse.sql b/tests/queries/0_stateless/02244_issue_35598_fuse.sql deleted file mode 100644 index a590854eb6c..00000000000 --- a/tests/queries/0_stateless/02244_issue_35598_fuse.sql +++ /dev/null @@ -1,5 +0,0 @@ -SELECT sum(x), count(x), avg(x) FROM (SELECT number :: Decimal32(0) AS x FROM numbers(0)) -SETTINGS optimize_syntax_fuse_functions = 0, optimize_fuse_sum_count_avg = 0; - -SELECT sum(x), count(x), avg(x) FROM (SELECT number :: Decimal32(0) AS x FROM numbers(0)) -SETTINGS optimize_syntax_fuse_functions = 1, optimize_fuse_sum_count_avg = 1; diff --git a/tests/queries/0_stateless/02476_fuse_sum_count.sql b/tests/queries/0_stateless/02476_fuse_sum_count.sql index ee65d32d0cf..315bbd10a65 100644 --- a/tests/queries/0_stateless/02476_fuse_sum_count.sql +++ b/tests/queries/0_stateless/02476_fuse_sum_count.sql @@ -1,5 +1,5 @@ SET allow_experimental_analyzer = 1; -SET optimize_syntax_fuse_functions = 1, optimize_fuse_sum_count_avg = 1; +SET optimize_syntax_fuse_functions = 1; DROP TABLE IF EXISTS fuse_tbl; diff --git a/tests/queries/0_stateless/02477_fuse_quantiles.sql b/tests/queries/0_stateless/02477_fuse_quantiles.sql index efd861ad7f3..c0719d771d7 100644 --- a/tests/queries/0_stateless/02477_fuse_quantiles.sql +++ b/tests/queries/0_stateless/02477_fuse_quantiles.sql @@ -1,5 +1,5 @@ SET allow_experimental_analyzer = 1; -SET optimize_syntax_fuse_functions = 1, optimize_fuse_sum_count_avg = 1; +SET optimize_syntax_fuse_functions = 1; DROP TABLE IF EXISTS fuse_tbl; diff --git a/tests/queries/0_stateless/02517_fuse_bug_44712.reference b/tests/queries/0_stateless/02517_fuse_bug_44712.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02517_fuse_bug_44712.sql b/tests/queries/0_stateless/02517_fuse_bug_44712.sql deleted file mode 100644 index 894bf9e06d5..00000000000 --- a/tests/queries/0_stateless/02517_fuse_bug_44712.sql +++ /dev/null @@ -1,10 +0,0 @@ -DROP TABLE IF EXISTS fuse_tbl__fuzz_35; - -CREATE TABLE fuse_tbl__fuzz_35 (`a` UInt8, `b` Nullable(Int16)) ENGINE = Log; -INSERT INTO fuse_tbl__fuzz_35 SELECT number, number + 1 FROM numbers(1000); - -set allow_experimental_analyzer = 0, optimize_syntax_fuse_functions = 1, optimize_fuse_sum_count_avg = 1; - -SELECT quantile(0.5)(b), quantile(0.9)(b) FROM (SELECT x + 2147483648 AS b FROM (SELECT quantile(0.5)(b) AS x FROM fuse_tbl__fuzz_35) GROUP BY x) FORMAT Null; - -DROP TABLE IF EXISTS fuse_tbl__fuzz_35;