Implemented backup version 2: now files in backups are named by their checksums.

It will allow to store duplicate files only one time.
This commit is contained in:
Vitaly Baranov 2022-04-16 22:18:42 +02:00
parent acd28d8a1d
commit f14613f433
4 changed files with 233 additions and 140 deletions

View File

@ -35,7 +35,10 @@ namespace ErrorCodes
namespace
{
const UInt64 BACKUP_VERSION = 1;
const UInt64 INITIAL_BACKUP_VERSION = 1;
const UInt64 CURRENT_BACKUP_VERSION = 2;
const UInt64 BACKUP_VERSION_USE_CHECKSUMS_AS_DATA_FILE_NAMES = 2;
UInt128 unhexChecksum(const String & checksum)
{
@ -51,18 +54,18 @@ class BackupImpl::BackupEntryFromBackupImpl : public IBackupEntry
public:
BackupEntryFromBackupImpl(
const std::shared_ptr<const BackupImpl> & backup_,
const String & file_name_,
const String & data_file_name_,
UInt64 size_,
const std::optional<UInt128> checksum_,
const UInt128 checksum_,
BackupEntryPtr base_backup_entry_ = {})
: backup(backup_), file_name(file_name_), size(size_), checksum(checksum_),
: backup(backup_), data_file_name(data_file_name_), size(size_), checksum(checksum_),
base_backup_entry(std::move(base_backup_entry_))
{
}
std::unique_ptr<ReadBuffer> getReadBuffer() const override
{
auto read_buffer = backup->readFileImpl(file_name);
auto read_buffer = backup->readFileImpl(data_file_name);
if (base_backup_entry)
{
auto base_backup_read_buffer = base_backup_entry->getReadBuffer();
@ -76,15 +79,93 @@ public:
private:
const std::shared_ptr<const BackupImpl> backup;
const String file_name;
const String data_file_name;
const UInt64 size;
const std::optional<UInt128> checksum;
const UInt128 checksum;
BackupEntryPtr base_backup_entry;
};
class BackupImpl::LocalFileInfos : public IFileInfos
{
public:
LocalFileInfos() = default;
~LocalFileInfos() override = default;
void add(FileInfo && file_info, bool & is_new_checksum) override
{
file_names.emplace(file_info.file_name, file_info.checksum);
is_new_checksum = (file_info.checksum && !file_infos.contains(file_info.checksum));
if (is_new_checksum)
file_infos.emplace(file_info.checksum, std::move(file_info));
}
std::vector<FileInfo> getAllFileInfos() override
{
std::vector<FileInfo> res;
for (const auto & [file_name, checksum] : file_names)
{
FileInfo info = file_infos.at(checksum);
info.file_name = file_name;
res.push_back(std::move(info));
}
return res;
}
Strings listFiles(const String & prefix, const String & terminator) override
{
Strings elements;
for (auto it = file_names.lower_bound(prefix); it != file_names.end(); ++it)
{
const String & name = it->first;
if (!name.starts_with(prefix))
break;
size_t start_pos = prefix.length();
size_t end_pos = String::npos;
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))
continue;
elements.push_back(String{new_element});
}
return elements;
}
std::optional<UInt128> getChecksumByFileName(const String & file_name) override
{
auto it = file_names.find(file_name);
if (it == file_names.end())
return std::nullopt;
return it->second;
}
std::optional<FileInfo> getFileInfoByChecksum(const UInt128 & checksum) override
{
auto it = file_infos.find(checksum);
if (it == file_infos.end())
return std::nullopt;
return it->second;
}
std::optional<FileInfo> getFileInfoByFileName(const String & file_name) override
{
auto it = file_names.find(file_name);
if (it == file_names.end())
return std::nullopt;
FileInfo info = file_infos.at(it->second);
info.file_name = file_name;
return info;
}
private:
std::map<String /* file_name */, UInt128 /* checksum */> file_names; /// Should be ordered alphabetically, see listFiles(). For empty files we assume checksum = 0.
std::unordered_map<UInt128 /* checksum */, FileInfo> file_infos; /// Information about files. Without empty files.
};
BackupImpl::BackupImpl(const String & backup_name_, const ContextPtr & context_, const std::optional<BackupInfo> & base_backup_info_)
: backup_name(backup_name_), context(context_), base_backup_info_param(base_backup_info_)
: backup_name(backup_name_), context(context_), base_backup_info_param(base_backup_info_), file_infos(std::make_unique<LocalFileInfos>())
{
}
@ -107,7 +188,6 @@ void BackupImpl::open(OpenMode open_mode_)
timestamp = std::time(nullptr);
uuid = UUIDHelpers::generateV4();
writing_finalized = false;
written_files.clear();
}
if (open_mode_ == OpenMode::READ)
@ -146,14 +226,14 @@ void BackupImpl::close()
if (open_mode == OpenMode::NONE)
return;
closeImpl(written_files, writing_finalized);
closeImpl({}/*written_files*/, writing_finalized);
uuid = UUIDHelpers::Nil;
timestamp = 0;
base_backup_info.reset();
base_backup.reset();
base_backup_uuid.reset();
file_infos.clear();
//file_infos.clear();
open_mode = OpenMode::NONE;
}
@ -172,14 +252,14 @@ time_t BackupImpl::getTimestamp() const
void BackupImpl::writeBackupMetadata()
{
Poco::AutoPtr<Poco::Util::XMLConfiguration> config{new Poco::Util::XMLConfiguration()};
config->setUInt("version", BACKUP_VERSION);
config->setUInt("version", CURRENT_BACKUP_VERSION);
config->setString("timestamp", toString(LocalDateTime{timestamp}));
config->setString("uuid", toString(uuid));
if (base_backup_info)
{
bool base_backup_in_use = false;
for (const auto & [name, info] : file_infos)
for (const auto & info : file_infos->getAllFileInfos())
{
if (info.base_size)
base_backup_in_use = true;
@ -193,10 +273,10 @@ void BackupImpl::writeBackupMetadata()
}
size_t index = 0;
for (const auto & [name, info] : file_infos)
for (const auto & info : file_infos->getAllFileInfos())
{
String prefix = index ? "contents.file[" + std::to_string(index) + "]." : "contents.file.";
config->setString(prefix + "name", name);
config->setString(prefix + "name", info.file_name);
config->setUInt(prefix + "size", info.size);
if (info.size)
{
@ -217,7 +297,7 @@ void BackupImpl::writeBackupMetadata()
std::ostringstream stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
config->save(stream);
String str = stream.str();
written_files.push_back(".backup");
//written_files.push_back(".backup");
auto out = writeFileImpl(".backup");
out->write(str.data(), str.size());
}
@ -231,8 +311,8 @@ void BackupImpl::readBackupMetadata()
Poco::AutoPtr<Poco::Util::XMLConfiguration> config{new Poco::Util::XMLConfiguration()};
config->load(stream);
UInt64 version = config->getUInt("version");
if (version != BACKUP_VERSION)
version = config->getUInt("version");
if ((version < INITIAL_BACKUP_VERSION) || (version > CURRENT_BACKUP_VERSION))
throw Exception(ErrorCodes::BACKUP_VERSION_NOT_SUPPORTED, "Backup {}: Version {} is not supported", getName(), version);
timestamp = parse<LocalDateTime>(config->getString("timestamp")).to_time_t();
@ -244,7 +324,7 @@ void BackupImpl::readBackupMetadata()
if (config->has("base_backup_uuid"))
base_backup_uuid = parse<UUID>(config->getString("base_backup_uuid"));
file_infos.clear();
//file_infos.clear();
Poco::Util::AbstractConfiguration::Keys keys;
config->keys("contents", keys);
for (const auto & key : keys)
@ -252,24 +332,25 @@ void BackupImpl::readBackupMetadata()
if ((key == "file") || key.starts_with("file["))
{
String prefix = "contents." + key + ".";
String name = config->getString(prefix + "name");
FileInfo info;
info.file_name = config->getString(prefix + "name");
info.size = config->getUInt(prefix + "size");
if (info.size)
info.checksum = info.size ? unhexChecksum(config->getString(prefix + "checksum")) : UInt128{0};
bool use_base = config->getBool(prefix + "use_base", false);
info.base_size = config->getUInt(prefix + "base_size", use_base ? info.size : 0);
if (info.base_size)
use_base = true;
if (use_base)
{
info.checksum = unhexChecksum(config->getString(prefix + "checksum"));
bool use_base = config->getBool(prefix + "use_base", false);
info.base_size = config->getUInt(prefix + "base_size", use_base ? info.size : 0);
if (info.base_size)
{
if (info.base_size == info.size)
info.base_checksum = info.checksum;
else
info.base_checksum = unhexChecksum(config->getString(prefix + "base_checksum"));
}
if (info.base_size == info.size)
info.base_checksum = info.checksum;
else
info.base_checksum = unhexChecksum(config->getString(prefix + "base_checksum"));
}
file_infos.emplace(name, info);
file_checksums.emplace(info.checksum, name);
file_infos->add(std::move(info));
}
}
}
@ -278,73 +359,64 @@ Strings BackupImpl::listFiles(const String & prefix, const String & terminator)
{
if (!prefix.ends_with('/') && !prefix.empty())
throw Exception("prefix should end with '/'", ErrorCodes::BAD_ARGUMENTS);
std::lock_guard lock{mutex};
Strings elements;
for (auto it = file_infos.lower_bound(prefix); it != file_infos.end(); ++it)
{
const String & name = it->first;
if (!name.starts_with(prefix))
break;
size_t start_pos = prefix.length();
size_t end_pos = String::npos;
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))
continue;
elements.push_back(String{new_element});
}
return elements;
return file_infos->listFiles(prefix, terminator);
}
bool BackupImpl::fileExists(const String & file_name) const
{
std::lock_guard lock{mutex};
return file_infos.contains(file_name);
return file_infos->getChecksumByFileName(file_name).has_value();
}
bool BackupImpl::fileExistsByChecksum(const UInt128 & checksum) const
{
return file_infos->getFileInfoByChecksum(checksum).has_value();
}
size_t BackupImpl::getFileSize(const String & file_name) const
{
std::lock_guard lock{mutex};
auto it = file_infos.find(file_name);
if (it == file_infos.end())
auto info = file_infos->getFileInfoByFileName(file_name);
if (!info)
throw Exception(
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", getName(), quoteString(file_name));
return it->second.size;
return info->size;
}
size_t BackupImpl::getFileSizeByChecksum(const UInt128 & checksum) const
{
auto info = file_infos->getFileInfoByChecksum(checksum);
if (!info)
throw Exception(
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", getName(), getHexUIntLowercase(checksum));
return info->size;
}
UInt128 BackupImpl::getFileChecksum(const String & file_name) const
{
std::lock_guard lock{mutex};
auto it = file_infos.find(file_name);
if (it == file_infos.end())
auto info = file_infos->getFileInfoByFileName(file_name);
if (!info)
throw Exception(
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", getName(), quoteString(file_name));
return it->second.checksum;
return info->checksum;
}
std::optional<String> BackupImpl::findFileByChecksum(const UInt128 & checksum) const
{
std::lock_guard lock{mutex};
auto it = file_checksums.find(checksum);
if (it == file_checksums.end())
return std::nullopt;
return it->second;
}
BackupEntryPtr BackupImpl::readFile(const String & file_name) const
{
return readFileByChecksum(getFileChecksum(file_name));
}
BackupEntryPtr BackupImpl::readFileByChecksum(const UInt128 & checksum) const
{
std::lock_guard lock{mutex};
if (open_mode != OpenMode::READ)
throw Exception("Backup is not opened for reading", ErrorCodes::LOGICAL_ERROR);
auto it = file_infos.find(file_name);
if (it == file_infos.end())
auto info_opt = file_infos->getFileInfoByChecksum(checksum);
if (!info_opt)
throw Exception(
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", getName(), quoteString(file_name));
ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Backup {}: Entry {} not found in the backup", getName(), getHexUIntLowercase(checksum));
const auto & info = it->second;
const auto & info = *info_opt;
if (!info.size)
{
/// Entry's data is empty.
@ -354,8 +426,9 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const
if (!info.base_size)
{
/// Data goes completely from this backup, the base backup isn't used.
String data_file_name = (version >= BACKUP_VERSION_USE_CHECKSUMS_AS_DATA_FILE_NAMES) ? getHexUIntLowercase(checksum) : info.file_name;
return std::make_unique<BackupEntryFromBackupImpl>(
std::static_pointer_cast<const BackupImpl>(shared_from_this()), file_name, info.size, info.checksum);
std::static_pointer_cast<const BackupImpl>(shared_from_this()), data_file_name, info.size, info.checksum);
}
if (info.size < info.base_size)
@ -363,7 +436,7 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const
throw Exception(
ErrorCodes::BACKUP_DAMAGED,
"Backup {}: Entry {} has its data size less than in the base backup {}: {} < {}",
getName(), quoteString(file_name), base_backup->getName(), info.size, info.base_size);
getName(), getHexUIntLowercase(checksum), base_backup->getName(), info.size, info.base_size);
}
if (!base_backup)
@ -371,26 +444,25 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const
throw Exception(
ErrorCodes::NO_BASE_BACKUP,
"Backup {}: Entry {} is marked to be read from a base backup, but there is no base backup specified",
getName(), quoteString(file_name));
getName(), getHexUIntLowercase(checksum));
}
auto base_file_name = base_backup->findFileByChecksum(info.base_checksum);
if (!base_file_name)
if (!base_backup->fileExistsByChecksum(info.base_checksum))
{
throw Exception(
ErrorCodes::WRONG_BASE_BACKUP,
"Backup {}: Entry {} is marked to be read from a base backup, but doesn't exist there",
getName(), quoteString(file_name));
getName(), getHexUIntLowercase(checksum));
}
auto base_entry = base_backup->readFile(*base_file_name);
auto base_entry = base_backup->readFileByChecksum(info.base_checksum);
auto base_size = base_entry->getSize();
if (base_size != info.base_size)
{
throw Exception(
ErrorCodes::WRONG_BASE_BACKUP,
"Backup {}: Entry {} has unexpected size in the base backup {}: {} (expected size: {})",
getName(), quoteString(file_name), base_backup->getName(), base_size, info.base_size);
getName(), getHexUIntLowercase(checksum), base_backup->getName(), base_size, info.base_size);
}
if (info.size == info.base_size)
@ -399,10 +471,13 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const
return base_entry;
}
/// The beginning of the data goes from the base backup,
/// and the ending goes from this backup.
return std::make_unique<BackupEntryFromBackupImpl>(
static_pointer_cast<const BackupImpl>(shared_from_this()), file_name, info.size, info.checksum, std::move(base_entry));
{
/// The beginning of the data goes from the base backup,
/// and the ending goes from this backup.
String data_file_name = (version >= BACKUP_VERSION_USE_CHECKSUMS_AS_DATA_FILE_NAMES) ? getHexUIntLowercase(checksum) : info.file_name;
return std::make_unique<BackupEntryFromBackupImpl>(
static_pointer_cast<const BackupImpl>(shared_from_this()), data_file_name, info.size, info.checksum, std::move(base_entry));
}
}
@ -412,17 +487,28 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry)
if (open_mode != OpenMode::WRITE)
throw Exception("Backup is not opened for writing", ErrorCodes::LOGICAL_ERROR);
if (file_infos.contains(file_name))
if (file_infos->getChecksumByFileName(file_name))
throw Exception(
ErrorCodes::BACKUP_ENTRY_ALREADY_EXISTS, "Backup {}: Entry {} already exists", getName(), quoteString(file_name));
UInt64 size = entry->getSize();
std::optional<UInt128> checksum = entry->getChecksum();
FileInfo info;
info.file_name = file_name;
size_t size = entry->getSize();
info.size = size;
/// Check if the entry's data is empty.
if (!size)
if (!info.size)
{
file_infos.emplace(file_name, FileInfo{});
file_infos->add(std::move(info));
return;
}
/// Maybe we have a copy of this file in the backup already.
std::optional<UInt128> checksum = entry->getChecksum();
if (checksum && file_infos->getFileInfoByChecksum(*checksum))
{
info.checksum = *checksum;
file_infos->add(std::move(info));
return;
}
@ -477,36 +563,43 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry)
checksum = hashing_read_buffer->getHash();
}
hashing_read_buffer.reset();
info.checksum = *checksum;
/// Maybe we have a copy of this file in the backup already.
if (file_infos->getFileInfoByChecksum(*checksum))
{
file_infos->add(std::move(info));
return;
}
/// Check if a entry with the same checksum exists in the base backup.
if (base_backup && !use_base)
if (base_backup && !use_base && base_backup->fileExistsByChecksum(*checksum))
{
if (auto base_file_name = base_backup->findFileByChecksum(*checksum))
{
if (size == base_backup->getFileSize(*base_file_name))
{
/// The entry's data has not changed since the base backup,
/// but the entry itself has been moved or renamed.
base_size = size;
base_checksum = *checksum;
use_base = true;
}
}
/// The entry's data has not changed since the base backup,
/// but the entry itself has been moved or renamed.
base_size = size;
base_checksum = *checksum;
use_base = true;
}
if (use_base)
{
info.base_size = base_size;
info.base_checksum = base_checksum;
}
if (use_base && (size == base_size))
{
/// The entry's data has not been changed since the base backup.
FileInfo info;
info.size = size;
info.checksum = *checksum;
info.base_size = base_size;
info.base_checksum = base_checksum;
file_infos.emplace(file_name, info);
file_checksums.emplace(*checksum, file_name);
file_infos->add(std::move(info));
return;
}
bool is_new_checksum;
file_infos->add(std::move(info), is_new_checksum);
if (!is_new_checksum)
return; /// We copy data only if it's a new checksum.
/// Either the entry wasn't exist in the base backup
/// or the entry has data appended to the end of the data from the base backup.
/// In both those cases we have to copy data to this backup.
@ -527,21 +620,8 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry)
}
/// Copy the entry's data after `copy_pos`.
written_files.push_back(file_name);
auto out = writeFileImpl(file_name);
auto out = writeFileImpl(getHexUIntLowercase(*checksum));
copyData(*read_buffer, *out);
/// Done!
FileInfo info;
info.size = size;
info.checksum = *checksum;
if (use_base)
{
info.base_size = base_size;
info.base_checksum = base_checksum;
}
file_infos.emplace(file_name, info);
file_checksums.emplace(*checksum, file_name);
}

