diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.cpp b/src/Backups/BackupEntryFromAppendOnlyFile.cpp index 331a4a69d06..1d73ab52820 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.cpp +++ b/src/Backups/BackupEntryFromAppendOnlyFile.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include @@ -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 unencrypted_file_size) + UInt64 calculateSize(const DiskPtr & disk, const String & file_path, bool copy_encrypted, std::optional 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 & file_size_) + const DiskPtr & disk_, const String & file_path_, bool copy_encrypted_, const std::optional & 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 BackupEntryFromAppendOnlyFile::getReadBuffer(const ReadSettings & read_settings) const { std::unique_ptr 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)); diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.h b/src/Backups/BackupEntryFromAppendOnlyFile.h index 8a78478dcc5..257c392f24c 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.h +++ b/src/Backups/BackupEntryFromAppendOnlyFile.h @@ -15,6 +15,7 @@ public: BackupEntryFromAppendOnlyFile( const DiskPtr & disk_, const String & file_path_, + bool copy_encrypted_ = false, const std::optional & 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; }; diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index 24965f09356..cc635dd8541 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -1,6 +1,5 @@ #include #include -#include namespace DB @@ -20,11 +19,13 @@ namespace BackupEntryFromImmutableFile::BackupEntryFromImmutableFile( const DiskPtr & disk_, const String & file_path_, + bool copy_encrypted_, const std::optional & file_size_, const std::optional & 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 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::getChecksum(); - else if (data_source_description.is_encrypted) + else if (copy_encrypted) checksum = combineChecksums(*checksum, disk->getEncryptedFileIV(file_path)); checksum_adjusted = true; } diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index 5cdb4adc7c4..850a86a3264 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -18,6 +18,7 @@ public: BackupEntryFromImmutableFile( const DiskPtr & disk_, const String & file_path_, + bool copy_encrypted_ = false, const std::optional & file_size_ = {}, const std::optional & checksum_ = {}); @@ -30,7 +31,7 @@ public: std::optional 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 file_size; mutable std::optional checksum; mutable bool file_size_adjusted = false; diff --git a/src/Backups/BackupEntryFromSmallFile.cpp b/src/Backups/BackupEntryFromSmallFile.cpp index 3dcee7147fc..22487767689 100644 --- a/src/Backups/BackupEntryFromSmallFile.cpp +++ b/src/Backups/BackupEntryFromSmallFile.cpp @@ -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)) { } diff --git a/src/Backups/BackupEntryFromSmallFile.h b/src/Backups/BackupEntryFromSmallFile.h index 4f936718fbb..d6651ab8cb5 100644 --- a/src/Backups/BackupEntryFromSmallFile.h +++ b/src/Backups/BackupEntryFromSmallFile.h @@ -14,13 +14,13 @@ class BackupEntryFromSmallFile : public BackupEntryWithChecksumCalculation 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; }; diff --git a/src/Backups/BackupEntryWithChecksumCalculation.cpp b/src/Backups/BackupEntryWithChecksumCalculation.cpp index 2c0c2eab8ff..1e634e6bb73 100644 --- a/src/Backups/BackupEntryWithChecksumCalculation.cpp +++ b/src/Backups/BackupEntryWithChecksumCalculation.cpp @@ -25,15 +25,21 @@ std::optional BackupEntryWithChecksumCalculation::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(); diff --git a/src/Backups/BackupIO_Default.cpp b/src/Backups/BackupIO_Default.cpp index 3b4851e9441..f7ba061cf3a 100644 --- a/src/Backups/BackupIO_Default.cpp +++ b/src/Backups/BackupIO_Default.cpp @@ -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); diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 57d85305e25..882342467fe 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -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) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 1b97256c75b..2c899687e6e 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -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; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 47b4ed80ebf..4d74fe8bbab 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -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 readEncryptedFile( + virtual std::unique_ptr readEncryptedFile( /// NOLINT const String & path, const ReadSettings & settings = ReadSettings{}, std::optional read_hint = {}, std::optional file_size = {}) const; /// Writes an already encrypted file to the disk (only for encrypted disks). - virtual std::unique_ptr writeEncryptedFile( + virtual std::unique_ptr writeEncryptedFile( /// NOLINT const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, WriteMode mode = WriteMode::Rewrite, diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index c1e2c5b8cf8..ebe55ea7dc7 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -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(disk, filepath_on_disk)); + backup_entries.emplace_back(filepath_in_backup, std::make_unique(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(disk, filepath_on_disk, file_size, file_hash); + BackupEntryPtr backup_entry = std::make_unique(disk, filepath_on_disk, copy_encrypted, file_size, file_hash); if (temp_dir_owner) backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 52544bb2457..09eb7f008bc 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -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( diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 9c267d94e63..7c85469d890 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -65,6 +65,7 @@ using SyncGuardPtr = std::unique_ptr; class IBackupEntry; using BackupEntryPtr = std::shared_ptr; using BackupEntries = std::vector>; +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. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e36bc8baeb4..b8208052f19 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -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> 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); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index ce5c6a730e9..04b008b623c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -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; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 6d77d2b97b0..f698f1881fa 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -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( - 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( - 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(disk, files_info_path)); + data_path_in_backup_fs / fileName(files_info_path), std::make_unique(disk, files_info_path, copy_encrypted)); /// columns.txt backup_entries_collector.addBackupEntry( diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index ebc780f5ab1..c9654cfd105 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -23,7 +23,7 @@ #include #include #include -#include +#include #include #include #include @@ -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(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(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(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(temp_disk, index_mrk_path)}; } /// Writing columns.txt diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b4dc2830bd6..2c19d3ba122 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -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 & 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)); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fba6e6cbafc..fcb7adbd69f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -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(); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 23bf88b9db5..b2e7c202800 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -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( - 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( - 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(disk, files_info_path)); + data_path_in_backup_fs / fileName(files_info_path), std::make_unique(disk, files_info_path, copy_encrypted)); /// columns.txt backup_entries_collector.addBackupEntry( diff --git a/tests/integration/test_encrypted_disk/test.py b/tests/integration/test_encrypted_disk/test.py index 90da5849c7f..66ff073f02b 100644 --- a/tests/integration/test_encrypted_disk/test.py +++ b/tests/integration/test_encrypted_disk/test.py @@ -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("true") != -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("true") != -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')"