ClickHouse/dbms/src/Interpreters/InterpreterDropQuery.cpp

237 lines
8.8 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>
#include <Parsers/ASTDropQuery.h>
#include <Storages/IStorage.h>
#include <Common/escapeForFileName.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
{
extern const int TABLE_WAS_NOT_DROPPED;
extern const int DATABASE_NOT_EMPTY;
extern const int UNKNOWN_DATABASE;
extern const int READONLY;
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;
2018-09-11 18:37:19 +00:00
extern const int QUERY_IS_PROHIBITED;
}
2011-11-05 23:31:19 +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()
{
ASTDropQuery & drop = typeid_cast<ASTDropQuery &>(*query_ptr);
checkAccess(drop);
2017-04-21 12:39:28 +00:00
if (!drop.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, context, {drop.database});
2018-04-21 00:35:20 +00:00
if (!drop.table.empty())
return executeToTable(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary);
else if (!drop.database.empty())
return executeToDatabase(drop.database, drop.kind, drop.if_exists);
else
throw Exception("Database and table names is empty.", ErrorCodes::LOGICAL_ERROR);
}
2018-04-21 00:35:20 +00:00
BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool if_temporary)
{
if (if_temporary || database_name_.empty())
{
2018-04-21 00:35:20 +00:00
auto & session_context = context.hasSessionContext() ? context.getSessionContext() : context;
if (session_context.isExternalTableExist(table_name))
return executeToTemporaryTable(table_name, kind);
}
2018-04-21 00:35:20 +00:00
String database_name = database_name_.empty() ? context.getCurrentDatabase() : database_name_;
2018-09-18 18:33:15 +00:00
auto ddl_guard = context.getDDLGuard(database_name, table_name);
2018-04-21 00:35:20 +00:00
DatabaseAndTable database_and_table = tryGetDatabaseAndTable(database_name, table_name, if_exists);
if (database_and_table.first && database_and_table.second)
{
2018-04-21 00:35:20 +00:00
if (kind == ASTDropQuery::Kind::Detach)
{
2018-04-21 00:35:20 +00:00
database_and_table.second->shutdown();
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = database_and_table.second->lockForAlter(context.getCurrentQueryId());
2018-04-21 00:35:20 +00:00
/// Drop table from memory, don't touch data and metadata
database_and_table.first->detachTable(database_and_table.second->getTableName());
}
else if (kind == ASTDropQuery::Kind::Truncate)
{
2018-08-03 09:54:46 +00:00
database_and_table.second->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 = database_and_table.second->lockForAlter(context.getCurrentQueryId());
2018-04-21 00:35:20 +00:00
/// Drop table data, don't touch metadata
database_and_table.second->truncate(query_ptr, context);
2018-04-21 00:35:20 +00:00
}
else if (kind == ASTDropQuery::Kind::Drop)
{
2018-08-03 09:54:46 +00:00
database_and_table.second->checkTableCanBeDropped();
2018-04-21 00:35:20 +00:00
database_and_table.second->shutdown();
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = database_and_table.second->lockForAlter(context.getCurrentQueryId());
2018-11-27 18:44:19 +00:00
/// Delete table metadata and table itself from memory
2018-04-21 00:35:20 +00:00
database_and_table.first->removeTable(context, database_and_table.second->getTableName());
/// Delete table data
database_and_table.second->drop();
database_and_table.second->is_dropped = true;
String database_data_path = database_and_table.first->getDataPath();
/// If it is not virtual database like Dictionary then drop remaining data dir
if (!database_data_path.empty())
2018-02-02 13:17:45 +00:00
{
2018-04-21 00:35:20 +00:00
String table_data_path = database_data_path + "/" + escapeForFileName(database_and_table.second->getTableName());
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::executeToTemporaryTable(String & table_name, ASTDropQuery::Kind kind)
{
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;
StoragePtr table = context_handle.tryGetExternalTable(table_name);
2018-04-21 00:35:20 +00:00
if (table)
{
if (kind == ASTDropQuery::Kind::Truncate)
{
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = table->lockForAlter(context.getCurrentQueryId());
/// Drop table data, don't touch metadata
table->truncate(query_ptr, context);
}
else if (kind == ASTDropQuery::Kind::Drop)
{
context_handle.tryRemoveExternalTable(table_name);
table->shutdown();
/// If table was already dropped by anyone, an exception will be thrown
auto table_lock = table->lockForAlter(context.getCurrentQueryId());
/// Delete table data
table->drop();
table->is_dropped = true;
}
}
}
2018-04-21 00:35:20 +00:00
return {};
}
2018-04-21 00:35:20 +00:00
BlockIO InterpreterDropQuery::executeToDatabase(String & database_name, ASTDropQuery::Kind kind, bool if_exists)
{
2018-09-18 18:33:15 +00:00
auto ddl_guard = context.getDDLGuard(database_name, "");
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)
{
2018-04-21 00:35:20 +00:00
throw Exception("Unable to truncate database.", ErrorCodes::SYNTAX_ERROR);
}
2018-04-21 00:35:20 +00:00
else if (kind == ASTDropQuery::Kind::Detach)
{
2018-04-21 00:35:20 +00:00
context.detachDatabase(database_name);
database->shutdown();
}
2018-04-21 00:35:20 +00:00
else if (kind == ASTDropQuery::Kind::Drop)
{
for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next())
{
String current_table_name = iterator->table()->getTableName();
executeToTable(database_name, current_table_name, kind, false, false);
}
2018-04-21 00:35:20 +00:00
auto context_lock = context.getLock();
2018-04-21 00:35:20 +00:00
/// Someone could have time to delete the database before us.
context.assertDatabaseExists(database_name);
2018-04-21 00:35:20 +00:00
/// Someone could have time to create a table in the database to be deleted while we deleted the tables without the context lock.
if (!context.getDatabase(database_name)->empty(context))
throw Exception("New table appeared in database being dropped. Try dropping it again.", ErrorCodes::DATABASE_NOT_EMPTY);
2018-04-21 00:35:20 +00:00
/// Delete database information from the RAM
context.detachDatabase(database_name);
2018-04-21 00:35:20 +00:00
database->shutdown();
2018-06-09 15:48:22 +00:00
2018-04-21 00:35:20 +00:00
/// Delete the database.
2018-06-09 15:48:22 +00:00
database->drop();
2018-06-09 15:48:22 +00:00
/// Old ClickHouse versions did not store database.sql files
Poco::File database_metadata_file(context.getPath() + "metadata/" + escapeForFileName(database_name) + ".sql");
if (database_metadata_file.exists())
database_metadata_file.remove(false);
}
}
2018-04-21 00:35:20 +00:00
return {};
}
2018-04-21 00:35:20 +00:00
DatabasePtr InterpreterDropQuery::tryGetDatabase(String & database_name, bool if_exists)
{
return if_exists ? context.tryGetDatabase(database_name) : context.getDatabase(database_name);
}
2018-04-21 00:35:20 +00:00
DatabaseAndTable InterpreterDropQuery::tryGetDatabaseAndTable(String & database_name, String & table_name, bool if_exists)
{
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)
throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist.",
ErrorCodes::UNKNOWN_TABLE);
return {std::move(database), std::move(table)};
}
return {};
}
void InterpreterDropQuery::checkAccess(const ASTDropQuery & drop)
{
const Settings & settings = context.getSettingsRef();
auto readonly = settings.readonly;
2018-09-11 18:37:19 +00:00
bool allow_ddl = settings.allow_ddl;
2015-12-13 12:02:10 +00:00
2017-12-22 19:20:18 +00:00
/// It's allowed to drop temporary tables.
2018-09-11 18:37:19 +00:00
if ((!readonly && allow_ddl) || (drop.database.empty() && context.tryGetExternalTable(drop.table) && readonly >= 2))
return;
2018-09-11 18:37:19 +00:00
if (readonly)
throw Exception("Cannot drop table in readonly mode", ErrorCodes::READONLY);
throw Exception("Cannot drop table. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
}
2017-12-22 19:20:18 +00:00
2011-11-05 23:31:19 +00:00
}