View File

@ -33,10 +33,12 @@ public:
UUID getUUID() const override { return uuid; }
Strings listFiles(const String & prefix, const String & terminator) const override;
bool fileExists(const String & file_name) const override;
bool fileExistsByChecksum(const UInt128 & checksum) const override;
size_t getFileSize(const String & file_name) const override;
size_t getFileSizeByChecksum(const UInt128 & checksum) const override;
UInt128 getFileChecksum(const String & file_name) const override;
std::optional<String> findFileByChecksum(const UInt128 & checksum) const override;
BackupEntryPtr readFile(const String & file_name) const override;
BackupEntryPtr readFileByChecksum(const UInt128 & checksum) const override;
void writeFile(const String & file_name, BackupEntryPtr entry) override;
void finalizeWriting() override;
@ -65,6 +67,8 @@ private:
struct FileInfo
{
String file_name;
UInt64 size = 0;
UInt128 checksum{0, 0};
@ -73,7 +77,21 @@ private:
UInt128 base_checksum{0, 0};
};
class IFileInfos
{
public:
virtual ~IFileInfos() {}
virtual void add(FileInfo && file_info, bool & is_new_checksum) = 0;
void add(FileInfo && file_info) { bool dummy; add(std::move(file_info), dummy); }
virtual std::vector<FileInfo> getAllFileInfos() = 0;
virtual Strings listFiles(const String & prefix, const String & terminator) = 0;
virtual std::optional<UInt128> getChecksumByFileName(const String & file_name) = 0;
virtual std::optional<FileInfo> getFileInfoByChecksum(const UInt128 & checksum) = 0;
virtual std::optional<FileInfo> getFileInfoByFileName(const String & file_name) = 0;
};
class BackupEntryFromBackupImpl;
class LocalFileInfos;
const String backup_name;
ContextPtr context;
@ -81,12 +99,11 @@ private:
OpenMode open_mode = OpenMode::NONE;
UUID uuid = {};
time_t timestamp = 0;
UInt64 version = 1;
std::optional<BackupInfo> base_backup_info;
std::shared_ptr<const IBackup> base_backup;
std::optional<UUID> base_backup_uuid;
std::map<String, FileInfo> file_infos; /// Should be ordered alphabetically, see listFiles().
std::unordered_map<UInt128, String> file_checksums;
Strings written_files;
std::unique_ptr<IFileInfos> file_infos;
bool writing_finalized = false;
};

