Remove metadata leftovers on drop database (#30054)

* remove metadata leftovers on drop database

* Update InterpreterDropQuery.cpp

* Update DatabaseCatalog.cpp
This commit is contained in:
tavplubix 2021-10-13 13:34:18 +03:00 committed by GitHub
parent a16ce6b54b
commit 7742b96497
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 60 additions and 11 deletions

View File

@ -523,6 +523,8 @@ class IColumn;
M(Int64, remote_fs_read_backoff_threshold, 10000, "Max wait time when trying to read data for remote disk", 0) \
M(Int64, remote_fs_read_backoff_max_tries, 5, "Max attempts to read with backoff", 0) \
\
M(Bool, force_remove_data_recursively_on_drop, false, "Recursively remove data on DROP query. Avoids 'Directory not empty' error, but may silently remove detached data", 0) \
\
/** Experimental functions */ \
M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \
M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \

View File

@ -71,6 +71,7 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m
/// Before 20.7 it's possible that .sql metadata file does not exist for some old database.
/// In this case Ordinary database is created on server startup if the corresponding metadata directory exists.
/// So we should remove metadata directory if database creation failed.
/// TODO remove this code
created = fs::create_directory(metadata_path);
DatabasePtr impl = getImpl(create, metadata_path, context);

View File

@ -39,6 +39,7 @@ namespace ErrorCodes
extern const int SYNTAX_ERROR;
extern const int TABLE_ALREADY_EXISTS;
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
extern const int DATABASE_NOT_EMPTY;
}
@ -544,8 +545,28 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const
void DatabaseOnDisk::drop(ContextPtr local_context)
{
assert(tables.empty());
fs::remove(local_context->getPath() + getDataPath());
fs::remove(getMetadataPath());
if (local_context->getSettingsRef().force_remove_data_recursively_on_drop)
{
fs::remove_all(local_context->getPath() + getDataPath());
fs::remove_all(getMetadataPath());
}
else
{
try
{
fs::remove(local_context->getPath() + getDataPath());
fs::remove(getMetadataPath());
}
catch (const fs::filesystem_error & e)
{
if (e.code() != std::errc::directory_not_empty)
throw Exception(Exception::CreateFromSTDTag{}, e);
throw Exception(ErrorCodes::DATABASE_NOT_EMPTY, "Cannot drop: {}. "
"Probably database contain some detached tables or metadata leftovers from Ordinary engine. "
"If you want to remove all data anyway, try to attach database back and drop it again "
"with enabled force_remove_data_recursively_on_drop setting", e.what());
}
}
}
String DatabaseOnDisk::getObjectMetadataPath(const String & object_name) const

View File

@ -329,7 +329,7 @@ void DatabaseCatalog::attachDatabase(const String & database_name, const Databas
}
DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool drop, bool check_empty)
DatabasePtr DatabaseCatalog::detachDatabase(ContextPtr local_context, const String & database_name, bool drop, bool check_empty)
{
if (database_name == TEMPORARY_DATABASE)
throw Exception("Cannot detach database with temporary tables.", ErrorCodes::DATABASE_ACCESS_DENIED);
@ -365,12 +365,14 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d
if (drop)
{
/// Delete the database.
db->drop(getContext());
db->drop(local_context);
/// Old ClickHouse versions did not store database.sql files
/// Remove metadata dir (if exists) to avoid recreation of .sql file on server startup
fs::path database_metadata_dir = fs::path(getContext()->getPath()) / "metadata" / escapeForFileName(database_name);
fs::remove(database_metadata_dir);
fs::path database_metadata_file = fs::path(getContext()->getPath()) / "metadata" / (escapeForFileName(database_name) + ".sql");
if (fs::exists(database_metadata_file))
fs::remove(database_metadata_file);
fs::remove(database_metadata_file);
}
return db;

View File

@ -147,7 +147,7 @@ public:
DatabasePtr getSystemDatabase() const;
void attachDatabase(const String & database_name, const DatabasePtr & database);
DatabasePtr detachDatabase(const String & database_name, bool drop = false, bool check_empty = true);
DatabasePtr detachDatabase(ContextPtr local_context, const String & database_name, bool drop = false, bool check_empty = true);
void updateDatabaseName(const String & old_name, const String & new_name);
/// database_name must be not empty

View File

@ -295,7 +295,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
assert(removed);
}
if (added)
DatabaseCatalog::instance().detachDatabase(database_name, false, false);
DatabaseCatalog::instance().detachDatabase(getContext(), database_name, false, false);
throw;
}

View File

@ -369,7 +369,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query,
database->assertCanBeDetached(true);
/// DETACH or DROP database itself
DatabaseCatalog::instance().detachDatabase(database_name, drop, database->shouldBeEmptyOnDetach());
DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach());
}
}

View File

@ -72,6 +72,7 @@ static void loadDatabase(
}
else if (fs::exists(fs::path(database_path)))
{
/// TODO Remove this code (it's required for compatibility with versions older than 20.7)
/// Database exists, but .sql file is absent. It's old-style Ordinary database (e.g. system or default)
database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database) + " ENGINE = Ordinary";
}

View File

@ -0,0 +1,22 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', stay_alive=True)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_drop_memory_database(start_cluster):
node.query("CREATE DATABASE test ENGINE Memory")
node.query("CREATE TABLE test.test_table(a String) ENGINE Memory")
node.query("DROP DATABASE test")
node.restart_clickhouse(kill=True)
assert node.query("SHOW DATABASES LIKE 'test'").strip() == ""

View File

@ -131,7 +131,7 @@ SELECT 'And detach permanently again to check how database drop will behave';
DETACH table test1601_detach_permanently_ordinary.test_name_reuse PERMANENTLY;
SELECT 'DROP database - Directory not empty error, but database detached';
DROP DATABASE test1601_detach_permanently_ordinary; -- { serverError 1001 }
DROP DATABASE test1601_detach_permanently_ordinary; -- { serverError 219 }
ATTACH DATABASE test1601_detach_permanently_ordinary;
@ -205,7 +205,7 @@ SELECT 'And detach permanently again to check how database drop will behave';
DETACH table test1601_detach_permanently_lazy.test_name_reuse PERMANENTLY;
SELECT 'DROP database - Directory not empty error, but database deteched';
DROP DATABASE test1601_detach_permanently_lazy; -- { serverError 1001 }
DROP DATABASE test1601_detach_permanently_lazy; -- { serverError 219 }
ATTACH DATABASE test1601_detach_permanently_lazy;