Merge pull request #50206 from ClickHouse/transactions_for_encrypted_disk

Use transactions for encrypted disks
This commit is contained in:
alesapin 2023-05-26 13:13:58 +02:00 committed by GitHub
commit e71bb1cd52
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 471 additions and 122 deletions

View File

@ -138,19 +138,6 @@ namespace
}
}
String getCurrentKey(const String & path, const DiskEncryptedSettings & settings)
{
auto it = settings.keys.find(settings.current_key_id);
if (it == settings.keys.end())
throw Exception(
ErrorCodes::DATA_ENCRYPTION_ERROR,
"Not found a key with the current ID {} required to cipher file {}",
settings.current_key_id,
quoteString(path));
return it->second;
}
String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings)
{
auto it = settings.keys.find(header.key_id);
@ -203,18 +190,19 @@ private:
};
DiskEncrypted::DiskEncrypted(
const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_)
: DiskEncrypted(name_, parseDiskEncryptedSettings(name_, config_, config_prefix_, map_))
const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_, bool use_fake_transaction_)
: DiskEncrypted(name_, parseDiskEncryptedSettings(name_, config_, config_prefix_, map_), use_fake_transaction_)
{
}
DiskEncrypted::DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_)
DiskEncrypted::DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_, bool use_fake_transaction_)
: IDisk(name_)
, delegate(settings_->wrapped_disk)
, encrypted_name(name_)
, disk_path(settings_->disk_path)
, disk_absolute_path(settings_->wrapped_disk->getPath() + settings_->disk_path)
, current_settings(std::move(settings_))
, use_fake_transaction(use_fake_transaction_)
{
delegate->createDirectories(disk_path);
}
@ -309,38 +297,6 @@ std::unique_ptr<ReadBufferFromFileBase> DiskEncrypted::readFile(
return std::make_unique<ReadBufferFromEncryptedFile>(settings.local_fs_buffer_size, std::move(buffer), key, header);
}
std::unique_ptr<WriteBufferFromFileBase> DiskEncrypted::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &)
{
auto wrapped_path = wrappedPath(path);
FileEncryption::Header header;
String key;
UInt64 old_file_size = 0;
auto settings = current_settings.get();
if (mode == WriteMode::Append && exists(path))
{
old_file_size = getFileSize(path);
if (old_file_size)
{
/// Append mode: we continue to use the same header.
auto read_buffer = delegate->readFile(wrapped_path, ReadSettings().adjustBufferSize(FileEncryption::Header::kSize));
header = readHeader(*read_buffer);
key = getKey(path, header, *settings);
}
}
if (!old_file_size)
{
/// Rewrite mode: we generate a new header.
key = getCurrentKey(path, *settings);
header.algorithm = settings->current_algorithm;
header.key_id = settings->current_key_id;
header.key_hash = calculateKeyHash(key);
header.init_vector = InitVector::random();
}
auto buffer = delegate->writeFile(wrapped_path, buf_size, mode);
return std::make_unique<WriteBufferFromEncryptedFile>(buf_size, std::move(buffer), key, header, old_file_size);
}
size_t DiskEncrypted::getFileSize(const String & path) const
{
auto wrapped_path = wrappedPath(path);
@ -416,7 +372,7 @@ void registerDiskEncrypted(DiskFactory & factory, bool global_skip_access_check)
const DisksMap & map) -> DiskPtr
{
bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false);
DiskPtr disk = std::make_shared<DiskEncrypted>(name, config, config_prefix, map);
DiskPtr disk = std::make_shared<DiskEncrypted>(name, config, config_prefix, map, config.getBool(config_prefix + ".use_fake_transaction", true));
disk->startup(context, skip_access_check);
return disk;
};

View File

