diff --git a/src/Backups/BackupCoordinationDistributed.cpp b/src/Backups/BackupCoordinationDistributed.cpp index 4391ae9dc02..5ea5a564e52 100644 --- a/src/Backups/BackupCoordinationDistributed.cpp +++ b/src/Backups/BackupCoordinationDistributed.cpp @@ -301,12 +301,19 @@ std::vector BackupCoordinationDistributed::getAllFileInfos() const return file_infos; } -Strings BackupCoordinationDistributed::listFiles(const String & prefix, const String & terminator) const +Strings BackupCoordinationDistributed::listFiles(const String & directory, bool recursive) const { auto zookeeper = get_zookeeper(); Strings escaped_names = zookeeper->getChildren(zookeeper_path + "/file_names"); + String prefix = directory; + if (!prefix.empty() && !prefix.ends_with('/')) + prefix += '/'; + String terminator = recursive ? "" : "/"; + Strings elements; + std::unordered_set unique_elements; + for (const String & escaped_name : escaped_names) { String name = unescapeForFileName(escaped_name); @@ -317,15 +324,35 @@ Strings BackupCoordinationDistributed::listFiles(const String & prefix, const St if (!terminator.empty()) end_pos = name.find(terminator, start_pos); std::string_view new_element = std::string_view{name}.substr(start_pos, end_pos - start_pos); - if (!elements.empty() && (elements.back() == new_element)) + if (unique_elements.contains(new_element)) continue; elements.push_back(String{new_element}); + unique_elements.emplace(new_element); } ::sort(elements.begin(), elements.end()); return elements; } +bool BackupCoordinationDistributed::hasFiles(const String & directory) const +{ + auto zookeeper = get_zookeeper(); + Strings escaped_names = zookeeper->getChildren(zookeeper_path + "/file_names"); + + String prefix = directory; + if (!prefix.empty() && !prefix.ends_with('/')) + prefix += '/'; + + for (const String & escaped_name : escaped_names) + { + String name = unescapeForFileName(escaped_name); + if (name.starts_with(prefix)) + return true; + } + + return false; +} + std::optional BackupCoordinationDistributed::getFileInfo(const String & file_name) const { auto zookeeper = get_zookeeper(); diff --git a/src/Backups/BackupCoordinationDistributed.h b/src/Backups/BackupCoordinationDistributed.h index 0963ee5d015..78d8eb7452f 100644 --- a/src/Backups/BackupCoordinationDistributed.h +++ b/src/Backups/BackupCoordinationDistributed.h @@ -32,7 +32,8 @@ public: void updateFileInfo(const FileInfo & file_info) override; std::vector getAllFileInfos() const override; - Strings listFiles(const String & prefix, const String & terminator) const override; + Strings listFiles(const String & diretory, 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; std::optional getFileSizeAndChecksum(const String & file_name) const override; diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index 9a8f99e46bc..725a054c029 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -89,9 +89,14 @@ std::vector BackupCoordinationLocal::getAllFileInfos() const return res; } -Strings BackupCoordinationLocal::listFiles(const String & prefix, const String & terminator) const +Strings BackupCoordinationLocal::listFiles(const String & directory, bool recursive) const { std::lock_guard lock{mutex}; + String prefix = directory; + 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) { @@ -107,9 +112,25 @@ Strings BackupCoordinationLocal::listFiles(const String & prefix, const String & continue; elements.push_back(String{new_element}); } + return elements; } +bool BackupCoordinationLocal::hasFiles(const String & directory) const +{ + std::lock_guard lock{mutex}; + String prefix = directory; + if (!prefix.empty() && !prefix.ends_with('/')) + prefix += '/'; + + auto it = file_names.lower_bound(prefix); + if (it == file_names.end()) + return false; + + const String & name = it->first; + return name.starts_with(prefix); +} + std::optional BackupCoordinationLocal::getFileInfo(const String & file_name) const { std::lock_guard lock{mutex}; diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 8e6de3deea0..862dea3c6e3 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -32,7 +32,8 @@ public: void updateFileInfo(const FileInfo & file_info) override; std::vector getAllFileInfos() const override; - Strings listFiles(const String & prefix, const String & terminator) 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/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 35b85298e8c..9104a42fc5f 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -10,6 +10,9 @@ #include #include #include +#include + +namespace fs = std::filesystem; namespace DB @@ -130,18 +133,14 @@ std::string_view BackupEntriesCollector::toString(Stage stage) /// it's either empty or has the format "shards//replicas//". void BackupEntriesCollector::calculateRootPathInBackup() { - if (backup_settings.host_id.empty()) - { - root_path_in_backup = ""; - } - else + root_path_in_backup = "/"; + if (!backup_settings.host_id.empty()) { auto [shard_num, replica_num] = BackupSettings::Util::findShardNumAndReplicaNum(backup_settings.cluster_host_ids, backup_settings.host_id); - root_path_in_backup = fmt::format("shards/{}/replicas/{}/", shard_num, replica_num); + root_path_in_backup = root_path_in_backup / fs::path{"shards"} / std::to_string(shard_num) / "replicas" / std::to_string(replica_num); } - - LOG_TRACE(log, "Will use path in backup: {}", quoteString(root_path_in_backup)); + LOG_TRACE(log, "Will use path in backup: {}", doubleQuoteString(String{root_path_in_backup})); } /// Finds databases and tables which we will put to the backup. @@ -254,8 +253,8 @@ void BackupEntriesCollector::collectTableInfo( storage->adjustCreateQueryForBackup(create_table_query); auto new_table_name = renaming_settings.getNewTableName(table_name); - String data_path_in_backup - = root_path_in_backup + "data/" + escapeForFileName(new_table_name.first) + "/" + escapeForFileName(new_table_name.second); + fs::path data_path_in_backup + = root_path_in_backup / "data" / escapeForFileName(new_table_name.first) / escapeForFileName(new_table_name.second); /// Check that information is consistent. const auto & create = create_table_query->as(); @@ -421,7 +420,7 @@ void BackupEntriesCollector::makeBackupEntriesForDatabasesDefs() renameInCreateQuery(new_create_query, renaming_settings, context); String new_database_name = renaming_settings.getNewDatabaseName(database_name); - String metadata_path_in_backup = root_path_in_backup + "metadata/" + escapeForFileName(new_database_name) + ".sql"; + auto metadata_path_in_backup = root_path_in_backup / "metadata" / (escapeForFileName(new_database_name) + ".sql"); backup_entries.emplace_back(metadata_path_in_backup, std::make_shared(serializeAST(*new_create_query))); } @@ -483,8 +482,8 @@ void BackupEntriesCollector::addBackupEntryForCreateQuery(const ASTPtr & create_ const auto & create = new_create_query->as(); String new_table_name = create.getTable(); String new_database_name = create.getDatabase(); - String metadata_path_in_backup - = root_path_in_backup + "metadata/" + escapeForFileName(new_database_name) + "/" + escapeForFileName(new_table_name) + ".sql"; + auto metadata_path_in_backup + = root_path_in_backup / "metadata" / escapeForFileName(new_database_name) / (escapeForFileName(new_table_name) + ".sql"); addBackupEntry(metadata_path_in_backup, std::make_shared(serializeAST(*create_query))); } diff --git a/src/Backups/BackupEntriesCollector.h b/src/Backups/BackupEntriesCollector.h index d9355744826..3ba113bff61 100644 --- a/src/Backups/BackupEntriesCollector.h +++ b/src/Backups/BackupEntriesCollector.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -99,7 +100,7 @@ private: Poco::Logger * log; Stage current_stage = Stage::kPreparing; - String root_path_in_backup; + std::filesystem::path root_path_in_backup; DDLRenamingSettings renaming_settings; struct DatabaseInfo @@ -114,7 +115,7 @@ private: StoragePtr storage; TableLockHolder table_lock; ASTPtr create_table_query; - String data_path_in_backup; + std::filesystem::path data_path_in_backup; std::optional partitions; }; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index dfed49bf5f5..7c9723984bf 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -66,6 +66,14 @@ namespace { return hexChecksum(size_and_checksum.second) + std::to_string(size_and_checksum.first); } + + /// We store entries' file names in the backup without leading slashes. + String removeLeadingSlash(const String & path) + { + if (path.starts_with('/')) + return path.substr(1); + return path; + } } @@ -243,10 +251,12 @@ void BackupImpl::writeBackupMetadata() config->setString("timestamp", toString(LocalDateTime{timestamp})); config->setString("uuid", toString(*uuid)); + auto all_file_infos = coordination->getAllFileInfos(); + if (base_backup_info) { bool base_backup_in_use = false; - for (const auto & info : coordination->getAllFileInfos()) + for (const auto & info : all_file_infos) { if (info.base_size) base_backup_in_use = true; @@ -260,7 +270,7 @@ void BackupImpl::writeBackupMetadata() } size_t index = 0; - for (const auto & info : coordination->getAllFileInfos()) + for (const auto & info : all_file_infos) { String prefix = index ? "contents.file[" + std::to_string(index) + "]." : "contents.file."; config->setUInt(prefix + "size", info.size); @@ -370,18 +380,25 @@ void BackupImpl::readBackupMetadata() } } -Strings BackupImpl::listFiles(const String & prefix, const String & terminator) const +Strings BackupImpl::listFiles(const String & directory, bool recursive) const { std::lock_guard lock{mutex}; - if (!prefix.ends_with('/') && !prefix.empty()) - throw Exception("prefix should end with '/'", ErrorCodes::BAD_ARGUMENTS); - return coordination->listFiles(prefix, terminator); + auto adjusted_dir = removeLeadingSlash(directory); + return coordination->listFiles(adjusted_dir, recursive); +} + +bool BackupImpl::hasFiles(const String & directory) const +{ + std::lock_guard lock{mutex}; + auto adjusted_dir = removeLeadingSlash(directory); + return coordination->hasFiles(adjusted_dir); } bool BackupImpl::fileExists(const String & file_name) const { std::lock_guard lock{mutex}; - return coordination->getFileInfo(file_name).has_value(); + auto adjusted_path = removeLeadingSlash(file_name); + return coordination->getFileInfo(adjusted_path).has_value(); } bool BackupImpl::fileExists(const SizeAndChecksum & size_and_checksum) const @@ -393,7 +410,8 @@ bool BackupImpl::fileExists(const SizeAndChecksum & size_and_checksum) const UInt64 BackupImpl::getFileSize(const String & file_name) const { std::lock_guard lock{mutex}; - auto info = coordination->getFileInfo(file_name); + auto adjusted_path = removeLeadingSlash(file_name); + auto info = coordination->getFileInfo(adjusted_path); if (!info) throw Exception( ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", backup_name, quoteString(file_name)); @@ -403,7 +421,8 @@ UInt64 BackupImpl::getFileSize(const String & file_name) const UInt128 BackupImpl::getFileChecksum(const String & file_name) const { std::lock_guard lock{mutex}; - auto info = coordination->getFileInfo(file_name); + auto adjusted_path = removeLeadingSlash(file_name); + auto info = coordination->getFileInfo(adjusted_path); if (!info) throw Exception( ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", backup_name, quoteString(file_name)); @@ -413,7 +432,8 @@ UInt128 BackupImpl::getFileChecksum(const String & file_name) const SizeAndChecksum BackupImpl::getFileSizeAndChecksum(const String & file_name) const { std::lock_guard lock{mutex}; - auto info = coordination->getFileInfo(file_name); + auto adjusted_path = removeLeadingSlash(file_name); + auto info = coordination->getFileInfo(adjusted_path); if (!info) throw Exception( ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", backup_name, quoteString(file_name)); @@ -492,12 +512,13 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) if (writing_finalized) throw Exception("Backup is already finalized", ErrorCodes::LOGICAL_ERROR); - if (coordination->getFileInfo(file_name)) + auto adjusted_path = removeLeadingSlash(file_name); + if (coordination->getFileInfo(adjusted_path)) throw Exception( ErrorCodes::BACKUP_ENTRY_ALREADY_EXISTS, "Backup {}: Entry {} already exists", backup_name, quoteString(file_name)); FileInfo info; - info.file_name = file_name; + info.file_name = adjusted_path; size_t size = entry->getSize(); info.size = size; @@ -518,13 +539,13 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) } /// Check if a entry with such name exists in the base backup. - bool base_exists = (base_backup && base_backup->fileExists(file_name)); + bool base_exists = (base_backup && base_backup->fileExists(adjusted_path)); UInt64 base_size = 0; UInt128 base_checksum{0, 0}; if (base_exists) { - base_size = base_backup->getFileSize(file_name); - base_checksum = base_backup->getFileChecksum(file_name); + base_size = base_backup->getFileSize(adjusted_path); + base_checksum = base_backup->getFileChecksum(adjusted_path); } std::unique_ptr read_buffer; /// We'll set that later. @@ -658,7 +679,7 @@ void BackupImpl::finalizeWriting() if (writing_finalized) throw Exception("Backup is already finalized", ErrorCodes::LOGICAL_ERROR); - if (coordination->getAllFileInfos().empty()) + if (!coordination->hasFiles("")) throw Exception("Backup must not be empty", ErrorCodes::BACKUP_IS_EMPTY); if (!is_internal_backup) diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index bb31a76ead1..f8c5bc0cf5f 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -57,7 +57,8 @@ public: OpenMode getOpenMode() const override { return open_mode; } time_t getTimestamp() const override; UUID getUUID() const override { return *uuid; } - Strings listFiles(const String & prefix, const String & terminator) const override; + Strings listFiles(const String & directory, bool recursive) const override; + bool hasFiles(const String & directory) const override; bool fileExists(const String & file_name) const override; bool fileExists(const SizeAndChecksum & size_and_checksum) const override; UInt64 getFileSize(const String & file_name) const override; diff --git a/src/Backups/IBackup.h b/src/Backups/IBackup.h index 857a47134ad..467c8fea4cd 100644 --- a/src/Backups/IBackup.h +++ b/src/Backups/IBackup.h @@ -36,14 +36,13 @@ public: /// Returns UUID of the backup. virtual UUID getUUID() const = 0; - /// Returns names of entries stored in the backup. - /// If `prefix` isn't empty the function will return only the names starting with - /// the prefix (but without the prefix itself). - /// If the `terminator` isn't empty the function will returns only parts of the names - /// before the terminator. For example, list("", "") returns names of all the entries - /// in the backup; and list("data/", "/") return kind of a list of folders and - /// files stored in the "data/" directory inside the backup. - virtual Strings listFiles(const String & prefix = "", const String & terminator = "/") const = 0; /// NOLINT + /// Returns names of entries stored in a specified directory in the backup. + /// If `directory` is empty or '/' the functions returns entries in the backup's root. + virtual Strings listFiles(const String & directory, bool recursive = false) const = 0; + + /// Checks if a specified directory contains any files. + /// The function returns the same as `!listFiles(directory).empty()`. + virtual bool hasFiles(const String & directory) const = 0; using SizeAndChecksum = std::pair; diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 1216c7331c2..92b7139ed5f 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -80,7 +80,8 @@ public: virtual void updateFileInfo(const FileInfo & file_info) = 0; virtual std::vector getAllFileInfos() const = 0; - virtual Strings listFiles(const String & prefix, const String & terminator) const = 0; + virtual Strings listFiles(const String & directory, bool recursive) const = 0; + virtual bool hasFiles(const String & directory) const = 0; using SizeAndChecksum = std::pair; diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index ed2cf4e0573..6ef7ef04895 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -15,6 +15,10 @@ #include #include #include +#include +#include + +namespace fs = std::filesystem; namespace DB @@ -162,11 +166,11 @@ void RestorerFromBackup::findRootPathsInBackup() root_paths_in_backup.clear(); /// Start with "" as the root path and then we will add shard- and replica-related part to it. - String root_path = ""; + fs::path root_path = "/"; root_paths_in_backup.push_back(root_path); /// Add shard-related part to the root path. - Strings shards_in_backup = backup->listFiles(root_path + "shards/"); + Strings shards_in_backup = backup->listFiles(root_path / "shards"); if (shards_in_backup.empty()) { if (restore_settings.shard_num_in_backup > 1) @@ -183,12 +187,12 @@ void RestorerFromBackup::findRootPathsInBackup() shard_name = std::to_string(shard_num); if (std::find(shards_in_backup.begin(), shards_in_backup.end(), shard_name) == shards_in_backup.end()) throw Exception(ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "No shard #{} in backup", shard_name); - root_path += "shards/" + shard_name + "/"; + root_path = root_path / "shards" / shard_name; root_paths_in_backup.push_back(root_path); } /// Add replica-related part to the root path. - Strings replicas_in_backup = backup->listFiles(root_path + "replicas/"); + Strings replicas_in_backup = backup->listFiles(root_path / "replicas"); if (replicas_in_backup.empty()) { if (restore_settings.replica_num_in_backup > 1) @@ -209,17 +213,23 @@ void RestorerFromBackup::findRootPathsInBackup() if (std::find(replicas_in_backup.begin(), replicas_in_backup.end(), replica_name) == replicas_in_backup.end()) replica_name = replicas_in_backup.front(); } - root_path += "replicas/" + replica_name + "/"; + root_path = root_path / "replicas" / replica_name; root_paths_in_backup.push_back(root_path); } /// Revert the list of root paths, because we need it in the following order: - /// "shards//replicas//" (first we search tables here) - /// "shards//" (then here) - /// "" (and finally here) + /// "/shards//replicas//" (first we search tables here) + /// "/shards//" (then here) + /// "/" (and finally here) std::reverse(root_paths_in_backup.begin(), root_paths_in_backup.end()); - LOG_TRACE(log, "Will use paths in backup: {}", joinQuotedStrings(root_paths_in_backup)); + LOG_TRACE( + log, + "Will use paths in backup: {}", + boost::algorithm::join( + root_paths_in_backup + | boost::adaptors::transformed([](const fs::path & path) -> String { return doubleQuoteString(String{path}); }), + ", ")); } void RestorerFromBackup::collectDatabaseAndTableInfos() @@ -253,16 +263,16 @@ void RestorerFromBackup::collectDatabaseAndTableInfos() void RestorerFromBackup::collectTableInfo(const DatabaseAndTableName & table_name_in_backup, const std::optional & partitions) { - std::optional metadata_path; - std::optional root_path_in_use; - for (const String & root_path : root_paths_in_backup) + std::optional metadata_path; + std::optional root_path_in_use; + for (const auto & root_path_in_backup : root_paths_in_backup) { - String try_metadata_path = root_path + "metadata/" + escapeForFileName(table_name_in_backup.first) + "/" - + escapeForFileName(table_name_in_backup.second) + ".sql"; + fs::path try_metadata_path = root_path_in_backup / "metadata" / escapeForFileName(table_name_in_backup.first) + / (escapeForFileName(table_name_in_backup.second) + ".sql"); if (backup->fileExists(try_metadata_path)) { metadata_path = try_metadata_path; - root_path_in_use = root_path; + root_path_in_use = root_path_in_backup; break; } } @@ -277,8 +287,8 @@ void RestorerFromBackup::collectTableInfo(const DatabaseAndTableName & table_nam } DatabaseAndTableName table_name = renaming_settings.getNewTableName(table_name_in_backup); - String data_path_in_backup = *root_path_in_use + "data/" + escapeForFileName(table_name_in_backup.first) + "/" - + escapeForFileName(table_name_in_backup.second); + fs::path data_path_in_backup + = *root_path_in_use / "data" / escapeForFileName(table_name_in_backup.first) / escapeForFileName(table_name_in_backup.second); auto read_buffer = backup->readFile(*metadata_path)->getReadBuffer(); String create_query_str; @@ -317,15 +327,15 @@ void RestorerFromBackup::collectTableInfo(const DatabaseAndTableName & table_nam void RestorerFromBackup::collectDatabaseInfo(const String & database_name_in_backup, const std::set & except_table_names) { - std::optional metadata_path; + std::optional metadata_path; std::unordered_set table_names_in_backup; - for (const String & root_path_in_backup : root_paths_in_backup) + for (const auto & root_path_in_backup : root_paths_in_backup) { - String try_metadata_path = root_path_in_backup + "metadata/" + escapeForFileName(database_name_in_backup) + ".sql"; + fs::path try_metadata_path = root_path_in_backup / "metadata" / (escapeForFileName(database_name_in_backup) + ".sql"); if (!metadata_path && backup->fileExists(try_metadata_path)) metadata_path = try_metadata_path; - Strings file_names = backup->listFiles(root_path_in_backup + "metadata/" + escapeForFileName(database_name_in_backup) + "/"); + Strings file_names = backup->listFiles(root_path_in_backup / "metadata" / escapeForFileName(database_name_in_backup)); for (const String & file_name : file_names) { constexpr const std::string_view sql_ext = ".sql"; @@ -385,9 +395,9 @@ void RestorerFromBackup::collectDatabaseInfo(const String & database_name_in_bac void RestorerFromBackup::collectAllDatabasesInfo(const std::set & except_database_names) { std::unordered_set database_names_in_backup; - for (const String & root_path_in_backup : root_paths_in_backup) + for (const auto & root_path_in_backup : root_paths_in_backup) { - Strings file_names = backup->listFiles(root_path_in_backup + "metadata/"); + Strings file_names = backup->listFiles(root_path_in_backup / "metadata"); for (String & file_name : file_names) { constexpr const std::string_view sql_ext = ".sql"; diff --git a/src/Backups/RestorerFromBackup.h b/src/Backups/RestorerFromBackup.h index 3562b8b68ea..c31ae70e522 100644 --- a/src/Backups/RestorerFromBackup.h +++ b/src/Backups/RestorerFromBackup.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -88,7 +89,7 @@ private: Poco::Logger * log; Stage current_stage = Stage::kPreparing; - Strings root_paths_in_backup; + std::vector root_paths_in_backup; DDLRenamingSettings renaming_settings; void setStage(Stage new_stage, const String & error_message = {}); @@ -111,7 +112,7 @@ private: { ASTPtr create_table_query; std::optional partitions; - String data_path_in_backup; + std::filesystem::path data_path_in_backup; }; std::map table_infos; diff --git a/src/Common/quoteString.cpp b/src/Common/quoteString.cpp index 2be748d1b93..e3e6e0b3249 100644 --- a/src/Common/quoteString.cpp +++ b/src/Common/quoteString.cpp @@ -44,16 +44,4 @@ String backQuoteIfNeed(const StringRef & x) return res; } -String joinQuotedStrings(const std::vector & strings, std::string_view separator) -{ - String res; - for (const auto & str : strings) - { - if (!res.empty()) - res += separator; - res += quoteString(str); - } - return res; -} - } diff --git a/src/Common/quoteString.h b/src/Common/quoteString.h index 7034b400a54..73c0de03d45 100644 --- a/src/Common/quoteString.h +++ b/src/Common/quoteString.h @@ -24,7 +24,4 @@ String backQuote(const StringRef & x); /// Quote the identifier with backquotes, if required. String backQuoteIfNeed(const StringRef & x); -/// Quotes each string and joins quoted strings using a specified separator between them. -String joinQuotedStrings(const std::vector & strings, std::string_view separator = ", "); - } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0199ba88b67..b7c70d1c5f6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4061,17 +4061,10 @@ Pipe MergeTreeData::alterPartition( void MergeTreeData::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) { - BackupEntries backup_entries = backupParts(backup_entries_collector.getContext(), partitions); - fs::path data_path_in_backup_fs = data_path_in_backup; - for (auto & pair: backup_entries) - { - auto & file_name = pair.first; - file_name = data_path_in_backup_fs / file_name; - } - backup_entries_collector.addBackupEntries(std::move(backup_entries)); + backup_entries_collector.addBackupEntries(backupParts(backup_entries_collector.getContext(), data_path_in_backup, partitions)); } -BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const std::optional & partitions) const +BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const String & data_path_in_backup, const std::optional & partitions) const { DataPartsVector data_parts; if (partitions) @@ -4081,6 +4074,7 @@ BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const BackupEntries backup_entries; std::map> temp_dirs; + fs::path data_path_in_backup_fs = data_path_in_backup; for (const auto & part : data_parts) { @@ -4098,7 +4092,7 @@ BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const for (const auto & [filepath, checksum] : part->checksums.files) { - String relative_filepath = fs::path(part->relative_path) / filepath; + String relative_filepath = data_path_in_backup_fs / part->relative_path / filepath; String hardlink_filepath = temp_part_dir / filepath; disk->createHardLink(part_dir / filepath, hardlink_filepath); UInt128 file_hash{checksum.file_hash.first, checksum.file_hash.second}; @@ -4109,7 +4103,7 @@ BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const for (const auto & filepath : part->getFileNamesWithoutChecksums()) { - String relative_filepath = fs::path(part->relative_path) / filepath; + auto relative_filepath = data_path_in_backup_fs / part->relative_path / filepath; backup_entries.emplace_back( relative_filepath, std::make_unique(disk, part_dir / filepath)); } @@ -4121,7 +4115,7 @@ BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const void MergeTreeData::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) { auto backup = restorer.getBackup(); - if (!restorer.isNonEmptyTableAllowed() && getTotalActiveSizeInBytes() && !backup->listFiles(data_path_in_backup + '/').empty()) + if (!restorer.isNonEmptyTableAllowed() && getTotalActiveSizeInBytes() && backup->hasFiles(data_path_in_backup)) restorer.throwTableIsNotEmpty(getStorageID()); restorePartsFromBackup(restorer, data_path_in_backup, partitions); @@ -4185,7 +4179,7 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const partition_ids = getPartitionIDsFromQuery(*partitions, restorer.getContext()); auto backup = restorer.getBackup(); - Strings part_names = backup->listFiles(data_path_in_backup + '/'); + Strings part_names = backup->listFiles(data_path_in_backup); auto restored_parts_holder = std::make_shared(std::static_pointer_cast(shared_from_this()), backup, part_names.size()); @@ -4221,7 +4215,7 @@ void MergeTreeData::restorePartFromBackup(std::shared_ptr r auto backup = restored_parts_holder->getBackup(); UInt64 total_size_of_part = 0; - Strings filenames = backup->listFiles(part_path_in_backup + '/', ""); + Strings filenames = backup->listFiles(part_path_in_backup, /* recursive= */ true); fs::path part_path_in_backup_fs = part_path_in_backup; for (const String & filename : filenames) total_size_of_part += backup->getFileSize(part_path_in_backup_fs / filename); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 79c81da5309..d531fc9f339 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1230,7 +1230,7 @@ protected: bool movePartsToSpace(const DataPartsVector & parts, SpacePtr space); /// Makes backup entries to backup the parts of this table. - BackupEntries backupParts(const ContextPtr & local_context, const std::optional & partitions) const; + BackupEntries backupParts(const ContextPtr & local_context, const String & data_path_in_backup, const std::optional & partitions) const; class RestoredPartsHolder; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 80ee7d0caa3..55782f35d8e 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -996,7 +996,7 @@ void StorageLog::restoreDataFromBackup(RestorerFromBackup & restorer, const Stri return; auto backup = restorer.getBackup(); - if (!restorer.isNonEmptyTableAllowed() && total_bytes && !backup->listFiles(data_path_in_backup + '/').empty()) + if (!restorer.isNonEmptyTableAllowed() && total_bytes && backup->hasFiles(data_path_in_backup)) RestorerFromBackup::throwTableIsNotEmpty(getStorageID()); auto lock_timeout = getLockTimeout(restorer.getContext()); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 86ec7ab8ae5..645fbf9c6ef 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -497,7 +497,7 @@ void StorageMemory::restoreDataFromBackup(RestorerFromBackup & restorer, const S RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName()); auto backup = restorer.getBackup(); - if (!restorer.isNonEmptyTableAllowed() && total_size_bytes && !backup->listFiles(data_path_in_backup + '/').empty()) + if (!restorer.isNonEmptyTableAllowed() && total_size_bytes && backup->hasFiles(data_path_in_backup)) RestorerFromBackup::throwTableIsNotEmpty(getStorageID()); restorer.addDataRestoreTask( diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3f6e6fc0a7b..632cc702be7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8283,7 +8283,7 @@ void StorageReplicatedMergeTree::backupData( /// First we generate backup entries in the same way as an ordinary MergeTree does. /// But then we don't add them to the BackupEntriesCollector right away, /// because we need to coordinate them with other replicas (other replicas can have better parts). - auto backup_entries = backupParts(backup_entries_collector.getContext(), partitions); + auto backup_entries = backupParts(backup_entries_collector.getContext(), "", partitions); auto coordination = backup_entries_collector.getBackupCoordination(); String full_zk_path = getZooKeeperName() + getZooKeeperPath(); @@ -8380,7 +8380,7 @@ void StorageReplicatedMergeTree::restoreDataFromBackup(RestorerFromBackup & rest empty = false; } auto backup = restorer.getBackup(); - if (!empty && !backup->listFiles(data_path_in_backup + '/').empty()) + if (!empty && backup->hasFiles(data_path_in_backup)) restorer.throwTableIsNotEmpty(getStorageID()); } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index e7dce95ee7d..be7fdbeccf1 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -596,7 +596,7 @@ void StorageStripeLog::restoreDataFromBackup(RestorerFromBackup & restorer, cons RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName()); auto backup = restorer.getBackup(); - if (!restorer.isNonEmptyTableAllowed() && total_bytes && !backup->listFiles(data_path_in_backup + '/').empty()) + if (!restorer.isNonEmptyTableAllowed() && total_bytes && backup->hasFiles(data_path_in_backup)) RestorerFromBackup::throwTableIsNotEmpty(getStorageID()); auto lock_timeout = getLockTimeout(restorer.getContext());