2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTRenameQuery.h>
|
|
|
|
#include <Databases/IDatabase.h>
|
2017-05-23 18:24:43 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2024-03-19 16:04:29 +00:00
|
|
|
#include <Interpreters/DatabaseCatalog.h>
|
2024-01-09 06:33:48 +00:00
|
|
|
#include <Interpreters/InterpreterFactory.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#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>
|
2021-10-31 08:51:20 +00:00
|
|
|
#include <Access/Common/AccessRightsElement.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2020-07-04 16:32:23 +00:00
|
|
|
#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;
|
2022-09-05 10:38:44 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2021-01-26 17:51:25 +00:00
|
|
|
}
|
2012-06-18 06:20:23 +00:00
|
|
|
|
2021-04-10 23:33:54 +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
|
|
|
|
2019-03-15 16:14:13 +00:00
|
|
|
if (!rename.cluster.empty())
|
2022-04-22 12:15:29 +00:00
|
|
|
{
|
|
|
|
DDLQueryOnClusterParams params;
|
2022-09-05 10:38:44 +00:00
|
|
|
params.access_to_check = getRequiredAccess(rename.database ? RenameType::RenameDatabase : RenameType::RenameTable);
|
2022-04-22 12:15:29 +00:00
|
|
|
return executeDDLQueryOnCluster(query_ptr, getContext(), params);
|
|
|
|
}
|
2018-04-17 19:33:58 +00:00
|
|
|
|
2022-09-05 10:38:44 +00:00
|
|
|
getContext()->checkAccess(getRequiredAccess(rename.database ? RenameType::RenameDatabase : RenameType::RenameTable));
|
2017-04-21 12:39:28 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
String path = getContext()->getPath();
|
|
|
|
String current_database = getContext()->getCurrentDatabase();
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2017-01-23 18:05:07 +00:00
|
|
|
/** 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;
|
2024-03-08 11:54:20 +00:00
|
|
|
descriptions.reserve(rename.getElements().size());
|
2015-04-08 01:06:15 +00:00
|
|
|
|
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;
|
|
|
|
|
2024-03-08 11:54:20 +00:00
|
|
|
for (const auto & elem : rename.getElements())
|
2012-06-18 06:20:23 +00:00
|
|
|
{
|
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();
|
2016-03-22 01:33:17 +00:00
|
|
|
|
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);
|
2016-03-22 01:33:17 +00:00
|
|
|
|
2019-08-31 10:37:28 +00:00
|
|
|
table_guards[from];
|
|
|
|
table_guards[to];
|
2015-04-08 01:06:15 +00:00
|
|
|
}
|
2012-06-18 06:20:23 +00:00
|
|
|
|
2020-02-10 18:31:52 +00:00
|
|
|
auto & database_catalog = DatabaseCatalog::instance();
|
|
|
|
|
2019-08-31 10:37:28 +00:00
|
|
|
/// 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)
|
2015-04-08 01:06:15 +00:00
|
|
|
{
|
2021-11-11 16:29:56 +00:00
|
|
|
if (elem.if_exists)
|
|
|
|
{
|
|
|
|
assert(!rename.exchange);
|
|
|
|
if (!database_catalog.isTableExist(StorageID(elem.from_database_name, elem.from_table_name), getContext()))
|
|
|
|
continue;
|
|
|
|
}
|
2014-03-20 10:59:45 +00:00
|
|
|
|
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;
|
2021-04-10 23:33:54 +00:00
|
|
|
database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name), getContext());
|
2021-07-01 13:21:38 +00:00
|
|
|
}
|
2014-03-20 10:59:45 +00:00
|
|
|
|
2020-05-11 12:55:17 +00:00
|
|
|
DatabasePtr database = database_catalog.getDatabase(elem.from_database_name);
|
2022-09-15 19:15:57 +00:00
|
|
|
if (database->shouldReplicateQuery(getContext(), query_ptr))
|
2020-10-22 15:08:00 +00:00
|
|
|
{
|
2021-01-26 17:51:25 +00:00
|
|
|
if (1 < descriptions.size())
|
2021-11-11 16:29:56 +00:00
|
|
|
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-01 19:29:47 +00:00
|
|
|
|
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();
|
2022-09-16 14:25:32 +00:00
|
|
|
return database->tryEnqueueReplicatedDDL(query_ptr, getContext());
|
2020-10-22 15:08:00 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2023-02-01 23:30:49 +00:00
|
|
|
StorageID from_table_id{elem.from_database_name, elem.from_table_name};
|
|
|
|
StorageID to_table_id{elem.to_database_name, elem.to_table_name};
|
|
|
|
std::vector<StorageID> ref_dependencies;
|
|
|
|
std::vector<StorageID> loading_dependencies;
|
|
|
|
|
2021-11-02 12:58:45 +00:00
|
|
|
if (!exchange_tables)
|
2023-02-01 23:30:49 +00:00
|
|
|
{
|
|
|
|
bool check_ref_deps = getContext()->getSettingsRef().check_referential_table_dependencies;
|
|
|
|
bool check_loading_deps = !check_ref_deps && getContext()->getSettingsRef().check_table_dependencies;
|
|
|
|
std::tie(ref_dependencies, loading_dependencies) = database_catalog.removeDependencies(from_table_id, check_ref_deps, check_loading_deps);
|
|
|
|
}
|
2021-11-02 12:58:45 +00:00
|
|
|
|
Fix table dependencies in case of failed RENAME TABLE
CI found this [1]
[ 382620 ] {} <Error> Application: Caught exception while loading metadata: Code: 60. DB::Exception: Table test_25.join doesn\'t exist: While processing _CAST(joinGet(test_25.join, \'m\', CAST(\'42\', \'int\')) AS m_tmp_alter1351896088559986425, \'Int32\') AS m: default expression and column type are incomp
[ 382620 ] {} <Error> Application: Code: 60. DB::Exception: Table test_25.join doesn\'t exist: While processing _CAST(joinGet(test_25.join, \'m\', CAST(\'42\', \'int\')) AS m_tmp_alter1351896088559986425, \'Int32\') AS m: default expression and column type are incompatible.: Cannot attach table `test_25`.`t
[1]: https://s3.amazonaws.com/clickhouse-test-reports/48670/d7f865037266ed87538cf4df7ec7e8165681871b/stress_test__asan_.html
The problem here is that after failed RENAME dependencies got lost:
[4e937f39d866] 2023.04.12 00:51:06.833624 [ 13419 ] {61429225-64cb-4fce-b60d-01e0dac6e52c} <Error> executeQuery: Code: 241. DB::Exception: Memory limit (total) exceeded: would use 34.65 GiB (attempt to allocate chunk of 2097419 bytes), maximum: 34.29 GiB. OvercommitTracker decision: Memory overcommit has freed not enough memory. (MEMORY_LIMIT_EXCEEDED) (version 23.4.1.1) (from [::1]:45710) (comment: 01160_table_dependencies.sh) (in query: create database test_25_1), Stack trace (when copying this message, always include the lines below):
[4e937f39d866] 2023.04.12 00:51:07.351914 [ 5151 ] {66d8bdd4-668e-4239-a8af-6b8f17bb5222} <Error> executeQuery: Code: 81. DB::Exception: Database test_25_1 doesn't exist. (UNKNOWN_DATABASE) (version 23.4.1.1) (from [::1]:45762) (comment: 01160_table_dependencies.sh) (in query: rename table t to test_25_1.t), Stack trace (when copying this message, always include the lines below):
And from the test output:
OK
OK
OK
OK
a [] [] []
data_02344 [] [] []
date_table [] [] []
dict1 [] ['dict_src'] ['join']
dict2 [] ['join'] []
dict_src [] [] ['dict1']
dist_02346 [] [] []
join [] ['dict1'] ['dict2','s']
^^^^^^^^^^^^^ no "t"
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-04-12 09:07:18 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
database->renameTable(
|
|
|
|
getContext(),
|
|
|
|
elem.from_table_name,
|
|
|
|
*database_catalog.getDatabase(elem.to_database_name),
|
|
|
|
elem.to_table_name,
|
|
|
|
exchange_tables,
|
|
|
|
rename.dictionary);
|
|
|
|
|
|
|
|
DatabaseCatalog::instance().addDependencies(to_table_id, ref_dependencies, loading_dependencies);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
/// Restore dependencies if RENAME fails
|
|
|
|
DatabaseCatalog::instance().addDependencies(from_table_id, ref_dependencies, loading_dependencies);
|
|
|
|
throw;
|
|
|
|
}
|
2020-05-11 12:55:17 +00:00
|
|
|
}
|
2012-06-18 06:20:23 +00:00
|
|
|
}
|
2020-07-04 16:32:23 +00:00
|
|
|
|
2015-06-18 02:11:05 +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();
|
|
|
|
|
2021-11-04 14:35:34 +00:00
|
|
|
auto db = descriptions.front().if_exists ? catalog.tryGetDatabase(old_name) : catalog.getDatabase(old_name);
|
|
|
|
|
2021-11-08 08:51:22 +00:00
|
|
|
if (db)
|
|
|
|
{
|
2021-11-04 14:35:34 +00:00
|
|
|
catalog.assertDatabaseDoesntExist(new_name);
|
2021-11-15 10:35:45 +00:00
|
|
|
db->renameDatabase(getContext(), new_name);
|
2021-11-04 14:35:34 +00:00
|
|
|
}
|
|
|
|
|
2020-07-07 12:11:58 +00:00
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2022-09-05 10:38:44 +00:00
|
|
|
AccessRightsElements InterpreterRenameQuery::getRequiredAccess(InterpreterRenameQuery::RenameType type) const
|
2020-01-24 16:20:36 +00:00
|
|
|
{
|
|
|
|
AccessRightsElements required_access;
|
|
|
|
const auto & rename = query_ptr->as<const ASTRenameQuery &>();
|
2024-03-08 11:54:20 +00:00
|
|
|
for (const auto & elem : rename.getElements())
|
2020-01-24 16:20:36 +00:00
|
|
|
{
|
2022-09-05 10:38:44 +00:00
|
|
|
if (type == RenameType::RenameTable)
|
|
|
|
{
|
2023-02-14 13:01:06 +00:00
|
|
|
required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.from.getDatabase(), elem.from.getTable());
|
|
|
|
required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.to.getDatabase(), elem.to.getTable());
|
2022-09-05 10:38:44 +00:00
|
|
|
if (rename.exchange)
|
|
|
|
{
|
2023-07-25 04:10:04 +00:00
|
|
|
required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.from.getDatabase(), elem.from.getTable());
|
2023-02-14 13:01:06 +00:00
|
|
|
required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.to.getDatabase(), elem.to.getTable());
|
2022-09-05 10:38:44 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
else if (type == RenameType::RenameDatabase)
|
|
|
|
{
|
2023-02-14 13:01:06 +00:00
|
|
|
required_access.emplace_back(AccessType::SELECT | AccessType::DROP_DATABASE, elem.from.getDatabase());
|
|
|
|
required_access.emplace_back(AccessType::CREATE_DATABASE | AccessType::INSERT, elem.to.getDatabase());
|
2022-09-05 10:38:44 +00:00
|
|
|
}
|
|
|
|
else
|
2020-07-07 12:11:58 +00:00
|
|
|
{
|
2022-09-05 10:38:44 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown type of rename query");
|
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
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
void InterpreterRenameQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const
|
2020-12-14 03:30:39 +00:00
|
|
|
{
|
|
|
|
const auto & rename = ast->as<const ASTRenameQuery &>();
|
2024-03-08 11:54:20 +00:00
|
|
|
for (const auto & element : rename.getElements())
|
2020-12-14 03:30:39 +00:00
|
|
|
{
|
|
|
|
{
|
2023-02-14 13:01:06 +00:00
|
|
|
String database = backQuoteIfNeed(!element.from.database ? getContext()->getCurrentDatabase() : element.from.getDatabase());
|
2020-12-14 03:30:39 +00:00
|
|
|
elem.query_databases.insert(database);
|
2023-02-14 13:01:06 +00:00
|
|
|
elem.query_tables.insert(database + "." + backQuoteIfNeed(element.from.getTable()));
|
2020-12-14 03:30:39 +00:00
|
|
|
}
|
|
|
|
{
|
2023-02-14 13:01:06 +00:00
|
|
|
String database = backQuoteIfNeed(!element.to.database ? getContext()->getCurrentDatabase() : element.to.getDatabase());
|
2020-12-14 03:30:39 +00:00
|
|
|
elem.query_databases.insert(database);
|
2023-02-14 13:01:06 +00:00
|
|
|
elem.query_tables.insert(database + "." + backQuoteIfNeed(element.to.getTable()));
|
2020-12-14 03:30:39 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2024-01-09 06:33:48 +00:00
|
|
|
void registerInterpreterRenameQuery(InterpreterFactory & factory)
|
|
|
|
{
|
|
|
|
auto create_fn = [] (const InterpreterFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
return std::make_unique<InterpreterRenameQuery>(args.query, args.context);
|
|
|
|
};
|
|
|
|
factory.registerInterpreter("InterpreterRenameQuery", create_fn);
|
|
|
|
}
|
|
|
|
|
2012-06-18 06:20:23 +00:00
|
|
|
}
|