2011-11-05 23:31:19 +00:00
|
|
|
#include <Poco/File.h>
|
|
|
|
|
2017-12-20 07:39:52 +00:00
|
|
|
#include <Databases/IDatabase.h>
|
2017-05-23 18:01:50 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2020-11-03 13:47:26 +00:00
|
|
|
#include <Interpreters/executeDDLQueryOnCluster.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/InterpreterDropQuery.h>
|
2019-10-15 18:04:17 +00:00
|
|
|
#include <Interpreters/ExternalDictionariesLoader.h>
|
2020-12-14 03:30:39 +00:00
|
|
|
#include <Interpreters/QueryLog.h>
|
2020-01-26 09:49:53 +00:00
|
|
|
#include <Access/AccessRightsElement.h>
|
2017-12-20 07:39:52 +00:00
|
|
|
#include <Parsers/ASTDropQuery.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/IStorage.h>
|
2017-12-20 07:39:52 +00:00
|
|
|
#include <Common/escapeForFileName.h>
|
2019-10-08 18:42:22 +00:00
|
|
|
#include <Common/quoteString.h>
|
2017-12-20 07:39:52 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2020-07-04 16:32:23 +00:00
|
|
|
#include <Databases/DatabaseReplicated.h>
|
2011-11-05 23:31:19 +00:00
|
|
|
|
2020-09-15 14:22:20 +00:00
|
|
|
#if !defined(ARCADIA_BUILD)
|
|
|
|
# include "config_core.h"
|
|
|
|
#endif
|
|
|
|
|
|
|
|
#if USE_MYSQL
|
|
|
|
# include <Databases/MySQL/DatabaseMaterializeMySQL.h>
|
|
|
|
#endif
|
2011-11-05 23:31:19 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2017-12-22 19:20:18 +00:00
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2018-04-21 00:35:20 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2018-06-09 16:03:07 +00:00
|
|
|
extern const int SYNTAX_ERROR;
|
2018-04-21 00:35:20 +00:00
|
|
|
extern const int UNKNOWN_TABLE;
|
2019-10-11 13:21:52 +00:00
|
|
|
extern const int UNKNOWN_DICTIONARY;
|
2020-12-07 13:55:24 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2021-02-01 19:29:47 +00:00
|
|
|
extern const int INCORRECT_QUERY;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2011-11-05 23:31:19 +00:00
|
|
|
|
2020-03-18 00:57:00 +00:00
|
|
|
static DatabasePtr tryGetDatabase(const String & database_name, bool if_exists)
|
|
|
|
{
|
|
|
|
return if_exists ? DatabaseCatalog::instance().tryGetDatabase(database_name) : DatabaseCatalog::instance().getDatabase(database_name);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-12-20 07:39:52 +00:00
|
|
|
InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {}
|
2011-11-05 23:31:19 +00:00
|
|
|
|
|
|
|
|
2015-06-18 02:11:05 +00:00
|
|
|
BlockIO InterpreterDropQuery::execute()
|
2017-04-13 16:12:56 +00:00
|
|
|
{
|
2019-03-15 16:14:13 +00:00
|
|
|
auto & drop = query_ptr->as<ASTDropQuery &>();
|
|
|
|
if (!drop.cluster.empty())
|
2020-01-24 16:20:36 +00:00
|
|
|
return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccessForDDLOnCluster());
|
2017-04-13 16:12:56 +00:00
|
|
|
|
2020-09-29 22:30:34 +00:00
|
|
|
if (context.getSettingsRef().database_atomic_wait_for_drop_and_detach_synchronously)
|
|
|
|
drop.no_delay = true;
|
|
|
|
|
2019-03-15 16:14:13 +00:00
|
|
|
if (!drop.table.empty())
|
2019-10-11 13:21:52 +00:00
|
|
|
{
|
|
|
|
if (!drop.is_dictionary)
|
2020-10-13 13:31:02 +00:00
|
|
|
return executeToTable(drop);
|
2020-12-07 13:55:24 +00:00
|
|
|
else if (drop.permanently && drop.kind == ASTDropQuery::Kind::Detach)
|
|
|
|
throw Exception("DETACH PERMANENTLY is not implemented for dictionaries", ErrorCodes::NOT_IMPLEMENTED);
|
2019-10-11 13:21:52 +00:00
|
|
|
else
|
|
|
|
return executeToDictionary(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
|
|
|
|
}
|
2019-03-15 16:14:13 +00:00
|
|
|
else if (!drop.database.empty())
|
2020-10-27 19:56:54 +00:00
|
|
|
return executeToDatabase(drop);
|
2018-04-21 00:35:20 +00:00
|
|
|
else
|
2020-03-23 22:28:30 +00:00
|
|
|
throw Exception("Nothing to drop, both names are empty", ErrorCodes::LOGICAL_ERROR);
|
2018-04-21 00:35:20 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
|
2020-10-27 19:56:54 +00:00
|
|
|
void InterpreterDropQuery::waitForTableToBeActuallyDroppedOrDetached(const ASTDropQuery & query, const DatabasePtr & db, const UUID & uuid_to_wait)
|
|
|
|
{
|
|
|
|
if (uuid_to_wait == UUIDHelpers::Nil)
|
|
|
|
return;
|
|
|
|
|
|
|
|
if (query.kind == ASTDropQuery::Kind::Drop)
|
|
|
|
DatabaseCatalog::instance().waitTableFinallyDropped(uuid_to_wait);
|
|
|
|
else if (query.kind == ASTDropQuery::Kind::Detach)
|
2020-11-24 12:28:31 +00:00
|
|
|
db->waitDetachedTableNotInUse(uuid_to_wait);
|
2020-10-27 19:56:54 +00:00
|
|
|
}
|
|
|
|
|
2020-10-13 13:31:02 +00:00
|
|
|
BlockIO InterpreterDropQuery::executeToTable(const ASTDropQuery & query)
|
2020-10-27 19:56:54 +00:00
|
|
|
{
|
|
|
|
DatabasePtr database;
|
|
|
|
UUID table_to_wait_on = UUIDHelpers::Nil;
|
|
|
|
auto res = executeToTableImpl(query, database, table_to_wait_on);
|
|
|
|
if (query.no_delay)
|
|
|
|
waitForTableToBeActuallyDroppedOrDetached(query, database, table_to_wait_on);
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
BlockIO InterpreterDropQuery::executeToTableImpl(const ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait)
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
2020-10-23 12:53:12 +00:00
|
|
|
/// NOTE: it does not contain UUID, we will resolve it with locked DDLGuard
|
2020-10-13 13:31:02 +00:00
|
|
|
auto table_id = StorageID(query);
|
|
|
|
if (query.temporary || table_id.database_name.empty())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-10-13 13:31:02 +00:00
|
|
|
if (context.tryResolveStorageID(table_id, Context::ResolveExternal))
|
|
|
|
return executeToTemporaryTable(table_id.getTableName(), query.kind);
|
|
|
|
else
|
|
|
|
table_id.database_name = context.getCurrentDatabase();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-03-20 00:07:52 +00:00
|
|
|
if (query.temporary)
|
2020-01-29 19:54:40 +00:00
|
|
|
{
|
2020-03-20 00:07:52 +00:00
|
|
|
if (query.if_exists)
|
2020-01-24 16:20:36 +00:00
|
|
|
return {};
|
2020-10-13 13:31:02 +00:00
|
|
|
throw Exception("Temporary table " + backQuoteIfNeed(table_id.table_name) + " doesn't exist",
|
2020-01-29 19:54:40 +00:00
|
|
|
ErrorCodes::UNKNOWN_TABLE);
|
|
|
|
}
|
|
|
|
|
2020-03-30 14:53:05 +00:00
|
|
|
auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr);
|
2018-09-18 18:33:15 +00:00
|
|
|
|
2020-04-13 14:09:56 +00:00
|
|
|
/// If table was already dropped by anyone, an exception will be thrown
|
2020-05-28 23:01:18 +00:00
|
|
|
auto [database, table] = query.if_exists ? DatabaseCatalog::instance().tryGetDatabaseAndTable(table_id, context)
|
|
|
|
: DatabaseCatalog::instance().getDatabaseAndTable(table_id, context);
|
2018-04-21 00:35:20 +00:00
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
if (database && table)
|
2017-08-15 12:34:28 +00:00
|
|
|
{
|
2021-02-01 19:29:47 +00:00
|
|
|
if (query.as<ASTDropQuery &>().is_view && !table->isView())
|
2020-03-24 14:23:18 +00:00
|
|
|
throw Exception("Table " + table_id.getNameForLogs() + " is not a View", ErrorCodes::LOGICAL_ERROR);
|
2020-03-23 22:28:30 +00:00
|
|
|
|
2020-10-23 12:53:12 +00:00
|
|
|
/// Now get UUID, so we can wait for table data to be finally dropped
|
2020-10-23 14:19:02 +00:00
|
|
|
table_id.uuid = database->tryGetTableUUID(table_id.table_name);
|
2020-10-13 17:45:59 +00:00
|
|
|
|
2021-02-01 19:29:47 +00:00
|
|
|
/// Prevents recursive drop from drop database query. The original query must specify a table.
|
2021-02-02 10:32:42 +00:00
|
|
|
bool is_drop_or_detach_database = query_ptr->as<ASTDropQuery>()->table.empty();
|
2021-02-01 19:29:47 +00:00
|
|
|
bool is_replicated_ddl_query = typeid_cast<DatabaseReplicated *>(database.get()) &&
|
|
|
|
context.getClientInfo().query_kind != ClientInfo::QueryKind::REPLICATED_LOG_QUERY &&
|
|
|
|
!is_drop_or_detach_database;
|
|
|
|
if (is_replicated_ddl_query)
|
|
|
|
{
|
|
|
|
if (query.kind == ASTDropQuery::Kind::Detach && !query.permanently)
|
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY, "DETACH TABLE is not allowed for Replicated databases. "
|
|
|
|
"Use DETACH TABLE PERMANENTLY or SYSTEM RESTART REPLICA");
|
|
|
|
|
|
|
|
ddl_guard.reset();
|
|
|
|
table.reset();
|
|
|
|
return typeid_cast<DatabaseReplicated *>(database.get())->propose(query.clone());
|
|
|
|
}
|
|
|
|
|
2020-03-20 00:07:52 +00:00
|
|
|
if (query.kind == ASTDropQuery::Kind::Detach)
|
2017-08-15 17:00:18 +00:00
|
|
|
{
|
2020-03-13 15:41:36 +00:00
|
|
|
context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id);
|
2020-11-11 14:34:58 +00:00
|
|
|
table->checkTableCanBeDetached();
|
2020-01-24 16:20:36 +00:00
|
|
|
table->shutdown();
|
2020-06-27 13:39:41 +00:00
|
|
|
TableExclusiveLockHolder table_lock;
|
2021-01-18 14:09:39 +00:00
|
|
|
|
2020-09-14 19:25:02 +00:00
|
|
|
if (database->getUUID() == UUIDHelpers::Nil)
|
2020-04-12 13:35:44 +00:00
|
|
|
table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
2020-11-30 17:52:32 +00:00
|
|
|
|
|
|
|
if (query.permanently)
|
|
|
|
{
|
|
|
|
/// Drop table from memory, don't touch data, metadata file renamed and will be skipped during server restart
|
|
|
|
database->detachTablePermanently(table_id.table_name);
|
|
|
|
}
|
2020-10-22 15:08:00 +00:00
|
|
|
else
|
2020-11-30 17:52:32 +00:00
|
|
|
{
|
|
|
|
/// Drop table from memory, don't touch data and metadata
|
2020-06-07 11:20:05 +00:00
|
|
|
database->detachTable(table_id.table_name);
|
2020-11-30 17:52:32 +00:00
|
|
|
}
|
2018-04-21 00:35:20 +00:00
|
|
|
}
|
2020-03-20 00:07:52 +00:00
|
|
|
else if (query.kind == ASTDropQuery::Kind::Truncate)
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
2020-04-02 18:56:56 +00:00
|
|
|
context.checkAccess(AccessType::TRUNCATE, table_id);
|
2020-01-24 16:20:36 +00:00
|
|
|
table->checkTableCanBeDropped();
|
2018-06-09 15:48:22 +00:00
|
|
|
|
2020-04-09 18:10:27 +00:00
|
|
|
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
2020-06-18 10:29:13 +00:00
|
|
|
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
2018-04-21 00:35:20 +00:00
|
|
|
/// Drop table data, don't touch metadata
|
2021-02-01 19:29:47 +00:00
|
|
|
table->truncate(query_ptr, metadata_snapshot, context, table_lock);
|
2018-04-21 00:35:20 +00:00
|
|
|
}
|
2020-03-20 00:07:52 +00:00
|
|
|
else if (query.kind == ASTDropQuery::Kind::Drop)
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
2020-03-13 15:41:36 +00:00
|
|
|
context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id);
|
2020-01-24 16:20:36 +00:00
|
|
|
table->checkTableCanBeDropped();
|
2018-04-21 00:35:20 +00:00
|
|
|
|
2020-03-30 14:53:05 +00:00
|
|
|
table->shutdown();
|
|
|
|
|
2020-06-18 16:10:47 +00:00
|
|
|
TableExclusiveLockHolder table_lock;
|
2020-09-14 19:25:02 +00:00
|
|
|
if (database->getUUID() == UUIDHelpers::Nil)
|
2020-04-12 13:35:44 +00:00
|
|
|
table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
2018-11-28 15:59:57 +00:00
|
|
|
|
2021-02-01 19:29:47 +00:00
|
|
|
database->dropTable(context, table_id.table_name, query.no_delay);
|
2018-04-21 00:35:20 +00:00
|
|
|
}
|
|
|
|
|
2020-10-27 19:56:54 +00:00
|
|
|
db = database;
|
|
|
|
uuid_to_wait = table_id.uuid;
|
2020-09-29 13:42:58 +00:00
|
|
|
}
|
|
|
|
|
2018-04-21 00:35:20 +00:00
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2019-10-11 13:21:52 +00:00
|
|
|
|
|
|
|
BlockIO InterpreterDropQuery::executeToDictionary(
|
2020-01-24 16:20:36 +00:00
|
|
|
const String & database_name_,
|
|
|
|
const String & dictionary_name,
|
2019-10-11 13:21:52 +00:00
|
|
|
ASTDropQuery::Kind kind,
|
|
|
|
bool if_exists,
|
|
|
|
bool is_temporary,
|
|
|
|
bool no_ddl_lock)
|
|
|
|
{
|
|
|
|
if (is_temporary)
|
|
|
|
throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR);
|
|
|
|
|
2020-02-10 13:10:17 +00:00
|
|
|
String database_name = context.resolveDatabase(database_name_);
|
2019-10-11 13:21:52 +00:00
|
|
|
|
2020-02-10 18:31:52 +00:00
|
|
|
auto ddl_guard = (!no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(database_name, dictionary_name) : nullptr);
|
2019-10-11 13:21:52 +00:00
|
|
|
|
2019-12-12 16:50:20 +00:00
|
|
|
DatabasePtr database = tryGetDatabase(database_name, if_exists);
|
2019-10-11 13:21:52 +00:00
|
|
|
|
2020-04-23 16:51:48 +00:00
|
|
|
if (!database || !database->isDictionaryExist(dictionary_name))
|
2019-10-11 13:21:52 +00:00
|
|
|
{
|
|
|
|
if (!if_exists)
|
|
|
|
throw Exception(
|
|
|
|
"Dictionary " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(dictionary_name) + " doesn't exist.",
|
|
|
|
ErrorCodes::UNKNOWN_DICTIONARY);
|
|
|
|
else
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
|
|
|
if (kind == ASTDropQuery::Kind::Detach)
|
|
|
|
{
|
|
|
|
/// Drop dictionary from memory, don't touch data and metadata
|
2020-02-19 21:11:29 +00:00
|
|
|
context.checkAccess(AccessType::DROP_DICTIONARY, database_name, dictionary_name);
|
2020-04-12 20:50:32 +00:00
|
|
|
database->detachDictionary(dictionary_name);
|
2019-10-11 13:21:52 +00:00
|
|
|
}
|
|
|
|
else if (kind == ASTDropQuery::Kind::Truncate)
|
|
|
|
{
|
|
|
|
throw Exception("Cannot TRUNCATE dictionary", ErrorCodes::SYNTAX_ERROR);
|
|
|
|
}
|
|
|
|
else if (kind == ASTDropQuery::Kind::Drop)
|
|
|
|
{
|
2020-01-24 16:20:36 +00:00
|
|
|
context.checkAccess(AccessType::DROP_DICTIONARY, database_name, dictionary_name);
|
2019-10-11 13:21:52 +00:00
|
|
|
database->removeDictionary(context, dictionary_name);
|
|
|
|
}
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name, ASTDropQuery::Kind kind)
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
|
|
|
if (kind == ASTDropQuery::Kind::Detach)
|
|
|
|
throw Exception("Unable to detach temporary table.", ErrorCodes::SYNTAX_ERROR);
|
2018-07-10 14:24:19 +00:00
|
|
|
else
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
2018-07-10 14:24:19 +00:00
|
|
|
auto & context_handle = context.hasSessionContext() ? context.getSessionContext() : context;
|
2020-02-12 18:14:12 +00:00
|
|
|
auto resolved_id = context_handle.tryResolveStorageID(StorageID("", table_name), Context::ResolveExternal);
|
|
|
|
if (resolved_id)
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
2020-05-28 23:01:18 +00:00
|
|
|
StoragePtr table = DatabaseCatalog::instance().getTable(resolved_id, context);
|
2018-07-10 14:24:19 +00:00
|
|
|
if (kind == ASTDropQuery::Kind::Truncate)
|
|
|
|
{
|
2020-04-12 13:35:44 +00:00
|
|
|
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
2018-07-10 14:24:19 +00:00
|
|
|
/// Drop table data, don't touch metadata
|
2020-06-18 10:29:13 +00:00
|
|
|
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
|
|
|
table->truncate(query_ptr, metadata_snapshot, context, table_lock);
|
2018-07-10 14:24:19 +00:00
|
|
|
}
|
|
|
|
else if (kind == ASTDropQuery::Kind::Drop)
|
|
|
|
{
|
2020-01-30 19:00:51 +00:00
|
|
|
context_handle.removeExternalTable(table_name);
|
2018-07-10 14:24:19 +00:00
|
|
|
table->shutdown();
|
2020-04-12 13:35:44 +00:00
|
|
|
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
2018-07-10 14:24:19 +00:00
|
|
|
/// Delete table data
|
2020-01-22 11:30:11 +00:00
|
|
|
table->drop();
|
2018-07-10 14:24:19 +00:00
|
|
|
table->is_dropped = true;
|
|
|
|
}
|
2017-08-15 17:00:18 +00:00
|
|
|
}
|
2017-08-15 12:34:28 +00:00
|
|
|
}
|
|
|
|
|
2018-04-21 00:35:20 +00:00
|
|
|
return {};
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-03-18 00:57:00 +00:00
|
|
|
|
2020-10-27 19:56:54 +00:00
|
|
|
BlockIO InterpreterDropQuery::executeToDatabase(const ASTDropQuery & query)
|
|
|
|
{
|
|
|
|
DatabasePtr database;
|
|
|
|
std::vector<UUID> tables_to_wait;
|
|
|
|
BlockIO res;
|
|
|
|
try
|
|
|
|
{
|
|
|
|
res = executeToDatabaseImpl(query, database, tables_to_wait);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
if (query.no_delay)
|
|
|
|
{
|
2020-10-29 13:37:59 +00:00
|
|
|
for (const auto & table_uuid : tables_to_wait)
|
2020-10-27 19:56:54 +00:00
|
|
|
waitForTableToBeActuallyDroppedOrDetached(query, database, table_uuid);
|
|
|
|
}
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (query.no_delay)
|
|
|
|
{
|
2020-10-29 13:37:59 +00:00
|
|
|
for (const auto & table_uuid : tables_to_wait)
|
2020-10-27 19:56:54 +00:00
|
|
|
waitForTableToBeActuallyDroppedOrDetached(query, database, table_uuid);
|
|
|
|
}
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, DatabasePtr & database, std::vector<UUID> & uuids_to_wait)
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
2020-10-27 19:56:54 +00:00
|
|
|
const auto & database_name = query.database;
|
2020-02-10 18:31:52 +00:00
|
|
|
auto ddl_guard = DatabaseCatalog::instance().getDDLGuard(database_name, "");
|
2018-09-18 18:33:15 +00:00
|
|
|
|
2020-10-27 19:56:54 +00:00
|
|
|
database = tryGetDatabase(database_name, query.if_exists);
|
|
|
|
if (database)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-10-27 19:56:54 +00:00
|
|
|
if (query.kind == ASTDropQuery::Kind::Truncate)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-03-23 22:28:30 +00:00
|
|
|
throw Exception("Unable to truncate database", ErrorCodes::SYNTAX_ERROR);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2020-10-27 19:56:54 +00:00
|
|
|
else if (query.kind == ASTDropQuery::Kind::Detach || query.kind == ASTDropQuery::Kind::Drop)
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
2020-10-27 19:56:54 +00:00
|
|
|
bool drop = query.kind == ASTDropQuery::Kind::Drop;
|
2020-01-24 16:20:36 +00:00
|
|
|
context.checkAccess(AccessType::DROP_DATABASE, database_name);
|
|
|
|
|
2020-12-07 13:55:24 +00:00
|
|
|
if (query.kind == ASTDropQuery::Kind::Detach && query.permanently)
|
|
|
|
throw Exception("DETACH PERMANENTLY is not implemented for databases", ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
|
2020-09-15 14:22:20 +00:00
|
|
|
#if USE_MYSQL
|
2020-09-14 19:25:02 +00:00
|
|
|
if (database->getEngineName() == "MaterializeMySQL")
|
|
|
|
stopDatabaseSynchronization(database);
|
2020-09-15 14:22:20 +00:00
|
|
|
#endif
|
2020-09-14 19:25:02 +00:00
|
|
|
|
2020-04-12 20:50:32 +00:00
|
|
|
if (database->shouldBeEmptyOnDetach())
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
2020-04-12 20:50:32 +00:00
|
|
|
/// DETACH or DROP all tables and dictionaries inside database.
|
|
|
|
/// First we should DETACH or DROP dictionaries because StorageDictionary
|
|
|
|
/// must be detached only by detaching corresponding dictionary.
|
2020-04-20 11:09:09 +00:00
|
|
|
for (auto iterator = database->getDictionariesIterator(); iterator->isValid(); iterator->next())
|
2020-04-12 20:50:32 +00:00
|
|
|
{
|
|
|
|
String current_dictionary = iterator->name();
|
2020-10-27 19:56:54 +00:00
|
|
|
executeToDictionary(database_name, current_dictionary, query.kind, false, false, false);
|
2020-04-12 20:50:32 +00:00
|
|
|
}
|
|
|
|
|
2020-10-27 19:56:54 +00:00
|
|
|
ASTDropQuery query_for_table;
|
|
|
|
query_for_table.kind = query.kind;
|
|
|
|
query_for_table.if_exists = true;
|
|
|
|
query_for_table.database = database_name;
|
|
|
|
query_for_table.no_delay = query.no_delay;
|
2020-10-18 20:18:02 +00:00
|
|
|
|
2020-05-28 20:10:45 +00:00
|
|
|
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
2020-04-12 20:50:32 +00:00
|
|
|
{
|
2020-10-27 19:56:54 +00:00
|
|
|
DatabasePtr db;
|
|
|
|
UUID table_to_wait = UUIDHelpers::Nil;
|
|
|
|
query_for_table.table = iterator->name();
|
|
|
|
executeToTableImpl(query_for_table, db, table_to_wait);
|
|
|
|
uuids_to_wait.push_back(table_to_wait);
|
2020-04-12 20:50:32 +00:00
|
|
|
}
|
2019-10-11 15:22:30 +00:00
|
|
|
}
|
|
|
|
|
2020-08-18 15:15:27 +00:00
|
|
|
/// Protects from concurrent CREATE TABLE queries
|
|
|
|
auto db_guard = DatabaseCatalog::instance().getExclusiveDDLGuardForDatabase(database_name);
|
|
|
|
|
2020-09-14 19:25:02 +00:00
|
|
|
if (!drop)
|
|
|
|
database->assertCanBeDetached(true);
|
2020-04-10 23:02:15 +00:00
|
|
|
|
2020-02-13 21:00:03 +00:00
|
|
|
/// DETACH or DROP database itself
|
2020-04-12 20:50:32 +00:00
|
|
|
DatabaseCatalog::instance().detachDatabase(database_name, drop, database->shouldBeEmptyOnDetach());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-04-21 00:35:20 +00:00
|
|
|
return {};
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
AccessRightsElements InterpreterDropQuery::getRequiredAccessForDDLOnCluster() const
|
|
|
|
{
|
|
|
|
AccessRightsElements required_access;
|
|
|
|
const auto & drop = query_ptr->as<const ASTDropQuery &>();
|
2017-12-20 07:39:52 +00:00
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
if (drop.table.empty())
|
|
|
|
{
|
|
|
|
if (drop.kind == ASTDropQuery::Kind::Detach)
|
2020-02-19 21:11:29 +00:00
|
|
|
required_access.emplace_back(AccessType::DROP_DATABASE, drop.database);
|
2020-01-24 16:20:36 +00:00
|
|
|
else if (drop.kind == ASTDropQuery::Kind::Drop)
|
|
|
|
required_access.emplace_back(AccessType::DROP_DATABASE, drop.database);
|
|
|
|
}
|
|
|
|
else if (drop.is_dictionary)
|
|
|
|
{
|
|
|
|
if (drop.kind == ASTDropQuery::Kind::Detach)
|
2020-02-19 21:11:29 +00:00
|
|
|
required_access.emplace_back(AccessType::DROP_DICTIONARY, drop.database, drop.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
else if (drop.kind == ASTDropQuery::Kind::Drop)
|
|
|
|
required_access.emplace_back(AccessType::DROP_DICTIONARY, drop.database, drop.table);
|
|
|
|
}
|
|
|
|
else if (!drop.temporary)
|
|
|
|
{
|
|
|
|
/// It can be view or table.
|
|
|
|
if (drop.kind == ASTDropQuery::Kind::Drop)
|
|
|
|
required_access.emplace_back(AccessType::DROP_TABLE | AccessType::DROP_VIEW, drop.database, drop.table);
|
|
|
|
else if (drop.kind == ASTDropQuery::Kind::Truncate)
|
2020-04-02 18:56:56 +00:00
|
|
|
required_access.emplace_back(AccessType::TRUNCATE, drop.database, drop.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
else if (drop.kind == ASTDropQuery::Kind::Detach)
|
2020-02-19 21:11:29 +00:00
|
|
|
required_access.emplace_back(AccessType::DROP_TABLE | AccessType::DROP_VIEW, drop.database, drop.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
}
|
2018-09-11 18:37:19 +00:00
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
return required_access;
|
2017-12-20 07:39:52 +00:00
|
|
|
}
|
2017-12-22 19:20:18 +00:00
|
|
|
|
2020-12-14 03:30:39 +00:00
|
|
|
void InterpreterDropQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, const Context &) const
|
|
|
|
{
|
|
|
|
elem.query_kind = "Drop";
|
|
|
|
}
|
|
|
|
|
2011-11-05 23:31:19 +00:00
|
|
|
}
|