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>
|
2017-12-20 07:39:52 +00:00
|
|
|
#include <Interpreters/DDLWorker.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>
|
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>
|
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
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int TABLE_WAS_NOT_DROPPED;
|
|
|
|
extern const int DATABASE_NOT_EMPTY;
|
|
|
|
extern const int UNKNOWN_DATABASE;
|
2017-12-20 07:39:52 +00:00
|
|
|
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;
|
2019-10-11 13:21:52 +00:00
|
|
|
extern const int UNKNOWN_DICTIONARY;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2011-11-05 23:31:19 +00:00
|
|
|
|
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 &>();
|
2017-04-13 16:12:56 +00:00
|
|
|
|
2019-03-15 16:14:13 +00:00
|
|
|
checkAccess(drop);
|
2017-12-20 07:39:52 +00:00
|
|
|
|
2019-03-15 16:14:13 +00:00
|
|
|
if (!drop.cluster.empty())
|
|
|
|
return executeDDLQueryOnCluster(query_ptr, context, {drop.database});
|
2017-04-13 16:12:56 +00:00
|
|
|
|
2019-03-15 16:14:13 +00:00
|
|
|
if (!drop.table.empty())
|
2019-10-11 13:21:52 +00:00
|
|
|
{
|
|
|
|
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);
|
|
|
|
}
|
2019-03-15 16:14:13 +00:00
|
|
|
else if (!drop.database.empty())
|
|
|
|
return executeToDatabase(drop.database, drop.kind, drop.if_exists);
|
2018-04-21 00:35:20 +00:00
|
|
|
else
|
2019-10-11 13:21:52 +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
|
|
|
|
|
|
|
|
2019-10-11 13:21:52 +00:00
|
|
|
BlockIO InterpreterDropQuery::executeToTable(
|
|
|
|
String & database_name_,
|
|
|
|
String & table_name,
|
|
|
|
ASTDropQuery::Kind kind,
|
|
|
|
bool if_exists,
|
|
|
|
bool if_temporary,
|
|
|
|
bool no_ddl_lock)
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
|
|
|
if (if_temporary || database_name_.empty())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
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);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-04-21 00:35:20 +00:00
|
|
|
String database_name = database_name_.empty() ? context.getCurrentDatabase() : database_name_;
|
|
|
|
|
2019-03-11 16:50:31 +00:00
|
|
|
auto ddl_guard = (!no_ddl_lock ? context.getDDLGuard(database_name, table_name) : nullptr);
|
2018-09-18 18:33:15 +00:00
|
|
|
|
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)
|
2017-08-15 12:34:28 +00:00
|
|
|
{
|
2018-04-21 00:35:20 +00:00
|
|
|
if (kind == ASTDropQuery::Kind::Detach)
|
2017-08-15 17:00:18 +00:00
|
|
|
{
|
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
|
2019-03-05 10:12:20 +00:00
|
|
|
auto table_lock = database_and_table.second->lockExclusively(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
|
2019-03-05 10:12:20 +00:00
|
|
|
auto table_lock = database_and_table.second->lockExclusively(context.getCurrentQueryId());
|
2018-04-21 00:35:20 +00:00
|
|
|
/// Drop table data, don't touch metadata
|
2019-08-27 20:43:08 +00:00
|
|
|
database_and_table.second->truncate(query_ptr, context, table_lock);
|
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
|
2018-11-28 15:59:57 +00:00
|
|
|
|
2019-03-05 10:12:20 +00:00
|
|
|
auto table_lock = database_and_table.second->lockExclusively(context.getCurrentQueryId());
|
2018-11-28 15:59:57 +00:00
|
|
|
|
2019-08-28 18:54:30 +00:00
|
|
|
const std::string metadata_file_without_extension =
|
|
|
|
database_and_table.first->getMetadataPath()
|
|
|
|
+ escapeForFileName(database_and_table.second->getTableName());
|
|
|
|
|
|
|
|
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
|
|
|
|
2019-08-28 18:34:42 +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
|
2019-08-28 18:34:42 +00:00
|
|
|
database_and_table.second->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;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// 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());
|
|
|
|
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 {};
|
|
|
|
}
|
|
|
|
|
2019-10-11 13:21:52 +00:00
|
|
|
|
|
|
|
BlockIO InterpreterDropQuery::executeToDictionary(
|
|
|
|
String & database_name_,
|
|
|
|
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 = database_name_.empty() ? context.getCurrentDatabase() : database_name_;
|
|
|
|
|
|
|
|
auto ddl_guard = (!no_ddl_lock ? context.getDDLGuard(database_name, dictionary_name) : nullptr);
|
|
|
|
|
|
|
|
DatabasePtr database = tryGetDatabase(database_name, false);
|
|
|
|
|
|
|
|
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
|
2019-10-16 17:06:52 +00:00
|
|
|
database->detachDictionary(dictionary_name, context);
|
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)
|
|
|
|
{
|
|
|
|
database->removeDictionary(context, dictionary_name);
|
|
|
|
}
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2018-04-21 00:35:20 +00:00
|
|
|
BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDropQuery::Kind kind)
|
|
|
|
{
|
|
|
|
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;
|
|
|
|
StoragePtr table = context_handle.tryGetExternalTable(table_name);
|
2018-04-21 00:35:20 +00:00
|
|
|
if (table)
|
|
|
|
{
|
2018-07-10 14:24:19 +00:00
|
|
|
if (kind == ASTDropQuery::Kind::Truncate)
|
|
|
|
{
|
|
|
|
/// If table was already dropped by anyone, an exception will be thrown
|
2019-03-05 10:12:20 +00:00
|
|
|
auto table_lock = table->lockExclusively(context.getCurrentQueryId());
|
2018-07-10 14:24:19 +00:00
|
|
|
/// Drop table data, don't touch metadata
|
2019-08-27 20:43:08 +00:00
|
|
|
table->truncate(query_ptr, context, table_lock);
|
2018-07-10 14:24:19 +00:00
|
|
|
}
|
|
|
|
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
|
2019-03-05 10:12:20 +00:00
|
|
|
auto table_lock = table->lockExclusively(context.getCurrentQueryId());
|
2018-07-10 14:24:19 +00:00
|
|
|
/// Delete table data
|
2019-08-27 20:43:08 +00:00
|
|
|
table->drop(table_lock);
|
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
|
|
|
|
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))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-04-21 00:35:20 +00:00
|
|
|
if (kind == ASTDropQuery::Kind::Truncate)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-04-21 00:35:20 +00:00
|
|
|
throw Exception("Unable to truncate database.", ErrorCodes::SYNTAX_ERROR);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2018-04-21 00:35:20 +00:00
|
|
|
else if (kind == ASTDropQuery::Kind::Detach)
|
2017-06-13 16:36:15 +00:00
|
|
|
{
|
2018-04-21 00:35:20 +00:00
|
|
|
context.detachDatabase(database_name);
|
|
|
|
database->shutdown();
|
2017-06-13 16:36:15 +00:00
|
|
|
}
|
2018-04-21 00:35:20 +00:00
|
|
|
else if (kind == ASTDropQuery::Kind::Drop)
|
|
|
|
{
|
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
|
|
|
}
|
2017-06-13 16:36:15 +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);
|
|
|
|
}
|
|
|
|
|
2018-04-21 00:35:20 +00:00
|
|
|
auto context_lock = context.getLock();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-21 00:35:20 +00:00
|
|
|
/// Someone could have time to delete the database before us.
|
|
|
|
context.assertDatabaseExists(database_name);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-21 00:35:20 +00:00
|
|
|
/// Delete database information from the RAM
|
|
|
|
context.detachDatabase(database_name);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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);
|
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
|
|
|
|
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);
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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-02-08 14:14:08 +00:00
|
|
|
|
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);
|
2018-01-25 19:01:11 +00:00
|
|
|
|
2018-09-07 01:39:48 +00:00
|
|
|
return {std::move(database), std::move(table)};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
return {};
|
2014-03-20 10:59:45 +00:00
|
|
|
}
|
|
|
|
|
2017-12-20 07:39:52 +00:00
|
|
|
void InterpreterDropQuery::checkAccess(const ASTDropQuery & drop)
|
|
|
|
{
|
|
|
|
const Settings & settings = context.getSettingsRef();
|
2018-03-11 00:15:26 +00:00
|
|
|
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))
|
2017-12-20 07:39:52 +00:00
|
|
|
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-20 07:39:52 +00:00
|
|
|
}
|
2017-12-22 19:20:18 +00:00
|
|
|
|
2011-11-05 23:31:19 +00:00
|
|
|
}
|