mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
Remove metadata leftovers on drop database (#30054)
* remove metadata leftovers on drop database * Update InterpreterDropQuery.cpp * Update DatabaseCatalog.cpp
This commit is contained in:
parent
a16ce6b54b
commit
7742b96497
@ -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) \
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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";
|
||||
}
|
||||
|
0
tests/integration/test_restart_server/__init__.py
Executable file
0
tests/integration/test_restart_server/__init__.py
Executable file
22
tests/integration/test_restart_server/test.py
Executable file
22
tests/integration/test_restart_server/test.py
Executable 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() == ""
|
||||
|
@ -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;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user