Fix starting the server with tables having columns' default expressions containing dictGet().

Allow getting return type of dictGet() without loading dictionary.
This commit is contained in:
Vitaly Baranov 2021-01-29 15:44:56 +03:00
parent 742b740753
commit 7c8deae0e1
6 changed files with 87 additions and 36 deletions

View File

@ -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<const ASTCreateQuery &>();
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<const ASTCreateQuery &>();
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);
}
}
}

View File

@ -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

View File

@ -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;

View File

@ -121,25 +121,26 @@ public:
return getDictionary(dict_name_col->getValue<String>())->isInjective(attr_name_col->getValue<String>());
}
DictionaryAttribute getDictionaryAttribute(std::shared_ptr<const IDictionaryBase> 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<bool> 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<ColumnString>(arguments[0].column.get()))
dictionary_name = name_col->getValue<String>();
else
@ -278,16 +276,14 @@ public:
+ ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
String attribute_name;
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[1].column.get()))
attribute_name = name_col->getValue<String>();
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

View File

@ -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

View File

@ -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;