Move checksum calculation to IBackupEntry.

This commit is contained in:
Vitaly Baranov 2023-05-01 14:23:59 +02:00
parent 002fd19cb7
commit 517e119e03
18 changed files with 228 additions and 129 deletions

View File

@ -1,25 +1,52 @@
#include <Backups/BackupEntryFromAppendOnlyFile.h>
#include <Disks/DiskEncrypted.h>
#include <IO/LimitSeekableReadBuffer.h>
namespace DB
{
namespace
{
UInt64 calculateSize(const DiskPtr & disk, const String & file_path, const std::optional<UInt64> & file_size, bool disk_is_encrypted)
{
if (file_size)
{
if (disk_is_encrypted)
return DiskEncrypted::convertFileSizeToEncryptedFileSize(*file_size);
else
return *file_size;
}
else
{
if (disk_is_encrypted)
return disk->getEncryptedFileSize(file_path);
else
return disk->getFileSize(file_path);
}
}
}
BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile(
const DiskPtr & disk_,
const String & file_path_,
const ReadSettings & settings_,
const std::optional<UInt64> & file_size_,
const std::optional<UInt128> & checksum_)
: BackupEntryFromImmutableFile(disk_, file_path_, settings_, file_size_, checksum_)
, limit(BackupEntryFromImmutableFile::getSize())
const DiskPtr & disk_, const String & file_path_, const ReadSettings & settings_, const std::optional<UInt64> & file_size_)
: disk(disk_)
, file_path(file_path_)
, data_source_description(disk->getDataSourceDescription())
, settings(settings_)
, size(calculateSize(disk_, file_path_, file_size_, data_source_description.is_encrypted))
{
}
BackupEntryFromAppendOnlyFile::~BackupEntryFromAppendOnlyFile() = default;
std::unique_ptr<SeekableReadBuffer> BackupEntryFromAppendOnlyFile::getReadBuffer() const
{
auto buf = BackupEntryFromImmutableFile::getReadBuffer();
return std::make_unique<LimitSeekableReadBuffer>(std::move(buf), 0, limit);
std::unique_ptr<SeekableReadBuffer> buf;
if (data_source_description.is_encrypted)
buf = disk->readEncryptedFile(file_path, settings);
else
buf = disk->readFile(file_path, settings);
return std::make_unique<LimitSeekableReadBuffer>(std::move(buf), 0, size);
}
}

View File

@ -1,6 +1,6 @@
#pragma once
#include <Backups/BackupEntryFromImmutableFile.h>
#include <Backups/BackupEntryWithChecksumCalculation.h>
namespace DB
@ -8,24 +8,34 @@ namespace DB
/// Represents a file prepared to be included in a backup, assuming that until this backup entry is destroyed
/// the file can be appended with new data, but the bytes which are already in the file won't be changed.
class BackupEntryFromAppendOnlyFile : public BackupEntryFromImmutableFile
class BackupEntryFromAppendOnlyFile : public BackupEntryWithChecksumCalculation<IBackupEntry>
{
public:
/// The constructor is allowed to not set `file_size_` or `checksum_`, in that case it will be calculated from the data.
/// The constructor is allowed to not set `file_size_`, in that case it will be calculated from the data.
BackupEntryFromAppendOnlyFile(
const DiskPtr & disk_,
const String & file_path_,
const ReadSettings & settings_,
const std::optional<UInt64> & file_size_ = {},
const std::optional<UInt128> & checksum_ = {});
const std::optional<UInt64> & file_size_ = {});
~BackupEntryFromAppendOnlyFile() override;
UInt64 getSize() const override { return limit; }
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override;
UInt64 getSize() const override { return size; }
bool isFromImmutableFile() const override { return false; }
DataSourceDescription getDataSourceDescription() const override { return data_source_description; }
bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; }
bool isFromFile() const override { return true; }
DiskPtr getDisk() const override { return disk; }
String getFilePath() const override { return file_path; }
private:
const UInt64 limit;
const DiskPtr disk;
const String file_path;
const DataSourceDescription data_source_description;
const ReadSettings settings;
const UInt64 size;
};
}

View File

