Fix style.

This commit is contained in:
Vitaly Baranov 2022-06-19 15:48:52 +02:00
parent b0d19328b8
commit 8a7c970ce0
33 changed files with 63 additions and 78 deletions

View File

@ -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});
}

View File

@ -529,7 +529,7 @@ std::vector<std::pair<UUID, AccessEntityPtr>> IAccessStorage::readAllForBackup(A
{
if (!isBackupAllowed())
throwBackupNotAllowed();
auto res = readAllWithIDs(type);
boost::range::remove_erase_if(res, [](const std::pair<UUID, AccessEntityPtr> & x) { return !x.second->isBackupAllowed(); });
return res;

View File

@ -111,7 +111,7 @@ public:
/// Reads all entities and returns them with their IDs.
template <typename EntityClassT>
std::vector<std::pair<UUID, std::shared_ptr<const EntityClassT>>> readAllWithIDs() const;
std::vector<std::pair<UUID, AccessEntityPtr>> 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<std::pair<UUID, std::shared_ptr<const EntityClassT>>> IAccessStorage
if (auto entity = tryRead<EntityClassT>(id))
entities.emplace_back(id, entity);
}
return entities;
return entities;
}
}

View File

@ -388,7 +388,7 @@ std::vector<std::pair<UUID, AccessEntityPtr>> MultipleAccessStorage::readAllForB
std::vector<std::pair<UUID, AccessEntityPtr>> res;
auto storages = getStoragesInternal();
size_t count = 0;
for (const auto & storage : *storages)
{
if (storage->isBackupAllowed())

View File

@ -623,7 +623,7 @@ void ReplicatedAccessStorage::insertFromBackup(const std::vector<std::pair<UUID,
bool replace_if_exists = (restore_settings.create_access == RestoreAccessCreationMode::kReplace);
bool throw_if_exists = (restore_settings.create_access == RestoreAccessCreationMode::kCreate);
for (const auto & [id, entity] : entities_from_backup)
insertWithID(id, entity, replace_if_exists, throw_if_exists);
}

View File

@ -19,7 +19,7 @@ struct Role : public IAccessEntity
std::shared_ptr<IAccessEntity> clone() const override { return cloneImpl<Role>(); }
static constexpr const auto TYPE = AccessEntityType::ROLE;
AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const override;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
bool isBackupAllowed() const override { return settings.isBackupAllowed(); }

View File

@ -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

View File

@ -32,7 +32,7 @@ public:
void updateFileInfo(const FileInfo & file_info) override;
std::vector<FileInfo> 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<FileInfo> getFileInfo(const String & file_name) const override;
std::optional<FileInfo> getFileInfo(const SizeAndChecksum & size_and_checksum) const override;

View File

@ -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;
}

View File

@ -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<PartNameAndChecksum> & part_names_and_checksums) override;
Strings getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const override;

View File

@ -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<int>(stage));
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown backup stage: {}", static_cast<int>(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

View File

@ -130,7 +130,7 @@ private:
std::optional<std::set<String>> previous_database_names;
std::optional<std::set<TableKey>> previous_table_names;
bool consistent = false;
BackupEntries backup_entries;
std::queue<std::function<void()>> post_collecting_tasks;
};

View File

@ -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;
}

View File

@ -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.

View File

@ -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<RestoreCoordinationDistributed>(
@ -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)

View File

@ -1,24 +1,11 @@
#include <Backups/RestoreCoordinationDistributed.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/escapeForFileName.h>
#include <Common/logger_useful.h>
#include <Interpreters/StorageID.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/ReadBufferFromString.h>
#include <base/chrono_io.h>
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_)

View File

@ -1,18 +1,9 @@
#include <Backups/RestoreCoordinationLocal.h>
#include <Common/Exception.h>
#include <Common/logger_useful.h>
#include <Interpreters/StorageID.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
RestoreCoordinationLocal::RestoreCoordinationLocal() = default;
RestoreCoordinationLocal::~RestoreCoordinationLocal() = default;

View File

@ -48,7 +48,7 @@ namespace
return;
}
}
if (field.getType() == Field::Types::UInt64)
{
UInt64 number = field.get<UInt64>();
@ -122,7 +122,7 @@ namespace
return;
}
}
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Cannot parse creation mode from {}", field);
}

View File

@ -28,7 +28,7 @@ enum class RestoreAccessCreationMode
{
/// RESTORE will throw an exception if some user already exists.
kCreate,
/// RESTORE will skip existing users.
kCreateIfNotExists,

View File

@ -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<int>(stage));
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown restore stage: {}", static_cast<int>(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(

View File

@ -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);

View File

@ -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<ASTExpressionList>();
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<ASTPair>();
if (!pair || !pair->second)
continue;

View File

@ -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;

View File

@ -41,7 +41,7 @@ public:
void createTableRestoredFromBackup(const ASTPtr & create_table_query, const RestorerFromBackup & restorer) override;
void shutdown() override;
~DatabaseWithOwnTablesBase() override;
protected:

View File

@ -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<ASTCreateQuery &>();
create.uuid = UUIDHelpers::Nil;

View File

@ -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;
}

View File

@ -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 : "");

View File

@ -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

View File

@ -4108,7 +4108,7 @@ BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const
relative_filepath, std::make_unique<BackupEntryFromSmallFile>(disk, part_dir / filepath));
}
}
return backup_entries;
}

View File

@ -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,

View File

@ -38,7 +38,6 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int NOT_IMPLEMENTED;
}

View File

@ -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;
}

View File

@ -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"