mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
monor improvements
This commit is contained in:
parent
a1076005c7
commit
53725bdea1
@ -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);
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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*/)
|
||||
|
@ -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
|
||||
|
@ -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()
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
test MergeTree with uuid
|
||||
02681_undrop_uuid
|
||||
OK
|
||||
1
|
||||
2
|
||||
3
|
||||
|
@ -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;"
|
||||
|
Loading…
Reference in New Issue
Block a user