@ -1,8 +1,6 @@
#include <Backups/BackupEntryFromImmutableFile.h>
#include <Disks/IDisk.h>
#include <Disks/IO/createReadBufferFromFileBase.h>
#include <Poco/File.h>
#include <Common/filesystemHelpers.h>
#include <Disks/DiskEncrypted.h>
namespace DB
@ -18,26 +16,13 @@ BackupEntryFromImmutableFile::BackupEntryFromImmutableFile(
, file_path(file_path_)
, data_source_description(disk->getDataSourceDescription())
, settings(settings_)
, file_size(data_source_description.is_encrypted ? std::optional<UInt64>{} : file_size_)
, checksum(data_source_description.is_encrypted ? std::optional<UInt128>{} : checksum_)
, file_size(file_size_)
, checksum(checksum_)
{
}
BackupEntryFromImmutableFile::~BackupEntryFromImmutableFile() = default;
UInt64 BackupEntryFromImmutableFile::getSize() const
{
std::lock_guard lock{get_file_size_mutex};
if (!file_size)
{
if (data_source_description.is_encrypted)
file_size = disk->getEncryptedFileSize(file_path);
else
file_size = disk->getFileSize(file_path);
}
return *file_size;
}
std::unique_ptr<SeekableReadBuffer> BackupEntryFromImmutableFile::getReadBuffer() const
{
if (data_source_description.is_encrypted)
@ -46,4 +31,43 @@ std::unique_ptr<SeekableReadBuffer> BackupEntryFromImmutableFile::getReadBuffer(
return disk->readFile(file_path, settings);
}
UInt64 BackupEntryFromImmutableFile::getSize() const
{
std::lock_guard lock{size_and_checksum_mutex};
if (!file_size_adjusted)
{
if (!file_size)
file_size = disk->getFileSize(file_path);
if (data_source_description.is_encrypted)
*file_size = DiskEncrypted::convertFileSizeToEncryptedFileSize(*file_size);
file_size_adjusted = true;
}
return *file_size;
}
UInt128 BackupEntryFromImmutableFile::getChecksum() const
{
std::lock_guard lock{size_and_checksum_mutex};
if (!checksum_adjusted)
{
/// TODO: We should not just ignore `checksum` if `data_source_description.is_encrypted == true`, we should use it somehow.
if (!checksum || data_source_description.is_encrypted)
checksum = BackupEntryWithChecksumCalculation<IBackupEntry>::getChecksum();
checksum_adjusted = true;
}
return *checksum;
}
std::optional<UInt128> BackupEntryFromImmutableFile::getPartialChecksum(size_t prefix_length) const
{
if (prefix_length == 0)
return 0;
if (prefix_length >= getSize())
return getChecksum();
/// For immutable files we don't use partial checksums.
return std::nullopt;
}
}

View File

@ -1,6 +1,6 @@
#pragma once
#include <Backups/IBackupEntry.h>
#include <Backups/BackupEntryWithChecksumCalculation.h>
#include <IO/ReadSettings.h>
#include <base/defines.h>
#include <mutex>
@ -11,7 +11,7 @@ class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;
/// Represents a file prepared to be included in a backup, assuming that until this backup entry is destroyed the file won't be changed.
class BackupEntryFromImmutableFile : public IBackupEntry
class BackupEntryFromImmutableFile : public BackupEntryWithChecksumCalculation<IBackupEntry>
{
public:
/// The constructor is allowed to not set `file_size_` or `checksum_`, in that case it will be calculated from the data.
@ -24,13 +24,14 @@ public:
~BackupEntryFromImmutableFile() override;
UInt64 getSize() const override;
std::optional<UInt128> getChecksum() const override { return checksum; }
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override;
bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; }
UInt64 getSize() const override;
UInt128 getChecksum() const override;
std::optional<UInt128> getPartialChecksum(size_t prefix_length) const override;
DataSourceDescription getDataSourceDescription() const override { return data_source_description; }
bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; }
bool isFromFile() const override { return true; }
bool isFromImmutableFile() const override { return true; }
@ -41,10 +42,12 @@ private:
const DiskPtr disk;
const String file_path;
const DataSourceDescription data_source_description;
ReadSettings settings;
mutable std::optional<UInt64> file_size TSA_GUARDED_BY(get_file_size_mutex);
mutable std::mutex get_file_size_mutex;
const std::optional<UInt128> checksum;
const ReadSettings settings;
mutable std::optional<UInt64> file_size;
mutable std::optional<UInt64> checksum;
mutable bool file_size_adjusted = false;
mutable bool checksum_adjusted = false;
mutable std::mutex size_and_checksum_mutex;
};
}

View File

