mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Support REPLACE DICTIONARY, CREATE OR REPLACE DICTIONARY queries
This commit is contained in:
parent
6662e7ab0f
commit
008adabec2
@ -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 (...)
|
||||
|
@ -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)
|
||||
|
@ -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))
|
||||
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<ASTTableIdentifier>()->getTableId();
|
||||
query->database = dict_id.database_name;
|
||||
|
@ -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<SourceFromInputStream>(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<const IDictionary>(result.object);
|
||||
dictionary->updateDictionaryName(new_table_id);
|
||||
}
|
||||
|
||||
IStorage::renameInMemory(new_table_id);
|
||||
}
|
||||
|
||||
void registerStorageDictionary(StorageFactory & factory)
|
||||
|
@ -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;
|
||||
|
@ -0,0 +1,2 @@
|
||||
0 Value0
|
||||
0 Value1
|
50
tests/queries/0_stateless/01913_replace_dictionary.sql
Normal file
50
tests/queries/0_stateless/01913_replace_dictionary.sql
Normal file
@ -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;
|
Loading…
Reference in New Issue
Block a user