ClickHouse/src/Interpreters/InterpreterDropQuery.cpp

330 lines
13 KiB
C++
Raw Normal View History

2011-11-05 23:31:19 +00:00
#include <Poco/File.h>
#include <Databases/IDatabase.h>
2017-05-23 18:01:50 +00:00
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/InterpreterDropQuery.h>
2019-10-15 18:04:17 +00:00
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Access/AccessRightsElement.h>
#include <Parsers/ASTDropQuery.h>
#include <Storages/IStorage.h>
#include <Common/escapeForFileName.h>
#include <Common/quoteString.h>
#include <Common/typeid_cast.h>
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;
extern const int UNKNOWN_DICTIONARY;
}
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);
}
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()
{
auto & drop = query_ptr->as<ASTDropQuery &>();
if (!drop.cluster.empty())
2020-01-24 16:20:36 +00:00
return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccessForDDLOnCluster());
if (!drop.table.empty())
{
if (!drop.is_dictionary)
return executeToTable(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
else
return executeToDictionary(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
}
else if (!drop.database.empty())
return executeToDatabase(drop.database, drop.kind, drop.if_exists);
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
}
BlockIO InterpreterDropQuery::executeToTable(
2020-01-24 16:20:36 +00:00
const String & database_name_,
const String & table_name,
ASTDropQuery::Kind kind,
bool if_exists,
bool is_temporary,
bool no_ddl_lock)
2018-04-21 00:35:20 +00:00
{
if (is_temporary || database_name_.empty())
{
2020-03-12 12:16:16 +00:00
if (context.tryResolveStorageID({"", table_name}, Context::ResolveExternal))
2018-04-21 00:35:20 +00:00
return executeToTemporaryTable(table_name, kind);
}
if (is_temporary)
{
if (if_exists)
2020-01-24 16:20:36 +00:00
return {};
2020-03-23 22:28:30 +00:00
throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " doesn't exist",
ErrorCodes::UNKNOWN_TABLE);
}
String database_name = context.resolveDatabase(database_name_);
2018-04-21 00:35:20 +00:00
2020-02-10 18:31:52 +00:00
auto ddl_guard = (!no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(database_name, table_name) : nullptr);
2018-09-18 18:33:15 +00:00
2020-01-24 16:20:36 +00:00
auto [database, table] = tryGetDatabaseAndTable(database_name, table_name, if_exists);
2018-04-21 00:35:20 +00:00
2020-01-24 16:20:36 +00:00
if (database && table)
{
2020-03-23 22:28:30 +00:00
if (query_ptr->as<ASTDropQuery &>().is_view && !table->isView())
throw Exception("Table " + backQuoteIfNeed(table_name) + " is not a View", ErrorCodes::LOGICAL_ERROR);
2020-01-24 16:20:36 +00:00
auto table_id = table->getStorageID();
2018-04-21 00:35:20 +00:00
if (kind == ASTDropQuery::Kind::Detach)
{
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->shutdown();
2018-04-21 00:35:20 +00:00
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
2018-04-21 00:35:20 +00:00
/// Drop table from memory, don't touch data and metadata
2020-01-24 16:20:36 +00:00
database->detachTable(table_name);
2018-04-21 00:35:20 +00:00
}
else if (kind == ASTDropQuery::Kind::Truncate)
{
context.checkAccess(AccessType::TRUNCATE, table_id);
2020-01-24 16:20:36 +00:00
table->checkTableCanBeDropped();
2018-06-09 15:48:22 +00:00
2018-04-21 00:35:20 +00:00
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
2018-04-21 00:35:20 +00:00
/// Drop table data, don't touch metadata
2020-01-24 16:20:36 +00:00
table->truncate(query_ptr, context, table_lock);
2018-04-21 00:35:20 +00:00
}
else if (kind == ASTDropQuery::Kind::Drop)
{
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-01-24 16:20:36 +00:00
table->shutdown();
2018-04-21 00:35:20 +00:00
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
2019-08-28 18:54:30 +00:00
2020-01-24 16:20:36 +00:00
const std::string metadata_file_without_extension = database->getMetadataPath() + escapeForFileName(table_id.table_name);
2019-08-28 18:54:30 +00:00
const auto prev_metadata_name = metadata_file_without_extension + ".sql";
const auto drop_metadata_name = metadata_file_without_extension + ".sql.tmp_drop";
2019-08-09 11:17:33 +00:00
/// Try to rename metadata file and delete the data
2019-08-09 11:17:33 +00:00
try
{
2019-08-28 18:54:30 +00:00
/// There some kind of tables that have no metadata - ignore renaming
2019-08-13 08:03:31 +00:00
if (Poco::File(prev_metadata_name).exists())
2019-08-12 12:56:10 +00:00
Poco::File(prev_metadata_name).renameTo(drop_metadata_name);
2019-08-09 11:17:33 +00:00
/// Delete table data
2020-01-24 16:20:36 +00:00
table->drop(table_lock);
2019-08-09 11:17:33 +00:00
}
catch (...)
{
2019-08-13 08:03:31 +00:00
if (Poco::File(drop_metadata_name).exists())
2019-08-12 10:13:03 +00:00
Poco::File(drop_metadata_name).renameTo(prev_metadata_name);
2019-08-09 11:17:33 +00:00
throw;
}
2020-01-24 16:20:36 +00:00
String table_data_path_relative = database->getTableDataPath(table_name);
2019-12-25 16:13:48 +00:00
2019-08-09 11:17:33 +00:00
/// Delete table metadata and table itself from memory
2020-01-24 16:20:36 +00:00
database->removeTable(context, table_name);
table->is_dropped = true;
2018-04-21 00:35:20 +00:00
/// If it is not virtual database like Dictionary then drop remaining data dir
2019-12-25 16:13:48 +00:00
if (!table_data_path_relative.empty())
2018-02-02 13:17:45 +00:00
{
2019-12-25 16:13:48 +00:00
String table_data_path = context.getPath() + table_data_path_relative;
2018-04-21 00:35:20 +00:00
if (Poco::File(table_data_path).exists())
Poco::File(table_data_path).remove(true);
2018-02-02 13:17:45 +00:00
}
2018-04-21 00:35:20 +00:00
}
}
return {};
}
BlockIO InterpreterDropQuery::executeToDictionary(
2020-01-24 16:20:36 +00:00
const String & database_name_,
const String & dictionary_name,
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);
String database_name = context.resolveDatabase(database_name_);
2020-02-10 18:31:52 +00:00
auto ddl_guard = (!no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(database_name, dictionary_name) : nullptr);
DatabasePtr database = tryGetDatabase(database_name, if_exists);
if (!database || !database->isDictionaryExist(context, dictionary_name))
{
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
context.checkAccess(AccessType::DROP_DICTIONARY, database_name, dictionary_name);
2019-10-16 17:06:52 +00:00
database->detachDictionary(dictionary_name, context);
}
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);
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);
else
2018-04-21 00:35:20 +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-02-17 13:52:59 +00:00
StoragePtr table = DatabaseCatalog::instance().getTable(resolved_id);
if (kind == ASTDropQuery::Kind::Truncate)
{
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock =
table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
/// Drop table data, don't touch metadata
2019-08-27 20:43:08 +00:00
table->truncate(query_ptr, context, table_lock);
}
else if (kind == ASTDropQuery::Kind::Drop)
{
context_handle.removeExternalTable(table_name);
table->shutdown();
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock =
table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
/// Delete table data
2019-08-27 20:43:08 +00:00
table->drop(table_lock);
table->is_dropped = true;
}
}
}
2018-04-21 00:35:20 +00:00
return {};
}
2020-03-18 00:57:00 +00:00
2020-01-24 16:20:36 +00:00
BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, ASTDropQuery::Kind kind, bool if_exists)
2018-04-21 00:35:20 +00:00
{
2020-02-10 18:31:52 +00:00
auto ddl_guard = DatabaseCatalog::instance().getDDLGuard(database_name, "");
2018-09-18 18:33:15 +00:00
2018-04-21 00:35:20 +00:00
if (auto database = tryGetDatabase(database_name, if_exists))
{
2018-04-21 00:35:20 +00:00
if (kind == ASTDropQuery::Kind::Truncate)
{
2020-03-23 22:28:30 +00:00
throw Exception("Unable to truncate database", ErrorCodes::SYNTAX_ERROR);
}
2020-02-13 21:00:03 +00:00
else if (kind == ASTDropQuery::Kind::Detach || kind == ASTDropQuery::Kind::Drop)
{
2020-02-13 21:00:03 +00:00
bool drop = kind == ASTDropQuery::Kind::Drop;
2020-01-24 16:20:36 +00:00
context.checkAccess(AccessType::DROP_DATABASE, database_name);
2020-02-13 21:00:03 +00:00
/// DETACH or DROP all tables and dictionaries inside database
2019-10-10 17:33:01 +00:00
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
2018-04-21 00:35:20 +00:00
{
2019-10-01 12:44:17 +00:00
String current_table_name = iterator->name();
2019-03-11 16:50:31 +00:00
executeToTable(database_name, current_table_name, kind, false, false, false);
2018-04-21 00:35:20 +00:00
}
2019-10-11 15:22:30 +00:00
for (auto iterator = database->getDictionariesIterator(context); iterator->isValid(); iterator->next())
{
String current_dictionary = iterator->name();
executeToDictionary(database_name, current_dictionary, kind, false, false, false);
}
2020-02-13 21:00:03 +00:00
/// DETACH or DROP database itself
DatabaseCatalog::instance().detachDatabase(database_name, drop);
}
}
2018-04-21 00:35:20 +00:00
return {};
}
2020-01-24 16:20:36 +00:00
DatabaseAndTable InterpreterDropQuery::tryGetDatabaseAndTable(const String & database_name, const String & table_name, bool if_exists)
2018-04-21 00:35:20 +00:00
{
DatabasePtr database = tryGetDatabase(database_name, if_exists);
2018-04-21 00:35:20 +00:00
if (database)
{
StoragePtr table = database->tryGetTable(context, table_name);
if (!table && !if_exists)
2020-03-23 22:28:30 +00:00
throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist",
2018-04-21 00:35:20 +00:00
ErrorCodes::UNKNOWN_TABLE);
return {std::move(database), std::move(table)};
}
return {};
}
2015-12-13 12:02:10 +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 &>();
2020-01-24 16:20:36 +00:00
if (drop.table.empty())
{
if (drop.kind == ASTDropQuery::Kind::Detach)
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)
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)
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)
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-22 19:20:18 +00:00
2011-11-05 23:31:19 +00:00
}