ClickHouse/src/Interpreters/InterpreterRenameQuery.cpp

179 lines
6.6 KiB
C++
Raw Normal View History

#include <Parsers/ASTRenameQuery.h>
#include <Databases/IDatabase.h>
2017-05-23 18:24:43 +00:00
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterRenameQuery.h>
#include <Storages/IStorage.h>
2020-11-03 13:47:26 +00:00
#include <Interpreters/executeDDLQueryOnCluster.h>
2020-12-14 03:30:39 +00:00
#include <Interpreters/QueryLog.h>
#include <Access/AccessRightsElement.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Databases/DatabaseReplicated.h>
2012-06-18 06:20:23 +00:00
namespace DB
{
2021-01-26 17:51:25 +00:00
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
2012-06-18 06:20:23 +00:00
InterpreterRenameQuery::InterpreterRenameQuery(const ASTPtr & query_ptr_, ContextPtr context_)
: WithContext(context_), query_ptr(query_ptr_)
2012-06-18 06:20:23 +00:00
{
}
2015-06-18 02:11:05 +00:00
BlockIO InterpreterRenameQuery::execute()
2012-06-18 06:20:23 +00:00
{
2020-01-24 16:20:36 +00:00
const auto & rename = query_ptr->as<const ASTRenameQuery &>();
2017-04-21 12:39:28 +00:00
if (!rename.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccess());
getContext()->checkAccess(getRequiredAccess());
2017-04-21 12:39:28 +00:00
String path = getContext()->getPath();
String current_database = getContext()->getCurrentDatabase();
/** In case of error while renaming, it is possible that only part of tables was renamed
* or we will be in inconsistent state. (It is worth to be fixed.)
*/
2012-06-18 06:20:23 +00:00
2020-07-07 12:11:58 +00:00
RenameDescriptions descriptions;
descriptions.reserve(rename.elements.size());
2021-02-02 19:39:04 +00:00
/// Don't allow to drop tables (that we are renaming); don't allow to create tables in places where tables will be renamed.
TableGuards table_guards;
for (const auto & elem : rename.elements)
{
2017-12-01 21:13:25 +00:00
descriptions.emplace_back(elem, current_database);
2020-01-24 16:20:36 +00:00
const auto & description = descriptions.back();
2020-01-24 16:20:36 +00:00
UniqueTableName from(description.from_database_name, description.from_table_name);
UniqueTableName to(description.to_database_name, description.to_table_name);
table_guards[from];
table_guards[to];
}
2012-06-18 06:20:23 +00:00
2020-02-10 18:31:52 +00:00
auto & database_catalog = DatabaseCatalog::instance();
/// Must do it in consistent order.
for (auto & table_guard : table_guards)
2020-02-10 18:31:52 +00:00
table_guard.second = database_catalog.getDDLGuard(table_guard.first.database_name, table_guard.first.table_name);
2012-06-18 06:20:23 +00:00
2020-07-07 12:11:58 +00:00
if (rename.database)
return executeToDatabase(rename, descriptions);
else
2021-02-02 19:39:04 +00:00
return executeToTables(rename, descriptions, table_guards);
2020-07-07 12:11:58 +00:00
}
2021-02-02 19:39:04 +00:00
BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions, TableGuards & ddl_guards)
2020-07-07 12:11:58 +00:00
{
2021-08-04 18:14:59 +00:00
assert(!rename.rename_if_cannot_exchange || descriptions.size() == 1);
2021-08-05 11:14:50 +00:00
assert(!(rename.rename_if_cannot_exchange && rename.exchange));
2020-07-07 12:11:58 +00:00
auto & database_catalog = DatabaseCatalog::instance();
2020-07-13 14:24:00 +00:00
for (const auto & elem : descriptions)
{
2021-08-04 18:14:59 +00:00
bool exchange_tables;
if (rename.exchange)
2021-07-01 13:21:38 +00:00
{
2021-08-04 18:14:59 +00:00
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;
2021-07-01 13:21:38 +00:00
}
else
{
2021-08-04 18:14:59 +00:00
exchange_tables = false;
database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name), getContext());
2021-07-01 13:21:38 +00:00
}
2020-05-11 12:55:17 +00:00
DatabasePtr database = database_catalog.getDatabase(elem.from_database_name);
if (typeid_cast<DatabaseReplicated *>(database.get())
2021-07-30 16:34:18 +00:00
&& !getContext()->getClientInfo().is_replicated_database_internal)
2020-10-22 15:08:00 +00:00
{
2021-01-26 17:51:25 +00:00
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);
2021-02-02 19:39:04 +00:00
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());
2020-10-22 15:08:00 +00:00
}
else
{
database->renameTable(
getContext(),
elem.from_table_name,
*database_catalog.getDatabase(elem.to_database_name),
elem.to_table_name,
2021-07-01 13:21:38 +00:00
exchange_tables,
2020-07-16 21:41:26 +00:00
rename.dictionary);
2020-05-11 12:55:17 +00:00
}
}
return {};
2012-06-18 06:20:23 +00:00
}
2020-07-07 12:11:58 +00:00
BlockIO InterpreterRenameQuery::executeToDatabase(const ASTRenameQuery &, const RenameDescriptions & descriptions)
{
assert(descriptions.size() == 1);
assert(descriptions.front().from_table_name.empty());
assert(descriptions.front().to_table_name.empty());
2020-07-08 14:28:07 +00:00
2020-07-07 12:11:58 +00:00
const auto & old_name = descriptions.front().from_database_name;
const auto & new_name = descriptions.back().to_database_name;
2020-07-08 14:28:07 +00:00
auto & catalog = DatabaseCatalog::instance();
auto db = catalog.getDatabase(old_name);
catalog.assertDatabaseDoesntExist(new_name);
db->renameDatabase(new_name);
2020-07-07 12:11:58 +00:00
return {};
}
2020-01-24 16:20:36 +00:00
AccessRightsElements InterpreterRenameQuery::getRequiredAccess() const
{
AccessRightsElements required_access;
const auto & rename = query_ptr->as<const ASTRenameQuery &>();
for (const auto & elem : rename.elements)
{
required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.from.database, elem.from.table);
required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.to.database, elem.to.table);
2020-07-07 12:11:58 +00:00
if (rename.exchange)
{
2020-07-17 13:11:44 +00:00
required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.from.database, elem.from.table);
required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.to.database, elem.to.table);
2020-07-07 12:11:58 +00:00
}
2020-01-24 16:20:36 +00:00
}
return required_access;
}
2012-06-18 06:20:23 +00:00
void InterpreterRenameQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const
2020-12-14 03:30:39 +00:00
{
elem.query_kind = "Rename";
const auto & rename = ast->as<const ASTRenameQuery &>();
for (const auto & element : rename.elements)
{
{
String database = backQuoteIfNeed(element.from.database.empty() ? getContext()->getCurrentDatabase() : element.from.database);
2020-12-14 03:30:39 +00:00
elem.query_databases.insert(database);
elem.query_tables.insert(database + "." + backQuoteIfNeed(element.from.table));
}
{
String database = backQuoteIfNeed(element.to.database.empty() ? getContext()->getCurrentDatabase() : element.to.database);
2020-12-14 03:30:39 +00:00
elem.query_databases.insert(database);
elem.query_tables.insert(database + "." + backQuoteIfNeed(element.to.table));
}
}
}
2012-06-18 06:20:23 +00:00
}