From 5a18e0d5a58794c0e344a2312972592601087d4f Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Tue, 21 Mar 2023 20:14:26 +0800 Subject: [PATCH 1/8] support undrop table --- src/Access/Common/AccessType.h | 2 + src/Access/tests/gtest_access_rights_ops.cpp | 2 +- src/Core/Settings.h | 1 + src/Databases/DatabaseAtomic.cpp | 10 ++ src/Databases/DatabaseAtomic.h | 2 + src/Databases/IDatabase.h | 5 + src/Interpreters/DatabaseCatalog.cpp | 123 +++++++++++++++++- src/Interpreters/DatabaseCatalog.h | 2 + src/Interpreters/InterpreterFactory.cpp | 6 + src/Interpreters/InterpreterUndropQuery.cpp | 77 +++++++++++ src/Interpreters/InterpreterUndropQuery.h | 29 +++++ src/Parsers/ASTUndropQuery.cpp | 42 ++++++ src/Parsers/ASTUndropQuery.h | 30 +++++ src/Parsers/IAST.h | 1 + src/Parsers/ParserQueryWithOutput.cpp | 3 + src/Parsers/ParserUndropQuery.cpp | 81 ++++++++++++ src/Parsers/ParserUndropQuery.h | 20 +++ .../integration/test_grant_and_revoke/test.py | 2 +- .../integration/test_undrop_query/__init__.py | 0 .../configs/with_delay_config.xml | 3 + tests/integration/test_undrop_query/test.py | 59 +++++++++ .../01271_show_privileges.reference | 1 + .../02117_show_create_table_system.reference | 8 +- .../0_stateless/02681_undrop_query.reference | 32 +++++ .../0_stateless/02681_undrop_query.sql | 90 +++++++++++++ .../02681_undrop_query_uuid.reference | 5 + .../0_stateless/02681_undrop_query_uuid.sh | 17 +++ 27 files changed, 646 insertions(+), 7 deletions(-) create mode 100644 src/Interpreters/InterpreterUndropQuery.cpp create mode 100644 src/Interpreters/InterpreterUndropQuery.h create mode 100644 src/Parsers/ASTUndropQuery.cpp create mode 100644 src/Parsers/ASTUndropQuery.h create mode 100644 src/Parsers/ParserUndropQuery.cpp create mode 100644 src/Parsers/ParserUndropQuery.h create mode 100644 tests/integration/test_undrop_query/__init__.py create mode 100644 tests/integration/test_undrop_query/configs/with_delay_config.xml create mode 100644 tests/integration/test_undrop_query/test.py create mode 100644 tests/queries/0_stateless/02681_undrop_query.reference create mode 100644 tests/queries/0_stateless/02681_undrop_query.sql create mode 100644 tests/queries/0_stateless/02681_undrop_query_uuid.reference create mode 100755 tests/queries/0_stateless/02681_undrop_query_uuid.sh diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index a7827ee7c59..57fa75dc67b 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -104,6 +104,8 @@ enum class AccessType M(DROP_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_CONTROL) /* allows to execute DROP NAMED COLLECTION */\ M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\ \ + M(UNDROP_TABLE, "", TABLE, ALL) /* allows to execute {UNDROP} TABLE */\ + \ M(TRUNCATE, "TRUNCATE TABLE", TABLE, ALL) \ M(OPTIMIZE, "OPTIMIZE TABLE", TABLE, ALL) \ M(BACKUP, "", TABLE, ALL) /* allows to backup tables */\ diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 025f70af587..5f1f13ca5a2 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -48,7 +48,7 @@ TEST(AccessRights, Union) ASSERT_EQ(lhs.toString(), "GRANT INSERT ON *.*, " "GRANT SHOW, SELECT, ALTER, CREATE DATABASE, CREATE TABLE, CREATE VIEW, " - "CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, " + "CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, " "TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, " "SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " "SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, " diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9fa2ba0d32f..acc20c85165 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -720,6 +720,7 @@ class IColumn; M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ M(Bool, force_aggregation_in_order, false, "Force use of aggregation in order on remote nodes during distributed aggregation. PLEASE, NEVER CHANGE THIS SETTING VALUE MANUALLY!", IMPORTANT) \ M(UInt64, http_max_request_param_data_size, 10_MiB, "Limit on size of request data used as a query parameter in predefined HTTP requests.", 0) \ + M(Bool, allow_experimental_undrop_table_query, false, "Allow to use undrop query to restore dropped table in a limited time", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 7e20b6f6535..e5320dc6ff4 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -110,6 +110,16 @@ 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); diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index cb275812098..b8ff719989b 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -41,6 +41,8 @@ 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; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index b8880c4c4cc..4dbf78cb5c4 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -216,6 +216,11 @@ 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*/) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index b11a973c7b7..4b4032f65a8 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -51,6 +51,7 @@ namespace ErrorCodes extern const int DATABASE_ACCESS_DENIED; extern const int LOGICAL_ERROR; extern const int HAVE_DEPENDENT_OBJECTS; + extern const int FS_METADATA_ERROR; } TemporaryTableHolder::TemporaryTableHolder(ContextPtr context_, const TemporaryTableHolder::Creator & creator, const ASTPtr & query) @@ -869,6 +870,13 @@ String DatabaseCatalog::getPathForDroppedMetadata(const StorageID & table_id) co toString(table_id.uuid) + ".sql"; } +String DatabaseCatalog::getPathForMetadata(const StorageID & table_id) const +{ + return getContext()->getPath() + "metadata/" + + escapeForFileName(table_id.getDatabaseName()) + "/" + + escapeForFileName(table_id.getTableName()) + ".sql"; +} + void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr table, String dropped_metadata_path, bool ignore_delay) { assert(table_id.hasUUID()); @@ -936,6 +944,118 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr (*drop_task)->schedule(); } +void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) +{ + String latest_metadata_dropped_path; + StorageID dropped_table_id = table_id; + TableMarkedAsDropped dropped_table; + { + std::lock_guard lock(tables_marked_dropped_mutex); + time_t latest_drop_time = std::numeric_limits::min(); + auto it_table = tables_marked_dropped.end(); + for (auto it = tables_marked_dropped.begin(); it != tables_marked_dropped.end(); ++it) + { + auto storage_ptr = it->table; + if (it->table_id.uuid == table_id.uuid) + { + it_table = it; + dropped_table = *it; + break; + } + /// If table uuid exists, only find tables with equal uuid. + if (table_id.uuid != UUIDHelpers::Nil) + continue; + if (it->table_id.database_name == table_id.database_name && + it->table_id.table_name == table_id.table_name && + it->drop_time >= latest_drop_time) + { + latest_drop_time = it->drop_time; + it_table = it; + dropped_table = *it; + } + } + if (it_table == tables_marked_dropped.end()) + throw Exception(ErrorCodes::UNKNOWN_TABLE, + "The drop task of table {} is in progress, has been dropped or the database engine doesn't support it", + table_id.getNameForLogs()); + latest_metadata_dropped_path = it_table->metadata_path; + dropped_table_id = it_table->table_id; + tables_marked_dropped.erase(it_table); + [[maybe_unused]] auto removed = tables_marked_dropped_ids.erase(dropped_table_id.uuid); + assert(removed); + } + /// Remove the table from tables_marked_dropped and tables_marked_dropped_ids, + /// and the drop task for this table will no longer be scheduled. + LOG_INFO(log, "Trying Undrop table {} from {}", dropped_table_id.getNameForLogs(), latest_metadata_dropped_path); + String table_metadata_path = getPathForMetadata(dropped_table_id); + + auto enqueue = [&]() + { + /// In the dropTableDataTask method. + /// 1. We will first determine whether there are tables to be dropped in tables_marked_dropped. + /// 2. If one is exist, the table will be removed from tables_marked_dropped. + /// 3. And then execute dropTableFinally. + /// So undrop and drop do not cross-execute. + std::lock_guard lock(tables_marked_dropped_mutex); + tables_marked_dropped.emplace_back(dropped_table); + tables_marked_dropped_ids.insert(dropped_table_id.uuid); + CurrentMetrics::add(CurrentMetrics::TablesToDropQueueSize, 1); + }; + + ASTPtr ast = DatabaseOnDisk::parseQueryFromMetadata( + log, getContext(), latest_metadata_dropped_path, /*throw_on_error*/ true, /*remove_empty*/ false); + auto * create = typeid_cast(ast.get()); + if (!create) + { + enqueue(); + throw Exception( + ErrorCodes::FS_METADATA_ERROR, + "Cannot parse metadata of table {} from {}", + dropped_table_id.getNameForLogs(), + table_metadata_path); + } + + create->setDatabase(dropped_table_id.database_name); + create->setTable(dropped_table_id.table_name); + + 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(); + auto database = getDatabase(dropped_table_id.database_name, getContext()); + String relative_table_path = fs::path(database->getDataPath()) / DatabaseCatalog::instance().getPathForUUID(dropped_table.table_id.uuid); + auto storage = createTableFromAST( + *create, + dropped_table.table_id.database_name, + relative_table_path, + getContext(), + /* force_restore */ true).second; + database->undropTable(getContext(), dropped_table_id.table_name, storage, relative_table_path); + storage->startup(); + CurrentMetrics::sub(CurrentMetrics::TablesToDropQueueSize, 1); + } + catch (...) + { + enqueue(); + throw Exception( + ErrorCodes::FS_METADATA_ERROR, + "Cannot undrop table {} from {}", + dropped_table_id.getNameForLogs(), + table_metadata_path); + } +} + void DatabaseCatalog::dropTableDataTask() { /// Background task that removes data of tables which were marked as dropped by Atomic databases. @@ -948,7 +1068,8 @@ void DatabaseCatalog::dropTableDataTask() try { std::lock_guard lock(tables_marked_dropped_mutex); - assert(!tables_marked_dropped.empty()); + if (tables_marked_dropped.empty()) + return; time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); time_t min_drop_time = std::numeric_limits::max(); size_t tables_in_use_count = 0; diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 88645ff72af..51e9fbdb936 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -215,7 +215,9 @@ public: DatabaseAndTable tryGetByUUID(const UUID & uuid) const; String getPathForDroppedMetadata(const StorageID & table_id) const; + String getPathForMetadata(const StorageID & table_id) const; void enqueueDroppedTableCleanup(StorageID table_id, StoragePtr table, String dropped_metadata_path, bool ignore_delay = false); + void dequeueDroppedTableCleanup(StorageID table_id); void waitTableFinallyDropped(const UUID & uuid); diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 502de459156..91291090e02 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -60,6 +61,7 @@ #include #include #include +#include #include #include #include @@ -161,6 +163,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); diff --git a/src/Interpreters/InterpreterUndropQuery.cpp b/src/Interpreters/InterpreterUndropQuery.cpp new file mode 100644 index 00000000000..cc3b1b3ac24 --- /dev/null +++ b/src/Interpreters/InterpreterUndropQuery.cpp @@ -0,0 +1,77 @@ +#include +#include +#include +#include +#include + +#include "config.h" + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int TABLE_ALREADY_EXISTS; + extern const int SUPPORT_IS_DISABLED; +} + +InterpreterUndropQuery::InterpreterUndropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) +{ +} + + +BlockIO InterpreterUndropQuery::execute() +{ + if (!getContext()->getSettingsRef().allow_experimental_undrop_table_query) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Undrop table is experimental. " + "Set `allow_experimental_undrop_table_query` setting to enable it"); + + getContext()->checkAccess(AccessType::UNDROP_TABLE); + auto & undrop = query_ptr->as(); + if (!undrop.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext())) + { + DDLQueryOnClusterParams params; + params.access_to_check = getRequiredAccessForDDLOnCluster(); + return executeDDLQueryOnCluster(query_ptr, getContext(), params); + } + + if (undrop.table) + return executeToTable(undrop); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Nothing to undrop, both names are empty"); +} + +BlockIO InterpreterUndropQuery::executeToTable(ASTUndropQuery & query) +{ + auto table_id = StorageID(query); + auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name); + + auto context = getContext(); + if (table_id.database_name.empty()) + { + table_id.database_name = context->getCurrentDatabase(); + query.setDatabase(table_id.database_name); + } + + auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); + if (database->isTableExist(table_id.table_name, getContext())) + throw Exception( + ErrorCodes::TABLE_ALREADY_EXISTS, "Cannot Undrop table, {}.{} already exists", backQuote(table_id.database_name), backQuote(table_id.table_name)); + + database->checkMetadataFilenameAvailability(table_id.table_name); + + DatabaseCatalog::instance().dequeueDroppedTableCleanup(table_id); + return {}; +} + +AccessRightsElements InterpreterUndropQuery::getRequiredAccessForDDLOnCluster() const +{ + AccessRightsElements required_access; + const auto & undrop = query_ptr->as(); + + required_access.emplace_back(AccessType::UNDROP_TABLE, undrop.getDatabase(), undrop.getTable()); + return required_access; +} +} diff --git a/src/Interpreters/InterpreterUndropQuery.h b/src/Interpreters/InterpreterUndropQuery.h new file mode 100644 index 00000000000..a47617fd17f --- /dev/null +++ b/src/Interpreters/InterpreterUndropQuery.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class Context; +using DatabaseAndTable = std::pair; +class AccessRightsElements; + + +class InterpreterUndropQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterUndropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_); + + /// Undrop table. + BlockIO execute() override; + +private: + AccessRightsElements getRequiredAccessForDDLOnCluster() const; + ASTPtr query_ptr; + + BlockIO executeToTable(ASTUndropQuery & query); +}; +} diff --git a/src/Parsers/ASTUndropQuery.cpp b/src/Parsers/ASTUndropQuery.cpp new file mode 100644 index 00000000000..0b8a18b12c9 --- /dev/null +++ b/src/Parsers/ASTUndropQuery.cpp @@ -0,0 +1,42 @@ +#include +#include +#include + + +namespace DB +{ + +String ASTUndropQuery::getID(char delim) const +{ + return "UndropQuery" + (delim + getDatabase()) + delim + getTable(); +} + +ASTPtr ASTUndropQuery::clone() const +{ + auto res = std::make_shared(*this); + cloneOutputOptions(*res); + cloneTableOptions(*res); + return res; +} + +void ASTUndropQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : ""); + settings.ostr << "UNDROP "; + settings.ostr << "TABLE "; + settings.ostr << (settings.hilite ? hilite_none : ""); + + assert (table); + if (!database) + settings.ostr << backQuoteIfNeed(getTable()); + else + settings.ostr << backQuoteIfNeed(getDatabase()) + "." << backQuoteIfNeed(getTable()); + + if (uuid != UUIDHelpers::Nil) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "") + << quoteString(toString(uuid)); + + formatOnCluster(settings); +} + +} diff --git a/src/Parsers/ASTUndropQuery.h b/src/Parsers/ASTUndropQuery.h new file mode 100644 index 00000000000..7aac4c86c5b --- /dev/null +++ b/src/Parsers/ASTUndropQuery.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/** UNDROP query + */ +class ASTUndropQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster +{ +public: + /** Get the text that identifies this element. */ + String getID(char) const override; + ASTPtr clone() const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams & params) const override + { + return removeOnCluster(clone(), params.default_database); + } + + QueryKind getQueryKind() const override { return QueryKind::Undrop; } + +protected: + void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; + +} diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 5928506aa5b..606a822ecee 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -268,6 +268,7 @@ public: Delete, Create, Drop, + Undrop, Rename, Optimize, Check, diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 7024d8cbe11..d6cf9109be0 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -44,6 +45,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserAlterQuery alter_p; ParserRenameQuery rename_p; ParserDropQuery drop_p; + ParserUndropQuery undrop_p; ParserCheckQuery check_p; ParserOptimizeQuery optimize_p; ParserKillQueryQuery kill_query_p; @@ -71,6 +73,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec || alter_p.parse(pos, query, expected) || rename_p.parse(pos, query, expected) || drop_p.parse(pos, query, expected) + || undrop_p.parse(pos, query, expected) || check_p.parse(pos, query, expected) || kill_query_p.parse(pos, query, expected) || optimize_p.parse(pos, query, expected) diff --git a/src/Parsers/ParserUndropQuery.cpp b/src/Parsers/ParserUndropQuery.cpp new file mode 100644 index 00000000000..3784ab0f353 --- /dev/null +++ b/src/Parsers/ParserUndropQuery.cpp @@ -0,0 +1,81 @@ +#include + +#include +#include +#include "Parsers/ASTLiteral.h" + + +namespace DB +{ + +namespace +{ + +bool parseUndropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_table("TABLE"); + ParserToken s_dot(TokenType::Dot); + ParserIdentifier name_p(true); + + ASTPtr database; + ASTPtr table; + String cluster_str; + /// We can specify the table's uuid for exact undrop. + /// because the same name of a table can be created and deleted multiple times, + /// and can generate multiple different uuids. + UUID uuid = UUIDHelpers::Nil; + + if (!s_table.ignore(pos, expected)) + return false; + if (!name_p.parse(pos, table, expected)) + return false; + if (s_dot.ignore(pos, expected)) + { + database = table; + if (!name_p.parse(pos, table, expected)) + return false; + } + if (ParserKeyword("UUID").ignore(pos, expected)) + { + ParserStringLiteral uuid_p; + ASTPtr ast_uuid; + if (!uuid_p.parse(pos, ast_uuid, expected)) + return false; + uuid = parseFromString(ast_uuid->as()->value.get()); + } + if (ParserKeyword{"ON"}.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + auto query = std::make_shared(); + node = query; + + query->database = database; + query->table = table; + query->uuid = uuid; + + if (database) + query->children.push_back(database); + + assert (table); + query->children.push_back(table); + + query->cluster = cluster_str; + + return true; +} + +} + +bool ParserUndropQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_undrop("UNDROP"); + + if (s_undrop.ignore(pos, expected)) + return parseUndropQuery(pos, node, expected); + else + return false; +} + +} diff --git a/src/Parsers/ParserUndropQuery.h b/src/Parsers/ParserUndropQuery.h new file mode 100644 index 00000000000..4b289600396 --- /dev/null +++ b/src/Parsers/ParserUndropQuery.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/** Query like this: + * UNDROP TABLE [db.]name [UUID uuid] + */ +class ParserUndropQuery : public IParserBase +{ +protected: + const char * getName() const override{ return "UNDROP query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index 4d89e6255d3..4ad046fe5d2 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -184,7 +184,7 @@ def test_grant_all_on_table(): assert ( instance.query("SHOW GRANTS FOR B") == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, " - "DROP TABLE, DROP VIEW, DROP DICTIONARY, TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, " + "DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, " "SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, " "SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.table TO B\n" ) diff --git a/tests/integration/test_undrop_query/__init__.py b/tests/integration/test_undrop_query/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_undrop_query/configs/with_delay_config.xml b/tests/integration/test_undrop_query/configs/with_delay_config.xml new file mode 100644 index 00000000000..c672b187b01 --- /dev/null +++ b/tests/integration/test_undrop_query/configs/with_delay_config.xml @@ -0,0 +1,3 @@ + + 5 + diff --git a/tests/integration/test_undrop_query/test.py b/tests/integration/test_undrop_query/test.py new file mode 100644 index 00000000000..63d92d84541 --- /dev/null +++ b/tests/integration/test_undrop_query/test.py @@ -0,0 +1,59 @@ +import pytest +import uuid +import random +import logging +import time + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance("node", main_configs=["configs/with_delay_config.xml"]) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_undrop_drop_and_undrop_loop(started_cluster): + count = 0 + node.query("set allow_experimental_undrop_table_query = 1;") + while count < 10: + random_sec = random.randint(0, 10) + table_uuid = uuid.uuid1().__str__() + logging.info( + "random_sec: " + random_sec.__str__() + ", table_uuid: " + table_uuid + ) + node.query( + "create table test_undrop_loop" + + count.__str__() + + " UUID '" + + table_uuid + + "' (id Int32) Engine=MergeTree() order by id;" + ) + node.query("drop table test_undrop_loop" + count.__str__() + ";") + time.sleep(random_sec) + if random_sec >= 5: + error = node.query_and_get_error( + "undrop table test_undrop_loop" + + count.__str__() + + " uuid '" + + table_uuid + + "' settings allow_experimental_undrop_table_query = 1;" + ) + assert "UNKNOWN_TABLE" in error + else: + node.query( + "undrop table test_undrop_loop" + + count.__str__() + + " uuid '" + + table_uuid + + "' settings allow_experimental_undrop_table_query = 1;" + ) + count = count + 1 diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 94ab2d8b01d..553fee1f435 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -62,6 +62,7 @@ DROP DICTIONARY [] DICTIONARY DROP DROP FUNCTION [] GLOBAL DROP DROP NAMED COLLECTION [] NAMED_COLLECTION NAMED COLLECTION CONTROL DROP [] \N ALL +UNDROP TABLE [] TABLE ALL TRUNCATE ['TRUNCATE TABLE'] TABLE ALL OPTIMIZE ['OPTIMIZE TABLE'] TABLE ALL BACKUP [] TABLE ALL diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 6fb59657305..5247dff64ad 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -289,7 +289,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'TRUNCATE' = 64, 'OPTIMIZE' = 65, 'BACKUP' = 66, 'KILL QUERY' = 67, 'KILL TRANSACTION' = 68, 'MOVE PARTITION BETWEEN SHARDS' = 69, 'CREATE USER' = 70, 'ALTER USER' = 71, 'DROP USER' = 72, 'CREATE ROLE' = 73, 'ALTER ROLE' = 74, 'DROP ROLE' = 75, 'ROLE ADMIN' = 76, 'CREATE ROW POLICY' = 77, 'ALTER ROW POLICY' = 78, 'DROP ROW POLICY' = 79, 'CREATE QUOTA' = 80, 'ALTER QUOTA' = 81, 'DROP QUOTA' = 82, 'CREATE SETTINGS PROFILE' = 83, 'ALTER SETTINGS PROFILE' = 84, 'DROP SETTINGS PROFILE' = 85, 'SHOW USERS' = 86, 'SHOW ROLES' = 87, 'SHOW ROW POLICIES' = 88, 'SHOW QUOTAS' = 89, 'SHOW SETTINGS PROFILES' = 90, 'SHOW ACCESS' = 91, 'ACCESS MANAGEMENT' = 92, 'SHOW NAMED COLLECTIONS' = 93, 'SHOW NAMED COLLECTIONS SECRETS' = 94, 'NAMED COLLECTION CONTROL' = 95, 'SYSTEM SHUTDOWN' = 96, 'SYSTEM DROP DNS CACHE' = 97, 'SYSTEM DROP MARK CACHE' = 98, 'SYSTEM DROP UNCOMPRESSED CACHE' = 99, 'SYSTEM DROP MMAP CACHE' = 100, 'SYSTEM DROP QUERY CACHE' = 101, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 102, 'SYSTEM DROP FILESYSTEM CACHE' = 103, 'SYSTEM DROP SCHEMA CACHE' = 104, 'SYSTEM DROP S3 CLIENT CACHE' = 105, 'SYSTEM DROP CACHE' = 106, 'SYSTEM RELOAD CONFIG' = 107, 'SYSTEM RELOAD USERS' = 108, 'SYSTEM RELOAD SYMBOLS' = 109, 'SYSTEM RELOAD DICTIONARY' = 110, 'SYSTEM RELOAD MODEL' = 111, 'SYSTEM RELOAD FUNCTION' = 112, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 113, 'SYSTEM RELOAD' = 114, 'SYSTEM RESTART DISK' = 115, 'SYSTEM MERGES' = 116, 'SYSTEM TTL MERGES' = 117, 'SYSTEM FETCHES' = 118, 'SYSTEM MOVES' = 119, 'SYSTEM DISTRIBUTED SENDS' = 120, 'SYSTEM REPLICATED SENDS' = 121, 'SYSTEM SENDS' = 122, 'SYSTEM REPLICATION QUEUES' = 123, 'SYSTEM DROP REPLICA' = 124, 'SYSTEM SYNC REPLICA' = 125, 'SYSTEM RESTART REPLICA' = 126, 'SYSTEM RESTORE REPLICA' = 127, 'SYSTEM WAIT LOADING PARTS' = 128, 'SYSTEM SYNC DATABASE REPLICA' = 129, 'SYSTEM SYNC TRANSACTION LOG' = 130, 'SYSTEM SYNC FILE CACHE' = 131, 'SYSTEM FLUSH DISTRIBUTED' = 132, 'SYSTEM FLUSH LOGS' = 133, 'SYSTEM FLUSH' = 134, 'SYSTEM THREAD FUZZER' = 135, 'SYSTEM UNFREEZE' = 136, 'SYSTEM' = 137, 'dictGet' = 138, 'addressToLine' = 139, 'addressToLineWithInlines' = 140, 'addressToSymbol' = 141, 'demangle' = 142, 'INTROSPECTION' = 143, 'FILE' = 144, 'URL' = 145, 'REMOTE' = 146, 'MONGO' = 147, 'MEILISEARCH' = 148, 'MYSQL' = 149, 'POSTGRES' = 150, 'SQLITE' = 151, 'ODBC' = 152, 'JDBC' = 153, 'HDFS' = 154, 'S3' = 155, 'HIVE' = 156, 'SOURCES' = 157, 'CLUSTER' = 158, 'ALL' = 159, 'NONE' = 160), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'SHOW NAMED COLLECTIONS' = 93, 'SHOW NAMED COLLECTIONS SECRETS' = 94, 'ACCESS MANAGEMENT' = 95, 'SYSTEM SHUTDOWN' = 96, 'SYSTEM DROP DNS CACHE' = 97, 'SYSTEM DROP MARK CACHE' = 98, 'SYSTEM DROP UNCOMPRESSED CACHE' = 99, 'SYSTEM DROP MMAP CACHE' = 100, 'SYSTEM DROP QUERY CACHE' = 101, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 102, 'SYSTEM DROP FILESYSTEM CACHE' = 103, 'SYSTEM DROP SCHEMA CACHE' = 104, 'SYSTEM DROP S3 CLIENT CACHE' = 105, 'SYSTEM DROP CACHE' = 106, 'SYSTEM RELOAD CONFIG' = 107, 'SYSTEM RELOAD USERS' = 108, 'SYSTEM RELOAD SYMBOLS' = 109, 'SYSTEM RELOAD DICTIONARY' = 110, 'SYSTEM RELOAD MODEL' = 111, 'SYSTEM RELOAD FUNCTION' = 112, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 113, 'SYSTEM RELOAD' = 114, 'SYSTEM RESTART DISK' = 115, 'SYSTEM MERGES' = 116, 'SYSTEM TTL MERGES' = 117, 'SYSTEM FETCHES' = 118, 'SYSTEM MOVES' = 119, 'SYSTEM DISTRIBUTED SENDS' = 120, 'SYSTEM REPLICATED SENDS' = 121, 'SYSTEM SENDS' = 122, 'SYSTEM REPLICATION QUEUES' = 123, 'SYSTEM DROP REPLICA' = 124, 'SYSTEM SYNC REPLICA' = 125, 'SYSTEM RESTART REPLICA' = 126, 'SYSTEM RESTORE REPLICA' = 127, 'SYSTEM WAIT LOADING PARTS' = 128, 'SYSTEM SYNC DATABASE REPLICA' = 129, 'SYSTEM SYNC TRANSACTION LOG' = 130, 'SYSTEM SYNC FILE CACHE' = 131, 'SYSTEM FLUSH DISTRIBUTED' = 132, 'SYSTEM FLUSH LOGS' = 133, 'SYSTEM FLUSH' = 134, 'SYSTEM THREAD FUZZER' = 135, 'SYSTEM UNFREEZE' = 136, 'SYSTEM' = 137, 'dictGet' = 138, 'addressToLine' = 139, 'addressToLineWithInlines' = 140, 'addressToSymbol' = 141, 'demangle' = 142, 'INTROSPECTION' = 143, 'FILE' = 144, 'URL' = 145, 'REMOTE' = 146, 'MONGO' = 147, 'MEILISEARCH' = 148, 'MYSQL' = 149, 'POSTGRES' = 150, 'SQLITE' = 151, 'ODBC' = 152, 'JDBC' = 153, 'HDFS' = 154, 'S3' = 155, 'HIVE' = 156, 'SOURCES' = 157, 'CLUSTER' = 158, 'ALL' = 159, 'NONE' = 160), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -570,10 +570,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'TRUNCATE' = 64, 'OPTIMIZE' = 65, 'BACKUP' = 66, 'KILL QUERY' = 67, 'KILL TRANSACTION' = 68, 'MOVE PARTITION BETWEEN SHARDS' = 69, 'CREATE USER' = 70, 'ALTER USER' = 71, 'DROP USER' = 72, 'CREATE ROLE' = 73, 'ALTER ROLE' = 74, 'DROP ROLE' = 75, 'ROLE ADMIN' = 76, 'CREATE ROW POLICY' = 77, 'ALTER ROW POLICY' = 78, 'DROP ROW POLICY' = 79, 'CREATE QUOTA' = 80, 'ALTER QUOTA' = 81, 'DROP QUOTA' = 82, 'CREATE SETTINGS PROFILE' = 83, 'ALTER SETTINGS PROFILE' = 84, 'DROP SETTINGS PROFILE' = 85, 'SHOW USERS' = 86, 'SHOW ROLES' = 87, 'SHOW ROW POLICIES' = 88, 'SHOW QUOTAS' = 89, 'SHOW SETTINGS PROFILES' = 90, 'SHOW ACCESS' = 91, 'ACCESS MANAGEMENT' = 92, 'SHOW NAMED COLLECTIONS' = 93, 'SHOW NAMED COLLECTIONS SECRETS' = 94, 'NAMED COLLECTION CONTROL' = 95, 'SYSTEM SHUTDOWN' = 96, 'SYSTEM DROP DNS CACHE' = 97, 'SYSTEM DROP MARK CACHE' = 98, 'SYSTEM DROP UNCOMPRESSED CACHE' = 99, 'SYSTEM DROP MMAP CACHE' = 100, 'SYSTEM DROP QUERY CACHE' = 101, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 102, 'SYSTEM DROP FILESYSTEM CACHE' = 103, 'SYSTEM DROP SCHEMA CACHE' = 104, 'SYSTEM DROP S3 CLIENT CACHE' = 105, 'SYSTEM DROP CACHE' = 106, 'SYSTEM RELOAD CONFIG' = 107, 'SYSTEM RELOAD USERS' = 108, 'SYSTEM RELOAD SYMBOLS' = 109, 'SYSTEM RELOAD DICTIONARY' = 110, 'SYSTEM RELOAD MODEL' = 111, 'SYSTEM RELOAD FUNCTION' = 112, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 113, 'SYSTEM RELOAD' = 114, 'SYSTEM RESTART DISK' = 115, 'SYSTEM MERGES' = 116, 'SYSTEM TTL MERGES' = 117, 'SYSTEM FETCHES' = 118, 'SYSTEM MOVES' = 119, 'SYSTEM DISTRIBUTED SENDS' = 120, 'SYSTEM REPLICATED SENDS' = 121, 'SYSTEM SENDS' = 122, 'SYSTEM REPLICATION QUEUES' = 123, 'SYSTEM DROP REPLICA' = 124, 'SYSTEM SYNC REPLICA' = 125, 'SYSTEM RESTART REPLICA' = 126, 'SYSTEM RESTORE REPLICA' = 127, 'SYSTEM WAIT LOADING PARTS' = 128, 'SYSTEM SYNC DATABASE REPLICA' = 129, 'SYSTEM SYNC TRANSACTION LOG' = 130, 'SYSTEM SYNC FILE CACHE' = 131, 'SYSTEM FLUSH DISTRIBUTED' = 132, 'SYSTEM FLUSH LOGS' = 133, 'SYSTEM FLUSH' = 134, 'SYSTEM THREAD FUZZER' = 135, 'SYSTEM UNFREEZE' = 136, 'SYSTEM' = 137, 'dictGet' = 138, 'addressToLine' = 139, 'addressToLineWithInlines' = 140, 'addressToSymbol' = 141, 'demangle' = 142, 'INTROSPECTION' = 143, 'FILE' = 144, 'URL' = 145, 'REMOTE' = 146, 'MONGO' = 147, 'MEILISEARCH' = 148, 'MYSQL' = 149, 'POSTGRES' = 150, 'SQLITE' = 151, 'ODBC' = 152, 'JDBC' = 153, 'HDFS' = 154, 'S3' = 155, 'HIVE' = 156, 'SOURCES' = 157, 'CLUSTER' = 158, 'ALL' = 159, 'NONE' = 160), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'SHOW NAMED COLLECTIONS' = 93, 'SHOW NAMED COLLECTIONS SECRETS' = 94, 'ACCESS MANAGEMENT' = 95, 'SYSTEM SHUTDOWN' = 96, 'SYSTEM DROP DNS CACHE' = 97, 'SYSTEM DROP MARK CACHE' = 98, 'SYSTEM DROP UNCOMPRESSED CACHE' = 99, 'SYSTEM DROP MMAP CACHE' = 100, 'SYSTEM DROP QUERY CACHE' = 101, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 102, 'SYSTEM DROP FILESYSTEM CACHE' = 103, 'SYSTEM DROP SCHEMA CACHE' = 104, 'SYSTEM DROP S3 CLIENT CACHE' = 105, 'SYSTEM DROP CACHE' = 106, 'SYSTEM RELOAD CONFIG' = 107, 'SYSTEM RELOAD USERS' = 108, 'SYSTEM RELOAD SYMBOLS' = 109, 'SYSTEM RELOAD DICTIONARY' = 110, 'SYSTEM RELOAD MODEL' = 111, 'SYSTEM RELOAD FUNCTION' = 112, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 113, 'SYSTEM RELOAD' = 114, 'SYSTEM RESTART DISK' = 115, 'SYSTEM MERGES' = 116, 'SYSTEM TTL MERGES' = 117, 'SYSTEM FETCHES' = 118, 'SYSTEM MOVES' = 119, 'SYSTEM DISTRIBUTED SENDS' = 120, 'SYSTEM REPLICATED SENDS' = 121, 'SYSTEM SENDS' = 122, 'SYSTEM REPLICATION QUEUES' = 123, 'SYSTEM DROP REPLICA' = 124, 'SYSTEM SYNC REPLICA' = 125, 'SYSTEM RESTART REPLICA' = 126, 'SYSTEM RESTORE REPLICA' = 127, 'SYSTEM WAIT LOADING PARTS' = 128, 'SYSTEM SYNC DATABASE REPLICA' = 129, 'SYSTEM SYNC TRANSACTION LOG' = 130, 'SYSTEM SYNC FILE CACHE' = 131, 'SYSTEM FLUSH DISTRIBUTED' = 132, 'SYSTEM FLUSH LOGS' = 133, 'SYSTEM FLUSH' = 134, 'SYSTEM THREAD FUZZER' = 135, 'SYSTEM UNFREEZE' = 136, 'SYSTEM' = 137, 'dictGet' = 138, 'addressToLine' = 139, 'addressToLineWithInlines' = 140, 'addressToSymbol' = 141, 'demangle' = 142, 'INTROSPECTION' = 143, 'FILE' = 144, 'URL' = 145, 'REMOTE' = 146, 'MONGO' = 147, 'MEILISEARCH' = 148, 'MYSQL' = 149, 'POSTGRES' = 150, 'SQLITE' = 151, 'ODBC' = 152, 'JDBC' = 153, 'HDFS' = 154, 'S3' = 155, 'HIVE' = 156, 'SOURCES' = 157, 'CLUSTER' = 158, 'ALL' = 159, 'NONE' = 160), `aliases` Array(String), - `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'TRUNCATE' = 64, 'OPTIMIZE' = 65, 'BACKUP' = 66, 'KILL QUERY' = 67, 'KILL TRANSACTION' = 68, 'MOVE PARTITION BETWEEN SHARDS' = 69, 'CREATE USER' = 70, 'ALTER USER' = 71, 'DROP USER' = 72, 'CREATE ROLE' = 73, 'ALTER ROLE' = 74, 'DROP ROLE' = 75, 'ROLE ADMIN' = 76, 'CREATE ROW POLICY' = 77, 'ALTER ROW POLICY' = 78, 'DROP ROW POLICY' = 79, 'CREATE QUOTA' = 80, 'ALTER QUOTA' = 81, 'DROP QUOTA' = 82, 'CREATE SETTINGS PROFILE' = 83, 'ALTER SETTINGS PROFILE' = 84, 'DROP SETTINGS PROFILE' = 85, 'SHOW USERS' = 86, 'SHOW ROLES' = 87, 'SHOW ROW POLICIES' = 88, 'SHOW QUOTAS' = 89, 'SHOW SETTINGS PROFILES' = 90, 'SHOW ACCESS' = 91, 'ACCESS MANAGEMENT' = 92, 'SHOW NAMED COLLECTIONS' = 93, 'SHOW NAMED COLLECTIONS SECRETS' = 94, 'NAMED COLLECTION CONTROL' = 95, 'SYSTEM SHUTDOWN' = 96, 'SYSTEM DROP DNS CACHE' = 97, 'SYSTEM DROP MARK CACHE' = 98, 'SYSTEM DROP UNCOMPRESSED CACHE' = 99, 'SYSTEM DROP MMAP CACHE' = 100, 'SYSTEM DROP QUERY CACHE' = 101, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 102, 'SYSTEM DROP FILESYSTEM CACHE' = 103, 'SYSTEM DROP SCHEMA CACHE' = 104, 'SYSTEM DROP S3 CLIENT CACHE' = 105, 'SYSTEM DROP CACHE' = 106, 'SYSTEM RELOAD CONFIG' = 107, 'SYSTEM RELOAD USERS' = 108, 'SYSTEM RELOAD SYMBOLS' = 109, 'SYSTEM RELOAD DICTIONARY' = 110, 'SYSTEM RELOAD MODEL' = 111, 'SYSTEM RELOAD FUNCTION' = 112, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 113, 'SYSTEM RELOAD' = 114, 'SYSTEM RESTART DISK' = 115, 'SYSTEM MERGES' = 116, 'SYSTEM TTL MERGES' = 117, 'SYSTEM FETCHES' = 118, 'SYSTEM MOVES' = 119, 'SYSTEM DISTRIBUTED SENDS' = 120, 'SYSTEM REPLICATED SENDS' = 121, 'SYSTEM SENDS' = 122, 'SYSTEM REPLICATION QUEUES' = 123, 'SYSTEM DROP REPLICA' = 124, 'SYSTEM SYNC REPLICA' = 125, 'SYSTEM RESTART REPLICA' = 126, 'SYSTEM RESTORE REPLICA' = 127, 'SYSTEM WAIT LOADING PARTS' = 128, 'SYSTEM SYNC DATABASE REPLICA' = 129, 'SYSTEM SYNC TRANSACTION LOG' = 130, 'SYSTEM SYNC FILE CACHE' = 131, 'SYSTEM FLUSH DISTRIBUTED' = 132, 'SYSTEM FLUSH LOGS' = 133, 'SYSTEM FLUSH' = 134, 'SYSTEM THREAD FUZZER' = 135, 'SYSTEM UNFREEZE' = 136, 'SYSTEM' = 137, 'dictGet' = 138, 'addressToLine' = 139, 'addressToLineWithInlines' = 140, 'addressToSymbol' = 141, 'demangle' = 142, 'INTROSPECTION' = 143, 'FILE' = 144, 'URL' = 145, 'REMOTE' = 146, 'MONGO' = 147, 'MEILISEARCH' = 148, 'MYSQL' = 149, 'POSTGRES' = 150, 'SQLITE' = 151, 'ODBC' = 152, 'JDBC' = 153, 'HDFS' = 154, 'S3' = 155, 'HIVE' = 156, 'SOURCES' = 157, 'CLUSTER' = 158, 'ALL' = 159, 'NONE' = 160)) + `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5)), + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'SHOW NAMED COLLECTIONS' = 93, 'SHOW NAMED COLLECTIONS SECRETS' = 94, 'ACCESS MANAGEMENT' = 95, 'SYSTEM SHUTDOWN' = 96, 'SYSTEM DROP DNS CACHE' = 97, 'SYSTEM DROP MARK CACHE' = 98, 'SYSTEM DROP UNCOMPRESSED CACHE' = 99, 'SYSTEM DROP MMAP CACHE' = 100, 'SYSTEM DROP QUERY CACHE' = 101, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 102, 'SYSTEM DROP FILESYSTEM CACHE' = 103, 'SYSTEM DROP SCHEMA CACHE' = 104, 'SYSTEM DROP S3 CLIENT CACHE' = 105, 'SYSTEM DROP CACHE' = 106, 'SYSTEM RELOAD CONFIG' = 107, 'SYSTEM RELOAD USERS' = 108, 'SYSTEM RELOAD SYMBOLS' = 109, 'SYSTEM RELOAD DICTIONARY' = 110, 'SYSTEM RELOAD MODEL' = 111, 'SYSTEM RELOAD FUNCTION' = 112, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 113, 'SYSTEM RELOAD' = 114, 'SYSTEM RESTART DISK' = 115, 'SYSTEM MERGES' = 116, 'SYSTEM TTL MERGES' = 117, 'SYSTEM FETCHES' = 118, 'SYSTEM MOVES' = 119, 'SYSTEM DISTRIBUTED SENDS' = 120, 'SYSTEM REPLICATED SENDS' = 121, 'SYSTEM SENDS' = 122, 'SYSTEM REPLICATION QUEUES' = 123, 'SYSTEM DROP REPLICA' = 124, 'SYSTEM SYNC REPLICA' = 125, 'SYSTEM RESTART REPLICA' = 126, 'SYSTEM RESTORE REPLICA' = 127, 'SYSTEM WAIT LOADING PARTS' = 128, 'SYSTEM SYNC DATABASE REPLICA' = 129, 'SYSTEM SYNC TRANSACTION LOG' = 130, 'SYSTEM SYNC FILE CACHE' = 131, 'SYSTEM FLUSH DISTRIBUTED' = 132, 'SYSTEM FLUSH LOGS' = 133, 'SYSTEM FLUSH' = 134, 'SYSTEM THREAD FUZZER' = 135, 'SYSTEM UNFREEZE' = 136, 'SYSTEM' = 137, 'dictGet' = 138, 'addressToLine' = 139, 'addressToLineWithInlines' = 140, 'addressToSymbol' = 141, 'demangle' = 142, 'INTROSPECTION' = 143, 'FILE' = 144, 'URL' = 145, 'REMOTE' = 146, 'MONGO' = 147, 'MEILISEARCH' = 148, 'MYSQL' = 149, 'POSTGRES' = 150, 'SQLITE' = 151, 'ODBC' = 152, 'JDBC' = 153, 'HDFS' = 154, 'S3' = 155, 'HIVE' = 156, 'SOURCES' = 157, 'CLUSTER' = 158, 'ALL' = 159, 'NONE' = 160)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' diff --git a/tests/queries/0_stateless/02681_undrop_query.reference b/tests/queries/0_stateless/02681_undrop_query.reference new file mode 100644 index 00000000000..5df6c392eb9 --- /dev/null +++ b/tests/queries/0_stateless/02681_undrop_query.reference @@ -0,0 +1,32 @@ +test MergeTree undrop +02681_undrop_mergetree +1 +2 +3 +test detach +UPDATE num = 2 WHERE id = 1 +test MergeTree with cluster +02681_undrop_uuid_on_cluster +1 +2 +3 +test MergeTree without uuid on cluster +02681_undrop_no_uuid_on_cluster +1 +2 +3 +test ReplicatedMergeTree undrop +02681_undrop_replicatedmergetree +1 +2 +3 +test Log undrop +02681_undrop_log +1 +2 +3 +test Distributed undrop +02681_undrop_distributed +test MergeTree drop and undrop multiple times +02681_undrop_multiple +3 diff --git a/tests/queries/0_stateless/02681_undrop_query.sql b/tests/queries/0_stateless/02681_undrop_query.sql new file mode 100644 index 00000000000..5004eec77f1 --- /dev/null +++ b/tests/queries/0_stateless/02681_undrop_query.sql @@ -0,0 +1,90 @@ +-- Tags: no-ordinary-database, no-replicated-database, distributed, zookeeper + +set database_atomic_wait_for_drop_and_detach_synchronously = 0; +set allow_experimental_undrop_table_query = 1; + +select 'test MergeTree undrop'; +drop table if exists 02681_undrop_mergetree sync; +create table 02681_undrop_mergetree (id Int32) Engine=MergeTree() order by id; +insert into 02681_undrop_mergetree values (1),(2),(3); +drop table 02681_undrop_mergetree; +select table from system.marked_dropped_tables where table = '02681_undrop_mergetree' limit 1; +undrop table 02681_undrop_mergetree; +select * from 02681_undrop_mergetree order by id; +drop table 02681_undrop_mergetree sync; + +select 'test detach'; +drop table if exists 02681_undrop_detach sync; +create table 02681_undrop_detach (id Int32, num Int32) Engine=MergeTree() order by id; +insert into 02681_undrop_detach values (1, 1); +detach table 02681_undrop_detach; +undrop table 02681_undrop_detach; -- { serverError 57 } +attach table 02681_undrop_detach; +alter table 02681_undrop_detach update num = 2 where id = 1; +select command from system.mutations where table='02681_undrop_detach' limit 1; +drop table 02681_undrop_detach sync; + +select 'test MergeTree with cluster'; +drop table if exists 02681_undrop_uuid_on_cluster on cluster test_shard_localhost sync format Null; +create table 02681_undrop_uuid_on_cluster on cluster test_shard_localhost (id Int32) Engine=MergeTree() order by id format Null; +insert into 02681_undrop_uuid_on_cluster values (1),(2),(3); +drop table 02681_undrop_uuid_on_cluster on cluster test_shard_localhost format Null; +select table from system.marked_dropped_tables where table = '02681_undrop_uuid_on_cluster' limit 1; +undrop table 02681_undrop_uuid_on_cluster on cluster test_shard_localhost format Null; +select * from 02681_undrop_uuid_on_cluster order by id; +drop table 02681_undrop_uuid_on_cluster sync; + +select 'test MergeTree without uuid on cluster'; +drop table if exists 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost sync format Null; +create table 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost (id Int32) Engine=MergeTree() order by id format Null; +insert into 02681_undrop_no_uuid_on_cluster values (1),(2),(3); +drop table 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost format Null; +select table from system.marked_dropped_tables where table = '02681_undrop_no_uuid_on_cluster' limit 1; +undrop table 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost format Null; +select * from 02681_undrop_no_uuid_on_cluster order by id; +drop table 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost sync format Null; + +select 'test ReplicatedMergeTree undrop'; +drop table if exists 02681_undrop_replicatedmergetree sync; +create table 02681_undrop_replicatedmergetree (id Int32) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/02681_undrop_replicatedmergetree', 'test_undrop') order by id; +insert into 02681_undrop_replicatedmergetree values (1),(2),(3); +drop table 02681_undrop_replicatedmergetree; +select table from system.marked_dropped_tables where table = '02681_undrop_replicatedmergetree' limit 1; +undrop table 02681_undrop_replicatedmergetree; +select * from 02681_undrop_replicatedmergetree order by id; +drop table 02681_undrop_replicatedmergetree sync; + +select 'test Log undrop'; +drop table if exists 02681_undrop_log sync; +create table 02681_undrop_log (id Int32) Engine=Log(); +insert into 02681_undrop_log values (1),(2),(3); +drop table 02681_undrop_log; +select table from system.marked_dropped_tables where table = '02681_undrop_log' limit 1; +undrop table 02681_undrop_log; +select * from 02681_undrop_log order by id; +drop table 02681_undrop_log sync; + +select 'test Distributed undrop'; +drop table if exists 02681_undrop_distributed sync; +create table 02681_undrop_distributed (id Int32) Engine = Distributed(test_shard_localhost, currentDatabase(), 02681_undrop, rand()); +drop table 02681_undrop_distributed; +select table from system.marked_dropped_tables where table = '02681_undrop_distributed' limit 1; +undrop table 02681_undrop_distributed; +drop table 02681_undrop_distributed sync; + +select 'test MergeTree drop and undrop multiple times'; +drop table if exists 02681_undrop_multiple sync; +create table 02681_undrop_multiple (id Int32) Engine=MergeTree() order by id; +insert into 02681_undrop_multiple values (1); +drop table 02681_undrop_multiple; +create table 02681_undrop_multiple (id Int32) Engine=MergeTree() order by id; +insert into 02681_undrop_multiple values (2); +drop table 02681_undrop_multiple; +create table 02681_undrop_multiple (id Int32) Engine=MergeTree() order by id; +insert into 02681_undrop_multiple values (3); +drop table 02681_undrop_multiple; +select table from system.marked_dropped_tables where table = '02681_undrop_multiple' limit 1; +undrop table 02681_undrop_multiple; +select * from 02681_undrop_multiple order by id; +undrop table 02681_undrop_multiple; -- { serverError 57 } +drop table 02681_undrop_multiple sync; diff --git a/tests/queries/0_stateless/02681_undrop_query_uuid.reference b/tests/queries/0_stateless/02681_undrop_query_uuid.reference new file mode 100644 index 00000000000..9d36a21dbda --- /dev/null +++ b/tests/queries/0_stateless/02681_undrop_query_uuid.reference @@ -0,0 +1,5 @@ +test MergeTree with uuid +02681_undrop_uuid +1 +2 +3 diff --git a/tests/queries/0_stateless/02681_undrop_query_uuid.sh b/tests/queries/0_stateless/02681_undrop_query_uuid.sh new file mode 100755 index 00000000000..33ff0fa908e --- /dev/null +++ b/tests/queries/0_stateless/02681_undrop_query_uuid.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash +# Tags: no-ordinary-database, distributed, zookeeper + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +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;" +${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 "select table from system.marked_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 "select * from 02681_undrop_uuid order by id;" +${CLICKHOUSE_CLIENT} -q "drop table 02681_undrop_uuid sync;" From d1c3b1bd270853eb856daed2a0d59a2092324680 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Tue, 21 Mar 2023 20:34:17 +0800 Subject: [PATCH 2/8] fix test --- .../0_stateless/02117_show_create_table_system.reference | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 5247dff64ad..b07d6e01161 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -289,7 +289,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'SHOW NAMED COLLECTIONS' = 93, 'SHOW NAMED COLLECTIONS SECRETS' = 94, 'ACCESS MANAGEMENT' = 95, 'SYSTEM SHUTDOWN' = 96, 'SYSTEM DROP DNS CACHE' = 97, 'SYSTEM DROP MARK CACHE' = 98, 'SYSTEM DROP UNCOMPRESSED CACHE' = 99, 'SYSTEM DROP MMAP CACHE' = 100, 'SYSTEM DROP QUERY CACHE' = 101, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 102, 'SYSTEM DROP FILESYSTEM CACHE' = 103, 'SYSTEM DROP SCHEMA CACHE' = 104, 'SYSTEM DROP S3 CLIENT CACHE' = 105, 'SYSTEM DROP CACHE' = 106, 'SYSTEM RELOAD CONFIG' = 107, 'SYSTEM RELOAD USERS' = 108, 'SYSTEM RELOAD SYMBOLS' = 109, 'SYSTEM RELOAD DICTIONARY' = 110, 'SYSTEM RELOAD MODEL' = 111, 'SYSTEM RELOAD FUNCTION' = 112, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 113, 'SYSTEM RELOAD' = 114, 'SYSTEM RESTART DISK' = 115, 'SYSTEM MERGES' = 116, 'SYSTEM TTL MERGES' = 117, 'SYSTEM FETCHES' = 118, 'SYSTEM MOVES' = 119, 'SYSTEM DISTRIBUTED SENDS' = 120, 'SYSTEM REPLICATED SENDS' = 121, 'SYSTEM SENDS' = 122, 'SYSTEM REPLICATION QUEUES' = 123, 'SYSTEM DROP REPLICA' = 124, 'SYSTEM SYNC REPLICA' = 125, 'SYSTEM RESTART REPLICA' = 126, 'SYSTEM RESTORE REPLICA' = 127, 'SYSTEM WAIT LOADING PARTS' = 128, 'SYSTEM SYNC DATABASE REPLICA' = 129, 'SYSTEM SYNC TRANSACTION LOG' = 130, 'SYSTEM SYNC FILE CACHE' = 131, 'SYSTEM FLUSH DISTRIBUTED' = 132, 'SYSTEM FLUSH LOGS' = 133, 'SYSTEM FLUSH' = 134, 'SYSTEM THREAD FUZZER' = 135, 'SYSTEM UNFREEZE' = 136, 'SYSTEM' = 137, 'dictGet' = 138, 'addressToLine' = 139, 'addressToLineWithInlines' = 140, 'addressToSymbol' = 141, 'demangle' = 142, 'INTROSPECTION' = 143, 'FILE' = 144, 'URL' = 145, 'REMOTE' = 146, 'MONGO' = 147, 'MEILISEARCH' = 148, 'MYSQL' = 149, 'POSTGRES' = 150, 'SQLITE' = 151, 'ODBC' = 152, 'JDBC' = 153, 'HDFS' = 154, 'S3' = 155, 'HIVE' = 156, 'SOURCES' = 157, 'CLUSTER' = 158, 'ALL' = 159, 'NONE' = 160), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM' = 138, 'dictGet' = 139, 'addressToLine' = 140, 'addressToLineWithInlines' = 141, 'addressToSymbol' = 142, 'demangle' = 143, 'INTROSPECTION' = 144, 'FILE' = 145, 'URL' = 146, 'REMOTE' = 147, 'MONGO' = 148, 'MEILISEARCH' = 149, 'MYSQL' = 150, 'POSTGRES' = 151, 'SQLITE' = 152, 'ODBC' = 153, 'JDBC' = 154, 'HDFS' = 155, 'S3' = 156, 'HIVE' = 157, 'SOURCES' = 158, 'CLUSTER' = 159, 'ALL' = 160, 'NONE' = 161), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -570,10 +570,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'SHOW NAMED COLLECTIONS' = 93, 'SHOW NAMED COLLECTIONS SECRETS' = 94, 'ACCESS MANAGEMENT' = 95, 'SYSTEM SHUTDOWN' = 96, 'SYSTEM DROP DNS CACHE' = 97, 'SYSTEM DROP MARK CACHE' = 98, 'SYSTEM DROP UNCOMPRESSED CACHE' = 99, 'SYSTEM DROP MMAP CACHE' = 100, 'SYSTEM DROP QUERY CACHE' = 101, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 102, 'SYSTEM DROP FILESYSTEM CACHE' = 103, 'SYSTEM DROP SCHEMA CACHE' = 104, 'SYSTEM DROP S3 CLIENT CACHE' = 105, 'SYSTEM DROP CACHE' = 106, 'SYSTEM RELOAD CONFIG' = 107, 'SYSTEM RELOAD USERS' = 108, 'SYSTEM RELOAD SYMBOLS' = 109, 'SYSTEM RELOAD DICTIONARY' = 110, 'SYSTEM RELOAD MODEL' = 111, 'SYSTEM RELOAD FUNCTION' = 112, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 113, 'SYSTEM RELOAD' = 114, 'SYSTEM RESTART DISK' = 115, 'SYSTEM MERGES' = 116, 'SYSTEM TTL MERGES' = 117, 'SYSTEM FETCHES' = 118, 'SYSTEM MOVES' = 119, 'SYSTEM DISTRIBUTED SENDS' = 120, 'SYSTEM REPLICATED SENDS' = 121, 'SYSTEM SENDS' = 122, 'SYSTEM REPLICATION QUEUES' = 123, 'SYSTEM DROP REPLICA' = 124, 'SYSTEM SYNC REPLICA' = 125, 'SYSTEM RESTART REPLICA' = 126, 'SYSTEM RESTORE REPLICA' = 127, 'SYSTEM WAIT LOADING PARTS' = 128, 'SYSTEM SYNC DATABASE REPLICA' = 129, 'SYSTEM SYNC TRANSACTION LOG' = 130, 'SYSTEM SYNC FILE CACHE' = 131, 'SYSTEM FLUSH DISTRIBUTED' = 132, 'SYSTEM FLUSH LOGS' = 133, 'SYSTEM FLUSH' = 134, 'SYSTEM THREAD FUZZER' = 135, 'SYSTEM UNFREEZE' = 136, 'SYSTEM' = 137, 'dictGet' = 138, 'addressToLine' = 139, 'addressToLineWithInlines' = 140, 'addressToSymbol' = 141, 'demangle' = 142, 'INTROSPECTION' = 143, 'FILE' = 144, 'URL' = 145, 'REMOTE' = 146, 'MONGO' = 147, 'MEILISEARCH' = 148, 'MYSQL' = 149, 'POSTGRES' = 150, 'SQLITE' = 151, 'ODBC' = 152, 'JDBC' = 153, 'HDFS' = 154, 'S3' = 155, 'HIVE' = 156, 'SOURCES' = 157, 'CLUSTER' = 158, 'ALL' = 159, 'NONE' = 160), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM' = 138, 'dictGet' = 139, 'addressToLine' = 140, 'addressToLineWithInlines' = 141, 'addressToSymbol' = 142, 'demangle' = 143, 'INTROSPECTION' = 144, 'FILE' = 145, 'URL' = 146, 'REMOTE' = 147, 'MONGO' = 148, 'MEILISEARCH' = 149, 'MYSQL' = 150, 'POSTGRES' = 151, 'SQLITE' = 152, 'ODBC' = 153, 'JDBC' = 154, 'HDFS' = 155, 'S3' = 156, 'HIVE' = 157, 'SOURCES' = 158, 'CLUSTER' = 159, 'ALL' = 160, 'NONE' = 161), `aliases` Array(String), - `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'SHOW NAMED COLLECTIONS' = 93, 'SHOW NAMED COLLECTIONS SECRETS' = 94, 'ACCESS MANAGEMENT' = 95, 'SYSTEM SHUTDOWN' = 96, 'SYSTEM DROP DNS CACHE' = 97, 'SYSTEM DROP MARK CACHE' = 98, 'SYSTEM DROP UNCOMPRESSED CACHE' = 99, 'SYSTEM DROP MMAP CACHE' = 100, 'SYSTEM DROP QUERY CACHE' = 101, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 102, 'SYSTEM DROP FILESYSTEM CACHE' = 103, 'SYSTEM DROP SCHEMA CACHE' = 104, 'SYSTEM DROP S3 CLIENT CACHE' = 105, 'SYSTEM DROP CACHE' = 106, 'SYSTEM RELOAD CONFIG' = 107, 'SYSTEM RELOAD USERS' = 108, 'SYSTEM RELOAD SYMBOLS' = 109, 'SYSTEM RELOAD DICTIONARY' = 110, 'SYSTEM RELOAD MODEL' = 111, 'SYSTEM RELOAD FUNCTION' = 112, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 113, 'SYSTEM RELOAD' = 114, 'SYSTEM RESTART DISK' = 115, 'SYSTEM MERGES' = 116, 'SYSTEM TTL MERGES' = 117, 'SYSTEM FETCHES' = 118, 'SYSTEM MOVES' = 119, 'SYSTEM DISTRIBUTED SENDS' = 120, 'SYSTEM REPLICATED SENDS' = 121, 'SYSTEM SENDS' = 122, 'SYSTEM REPLICATION QUEUES' = 123, 'SYSTEM DROP REPLICA' = 124, 'SYSTEM SYNC REPLICA' = 125, 'SYSTEM RESTART REPLICA' = 126, 'SYSTEM RESTORE REPLICA' = 127, 'SYSTEM WAIT LOADING PARTS' = 128, 'SYSTEM SYNC DATABASE REPLICA' = 129, 'SYSTEM SYNC TRANSACTION LOG' = 130, 'SYSTEM SYNC FILE CACHE' = 131, 'SYSTEM FLUSH DISTRIBUTED' = 132, 'SYSTEM FLUSH LOGS' = 133, 'SYSTEM FLUSH' = 134, 'SYSTEM THREAD FUZZER' = 135, 'SYSTEM UNFREEZE' = 136, 'SYSTEM' = 137, 'dictGet' = 138, 'addressToLine' = 139, 'addressToLineWithInlines' = 140, 'addressToSymbol' = 141, 'demangle' = 142, 'INTROSPECTION' = 143, 'FILE' = 144, 'URL' = 145, 'REMOTE' = 146, 'MONGO' = 147, 'MEILISEARCH' = 148, 'MYSQL' = 149, 'POSTGRES' = 150, 'SQLITE' = 151, 'ODBC' = 152, 'JDBC' = 153, 'HDFS' = 154, 'S3' = 155, 'HIVE' = 156, 'SOURCES' = 157, 'CLUSTER' = 158, 'ALL' = 159, 'NONE' = 160)) + `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)), + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM' = 138, 'dictGet' = 139, 'addressToLine' = 140, 'addressToLineWithInlines' = 141, 'addressToSymbol' = 142, 'demangle' = 143, 'INTROSPECTION' = 144, 'FILE' = 145, 'URL' = 146, 'REMOTE' = 147, 'MONGO' = 148, 'MEILISEARCH' = 149, 'MYSQL' = 150, 'POSTGRES' = 151, 'SQLITE' = 152, 'ODBC' = 153, 'JDBC' = 154, 'HDFS' = 155, 'S3' = 156, 'HIVE' = 157, 'SOURCES' = 158, 'CLUSTER' = 159, 'ALL' = 160, 'NONE' = 161)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' From 6b59836b3ab7849d5e9055893e81b625356ef86d Mon Sep 17 00:00:00 2001 From: chen Date: Wed, 22 Mar 2023 14:27:56 +0800 Subject: [PATCH 3/8] Update 02681_undrop_query_uuid.sh --- tests/queries/0_stateless/02681_undrop_query_uuid.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02681_undrop_query_uuid.sh b/tests/queries/0_stateless/02681_undrop_query_uuid.sh index 33ff0fa908e..42f2b2c2f6c 100755 --- a/tests/queries/0_stateless/02681_undrop_query_uuid.sh +++ b/tests/queries/0_stateless/02681_undrop_query_uuid.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-ordinary-database, distributed, zookeeper +# Tags: no-ordinary-database, no-replicated-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 5036d19af3fc8a6771f32cd1cb730affb164c5ab Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Sun, 26 Mar 2023 13:15:23 +0800 Subject: [PATCH 4/8] better --- src/Interpreters/Context.h | 4 + src/Interpreters/DatabaseCatalog.cpp | 98 ++++++++----------- src/Interpreters/DatabaseCatalog.h | 2 + src/Interpreters/InterpreterCreateQuery.cpp | 5 +- src/Interpreters/InterpreterUndropQuery.cpp | 5 +- .../System/StorageSystemDroppedTables.cpp | 3 + .../25400_marked_dropped_tables.reference | 1 + 7 files changed, 58 insertions(+), 60 deletions(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index bbfbd4defdc..b9ef5e88a2a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -402,6 +402,7 @@ 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. @@ -1014,6 +1015,9 @@ 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 diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 4b4032f65a8..27a8743089c 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -947,18 +948,17 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) { String latest_metadata_dropped_path; - StorageID dropped_table_id = table_id; TableMarkedAsDropped dropped_table; { std::lock_guard lock(tables_marked_dropped_mutex); time_t latest_drop_time = std::numeric_limits::min(); - auto it_table = tables_marked_dropped.end(); + auto it_dropped_table = tables_marked_dropped.end(); for (auto it = tables_marked_dropped.begin(); it != tables_marked_dropped.end(); ++it) { auto storage_ptr = it->table; if (it->table_id.uuid == table_id.uuid) { - it_table = it; + it_dropped_table = it; dropped_table = *it; break; } @@ -970,53 +970,43 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) it->drop_time >= latest_drop_time) { latest_drop_time = it->drop_time; - it_table = it; + it_dropped_table = it; dropped_table = *it; } } - if (it_table == tables_marked_dropped.end()) + if (it_dropped_table == tables_marked_dropped.end()) throw Exception(ErrorCodes::UNKNOWN_TABLE, "The drop task of table {} is in progress, has been dropped or the database engine doesn't support it", table_id.getNameForLogs()); - latest_metadata_dropped_path = it_table->metadata_path; - dropped_table_id = it_table->table_id; - tables_marked_dropped.erase(it_table); - [[maybe_unused]] auto removed = tables_marked_dropped_ids.erase(dropped_table_id.uuid); + latest_metadata_dropped_path = it_dropped_table->metadata_path; + /// If is_being_undropped of table is true, table drop task will skip this table. + it_dropped_table->is_being_undropped = true; + String table_metadata_path = getPathForMetadata(it_dropped_table->table_id); + + try + { + /// a table is successfully marked undropped, + /// if and only if its metadata file was moved to a database. + renameNoReplace(latest_metadata_dropped_path, table_metadata_path); + } + catch (...) + { + it_dropped_table->is_being_undropped = false; + throw Exception( + ErrorCodes::FS_METADATA_ERROR, + "Cannot undrop table {}, failed to rename {} to {}", + dropped_table.table_id.getNameForLogs(), + latest_metadata_dropped_path, + table_metadata_path); + } + + tables_marked_dropped.erase(it_dropped_table); + [[maybe_unused]] auto removed = tables_marked_dropped_ids.erase(dropped_table.table_id.uuid); assert(removed); - } - /// Remove the table from tables_marked_dropped and tables_marked_dropped_ids, - /// and the drop task for this table will no longer be scheduled. - LOG_INFO(log, "Trying Undrop table {} from {}", dropped_table_id.getNameForLogs(), latest_metadata_dropped_path); - String table_metadata_path = getPathForMetadata(dropped_table_id); - - auto enqueue = [&]() - { - /// In the dropTableDataTask method. - /// 1. We will first determine whether there are tables to be dropped in tables_marked_dropped. - /// 2. If one is exist, the table will be removed from tables_marked_dropped. - /// 3. And then execute dropTableFinally. - /// So undrop and drop do not cross-execute. - std::lock_guard lock(tables_marked_dropped_mutex); - tables_marked_dropped.emplace_back(dropped_table); - tables_marked_dropped_ids.insert(dropped_table_id.uuid); - CurrentMetrics::add(CurrentMetrics::TablesToDropQueueSize, 1); - }; - - ASTPtr ast = DatabaseOnDisk::parseQueryFromMetadata( - log, getContext(), latest_metadata_dropped_path, /*throw_on_error*/ true, /*remove_empty*/ false); - auto * create = typeid_cast(ast.get()); - if (!create) - { - enqueue(); - throw Exception( - ErrorCodes::FS_METADATA_ERROR, - "Cannot parse metadata of table {} from {}", - dropped_table_id.getNameForLogs(), - table_metadata_path); + CurrentMetrics::sub(CurrentMetrics::TablesToDropQueueSize, 1); } - create->setDatabase(dropped_table_id.database_name); - create->setTable(dropped_table_id.table_name); + LOG_INFO(log, "Trying Undrop table {} from {}", dropped_table.table_id.getNameForLogs(), latest_metadata_dropped_path); try { @@ -1033,26 +1023,22 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) } }; wait_dropped_table_not_in_use(); - auto database = getDatabase(dropped_table_id.database_name, getContext()); - String relative_table_path = fs::path(database->getDataPath()) / DatabaseCatalog::instance().getPathForUUID(dropped_table.table_id.uuid); - auto storage = createTableFromAST( - *create, - dropped_table.table_id.database_name, - relative_table_path, - getContext(), - /* force_restore */ true).second; - database->undropTable(getContext(), dropped_table_id.table_name, storage, relative_table_path); - storage->startup(); - CurrentMetrics::sub(CurrentMetrics::TablesToDropQueueSize, 1); + + String query = fmt::format("ATTACH TABLE {}", dropped_table.table_id.getFullTableName()); + 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); + executeQuery(query, query_context, true); + LOG_INFO(log, "Table {} was successfully Undropped.", dropped_table.table_id.getNameForLogs()); } catch (...) { - enqueue(); throw Exception( ErrorCodes::FS_METADATA_ERROR, "Cannot undrop table {} from {}", - dropped_table_id.getNameForLogs(), - table_metadata_path); + dropped_table.table_id.getNameForLogs(), + latest_metadata_dropped_path); } } @@ -1079,7 +1065,7 @@ void DatabaseCatalog::dropTableDataTask() bool old_enough = elem.drop_time <= current_time; min_drop_time = std::min(min_drop_time, elem.drop_time); tables_in_use_count += !not_in_use; - return not_in_use && old_enough; + return not_in_use && old_enough && !elem.is_being_undropped; }); if (it != tables_marked_dropped.end()) { diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 51e9fbdb936..516bb15911c 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -244,6 +244,8 @@ public: StoragePtr table; String metadata_path; time_t drop_time{}; + /// If is_being_undropped marks true, the drop task for table will no longer be scheduled. + bool is_being_undropped = false; }; using TablesMarkedAsDropped = std::list; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 7a4d65a4d57..0ba8b0cd40b 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1107,7 +1107,8 @@ 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 - ddl_guard = DatabaseCatalog::instance().getDDLGuard(database_name, create.getTable()); + if (!getContext()->isInDDLGuard()) + ddl_guard = DatabaseCatalog::instance().getDDLGuard(database_name, create.getTable()); bool if_not_exists = create.if_not_exists; @@ -1312,7 +1313,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, return true; } - if (!ddl_guard) + if (!ddl_guard && !getContext()->isInDDLGuard()) ddl_guard = DatabaseCatalog::instance().getDDLGuard(create.getDatabase(), create.getTable()); String data_path; diff --git a/src/Interpreters/InterpreterUndropQuery.cpp b/src/Interpreters/InterpreterUndropQuery.cpp index cc3b1b3ac24..c4c214e75ca 100644 --- a/src/Interpreters/InterpreterUndropQuery.cpp +++ b/src/Interpreters/InterpreterUndropQuery.cpp @@ -46,7 +46,6 @@ BlockIO InterpreterUndropQuery::execute() BlockIO InterpreterUndropQuery::executeToTable(ASTUndropQuery & query) { auto table_id = StorageID(query); - auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name); auto context = getContext(); if (table_id.database_name.empty()) @@ -55,10 +54,12 @@ BlockIO InterpreterUndropQuery::executeToTable(ASTUndropQuery & query) query.setDatabase(table_id.database_name); } + auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name); + auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (database->isTableExist(table_id.table_name, getContext())) throw Exception( - ErrorCodes::TABLE_ALREADY_EXISTS, "Cannot Undrop table, {}.{} already exists", backQuote(table_id.database_name), backQuote(table_id.table_name)); + ErrorCodes::TABLE_ALREADY_EXISTS, "Cannot Undrop table, {} already exists", table_id); database->checkMetadataFilenameAvailability(table_id.table_name); diff --git a/src/Storages/System/StorageSystemDroppedTables.cpp b/src/Storages/System/StorageSystemDroppedTables.cpp index 1d6c8824c76..14caec38f41 100644 --- a/src/Storages/System/StorageSystemDroppedTables.cpp +++ b/src/Storages/System/StorageSystemDroppedTables.cpp @@ -21,6 +21,7 @@ NamesAndTypesList StorageSystemDroppedTables::getNamesAndTypes() {"table", std::make_shared()}, {"uuid", std::make_shared()}, {"engine", std::make_shared()}, + {"is_being_undropped", std::make_shared()}, {"metadata_dropped_path", std::make_shared()}, {"table_dropped_time", std::make_shared()}, }; @@ -39,6 +40,7 @@ void StorageSystemDroppedTables::fillData(MutableColumns & res_columns, ContextP auto & column_table = assert_cast(*res_columns[index++]); auto & column_uuid = assert_cast(*res_columns[index++]).getData(); auto & column_engine = assert_cast(*res_columns[index++]); + auto & column_is_being_undropped = assert_cast(*res_columns[index++]); auto & column_metadata_dropped_path = assert_cast(*res_columns[index++]); auto & column_table_dropped_time = assert_cast(*res_columns[index++]); @@ -52,6 +54,7 @@ void StorageSystemDroppedTables::fillData(MutableColumns & res_columns, ContextP column_engine.insertData(table_mark_dropped.table->getName().data(), table_mark_dropped.table->getName().size()); else column_engine.insertData({}, 0); + column_is_being_undropped.insertValue(static_cast(table_mark_dropped.is_being_undropped)); column_metadata_dropped_path.insertData(table_mark_dropped.metadata_path.data(), table_mark_dropped.metadata_path.size()); column_table_dropped_time.insertValue(static_cast(table_mark_dropped.drop_time)); }; diff --git a/tests/queries/0_stateless/25400_marked_dropped_tables.reference b/tests/queries/0_stateless/25400_marked_dropped_tables.reference index 44906da9527..778a54222b3 100644 --- a/tests/queries/0_stateless/25400_marked_dropped_tables.reference +++ b/tests/queries/0_stateless/25400_marked_dropped_tables.reference @@ -4,5 +4,6 @@ database String table String uuid UUID engine String +is_being_undropped UInt8 metadata_dropped_path String table_dropped_time DateTime From 9382c6ffdcc77bd1105fe0cc7565089bd5bb32e4 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Mon, 27 Mar 2023 23:51:35 +0800 Subject: [PATCH 5/8] better --- src/Interpreters/DatabaseCatalog.cpp | 36 ++++++++----------- src/Interpreters/DatabaseCatalog.h | 2 -- .../System/StorageSystemDroppedTables.cpp | 3 -- .../0_stateless/02681_undrop_query.sql | 14 ++++---- .../0_stateless/02681_undrop_query_uuid.sh | 2 +- .../25400_marked_dropped_tables.reference | 1 - 6 files changed, 23 insertions(+), 35 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 27a8743089c..ed6568a0901 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -979,26 +979,12 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) "The drop task of table {} is in progress, has been dropped or the database engine doesn't support it", table_id.getNameForLogs()); latest_metadata_dropped_path = it_dropped_table->metadata_path; - /// If is_being_undropped of table is true, table drop task will skip this table. - it_dropped_table->is_being_undropped = true; String table_metadata_path = getPathForMetadata(it_dropped_table->table_id); - try - { - /// a table is successfully marked undropped, - /// if and only if its metadata file was moved to a database. - renameNoReplace(latest_metadata_dropped_path, table_metadata_path); - } - catch (...) - { - it_dropped_table->is_being_undropped = false; - throw Exception( - ErrorCodes::FS_METADATA_ERROR, - "Cannot undrop table {}, failed to rename {} to {}", - dropped_table.table_id.getNameForLogs(), - latest_metadata_dropped_path, - table_metadata_path); - } + /// a table is successfully marked undropped, + /// if and only if its metadata file was moved to a database. + /// This maybe throw exception. + renameNoReplace(latest_metadata_dropped_path, table_metadata_path); tables_marked_dropped.erase(it_dropped_table); [[maybe_unused]] auto removed = tables_marked_dropped_ids.erase(dropped_table.table_id.uuid); @@ -1024,12 +1010,20 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) }; wait_dropped_table_not_in_use(); - String query = fmt::format("ATTACH TABLE {}", dropped_table.table_id.getFullTableName()); + auto ast_attach = std::make_shared(); + 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); - executeQuery(query, query_context, true); + InterpreterCreateQuery interpreter(ast_attach, query_context); + interpreter.setForceAttach(true); + interpreter.setForceRestoreData(true); + interpreter.execute(); + LOG_INFO(log, "Table {} was successfully Undropped.", dropped_table.table_id.getNameForLogs()); } catch (...) @@ -1065,7 +1059,7 @@ void DatabaseCatalog::dropTableDataTask() bool old_enough = elem.drop_time <= current_time; min_drop_time = std::min(min_drop_time, elem.drop_time); tables_in_use_count += !not_in_use; - return not_in_use && old_enough && !elem.is_being_undropped; + return not_in_use && old_enough; }); if (it != tables_marked_dropped.end()) { diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 516bb15911c..51e9fbdb936 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -244,8 +244,6 @@ public: StoragePtr table; String metadata_path; time_t drop_time{}; - /// If is_being_undropped marks true, the drop task for table will no longer be scheduled. - bool is_being_undropped = false; }; using TablesMarkedAsDropped = std::list; diff --git a/src/Storages/System/StorageSystemDroppedTables.cpp b/src/Storages/System/StorageSystemDroppedTables.cpp index 14caec38f41..1d6c8824c76 100644 --- a/src/Storages/System/StorageSystemDroppedTables.cpp +++ b/src/Storages/System/StorageSystemDroppedTables.cpp @@ -21,7 +21,6 @@ NamesAndTypesList StorageSystemDroppedTables::getNamesAndTypes() {"table", std::make_shared()}, {"uuid", std::make_shared()}, {"engine", std::make_shared()}, - {"is_being_undropped", std::make_shared()}, {"metadata_dropped_path", std::make_shared()}, {"table_dropped_time", std::make_shared()}, }; @@ -40,7 +39,6 @@ void StorageSystemDroppedTables::fillData(MutableColumns & res_columns, ContextP auto & column_table = assert_cast(*res_columns[index++]); auto & column_uuid = assert_cast(*res_columns[index++]).getData(); auto & column_engine = assert_cast(*res_columns[index++]); - auto & column_is_being_undropped = assert_cast(*res_columns[index++]); auto & column_metadata_dropped_path = assert_cast(*res_columns[index++]); auto & column_table_dropped_time = assert_cast(*res_columns[index++]); @@ -54,7 +52,6 @@ void StorageSystemDroppedTables::fillData(MutableColumns & res_columns, ContextP column_engine.insertData(table_mark_dropped.table->getName().data(), table_mark_dropped.table->getName().size()); else column_engine.insertData({}, 0); - column_is_being_undropped.insertValue(static_cast(table_mark_dropped.is_being_undropped)); column_metadata_dropped_path.insertData(table_mark_dropped.metadata_path.data(), table_mark_dropped.metadata_path.size()); column_table_dropped_time.insertValue(static_cast(table_mark_dropped.drop_time)); }; diff --git a/tests/queries/0_stateless/02681_undrop_query.sql b/tests/queries/0_stateless/02681_undrop_query.sql index 5004eec77f1..ead1a8bb305 100644 --- a/tests/queries/0_stateless/02681_undrop_query.sql +++ b/tests/queries/0_stateless/02681_undrop_query.sql @@ -8,7 +8,7 @@ drop table if exists 02681_undrop_mergetree sync; create table 02681_undrop_mergetree (id Int32) Engine=MergeTree() order by id; insert into 02681_undrop_mergetree values (1),(2),(3); drop table 02681_undrop_mergetree; -select table from system.marked_dropped_tables where table = '02681_undrop_mergetree' limit 1; +select table from system.dropped_tables where table = '02681_undrop_mergetree' limit 1; undrop table 02681_undrop_mergetree; select * from 02681_undrop_mergetree order by id; drop table 02681_undrop_mergetree sync; @@ -29,7 +29,7 @@ drop table if exists 02681_undrop_uuid_on_cluster on cluster test_shard_localhos create table 02681_undrop_uuid_on_cluster on cluster test_shard_localhost (id Int32) Engine=MergeTree() order by id format Null; insert into 02681_undrop_uuid_on_cluster values (1),(2),(3); drop table 02681_undrop_uuid_on_cluster on cluster test_shard_localhost format Null; -select table from system.marked_dropped_tables where table = '02681_undrop_uuid_on_cluster' limit 1; +select table from system.dropped_tables where table = '02681_undrop_uuid_on_cluster' limit 1; undrop table 02681_undrop_uuid_on_cluster on cluster test_shard_localhost format Null; select * from 02681_undrop_uuid_on_cluster order by id; drop table 02681_undrop_uuid_on_cluster sync; @@ -39,7 +39,7 @@ drop table if exists 02681_undrop_no_uuid_on_cluster on cluster test_shard_local create table 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost (id Int32) Engine=MergeTree() order by id format Null; insert into 02681_undrop_no_uuid_on_cluster values (1),(2),(3); drop table 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost format Null; -select table from system.marked_dropped_tables where table = '02681_undrop_no_uuid_on_cluster' limit 1; +select table from system.dropped_tables where table = '02681_undrop_no_uuid_on_cluster' limit 1; undrop table 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost format Null; select * from 02681_undrop_no_uuid_on_cluster order by id; drop table 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost sync format Null; @@ -49,7 +49,7 @@ drop table if exists 02681_undrop_replicatedmergetree sync; create table 02681_undrop_replicatedmergetree (id Int32) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/02681_undrop_replicatedmergetree', 'test_undrop') order by id; insert into 02681_undrop_replicatedmergetree values (1),(2),(3); drop table 02681_undrop_replicatedmergetree; -select table from system.marked_dropped_tables where table = '02681_undrop_replicatedmergetree' limit 1; +select table from system.dropped_tables where table = '02681_undrop_replicatedmergetree' limit 1; undrop table 02681_undrop_replicatedmergetree; select * from 02681_undrop_replicatedmergetree order by id; drop table 02681_undrop_replicatedmergetree sync; @@ -59,7 +59,7 @@ drop table if exists 02681_undrop_log sync; create table 02681_undrop_log (id Int32) Engine=Log(); insert into 02681_undrop_log values (1),(2),(3); drop table 02681_undrop_log; -select table from system.marked_dropped_tables where table = '02681_undrop_log' limit 1; +select table from system.dropped_tables where table = '02681_undrop_log' limit 1; undrop table 02681_undrop_log; select * from 02681_undrop_log order by id; drop table 02681_undrop_log sync; @@ -68,7 +68,7 @@ select 'test Distributed undrop'; drop table if exists 02681_undrop_distributed sync; create table 02681_undrop_distributed (id Int32) Engine = Distributed(test_shard_localhost, currentDatabase(), 02681_undrop, rand()); drop table 02681_undrop_distributed; -select table from system.marked_dropped_tables where table = '02681_undrop_distributed' limit 1; +select table from system.dropped_tables where table = '02681_undrop_distributed' limit 1; undrop table 02681_undrop_distributed; drop table 02681_undrop_distributed sync; @@ -83,7 +83,7 @@ drop table 02681_undrop_multiple; create table 02681_undrop_multiple (id Int32) Engine=MergeTree() order by id; insert into 02681_undrop_multiple values (3); drop table 02681_undrop_multiple; -select table from system.marked_dropped_tables where table = '02681_undrop_multiple' limit 1; +select table from system.dropped_tables where table = '02681_undrop_multiple' limit 1; undrop table 02681_undrop_multiple; select * from 02681_undrop_multiple order by id; undrop table 02681_undrop_multiple; -- { serverError 57 } diff --git a/tests/queries/0_stateless/02681_undrop_query_uuid.sh b/tests/queries/0_stateless/02681_undrop_query_uuid.sh index 42f2b2c2f6c..c7d87bad3d4 100755 --- a/tests/queries/0_stateless/02681_undrop_query_uuid.sh +++ b/tests/queries/0_stateless/02681_undrop_query_uuid.sh @@ -11,7 +11,7 @@ uuid=$(${CLICKHOUSE_CLIENT} --query "SELECT reinterpretAsUUID(currentDatabase()) ${CLICKHOUSE_CLIENT} -q "create table 02681_undrop_uuid UUID '$uuid' (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 "select table from system.marked_dropped_tables where table = '02681_undrop_uuid' limit 1;" +${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 "select * from 02681_undrop_uuid order by id;" ${CLICKHOUSE_CLIENT} -q "drop table 02681_undrop_uuid sync;" diff --git a/tests/queries/0_stateless/25400_marked_dropped_tables.reference b/tests/queries/0_stateless/25400_marked_dropped_tables.reference index 778a54222b3..44906da9527 100644 --- a/tests/queries/0_stateless/25400_marked_dropped_tables.reference +++ b/tests/queries/0_stateless/25400_marked_dropped_tables.reference @@ -4,6 +4,5 @@ database String table String uuid UUID engine String -is_being_undropped UInt8 metadata_dropped_path String table_dropped_time DateTime From 62087c3f09641b41360ac5e4a5d96599baf6a5ed Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Tue, 28 Mar 2023 07:27:13 +0800 Subject: [PATCH 6/8] fix --- src/Interpreters/DatabaseCatalog.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index ed6568a0901..436bf244e4e 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include From 53725bdea135d10e2538a0b5b00fd0be62f6ccf7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 Mar 2023 23:47:45 +0200 Subject: [PATCH 7/8] monor improvements --- src/Databases/DatabaseAtomic.cpp | 10 ---- src/Databases/DatabaseAtomic.h | 2 - src/Databases/IDatabase.h | 5 -- src/Interpreters/Context.h | 4 -- src/Interpreters/DatabaseCatalog.cpp | 58 +++++++------------ src/Interpreters/InterpreterCreateQuery.cpp | 4 +- src/Interpreters/InterpreterCreateQuery.h | 6 ++ src/Interpreters/InterpreterUndropQuery.cpp | 4 +- .../02681_undrop_query_uuid.reference | 1 + .../0_stateless/02681_undrop_query_uuid.sh | 10 ++-- 10 files changed, 38 insertions(+), 66 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index e5320dc6ff4..7e20b6f6535 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -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); diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index b8ff719989b..cb275812098 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -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; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 4dbf78cb5c4..b8880c4c4cc 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -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*/) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b9ef5e88a2a..bbfbd4defdc 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -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 diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 436bf244e4e..f57bb92d310 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -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(); - 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(); + 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() diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 66ef2dd0143..77179ad2f32 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -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; diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 4d11387f44c..a5fa6576091 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -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; diff --git a/src/Interpreters/InterpreterUndropQuery.cpp b/src/Interpreters/InterpreterUndropQuery.cpp index c4c214e75ca..72b8e7eba4f 100644 --- a/src/Interpreters/InterpreterUndropQuery.cpp +++ b/src/Interpreters/InterpreterUndropQuery.cpp @@ -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); diff --git a/tests/queries/0_stateless/02681_undrop_query_uuid.reference b/tests/queries/0_stateless/02681_undrop_query_uuid.reference index 9d36a21dbda..beae016401b 100644 --- a/tests/queries/0_stateless/02681_undrop_query_uuid.reference +++ b/tests/queries/0_stateless/02681_undrop_query_uuid.reference @@ -1,5 +1,6 @@ test MergeTree with uuid 02681_undrop_uuid +OK 1 2 3 diff --git a/tests/queries/0_stateless/02681_undrop_query_uuid.sh b/tests/queries/0_stateless/02681_undrop_query_uuid.sh index c7d87bad3d4..05e0f9c0cd0 100755 --- a/tests/queries/0_stateless/02681_undrop_query_uuid.sh +++ b/tests/queries/0_stateless/02681_undrop_query_uuid.sh @@ -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;" From cfe0fb65718f1a2fc097029ba8f1304e5c6193da Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 29 Mar 2023 03:31:39 +0200 Subject: [PATCH 8/8] fix --- src/Interpreters/DatabaseCatalog.cpp | 1 - tests/queries/0_stateless/02681_undrop_query_uuid.sh | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index f57bb92d310..ac9b8615fe5 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -53,7 +53,6 @@ namespace ErrorCodes extern const int DATABASE_ACCESS_DENIED; extern const int LOGICAL_ERROR; extern const int HAVE_DEPENDENT_OBJECTS; - extern const int FS_METADATA_ERROR; } TemporaryTableHolder::TemporaryTableHolder(ContextPtr context_, const TemporaryTableHolder::Creator & creator, const ASTPtr & query) diff --git a/tests/queries/0_stateless/02681_undrop_query_uuid.sh b/tests/queries/0_stateless/02681_undrop_query_uuid.sh index 05e0f9c0cd0..a93f30ef459 100755 --- a/tests/queries/0_stateless/02681_undrop_query_uuid.sh +++ b/tests/queries/0_stateless/02681_undrop_query_uuid.sh @@ -9,11 +9,11 @@ echo 'test MergeTree with uuid' ${CLICKHOUSE_CLIENT} -q "drop table if exists 02681_undrop_uuid sync;" 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} --distributed_ddl_output_mode=none -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 on cluster test_shard_localhost settings database_atomic_wait_for_drop_and_detach_synchronously = 0;" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -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 '$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} --distributed_ddl_output_mode=none -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;"