diff --git a/src/Access/AccessBackup.cpp b/src/Access/AccessBackup.cpp index d2f95df8f99..0322ad7457b 100644 --- a/src/Access/AccessBackup.cpp +++ b/src/Access/AccessBackup.cpp @@ -25,7 +25,7 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } @@ -129,7 +129,7 @@ namespace String name; readStringInto(name, *buf); buf->ignore(); - + if (!res.entities.contains(id)) res.dependencies.emplace(id, std::pair{name, type}); } diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index d2bb1ad1bac..de4c3fbce11 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -529,7 +529,7 @@ std::vector> IAccessStorage::readAllForBackup(A { if (!isBackupAllowed()) throwBackupNotAllowed(); - + auto res = readAllWithIDs(type); boost::range::remove_erase_if(res, [](const std::pair & x) { return !x.second->isBackupAllowed(); }); return res; diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index 1caa5790145..7b43309204d 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -111,7 +111,7 @@ public: /// Reads all entities and returns them with their IDs. template std::vector>> readAllWithIDs() const; - + std::vector> readAllWithIDs(AccessEntityType type) const; /// Inserts an entity to the storage. Returns ID of a new entry in the storage. @@ -251,7 +251,7 @@ std::vector>> IAccessStorage if (auto entity = tryRead(id)) entities.emplace_back(id, entity); } - return entities; + return entities; } } diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index 9d70c1e2a43..b9558fb0340 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -388,7 +388,7 @@ std::vector> MultipleAccessStorage::readAllForB std::vector> res; auto storages = getStoragesInternal(); size_t count = 0; - + for (const auto & storage : *storages) { if (storage->isBackupAllowed()) diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index 36ffb2da902..6a9d716c2f9 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -623,7 +623,7 @@ void ReplicatedAccessStorage::insertFromBackup(const std::vector clone() const override { return cloneImpl(); } static constexpr const auto TYPE = AccessEntityType::ROLE; AccessEntityType getType() const override { return TYPE; } - + std::vector findDependencies() const override; void replaceDependencies(const std::unordered_map & old_to_new_ids) override; bool isBackupAllowed() const override { return settings.isBackupAllowed(); } diff --git a/src/Backups/BackupCoordinationDistributed.cpp b/src/Backups/BackupCoordinationDistributed.cpp index 5ea5a564e52..945239482fc 100644 --- a/src/Backups/BackupCoordinationDistributed.cpp +++ b/src/Backups/BackupCoordinationDistributed.cpp @@ -14,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int UNEXPECTED_NODE_IN_ZOOKEEPER; + extern const int LOGICAL_ERROR; } /// zookeeper_path/file_names/file_name->checksum_and_size diff --git a/src/Backups/BackupCoordinationDistributed.h b/src/Backups/BackupCoordinationDistributed.h index 78d8eb7452f..2872e1f3ae4 100644 --- a/src/Backups/BackupCoordinationDistributed.h +++ b/src/Backups/BackupCoordinationDistributed.h @@ -32,7 +32,7 @@ public: void updateFileInfo(const FileInfo & file_info) override; std::vector getAllFileInfos() const override; - Strings listFiles(const String & diretory, bool recursive) const override; + Strings listFiles(const String & directory, bool recursive) const override; bool hasFiles(const String & directory) const override; std::optional getFileInfo(const String & file_name) const override; std::optional getFileInfo(const SizeAndChecksum & size_and_checksum) const override; diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index 725a054c029..55a3c671a6e 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -96,7 +96,7 @@ Strings BackupCoordinationLocal::listFiles(const String & directory, bool recurs if (!prefix.empty() && !prefix.ends_with('/')) prefix += '/'; String terminator = recursive ? "" : "/"; - + Strings elements; for (auto it = file_names.lower_bound(prefix); it != file_names.end(); ++it) { @@ -112,7 +112,7 @@ Strings BackupCoordinationLocal::listFiles(const String & directory, bool recurs continue; elements.push_back(String{new_element}); } - + return elements; } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 862dea3c6e3..ddca3c45dd8 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -20,7 +20,7 @@ public: void syncStage(const String & current_host, int stage, const Strings & wait_hosts, std::chrono::seconds timeout) override; void syncStageError(const String & current_host, const String & error_message) override; - + void addReplicatedPartNames(const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) override; Strings getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const override; diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 042d2eb636c..322bc00ee3c 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -23,8 +23,8 @@ namespace ErrorCodes { extern const int CANNOT_COLLECT_OBJECTS_FOR_BACKUP; extern const int CANNOT_BACKUP_TABLE; - extern const int BACKUP_IS_EMPTY; extern const int TABLE_IS_DROPPED; + extern const int LOGICAL_ERROR; } @@ -49,7 +49,7 @@ std::string_view BackupEntriesCollector::toString(Stage stage) case Stage::kWritingBackup: return "Writing backup"; case Stage::kError: return "Error"; } - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown backup stage: {}", static_cast(stage)); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown backup stage: {}", static_cast(stage)); } @@ -126,9 +126,9 @@ void BackupEntriesCollector::setStage(Stage new_stage, const String & error_mess LOG_ERROR(log, "{} failed with error: {}", toString(current_stage), error_message); else LOG_TRACE(log, "{}", toString(new_stage)); - + current_stage = new_stage; - + if (new_stage == Stage::kError) { backup_coordination->syncStageError(backup_settings.host_id, error_message); @@ -250,7 +250,7 @@ void BackupEntriesCollector::collectTableInfo( : context->tryResolveStorageID(StorageID{table_name.database, table_name.table}, Context::ResolveGlobal); if (!resolved_id.empty()) std::tie(database, storage) = DatabaseCatalog::instance().tryGetDatabaseAndTable(resolved_id, context); - + if (storage) { try diff --git a/src/Backups/BackupEntriesCollector.h b/src/Backups/BackupEntriesCollector.h index da44182ce9a..1466815f3a7 100644 --- a/src/Backups/BackupEntriesCollector.h +++ b/src/Backups/BackupEntriesCollector.h @@ -130,7 +130,7 @@ private: std::optional> previous_database_names; std::optional> previous_table_names; bool consistent = false; - + BackupEntries backup_entries; std::queue> post_collecting_tasks; }; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 84367910a47..09c1cd5e9b5 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -37,7 +37,6 @@ namespace ErrorCodes extern const int BACKUP_ENTRY_ALREADY_EXISTS; extern const int BACKUP_ENTRY_NOT_FOUND; extern const int BACKUP_IS_EMPTY; - extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; } diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 85d45b10b2d..08a5836ef31 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -200,20 +200,20 @@ AccessRightsElements getRequiredAccessToBackup(const ASTBackupQuery::Elements & required_access.emplace_back(AccessType::BACKUP, element.database_name, element.table_name); break; } - + case ASTBackupQuery::TEMPORARY_TABLE: { /// It's always allowed to backup temporary tables. break; } - + case ASTBackupQuery::DATABASE: { /// TODO: It's better to process `element.except_tables` somehow. required_access.emplace_back(AccessType::BACKUP, element.database_name); break; } - + case ASTBackupQuery::ALL: { /// TODO: It's better to process `element.except_databases` & `element.except_tables` somehow. diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index cede51ce84a..c19d730cf7e 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -156,7 +156,7 @@ UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & c backup_entries = backup_entries_collector.getBackupEntries(); } - writeBackupEntries(backup, std::move(backup_entries), backups_thread_pool); + writeBackupEntries(backup, std::move(backup_entries), backups_thread_pool); } /// Finalize backup (write its metadata). @@ -251,13 +251,13 @@ UUID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr conte if (restore_coordination && !restore_settings.internal) restore_coordination->drop(); }); - + if (on_cluster && restore_settings.coordination_zk_path.empty()) { String root_zk_path = context_in_use->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); restore_settings.coordination_zk_path = root_zk_path + "/restore-" + toString(restore_uuid); } - + if (!restore_settings.coordination_zk_path.empty()) { restore_coordination = std::make_shared( @@ -314,7 +314,7 @@ UUID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr conte setStatus(restore_uuid, BackupStatus::FAILED_TO_RESTORE); if (!in_separate_thread) throw; - } + } }; if (restore_settings.async) diff --git a/src/Backups/RestoreCoordinationDistributed.cpp b/src/Backups/RestoreCoordinationDistributed.cpp index 2faa7786210..e131ce7fe24 100644 --- a/src/Backups/RestoreCoordinationDistributed.cpp +++ b/src/Backups/RestoreCoordinationDistributed.cpp @@ -1,24 +1,11 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include namespace DB { -namespace ErrorCodes -{ - extern const int FAILED_TO_SYNC_BACKUP_OR_RESTORE; -} - - RestoreCoordinationDistributed::RestoreCoordinationDistributed(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_) : zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index 93e5b0121f3..9cecc3f90c9 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -1,18 +1,9 @@ #include -#include -#include -#include namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - RestoreCoordinationLocal::RestoreCoordinationLocal() = default; RestoreCoordinationLocal::~RestoreCoordinationLocal() = default; diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index ed920482d3e..590d39f24f8 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -48,7 +48,7 @@ namespace return; } } - + if (field.getType() == Field::Types::UInt64) { UInt64 number = field.get(); @@ -122,7 +122,7 @@ namespace return; } } - + throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Cannot parse creation mode from {}", field); } diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index d94f860b189..5e941b79508 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -28,7 +28,7 @@ enum class RestoreAccessCreationMode { /// RESTORE will throw an exception if some user already exists. kCreate, - + /// RESTORE will skip existing users. kCreateIfNotExists, diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 639d256f7e0..49cb0ec77f9 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -33,6 +33,7 @@ namespace ErrorCodes extern const int BACKUP_ENTRY_NOT_FOUND; extern const int CANNOT_RESTORE_TABLE; extern const int CANNOT_RESTORE_DATABASE; + extern const int LOGICAL_ERROR; } @@ -84,7 +85,7 @@ std::string_view RestorerFromBackup::toString(Stage stage) case Stage::kInsertingDataToTables: return "Inserting data to tables"; case Stage::kError: return "Error"; } - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown backup stage: {}", static_cast(stage)); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown restore stage: {}", static_cast(stage)); } @@ -134,7 +135,7 @@ void RestorerFromBackup::run(bool only_check_access) /// Find all the databases and tables which we will read from the backup. setStage(Stage::kFindingTablesInBackup); collectDatabaseAndTableInfos(); - + /// Check access rights. checkAccessForCollectedInfos(); if (only_check_access) @@ -204,12 +205,12 @@ void RestorerFromBackup::setStage(Stage new_stage, const String & error_message) LOG_ERROR(log, "{} failed with error: {}", toString(current_stage), error_message); else LOG_TRACE(log, "{}", toString(new_stage)); - + current_stage = new_stage; if (!restore_coordination) return; - + if (new_stage == Stage::kError) { restore_coordination->syncStageError(restore_settings.host_id, error_message); @@ -231,7 +232,7 @@ void RestorerFromBackup::findRootPathsInBackup() std::tie(shard_num, replica_num) = BackupSettings::Util::findShardNumAndReplicaNum(restore_settings.cluster_host_ids, restore_settings.host_id); } - + root_paths_in_backup.clear(); /// Start with "" as the root path and then we will add shard- and replica-related part to it. @@ -459,7 +460,7 @@ void RestorerFromBackup::collectDatabaseInfo(const String & database_name_in_bac String database_name = renaming_map.getNewDatabaseName(database_name_in_backup); DatabaseInfo & database_info = database_infos[database_name]; - + if (database_info.create_database_query && (serializeAST(*database_info.create_database_query) != serializeAST(*create_database_query))) { throw Exception( diff --git a/src/Backups/RestorerFromBackup.h b/src/Backups/RestorerFromBackup.h index 15c76a05402..ae036aab674 100644 --- a/src/Backups/RestorerFromBackup.h +++ b/src/Backups/RestorerFromBackup.h @@ -80,7 +80,7 @@ public: kError = -1, }; static std::string_view toString(Stage stage); - + /// Throws an exception that a specified table engine doesn't support partitions. [[noreturn]] static void throwPartitionsNotSupported(const StorageID & storage_id, const String & table_engine); diff --git a/src/Databases/DDLRenamingVisitor.cpp b/src/Databases/DDLRenamingVisitor.cpp index 409bddd05b6..4dbd3dd769c 100644 --- a/src/Databases/DDLRenamingVisitor.cpp +++ b/src/Databases/DDLRenamingVisitor.cpp @@ -124,13 +124,13 @@ namespace auto info = getInfoIfClickHouseDictionarySource(config, data.global_context); if (!info || !info->is_local) return; - + auto * source_list = dictionary.source->elements->as(); if (!source_list) return; auto & source_elements = source_list->children; - + Field * database_name_field = nullptr; Field * table_name_field = nullptr; @@ -138,7 +138,7 @@ namespace { if (!source_element) continue; - + auto * pair = source_element->as(); if (!pair || !pair->second) continue; diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 05c55ac9e43..5c701c8d90c 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -944,7 +944,7 @@ void DatabaseReplicated::createTableRestoredFromBackup(const ASTPtr & create_tab while (!isTableExist(table_name, restorer.getContext())) { waitForReplicaToProcessAllEntries(50); - + if (use_timeout) { auto elapsed = std::chrono::steady_clock::now() - start_time; diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index a37ea516ed3..fcaa4af88bb 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -41,7 +41,7 @@ public: void createTableRestoredFromBackup(const ASTPtr & create_table_query, const RestorerFromBackup & restorer) override; void shutdown() override; - + ~DatabaseWithOwnTablesBase() override; protected: diff --git a/src/Databases/IDatabase.cpp b/src/Databases/IDatabase.cpp index 1b1d65310bd..1d5695188b7 100644 --- a/src/Databases/IDatabase.cpp +++ b/src/Databases/IDatabase.cpp @@ -25,7 +25,7 @@ StoragePtr IDatabase::getTable(const String & name, ContextPtr context) const ASTPtr IDatabase::getCreateDatabaseQueryForBackup() const { auto query = getCreateDatabaseQuery(); - + /// We don't want to see any UUIDs in backup (after RESTORE the table will have another UUID anyway). auto & create = query->as(); create.uuid = UUIDHelpers::Nil; diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 84a4f977ca2..17ab85148c3 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -32,8 +32,6 @@ namespace ErrorCodes extern const int TIMEOUT_EXCEEDED; extern const int UNFINISHED; extern const int QUERY_IS_PROHIBITED; - extern const int INVALID_SHARD_ID; - extern const int NO_SUCH_REPLICA; extern const int LOGICAL_ERROR; } diff --git a/src/Parsers/ASTBackupQuery.cpp b/src/Parsers/ASTBackupQuery.cpp index 5256ad9ddfe..6f6c871fceb 100644 --- a/src/Parsers/ASTBackupQuery.cpp +++ b/src/Parsers/ASTBackupQuery.cpp @@ -94,7 +94,7 @@ namespace { format.ostr << (format.hilite ? IAST::hilite_keyword : "") << "TEMPORARY TABLE " << (format.hilite ? IAST::hilite_none : ""); format.ostr << backQuoteIfNeed(element.table_name); - + if (element.new_table_name != element.table_name) { format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " AS " << (format.hilite ? IAST::hilite_none : ""); diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 630de4864fc..cd6c49d6e3b 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -24,7 +24,6 @@ namespace ErrorCodes extern const int TABLE_IS_DROPPED; extern const int NOT_IMPLEMENTED; extern const int DEADLOCK_AVOIDED; - extern const int CANNOT_BACKUP_TABLE; } bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ff17a452d59..fb761ed1ae5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4108,7 +4108,7 @@ BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const relative_filepath, std::make_unique(disk, part_dir / filepath)); } } - + return backup_entries; } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index ce3e9a6fb24..1324ebf5b28 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -51,7 +51,6 @@ namespace ErrorCodes extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_FILE_NAME; - extern const int NOT_IMPLEMENTED; } /// NOTE: The lock `StorageLog::rwlock` is NOT kept locked while reading, diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 847288a1e7a..5de8c3bda43 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -38,7 +38,6 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int NOT_IMPLEMENTED; } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index a733d3289cc..d569a81c4a7 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -55,7 +55,6 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_FILE_NAME; extern const int TIMEOUT_EXCEEDED; - extern const int NOT_IMPLEMENTED; } diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 70683fefd33..6264959fbce 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -482,23 +482,35 @@ def test_system_users(): node1.query("CREATE USER u2 SETTINGS allow_backup=false") expected_error = "necessary to have grant BACKUP ON system.users" - assert expected_error in node1.query_and_get_error(f"BACKUP TABLE system.users ON CLUSTER 'cluster' TO {backup_name}", user='u2') + assert expected_error in node1.query_and_get_error( + f"BACKUP TABLE system.users ON CLUSTER 'cluster' TO {backup_name}", user="u2" + ) node1.query("GRANT BACKUP ON system.users TO u2") - node1.query(f"BACKUP TABLE system.users ON CLUSTER 'cluster' TO {backup_name}", user='u2') + node1.query( + f"BACKUP TABLE system.users ON CLUSTER 'cluster' TO {backup_name}", user="u2" + ) node1.query("DROP USER u1") expected_error = "necessary to have grant CREATE USER ON *.*" - assert expected_error in node1.query_and_get_error(f"RESTORE TABLE system.users ON CLUSTER 'cluster' FROM {backup_name}", user='u2') + assert expected_error in node1.query_and_get_error( + f"RESTORE TABLE system.users ON CLUSTER 'cluster' FROM {backup_name}", user="u2" + ) node1.query("GRANT CREATE USER ON *.* TO u2") expected_error = "necessary to have grant SELECT ON default.tbl WITH GRANT OPTION" - assert expected_error in node1.query_and_get_error(f"RESTORE TABLE system.users ON CLUSTER 'cluster' FROM {backup_name}", user='u2') + assert expected_error in node1.query_and_get_error( + f"RESTORE TABLE system.users ON CLUSTER 'cluster' FROM {backup_name}", user="u2" + ) node1.query("GRANT SELECT ON tbl TO u2 WITH GRANT OPTION") - node1.query(f"RESTORE TABLE system.users ON CLUSTER 'cluster' FROM {backup_name}", user='u2') + node1.query( + f"RESTORE TABLE system.users ON CLUSTER 'cluster' FROM {backup_name}", user="u2" + ) - assert node1.query("SHOW CREATE USER u1") == "CREATE USER u1 SETTINGS custom_a = 123\n" + assert ( + node1.query("SHOW CREATE USER u1") == "CREATE USER u1 SETTINGS custom_a = 123\n" + ) assert node1.query("SHOW GRANTS FOR u1") == "GRANT SELECT ON default.tbl TO u1\n"