mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-30 03:22:14 +00:00
Simplify path calculations in backup.
This commit is contained in:
parent
592f568f83
commit
21f3bed435
@ -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();
|
||||
|
@ -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;
|
||||
|
@ -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};
|
||||
|
@ -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;
|
||||
|
@ -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)));
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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>;
|
||||
|
||||
|
@ -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>;
|
||||
|
||||
|
@ -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";
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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 = ", ");
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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());
|
||||
|
@ -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(
|
||||
|
@ -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());
|
||||
}
|
||||
|
||||
|
@ -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());
|
||||
|
Loading…
Reference in New Issue
Block a user