mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Add backup setting "decrypt_files_from_encrypted_disks"
This commit is contained in:
parent
019493efa3
commit
943707963f
@ -1,5 +1,5 @@
|
||||
#include <Backups/BackupEntryFromAppendOnlyFile.h>
|
||||
#include <Disks/DiskEncrypted.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <IO/LimitSeekableReadBuffer.h>
|
||||
|
||||
|
||||
@ -9,11 +9,11 @@ namespace DB
|
||||
namespace
|
||||
{
|
||||
/// For append-only files we must calculate its size on the construction of a backup entry.
|
||||
UInt64 calculateSize(const DiskPtr & disk, const String & file_path, bool is_encrypted, std::optional<UInt64> unencrypted_file_size)
|
||||
UInt64 calculateSize(const DiskPtr & disk, const String & file_path, bool copy_encrypted, std::optional<UInt64> unencrypted_file_size)
|
||||
{
|
||||
if (!unencrypted_file_size)
|
||||
return is_encrypted ? disk->getEncryptedFileSize(file_path) : disk->getFileSize(file_path);
|
||||
else if (is_encrypted)
|
||||
return copy_encrypted ? disk->getEncryptedFileSize(file_path) : disk->getFileSize(file_path);
|
||||
else if (copy_encrypted)
|
||||
return disk->getEncryptedFileSize(*unencrypted_file_size);
|
||||
else
|
||||
return *unencrypted_file_size;
|
||||
@ -21,11 +21,12 @@ namespace
|
||||
}
|
||||
|
||||
BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile(
|
||||
const DiskPtr & disk_, const String & file_path_, const std::optional<UInt64> & file_size_)
|
||||
const DiskPtr & disk_, const String & file_path_, bool copy_encrypted_, const std::optional<UInt64> & file_size_)
|
||||
: disk(disk_)
|
||||
, file_path(file_path_)
|
||||
, data_source_description(disk->getDataSourceDescription())
|
||||
, size(calculateSize(disk_, file_path_, data_source_description.is_encrypted, file_size_))
|
||||
, copy_encrypted(copy_encrypted_ && data_source_description.is_encrypted)
|
||||
, size(calculateSize(disk_, file_path_, copy_encrypted, file_size_))
|
||||
{
|
||||
}
|
||||
|
||||
@ -34,7 +35,7 @@ BackupEntryFromAppendOnlyFile::~BackupEntryFromAppendOnlyFile() = default;
|
||||
std::unique_ptr<SeekableReadBuffer> BackupEntryFromAppendOnlyFile::getReadBuffer(const ReadSettings & read_settings) const
|
||||
{
|
||||
std::unique_ptr<SeekableReadBuffer> buf;
|
||||
if (data_source_description.is_encrypted)
|
||||
if (copy_encrypted)
|
||||
buf = disk->readEncryptedFile(file_path, read_settings.adjustBufferSize(size));
|
||||
else
|
||||
buf = disk->readFile(file_path, read_settings.adjustBufferSize(size));
|
||||
|
@ -15,6 +15,7 @@ public:
|
||||
BackupEntryFromAppendOnlyFile(
|
||||
const DiskPtr & disk_,
|
||||
const String & file_path_,
|
||||
bool copy_encrypted_ = false,
|
||||
const std::optional<UInt64> & file_size_ = {});
|
||||
|
||||
~BackupEntryFromAppendOnlyFile() override;
|
||||
@ -23,7 +24,7 @@ public:
|
||||
UInt64 getSize() const override { return size; }
|
||||
|
||||
DataSourceDescription getDataSourceDescription() const override { return data_source_description; }
|
||||
bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; }
|
||||
bool isEncryptedByDisk() const override { return copy_encrypted; }
|
||||
|
||||
bool isFromFile() const override { return true; }
|
||||
DiskPtr getDisk() const override { return disk; }
|
||||
@ -33,6 +34,7 @@ private:
|
||||
const DiskPtr disk;
|
||||
const String file_path;
|
||||
const DataSourceDescription data_source_description;
|
||||
const bool copy_encrypted;
|
||||
const UInt64 size;
|
||||
};
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <Backups/BackupEntryFromImmutableFile.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/DiskEncrypted.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -20,11 +19,13 @@ namespace
|
||||
BackupEntryFromImmutableFile::BackupEntryFromImmutableFile(
|
||||
const DiskPtr & disk_,
|
||||
const String & file_path_,
|
||||
bool copy_encrypted_,
|
||||
const std::optional<UInt64> & file_size_,
|
||||
const std::optional<UInt128> & checksum_)
|
||||
: disk(disk_)
|
||||
, file_path(file_path_)
|
||||
, data_source_description(disk->getDataSourceDescription())
|
||||
, copy_encrypted(copy_encrypted_ && data_source_description.is_encrypted)
|
||||
, file_size(file_size_)
|
||||
, checksum(checksum_)
|
||||
{
|
||||
@ -34,7 +35,7 @@ BackupEntryFromImmutableFile::~BackupEntryFromImmutableFile() = default;
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> BackupEntryFromImmutableFile::getReadBuffer(const ReadSettings & read_settings) const
|
||||
{
|
||||
if (data_source_description.is_encrypted)
|
||||
if (copy_encrypted)
|
||||
return disk->readEncryptedFile(file_path, read_settings);
|
||||
else
|
||||
return disk->readFile(file_path, read_settings);
|
||||
@ -46,8 +47,8 @@ UInt64 BackupEntryFromImmutableFile::getSize() const
|
||||
if (!file_size_adjusted)
|
||||
{
|
||||
if (!file_size)
|
||||
file_size = data_source_description.is_encrypted ? disk->getEncryptedFileSize(file_path) : disk->getFileSize(file_path);
|
||||
else if (data_source_description.is_encrypted)
|
||||
file_size = copy_encrypted ? disk->getEncryptedFileSize(file_path) : disk->getFileSize(file_path);
|
||||
else if (copy_encrypted)
|
||||
file_size = disk->getEncryptedFileSize(*file_size);
|
||||
file_size_adjusted = true;
|
||||
}
|
||||
@ -61,7 +62,7 @@ UInt128 BackupEntryFromImmutableFile::getChecksum() const
|
||||
{
|
||||
if (!checksum)
|
||||
checksum = BackupEntryWithChecksumCalculation<IBackupEntry>::getChecksum();
|
||||
else if (data_source_description.is_encrypted)
|
||||
else if (copy_encrypted)
|
||||
checksum = combineChecksums(*checksum, disk->getEncryptedFileIV(file_path));
|
||||
checksum_adjusted = true;
|
||||
}
|
||||
|
@ -18,6 +18,7 @@ public:
|
||||
BackupEntryFromImmutableFile(
|
||||
const DiskPtr & disk_,
|
||||
const String & file_path_,
|
||||
bool copy_encrypted_ = false,
|
||||
const std::optional<UInt64> & file_size_ = {},
|
||||
const std::optional<UInt128> & checksum_ = {});
|
||||
|
||||
@ -30,7 +31,7 @@ public:
|
||||
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 isEncryptedByDisk() const override { return copy_encrypted; }
|
||||
|
||||
bool isFromFile() const override { return true; }
|
||||
bool isFromImmutableFile() const override { return true; }
|
||||
@ -41,6 +42,7 @@ private:
|
||||
const DiskPtr disk;
|
||||
const String file_path;
|
||||
const DataSourceDescription data_source_description;
|
||||
const bool copy_encrypted;
|
||||
mutable std::optional<UInt64> file_size;
|
||||
mutable std::optional<UInt64> checksum;
|
||||
mutable bool file_size_adjusted = false;
|
||||
|
@ -19,9 +19,9 @@ namespace
|
||||
return s;
|
||||
}
|
||||
|
||||
String readFile(const DiskPtr & disk, const String & file_path, bool read_encrypted)
|
||||
String readFile(const DiskPtr & disk, const String & file_path, bool copy_encrypted)
|
||||
{
|
||||
auto buf = read_encrypted ? disk->readEncryptedFile(file_path) : disk->readFile(file_path);
|
||||
auto buf = copy_encrypted ? disk->readEncryptedFile(file_path) : disk->readFile(file_path);
|
||||
String s;
|
||||
readStringUntilEOF(s, *buf);
|
||||
return s;
|
||||
@ -36,11 +36,12 @@ BackupEntryFromSmallFile::BackupEntryFromSmallFile(const String & file_path_)
|
||||
{
|
||||
}
|
||||
|
||||
BackupEntryFromSmallFile::BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_)
|
||||
BackupEntryFromSmallFile::BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_, bool copy_encrypted_)
|
||||
: disk(disk_)
|
||||
, file_path(file_path_)
|
||||
, data_source_description(disk_->getDataSourceDescription())
|
||||
, data(readFile(disk_, file_path, data_source_description.is_encrypted))
|
||||
, copy_encrypted(copy_encrypted_ && data_source_description.is_encrypted)
|
||||
, data(readFile(disk_, file_path, copy_encrypted))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -14,13 +14,13 @@ class BackupEntryFromSmallFile : public BackupEntryWithChecksumCalculation<IBack
|
||||
{
|
||||
public:
|
||||
explicit BackupEntryFromSmallFile(const String & file_path_);
|
||||
BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_);
|
||||
BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_, bool copy_encrypted_ = false);
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> getReadBuffer(const ReadSettings &) 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 isEncryptedByDisk() const override { return copy_encrypted; }
|
||||
|
||||
bool isFromFile() const override { return true; }
|
||||
DiskPtr getDisk() const override { return disk; }
|
||||
@ -30,6 +30,7 @@ private:
|
||||
const DiskPtr disk;
|
||||
const String file_path;
|
||||
const DataSourceDescription data_source_description;
|
||||
const bool copy_encrypted = false;
|
||||
const String data;
|
||||
};
|
||||
|
||||
|
@ -25,15 +25,21 @@ std::optional<UInt128> BackupEntryWithChecksumCalculation<Base>::getPartialCheck
|
||||
if (prefix_length == 0)
|
||||
return 0;
|
||||
|
||||
if (prefix_length >= this->getSize())
|
||||
size_t size = this->getSize();
|
||||
if (prefix_length >= size)
|
||||
return this->getChecksum();
|
||||
|
||||
auto read_buffer = this->getReadBuffer(ReadSettings{}.adjustBufferSize(prefix_length));
|
||||
std::lock_guard lock{checksum_calculation_mutex};
|
||||
|
||||
ReadSettings read_settings;
|
||||
if (calculated_checksum)
|
||||
read_settings.adjustBufferSize(calculated_checksum ? prefix_length : size);
|
||||
|
||||
auto read_buffer = this->getReadBuffer(read_settings);
|
||||
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();
|
||||
|
@ -82,12 +82,12 @@ void BackupWriterDefault::copyFileFromDisk(const String & path_in_backup, DiskPt
|
||||
{
|
||||
LOG_TRACE(log, "Copying file {} from disk {} through buffers", src_path, src_disk->getName());
|
||||
|
||||
auto create_read_buffer = [this, src_disk, src_path, file_size = start_pos + length, copy_encrypted]
|
||||
auto create_read_buffer = [src_disk, src_path, copy_encrypted, settings = read_settings.adjustBufferSize(start_pos + length)]
|
||||
{
|
||||
if (copy_encrypted)
|
||||
return src_disk->readEncryptedFile(src_path, read_settings, {}, file_size);
|
||||
return src_disk->readEncryptedFile(src_path, settings);
|
||||
else
|
||||
return src_disk->readFile(src_path, read_settings, {}, file_size);
|
||||
return src_disk->readFile(src_path, settings);
|
||||
};
|
||||
|
||||
copyDataToFile(path_in_backup, create_read_buffer, start_pos, length);
|
||||
|
@ -23,6 +23,7 @@ namespace ErrorCodes
|
||||
M(String, password) \
|
||||
M(Bool, structure_only) \
|
||||
M(Bool, async) \
|
||||
M(Bool, decrypt_files_from_encrypted_disks) \
|
||||
M(Bool, deduplicate_files) \
|
||||
M(UInt64, shard_num) \
|
||||
M(UInt64, replica_num) \
|
||||
|
@ -32,6 +32,9 @@ struct BackupSettings
|
||||
/// Whether the BACKUP command must return immediately without waiting until the backup has completed.
|
||||
bool async = false;
|
||||
|
||||
/// Whether the BACKUP command should decrypt files stored on encrypted disks.
|
||||
bool decrypt_files_from_encrypted_disks = false;
|
||||
|
||||
/// Whether the BACKUP will omit similar files (within one backup only).
|
||||
bool deduplicate_files = true;
|
||||
|
||||
|
@ -253,14 +253,14 @@ public:
|
||||
virtual void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0;
|
||||
|
||||
/// Reads a file from an encrypted disk without decrypting it (only for encrypted disks).
|
||||
virtual std::unique_ptr<ReadBufferFromFileBase> readEncryptedFile(
|
||||
virtual std::unique_ptr<ReadBufferFromFileBase> readEncryptedFile( /// NOLINT
|
||||
const String & path,
|
||||
const ReadSettings & settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const;
|
||||
|
||||
/// Writes an already encrypted file to the disk (only for encrypted disks).
|
||||
virtual std::unique_ptr<WriteBufferFromFileBase> writeEncryptedFile(
|
||||
virtual std::unique_ptr<WriteBufferFromFileBase> writeEncryptedFile( /// NOLINT
|
||||
const String & path,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
WriteMode mode = WriteMode::Rewrite,
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Backups/BackupEntryFromSmallFile.h>
|
||||
#include <Backups/BackupEntryFromImmutableFile.h>
|
||||
#include <Backups/BackupEntryWrappedWith.h>
|
||||
#include <Backups/BackupSettings.h>
|
||||
#include <Disks/SingleDiskVolume.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
|
||||
@ -322,8 +323,9 @@ void DataPartStorageOnDiskBase::backup(
|
||||
const MergeTreeDataPartChecksums & checksums,
|
||||
const NameSet & files_without_checksums,
|
||||
const String & path_in_backup,
|
||||
BackupEntries & backup_entries,
|
||||
const BackupSettings & backup_settings,
|
||||
bool make_temporary_hard_links,
|
||||
BackupEntries & backup_entries,
|
||||
TemporaryFilesOnDisks * temp_dirs) const
|
||||
{
|
||||
fs::path part_path_on_disk = fs::path{root_path} / part_dir;
|
||||
@ -364,6 +366,8 @@ void DataPartStorageOnDiskBase::backup(
|
||||
|
||||
files_to_backup = getActualFileNamesOnDisk(files_to_backup);
|
||||
|
||||
bool copy_encrypted = !backup_settings.decrypt_files_from_encrypted_disks;
|
||||
|
||||
for (const auto & filepath : files_to_backup)
|
||||
{
|
||||
auto filepath_on_disk = part_path_on_disk / filepath;
|
||||
@ -371,7 +375,7 @@ void DataPartStorageOnDiskBase::backup(
|
||||
|
||||
if (files_without_checksums.contains(filepath))
|
||||
{
|
||||
backup_entries.emplace_back(filepath_in_backup, std::make_unique<BackupEntryFromSmallFile>(disk, filepath_on_disk));
|
||||
backup_entries.emplace_back(filepath_in_backup, std::make_unique<BackupEntryFromSmallFile>(disk, filepath_on_disk, copy_encrypted));
|
||||
continue;
|
||||
}
|
||||
|
||||
@ -392,7 +396,7 @@ void DataPartStorageOnDiskBase::backup(
|
||||
file_hash = {it->second.file_hash.first, it->second.file_hash.second};
|
||||
}
|
||||
|
||||
BackupEntryPtr backup_entry = std::make_unique<BackupEntryFromImmutableFile>(disk, filepath_on_disk, file_size, file_hash);
|
||||
BackupEntryPtr backup_entry = std::make_unique<BackupEntryFromImmutableFile>(disk, filepath_on_disk, copy_encrypted, file_size, file_hash);
|
||||
|
||||
if (temp_dir_owner)
|
||||
backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner);
|
||||
|
@ -53,8 +53,9 @@ public:
|
||||
const MergeTreeDataPartChecksums & checksums,
|
||||
const NameSet & files_without_checksums,
|
||||
const String & path_in_backup,
|
||||
BackupEntries & backup_entries,
|
||||
const BackupSettings & backup_settings,
|
||||
bool make_temporary_hard_links,
|
||||
BackupEntries & backup_entries,
|
||||
TemporaryFilesOnDisks * temp_dirs) const override;
|
||||
|
||||
MutableDataPartStoragePtr freeze(
|
||||
|
@ -65,6 +65,7 @@ using SyncGuardPtr = std::unique_ptr<ISyncGuard>;
|
||||
class IBackupEntry;
|
||||
using BackupEntryPtr = std::shared_ptr<const IBackupEntry>;
|
||||
using BackupEntries = std::vector<std::pair<String, BackupEntryPtr>>;
|
||||
struct BackupSettings;
|
||||
|
||||
struct WriteSettings;
|
||||
|
||||
@ -200,8 +201,9 @@ public:
|
||||
const MergeTreeDataPartChecksums & checksums,
|
||||
const NameSet & files_without_checksums,
|
||||
const String & path_in_backup,
|
||||
BackupEntries & backup_entries,
|
||||
const BackupSettings & backup_settings,
|
||||
bool make_temporary_hard_links,
|
||||
BackupEntries & backup_entries,
|
||||
TemporaryFilesOnDisks * temp_dirs) const = 0;
|
||||
|
||||
/// Creates hardlinks into 'to/dir_path' for every file in data part.
|
||||
|
@ -5051,7 +5051,11 @@ Pipe MergeTreeData::alterPartition(
|
||||
}
|
||||
|
||||
|
||||
BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup, const ContextPtr & local_context)
|
||||
BackupEntries MergeTreeData::backupParts(
|
||||
const DataPartsVector & data_parts,
|
||||
const String & data_path_in_backup,
|
||||
const BackupSettings & backup_settings,
|
||||
const ContextPtr & local_context)
|
||||
{
|
||||
BackupEntries backup_entries;
|
||||
std::map<DiskPtr, std::shared_ptr<TemporaryFileOnDisk>> temp_dirs;
|
||||
@ -5089,8 +5093,9 @@ BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, con
|
||||
part->checksums,
|
||||
part->getFileNamesWithoutChecksums(),
|
||||
data_path_in_backup,
|
||||
backup_entries_from_part,
|
||||
backup_settings,
|
||||
make_temporary_hard_links,
|
||||
backup_entries_from_part,
|
||||
&temp_dirs);
|
||||
|
||||
auto projection_parts = part->getProjectionParts();
|
||||
@ -5100,8 +5105,9 @@ BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, con
|
||||
projection_part->checksums,
|
||||
projection_part->getFileNamesWithoutChecksums(),
|
||||
fs::path{data_path_in_backup} / part->name,
|
||||
backup_entries_from_part,
|
||||
backup_settings,
|
||||
make_temporary_hard_links,
|
||||
backup_entries_from_part,
|
||||
&temp_dirs);
|
||||
}
|
||||
|
||||
|
@ -1322,7 +1322,7 @@ protected:
|
||||
MovePartsOutcome movePartsToSpace(const DataPartsVector & parts, SpacePtr space);
|
||||
|
||||
/// Makes backup entries to backup the parts of this table.
|
||||
BackupEntries backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup, const ContextPtr & local_context);
|
||||
BackupEntries backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup, const BackupSettings & backup_settings, const ContextPtr & local_context);
|
||||
|
||||
class RestoredPartsHolder;
|
||||
|
||||
|
@ -944,6 +944,8 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c
|
||||
fs::path temp_dir = temp_dir_owner->getPath();
|
||||
disk->createDirectories(temp_dir);
|
||||
|
||||
bool copy_encrypted = !backup_entries_collector.getBackupSettings().decrypt_files_from_encrypted_disks;
|
||||
|
||||
/// *.bin
|
||||
for (const auto & data_file : data_files)
|
||||
{
|
||||
@ -952,7 +954,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c
|
||||
String hardlink_file_path = temp_dir / data_file_name;
|
||||
disk->createHardLink(data_file.path, hardlink_file_path);
|
||||
BackupEntryPtr backup_entry = std::make_unique<BackupEntryFromAppendOnlyFile>(
|
||||
disk, hardlink_file_path, file_checker.getFileSize(data_file.path));
|
||||
disk, hardlink_file_path, copy_encrypted, file_checker.getFileSize(data_file.path));
|
||||
backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner);
|
||||
backup_entries_collector.addBackupEntry(data_path_in_backup_fs / data_file_name, std::move(backup_entry));
|
||||
}
|
||||
@ -965,7 +967,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c
|
||||
String hardlink_file_path = temp_dir / marks_file_name;
|
||||
disk->createHardLink(marks_file_path, hardlink_file_path);
|
||||
BackupEntryPtr backup_entry = std::make_unique<BackupEntryFromAppendOnlyFile>(
|
||||
disk, hardlink_file_path, file_checker.getFileSize(marks_file_path));
|
||||
disk, hardlink_file_path, copy_encrypted, file_checker.getFileSize(marks_file_path));
|
||||
backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner);
|
||||
backup_entries_collector.addBackupEntry(data_path_in_backup_fs / marks_file_name, std::move(backup_entry));
|
||||
}
|
||||
@ -973,7 +975,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c
|
||||
/// sizes.json
|
||||
String files_info_path = file_checker.getPath();
|
||||
backup_entries_collector.addBackupEntry(
|
||||
data_path_in_backup_fs / fileName(files_info_path), std::make_unique<BackupEntryFromSmallFile>(disk, files_info_path));
|
||||
data_path_in_backup_fs / fileName(files_info_path), std::make_unique<BackupEntryFromSmallFile>(disk, files_info_path, copy_encrypted));
|
||||
|
||||
/// columns.txt
|
||||
backup_entries_collector.addBackupEntry(
|
||||
|
@ -23,7 +23,7 @@
|
||||
#include <Compression/CompressedReadBufferFromFile.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <Backups/BackupEntriesCollector.h>
|
||||
#include <Backups/BackupEntryFromImmutableFile.h>
|
||||
#include <Backups/BackupEntryFromAppendOnlyFile.h>
|
||||
#include <Backups/BackupEntryFromMemory.h>
|
||||
#include <Backups/BackupEntryFromSmallFile.h>
|
||||
#include <Backups/IBackup.h>
|
||||
@ -319,21 +319,25 @@ namespace
|
||||
IndexForNativeFormat index;
|
||||
{
|
||||
auto data_file_path = temp_dir / fs::path{file_paths[data_bin_pos]}.filename();
|
||||
auto data_out_compressed = temp_disk->writeFile(data_file_path);
|
||||
CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size};
|
||||
NativeWriter block_out{data_out, 0, metadata_snapshot->getSampleBlock(), false, &index};
|
||||
for (const auto & block : *blocks)
|
||||
block_out.write(block);
|
||||
backup_entries[data_bin_pos] = {file_paths[data_bin_pos], std::make_shared<BackupEntryFromImmutableFile>(temp_disk, data_file_path)};
|
||||
{
|
||||
auto data_out_compressed = temp_disk->writeFile(data_file_path);
|
||||
CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size};
|
||||
NativeWriter block_out{data_out, 0, metadata_snapshot->getSampleBlock(), false, &index};
|
||||
for (const auto & block : *blocks)
|
||||
block_out.write(block);
|
||||
}
|
||||
backup_entries[data_bin_pos] = {file_paths[data_bin_pos], std::make_shared<BackupEntryFromAppendOnlyFile>(temp_disk, data_file_path)};
|
||||
}
|
||||
|
||||
/// Writing index.mrk
|
||||
{
|
||||
auto index_mrk_path = temp_dir / fs::path{file_paths[index_mrk_pos]}.filename();
|
||||
auto index_mrk_out_compressed = temp_disk->writeFile(index_mrk_path);
|
||||
CompressedWriteBuffer index_mrk_out{*index_mrk_out_compressed};
|
||||
index.write(index_mrk_out);
|
||||
backup_entries[index_mrk_pos] = {file_paths[index_mrk_pos], std::make_shared<BackupEntryFromImmutableFile>(temp_disk, index_mrk_path)};
|
||||
{
|
||||
auto index_mrk_out_compressed = temp_disk->writeFile(index_mrk_path);
|
||||
CompressedWriteBuffer index_mrk_out{*index_mrk_out_compressed};
|
||||
index.write(index_mrk_out);
|
||||
}
|
||||
backup_entries[index_mrk_pos] = {file_paths[index_mrk_pos], std::make_shared<BackupEntryFromAppendOnlyFile>(temp_disk, index_mrk_path)};
|
||||
}
|
||||
|
||||
/// Writing columns.txt
|
||||
|
@ -2142,6 +2142,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_
|
||||
|
||||
void StorageMergeTree::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
{
|
||||
const auto & backup_settings = backup_entries_collector.getBackupSettings();
|
||||
auto local_context = backup_entries_collector.getContext();
|
||||
|
||||
DataPartsVector data_parts;
|
||||
@ -2154,7 +2155,7 @@ void StorageMergeTree::backupData(BackupEntriesCollector & backup_entries_collec
|
||||
for (const auto & data_part : data_parts)
|
||||
min_data_version = std::min(min_data_version, data_part->info.getDataVersion() + 1);
|
||||
|
||||
backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup, local_context));
|
||||
backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup, backup_settings, local_context));
|
||||
backup_entries_collector.addBackupEntries(backupMutations(min_data_version, data_path_in_backup));
|
||||
}
|
||||
|
||||
|
@ -9252,6 +9252,8 @@ 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).
|
||||
|
||||
const auto & backup_settings = backup_entries_collector.getBackupSettings();
|
||||
auto local_context = backup_entries_collector.getContext();
|
||||
|
||||
DataPartsVector data_parts;
|
||||
@ -9260,7 +9262,7 @@ void StorageReplicatedMergeTree::backupData(
|
||||
else
|
||||
data_parts = getVisibleDataPartsVector(local_context);
|
||||
|
||||
auto backup_entries = backupParts(data_parts, /* data_path_in_backup */ "", local_context);
|
||||
auto backup_entries = backupParts(data_parts, /* data_path_in_backup */ "", backup_settings, local_context);
|
||||
|
||||
auto coordination = backup_entries_collector.getBackupCoordination();
|
||||
String shared_id = getTableSharedID();
|
||||
|
@ -545,6 +545,8 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec
|
||||
fs::path temp_dir = temp_dir_owner->getPath();
|
||||
disk->createDirectories(temp_dir);
|
||||
|
||||
bool copy_encrypted = !backup_entries_collector.getBackupSettings().decrypt_files_from_encrypted_disks;
|
||||
|
||||
/// data.bin
|
||||
{
|
||||
/// We make a copy of the data file because it can be changed later in write() or in truncate().
|
||||
@ -552,7 +554,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec
|
||||
String hardlink_file_path = temp_dir / data_file_name;
|
||||
disk->createHardLink(data_file_path, hardlink_file_path);
|
||||
BackupEntryPtr backup_entry = std::make_unique<BackupEntryFromAppendOnlyFile>(
|
||||
disk, hardlink_file_path, file_checker.getFileSize(data_file_path));
|
||||
disk, hardlink_file_path, copy_encrypted, file_checker.getFileSize(data_file_path));
|
||||
backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner);
|
||||
backup_entries_collector.addBackupEntry(data_path_in_backup_fs / data_file_name, std::move(backup_entry));
|
||||
}
|
||||
@ -564,7 +566,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec
|
||||
String hardlink_file_path = temp_dir / index_file_name;
|
||||
disk->createHardLink(index_file_path, hardlink_file_path);
|
||||
BackupEntryPtr backup_entry = std::make_unique<BackupEntryFromAppendOnlyFile>(
|
||||
disk, hardlink_file_path, file_checker.getFileSize(index_file_path));
|
||||
disk, hardlink_file_path, copy_encrypted, file_checker.getFileSize(index_file_path));
|
||||
backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner);
|
||||
backup_entries_collector.addBackupEntry(data_path_in_backup_fs / index_file_name, std::move(backup_entry));
|
||||
}
|
||||
@ -572,7 +574,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec
|
||||
/// sizes.json
|
||||
String files_info_path = file_checker.getPath();
|
||||
backup_entries_collector.addBackupEntry(
|
||||
data_path_in_backup_fs / fileName(files_info_path), std::make_unique<BackupEntryFromSmallFile>(disk, files_info_path));
|
||||
data_path_in_backup_fs / fileName(files_info_path), std::make_unique<BackupEntryFromSmallFile>(disk, files_info_path, copy_encrypted));
|
||||
|
||||
/// columns.txt
|
||||
backup_entries_collector.addBackupEntry(
|
||||
|
@ -309,17 +309,24 @@ def test_restart():
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
"storage_policy,backup_type,storage_policy2",
|
||||
"backup_type,old_storage_policy,new_storage_policy,decrypt_files_from_encrypted_disks",
|
||||
[
|
||||
("encrypted_policy", "S3", "encrypted_policy"),
|
||||
("encrypted_policy", "S3", "s3_encrypted_default_path"),
|
||||
("s3_encrypted_default_path", "S3", "s3_encrypted_default_path"),
|
||||
("s3_encrypted_default_path", "S3", "encrypted_policy"),
|
||||
("s3_encrypted_default_path", "File", "encrypted_policy"),
|
||||
("local_policy", "File", "encrypted_policy"),
|
||||
("S3", "encrypted_policy", "encrypted_policy", False),
|
||||
("S3", "encrypted_policy", "s3_encrypted_default_path", False),
|
||||
("S3", "s3_encrypted_default_path", "s3_encrypted_default_path", False),
|
||||
("S3", "s3_encrypted_default_path", "encrypted_policy", False),
|
||||
("File", "s3_encrypted_default_path", "encrypted_policy", False),
|
||||
("File", "local_policy", "encrypted_policy", False),
|
||||
("File", "encrypted_policy", "local_policy", False),
|
||||
("File", "encrypted_policy", "local_policy", True),
|
||||
],
|
||||
)
|
||||
def test_backup_restore(storage_policy, backup_type, storage_policy2):
|
||||
def test_backup_restore(
|
||||
backup_type,
|
||||
old_storage_policy,
|
||||
new_storage_policy,
|
||||
decrypt_files_from_encrypted_disks,
|
||||
):
|
||||
node.query(
|
||||
f"""
|
||||
CREATE TABLE encrypted_test (
|
||||
@ -327,7 +334,7 @@ def test_backup_restore(storage_policy, backup_type, storage_policy2):
|
||||
data String
|
||||
) ENGINE=MergeTree()
|
||||
ORDER BY id
|
||||
SETTINGS storage_policy='{storage_policy}'
|
||||
SETTINGS storage_policy='{old_storage_policy}'
|
||||
"""
|
||||
)
|
||||
|
||||
@ -343,22 +350,38 @@ def test_backup_restore(storage_policy, backup_type, storage_policy2):
|
||||
elif backup_type == "File":
|
||||
backup_destination = f"File('/backups/{backup_name}/')"
|
||||
|
||||
node.query(f"BACKUP TABLE encrypted_test TO {backup_destination}")
|
||||
node.query(
|
||||
f"BACKUP TABLE encrypted_test TO {backup_destination} SETTINGS decrypt_files_from_encrypted_disks={int(decrypt_files_from_encrypted_disks)}"
|
||||
)
|
||||
|
||||
if backup_type == "File" and storage_policy.find("encrypted") != -1:
|
||||
storage_policy_changed = old_storage_policy != new_storage_policy
|
||||
old_disk_encrypted = old_storage_policy.find("encrypted") != -1
|
||||
new_disk_encrypted = new_storage_policy.find("encrypted") != -1
|
||||
|
||||
if backup_type == "File":
|
||||
root_path = os.path.join(node.cluster.instances_dir, "backups", backup_name)
|
||||
expect_encrypted_in_backup = (
|
||||
old_disk_encrypted and not decrypt_files_from_encrypted_disks
|
||||
)
|
||||
|
||||
with open(f"{root_path}/metadata/default/encrypted_test.sql") as file:
|
||||
assert file.read().startswith("CREATE TABLE default.encrypted_test")
|
||||
|
||||
with open(f"{root_path}/.backup") as file:
|
||||
found_encrypted_in_backup = (
|
||||
file.read().find("<encrypted_by_disk>true</encrypted_by_disk>") != -1
|
||||
)
|
||||
assert found_encrypted_in_backup == expect_encrypted_in_backup
|
||||
|
||||
with open(
|
||||
f"{root_path}/data/default/encrypted_test/all_1_1_0/data.bin", "rb"
|
||||
) as file:
|
||||
assert file.read().startswith(b"ENC")
|
||||
with open(f"{root_path}/metadata/default/encrypted_test.sql") as file:
|
||||
assert file.read().startswith("CREATE TABLE default.encrypted_test")
|
||||
with open(f"{root_path}/.backup") as file:
|
||||
assert file.read().find("<encrypted_by_disk>true</encrypted_by_disk>") != -1
|
||||
found_encrypted_in_backup = file.read().startswith(b"ENC")
|
||||
assert found_encrypted_in_backup == expect_encrypted_in_backup
|
||||
|
||||
node.query(f"DROP TABLE encrypted_test SYNC")
|
||||
|
||||
if storage_policy != storage_policy2:
|
||||
if storage_policy_changed:
|
||||
node.query(
|
||||
f"""
|
||||
CREATE TABLE encrypted_test (
|
||||
@ -366,56 +389,22 @@ def test_backup_restore(storage_policy, backup_type, storage_policy2):
|
||||
data String
|
||||
) ENGINE=MergeTree()
|
||||
ORDER BY id
|
||||
SETTINGS storage_policy='{storage_policy2}'
|
||||
SETTINGS storage_policy='{new_storage_policy}'
|
||||
"""
|
||||
)
|
||||
|
||||
node.query(
|
||||
f"RESTORE TABLE encrypted_test FROM {backup_destination} SETTINGS allow_different_table_def={int(storage_policy != storage_policy2)}"
|
||||
)
|
||||
restore_command = f"RESTORE TABLE encrypted_test FROM {backup_destination} SETTINGS allow_different_table_def={int(storage_policy_changed)}"
|
||||
|
||||
assert node.query(select_query) == "(0,'data'),(1,'data')"
|
||||
expect_error = None
|
||||
if (
|
||||
old_disk_encrypted
|
||||
and not new_disk_encrypted
|
||||
and not decrypt_files_from_encrypted_disks
|
||||
):
|
||||
expect_error = "can be restored only to an encrypted disk"
|
||||
|
||||
|
||||
def test_cannot_restore_encrypted_files_to_unencrypted_disk():
|
||||
node.query(
|
||||
"""
|
||||
CREATE TABLE encrypted_test (
|
||||
id Int64,
|
||||
data String
|
||||
) ENGINE=MergeTree()
|
||||
ORDER BY id
|
||||
SETTINGS storage_policy='encrypted_policy'
|
||||
"""
|
||||
)
|
||||
|
||||
node.query("INSERT INTO encrypted_test VALUES (0,'data'),(1,'data')")
|
||||
assert (
|
||||
node.query("SELECT * FROM encrypted_test ORDER BY id FORMAT Values")
|
||||
== "(0,'data'),(1,'data')"
|
||||
)
|
||||
|
||||
backup_name = new_backup_name()
|
||||
backup_destination = (
|
||||
f"S3('http://minio1:9001/root/backups/{backup_name}', 'minio', 'minio123')"
|
||||
)
|
||||
node.query(f"BACKUP TABLE encrypted_test TO {backup_destination}")
|
||||
|
||||
node.query(f"DROP TABLE encrypted_test SYNC")
|
||||
|
||||
node.query(
|
||||
f"""
|
||||
CREATE TABLE encrypted_test (
|
||||
id Int64,
|
||||
data String
|
||||
) ENGINE=MergeTree()
|
||||
ORDER BY id
|
||||
SETTINGS storage_policy='local_policy'
|
||||
"""
|
||||
)
|
||||
|
||||
expected_error = "can be restored only to an encrypted disk"
|
||||
assert expected_error in node.query_and_get_error(
|
||||
f"RESTORE TABLE encrypted_test FROM {backup_destination} SETTINGS allow_different_table_def=1"
|
||||
)
|
||||
>>>>>>> 9c08fb30995 (Add tests.)
|
||||
if expect_error:
|
||||
assert expect_error in node.query_and_get_error(restore_command)
|
||||
else:
|
||||
node.query(restore_command)
|
||||
assert node.query(select_query) == "(0,'data'),(1,'data')"
|
||||
|
Loading…
Reference in New Issue
Block a user