diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 58b4ad72a1b..df30f9e6306 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 95c2e0a3e09..4a446b6d1af 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 945e1c55494..b996c3a199a 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 3c955ffbcf0..ac59775a755 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -121,25 +121,26 @@ public: 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 @@ public: 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 @@ public: + ", 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 00000000000..608f9904d26 --- /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 00000000000..cd4feaf5709 --- /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;