ClickHouse/src/Interpreters/InterpreterDropQuery.cpp

458 lines
18 KiB
C++
Raw Normal View History

#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>
#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>
#include <Access/Common/AccessRightsElement.h>
#include <Parsers/ASTDropQuery.h>
#include <Storages/IStorage.h>
#include <Common/escapeForFileName.h>
#include <Common/quoteString.h>
#include <Common/typeid_cast.h>
#include <Databases/DatabaseReplicated.h>
2020-09-15 14:22:20 +00:00
2021-10-27 23:10:39 +00:00
#include "config_core.h"
2020-09-15 14:22:20 +00:00
#if USE_MYSQL
# include <Databases/MySQL/DatabaseMaterializedMySQL.h>
2020-09-15 14:22:20 +00:00
#endif
2011-11-05 23:31:19 +00:00
#if USE_LIBPQXX
2021-06-27 19:09:17 +00:00
# include <Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h>
#endif
2011-11-05 23:31:19 +00:00
namespace DB
{
2017-12-22 19:20:18 +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;
2020-12-07 13:55:24 +00:00
extern const int NOT_IMPLEMENTED;
extern const int INCORRECT_QUERY;
2021-08-23 11:26:54 +00:00
extern const int TABLE_IS_READ_ONLY;
}
namespace ActionLocks
{
extern const StorageActionBlockType PartsMerge;
}
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);
}
2021-05-31 14:49:02 +00:00
InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_)
{
}
2011-11-05 23:31:19 +00:00
2015-06-18 02:11:05 +00:00
BlockIO InterpreterDropQuery::execute()
{
auto & drop = query_ptr->as<ASTDropQuery &>();
if (!drop.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, getContext(), getRequiredAccessForDDLOnCluster());
if (getContext()->getSettingsRef().database_atomic_wait_for_drop_and_detach_synchronously)
2020-09-29 22:30:34 +00:00
drop.no_delay = true;
2021-11-11 13:28:18 +00:00
if (drop.table)
2021-04-21 13:45:13 +00:00
return executeToTable(drop);
2021-11-11 13:28:18 +00:00
else if (drop.database)
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
}
2020-10-27 19:56:54 +00:00
void InterpreterDropQuery::waitForTableToBeActuallyDroppedOrDetached(const ASTDropQuery & query, const DatabasePtr & db, const UUID & uuid_to_wait)
2018-04-21 00:35:20 +00:00
{
2020-10-27 19:56:54 +00:00
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)
db->waitDetachedTableNotInUse(uuid_to_wait);
2020-10-27 19:56:54 +00:00
}
2021-03-16 22:01:48 +00:00
BlockIO InterpreterDropQuery::executeToTable(ASTDropQuery & query)
2020-10-27 19:56:54 +00:00
{
DatabasePtr database;
UUID table_to_wait_on = UUIDHelpers::Nil;
auto res = executeToTableImpl(getContext(), query, database, table_to_wait_on);
2020-10-27 19:56:54 +00:00
if (query.no_delay)
waitForTableToBeActuallyDroppedOrDetached(query, database, table_to_wait_on);
return res;
}
BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, 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())
{
if (context_->tryResolveStorageID(table_id, Context::ResolveExternal))
2020-10-13 13:31:02 +00:00
return executeToTemporaryTable(table_id.getTableName(), query.kind);
else
query.setDatabase(table_id.database_name = context_->getCurrentDatabase());
}
2020-03-20 00:07:52 +00:00
if (query.temporary)
{
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",
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
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)
{
2021-04-22 14:09:30 +00:00
auto & ast_drop_query = query.as<ASTDropQuery &>();
if (ast_drop_query.is_view && !table->isView())
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Table {} is not a View",
table_id.getNameForLogs());
if (ast_drop_query.is_dictionary && !table->isDictionary())
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Table {} is not a Dictionary",
table_id.getNameForLogs());
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
/// Prevents recursive drop from drop database query. The original query must specify a table.
2021-11-11 13:28:18 +00:00
bool is_drop_or_detach_database = !query_ptr->as<ASTDropQuery>()->table;
bool is_replicated_ddl_query = typeid_cast<DatabaseReplicated *>(database.get()) &&
!context_->getClientInfo().is_replicated_database_internal &&
!is_drop_or_detach_database;
2021-04-21 13:45:13 +00:00
AccessFlags drop_storage;
if (table->isView())
drop_storage = AccessType::DROP_VIEW;
else if (table->isDictionary())
drop_storage = AccessType::DROP_DICTIONARY;
else
drop_storage = AccessType::DROP_TABLE;
if (is_replicated_ddl_query)
{
2021-02-08 19:36:17 +00:00
if (query.kind == ASTDropQuery::Kind::Detach)
context_->checkAccess(drop_storage, table_id);
2021-02-08 19:36:17 +00:00
else if (query.kind == ASTDropQuery::Kind::Truncate)
context_->checkAccess(AccessType::TRUNCATE, table_id);
2021-02-08 19:36:17 +00:00
else if (query.kind == ASTDropQuery::Kind::Drop)
context_->checkAccess(drop_storage, table_id);
2021-02-08 19:36:17 +00:00
2021-02-02 19:39:04 +00:00
ddl_guard->releaseTableLock();
table.reset();
return typeid_cast<DatabaseReplicated *>(database.get())->tryEnqueueReplicatedDDL(query.clone(), context_);
}
2020-03-20 00:07:52 +00:00
if (query.kind == ASTDropQuery::Kind::Detach)
{
context_->checkAccess(drop_storage, table_id);
2021-04-21 13:45:13 +00:00
2021-04-22 09:19:59 +00:00
if (table->isDictionary())
{
2021-04-26 10:20:57 +00:00
/// If DROP DICTIONARY query is not used, check if Dictionary can be dropped with DROP TABLE query
2021-04-22 09:19:59 +00:00
if (!query.is_dictionary)
2021-04-27 19:09:03 +00:00
table->checkTableCanBeDetached();
2021-04-22 09:19:59 +00:00
}
else
2021-04-27 19:09:03 +00:00
table->checkTableCanBeDetached();
2021-04-21 13:45:13 +00:00
2021-05-13 07:03:00 +00:00
table->flushAndShutdown();
2020-06-18 16:10:47 +00:00
TableExclusiveLockHolder table_lock;
if (database->getUUID() == UUIDHelpers::Nil)
table_lock = table->lockExclusively(context_->getCurrentQueryId(), context_->getSettingsRef().lock_acquire_timeout);
2020-11-30 17:52:32 +00:00
if (query.permanently)
{
/// Server may fail to restart of DETACH PERMANENTLY if table has dependent ones
2021-11-02 12:58:45 +00:00
DatabaseCatalog::instance().tryRemoveLoadingDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies,
is_drop_or_detach_database);
2020-11-30 17:52:32 +00:00
/// Drop table from memory, don't touch data, metadata file renamed and will be skipped during server restart
database->detachTablePermanently(context_, table_id.table_name);
2020-11-30 17:52:32 +00:00
}
else
{
/// Drop table from memory, don't touch data and metadata
database->detachTable(context_, 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
{
if (table->isDictionary())
throw Exception("Cannot TRUNCATE dictionary", ErrorCodes::SYNTAX_ERROR);
context_->checkAccess(AccessType::TRUNCATE, table_id);
2021-09-04 09:02:07 +00:00
if (table->isStaticStorage())
2021-08-23 11:26:54 +00:00
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only");
2021-04-21 13:45:13 +00:00
2020-01-24 16:20:36 +00:00
table->checkTableCanBeDropped();
2018-06-09 15:48:22 +00:00
TableExclusiveLockHolder table_lock;
/// We don't need this lock for ReplicatedMergeTree
if (!table->supportsReplication())
{
/// And for simple MergeTree we can stop merges before acquiring the lock
auto merges_blocker = table->getActionLock(ActionLocks::PartsMerge);
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
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
{
context_->checkAccess(drop_storage, table_id);
2021-04-21 13:45:13 +00:00
2021-04-22 09:19:59 +00:00
if (table->isDictionary())
{
2021-04-26 10:20:57 +00:00
/// If DROP DICTIONARY query is not used, check if Dictionary can be dropped with DROP TABLE query
2021-04-22 09:19:59 +00:00
if (!query.is_dictionary)
table->checkTableCanBeDropped();
}
else
2021-04-21 13:45:13 +00:00
table->checkTableCanBeDropped();
2018-04-21 00:35:20 +00:00
2021-05-13 07:03:00 +00:00
table->flushAndShutdown();
2020-03-30 14:53:05 +00:00
2020-06-18 16:10:47 +00:00
TableExclusiveLockHolder table_lock;
if (database->getUUID() == UUIDHelpers::Nil)
table_lock = table->lockExclusively(context_->getCurrentQueryId(), context_->getSettingsRef().lock_acquire_timeout);
2021-11-02 12:58:45 +00:00
DatabaseCatalog::instance().tryRemoveLoadingDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies,
is_drop_or_detach_database);
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;
}
2018-04-21 00:35:20 +00:00
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);
else
2018-04-21 00:35:20 +00:00
{
auto context_handle = getContext()->hasSessionContext() ? getContext()->getSessionContext() : getContext();
auto resolved_id = context_handle->tryResolveStorageID(StorageID("", table_name), Context::ResolveExternal);
2020-02-12 18:14:12 +00:00
if (resolved_id)
2018-04-21 00:35:20 +00:00
{
StoragePtr table = DatabaseCatalog::instance().getTable(resolved_id, getContext());
if (kind == ASTDropQuery::Kind::Truncate)
{
auto table_lock
= table->lockExclusively(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout);
/// 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, getContext(), table_lock);
}
else if (kind == ASTDropQuery::Kind::Drop)
{
context_handle->removeExternalTable(table_name);
2021-05-13 07:03:00 +00:00
table->flushAndShutdown();
auto table_lock = table->lockExclusively(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout);
/// Delete table data
2020-01-22 11:30:11 +00:00
table->drop();
table->is_dropped = true;
}
}
}
2018-04-21 00:35:20 +00:00
return {};
}
2020-03-18 00:57:00 +00:00
2020-10-27 19:56:54 +00:00
BlockIO InterpreterDropQuery::executeToDatabase(const ASTDropQuery & query)
2018-04-21 00:35:20 +00:00
{
2020-10-27 19:56:54 +00:00
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
{
const auto & database_name = query.getDatabase();
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)
{
2020-10-27 19:56:54 +00:00
if (query.kind == ASTDropQuery::Kind::Truncate)
{
2020-03-23 22:28:30 +00:00
throw Exception("Unable to truncate database", ErrorCodes::SYNTAX_ERROR);
}
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;
getContext()->checkAccess(AccessType::DROP_DATABASE, database_name);
2020-01-24 16:20:36 +00:00
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);
if (database->hasReplicationThread())
database->stopReplication();
if (database->shouldBeEmptyOnDetach())
2018-04-21 00:35:20 +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.setDatabase(database_name);
2020-10-27 19:56:54 +00:00
query_for_table.no_delay = query.no_delay;
/// Flush should not be done if shouldBeEmptyOnDetach() == false,
/// since in this case getTablesIterator() may do some additional work,
/// see DatabaseMaterializedMySQL::getTablesIterator()
for (auto iterator = database->getTablesIterator(getContext()); iterator->isValid(); iterator->next())
{
iterator->table()->flush();
}
auto table_context = Context::createCopy(getContext());
table_context->setInternalQuery(true);
for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next())
{
2020-10-27 19:56:54 +00:00
DatabasePtr db;
UUID table_to_wait = UUIDHelpers::Nil;
query_for_table.setTable(iterator->name());
2021-04-21 13:45:13 +00:00
query_for_table.is_dictionary = iterator->table()->isDictionary();
executeToTableImpl(table_context, query_for_table, db, table_to_wait);
2020-10-27 19:56:54 +00:00
uuids_to_wait.push_back(table_to_wait);
}
2019-10-11 15:22:30 +00:00
}
if (!drop && query.no_delay)
{
/// Avoid "some tables are still in use" when sync mode is enabled
for (const auto & table_uuid : uuids_to_wait)
database->waitDetachedTableNotInUse(table_uuid);
}
2020-08-18 15:15:27 +00:00
/// Protects from concurrent CREATE TABLE queries
auto db_guard = DatabaseCatalog::instance().getExclusiveDDLGuardForDatabase(database_name);
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
DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach());
}
}
2018-04-21 00:35:20 +00:00
return {};
}
2020-01-24 16:20:36 +00:00
AccessRightsElements InterpreterDropQuery::getRequiredAccessForDDLOnCluster() const
{
AccessRightsElements required_access;
const auto & drop = query_ptr->as<const ASTDropQuery &>();
2021-11-11 13:28:18 +00:00
if (!drop.table)
2020-01-24 16:20:36 +00:00
{
if (drop.kind == ASTDropQuery::Kind::Detach)
required_access.emplace_back(AccessType::DROP_DATABASE, drop.getDatabase());
2020-01-24 16:20:36 +00:00
else if (drop.kind == ASTDropQuery::Kind::Drop)
required_access.emplace_back(AccessType::DROP_DATABASE, drop.getDatabase());
2020-01-24 16:20:36 +00:00
}
else if (drop.is_dictionary)
{
if (drop.kind == ASTDropQuery::Kind::Detach)
required_access.emplace_back(AccessType::DROP_DICTIONARY, drop.getDatabase(), drop.getTable());
2020-01-24 16:20:36 +00:00
else if (drop.kind == ASTDropQuery::Kind::Drop)
required_access.emplace_back(AccessType::DROP_DICTIONARY, drop.getDatabase(), drop.getTable());
2020-01-24 16:20:36 +00:00
}
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.getDatabase(), drop.getTable());
2020-01-24 16:20:36 +00:00
else if (drop.kind == ASTDropQuery::Kind::Truncate)
required_access.emplace_back(AccessType::TRUNCATE, drop.getDatabase(), drop.getTable());
2020-01-24 16:20:36 +00:00
else if (drop.kind == ASTDropQuery::Kind::Detach)
required_access.emplace_back(AccessType::DROP_TABLE | AccessType::DROP_VIEW, drop.getDatabase(), drop.getTable());
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-22 19:20:18 +00:00
void InterpreterDropQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const
2020-12-14 03:30:39 +00:00
{
elem.query_kind = "Drop";
}
2021-04-11 07:44:40 +00:00
void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool no_delay)
{
if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context))
{
/// We create and execute `drop` query for internal table.
auto drop_query = std::make_shared<ASTDropQuery>();
drop_query->setDatabase(target_table_id.database_name);
drop_query->setTable(target_table_id.table_name);
drop_query->kind = kind;
drop_query->no_delay = no_delay;
drop_query->if_exists = true;
ASTPtr ast_drop_query = drop_query;
/// FIXME We have to use global context to execute DROP query for inner table
/// to avoid "Not enough privileges" error if current user has only DROP VIEW ON mat_view_name privilege
/// and not allowed to drop inner table explicitly. Allowing to drop inner table without explicit grant
/// looks like expected behaviour and we have tests for it.
2021-04-11 07:44:40 +00:00
auto drop_context = Context::createCopy(global_context);
drop_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
if (auto txn = current_context->getZooKeeperMetadataTransaction())
{
/// For Replicated database
2021-07-30 16:34:18 +00:00
drop_context->getClientInfo().is_replicated_database_internal = true;
drop_context->setQueryContext(std::const_pointer_cast<Context>(current_context));
2021-04-11 07:44:40 +00:00
drop_context->initZooKeeperMetadataTransaction(txn, true);
}
InterpreterDropQuery drop_interpreter(ast_drop_query, drop_context);
drop_interpreter.execute();
}
}
2011-11-05 23:31:19 +00:00
}