Simplify path calculations in backup.

This commit is contained in:
Vitaly Baranov 2022-06-06 11:50:20 +02:00
parent 592f568f83
commit 21f3bed435
20 changed files with 168 additions and 106 deletions

View File

@ -301,12 +301,19 @@ std::vector<FileInfo> 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<std::string_view> 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<FileInfo> BackupCoordinationDistributed::getFileInfo(const String & file_name) const
{
auto zookeeper = get_zookeeper();

View File

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

View File

@ -89,9 +89,14 @@ std::vector<FileInfo> 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<FileInfo> BackupCoordinationLocal::getFileInfo(const String & file_name) const
{
std::lock_guard lock{mutex};

View File

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

View File

@ -10,6 +10,9 @@
#include <base/insertAtEnd.h>
#include <Common/escapeForFileName.h>
#include <boost/range/algorithm/copy.hpp>
#include <filesystem>
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/<shard_num>/replicas/<replica_num>/".
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<const ASTCreateQuery &>();
@ -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<BackupEntryFromMemory>(serializeAST(*new_create_query)));
}
@ -483,8 +482,8 @@ void BackupEntriesCollector::addBackupEntryForCreateQuery(const ASTPtr & create_
const auto & create = new_create_query->as<const ASTCreateQuery &>();
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<BackupEntryFromMemory>(serializeAST(*create_query)));
}

View File

@ -5,6 +5,7 @@
#include <Parsers/ASTBackupQuery.h>
#include <Storages/IStorage_fwd.h>
#include <Storages/TableLockHolder.h>
#include <filesystem>
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<ASTs> partitions;
};

View File

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

View File

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

View File

@ -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<UInt64, UInt128>;

View File

@ -80,7 +80,8 @@ public:
virtual void updateFileInfo(const FileInfo & file_info) = 0;
virtual std::vector<FileInfo> 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<UInt64, UInt128>;

View File

@ -15,6 +15,10 @@
#include <Common/escapeForFileName.h>
#include <Common/quoteString.h>
#include <base/insertAtEnd.h>
#include <boost/algorithm/string/join.hpp>
#include <filesystem>
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/<shard_num>/replicas/<replica_num>/" (first we search tables here)
/// "shards/<shard_num>/" (then here)
/// "" (and finally here)
/// "/shards/<shard_num>/replicas/<replica_num>/" (first we search tables here)
/// "/shards/<shard_num>/" (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<ASTs> & partitions)
{
std::optional<String> metadata_path;
std::optional<String> root_path_in_use;
for (const String & root_path : root_paths_in_backup)
std::optional<fs::path> metadata_path;
std::optional<fs::path> 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<String> & except_table_names)
{
std::optional<String> metadata_path;
std::optional<fs::path> metadata_path;
std::unordered_set<String> 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<String> & except_database_names)
{
std::unordered_set<String> 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";

View File

@ -6,6 +6,7 @@
#include <Storages/TableLockHolder.h>
#include <Storages/IStorage_fwd.h>
#include <Interpreters/Context_fwd.h>
#include <filesystem>
namespace DB
@ -88,7 +89,7 @@ private:
Poco::Logger * log;
Stage current_stage = Stage::kPreparing;
Strings root_paths_in_backup;
std::vector<std::filesystem::path> 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<ASTs> partitions;
String data_path_in_backup;
std::filesystem::path data_path_in_backup;
};
std::map<DatabaseAndTableName, TableInfo> table_infos;

View File

@ -44,16 +44,4 @@ String backQuoteIfNeed(const StringRef & x)
return res;
}
String joinQuotedStrings(const std::vector<String> & strings, std::string_view separator)
{
String res;
for (const auto & str : strings)
{
if (!res.empty())
res += separator;
res += quoteString(str);
}
return res;
}
}

View File

@ -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<String> & strings, std::string_view separator = ", ");
}

View File

@ -4061,17 +4061,10 @@ Pipe MergeTreeData::alterPartition(
void MergeTreeData::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & 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<ASTs> & partitions) const
BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const String & data_path_in_backup, const std::optional<ASTs> & partitions) const
{
DataPartsVector data_parts;
if (partitions)
@ -4081,6 +4074,7 @@ BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const
BackupEntries backup_entries;
std::map<DiskPtr, std::shared_ptr<TemporaryFileOnDisk>> 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<BackupEntryFromSmallFile>(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<ASTs> & 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<RestoredPartsHolder>(std::static_pointer_cast<MergeTreeData>(shared_from_this()), backup, part_names.size());
@ -4221,7 +4215,7 @@ void MergeTreeData::restorePartFromBackup(std::shared_ptr<RestoredPartsHolder> 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);

View File

@ -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<ASTs> & partitions) const;
BackupEntries backupParts(const ContextPtr & local_context, const String & data_path_in_backup, const std::optional<ASTs> & partitions) const;
class RestoredPartsHolder;

View File

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

View File

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

View File

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

View File

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