View File

@ -53,20 +53,20 @@ public:
/// Checks if an entry with a specified name exists.
virtual bool fileExists(const String & file_name) const = 0;
virtual bool fileExistsByChecksum(const UInt128 & checksum) const = 0;
/// Returns the size of the entry's data.
/// This function does the same as `read(file_name)->getSize()` but faster.
virtual size_t getFileSize(const String & file_name) const = 0;
virtual size_t getFileSizeByChecksum(const UInt128 & checksum) const = 0;
/// Returns the checksum of the entry's data.
/// This function does the same as `read(file_name)->getCheckum()` but faster.
virtual UInt128 getFileChecksum(const String & file_name) const = 0;
/// Finds a file by its checksum, returns nullopt if not found.
virtual std::optional<String> findFileByChecksum(const UInt128 & checksum) const = 0;
/// Reads an entry from the backup.
virtual BackupEntryPtr readFile(const String & file_name) const = 0;
virtual BackupEntryPtr readFileByChecksum(const UInt128 & checksum) const = 0;
/// Puts a new entry to the backup.
virtual void writeFile(const String & file_name, BackupEntryPtr entry) = 0;

View File

@ -162,16 +162,12 @@ def test_incremental_backup_after_renaming_table():
# Files in a base backup can be searched by checksum, so an incremental backup with a renamed table actually
# contains only its changed metadata.
assert os.path.isdir(os.path.join(get_backup_dir(backup_name), "metadata")) == True
assert os.path.isdir(os.path.join(get_backup_dir(backup_name), "data")) == True
assert (
os.path.isdir(os.path.join(get_backup_dir(incremental_backup_name), "metadata"))
== True
)
assert (
os.path.isdir(os.path.join(get_backup_dir(incremental_backup_name), "data"))
== False
)
contents = os.listdir(get_backup_dir(incremental_backup_name))
assert '.backup' in contents
contents.remove('.backup')
assert len(contents) == 1
with open(os.path.join(get_backup_dir(incremental_backup_name), contents[0])) as table_def_in_backup:
assert table_def_in_backup.read().startswith('CREATE TABLE test.table2')
instance.query("DROP TABLE test.table2")
instance.query(f"RESTORE TABLE test.table2 FROM {incremental_backup_name}")