From 7c8deae0e1eac1abdee2a266a86b5d7b7daf7685 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 29 Jan 2021 15:44:56 +0300 Subject: [PATCH 1/3] Fix starting the server with tables having columns' default expressions containing dictGet(). Allow getting return type of dictGet() without loading dictionary. --- src/Databases/DatabaseOrdinary.cpp | 33 +++++++++++------- src/Dictionaries/DictionaryStructure.cpp | 11 ++++-- src/Dictionaries/DictionaryStructure.h | 3 +- src/Functions/FunctionsExternalDictionaries.h | 34 ++++++++----------- ...76_dictget_in_default_expression.reference | 11 ++++++ .../01676_dictget_in_default_expression.sql | 31 +++++++++++++++++ 6 files changed, 87 insertions(+), 36 deletions(-) create mode 100644 tests/queries/0_stateless/01676_dictget_in_default_expression.reference create mode 100644 tests/queries/0_stateless/01676_dictget_in_default_expression.sql diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 58b4ad72a1b8..df30f9e6306c 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -172,6 +172,26 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto ThreadPool pool; + /// We must attach dictionaries before attaching tables + /// because while we're attaching tables we may need to have some dictionaries attached + /// (for example, dictionaries can be used in the default expressions for some tables). + /// On the other hand we can attach any dictionary (even sourced from ClickHouse table) + /// without having any tables attached. It is so because attaching of a dictionary means + /// loading of its config only, it doesn't involve loading the dictionary itself. + + /// Attach dictionaries. + for (const auto & [name, query] : file_names) + { + auto create_query = query->as(); + if (create_query.is_dictionary) + { + tryAttachDictionary(query, *this, getMetadataPath() + name, context); + + /// Messages, so that it's not boring to wait for the server to load for a long time. + logAboutProgress(log, ++dictionaries_processed, total_dictionaries, watch); + } + } + /// Attach tables. for (const auto & name_with_query : file_names) { @@ -196,19 +216,6 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto /// After all tables was basically initialized, startup them. startupTables(pool); - - /// Attach dictionaries. - for (const auto & [name, query] : file_names) - { - auto create_query = query->as(); - if (create_query.is_dictionary) - { - tryAttachDictionary(query, *this, getMetadataPath() + name, context); - - /// Messages, so that it's not boring to wait for the server to load for a long time. - logAboutProgress(log, ++dictionaries_processed, total_dictionaries, watch); - } - } } diff --git a/src/Dictionaries/DictionaryStructure.cpp b/src/Dictionaries/DictionaryStructure.cpp index 95c2e0a3e09e..4a446b6d1afb 100644 --- a/src/Dictionaries/DictionaryStructure.cpp +++ b/src/Dictionaries/DictionaryStructure.cpp @@ -218,7 +218,7 @@ void DictionaryStructure::validateKeyTypes(const DataTypes & key_types) const } } -const DictionaryAttribute & DictionaryStructure::getAttribute(const String& attribute_name, const DataTypePtr & type) const +const DictionaryAttribute & DictionaryStructure::getAttribute(const String & attribute_name) const { auto find_iter = std::find_if(attributes.begin(), attributes.end(), [&](const auto & attribute) { return attribute.name == attribute_name; }); @@ -226,13 +226,18 @@ const DictionaryAttribute & DictionaryStructure::getAttribute(const String& attr if (find_iter == attributes.end()) throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; - const auto & attribute = *find_iter; + return *find_iter; +} + +const DictionaryAttribute & DictionaryStructure::getAttribute(const String& attribute_name, const DataTypePtr & type) const +{ + const auto & attribute = getAttribute(attribute_name); if (!areTypesEqual(attribute.type, type)) throw Exception{"Attribute type does not match, expected " + attribute.type->getName() + ", found " + type->getName(), ErrorCodes::TYPE_MISMATCH}; - return *find_iter; + return attribute; } std::string DictionaryStructure::getKeyDescription() const diff --git a/src/Dictionaries/DictionaryStructure.h b/src/Dictionaries/DictionaryStructure.h index 945e1c55494f..b996c3a199a1 100644 --- a/src/Dictionaries/DictionaryStructure.h +++ b/src/Dictionaries/DictionaryStructure.h @@ -154,7 +154,8 @@ struct DictionaryStructure final DictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); void validateKeyTypes(const DataTypes & key_types) const; - const DictionaryAttribute &getAttribute(const String& attribute_name, const DataTypePtr & type) const; + const DictionaryAttribute & getAttribute(const String & attribute_name) const; + const DictionaryAttribute & getAttribute(const String & attribute_name, const DataTypePtr & type) const; std::string getKeyDescription() const; bool isKeySizeFixed() const; size_t getKeySize() const; diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 3c955ffbcf08..ac59775a7552 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -121,25 +121,26 @@ class FunctionDictHelper return getDictionary(dict_name_col->getValue())->isInjective(attr_name_col->getValue()); } - DictionaryAttribute getDictionaryAttribute(std::shared_ptr dictionary, const String& attribute_name) const + DictionaryStructure getDictionaryStructure(const String & dictionary_name) const { - const DictionaryStructure & structure = dictionary->getStructure(); - - auto find_iter = std::find_if(structure.attributes.begin(), structure.attributes.end(), [&](const auto &attribute) - { - return attribute.name == attribute_name; - }); - - if (find_iter == structure.attributes.end()) - throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; - - return *find_iter; + String resolved_name = DatabaseCatalog::instance().resolveDictionaryName(dictionary_name); + auto load_result = external_loader.getLoadResult(resolved_name); + if (!load_result.config) + throw Exception("Dictionary " + backQuote(dictionary_name) + " not found", ErrorCodes::BAD_ARGUMENTS); + return ExternalDictionariesLoader::getDictionaryStructure(*load_result.config); } + private: const Context & context; const ExternalDictionariesLoader & external_loader; /// Access cannot be not granted, since in this case checkAccess() will throw and access_checked will not be updated. std::atomic access_checked = false; + + /// We must not cache dictionary or dictionary's structure here, because there are places + /// where ExpressionActionsPtr is cached (StorageDistributed caching it for sharding_key_expr and + /// optimize_skip_unused_shards), and if the dictionary will be cached within "query" then + /// cached ExpressionActionsPtr will always have first version of the query and the dictionary + /// will not be updated after reload (see https://github.com/ClickHouse/ClickHouse/pull/16205) }; @@ -267,10 +268,7 @@ class FunctionDictGetNoType final : public IFunction if (arguments.size() < 3) throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - /// TODO: We can load only dictionary structure - String dictionary_name; - if (const auto * name_col = checkAndGetColumnConst(arguments[0].column.get())) dictionary_name = name_col->getValue(); else @@ -278,16 +276,14 @@ class FunctionDictGetNoType final : public IFunction + ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; String attribute_name; - if (const auto * name_col = checkAndGetColumnConst(arguments[1].column.get())) attribute_name = name_col->getValue(); else throw Exception{"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - auto dictionary = helper.getDictionary(dictionary_name); - - return helper.getDictionaryAttribute(dictionary, attribute_name).type; + /// We're extracting the return type from the dictionary's config, without loading the dictionary. + return helper.getDictionaryStructure(dictionary_name).getAttribute(attribute_name).type; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override diff --git a/tests/queries/0_stateless/01676_dictget_in_default_expression.reference b/tests/queries/0_stateless/01676_dictget_in_default_expression.reference new file mode 100644 index 000000000000..608f9904d267 --- /dev/null +++ b/tests/queries/0_stateless/01676_dictget_in_default_expression.reference @@ -0,0 +1,11 @@ +2 20 +3 15 +status: +LOADED +status_after_detach_and_attach: +NOT_LOADED +2 20 +3 15 +4 40 +status: +LOADED diff --git a/tests/queries/0_stateless/01676_dictget_in_default_expression.sql b/tests/queries/0_stateless/01676_dictget_in_default_expression.sql new file mode 100644 index 000000000000..cd4feaf57099 --- /dev/null +++ b/tests/queries/0_stateless/01676_dictget_in_default_expression.sql @@ -0,0 +1,31 @@ +DROP DATABASE IF EXISTS test_01676 SYNC; + +CREATE DATABASE test_01676; + +CREATE TABLE test_01676.dict_data (key UInt64, value UInt64) ENGINE=MergeTree ORDER BY tuple(); +INSERT INTO test_01676.dict_data VALUES (2,20), (3,30), (4,40), (5,50); + +CREATE DICTIONARY test_01676.dict (key UInt64, value UInt64) PRIMARY KEY key SOURCE(CLICKHOUSE(DB 'test_01676' TABLE 'dict_data' HOST '127.0.0.1' PORT tcpPort())) LIFETIME(0) LAYOUT(HASHED()); + +CREATE TABLE test_01676.table (x UInt64, y UInt64 DEFAULT dictGet('test_01676.dict', 'value', x)) ENGINE=MergeTree ORDER BY tuple(); +INSERT INTO test_01676.table (x) VALUES (2); +INSERT INTO test_01676.table VALUES (toUInt64(3), toUInt64(15)); + +SELECT * FROM test_01676.table ORDER BY x; + +SELECT 'status:'; +SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict'; + +DETACH DATABASE test_01676; +ATTACH DATABASE test_01676; + +SELECT 'status_after_detach_and_attach:'; +SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict'; + +INSERT INTO test_01676.table (x) VALUES (toInt64(4)); +SELECT * FROM test_01676.table ORDER BY x; + +SELECT 'status:'; +SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict'; + +DROP DATABASE test_01676; From 95177dc0dec4abbee4ee2e2fee6b5badc9c0d9a4 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 30 Jan 2021 02:25:28 +0300 Subject: [PATCH 2/3] Fix access to the key as an attribute in IPAddressDictionary. --- src/Dictionaries/DictionaryFactory.cpp | 2 +- src/Dictionaries/DictionaryStructure.cpp | 37 ++++++++----- src/Dictionaries/DictionaryStructure.h | 1 + src/Dictionaries/IPAddressDictionary.cpp | 52 ++++++++----------- src/Dictionaries/IPAddressDictionary.h | 6 +-- .../ExternalDictionariesLoader.cpp | 2 +- 6 files changed, 50 insertions(+), 50 deletions(-) diff --git a/src/Dictionaries/DictionaryFactory.cpp b/src/Dictionaries/DictionaryFactory.cpp index a889b63107ff..0ab7d1991861 100644 --- a/src/Dictionaries/DictionaryFactory.cpp +++ b/src/Dictionaries/DictionaryFactory.cpp @@ -41,7 +41,7 @@ DictionaryPtr DictionaryFactory::create( throw Exception{name + ": element dictionary.layout should have exactly one child element", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG}; - const DictionaryStructure dict_struct{config, config_prefix + ".structure"}; + const DictionaryStructure dict_struct{config, config_prefix}; DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create( name, config, config_prefix + ".source", dict_struct, context, config.getString(config_prefix + ".database", ""), check_source_config); diff --git a/src/Dictionaries/DictionaryStructure.cpp b/src/Dictionaries/DictionaryStructure.cpp index 4a446b6d1afb..3b2afdb1beac 100644 --- a/src/Dictionaries/DictionaryStructure.cpp +++ b/src/Dictionaries/DictionaryStructure.cpp @@ -135,17 +135,19 @@ DictionarySpecialAttribute::DictionarySpecialAttribute(const Poco::Util::Abstrac DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { - const auto has_id = config.has(config_prefix + ".id"); - const auto has_key = config.has(config_prefix + ".key"); + std::string structure_prefix = config_prefix + ".structure"; + + const auto has_id = config.has(structure_prefix + ".id"); + const auto has_key = config.has(structure_prefix + ".key"); if (has_key && has_id) throw Exception{"Only one of 'id' and 'key' should be specified", ErrorCodes::BAD_ARGUMENTS}; if (has_id) - id.emplace(config, config_prefix + ".id"); + id.emplace(config, structure_prefix + ".id"); else if (has_key) { - key.emplace(getAttributes(config, config_prefix + ".key", false, false)); + key.emplace(getAttributes(config, structure_prefix + ".key", false, false)); if (key->empty()) throw Exception{"Empty 'key' supplied", ErrorCodes::BAD_ARGUMENTS}; } @@ -158,11 +160,11 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration throw Exception{"'id' cannot be empty", ErrorCodes::BAD_ARGUMENTS}; const char * range_default_type = "Date"; - if (config.has(config_prefix + ".range_min")) - range_min.emplace(makeDictionaryTypedSpecialAttribute(config, config_prefix + ".range_min", range_default_type)); + if (config.has(structure_prefix + ".range_min")) + range_min.emplace(makeDictionaryTypedSpecialAttribute(config, structure_prefix + ".range_min", range_default_type)); - if (config.has(config_prefix + ".range_max")) - range_max.emplace(makeDictionaryTypedSpecialAttribute(config, config_prefix + ".range_max", range_default_type)); + if (config.has(structure_prefix + ".range_max")) + range_max.emplace(makeDictionaryTypedSpecialAttribute(config, structure_prefix + ".range_max", range_default_type)); if (range_min.has_value() != range_max.has_value()) { @@ -194,10 +196,13 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration has_expressions = true; } - attributes = getAttributes(config, config_prefix); + attributes = getAttributes(config, structure_prefix); if (attributes.empty()) throw Exception{"Dictionary has no attributes defined", ErrorCodes::BAD_ARGUMENTS}; + + if (config.getBool(config_prefix + ".layout.ip_trie.access_to_key_from_attributes", false)) + access_to_key_from_attributes = true; } @@ -222,14 +227,20 @@ const DictionaryAttribute & DictionaryStructure::getAttribute(const String & att { auto find_iter = std::find_if(attributes.begin(), attributes.end(), [&](const auto & attribute) { return attribute.name == attribute_name; }); + if (find_iter != attributes.end()) + return *find_iter; - if (find_iter == attributes.end()) - throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; + if (key && access_to_key_from_attributes) + { + find_iter = std::find_if(key->begin(), key->end(), [&](const auto & attribute) { return attribute.name == attribute_name; }); + if (find_iter != key->end()) + return *find_iter; + } - return *find_iter; + throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; } -const DictionaryAttribute & DictionaryStructure::getAttribute(const String& attribute_name, const DataTypePtr & type) const +const DictionaryAttribute & DictionaryStructure::getAttribute(const String & attribute_name, const DataTypePtr & type) const { const auto & attribute = getAttribute(attribute_name); diff --git a/src/Dictionaries/DictionaryStructure.h b/src/Dictionaries/DictionaryStructure.h index b996c3a199a1..ad3333792aa4 100644 --- a/src/Dictionaries/DictionaryStructure.h +++ b/src/Dictionaries/DictionaryStructure.h @@ -150,6 +150,7 @@ struct DictionaryStructure final std::optional range_min; std::optional range_max; bool has_expressions = false; + bool access_to_key_from_attributes = false; DictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index fa9f88991429..6447c76ee73e 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -247,21 +247,15 @@ IPAddressDictionary::IPAddressDictionary( const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, - bool require_nonempty_, - bool access_to_key_from_attributes_) + bool require_nonempty_) : IDictionaryBase(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) , require_nonempty(require_nonempty_) - , access_to_key_from_attributes(access_to_key_from_attributes_) + , access_to_key_from_attributes(dict_struct_.access_to_key_from_attributes) , logger(&Poco::Logger::get("IPAddressDictionary")) { - if (access_to_key_from_attributes) - { - dict_struct.attributes.emplace_back(dict_struct.key->front()); - } - createAttributes(); loadData(); @@ -367,18 +361,23 @@ ColumnUInt8::Ptr IPAddressDictionary::hasKeys(const Columns & key_columns, const void IPAddressDictionary::createAttributes() { - const auto size = dict_struct.attributes.size(); - attributes.reserve(size); - - for (const auto & attribute : dict_struct.attributes) + auto create_attributes_from_dictionary_attributes = [this](const std::vector & dict_attrs) { - attribute_index_by_name.emplace(attribute.name, attributes.size()); - attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value)); + attributes.reserve(attributes.size() + dict_attrs.size()); + for (const auto & attribute : dict_attrs) + { + attribute_index_by_name.emplace(attribute.name, attributes.size()); + attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value)); - if (attribute.hierarchical) - throw Exception{full_name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(), - ErrorCodes::TYPE_MISMATCH}; - } + if (attribute.hierarchical) + throw Exception{full_name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(), + ErrorCodes::TYPE_MISMATCH}; + } + }; + + create_attributes_from_dictionary_attributes(dict_struct.attributes); + if (access_to_key_from_attributes) + create_attributes_from_dictionary_attributes(*dict_struct.key); } void IPAddressDictionary::loadData() @@ -396,19 +395,13 @@ void IPAddressDictionary::loadData() element_count += rows; const ColumnPtr key_column_ptr = block.safeGetByPosition(0).column; - - size_t attributes_size = dict_struct.attributes.size(); - if (access_to_key_from_attributes) - { - /// last attribute contains key and will be filled in code below - attributes_size--; - } - const auto attribute_column_ptrs = ext::map(ext::range(0, attributes_size), + const auto attribute_column_ptrs = ext::map( + ext::range(0, dict_struct.attributes.size()), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx + 1).column; }); for (const auto row : ext::range(0, rows)) { - for (const auto attribute_idx : ext::range(0, attribute_column_ptrs.size())) + for (const auto attribute_idx : ext::range(0, dict_struct.attributes.size())) { const auto & attribute_column = *attribute_column_ptrs[attribute_idx]; auto & attribute = attributes[attribute_idx]; @@ -991,11 +984,8 @@ void registerDictionaryTrie(DictionaryFactory & factory) const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - const auto & layout_prefix = config_prefix + ".layout.ip_trie"; - const bool access_to_key_from_attributes = config.getBool(layout_prefix + ".access_to_key_from_attributes", false); // This is specialised dictionary for storing IPv4 and IPv6 prefixes. - return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, - require_nonempty, access_to_key_from_attributes); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); }; factory.registerLayout("ip_trie", create_layout, true); } diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index 629a850267cd..6c5cfa765e86 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -28,8 +28,7 @@ class IPAddressDictionary final : public IDictionaryBase const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, - bool require_nonempty_, - bool access_to_key_from_attributes_); + bool require_nonempty_); std::string getKeyDescription() const { return key_description; } @@ -47,8 +46,7 @@ class IPAddressDictionary final : public IDictionaryBase std::shared_ptr clone() const override { - return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, - require_nonempty, access_to_key_from_attributes); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty); } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index c735dd76911b..4df4e5f8c1be 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -38,7 +38,7 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create( DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config) { - return {config, key_in_config + ".structure"}; + return {config, key_in_config}; } DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const ObjectConfig & config) From cbd4bace6f265561902455aa18a9a894c43e6acc Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 30 Jan 2021 21:26:08 +0300 Subject: [PATCH 3/3] Mark that test 01676_dictget_in_default_expression should not be run in parallel. --- tests/queries/skip_list.json | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 0ec0f760d8d3..600a2fd60282 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -566,6 +566,7 @@ "01602_show_create_view", "01603_rename_overwrite_bug", "01646_system_restart_replicas_smoke", // system restart replicas is a global query + "01676_dictget_in_default_expression", "attach", "ddl_dictionaries", "dictionary",