write uuid for ddl dictionaries

This commit is contained in:
Alexander Tokmakov 2020-07-14 00:21:01 +03:00
parent fdb02edc7c
commit 31e62e713e
8 changed files with 49 additions and 36 deletions

View File

@ -47,7 +47,7 @@ void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name,
attachTableUnlocked(
dictionary_name,
StorageDictionary::create(
StorageID(database_name, dictionary_name),
StorageID(attach_info.create_query),
full_name,
ExternalDictionariesLoader::getDictionaryStructure(*attach_info.config),
StorageDictionary::Location::SameDatabaseAndNameAsDictionary),

View File

@ -605,6 +605,32 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
}
}
void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const
{
const auto kind = create.is_dictionary ? "Dictionary" : "Table";
const auto kind_upper = create.is_dictionary ? "DICTIONARY" : "TABLE";
if (database->getEngineName() == "Atomic")
{
if (create.attach && create.uuid == UUIDHelpers::Nil)
throw Exception(ErrorCodes::INCORRECT_QUERY,
"UUID must be specified in ATTACH {} query for Atomic database engine",
kind_upper);
if (!create.attach && create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
}
else
{
bool is_on_cluster = context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
if (create.uuid != UUIDHelpers::Nil && !is_on_cluster)
throw Exception(ErrorCodes::INCORRECT_QUERY,
"{} UUID specified, but engine of database {} is not Atomic", create.database, kind);
/// Ignore UUID if it's ON CLUSTER query
create.uuid = UUIDHelpers::Nil;
}
}
BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
{
@ -665,23 +691,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
if (need_add_to_database)
{
database = DatabaseCatalog::instance().getDatabase(create.database);
if (database->getEngineName() == "Atomic")
{
/// TODO implement ATTACH FROM 'path/to/data': generate UUID and move table data to store/
if (create.attach && create.uuid == UUIDHelpers::Nil)
throw Exception("UUID must be specified in ATTACH TABLE query for Atomic database engine", ErrorCodes::INCORRECT_QUERY);
if (!create.attach && create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
}
else
{
bool is_on_cluster = context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
if (create.uuid != UUIDHelpers::Nil && !is_on_cluster)
throw Exception("Table UUID specified, but engine of database " + create.database + " is not Atomic", ErrorCodes::INCORRECT_QUERY);
/// Ignore UUID if it's ON CLUSTER query
create.uuid = UUIDHelpers::Nil;
}
assertOrSetUUID(create, database);
/** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing).
* If table doesnt exist, one thread is creating table, while others wait in DDLGuard.
@ -785,8 +795,6 @@ BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create)
BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create)
{
create.uuid = UUIDHelpers::Nil; //FIXME
String dictionary_name = create.table;
create.database = context.resolveDatabase(create.database);
@ -810,6 +818,12 @@ BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create)
auto query = DatabaseCatalog::instance().getDatabase(database_name)->getCreateDictionaryQuery(dictionary_name);
create = query->as<ASTCreateQuery &>();
create.attach = true;
}
assertOrSetUUID(create, database);
if (create.attach)
{
auto config = getDictionaryConfigurationFromAST(create);
auto modification_time = database->getObjectMetadataModificationTime(dictionary_name);
database->attachDictionary(dictionary_name, DictionaryAttachInfo{query_ptr, config, modification_time});

View File

@ -73,6 +73,8 @@ private:
/// Inserts data in created table if it's CREATE ... SELECT
BlockIO fillTableIfNeeded(const ASTCreateQuery & create);
void assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const;
ASTPtr query_ptr;
Context & context;

View File

@ -254,6 +254,9 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
settings.ostr << (settings.hilite ? hilite_keyword : "") << (attach ? "ATTACH " : "CREATE ") << "DICTIONARY "
<< (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "")
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
if (uuid != UUIDHelpers::Nil)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
<< quoteString(toString(uuid));
formatOnCluster(settings);
}

View File

@ -796,7 +796,7 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
ParserKeyword s_dictionary("DICTIONARY");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_on("ON");
ParserIdentifier name_p;
ParserCompoundIdentifier dict_name_p(true);
ParserToken s_left_paren(TokenType::OpeningRoundBracket);
ParserToken s_right_paren(TokenType::ClosingRoundBracket);
ParserToken s_dot(TokenType::Dot);
@ -805,7 +805,6 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
bool if_not_exists = false;
ASTPtr database;
ASTPtr name;
ASTPtr attributes;
ASTPtr dictionary;
@ -826,16 +825,9 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
if (s_if_not_exists.ignore(pos, expected))
if_not_exists = true;
if (!name_p.parse(pos, name, expected))
if (!dict_name_p.parse(pos, name, expected))
return false;
if (s_dot.ignore(pos))
{
database = name;
if (!name_p.parse(pos, name, expected))
return false;
}
if (s_on.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
@ -862,8 +854,10 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
query->is_dictionary = true;
query->attach = attach;
tryGetIdentifierNameInto(database, query->database);
tryGetIdentifierNameInto(name, query->table);
StorageID dict_id = getTableIdentifier(name);
query->database = dict_id.database_name;
query->table = dict_id.table_name;
query->uuid = dict_id.uuid;
query->if_not_exists = if_not_exists;
query->set(query->dictionary_attributes_list, attributes);

View File

@ -1,6 +1,6 @@
DROP DATABASE IF EXISTS dictdb;
CREATE DATABASE dictdb ENGINE=Ordinary;
CREATE DATABASE dictdb;
CREATE DICTIONARY dictdb.restricted_dict (
key UInt64,

View File

@ -1,5 +1,5 @@
DROP DATABASE IF EXISTS dict_db_01254;
CREATE DATABASE dict_db_01254 ENGINE=Ordinary;
CREATE DATABASE dict_db_01254;
CREATE TABLE dict_db_01254.dict_data (key UInt64, val UInt64) Engine=Memory();
CREATE DICTIONARY dict_db_01254.dict
@ -13,7 +13,7 @@ LIFETIME(MIN 0 MAX 0)
LAYOUT(FLAT());
DETACH DATABASE dict_db_01254;
ATTACH DATABASE dict_db_01254 ENGINE=Ordinary;
ATTACH DATABASE dict_db_01254;
SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict';
SYSTEM RELOAD DICTIONARY dict_db_01254.dict;

View File

@ -7,7 +7,7 @@ DROP TABLE IF EXISTS dictdb_01376.table_for_dict;
DROP DICTIONARY IF EXISTS dictdb_01376.dict_exists;
DROP DATABASE IF EXISTS dictdb_01376;
CREATE DATABASE dictdb_01376 ENGINE = Ordinary;
CREATE DATABASE dictdb_01376;
CREATE TABLE dictdb_01376.table_for_dict
(