monor improvements

This commit is contained in:
Alexander Tokmakov 2023-03-28 23:47:45 +02:00
parent a1076005c7
commit 53725bdea1
10 changed files with 38 additions and 66 deletions

View File

@ -110,16 +110,6 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String &
return table;
}
void DatabaseAtomic::undropTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String & relative_table_path)
{
std::lock_guard lock(mutex);
String table_metadata_path = getObjectMetadataPath(table_name);
String table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID());
renameNoReplace(table_metadata_path_drop, table_metadata_path);
DatabaseOrdinary::attachTableUnlocked(table_name, table);
table_name_to_path.emplace(std::make_pair(table_name, relative_table_path));
}
void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool sync)
{
auto table = tryGetTable(table_name, local_context);

View File

@ -41,8 +41,6 @@ public:
void attachTable(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path) override;
StoragePtr detachTable(ContextPtr context, const String & name) override;
void undropTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override;
String getTableDataPath(const String & table_name) const override;
String getTableDataPath(const ASTCreateQuery & query) const override;

View File

@ -216,11 +216,6 @@ public:
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no DETACH TABLE query for Database{}", getEngineName());
}
virtual void undropTable(ContextPtr /* context */, const String & /*name*/, const StoragePtr & /*table*/, [[maybe_unused]] const String & relative_table_path = {}) /// NOLINT
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no UNDROP TABLE query for Database{}", getEngineName());
}
/// Forget about the table without deleting it's data, but rename metadata file to prevent reloading it
/// with next restart. The database may not support this method.
virtual void detachTablePermanently(ContextPtr /*context*/, const String & /*name*/)

View File

@ -402,7 +402,6 @@ private:
/// Temporary data for query execution accounting.
TemporaryDataOnDiskScopePtr temp_data_on_disk;
bool in_ddl_guard = false;
public:
/// Some counters for current query execution.
/// Most of them are workarounds and should be removed in the future.
@ -1015,9 +1014,6 @@ public:
bool isInternalQuery() const { return is_internal_query; }
void setInternalQuery(bool internal) { is_internal_query = internal; }
bool isInDDLGuard() const { return in_ddl_guard; }
void setInDDLGuard(bool ddl_guard) { in_ddl_guard = ddl_guard; }
ActionLocksManagerPtr getActionLocksManager() const;
enum class ApplicationType

View File

@ -993,48 +993,30 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id)
CurrentMetrics::sub(CurrentMetrics::TablesToDropQueueSize, 1);
}
LOG_INFO(log, "Trying Undrop table {} from {}", dropped_table.table_id.getNameForLogs(), latest_metadata_dropped_path);
LOG_INFO(log, "Attaching undropped table {} (metadata moved from {})",
dropped_table.table_id.getNameForLogs(), latest_metadata_dropped_path);
try
{
auto wait_dropped_table_not_in_use = [&]()
{
while (true)
{
{
std::lock_guard lock(tables_marked_dropped_mutex);
if (dropped_table.table.unique())
return;
}
std::this_thread::sleep_for(std::chrono::milliseconds(100));
}
};
wait_dropped_table_not_in_use();
/// It's unsafe to create another instance while the old one exists
/// We cannot wait on shared_ptr's refcount, so it's busy wait
while (!dropped_table.table.unique())
std::this_thread::sleep_for(std::chrono::milliseconds(100));
dropped_table.table.reset();
auto ast_attach = std::make_shared<ASTCreateQuery>();
ast_attach->attach = true;
ast_attach->setDatabase(dropped_table.table_id.database_name);
ast_attach->setTable(dropped_table.table_id.table_name);
auto ast_attach = std::make_shared<ASTCreateQuery>();
ast_attach->attach = true;
ast_attach->setDatabase(dropped_table.table_id.database_name);
ast_attach->setTable(dropped_table.table_id.table_name);
auto query_context = Context::createCopy(getContext());
/// Attach table needs to acquire ddl guard, that has already been acquired in undrop table,
/// and cannot be acquired in the attach table again.
query_context->setInDDLGuard(true);
InterpreterCreateQuery interpreter(ast_attach, query_context);
interpreter.setForceAttach(true);
interpreter.setForceRestoreData(true);
interpreter.execute();
auto query_context = Context::createCopy(getContext());
/// Attach table needs to acquire ddl guard, that has already been acquired in undrop table,
/// and cannot be acquired in the attach table again.
InterpreterCreateQuery interpreter(ast_attach, query_context);
interpreter.setForceAttach(true);
interpreter.setForceRestoreData(true);
interpreter.setDontNeedDDLGuard(); /// It's already locked by caller
interpreter.execute();
LOG_INFO(log, "Table {} was successfully Undropped.", dropped_table.table_id.getNameForLogs());
}
catch (...)
{
throw Exception(
ErrorCodes::FS_METADATA_ERROR,
"Cannot undrop table {} from {}",
dropped_table.table_id.getNameForLogs(),
latest_metadata_dropped_path);
}
LOG_INFO(log, "Table {} was successfully undropped.", dropped_table.table_id.getNameForLogs());
}
void DatabaseCatalog::dropTableDataTask()