@ -5,13 +5,12 @@
namespace DB
{
BackupEntryFromMemory::BackupEntryFromMemory(const void * data_, size_t size_, const std::optional<UInt128> & checksum_)
: BackupEntryFromMemory(String{reinterpret_cast<const char *>(data_), size_}, checksum_)
BackupEntryFromMemory::BackupEntryFromMemory(const void * data_, size_t size_)
: BackupEntryFromMemory(String{reinterpret_cast<const char *>(data_), size_})
{
}
BackupEntryFromMemory::BackupEntryFromMemory(String data_, const std::optional<UInt128> & checksum_)
: data(std::move(data_)), checksum(checksum_)
BackupEntryFromMemory::BackupEntryFromMemory(String data_) : data(std::move(data_))
{
}

View File

@ -1,32 +1,26 @@
#pragma once
#include <Backups/IBackupEntry.h>
#include <IO/ReadBufferFromString.h>
#include <Backups/BackupEntryWithChecksumCalculation.h>
namespace DB
{
/// Represents small preloaded data to be included in a backup.
class BackupEntryFromMemory : public IBackupEntry
class BackupEntryFromMemory : public BackupEntryWithChecksumCalculation<IBackupEntry>
{
public:
/// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data.
BackupEntryFromMemory(const void * data_, size_t size_, const std::optional<UInt128> & checksum_ = {});
explicit BackupEntryFromMemory(String data_, const std::optional<UInt128> & checksum_ = {});
BackupEntryFromMemory(const void * data_, size_t size_);
explicit BackupEntryFromMemory(String data_);
UInt64 getSize() const override { return data.size(); }
std::optional<UInt128> getChecksum() const override { return checksum; }
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override;
UInt64 getSize() const override { return data.size(); }
DataSourceDescription getDataSourceDescription() const override
{
return DataSourceDescription{DataSourceType::RAM, "", false, false};
}
DataSourceDescription getDataSourceDescription() const override { return DataSourceDescription{DataSourceType::RAM, "", false, false}; }
private:
const String data;
const std::optional<UInt128> checksum;
};
}

View File

@ -29,21 +29,18 @@ namespace
}
BackupEntryFromSmallFile::BackupEntryFromSmallFile(const String & file_path_, const std::optional<UInt128> & checksum_)
BackupEntryFromSmallFile::BackupEntryFromSmallFile(const String & file_path_)
: file_path(file_path_)
, data_source_description(DiskLocal::getLocalDataSourceDescription(file_path_))
, data(readFile(file_path_))
, checksum(checksum_)
{
}
BackupEntryFromSmallFile::BackupEntryFromSmallFile(
const DiskPtr & disk_, const String & file_path_, const std::optional<UInt128> & checksum_)
BackupEntryFromSmallFile::BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_)
: disk(disk_)
, file_path(file_path_)
, data_source_description(disk_->getDataSourceDescription())
, data(readFile(disk_, file_path, data_source_description.is_encrypted))
, checksum(data_source_description.is_encrypted ? std::optional<UInt128>{} : checksum_)
{
}

View File

@ -1,6 +1,6 @@
#pragma once
#include <Backups/IBackupEntry.h>
#include <Backups/BackupEntryWithChecksumCalculation.h>
namespace DB
@ -10,37 +10,27 @@ using DiskPtr = std::shared_ptr<IDisk>;
/// Represents a file prepared to be included in a backup,
/// assuming that the file is small and can be easily loaded into memory.
class BackupEntryFromSmallFile : public IBackupEntry
class BackupEntryFromSmallFile : public BackupEntryWithChecksumCalculation<IBackupEntry>
{
public:
/// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data.
explicit BackupEntryFromSmallFile(
const String & file_path_,
const std::optional<UInt128> & checksum_ = {});
explicit BackupEntryFromSmallFile(const String & file_path_);
BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_);
BackupEntryFromSmallFile(
const DiskPtr & disk_,
const String & file_path_,
const std::optional<UInt128> & checksum_ = {});
UInt64 getSize() const override { return data.size(); }
std::optional<UInt128> getChecksum() const override { return checksum; }
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override;
UInt64 getSize() const override { return data.size(); }
DataSourceDescription getDataSourceDescription() const override { return data_source_description; }
bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; }
bool isFromFile() const override { return true; }
DiskPtr getDisk() const override { return disk; }
String getFilePath() const override { return file_path; }
DataSourceDescription getDataSourceDescription() const override { return data_source_description; }
private:
const DiskPtr disk;
const String file_path;
const DataSourceDescription data_source_description;
const String data;
const std::optional<UInt128> checksum;
};
}

View File