@ -6,22 +6,14 @@
#include <Disks/IDisk.h>
#include <Common/MultiVersion.h>
#include <Disks/FakeDiskTransaction.h>
#include <Disks/DiskEncryptedTransaction.h>
namespace DB
{
class ReadBufferFromFileBase;
class WriteBufferFromFileBase;
namespace FileEncryption { enum class Algorithm; }
struct DiskEncryptedSettings
{
DiskPtr wrapped_disk;
String disk_path;
std::unordered_map<UInt64, String> keys;
UInt64 current_key_id;
FileEncryption::Algorithm current_algorithm;
};
/// Encrypted disk ciphers all written files on the fly and writes the encrypted files to an underlying (normal) disk.
/// And when we read files from an encrypted disk it deciphers them automatically,
@ -29,8 +21,8 @@ struct DiskEncryptedSettings
class DiskEncrypted : public IDisk
{
public:
DiskEncrypted(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_);
DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_);
DiskEncrypted(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_, bool use_fake_transaction_);
DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_, bool use_fake_transaction_);
const String & getName() const override { return encrypted_name; }
const String & getPath() const override { return disk_absolute_path; }
@ -59,28 +51,30 @@ public:
void createDirectory(const String & path) override
{
auto wrapped_path = wrappedPath(path);
delegate->createDirectory(wrapped_path);
auto tx = createEncryptedTransaction();
tx->createDirectory(path);
tx->commit();
}
void createDirectories(const String & path) override
{
auto wrapped_path = wrappedPath(path);
delegate->createDirectories(wrapped_path);
auto tx = createEncryptedTransaction();
tx->createDirectories(path);
tx->commit();
}
void clearDirectory(const String & path) override
{
auto wrapped_path = wrappedPath(path);
delegate->clearDirectory(wrapped_path);
auto tx = createEncryptedTransaction();
tx->clearDirectory(path);
tx->commit();
}
void moveDirectory(const String & from_path, const String & to_path) override
{
auto wrapped_from_path = wrappedPath(from_path);
auto wrapped_to_path = wrappedPath(to_path);
delegate->moveDirectory(wrapped_from_path, wrapped_to_path);
auto tx = createEncryptedTransaction();
tx->moveDirectory(from_path, to_path);
tx->commit();
}
DirectoryIteratorPtr iterateDirectory(const String & path) const override
@ -91,22 +85,23 @@ public:
void createFile(const String & path) override
{
auto wrapped_path = wrappedPath(path);
delegate->createFile(wrapped_path);
auto tx = createEncryptedTransaction();
tx->createFile(path);
tx->commit();
}
void moveFile(const String & from_path, const String & to_path) override
{
auto wrapped_from_path = wrappedPath(from_path);
auto wrapped_to_path = wrappedPath(to_path);
delegate->moveFile(wrapped_from_path, wrapped_to_path);
auto tx = createEncryptedTransaction();
tx->moveFile(from_path, to_path);
tx->commit();
}
void replaceFile(const String & from_path, const String & to_path) override
{
auto wrapped_from_path = wrappedPath(from_path);
auto wrapped_to_path = wrappedPath(to_path);
delegate->replaceFile(wrapped_from_path, wrapped_to_path);
auto tx = createEncryptedTransaction();
tx->replaceFile(from_path, to_path);
tx->commit();
}
void listFiles(const String & path, std::vector<String> & file_names) const override
@ -129,61 +124,67 @@ public:
const String & path,
size_t buf_size,
WriteMode mode,
const WriteSettings & settings) override;
const WriteSettings & settings) override
{
auto tx = createEncryptedTransaction();
auto result = tx->writeFile(path, buf_size, mode, settings);
return result;
}
void removeFile(const String & path) override
{
auto wrapped_path = wrappedPath(path);
delegate->removeFile(wrapped_path);
auto tx = createEncryptedTransaction();
tx->removeFile(path);
tx->commit();
}
void removeFileIfExists(const String & path) override
{
auto wrapped_path = wrappedPath(path);
delegate->removeFileIfExists(wrapped_path);
auto tx = createEncryptedTransaction();
tx->removeFileIfExists(path);
tx->commit();
}
void removeDirectory(const String & path) override
{
auto wrapped_path = wrappedPath(path);
delegate->removeDirectory(wrapped_path);
auto tx = createEncryptedTransaction();
tx->removeDirectory(path);
tx->commit();
}
void removeRecursive(const String & path) override
{
auto wrapped_path = wrappedPath(path);
delegate->removeRecursive(wrapped_path);
auto tx = createEncryptedTransaction();
tx->removeRecursive(path);
tx->commit();
}
void removeSharedFile(const String & path, bool flag) override
{
auto wrapped_path = wrappedPath(path);
delegate->removeSharedFile(wrapped_path, flag);
auto tx = createEncryptedTransaction();
tx->removeSharedFile(path, flag);
tx->commit();
}
void removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override
{
auto wrapped_path = wrappedPath(path);
delegate->removeSharedRecursive(wrapped_path, keep_all_batch_data, file_names_remove_metadata_only);
auto tx = createEncryptedTransaction();
tx->removeSharedRecursive(path, keep_all_batch_data, file_names_remove_metadata_only);
tx->commit();
}
void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override
{
for (const auto & file : files)
{
auto wrapped_path = wrappedPath(file.path);
bool keep = keep_all_batch_data || file_names_remove_metadata_only.contains(fs::path(file.path).filename());
if (file.if_exists)
delegate->removeSharedFileIfExists(wrapped_path, keep);
else
delegate->removeSharedFile(wrapped_path, keep);
}
auto tx = createEncryptedTransaction();
tx->removeSharedFiles(files, keep_all_batch_data, file_names_remove_metadata_only);
tx->commit();
}
void removeSharedFileIfExists(const String & path, bool flag) override
{
auto wrapped_path = wrappedPath(path);
delegate->removeSharedFileIfExists(wrapped_path, flag);
auto tx = createEncryptedTransaction();
tx->removeSharedFileIfExists(path, flag);
tx->commit();
}
Strings getBlobPath(const String & path) const override
@ -194,8 +195,9 @@ public:
void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override
{
auto wrapped_path = wrappedPath(path);
delegate->writeFileUsingBlobWritingFunction(wrapped_path, mode, std::move(write_blob_function));
auto tx = createEncryptedTransaction();
tx->writeFileUsingBlobWritingFunction(path, mode, std::move(write_blob_function));
tx->commit();
}
std::unique_ptr<ReadBufferFromFileBase> readEncryptedFile(const String & path, const ReadSettings & settings) const override
@ -210,8 +212,9 @@ public:
WriteMode mode,
const WriteSettings & settings) const override
{
auto wrapped_path = wrappedPath(path);
return delegate->writeFile(wrapped_path, buf_size, mode, settings);
auto tx = createEncryptedTransaction();
auto buf = tx->writeEncryptedFile(path, buf_size, mode, settings);
return buf;
}
size_t getEncryptedFileSize(const String & path) const override
@ -228,8 +231,9 @@ public:
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override
{
auto wrapped_path = wrappedPath(path);
delegate->setLastModified(wrapped_path, timestamp);
auto tx = createEncryptedTransaction();
tx->setLastModified(path, timestamp);
tx->commit();
}
Poco::Timestamp getLastModified(const String & path) const override
@ -246,15 +250,16 @@ public:
void setReadOnly(const String & path) override
{
auto wrapped_path = wrappedPath(path);
delegate->setReadOnly(wrapped_path);
auto tx = createEncryptedTransaction();
tx->setReadOnly(path);
tx->commit();
}
void createHardLink(const String & src_path, const String & dst_path) override
{
auto wrapped_src_path = wrappedPath(src_path);
auto wrapped_dst_path = wrappedPath(dst_path);
delegate->createHardLink(wrapped_src_path, wrapped_dst_path);
auto tx = createEncryptedTransaction();
tx->createHardLink(src_path, dst_path);
tx->commit();
}
void truncateFile(const String & path, size_t size) override;
@ -289,11 +294,22 @@ public:
SyncGuardPtr getDirectorySyncGuard(const String & path) const override;
std::shared_ptr<DiskEncryptedTransaction> createEncryptedTransaction() const
{
auto delegate_transaction = delegate->createTransaction();
return std::make_shared<DiskEncryptedTransaction>(delegate_transaction, disk_path, *current_settings.get(), delegate.get());
}
DiskTransactionPtr createTransaction() override
{
/// Need to overwrite explicetly because this disk change
/// a lot of "delegate" methods.
return std::make_shared<FakeDiskTransaction>(*this);
if (use_fake_transaction)
{
return std::make_shared<FakeDiskTransaction>(*this);
}
else
{
return createEncryptedTransaction();
}
}
UInt64 getTotalSpace() const override
@ -331,10 +347,7 @@ public:
private:
String wrappedPath(const String & path) const
{
// if path starts_with disk_path -> got already wrapped path
if (!disk_path.empty() && path.starts_with(disk_path))
return path;
return disk_path + path;
return DiskEncryptedTransaction::wrappedPath(disk_path, path);
}
DiskPtr delegate;
@ -342,6 +355,7 @@ private:
const String disk_path;
const String disk_absolute_path;
MultiVersion<DiskEncryptedSettings> current_settings;
bool use_fake_transaction;
};
}