View File

@ -1107,7 +1107,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
/// For short syntax of ATTACH query we have to lock table name here, before reading metadata
/// and hold it until table is attached
if (!getContext()->isInDDLGuard())
if (likely(need_ddl_guard))
ddl_guard = DatabaseCatalog::instance().getDDLGuard(database_name, create.getTable());
bool if_not_exists = create.if_not_exists;
@ -1313,7 +1313,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
return true;
}
if (!ddl_guard && !getContext()->isInDDLGuard())
if (!ddl_guard && likely(need_ddl_guard))
ddl_guard = DatabaseCatalog::instance().getDDLGuard(create.getDatabase(), create.getTable());
String data_path;

View File

@ -61,6 +61,11 @@ public:
load_database_without_tables = load_database_without_tables_;
}
void setDontNeedDDLGuard()
{
need_ddl_guard = false;
}
/// Obtain information about columns, their types, default values and column comments,
/// for case when columns in CREATE query is specified explicitly.
static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, ContextPtr context, bool attach);
@ -112,6 +117,7 @@ private:
bool internal = false;
bool force_attach = false;
bool load_database_without_tables = false;
bool need_ddl_guard = true;
mutable String as_database_saved;
mutable String as_table_saved;

View File

@ -57,9 +57,11 @@ BlockIO InterpreterUndropQuery::executeToTable(ASTUndropQuery & query)
auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name);
auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name);
if (database->getEngineName() == "Replicated")
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Replicated database does not support UNDROP query");
if (database->isTableExist(table_id.table_name, getContext()))
throw Exception(
ErrorCodes::TABLE_ALREADY_EXISTS, "Cannot Undrop table, {} already exists", table_id);
ErrorCodes::TABLE_ALREADY_EXISTS, "Cannot undrop table, {} already exists", table_id);
database->checkMetadataFilenameAvailability(table_id.table_name);

View File

@ -1,5 +1,6 @@
test MergeTree with uuid
02681_undrop_uuid
OK
1
2
3

View File

@ -7,11 +7,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
echo 'test MergeTree with uuid'
${CLICKHOUSE_CLIENT} -q "drop table if exists 02681_undrop_uuid sync;"
uuid=$(${CLICKHOUSE_CLIENT} --query "SELECT reinterpretAsUUID(currentDatabase())")
${CLICKHOUSE_CLIENT} -q "create table 02681_undrop_uuid UUID '$uuid' (id Int32) Engine=MergeTree() order by id;"
uuid=$(${CLICKHOUSE_CLIENT} --query "SELECT generateUUIDv4()")
uuid2=$(${CLICKHOUSE_CLIENT} --query "SELECT generateUUIDv4()")
${CLICKHOUSE_CLIENT} -q "create table 02681_undrop_uuid UUID '$uuid' on cluster test_shard_localhost (id Int32) Engine=MergeTree() order by id;"
${CLICKHOUSE_CLIENT} -q "insert into 02681_undrop_uuid values (1),(2),(3);"
${CLICKHOUSE_CLIENT} -q "drop table 02681_undrop_uuid settings database_atomic_wait_for_drop_and_detach_synchronously = 0;"
${CLICKHOUSE_CLIENT} -q "drop table 02681_undrop_uuid on cluster test_shard_localhost settings database_atomic_wait_for_drop_and_detach_synchronously = 0;"
${CLICKHOUSE_CLIENT} -q "select table from system.dropped_tables where table = '02681_undrop_uuid' limit 1;"
${CLICKHOUSE_CLIENT} -q "undrop table 02681_undrop_uuid UUID '$uuid' settings allow_experimental_undrop_table_query = 1;"
${CLICKHOUSE_CLIENT} -q "undrop table 02681_undrop_uuid UUID '$uuid2' settings allow_experimental_undrop_table_query = 1;" 2>&1| grep -Faq "UNKNOWN_TABLE" && echo OK
${CLICKHOUSE_CLIENT} -q "undrop table 02681_undrop_uuid UUID '$uuid' on cluster test_shard_localhost settings allow_experimental_undrop_table_query = 1;"
${CLICKHOUSE_CLIENT} -q "select * from 02681_undrop_uuid order by id;"
${CLICKHOUSE_CLIENT} -q "drop table 02681_undrop_uuid sync;"