From 92e57817a21529409d659c7c759a62a82b0a3bde Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 8 May 2023 16:14:08 +0200 Subject: [PATCH 1/6] Support `dictionary` table function for `RegExpTreeDictionary` --- src/Core/Settings.h | 1 - src/Dictionaries/RegExpTreeDictionary.cpp | 53 ++++++++++++++++--- src/Dictionaries/RegExpTreeDictionary.h | 7 ++- .../ExternalDictionariesLoader.cpp | 30 ++++++++++- src/Interpreters/ExternalDictionariesLoader.h | 3 ++ .../TableFunctionDictionary.cpp | 4 +- ...4_regexp_dictionary_table_source.reference | 9 ++++ .../02504_regexp_dictionary_table_source.sql | 6 ++- 8 files changed, 94 insertions(+), 19 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ed5ad0278a2..8bcb38d5c5f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -969,7 +969,6 @@ class IColumn; M(Bool, output_format_bson_string_as_string, false, "Use BSON String type instead of Binary for String columns.", 0) \ M(Bool, input_format_bson_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format BSON.", 0) \ \ - M(Bool, regexp_dict_allow_other_sources, false, "Allow regexp_tree dictionary to use sources other than yaml source.", 0) \ M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 8263c87d0e6..79fefde23e1 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -31,6 +31,7 @@ #include +#include "Processors/Sources/BlocksListSource.h" #include "config.h" #if USE_VECTORSCAN @@ -117,6 +118,7 @@ struct RegExpTreeDictionary::RegexTreeNode { Field field; std::vector pieces; + String original_value; constexpr bool containsBackRefs() const { return !pieces.empty(); } }; @@ -208,12 +210,12 @@ void RegExpTreeDictionary::initRegexNodes(Block & block) auto string_pieces = createStringPieces(value, num_captures, regex, logger); if (!string_pieces.empty()) { - node->attributes[name_] = RegexTreeNode::AttributeValue{.field = values[j], .pieces = std::move(string_pieces)}; + node->attributes[name_] = RegexTreeNode::AttributeValue{.field = values[j], .pieces = std::move(string_pieces), .original_value = value}; } else { - Field field = parseStringToField(values[j].safeGet(), attr.type); - node->attributes[name_] = RegexTreeNode::AttributeValue{.field = std::move(field)}; + Field field = parseStringToField(value, attr.type); + node->attributes[name_] = RegexTreeNode::AttributeValue{.field = std::move(field), .original_value = value}; } } } @@ -651,6 +653,47 @@ std::unordered_map RegExpTreeDictionary::match( return result; } +Pipe RegExpTreeDictionary::read(const Names & , size_t max_block_size, size_t) const +{ + auto it = regex_nodes.begin(); + size_t block_size = 0; + BlocksList result; + for (;;) + { + Block block; + auto col_id = std::make_shared()->createColumn(); + auto col_pid = std::make_shared()->createColumn(); + auto col_regex = std::make_shared()->createColumn(); + auto col_keys = std::make_shared(std::make_shared())->createColumn(); + auto col_values = std::make_shared(std::make_shared())->createColumn(); + for (;it != regex_nodes.end() && block_size < max_block_size; it++, block_size++) + { + col_id->insert(it->first); + const auto & node = it->second; + col_pid->insert(node->parent_id); + col_regex->insert(node->regex); + std::vector keys, values; + for (const auto & [key, attr] : node->attributes) + { + keys.push_back(key); + values.push_back(attr.original_value); + } + col_keys->insert(Array(keys.begin(), keys.end())); + col_values->insert(Array(values.begin(), values.end())); + } + block.insert(ColumnWithTypeAndName(std::move(col_id),std::make_shared(),kId)); + block.insert(ColumnWithTypeAndName(std::move(col_pid),std::make_shared(),kParentId)); + block.insert(ColumnWithTypeAndName(std::move(col_regex),std::make_shared(),kRegExp)); + block.insert(ColumnWithTypeAndName(std::move(col_keys),std::make_shared(std::make_shared()),kKeys)); + block.insert(ColumnWithTypeAndName(std::move(col_values),std::make_shared(std::make_shared()),kValues)); + result.push_back(std::move(block)); + if (it == regex_nodes.end()) + break; + block_size = 0; + } + return Pipe(std::make_shared(std::move(result))); +} + Columns RegExpTreeDictionary::getColumns( const Strings & attribute_names, const DataTypes & result_types, @@ -717,10 +760,6 @@ void registerDictionaryRegExpTree(DictionaryFactory & factory) const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); - if (!context->getSettings().regexp_dict_allow_other_sources && typeid_cast(source_ptr.get()) == nullptr) - throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, - "regexp_tree dictionary doesn't accept sources other than yaml source. " - "To active it, please set regexp_dict_allow_other_sources=true"); return std::make_unique(dict_id, dict_struct, std::move(source_ptr), configuration, context->getSettings().regexp_dict_allow_hyperscan); }; diff --git a/src/Dictionaries/RegExpTreeDictionary.h b/src/Dictionaries/RegExpTreeDictionary.h index 4e8e20bba2d..893d1fb7a0b 100644 --- a/src/Dictionaries/RegExpTreeDictionary.h +++ b/src/Dictionaries/RegExpTreeDictionary.h @@ -22,6 +22,8 @@ #include #include +#include + namespace DB { @@ -91,10 +93,7 @@ public: throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary {} does not support method `hasKeys`", name); } - Pipe read(const Names &, size_t, size_t) const override - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary {} does not support method `read`", name); - } + Pipe read(const Names &, size_t, size_t) const override; ColumnPtr getColumn( const std::string & attribute_name, diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 080878c7d86..2129b3bbec6 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -4,9 +4,11 @@ #include #include #include +#include +#include +#include #include - -#include "config.h" +#include #if USE_MYSQL # include @@ -67,6 +69,30 @@ void ExternalDictionariesLoader::reloadDictionary(const std::string & dictionary loadOrReload(resolved_dictionary_name); } +ColumnsDescription ExternalDictionariesLoader::getActualTableStructure(const std::string & dictionary_name, ContextPtr query_context) const +{ + /// if it is regexp tree dictionary, we have a special table structure. + std::string resolved_name = resolveDictionaryName(dictionary_name, query_context->getCurrentDatabase()); + auto load_result = load(resolved_name); + if (load_result) + { + const auto dictionary = std::static_pointer_cast(load_result); + if (dictionary->getTypeName() == "RegExpTree") + { + return ColumnsDescription(NamesAndTypesList({ + {"id", std::make_shared()}, + {"parent_id", std::make_shared()}, + {"regexp", std::make_shared()}, + {"keys", std::make_shared(std::make_shared())}, + {"values", std::make_shared(std::make_shared())} + })); + } + } + /// otherwise, we get table structure by dictionary structure. + auto dictionary_structure = getDictionaryStructure(dictionary_name, query_context); + return ColumnsDescription(StorageDictionary::getNamesAndTypes(dictionary_structure)); +} + DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const std::string & dictionary_name, ContextPtr query_context) const { std::string resolved_name = resolveDictionaryName(dictionary_name, query_context->getCurrentDatabase()); diff --git a/src/Interpreters/ExternalDictionariesLoader.h b/src/Interpreters/ExternalDictionariesLoader.h index 659820d02ac..5635a307816 100644 --- a/src/Interpreters/ExternalDictionariesLoader.h +++ b/src/Interpreters/ExternalDictionariesLoader.h @@ -4,6 +4,7 @@ #include #include #include +#include "Storages/ColumnsDescription.h" #include @@ -29,6 +30,8 @@ public: QualifiedTableName qualifyDictionaryNameWithDatabase(const std::string & dictionary_name, ContextPtr context) const; + ColumnsDescription getActualTableStructure(const std::string & dictionary_name, ContextPtr context) const; + DictionaryStructure getDictionaryStructure(const std::string & dictionary_name, ContextPtr context) const; void assertDictionaryStructureExists(const std::string & dictionary_name, ContextPtr context) const; diff --git a/src/TableFunctions/TableFunctionDictionary.cpp b/src/TableFunctions/TableFunctionDictionary.cpp index 54c23cfb64b..5231e39dd8c 100644 --- a/src/TableFunctions/TableFunctionDictionary.cpp +++ b/src/TableFunctions/TableFunctionDictionary.cpp @@ -42,10 +42,8 @@ void TableFunctionDictionary::parseArguments(const ASTPtr & ast_function, Contex ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr context) const { const ExternalDictionariesLoader & external_loader = context->getExternalDictionariesLoader(); - auto dictionary_structure = external_loader.getDictionaryStructure(dictionary_name, context); - auto result = ColumnsDescription(StorageDictionary::getNamesAndTypes(dictionary_structure)); - return result; + return external_loader.getActualTableStructure(dictionary_name, context); } StoragePtr TableFunctionDictionary::executeImpl( diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.reference b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.reference index 28b98cfabf3..86a74291b07 100644 --- a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.reference +++ b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.reference @@ -1,3 +1,9 @@ +1 0 Linux/(\\d+[\\.\\d]*).+tlinux ['version','name'] ['\\1','TencentOS'] +2 0 (\\d+)/tclwebkit(\\d+[\\.\\d]*) ['comment','version','name'] ['test $1 and $2','$1','Andriod'] +3 2 33/tclwebkit ['version'] ['13'] +4 2 3[12]/tclwebkit ['version'] ['12'] +5 2 3[12]/tclwebkit ['version'] ['11'] +6 2 3[12]/tclwebkit ['version'] ['10'] ('TencentOS',101,'nothing') ('Andriod',13,'test 33 and 11.10') ('',NULL,'nothing') @@ -33,3 +39,6 @@ ('Andriod',43) ('Andriod',44) ('Andriod1',33,'matched 3') +1 0 (\\d+)/tclwebkit ['version','name'] ['$1','Andriod'] +2 0 33/tclwebkit ['comment','version'] ['matched 3','13'] +3 1 33/tclwebkit ['name'] ['Andriod1'] diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql index a8f1fb17a45..e7937b347a0 100644 --- a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql +++ b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql @@ -31,8 +31,9 @@ create dictionary regexp_dict1 PRIMARY KEY(regexp) SOURCE(CLICKHOUSE(QUERY concat('select * from ', currentDatabase() , '.regexp_dictionary_source_table'))) LIFETIME(0) -LAYOUT(regexp_tree) -SETTINGS(regexp_dict_allow_other_sources = true); +LAYOUT(regexp_tree); + +select * from dictionary(regexp_dict1); select dictGet('regexp_dict1', ('name', 'version', 'comment'), 'Linux/101.tlinux'); select dictGet('regexp_dict1', ('name', 'version', 'comment'), '33/tclwebkit11.10x'); @@ -79,6 +80,7 @@ select dictGet(regexp_dict1, ('name', 'version', 'comment'), '33/tclwebkit'); truncate table regexp_dictionary_source_table; SYSTEM RELOAD dictionary regexp_dict1; -- { serverError 489 } +select * from dictionary(regexp_dict1); DROP TABLE IF EXISTS regexp_dictionary_source_table; DROP TABLE IF EXISTS needle_table; From 71c96ceb6138512a2cd303ee196a9aed2cd18526 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 8 May 2023 16:21:42 +0200 Subject: [PATCH 2/6] Update src/Interpreters/ExternalDictionariesLoader.h --- src/Interpreters/ExternalDictionariesLoader.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExternalDictionariesLoader.h b/src/Interpreters/ExternalDictionariesLoader.h index 5635a307816..717aefddd8f 100644 --- a/src/Interpreters/ExternalDictionariesLoader.h +++ b/src/Interpreters/ExternalDictionariesLoader.h @@ -4,7 +4,7 @@ #include #include #include -#include "Storages/ColumnsDescription.h" +#include #include From ddce47f79ed74bf07aee4bfc91598a4617167dee Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 9 May 2023 20:17:54 +0200 Subject: [PATCH 3/6] refine table source for regexp tree dictionary --- .../ClickHouseDictionarySource.cpp | 14 ++++----- src/Dictionaries/ClickHouseDictionarySource.h | 4 +-- src/Dictionaries/ExternalQueryBuilder.h | 10 +++++-- src/Dictionaries/RegExpTreeDictionary.cpp | 30 ++++++++++++++++++- .../02504_regexp_dictionary_table_source.sql | 2 +- 5 files changed, 47 insertions(+), 13 deletions(-) diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 05cc03b6ced..65147ee664e 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -71,11 +71,11 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( : update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , configuration{configuration_} - , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks} + , query_builder(std::make_shared(dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks)) , sample_block{sample_block_} , context(context_) , pool{createPool(configuration)} - , load_all_query{query_builder.composeLoadAllQuery()} + , load_all_query{query_builder->composeLoadAllQuery()} { } @@ -84,7 +84,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionar , dict_struct{other.dict_struct} , configuration{other.configuration} , invalidate_query_response{other.invalidate_query_response} - , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks} + , query_builder(std::make_shared(dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks)) , sample_block{other.sample_block} , context(Context::createCopy(other.context)) , pool{createPool(configuration)} @@ -99,12 +99,12 @@ std::string ClickHouseDictionarySource::getUpdateFieldAndDate() time_t hr_time = std::chrono::system_clock::to_time_t(update_time) - configuration.update_lag; std::string str_time = DateLUT::instance().timeToString(hr_time); update_time = std::chrono::system_clock::now(); - return query_builder.composeUpdateQuery(configuration.update_field, str_time); + return query_builder->composeUpdateQuery(configuration.update_field, str_time); } else { update_time = std::chrono::system_clock::now(); - return query_builder.composeLoadAllQuery(); + return query_builder->composeLoadAllQuery(); } } @@ -121,13 +121,13 @@ QueryPipeline ClickHouseDictionarySource::loadUpdatedAll() QueryPipeline ClickHouseDictionarySource::loadIds(const std::vector & ids) { - return createStreamForQuery(query_builder.composeLoadIdsQuery(ids)); + return createStreamForQuery(query_builder->composeLoadIdsQuery(ids)); } QueryPipeline ClickHouseDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { - String query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::IN_WITH_TUPLES); + String query = query_builder->composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::IN_WITH_TUPLES); return createStreamForQuery(query); } diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index f37fae389ac..124d4c8db3f 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -78,11 +78,11 @@ private: const DictionaryStructure dict_struct; const Configuration configuration; mutable std::string invalidate_query_response; - ExternalQueryBuilder query_builder; + ExternalQueryBuilderPtr query_builder; Block sample_block; ContextMutablePtr context; ConnectionPoolWithFailoverPtr pool; - const std::string load_all_query; + std::string load_all_query; Poco::Logger * log = &Poco::Logger::get("ClickHouseDictionarySource"); /// RegExpTreeDictionary is the only dictionary whose structure of attributions differ from the input block. diff --git a/src/Dictionaries/ExternalQueryBuilder.h b/src/Dictionaries/ExternalQueryBuilder.h index 353367b12b6..05979d1fb86 100644 --- a/src/Dictionaries/ExternalQueryBuilder.h +++ b/src/Dictionaries/ExternalQueryBuilder.h @@ -36,6 +36,10 @@ struct ExternalQueryBuilder const std::string & where_, IdentifierQuotingStyle quoting_style_); + ExternalQueryBuilder(const ExternalQueryBuilder &) = default; + + virtual ~ExternalQueryBuilder() = default; + /** Generate a query to load all data. */ std::string composeLoadAllQuery() const; @@ -61,10 +65,10 @@ struct ExternalQueryBuilder std::string composeLoadKeysQuery(const Columns & key_columns, const std::vector & requested_rows, LoadKeysMethod method, size_t partition_key_prefix = 0) const; -private: +protected: const FormatSettings format_settings = {}; - void composeLoadAllQuery(WriteBuffer & out) const; + virtual void composeLoadAllQuery(WriteBuffer & out) const; /// In the following methods `beg` and `end` specifies which columns to write in expression @@ -93,4 +97,6 @@ private: void writeQuoted(const std::string & s, WriteBuffer & out) const; }; +using ExternalQueryBuilderPtr = std::shared_ptr; + } diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 68f21087f0c..a0c66f42e8b 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -31,7 +32,6 @@ #include -#include "Processors/Sources/BlocksListSource.h" #include "config.h" #if USE_VECTORSCAN @@ -87,6 +87,32 @@ namespace } } +struct ExternalRegexpQueryBuilder final : public ExternalQueryBuilder +{ + explicit ExternalRegexpQueryBuilder(const ExternalQueryBuilder & builder) : ExternalQueryBuilder(builder) {} + + void composeLoadAllQuery(WriteBuffer & out) const override + { + writeString("SELECT id, parent_id, regexp, keys, values FROM ", out); + if (!db.empty()) + { + writeQuoted(db, out); + writeChar('.', out); + } + if (!schema.empty()) + { + writeQuoted(schema, out); + writeChar('.', out); + } + writeQuoted(table, out); + if (!where.empty()) + { + writeString(" WHERE ", out); + writeString(where, out); + } + } +}; + struct RegExpTreeDictionary::RegexTreeNode { std::vector children; @@ -385,6 +411,8 @@ RegExpTreeDictionary::RegExpTreeDictionary( sample_block.insert(ColumnWithTypeAndName(std::make_shared(std::make_shared()), kKeys)); sample_block.insert(ColumnWithTypeAndName(std::make_shared(std::make_shared()), kValues)); ch_source->sample_block = std::move(sample_block); + ch_source->query_builder = std::make_shared(*ch_source->query_builder); + ch_source->load_all_query = ch_source->query_builder->composeLoadAllQuery(); } loadData(); diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql index e7937b347a0..15e8adce403 100644 --- a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql +++ b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql @@ -29,7 +29,7 @@ create dictionary regexp_dict1 comment String default 'nothing' ) PRIMARY KEY(regexp) -SOURCE(CLICKHOUSE(QUERY concat('select * from ', currentDatabase() , '.regexp_dictionary_source_table'))) +SOURCE(CLICKHOUSE(TABLE 'regexp_dictionary_source_table')) LIFETIME(0) LAYOUT(regexp_tree); From 2273884ded830d1bac328c58724128d61579fb91 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 10 May 2023 17:42:08 +0200 Subject: [PATCH 4/6] address comments --- .../ExternalDictionariesLoader.cpp | 32 +++---------------- src/Interpreters/ExternalDictionariesLoader.h | 4 +-- .../TableFunctionDictionary.cpp | 31 +++++++++++++++++- 3 files changed, 35 insertions(+), 32 deletions(-) diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 2129b3bbec6..21613806788 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -4,11 +4,11 @@ #include #include #include -#include -#include -#include #include -#include +#include + +#include "Storages/ColumnsDescription.h" +#include "config.h" #if USE_MYSQL # include @@ -69,30 +69,6 @@ void ExternalDictionariesLoader::reloadDictionary(const std::string & dictionary loadOrReload(resolved_dictionary_name); } -ColumnsDescription ExternalDictionariesLoader::getActualTableStructure(const std::string & dictionary_name, ContextPtr query_context) const -{ - /// if it is regexp tree dictionary, we have a special table structure. - std::string resolved_name = resolveDictionaryName(dictionary_name, query_context->getCurrentDatabase()); - auto load_result = load(resolved_name); - if (load_result) - { - const auto dictionary = std::static_pointer_cast(load_result); - if (dictionary->getTypeName() == "RegExpTree") - { - return ColumnsDescription(NamesAndTypesList({ - {"id", std::make_shared()}, - {"parent_id", std::make_shared()}, - {"regexp", std::make_shared()}, - {"keys", std::make_shared(std::make_shared())}, - {"values", std::make_shared(std::make_shared())} - })); - } - } - /// otherwise, we get table structure by dictionary structure. - auto dictionary_structure = getDictionaryStructure(dictionary_name, query_context); - return ColumnsDescription(StorageDictionary::getNamesAndTypes(dictionary_structure)); -} - DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const std::string & dictionary_name, ContextPtr query_context) const { std::string resolved_name = resolveDictionaryName(dictionary_name, query_context->getCurrentDatabase()); diff --git a/src/Interpreters/ExternalDictionariesLoader.h b/src/Interpreters/ExternalDictionariesLoader.h index 717aefddd8f..5f27d695c8b 100644 --- a/src/Interpreters/ExternalDictionariesLoader.h +++ b/src/Interpreters/ExternalDictionariesLoader.h @@ -4,7 +4,6 @@ #include #include #include -#include #include @@ -30,8 +29,6 @@ public: QualifiedTableName qualifyDictionaryNameWithDatabase(const std::string & dictionary_name, ContextPtr context) const; - ColumnsDescription getActualTableStructure(const std::string & dictionary_name, ContextPtr context) const; - DictionaryStructure getDictionaryStructure(const std::string & dictionary_name, ContextPtr context) const; void assertDictionaryStructureExists(const std::string & dictionary_name, ContextPtr context) const; @@ -53,6 +50,7 @@ protected: friend class StorageSystemDictionaries; friend class DatabaseDictionary; + friend class TableFunctionDictionary; }; } diff --git a/src/TableFunctions/TableFunctionDictionary.cpp b/src/TableFunctions/TableFunctionDictionary.cpp index 5231e39dd8c..90db9550a72 100644 --- a/src/TableFunctions/TableFunctionDictionary.cpp +++ b/src/TableFunctions/TableFunctionDictionary.cpp @@ -2,6 +2,10 @@ #include +#include +#include +#include + #include #include #include @@ -42,8 +46,33 @@ void TableFunctionDictionary::parseArguments(const ASTPtr & ast_function, Contex ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr context) const { const ExternalDictionariesLoader & external_loader = context->getExternalDictionariesLoader(); + std::string resolved_name = external_loader.resolveDictionaryName(dictionary_name, context->getCurrentDatabase()); + auto load_result = external_loader.load(resolved_name); + if (load_result) + { + /// for regexp tree dictionary, the table structure will be different with dictionary structure. it is: + /// - id. identifier of the tree node + /// - parent_id. + /// - regexp. the regular expression + /// - keys. the names of attributions of dictionary structure + /// - values. the values of each attribution + const auto dictionary = std::static_pointer_cast(load_result); + if (dictionary->getTypeName() == "RegExpTree") + { + return ColumnsDescription(NamesAndTypesList({ + {"id", std::make_shared()}, + {"parent_id", std::make_shared()}, + {"regexp", std::make_shared()}, + {"keys", std::make_shared(std::make_shared())}, + {"values", std::make_shared(std::make_shared())} + })); + } + } + + /// otherwise, we get table structure by dictionary structure. + auto dictionary_structure = external_loader.getDictionaryStructure(dictionary_name, context); + return ColumnsDescription(StorageDictionary::getNamesAndTypes(dictionary_structure)); - return external_loader.getActualTableStructure(dictionary_name, context); } StoragePtr TableFunctionDictionary::executeImpl( From 6b3160f7ceead041824ff82d355e74db086c5f06 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 11 May 2023 13:49:02 +0200 Subject: [PATCH 5/6] trivial change --- src/Interpreters/ExternalDictionariesLoader.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 21613806788..080878c7d86 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -5,9 +5,7 @@ #include #include #include -#include -#include "Storages/ColumnsDescription.h" #include "config.h" #if USE_MYSQL From ef74e64336db12cd0a623488f21d653b0df9e2aa Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 11 May 2023 22:18:08 +0200 Subject: [PATCH 6/6] address comments --- docs/en/sql-reference/dictionaries/index.md | 2 -- src/Dictionaries/RegExpTreeDictionary.cpp | 5 +++++ src/Dictionaries/RegExpTreeDictionary.h | 2 +- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 74ab7e3c948..5801b7866cb 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2218,8 +2218,6 @@ LAYOUT(regexp_tree) ... ``` -We only allow `YAMLRegExpTree` to work with regexp_tree dicitionary layout. If you want to use other sources, please set variable `regexp_dict_allow_other_sources` true. - **Source** We introduce a type of source called `YAMLRegExpTree` representing the structure of Regexp Tree dictionary. An Example of a valid yaml config is like: diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index a0c66f42e8b..9841cadcdca 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -683,9 +683,11 @@ std::unordered_map RegExpTreeDictionary::match( Pipe RegExpTreeDictionary::read(const Names & , size_t max_block_size, size_t) const { + auto it = regex_nodes.begin(); size_t block_size = 0; BlocksList result; + for (;;) { Block block; @@ -694,6 +696,7 @@ Pipe RegExpTreeDictionary::read(const Names & , size_t max_block_size, size_t) c auto col_regex = std::make_shared()->createColumn(); auto col_keys = std::make_shared(std::make_shared())->createColumn(); auto col_values = std::make_shared(std::make_shared())->createColumn(); + for (;it != regex_nodes.end() && block_size < max_block_size; it++, block_size++) { col_id->insert(it->first); @@ -709,6 +712,7 @@ Pipe RegExpTreeDictionary::read(const Names & , size_t max_block_size, size_t) c col_keys->insert(Array(keys.begin(), keys.end())); col_values->insert(Array(values.begin(), values.end())); } + block.insert(ColumnWithTypeAndName(std::move(col_id),std::make_shared(),kId)); block.insert(ColumnWithTypeAndName(std::move(col_pid),std::make_shared(),kParentId)); block.insert(ColumnWithTypeAndName(std::move(col_regex),std::make_shared(),kRegExp)); @@ -719,6 +723,7 @@ Pipe RegExpTreeDictionary::read(const Names & , size_t max_block_size, size_t) c break; block_size = 0; } + return Pipe(std::make_shared(std::move(result))); } diff --git a/src/Dictionaries/RegExpTreeDictionary.h b/src/Dictionaries/RegExpTreeDictionary.h index 893d1fb7a0b..683588e688f 100644 --- a/src/Dictionaries/RegExpTreeDictionary.h +++ b/src/Dictionaries/RegExpTreeDictionary.h @@ -93,7 +93,7 @@ public: throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary {} does not support method `hasKeys`", name); } - Pipe read(const Names &, size_t, size_t) const override; + Pipe read(const Names & columns, size_t max_block_size, size_t num_streams) const override; ColumnPtr getColumn( const std::string & attribute_name,