View File

@ -0,0 +1,120 @@
#include <Disks/DiskEncryptedTransaction.h>
#if USE_SSL
#include <IO/FileEncryptionCommon.h>
#include <Common/Exception.h>
#include <boost/algorithm/hex.hpp>
#include <IO/ReadBufferFromEncryptedFile.h>
#include <IO/ReadBufferFromFileDecorator.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromEncryptedFile.h>
#include <Common/quoteString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int DATA_ENCRYPTION_ERROR;
}
namespace
{
FileEncryption::Header readHeader(ReadBufferFromFileBase & read_buffer)
{
try
{
FileEncryption::Header header;
header.read(read_buffer);
return header;
}
catch (Exception & e)
{
e.addMessage("While reading the header of encrypted file " + quoteString(read_buffer.getFileName()));
throw;
}
}
String getCurrentKey(const String & path, const DiskEncryptedSettings & settings)
{
auto it = settings.keys.find(settings.current_key_id);
if (it == settings.keys.end())
throw Exception(
ErrorCodes::DATA_ENCRYPTION_ERROR,
"Not found a key with the current ID {} required to cipher file {}",
settings.current_key_id,
quoteString(path));
return it->second;
}
String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings)
{
auto it = settings.keys.find(header.key_id);
if (it == settings.keys.end())
throw Exception(
ErrorCodes::DATA_ENCRYPTION_ERROR,
"Not found a key with ID {} required to decipher file {}",
header.key_id,
quoteString(path));
String key = it->second;
if (FileEncryption::calculateKeyHash(key) != header.key_hash)
throw Exception(
ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong key with ID {}, could not decipher file {}", header.key_id, quoteString(path));
return key;
}
}
void DiskEncryptedTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path)
{
auto wrapped_from_path = wrappedPath(from_file_path);
auto wrapped_to_path = wrappedPath(to_file_path);
delegate_transaction->copyFile(wrapped_from_path, wrapped_to_path);
}
std::unique_ptr<WriteBufferFromFileBase> DiskEncryptedTransaction::writeFile( // NOLINT
const std::string & path,
size_t buf_size,
WriteMode mode,
const WriteSettings & settings,
bool autocommit)
{
auto wrapped_path = wrappedPath(path);
FileEncryption::Header header;
String key;
UInt64 old_file_size = 0;
if (mode == WriteMode::Append && delegate_disk->exists(wrapped_path))
{
size_t size = delegate_disk->getFileSize(wrapped_path);
old_file_size = size > FileEncryption::Header::kSize ? (size - FileEncryption::Header::kSize) : 0;
if (old_file_size)
{
/// Append mode: we continue to use the same header.
auto read_buffer = delegate_disk->readFile(wrapped_path, ReadSettings().adjustBufferSize(FileEncryption::Header::kSize));
header = readHeader(*read_buffer);
key = getKey(path, header, current_settings);
}
}
if (!old_file_size)
{
/// Rewrite mode: we generate a new header.
key = getCurrentKey(path, current_settings);
header.algorithm = current_settings.current_algorithm;
header.key_id = current_settings.current_key_id;
header.key_hash = FileEncryption::calculateKeyHash(key);
header.init_vector = FileEncryption::InitVector::random();
}
auto buffer = delegate_transaction->writeFile(wrapped_path, buf_size, mode, settings, autocommit);
return std::make_unique<WriteBufferFromEncryptedFile>(buf_size, std::move(buffer), key, header, old_file_size);
}
}
#endif

