Add backup setting "decrypt_files_from_encrypted_disks"

This commit is contained in:
Vitaly Baranov 2023-05-04 01:27:16 +02:00
parent 019493efa3
commit 943707963f
22 changed files with 152 additions and 121 deletions

View File

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

View File

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

View File

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

View File

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

View File

@ -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))
{
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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')"