@ -0,0 +1,48 @@
#include <Backups/BackupEntryWithChecksumCalculation.h>
#include <IO/HashingReadBuffer.h>
namespace DB
{
template <typename Base>
UInt128 BackupEntryWithChecksumCalculation<Base>::getChecksum() const
{
std::lock_guard lock{checksum_calculation_mutex};
if (!calculated_checksum)
{
auto read_buffer = this->getReadBuffer();
HashingReadBuffer hashing_read_buffer(*read_buffer);
hashing_read_buffer.ignoreAll();
calculated_checksum = hashing_read_buffer.getHash();
}
return *calculated_checksum;
}
template <typename Base>
std::optional<UInt128> BackupEntryWithChecksumCalculation<Base>::getPartialChecksum(size_t prefix_length) const
{
if (prefix_length == 0)
return 0;
if (prefix_length >= this->getSize())
return this->getChecksum();
auto read_buffer = this->getReadBuffer();
HashingReadBuffer hashing_read_buffer(*read_buffer);
hashing_read_buffer.ignore(prefix_length);
auto partial_checksum = hashing_read_buffer.getHash();
std::lock_guard lock{checksum_calculation_mutex};
if (!calculated_checksum)
{
hashing_read_buffer.ignoreAll();
calculated_checksum = hashing_read_buffer.getHash();
}
return partial_checksum;
}
template class BackupEntryWithChecksumCalculation<IBackupEntry>;
}

View File

@ -0,0 +1,22 @@
#pragma once
#include <Backups/IBackupEntry.h>
namespace DB
{
/// Calculates the checksum and the partial checksum for a backup entry based on ReadBuffer returned by getReadBuffer().
template <typename Base>
class BackupEntryWithChecksumCalculation : public Base
{
public:
UInt128 getChecksum() const override;
std::optional<UInt128> getPartialChecksum(size_t prefix_length) const override;
private:
mutable std::optional<UInt128> calculated_checksum;
mutable std::mutex checksum_calculation_mutex;
};
}

View File

@ -15,15 +15,16 @@ public:
BackupEntryWrappedWith(BackupEntryPtr entry_, T && custom_value_) : entry(entry_), custom_value(std::move(custom_value_)) { }
~BackupEntryWrappedWith() override = default;
UInt64 getSize() const override { return entry->getSize(); }
std::optional<UInt128> getChecksum() const override { return entry->getChecksum(); }
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override { return entry->getReadBuffer(); }
UInt64 getSize() const override { return entry->getSize(); }
UInt128 getChecksum() const override { return entry->getChecksum(); }
std::optional<UInt128> getPartialChecksum(size_t prefix_length) const override { return entry->getPartialChecksum(prefix_length); }
DataSourceDescription getDataSourceDescription() const override { return entry->getDataSourceDescription(); }
bool isEncryptedByDisk() const override { return entry->isEncryptedByDisk(); }
bool isFromFile() const override { return entry->isFromFile(); }
bool isFromImmutableFile() const override { return entry->isFromImmutableFile(); }
String getFilePath() const override { return entry->getFilePath(); }
DiskPtr getDisk() const override { return entry->getDisk(); }
DataSourceDescription getDataSourceDescription() const override { return entry->getDataSourceDescription(); }
private:
BackupEntryPtr entry;

View File

@ -7,7 +7,7 @@
#include <Common/scope_guard_safe.h>
#include <Common/setThreadName.h>
#include <Common/ThreadPool.h>
#include <IO/HashingReadBuffer.h>
#include <base/hex.h>
namespace DB
@ -49,44 +49,19 @@ namespace
struct ChecksumsForNewEntry
{
UInt128 full_checksum;
UInt128 prefix_checksum;
std::optional<UInt128> prefix_checksum;
};
/// Calculate checksum for backup entry if it's empty.
/// Also able to calculate additional checksum of some prefix.
ChecksumsForNewEntry calculateNewEntryChecksumsIfNeeded(const BackupEntryPtr & entry, size_t prefix_size)
{
ChecksumsForNewEntry res;
/// The partial checksum should be calculated before the full checksum to enable optimization in BackupEntryWithChecksumCalculation.
if (prefix_size > 0)
{
auto read_buffer = entry->getReadBuffer();
HashingReadBuffer hashing_read_buffer(*read_buffer);
hashing_read_buffer.ignore(prefix_size);
auto prefix_checksum = hashing_read_buffer.getHash();
if (entry->getChecksum() == std::nullopt)
{
hashing_read_buffer.ignoreAll();
auto full_checksum = hashing_read_buffer.getHash();
return ChecksumsForNewEntry{full_checksum, prefix_checksum};
}
else
{
return ChecksumsForNewEntry{*(entry->getChecksum()), prefix_checksum};
}
}
else
{
if (entry->getChecksum() == std::nullopt)
{
auto read_buffer = entry->getReadBuffer();
HashingReadBuffer hashing_read_buffer(*read_buffer);
hashing_read_buffer.ignoreAll();
return ChecksumsForNewEntry{hashing_read_buffer.getHash(), 0};
}
else
{
return ChecksumsForNewEntry{*(entry->getChecksum()), 0};
}
}
res.prefix_checksum = entry->getPartialChecksum(prefix_size);
res.full_checksum = entry->getChecksum();
return res;
}
/// We store entries' file names in the backup without leading slashes.