View File

@ -0,0 +1,259 @@
#pragma once
#include "config.h"
#if USE_SSL
#include <Disks/IDiskTransaction.h>
#include <Disks/IDisk.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
namespace DB
{
namespace FileEncryption { enum class Algorithm; }
struct DiskEncryptedSettings
{
DiskPtr wrapped_disk;
String disk_path;
std::unordered_map<UInt64, String> keys;
UInt64 current_key_id;
FileEncryption::Algorithm current_algorithm;
};
class DiskEncryptedTransaction : public IDiskTransaction
{
public:
static String wrappedPath(const String disk_path, const String & path)
{
// if path starts_with disk_path -> got already wrapped path
if (!disk_path.empty() && path.starts_with(disk_path))
return path;
return disk_path + path;
}
DiskEncryptedTransaction(DiskTransactionPtr delegate_transaction_, const std::string & disk_path_, DiskEncryptedSettings current_settings_, IDisk * delegate_disk_)
: delegate_transaction(delegate_transaction_)
, disk_path(disk_path_)
, current_settings(current_settings_)
, delegate_disk(delegate_disk_)
{}
/// Tries to commit all accumulated operations simultaneously.
/// If something fails rollback and throw exception.
void commit() override // NOLINT
{
delegate_transaction->commit();
}
void undo() override
{
delegate_transaction->undo();
}
~DiskEncryptedTransaction() override = default;
/// Create directory.
void createDirectory(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->createDirectory(wrapped_path);
}
/// Create directory and all parent directories if necessary.
void createDirectories(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->createDirectories(wrapped_path);
}
/// Remove all files from the directory. Directories are not removed.
void clearDirectory(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->clearDirectory(wrapped_path);
}
/// Move directory from `from_path` to `to_path`.
void moveDirectory(const std::string & from_path, const std::string & to_path) override
{
auto wrapped_from_path = wrappedPath(from_path);
auto wrapped_to_path = wrappedPath(to_path);
delegate_transaction->moveDirectory(wrapped_from_path, wrapped_to_path);
}
void moveFile(const std::string & from_path, const std::string & to_path) override
{
auto wrapped_from_path = wrappedPath(from_path);
auto wrapped_to_path = wrappedPath(to_path);
delegate_transaction->moveFile(wrapped_from_path, wrapped_to_path);
}
void createFile(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->createFile(wrapped_path);
}
/// Move the file from `from_path` to `to_path`.
/// If a file with `to_path` path already exists, it will be replaced.
void replaceFile(const std::string & from_path, const std::string & to_path) override
{
auto wrapped_from_path = wrappedPath(from_path);
auto wrapped_to_path = wrappedPath(to_path);
delegate_transaction->replaceFile(wrapped_from_path, wrapped_to_path);
}
/// Only copy of several files supported now. Disk interface support copy to another disk
/// but it's impossible to implement correctly in transactions because other disk can
/// use different metadata storage.
/// TODO: maybe remove it at all, we don't want copies
void copyFile(const std::string & from_file_path, const std::string & to_file_path) override;
/// Open the file for write and return WriteBufferFromFileBase object.
std::unique_ptr<WriteBufferFromFileBase> writeFile( /// NOLINT
const std::string & path,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
WriteMode mode = WriteMode::Rewrite,
const WriteSettings & settings = {},
bool autocommit = true) override;
/// Remove file. Throws exception if file doesn't exists or it's a directory.
void removeFile(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeFile(wrapped_path);
}
/// Remove file if it exists.
void removeFileIfExists(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeFileIfExists(wrapped_path);
}
/// Remove directory. Throws exception if it's not a directory or if directory is not empty.
void removeDirectory(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeDirectory(wrapped_path);
}
/// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists.
void removeRecursive(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeRecursive(wrapped_path);
}
/// Remove file. Throws exception if file doesn't exists or if directory is not empty.
/// Differs from removeFile for S3/HDFS disks
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
void removeSharedFile(const std::string & path, bool keep_shared_data) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeSharedFile(wrapped_path, keep_shared_data);
}
/// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists.
/// Differs from removeRecursive for S3/HDFS disks
/// Second bool param is a flag to remove (false) or keep (true) shared data on S3.
/// Third param determines which files cannot be removed even if second is true.
void removeSharedRecursive(const std::string & path, bool keep_all_shared_data, const NameSet & file_names_remove_metadata_only) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeSharedRecursive(wrapped_path, keep_all_shared_data, file_names_remove_metadata_only);
}
/// Remove file or directory if it exists.
/// Differs from removeFileIfExists for S3/HDFS disks
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
void removeSharedFileIfExists(const std::string & path, bool keep_shared_data) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeSharedFileIfExists(wrapped_path, keep_shared_data);
}
/// Batch request to remove multiple files.
/// May be much faster for blob storage.
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3.
/// Third param determines which files cannot be removed even if second is true.
void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override
{
for (const auto & file : files)
{
auto wrapped_path = wrappedPath(file.path);
bool keep = keep_all_batch_data || file_names_remove_metadata_only.contains(fs::path(file.path).filename());
if (file.if_exists)
delegate_transaction->removeSharedFileIfExists(wrapped_path, keep);
else
delegate_transaction->removeSharedFile(wrapped_path, keep);
}
}
/// Set last modified time to file or directory at `path`.
void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->setLastModified(wrapped_path, timestamp);
}
/// Just chmod.
void chmod(const String & path, mode_t mode) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->chmod(wrapped_path, mode);
}
/// Set file at `path` as read-only.
void setReadOnly(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->setReadOnly(wrapped_path);
}
/// Create hardlink from `src_path` to `dst_path`.
void createHardLink(const std::string & src_path, const std::string & dst_path) override
{
auto wrapped_src_path = wrappedPath(src_path);
auto wrapped_dst_path = wrappedPath(dst_path);
delegate_transaction->createHardLink(wrapped_src_path, wrapped_dst_path);
}
void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->writeFileUsingBlobWritingFunction(wrapped_path, mode, std::move(write_blob_function));
}
std::unique_ptr<WriteBufferFromFileBase> writeEncryptedFile(
const String & path,
size_t buf_size,
WriteMode mode,
const WriteSettings & settings) const
{
auto wrapped_path = wrappedPath(path);
return delegate_transaction->writeFile(wrapped_path, buf_size, mode, settings);
}
private:
String wrappedPath(const String & path) const
{
return wrappedPath(disk_path, path);
}
DiskTransactionPtr delegate_transaction;
std::string disk_path;
DiskEncryptedSettings current_settings;
IDisk * delegate_disk;
};
}
#endif

View File

@ -40,7 +40,7 @@ protected:
settings->keys[0] = key;
settings->current_key_id = 0;
settings->disk_path = path;
encrypted_disk = std::make_shared<DiskEncrypted>("encrypted_disk", std::move(settings));
encrypted_disk = std::make_shared<DiskEncrypted>("encrypted_disk", std::move(settings), true);
}
String getFileNames()