diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 28d88bdd8df..38e5b266e13 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1101,6 +1101,7 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, [[maybe_unused]] bool done = doCreateTable(create, properties); assert(done); ast_drop->table = create.table; + ast_drop->is_dictionary = create.is_dictionary; ast_drop->database = create.database; ast_drop->kind = ASTDropQuery::Drop; created = true; @@ -1113,14 +1114,18 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create, ASTRenameQuery::Table{create.database, create.table}, ASTRenameQuery::Table{create.database, table_to_replace_name} }; + ast_rename->elements.push_back(std::move(elem)); ast_rename->exchange = true; + ast_rename->dictionary = create.is_dictionary; + InterpreterRenameQuery(ast_rename, getContext()).execute(); replaced = true; InterpreterDropQuery(ast_drop, getContext()).execute(); create.table = table_to_replace_name; + return fillTableIfNeeded(create); } catch (...) diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index d6d424beb3a..8d8a0a1840a 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -305,8 +305,16 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat } else { + String action = "CREATE"; + if (attach) + action = "ATTACH"; + else if (replace_table && create_or_replace) + action = "CREATE OR REPLACE"; + else if (replace_table) + action = "REPLACE"; + /// Always DICTIONARY - settings.ostr << (settings.hilite ? hilite_keyword : "") << (attach ? "ATTACH " : "CREATE ") << "DICTIONARY " + settings.ostr << (settings.hilite ? hilite_keyword : "") << action << "DICTIONARY " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "") << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); if (uuid != UUIDHelpers::Nil) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 23a53ed3244..d4525883e36 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -971,6 +971,8 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E { ParserKeyword s_create("CREATE"); ParserKeyword s_attach("ATTACH"); + ParserKeyword s_replace("REPLACE"); + ParserKeyword s_or_replace("OR REPLACE"); ParserKeyword s_dictionary("DICTIONARY"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); ParserKeyword s_on("ON"); @@ -982,6 +984,8 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E ParserDictionary dictionary_p; bool if_not_exists = false; + bool replace = false; + bool or_replace = false; ASTPtr name; ASTPtr attributes; @@ -989,13 +993,21 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E String cluster_str; bool attach = false; - if (!s_create.ignore(pos, expected)) + + if (s_create.ignore(pos, expected)) { - if (s_attach.ignore(pos, expected)) - attach = true; - else - return false; + if (s_or_replace.ignore(pos, expected)) + { + replace = true; + or_replace = true; + } } + else if (s_attach.ignore(pos, expected)) + attach = true; + else if (s_replace.ignore(pos, expected)) + replace = true; + else + return false; if (!s_dictionary.ignore(pos, expected)) return false; @@ -1031,6 +1043,8 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E node = query; query->is_dictionary = true; query->attach = attach; + query->create_or_replace = or_replace; + query->replace_table = replace; auto dict_id = name->as()->getTableId(); query->database = dict_id.database_name; diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 5aeaff590e1..2c37f0ef641 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -167,7 +167,7 @@ Pipe StorageDictionary::read( const size_t max_block_size, const unsigned /*threads*/) { - auto dictionary = getContext()->getExternalDictionariesLoader().getDictionary(dictionary_name, local_context); + auto dictionary = getContext()->getExternalDictionariesLoader().getDictionary(getStorageID().getInternalDictionaryName(), local_context); auto stream = dictionary->getBlockInputStream(column_names, max_block_size); /// TODO: update dictionary interface for processors. return Pipe(std::make_shared(stream)); @@ -215,23 +215,30 @@ LoadablesConfigurationPtr StorageDictionary::getConfiguration() const void StorageDictionary::renameInMemory(const StorageID & new_table_id) { + auto previous_table_id = getStorageID(); + auto previous_dictionary_name = getStorageID().getInternalDictionaryName(); + auto new_dictionary_name = new_table_id.getInternalDictionaryName(); + + IStorage::renameInMemory(new_table_id); + + dictionary_name = new_dictionary_name; + if (configuration) { configuration->setString("dictionary.database", new_table_id.database_name); configuration->setString("dictionary.name", new_table_id.table_name); const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); - external_dictionaries_loader.reloadConfig(getStorageID().getInternalDictionaryName()); + external_dictionaries_loader.reloadConfig(previous_dictionary_name); + + auto result = external_dictionaries_loader.getLoadResult(new_dictionary_name); - auto result = external_dictionaries_loader.getLoadResult(getStorageID().getInternalDictionaryName()); if (!result.object) return; const auto dictionary = std::static_pointer_cast(result.object); dictionary->updateDictionaryName(new_table_id); } - - IStorage::renameInMemory(new_table_id); } void registerStorageDictionary(StorageFactory & factory) diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index e2ba2964b1d..d074dec2c34 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -45,7 +45,7 @@ public: Poco::Timestamp getUpdateTime() const; LoadablesConfigurationPtr getConfiguration() const; - const String & getDictionaryName() const { return dictionary_name; } + String getDictionaryName() const { return dictionary_name; } /// Specifies where the table is located relative to the dictionary. enum class Location @@ -66,7 +66,7 @@ public: }; private: - const String dictionary_name; + String dictionary_name; const Location location; mutable std::mutex dictionary_config_mutex; diff --git a/tests/queries/0_stateless/01913_replace_dictionary.reference b/tests/queries/0_stateless/01913_replace_dictionary.reference new file mode 100644 index 00000000000..2d33c16ccc2 --- /dev/null +++ b/tests/queries/0_stateless/01913_replace_dictionary.reference @@ -0,0 +1,2 @@ +0 Value0 +0 Value1 diff --git a/tests/queries/0_stateless/01913_replace_dictionary.sql b/tests/queries/0_stateless/01913_replace_dictionary.sql new file mode 100644 index 00000000000..22b0bd002ae --- /dev/null +++ b/tests/queries/0_stateless/01913_replace_dictionary.sql @@ -0,0 +1,50 @@ +DROP DATABASE IF EXISTS 01913_db; +CREATE DATABASE 01913_db ENGINE=Atomic; + +DROP TABLE IF EXISTS 01913_db.test_source_table_1; +CREATE TABLE 01913_db.test_source_table_1 +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO 01913_db.test_source_table_1 VALUES (0, 'Value0'); + +DROP DICTIONARY IF EXISTS 01913_db.test_dictionary; +CREATE DICTIONARY 01913_db.test_dictionary +( + id UInt64, + value String +) +PRIMARY KEY id +LAYOUT(DIRECT()) +SOURCE(CLICKHOUSE(DB '01913_db' TABLE 'test_source_table_1')); + +SELECT * FROM 01913_db.test_dictionary; + +DROP TABLE IF EXISTS 01913_db.test_source_table_2; +CREATE TABLE 01913_db.test_source_table_2 +( + id UInt64, + value_1 String +) ENGINE=TinyLog; + +INSERT INTO 01913_db.test_source_table_2 VALUES (0, 'Value1'); + +REPLACE DICTIONARY 01913_db.test_dictionary +( + id UInt64, + value_1 String +) +PRIMARY KEY id +LAYOUT(HASHED()) +SOURCE(CLICKHOUSE(DB '01913_db' TABLE 'test_source_table_2')) +LIFETIME(0); + +SELECT * FROM 01913_db.test_dictionary; + +DROP DICTIONARY 01913_db.test_dictionary; +DROP TABLE 01913_db.test_source_table_1; +DROP TABLE 01913_db.test_source_table_2; + +DROP DATABASE 01913_db;