View File

@ -1,10 +1,7 @@
#include <Backups/BackupIO_File.h>
#include <Disks/DiskLocal.h>
//#include <Disks/IDisk.h>
#include <Disks/IO/createReadBufferFromFileBase.h>
#include <IO/WriteBufferFromFile.h>
//#include <IO/copyData.h>
//#include <Common/filesystemHelpers.h>
#include <Common/logger_useful.h>

View File

@ -17,11 +17,12 @@ class IBackupEntriesLazyBatch::BackupEntryFromBatch : public IBackupEntry
public:
BackupEntryFromBatch(const std::shared_ptr<IBackupEntriesLazyBatch> & batch_, size_t index_) : batch(batch_), index(index_) { }
UInt64 getSize() const override { return getInternalBackupEntry()->getSize(); }
std::optional<UInt128> getChecksum() const override { return getInternalBackupEntry()->getChecksum(); }
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override { return getInternalBackupEntry()->getReadBuffer(); }
bool isEncryptedByDisk() const override { return getInternalBackupEntry()->isEncryptedByDisk(); }
UInt64 getSize() const override { return getInternalBackupEntry()->getSize(); }
UInt128 getChecksum() const override { return getInternalBackupEntry()->getChecksum(); }
std::optional<UInt128> getPartialChecksum(size_t prefix_length) const override { return getInternalBackupEntry()->getPartialChecksum(prefix_length); }
DataSourceDescription getDataSourceDescription() const override { return getInternalBackupEntry()->getDataSourceDescription(); }
bool isEncryptedByDisk() const override { return getInternalBackupEntry()->isEncryptedByDisk(); }
bool isFromFile() const override { return getInternalBackupEntry()->isFromFile(); }
bool isFromImmutableFile() const override { return getInternalBackupEntry()->isFromImmutableFile(); }
String getFilePath() const override { return getInternalBackupEntry()->getFilePath(); }

View File

@ -20,9 +20,12 @@ public:
/// Returns the size of the data.
virtual UInt64 getSize() const = 0;
/// Returns the checksum of the data if it's precalculated.
/// Can return nullopt which means the checksum should be calculated from the read buffer.
virtual std::optional<UInt128> getChecksum() const { return {}; }
/// Returns the checksum of the data.
virtual UInt128 getChecksum() const = 0;
/// Returns a partial checksum, i.e. the checksum calculated for a prefix part of the data.
/// Can return nullopt if the partial checksum is too difficult to calculate.
virtual std::optional<UInt128> getPartialChecksum(size_t /* prefix_length */) const { return {}; }
/// Returns a read buffer for reading the data.
virtual std::unique_ptr<SeekableReadBuffer> getReadBuffer() const = 0;

View File

@ -348,6 +348,13 @@ size_t DiskEncrypted::getFileSize(const String & path) const
return size > FileEncryption::Header::kSize ? (size - FileEncryption::Header::kSize) : 0;
}
size_t DiskEncrypted::convertFileSizeToEncryptedFileSize(size_t file_size)
{
if (file_size)
return file_size + FileEncryption::Header::kSize;
return 0;
}
void DiskEncrypted::truncateFile(const String & path, size_t size)
{
auto wrapped_path = wrappedPath(path);

View File

@ -223,6 +223,8 @@ public:
return delegate->getFileSize(wrapped_path);
}
static size_t convertFileSizeToEncryptedFileSize(size_t file_size);
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override
{
auto wrapped_path = wrappedPath(path);

View File

@ -9278,10 +9278,9 @@ void StorageReplicatedMergeTree::backupData(
auto & hash = part_names_with_hashes_calculating[part_name];
if (relative_path.ends_with(".bin"))
{
auto checksum = backup_entry->getChecksum();
hash.update(relative_path);
hash.update(backup_entry->getSize());
hash.update(*checksum);
hash.update(backup_entry->getChecksum());
}
continue;
}