mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
rename/exchange database/table/dictionary support IF EXISTS syntax
This commit is contained in:
parent
59c8ed9b0c
commit
ec2c70ab75
@ -78,45 +78,68 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c
|
||||
|
||||
for (const auto & elem : descriptions)
|
||||
{
|
||||
bool exchange_tables;
|
||||
if (rename.exchange)
|
||||
bool ignore = elem.if_exists;
|
||||
|
||||
if (ignore)
|
||||
{
|
||||
exchange_tables = true;
|
||||
}
|
||||
else if (rename.rename_if_cannot_exchange)
|
||||
{
|
||||
exchange_tables = database_catalog.isTableExist(StorageID(elem.to_database_name, elem.to_table_name), getContext());
|
||||
renamed_instead_of_exchange = !exchange_tables;
|
||||
}
|
||||
else
|
||||
{
|
||||
exchange_tables = false;
|
||||
database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name), getContext());
|
||||
if (rename.exchange)
|
||||
{
|
||||
ignore = rename.dictionary ? (!database_catalog.isDictionaryExist(StorageID(elem.to_database_name, elem.to_table_name)) ||
|
||||
!database_catalog.isDictionaryExist(StorageID(elem.from_database_name, elem.from_table_name))) :
|
||||
(!database_catalog.isTableExist(StorageID(elem.to_database_name, elem.to_table_name), getContext()) ||
|
||||
!database_catalog.isTableExist(StorageID(elem.from_database_name, elem.from_table_name), getContext()));
|
||||
}
|
||||
else
|
||||
{
|
||||
ignore = rename.dictionary ? (!database_catalog.isDictionaryExist(StorageID(elem.to_database_name, elem.to_table_name))) :
|
||||
(!database_catalog.isTableExist(StorageID(elem.to_database_name, elem.to_table_name), getContext()));
|
||||
}
|
||||
}
|
||||
|
||||
DatabasePtr database = database_catalog.getDatabase(elem.from_database_name);
|
||||
if (typeid_cast<DatabaseReplicated *>(database.get())
|
||||
&& !getContext()->getClientInfo().is_replicated_database_internal)
|
||||
if (!ignore)
|
||||
{
|
||||
if (1 < descriptions.size())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Database {} is Replicated, "
|
||||
"it does not support renaming of multiple tables in single query.", elem.from_database_name);
|
||||
bool exchange_tables;
|
||||
if (rename.exchange)
|
||||
{
|
||||
exchange_tables = true;
|
||||
}
|
||||
else if (rename.rename_if_cannot_exchange)
|
||||
{
|
||||
exchange_tables = database_catalog.isTableExist(StorageID(elem.to_database_name, elem.to_table_name), getContext());
|
||||
renamed_instead_of_exchange = !exchange_tables;
|
||||
}
|
||||
else
|
||||
{
|
||||
exchange_tables = false;
|
||||
database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name), getContext());
|
||||
}
|
||||
|
||||
UniqueTableName from(elem.from_database_name, elem.from_table_name);
|
||||
UniqueTableName to(elem.to_database_name, elem.to_table_name);
|
||||
ddl_guards[from]->releaseTableLock();
|
||||
ddl_guards[to]->releaseTableLock();
|
||||
return typeid_cast<DatabaseReplicated *>(database.get())->tryEnqueueReplicatedDDL(query_ptr, getContext());
|
||||
}
|
||||
else
|
||||
{
|
||||
database->renameTable(
|
||||
getContext(),
|
||||
elem.from_table_name,
|
||||
*database_catalog.getDatabase(elem.to_database_name),
|
||||
elem.to_table_name,
|
||||
exchange_tables,
|
||||
rename.dictionary);
|
||||
DatabasePtr database = database_catalog.getDatabase(elem.from_database_name);
|
||||
if (typeid_cast<DatabaseReplicated *>(database.get()) && !getContext()->getClientInfo().is_replicated_database_internal)
|
||||
{
|
||||
if (1 < descriptions.size())
|
||||
throw Exception(
|
||||
ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Database {} is Replicated, "
|
||||
"it does not support renaming of multiple tables in single query.",
|
||||
elem.from_database_name);
|
||||
|
||||
UniqueTableName from(elem.from_database_name, elem.from_table_name);
|
||||
UniqueTableName to(elem.to_database_name, elem.to_table_name);
|
||||
ddl_guards[from]->releaseTableLock();
|
||||
ddl_guards[to]->releaseTableLock();
|
||||
return typeid_cast<DatabaseReplicated *>(database.get())->tryEnqueueReplicatedDDL(query_ptr, getContext());
|
||||
}
|
||||
else
|
||||
{
|
||||
database->renameTable(
|
||||
getContext(),
|
||||
elem.from_table_name,
|
||||
*database_catalog.getDatabase(elem.to_database_name),
|
||||
elem.to_table_name,
|
||||
exchange_tables,
|
||||
rename.dictionary);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -133,9 +156,13 @@ BlockIO InterpreterRenameQuery::executeToDatabase(const ASTRenameQuery &, const
|
||||
const auto & new_name = descriptions.back().to_database_name;
|
||||
auto & catalog = DatabaseCatalog::instance();
|
||||
|
||||
auto db = catalog.getDatabase(old_name);
|
||||
catalog.assertDatabaseDoesntExist(new_name);
|
||||
db->renameDatabase(new_name);
|
||||
auto db = descriptions.front().if_exists ? catalog.tryGetDatabase(old_name) : catalog.getDatabase(old_name);
|
||||
|
||||
if (db) {
|
||||
catalog.assertDatabaseDoesntExist(new_name);
|
||||
db->renameDatabase(new_name);
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
|
@ -31,7 +31,8 @@ struct RenameDescription
|
||||
from_database_name(elem.from.database.empty() ? current_database : elem.from.database),
|
||||
from_table_name(elem.from.table),
|
||||
to_database_name(elem.to.database.empty() ? current_database : elem.to.database),
|
||||
to_table_name(elem.to.table)
|
||||
to_table_name(elem.to.table),
|
||||
if_exists(elem.if_exists)
|
||||
{}
|
||||
|
||||
String from_database_name;
|
||||
@ -39,6 +40,7 @@ struct RenameDescription
|
||||
|
||||
String to_database_name;
|
||||
String to_table_name;
|
||||
bool if_exists;
|
||||
};
|
||||
|
||||
using RenameDescriptions = std::vector<RenameDescription>;
|
||||
|
@ -25,6 +25,7 @@ public:
|
||||
{
|
||||
Table from;
|
||||
Table to;
|
||||
bool if_exists{false}; /// If this directive is used, one will not get an error if the table/database/dictionary to be renamed/exchanged doesn't exist.
|
||||
};
|
||||
|
||||
using Elements = std::vector<Element>;
|
||||
|
@ -44,6 +44,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserKeyword s_rename_dictionary("RENAME DICTIONARY");
|
||||
ParserKeyword s_exchange_dictionaries("EXCHANGE DICTIONARIES");
|
||||
ParserKeyword s_rename_database("RENAME DATABASE");
|
||||
ParserKeyword s_if_exists("IF EXISTS");
|
||||
ParserKeyword s_to("TO");
|
||||
ParserKeyword s_and("AND");
|
||||
ParserToken s_comma(TokenType::Comma);
|
||||
@ -67,6 +68,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ASTPtr from_db;
|
||||
ASTPtr to_db;
|
||||
ParserIdentifier db_name_p;
|
||||
bool if_exists = s_if_exists.ignore(pos, expected);
|
||||
if (!db_name_p.parse(pos, from_db, expected))
|
||||
return false;
|
||||
if (!s_to.ignore(pos, expected))
|
||||
@ -84,6 +86,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
auto query = std::make_shared<ASTRenameQuery>();
|
||||
query->database = true;
|
||||
query->elements.emplace({});
|
||||
query->elements.front().if_exists = if_exists;
|
||||
tryGetIdentifierNameInto(from_db, query->elements.front().from.database);
|
||||
tryGetIdentifierNameInto(to_db, query->elements.front().to.database);
|
||||
query->cluster = cluster_str;
|
||||
@ -103,6 +106,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
break;
|
||||
|
||||
ASTRenameQuery::Element& ref = elements.emplace_back();
|
||||
ref.if_exists = s_if_exists.ignore( pos, expected);
|
||||
|
||||
if (!parseDatabaseAndTable(ref.from, pos, expected)
|
||||
|| !ignore_delim()
|
||||
|
@ -13,17 +13,23 @@ CREATE TABLE t2 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBloc
|
||||
EXCHANGE TABLES t1 AND t3; -- { serverError 60 }
|
||||
EXCHANGE TABLES t4 AND t2; -- { serverError 60 }
|
||||
RENAME TABLE t0 TO t1; -- { serverError 57 }
|
||||
DROP TABLE t1;
|
||||
RENAME TABLE t0 TO t1;
|
||||
|
||||
RENAME DATABASE IF EXISTS test_non_exists TO test_non_exists_renamed;
|
||||
EXCHANGE TABLES IF EXISTS t1 AND t3;
|
||||
EXCHANGE TABLES IF EXISTS t4 AND t2;
|
||||
RENAME TABLE IF EXISTS t3 TO t4;
|
||||
|
||||
DROP TABLE IF EXISTS t1;
|
||||
RENAME TABLE IF EXISTS t0 TO t1;
|
||||
SELECT * FROM t1;
|
||||
SELECT * FROM t2;
|
||||
|
||||
EXCHANGE TABLES t1 AND t2;
|
||||
EXCHANGE TABLES IF EXISTS t1 AND t2;
|
||||
SELECT * FROM t1;
|
||||
SELECT * FROM t2;
|
||||
|
||||
RENAME TABLE t1 TO t1tmp, t2 TO t2tmp;
|
||||
RENAME TABLE t1tmp TO t2, t2tmp TO t1;
|
||||
RENAME TABLE IF EXISTS t1 TO t1tmp, t2 TO t2tmp;
|
||||
RENAME TABLE IF EXISTS t1tmp TO t2, t2tmp TO t1;
|
||||
SELECT * FROM t1;
|
||||
SELECT * FROM t2;
|
||||
|
||||
@ -44,8 +50,8 @@ EXCHANGE TABLES test_01109_other_atomic.t3 AND test_01109_ordinary.t4; -- { serv
|
||||
EXCHANGE TABLES test_01109_ordinary.t4 AND test_01109_other_atomic.t3; -- { serverError 48 }
|
||||
EXCHANGE TABLES test_01109_ordinary.t4 AND test_01109_ordinary.t4; -- { serverError 48 }
|
||||
|
||||
EXCHANGE TABLES t1 AND test_01109_other_atomic.t3;
|
||||
EXCHANGE TABLES t2 AND t2;
|
||||
EXCHANGE TABLES IF EXISTS t1 AND test_01109_other_atomic.t3;
|
||||
EXCHANGE TABLES IF EXISTS t2 AND t2;
|
||||
SELECT * FROM t1;
|
||||
SELECT * FROM t2;
|
||||
SELECT * FROM test_01109_other_atomic.t3;
|
||||
|
Loading…
Reference in New Issue
Block a user