mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Add comments for encrypted disks; some improvements in code.
This commit is contained in:
parent
c6177bd0cc
commit
9cc258327e
@ -5,6 +5,7 @@
|
||||
#include <IO/FileEncryptionCommon.h>
|
||||
#include <IO/ReadBufferFromEncryptedFile.h>
|
||||
#include <IO/WriteBufferFromEncryptedFile.h>
|
||||
#include <boost/algorithm/hex.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -12,13 +13,81 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int INCORRECT_DISK_INDEX;
|
||||
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
using DiskEncryptedPtr = std::shared_ptr<DiskEncrypted>;
|
||||
using namespace FileEncryption;
|
||||
namespace
|
||||
{
|
||||
using DiskEncryptedPtr = std::shared_ptr<DiskEncrypted>;
|
||||
using namespace FileEncryption;
|
||||
|
||||
String unhexKey(const String & hex, const String & disk_name)
|
||||
{
|
||||
try
|
||||
{
|
||||
return boost::algorithm::unhex(hex);
|
||||
}
|
||||
catch (const std::exception &)
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read key_hex for disk {}, check for valid characters [0-9a-fA-F] and length", disk_name);
|
||||
}
|
||||
}
|
||||
|
||||
struct DiskEncryptedSettings
|
||||
{
|
||||
String key;
|
||||
DiskPtr wrapped_disk;
|
||||
String path_on_wrapped_disk;
|
||||
|
||||
DiskEncryptedSettings(
|
||||
const String & disk_name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const DisksMap & map)
|
||||
{
|
||||
String wrapped_disk_name = config.getString(config_prefix + ".disk", "");
|
||||
if (wrapped_disk_name.empty())
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Name of the wrapped disk must not be empty. An encrypted disk is a wrapper over another disk. "
|
||||
"Disk {}", disk_name);
|
||||
|
||||
key = config.getString(config_prefix + ".key", "");
|
||||
String key_hex = config.getString(config_prefix + ".key_hex", "");
|
||||
if (!key.empty() && !key_hex.empty())
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Both 'key' and 'key_hex' are specified. There should be only one. Disk {}", disk_name);
|
||||
|
||||
if (!key_hex.empty())
|
||||
{
|
||||
assert(key.empty());
|
||||
key = unhexKey(key_hex, disk_name);
|
||||
}
|
||||
|
||||
if (key.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key of the encrypted disk must not be empty. Disk {}", disk_name);
|
||||
|
||||
if (!FileEncryption::isKeyLengthSupported(key.length()))
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Key length is not supported, supported only keys of length 16, 24, or 32 bytes. Disk {}", disk_name);
|
||||
|
||||
auto wrapped_disk_it = map.find(wrapped_disk_name);
|
||||
if (wrapped_disk_it == map.end())
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"The wrapped disk must have been announced earlier. No disk with name {}. Disk {}",
|
||||
wrapped_disk_name, disk_name);
|
||||
wrapped_disk = wrapped_disk_it->second;
|
||||
|
||||
path_on_wrapped_disk = config.getString(config_prefix + ".path", "");
|
||||
}
|
||||
};
|
||||
|
||||
bool inline isSameDiskType(const IDisk & one, const IDisk & another)
|
||||
{
|
||||
return typeid(one) == typeid(another);
|
||||
}
|
||||
}
|
||||
|
||||
class DiskEncryptedReservation : public IReservation
|
||||
{
|
||||
@ -57,23 +126,45 @@ ReservationPtr DiskEncrypted::reserve(UInt64 bytes)
|
||||
DiskEncrypted::DiskEncrypted(const String & name_, DiskPtr disk_, const String & key_, const String & path_)
|
||||
: DiskDecorator(disk_)
|
||||
, name(name_), key(key_), disk_path(path_)
|
||||
, disk_absolute_path(delegate->getPath() + disk_path)
|
||||
{
|
||||
initialize();
|
||||
}
|
||||
|
||||
void DiskEncrypted::initialize()
|
||||
{
|
||||
disk_absolute_path = delegate->getPath() + disk_path;
|
||||
|
||||
// use wrapped_disk as an EncryptedDisk store
|
||||
if (disk_path.empty())
|
||||
return;
|
||||
|
||||
if (disk_path.back() != '/')
|
||||
throw Exception("Disk path must ends with '/', but '" + disk_path + "' doesn't.", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Disk path must ends with '/', but '" + disk_path + "' doesn't.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
delegate->createDirectories(disk_path);
|
||||
}
|
||||
|
||||
void DiskEncrypted::copy(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path)
|
||||
{
|
||||
/// Check if we can copy the file without deciphering.
|
||||
if (isSameDiskType(*this, *to_disk))
|
||||
{
|
||||
/// Disk type is the same, check if the key is the same too.
|
||||
if (auto * to_encrypted_disk = typeid_cast<DiskEncrypted *>(to_disk.get()))
|
||||
{
|
||||
if (key == to_encrypted_disk->key)
|
||||
{
|
||||
/// Key is the same so we can simply copy the encrypted file.
|
||||
delegate->copy(wrappedPath(from_path), to_encrypted_disk->delegate, wrappedPath(to_path));
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Copy the file through buffers with deciphering.
|
||||
copyThroughBuffers(from_path, to_disk, to_path);
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> DiskEncrypted::readFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
@ -85,37 +176,28 @@ std::unique_ptr<ReadBufferFromFileBase> DiskEncrypted::readFile(
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
auto buffer = delegate->readFile(wrapped_path, buf_size, estimated_size, aio_threshold, mmap_threshold, mmap_cache);
|
||||
|
||||
String iv;
|
||||
size_t offset = 0;
|
||||
|
||||
if (exists(path) && getFileSize(path))
|
||||
{
|
||||
iv = readIV(kIVSize, *buffer);
|
||||
offset = kIVSize;
|
||||
}
|
||||
else
|
||||
iv = randomString(kIVSize);
|
||||
|
||||
return std::make_unique<ReadBufferFromEncryptedFile>(buf_size, std::move(buffer), iv, key, offset);
|
||||
InitVector iv;
|
||||
iv.read(*buffer);
|
||||
return std::make_unique<ReadBufferFromEncryptedFile>(buf_size, std::move(buffer), key, iv);
|
||||
}
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DiskEncrypted::writeFile(const String & path, size_t buf_size, WriteMode mode)
|
||||
{
|
||||
String iv;
|
||||
size_t start_offset = 0;
|
||||
InitVector iv;
|
||||
UInt64 old_file_size = 0;
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
|
||||
if (mode == WriteMode::Append && exists(path) && getFileSize(path))
|
||||
{
|
||||
auto read_buffer = delegate->readFile(wrapped_path, kIVSize);
|
||||
iv = readIV(kIVSize, *read_buffer);
|
||||
start_offset = getFileSize(path);
|
||||
auto read_buffer = delegate->readFile(wrapped_path, InitVector::kSize);
|
||||
iv.read(*read_buffer);
|
||||
old_file_size = getFileSize(path);
|
||||
}
|
||||
else
|
||||
iv = randomString(kIVSize);
|
||||
iv = InitVector::random();
|
||||
|
||||
auto buffer = delegate->writeFile(wrapped_path, buf_size, mode);
|
||||
return std::make_unique<WriteBufferFromEncryptedFile>(buf_size, std::move(buffer), iv, key, start_offset);
|
||||
return std::make_unique<WriteBufferFromEncryptedFile>(buf_size, std::move(buffer), key, iv, old_file_size);
|
||||
}
|
||||
|
||||
|
||||
@ -123,13 +205,13 @@ size_t DiskEncrypted::getFileSize(const String & path) const
|
||||
{
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
size_t size = delegate->getFileSize(wrapped_path);
|
||||
return size > kIVSize ? (size - kIVSize) : 0;
|
||||
return size > InitVector::kSize ? (size - InitVector::kSize) : 0;
|
||||
}
|
||||
|
||||
void DiskEncrypted::truncateFile(const String & path, size_t size)
|
||||
{
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
delegate->truncateFile(wrapped_path, size ? (size + kIVSize) : 0);
|
||||
delegate->truncateFile(wrapped_path, size ? (size + InitVector::kSize) : 0);
|
||||
}
|
||||
|
||||
SyncGuardPtr DiskEncrypted::getDirectorySyncGuard(const String & path) const
|
||||
@ -144,22 +226,10 @@ void DiskEncrypted::applyNewSettings(
|
||||
const String & config_prefix,
|
||||
const DisksMap & map)
|
||||
{
|
||||
String wrapped_disk_name = config.getString(config_prefix + ".disk", "");
|
||||
if (wrapped_disk_name.empty())
|
||||
throw Exception("The wrapped disk name can not be empty. An encrypted disk is a wrapper over another disk. "
|
||||
"Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
key = config.getString(config_prefix + ".key", "");
|
||||
if (key.empty())
|
||||
throw Exception("Encrypted disk key can not be empty. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
auto wrapped_disk = map.find(wrapped_disk_name);
|
||||
if (wrapped_disk == map.end())
|
||||
throw Exception("The wrapped disk must have been announced earlier. No disk with name " + wrapped_disk_name + ". Disk " + name,
|
||||
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
delegate = wrapped_disk->second;
|
||||
|
||||
disk_path = config.getString(config_prefix + ".path", "");
|
||||
DiskEncryptedSettings settings{name, config, config_prefix, map};
|
||||
key = settings.key;
|
||||
delegate = settings.wrapped_disk;
|
||||
disk_path = settings.path_on_wrapped_disk;
|
||||
initialize();
|
||||
}
|
||||
|
||||
@ -169,28 +239,10 @@ void registerDiskEncrypted(DiskFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
ContextPtr /*context*/,
|
||||
const DisksMap & map) -> DiskPtr {
|
||||
|
||||
String wrapped_disk_name = config.getString(config_prefix + ".disk", "");
|
||||
if (wrapped_disk_name.empty())
|
||||
throw Exception("The wrapped disk name can not be empty. An encrypted disk is a wrapper over another disk. "
|
||||
"Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
String key = config.getString(config_prefix + ".key", "");
|
||||
if (key.empty())
|
||||
throw Exception("Encrypted disk key can not be empty. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
if (key.size() != cipherKeyLength(defaultCipher()))
|
||||
throw Exception("Expected key with size " + std::to_string(cipherKeyLength(defaultCipher())) + ", got key with size " + std::to_string(key.size()),
|
||||
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
auto wrapped_disk = map.find(wrapped_disk_name);
|
||||
if (wrapped_disk == map.end())
|
||||
throw Exception("The wrapped disk must have been announced earlier. No disk with name " + wrapped_disk_name + ". Disk " + name,
|
||||
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
String relative_path = config.getString(config_prefix + ".path", "");
|
||||
|
||||
return std::make_shared<DiskEncrypted>(name, wrapped_disk->second, key, relative_path);
|
||||
const DisksMap & map) -> DiskPtr
|
||||
{
|
||||
DiskEncryptedSettings settings{name, config, config_prefix, map};
|
||||
return std::make_shared<DiskEncrypted>(name, settings.wrapped_disk, settings.key, settings.path_on_wrapped_disk);
|
||||
};
|
||||
factory.registerDiskType("encrypted", creator);
|
||||
}
|
||||
|
@ -14,6 +14,9 @@ namespace DB
|
||||
class ReadBufferFromFileBase;
|
||||
class WriteBufferFromFileBase;
|
||||
|
||||
/// 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,
|
||||
/// so we can work with a encrypted disk like it's a normal disk.
|
||||
class DiskEncrypted : public DiskDecorator
|
||||
{
|
||||
public:
|
||||
@ -102,10 +105,7 @@ public:
|
||||
delegate->listFiles(wrapped_path, file_names);
|
||||
}
|
||||
|
||||
void copy(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path) override
|
||||
{
|
||||
IDisk::copy(from_path, to_disk, to_path);
|
||||
}
|
||||
void copy(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path) override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
|
@ -309,7 +309,7 @@ void DiskLocal::copy(const String & from_path, const std::shared_ptr<IDisk> & to
|
||||
fs::copy(from, to, fs::copy_options::recursive | fs::copy_options::overwrite_existing); /// Use more optimal way.
|
||||
}
|
||||
else
|
||||
IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers.
|
||||
copyThroughBuffers(from_path, to_disk, to_path); /// Base implementation.
|
||||
}
|
||||
|
||||
SyncGuardPtr DiskLocal::getDirectorySyncGuard(const String & path) const
|
||||
|
@ -58,7 +58,7 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p
|
||||
}
|
||||
}
|
||||
|
||||
void IDisk::copy(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path)
|
||||
void IDisk::copyThroughBuffers(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path)
|
||||
{
|
||||
auto & exec = to_disk->getExecutor();
|
||||
ResultsCollector results;
|
||||
@ -71,6 +71,11 @@ void IDisk::copy(const String & from_path, const std::shared_ptr<IDisk> & to_dis
|
||||
result.get();
|
||||
}
|
||||
|
||||
void IDisk::copy(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path)
|
||||
{
|
||||
copyThroughBuffers(from_path, to_disk, to_path);
|
||||
}
|
||||
|
||||
void IDisk::truncateFile(const String &, size_t)
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate operation is not implemented for disk of type {}", getType());
|
||||
|
@ -242,6 +242,11 @@ protected:
|
||||
/// Returns executor to perform asynchronous operations.
|
||||
virtual Executor & getExecutor() { return *executor; }
|
||||
|
||||
/// Base implementation of the function copy().
|
||||
/// It just opens two files, reads data by portions from the first file, and writes it to the second one.
|
||||
/// A derived class may override copy() to provide a faster implementation.
|
||||
void copyThroughBuffers(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path);
|
||||
|
||||
private:
|
||||
std::unique_ptr<Executor> executor;
|
||||
};
|
||||
|
@ -1,8 +1,8 @@
|
||||
#include <IO/FileEncryptionCommon.h>
|
||||
|
||||
#if USE_SSL
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
@ -23,244 +23,281 @@ namespace FileEncryption
|
||||
|
||||
namespace
|
||||
{
|
||||
String toBigEndianString(UInt128 value)
|
||||
constexpr const size_t kBlockSize = 16;
|
||||
|
||||
size_t blockOffset(size_t pos) { return pos % kBlockSize; }
|
||||
size_t blocks(size_t pos) { return pos / kBlockSize; }
|
||||
|
||||
size_t partBlockSize(size_t size, size_t off)
|
||||
{
|
||||
WriteBufferFromOwnString out;
|
||||
writeBinaryBigEndian(value, out);
|
||||
return std::move(out.str());
|
||||
assert(off < kBlockSize);
|
||||
/// write the part as usual block
|
||||
if (off == 0)
|
||||
return 0;
|
||||
return off + size <= kBlockSize ? size : (kBlockSize - off) % kBlockSize;
|
||||
}
|
||||
|
||||
UInt128 fromBigEndianString(const String & str)
|
||||
size_t encryptBlocks(EVP_CIPHER_CTX * evp_ctx, const char * data, size_t size, WriteBuffer & out)
|
||||
{
|
||||
ReadBufferFromMemory in{str.data(), str.length()};
|
||||
UInt128 result;
|
||||
readBinaryBigEndian(result, in);
|
||||
return result;
|
||||
const uint8_t * in = reinterpret_cast<const uint8_t *>(data);
|
||||
size_t in_size = 0;
|
||||
size_t out_size = 0;
|
||||
|
||||
while (in_size < size)
|
||||
{
|
||||
out.nextIfAtEnd();
|
||||
size_t part_size = std::min(size - in_size, out.available());
|
||||
uint8_t * ciphertext = reinterpret_cast<uint8_t *>(out.position());
|
||||
int ciphertext_size = 0;
|
||||
if (!EVP_EncryptUpdate(evp_ctx, ciphertext, &ciphertext_size, &in[in_size], part_size))
|
||||
throw Exception("Failed to encrypt", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
in_size += part_size;
|
||||
if (ciphertext_size)
|
||||
{
|
||||
out.position() += ciphertext_size;
|
||||
out_size += ciphertext_size;
|
||||
}
|
||||
}
|
||||
|
||||
return out_size;
|
||||
}
|
||||
|
||||
size_t encryptBlockWithPadding(EVP_CIPHER_CTX * evp_ctx, const char * data, size_t size, size_t pad_left, WriteBuffer & out)
|
||||
{
|
||||
assert((size <= kBlockSize) && (size + pad_left <= kBlockSize));
|
||||
uint8_t padded_data[kBlockSize] = {};
|
||||
memcpy(&padded_data[pad_left], data, size);
|
||||
size_t padded_data_size = pad_left + size;
|
||||
|
||||
uint8_t ciphertext[kBlockSize];
|
||||
int ciphertext_size = 0;
|
||||
if (!EVP_EncryptUpdate(evp_ctx, ciphertext, &ciphertext_size, padded_data, padded_data_size))
|
||||
throw Exception("Failed to encrypt", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
if (!ciphertext_size)
|
||||
return 0;
|
||||
|
||||
if (static_cast<size_t>(ciphertext_size) < pad_left)
|
||||
throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Unexpected size of encrypted data: {} < {}", ciphertext_size, pad_left);
|
||||
|
||||
uint8_t * ciphertext_begin = &ciphertext[pad_left];
|
||||
ciphertext_size -= pad_left;
|
||||
out.write(reinterpret_cast<const char *>(ciphertext_begin), ciphertext_size);
|
||||
return ciphertext_size;
|
||||
}
|
||||
|
||||
size_t encryptFinal(EVP_CIPHER_CTX * evp_ctx, WriteBuffer & out)
|
||||
{
|
||||
uint8_t ciphertext[kBlockSize];
|
||||
int ciphertext_size = 0;
|
||||
if (!EVP_EncryptFinal_ex(evp_ctx,
|
||||
ciphertext, &ciphertext_size))
|
||||
throw Exception("Failed to finalize encrypting", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
if (ciphertext_size)
|
||||
out.write(reinterpret_cast<const char *>(ciphertext), ciphertext_size);
|
||||
return ciphertext_size;
|
||||
}
|
||||
|
||||
size_t decryptBlocks(EVP_CIPHER_CTX * evp_ctx, const char * data, size_t size, char * out)
|
||||
{
|
||||
const uint8_t * in = reinterpret_cast<const uint8_t *>(data);
|
||||
uint8_t * plaintext = reinterpret_cast<uint8_t *>(out);
|
||||
int plaintext_size = 0;
|
||||
if (!EVP_DecryptUpdate(evp_ctx, plaintext, &plaintext_size, in, size))
|
||||
throw Exception("Failed to decrypt", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
return plaintext_size;
|
||||
}
|
||||
|
||||
size_t decryptBlockWithPadding(EVP_CIPHER_CTX * evp_ctx, const char * data, size_t size, size_t pad_left, char * out)
|
||||
{
|
||||
assert((size <= kBlockSize) && (size + pad_left <= kBlockSize));
|
||||
uint8_t padded_data[kBlockSize] = {};
|
||||
memcpy(&padded_data[pad_left], data, size);
|
||||
size_t padded_data_size = pad_left + size;
|
||||
|
||||
uint8_t plaintext[kBlockSize];
|
||||
int plaintext_size = 0;
|
||||
if (!EVP_DecryptUpdate(evp_ctx, plaintext, &plaintext_size, padded_data, padded_data_size))
|
||||
throw Exception("Failed to decrypt", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
if (!plaintext_size)
|
||||
return 0;
|
||||
|
||||
if (static_cast<size_t>(plaintext_size) < pad_left)
|
||||
throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Unexpected size of decrypted data: {} < {}", plaintext_size, pad_left);
|
||||
|
||||
const uint8_t * plaintext_begin = &plaintext[pad_left];
|
||||
plaintext_size -= pad_left;
|
||||
memcpy(out, plaintext_begin, plaintext_size);
|
||||
return plaintext_size;
|
||||
}
|
||||
|
||||
size_t decryptFinal(EVP_CIPHER_CTX * evp_ctx, char * out)
|
||||
{
|
||||
uint8_t plaintext[kBlockSize];
|
||||
int plaintext_size = 0;
|
||||
if (!EVP_DecryptFinal_ex(evp_ctx, plaintext, &plaintext_size))
|
||||
throw Exception("Failed to finalize decrypting", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
if (plaintext_size)
|
||||
memcpy(out, plaintext, plaintext_size);
|
||||
return plaintext_size;
|
||||
}
|
||||
}
|
||||
|
||||
InitVector::InitVector(const String & iv_) : iv(fromBigEndianString(iv_)) {}
|
||||
|
||||
const String & InitVector::str() const
|
||||
String InitVector::toString() const
|
||||
{
|
||||
local = toBigEndianString(iv + counter);
|
||||
return local;
|
||||
static_assert(sizeof(counter) == InitVector::kSize);
|
||||
WriteBufferFromOwnString out;
|
||||
writeBinaryBigEndian(counter, out);
|
||||
return std::move(out.str());
|
||||
}
|
||||
|
||||
Encryption::Encryption(const String & iv_, const EncryptionKey & key_, size_t offset_)
|
||||
: evp_cipher(defaultCipher())
|
||||
, init_vector(iv_)
|
||||
, key(key_)
|
||||
, block_size(cipherIVLength(evp_cipher))
|
||||
InitVector InitVector::fromString(const String & str)
|
||||
{
|
||||
if (iv_.size() != cipherIVLength(evp_cipher))
|
||||
throw DB::Exception("Expected iv with size " + std::to_string(cipherIVLength(evp_cipher)) + ", got iv with size " + std::to_string(iv_.size()),
|
||||
DB::ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
if (key_.size() != cipherKeyLength(evp_cipher))
|
||||
throw DB::Exception("Expected key with size " + std::to_string(cipherKeyLength(evp_cipher)) + ", got iv with size " + std::to_string(key_.size()),
|
||||
DB::ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
offset = offset_;
|
||||
if (str.length() != InitVector::kSize)
|
||||
throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Expected iv with size {}, got iv with size {}", InitVector::kSize, str.length());
|
||||
ReadBufferFromMemory in{str.data(), str.length()};
|
||||
UInt128 counter;
|
||||
readBinaryBigEndian(counter, in);
|
||||
return InitVector{counter};
|
||||
}
|
||||
|
||||
size_t Encryption::partBlockSize(size_t size, size_t off) const
|
||||
void InitVector::read(ReadBuffer & in)
|
||||
{
|
||||
assert(off < block_size);
|
||||
/// write the part as usual block
|
||||
if (off == 0)
|
||||
return 0;
|
||||
return off + size <= block_size ? size : (block_size - off) % block_size;
|
||||
readBinaryBigEndian(counter, in);
|
||||
}
|
||||
|
||||
void Encryptor::encrypt(const char * plaintext, WriteBuffer & buf, size_t size)
|
||||
void InitVector::write(WriteBuffer & out) const
|
||||
{
|
||||
if (!size)
|
||||
return;
|
||||
|
||||
auto iv = InitVector(init_vector);
|
||||
auto off = blockOffset(offset);
|
||||
iv.set(blocks(offset));
|
||||
|
||||
size_t part_size = partBlockSize(size, off);
|
||||
if (off)
|
||||
{
|
||||
buf.write(encryptPartialBlock(plaintext, part_size, iv, off).data(), part_size);
|
||||
offset += part_size;
|
||||
size -= part_size;
|
||||
iv.inc();
|
||||
}
|
||||
|
||||
if (size)
|
||||
{
|
||||
buf.write(encryptNBytes(plaintext + part_size, size, iv).data(), size);
|
||||
offset += size;
|
||||
}
|
||||
writeBinaryBigEndian(counter, out);
|
||||
}
|
||||
|
||||
String Encryptor::encryptPartialBlock(const char * partial_block, size_t size, const InitVector & iv, size_t off) const
|
||||
InitVector InitVector::random()
|
||||
{
|
||||
if (size > block_size)
|
||||
throw Exception("Expected partial block, got block with size > block_size: size = " + std::to_string(size) + " and offset = " + std::to_string(off),
|
||||
ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
String plaintext(block_size, '\0');
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
plaintext[i + off] = partial_block[i];
|
||||
|
||||
return String(encryptNBytes(plaintext.data(), block_size, iv), off, size);
|
||||
}
|
||||
|
||||
String Encryptor::encryptNBytes(const char * data, size_t bytes, const InitVector & iv) const
|
||||
{
|
||||
String ciphertext(bytes, '\0');
|
||||
auto * ciphertext_ref = ciphertext.data();
|
||||
|
||||
auto evp_ctx_ptr = std::unique_ptr<EVP_CIPHER_CTX, decltype(&::EVP_CIPHER_CTX_free)>(EVP_CIPHER_CTX_new(), &EVP_CIPHER_CTX_free);
|
||||
auto * evp_ctx = evp_ctx_ptr.get();
|
||||
|
||||
if (EVP_EncryptInit_ex(evp_ctx, evp_cipher, nullptr, nullptr, nullptr) != 1)
|
||||
throw Exception("Failed to initialize encryption context with cipher", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
if (EVP_EncryptInit_ex(evp_ctx, nullptr, nullptr,
|
||||
reinterpret_cast<const unsigned char*>(key.str().data()),
|
||||
reinterpret_cast<const unsigned char*>(iv.str().data())) != 1)
|
||||
throw Exception("Failed to set key and IV for encryption", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
int output_len = 0;
|
||||
if (EVP_EncryptUpdate(evp_ctx,
|
||||
reinterpret_cast<unsigned char*>(ciphertext_ref), &output_len,
|
||||
reinterpret_cast<const unsigned char*>(data), static_cast<int>(bytes)) != 1)
|
||||
throw Exception("Failed to encrypt", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
ciphertext_ref += output_len;
|
||||
|
||||
int final_output_len = 0;
|
||||
if (EVP_EncryptFinal_ex(evp_ctx,
|
||||
reinterpret_cast<unsigned char*>(ciphertext_ref), &final_output_len) != 1)
|
||||
throw Exception("Failed to fetch ciphertext", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
if (output_len < 0 || final_output_len < 0 || static_cast<size_t>(output_len) + static_cast<size_t>(final_output_len) != bytes)
|
||||
throw Exception("Only part of the data was encrypted", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
return ciphertext;
|
||||
}
|
||||
|
||||
void Decryptor::decrypt(const char * ciphertext, BufferBase::Position buf, size_t size, size_t off)
|
||||
{
|
||||
if (!size)
|
||||
return;
|
||||
|
||||
auto iv = InitVector(init_vector);
|
||||
iv.set(blocks(off));
|
||||
off = blockOffset(off);
|
||||
|
||||
size_t part_size = partBlockSize(size, off);
|
||||
if (off)
|
||||
{
|
||||
decryptPartialBlock(buf, ciphertext, part_size, iv, off);
|
||||
size -= part_size;
|
||||
if (part_size + off == block_size)
|
||||
iv.inc();
|
||||
}
|
||||
|
||||
if (size)
|
||||
decryptNBytes(buf, ciphertext + part_size, size, iv);
|
||||
}
|
||||
|
||||
void Decryptor::decryptPartialBlock(BufferBase::Position & to, const char * partial_block, size_t size, const InitVector & iv, size_t off) const
|
||||
{
|
||||
if (size > block_size)
|
||||
throw Exception("Expecter partial block, got block with size > block_size: size = " + std::to_string(size) + " and offset = " + std::to_string(off),
|
||||
ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
String ciphertext(block_size, '\0');
|
||||
String plaintext(block_size, '\0');
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
ciphertext[i + off] = partial_block[i];
|
||||
|
||||
auto * plaintext_ref = plaintext.data();
|
||||
decryptNBytes(plaintext_ref, ciphertext.data(), off + size, iv);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
*(to++) = plaintext[i + off];
|
||||
}
|
||||
|
||||
void Decryptor::decryptNBytes(BufferBase::Position & to, const char * data, size_t bytes, const InitVector & iv) const
|
||||
{
|
||||
auto evp_ctx_ptr = std::unique_ptr<EVP_CIPHER_CTX, decltype(&::EVP_CIPHER_CTX_free)>(EVP_CIPHER_CTX_new(), &EVP_CIPHER_CTX_free);
|
||||
auto * evp_ctx = evp_ctx_ptr.get();
|
||||
|
||||
if (EVP_DecryptInit_ex(evp_ctx, evp_cipher, nullptr, nullptr, nullptr) != 1)
|
||||
throw Exception("Failed to initialize decryption context with cipher", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
if (EVP_DecryptInit_ex(evp_ctx, nullptr, nullptr,
|
||||
reinterpret_cast<const unsigned char*>(key.str().data()),
|
||||
reinterpret_cast<const unsigned char*>(iv.str().data())) != 1)
|
||||
throw Exception("Failed to set key and IV for decryption", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
int output_len = 0;
|
||||
if (EVP_DecryptUpdate(evp_ctx,
|
||||
reinterpret_cast<unsigned char*>(to), &output_len,
|
||||
reinterpret_cast<const unsigned char*>(data), static_cast<int>(bytes)) != 1)
|
||||
throw Exception("Failed to decrypt", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
to += output_len;
|
||||
|
||||
int final_output_len = 0;
|
||||
if (EVP_DecryptFinal_ex(evp_ctx,
|
||||
reinterpret_cast<unsigned char*>(to), &final_output_len) != 1)
|
||||
throw Exception("Failed to fetch plaintext", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
if (output_len < 0 || final_output_len < 0 || static_cast<size_t>(output_len) + static_cast<size_t>(final_output_len) != bytes)
|
||||
throw Exception("Only part of the data was decrypted", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
}
|
||||
|
||||
String readIV(size_t size, ReadBuffer & in)
|
||||
{
|
||||
String iv(size, 0);
|
||||
in.readStrict(reinterpret_cast<char *>(iv.data()), size);
|
||||
return iv;
|
||||
}
|
||||
|
||||
String randomString(size_t size)
|
||||
{
|
||||
String iv(size, 0);
|
||||
|
||||
std::random_device rd;
|
||||
std::mt19937 gen{rd()};
|
||||
std::uniform_int_distribution<size_t> dis;
|
||||
std::uniform_int_distribution<UInt128::base_type> dis;
|
||||
UInt128 counter;
|
||||
for (size_t i = 0; i != std::size(counter.items); ++i)
|
||||
counter.items[i] = dis(gen);
|
||||
return InitVector{counter};
|
||||
}
|
||||
|
||||
char * ptr = iv.data();
|
||||
while (size)
|
||||
|
||||
Encryptor::Encryptor(const String & key_, const InitVector & iv_)
|
||||
: key(key_)
|
||||
, init_vector(iv_)
|
||||
{
|
||||
if (key_.length() == 16)
|
||||
evp_cipher = EVP_aes_128_ctr();
|
||||
else if (key_.length() == 24)
|
||||
evp_cipher = EVP_aes_192_ctr();
|
||||
else if (key_.length() == 32)
|
||||
evp_cipher = EVP_aes_256_ctr();
|
||||
else
|
||||
throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Key length {} is not supported, supported only keys of length 128, 192, or 256 bits", key_.length());
|
||||
|
||||
size_t cipher_key_length = static_cast<size_t>(EVP_CIPHER_key_length(evp_cipher));
|
||||
if (cipher_key_length != key_.length())
|
||||
throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Got unexpected key length from cipher: {} != {}", cipher_key_length, key_.length());
|
||||
|
||||
size_t cipher_iv_length = static_cast<size_t>(EVP_CIPHER_iv_length(evp_cipher));
|
||||
if (cipher_iv_length != InitVector::kSize)
|
||||
throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Got unexpected init vector's length from cipher: {} != {}", cipher_iv_length, InitVector::kSize);
|
||||
}
|
||||
|
||||
void Encryptor::encrypt(const char * data, size_t size, WriteBuffer & out)
|
||||
{
|
||||
if (!size)
|
||||
return;
|
||||
|
||||
auto current_iv = (init_vector + blocks(offset)).toString();
|
||||
|
||||
auto evp_ctx_ptr = std::unique_ptr<EVP_CIPHER_CTX, decltype(&::EVP_CIPHER_CTX_free)>(EVP_CIPHER_CTX_new(), &EVP_CIPHER_CTX_free);
|
||||
auto * evp_ctx = evp_ctx_ptr.get();
|
||||
|
||||
if (!EVP_EncryptInit_ex(evp_ctx, evp_cipher, nullptr, nullptr, nullptr))
|
||||
throw Exception("Failed to initialize encryption context with cipher", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
if (!EVP_EncryptInit_ex(evp_ctx, nullptr, nullptr,
|
||||
reinterpret_cast<const uint8_t*>(key.c_str()), reinterpret_cast<const uint8_t*>(current_iv.c_str())))
|
||||
throw Exception("Failed to set key and IV for encryption", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
size_t in_size = 0;
|
||||
size_t out_size = 0;
|
||||
|
||||
auto off = blockOffset(offset);
|
||||
if (off)
|
||||
{
|
||||
auto value = dis(gen);
|
||||
size_t n = std::min(size, sizeof(value));
|
||||
memcpy(ptr, &value, n);
|
||||
ptr += n;
|
||||
size -= n;
|
||||
size_t in_part_size = partBlockSize(size, off);
|
||||
size_t out_part_size = encryptBlockWithPadding(evp_ctx, &data[in_size], in_part_size, off, out);
|
||||
in_size += in_part_size;
|
||||
out_size += out_part_size;
|
||||
}
|
||||
|
||||
return iv;
|
||||
if (in_size < size)
|
||||
{
|
||||
size_t in_part_size = size - in_size;
|
||||
size_t out_part_size = encryptBlocks(evp_ctx, &data[in_size], in_part_size, out);
|
||||
in_size += in_part_size;
|
||||
out_size += out_part_size;
|
||||
}
|
||||
|
||||
out_size += encryptFinal(evp_ctx, out);
|
||||
|
||||
if (out_size != in_size)
|
||||
throw Exception("Only part of the data was encrypted", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
offset += in_size;
|
||||
}
|
||||
|
||||
void writeIV(const String & iv, WriteBuffer & out)
|
||||
void Encryptor::decrypt(const char * data, size_t size, char * out)
|
||||
{
|
||||
out.write(iv.data(), iv.length());
|
||||
if (!size)
|
||||
return;
|
||||
|
||||
auto current_iv = (init_vector + blocks(offset)).toString();
|
||||
|
||||
auto evp_ctx_ptr = std::unique_ptr<EVP_CIPHER_CTX, decltype(&::EVP_CIPHER_CTX_free)>(EVP_CIPHER_CTX_new(), &EVP_CIPHER_CTX_free);
|
||||
auto * evp_ctx = evp_ctx_ptr.get();
|
||||
|
||||
if (!EVP_DecryptInit_ex(evp_ctx, evp_cipher, nullptr, nullptr, nullptr))
|
||||
throw Exception("Failed to initialize decryption context with cipher", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
if (!EVP_DecryptInit_ex(evp_ctx, nullptr, nullptr,
|
||||
reinterpret_cast<const uint8_t*>(key.c_str()), reinterpret_cast<const uint8_t*>(current_iv.c_str())))
|
||||
throw Exception("Failed to set key and IV for decryption", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
|
||||
size_t in_size = 0;
|
||||
size_t out_size = 0;
|
||||
|
||||
auto off = blockOffset(offset);
|
||||
if (off)
|
||||
{
|
||||
size_t in_part_size = partBlockSize(size, off);
|
||||
size_t out_part_size = decryptBlockWithPadding(evp_ctx, &data[in_size], in_part_size, off, &out[out_size]);
|
||||
in_size += in_part_size;
|
||||
out_size += out_part_size;
|
||||
}
|
||||
|
||||
if (in_size < size)
|
||||
{
|
||||
size_t in_part_size = size - in_size;
|
||||
size_t out_part_size = decryptBlocks(evp_ctx, &data[in_size], in_part_size, &out[out_size]);
|
||||
in_size += in_part_size;
|
||||
out_size += out_part_size;
|
||||
}
|
||||
|
||||
out_size += decryptFinal(evp_ctx, &out[out_size]);
|
||||
|
||||
if (out_size != in_size)
|
||||
throw Exception("Only part of the data was decrypted", ErrorCodes::DATA_ENCRYPTION_ERROR);
|
||||
offset += in_size;
|
||||
}
|
||||
|
||||
size_t cipherKeyLength(const EVP_CIPHER * evp_cipher)
|
||||
bool isKeyLengthSupported(size_t key_length)
|
||||
{
|
||||
return static_cast<size_t>(EVP_CIPHER_key_length(evp_cipher));
|
||||
}
|
||||
|
||||
size_t cipherIVLength(const EVP_CIPHER * evp_cipher)
|
||||
{
|
||||
return static_cast<size_t>(EVP_CIPHER_iv_length(evp_cipher));
|
||||
}
|
||||
|
||||
const EVP_CIPHER * defaultCipher()
|
||||
{
|
||||
return EVP_aes_128_ctr();
|
||||
return (key_length == 16) || (key_length == 24) || (key_length == 32);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -16,87 +16,82 @@ class WriteBuffer;
|
||||
namespace FileEncryption
|
||||
{
|
||||
|
||||
constexpr size_t kIVSize = sizeof(UInt128);
|
||||
|
||||
/// Initialization vector. Its size is always 16 bytes.
|
||||
class InitVector
|
||||
{
|
||||
public:
|
||||
InitVector(const String & iv_);
|
||||
const String & str() const;
|
||||
void inc() { ++counter; }
|
||||
void inc(size_t n) { counter += n; }
|
||||
void set(size_t n) { counter = n; }
|
||||
static constexpr const size_t kSize = 16;
|
||||
|
||||
InitVector() = default;
|
||||
explicit InitVector(const UInt128 & counter_) { set(counter_); }
|
||||
|
||||
void set(const UInt128 & counter_) { counter = counter_; }
|
||||
UInt128 get() const { return counter; }
|
||||
|
||||
void read(ReadBuffer & in);
|
||||
void write(WriteBuffer & out) const;
|
||||
|
||||
/// Write 16 bytes of the counter to a string in big endian order.
|
||||
/// We need big endian because the used cipher algorithms treat an initialization vector as a counter in big endian.
|
||||
String toString() const;
|
||||
|
||||
/// Converts a string of 16 bytes length in big endian order to a counter.
|
||||
static InitVector fromString(const String & str_);
|
||||
|
||||
/// Adds a specified offset to the counter.
|
||||
InitVector & operator++() { ++counter; return *this; }
|
||||
InitVector operator++(int) { InitVector res = *this; ++counter; return res; }
|
||||
InitVector & operator+=(size_t offset) { counter += offset; return *this; }
|
||||
InitVector operator+(size_t offset) const { InitVector res = *this; return res += offset; }
|
||||
|
||||
/// Generates a random initialization vector.
|
||||
static InitVector random();
|
||||
|
||||
private:
|
||||
UInt128 iv;
|
||||
UInt128 counter = 0;
|
||||
mutable String local;
|
||||
};
|
||||
|
||||
|
||||
class EncryptionKey
|
||||
/// Encrypts or decrypts data.
|
||||
class Encryptor
|
||||
{
|
||||
public:
|
||||
EncryptionKey(const String & key_) : key(key_) { }
|
||||
size_t size() const { return key.size(); }
|
||||
const String & str() const { return key; }
|
||||
/// The `key` should have length 128 or 192 or 256.
|
||||
/// According to the key's length aes_128_ctr or aes_192_ctr or aes_256_ctr will be used for encryption.
|
||||
/// We chose to use CTR cipther algorithms because they have the following features which are important for us:
|
||||
/// - No right padding, so we can append encrypted files without deciphering;
|
||||
/// - One byte is always ciphered as one byte, so we get random access to encrypted files easily.
|
||||
Encryptor(const String & key_, const InitVector & iv_);
|
||||
|
||||
/// Sets the current position in the data stream from the very beginning of data.
|
||||
/// It affects how the data will be encrypted or decrypted because
|
||||
/// the initialization vector is increased by an index of the current block
|
||||
/// and the index of the current block is calculated from this offset.
|
||||
void setOffset(size_t offset_) { offset = offset_; }
|
||||
|
||||
/// Encrypts some data.
|
||||
/// Also the function moves `offset` by `size` (for successive encryptions).
|
||||
void encrypt(const char * data, size_t size, WriteBuffer & out);
|
||||
|
||||
/// Decrypts some data.
|
||||
/// The used cipher algorithms generate the same number of bytes in output as they were in input,
|
||||
/// so the function always writes `size` bytes of the plaintext to `out`.
|
||||
/// Also the function moves `offset` by `size` (for successive decryptions).
|
||||
void decrypt(const char * data, size_t size, char * out);
|
||||
|
||||
private:
|
||||
String key;
|
||||
};
|
||||
|
||||
|
||||
class Encryption
|
||||
{
|
||||
public:
|
||||
Encryption(const String & iv_, const EncryptionKey & key_, size_t offset_);
|
||||
|
||||
protected:
|
||||
size_t blockOffset(size_t pos) const { return pos % block_size; }
|
||||
size_t blocks(size_t pos) const { return pos / block_size; }
|
||||
size_t partBlockSize(size_t size, size_t off) const;
|
||||
const EVP_CIPHER * get() const { return evp_cipher; }
|
||||
|
||||
const String key;
|
||||
const InitVector init_vector;
|
||||
const EVP_CIPHER * evp_cipher;
|
||||
const String init_vector;
|
||||
const EncryptionKey key;
|
||||
size_t block_size;
|
||||
|
||||
/// absolute offset
|
||||
/// The current position in the data stream from the very beginning of data.
|
||||
size_t offset = 0;
|
||||
};
|
||||
|
||||
|
||||
class Encryptor : public Encryption
|
||||
{
|
||||
public:
|
||||
using Encryption::Encryption;
|
||||
void encrypt(const char * plaintext, WriteBuffer & buf, size_t size);
|
||||
|
||||
private:
|
||||
String encryptPartialBlock(const char * partial_block, size_t size, const InitVector & iv, size_t off) const;
|
||||
String encryptNBytes(const char * data, size_t bytes, const InitVector & iv) const;
|
||||
};
|
||||
|
||||
|
||||
class Decryptor : public Encryption
|
||||
{
|
||||
public:
|
||||
Decryptor(const String & iv_, const EncryptionKey & key_) : Encryption(iv_, key_, 0) { }
|
||||
void decrypt(const char * ciphertext, char * buf, size_t size, size_t off);
|
||||
|
||||
private:
|
||||
void decryptPartialBlock(char *& to, const char * partial_block, size_t size, const InitVector & iv, size_t off) const;
|
||||
void decryptNBytes(char *& to, const char * data, size_t bytes, const InitVector & iv) const;
|
||||
};
|
||||
|
||||
|
||||
String readIV(size_t size, ReadBuffer & in);
|
||||
String randomString(size_t size);
|
||||
void writeIV(const String & iv, WriteBuffer & out);
|
||||
size_t cipherKeyLength(const EVP_CIPHER * evp_cipher);
|
||||
size_t cipherIVLength(const EVP_CIPHER * evp_cipher);
|
||||
const EVP_CIPHER * defaultCipher();
|
||||
/// Checks whether a passed key length is supported, i.e.
|
||||
/// whether its length is 128 or 192 or 256 bits (16 or 24 or 32 bytes).
|
||||
bool isKeyLengthSupported(size_t key_length);
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -9,91 +9,95 @@ namespace ErrorCodes
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
using InitVector = FileEncryption::InitVector;
|
||||
|
||||
ReadBufferFromEncryptedFile::ReadBufferFromEncryptedFile(
|
||||
size_t buf_size_,
|
||||
size_t buffer_size_,
|
||||
std::unique_ptr<ReadBufferFromFileBase> in_,
|
||||
const String & init_vector_,
|
||||
const FileEncryption::EncryptionKey & key_,
|
||||
const size_t iv_offset_)
|
||||
: ReadBufferFromFileBase(buf_size_, nullptr, 0)
|
||||
const String & key_,
|
||||
const InitVector & init_vector_)
|
||||
: ReadBufferFromFileBase(buffer_size_, nullptr, 0)
|
||||
, in(std::move(in_))
|
||||
, buf_size(buf_size_)
|
||||
, decryptor(FileEncryption::Decryptor(init_vector_, key_))
|
||||
, iv_offset(iv_offset_)
|
||||
, encrypted_buffer(buffer_size_)
|
||||
, encryptor(key_, init_vector_)
|
||||
{
|
||||
/// We should start reading from `in` at the offset == InitVector::kSize.
|
||||
need_seek = true;
|
||||
}
|
||||
|
||||
off_t ReadBufferFromEncryptedFile::seek(off_t off, int whence)
|
||||
{
|
||||
if (whence == SEEK_CUR)
|
||||
{
|
||||
if (off < 0 && -off > getPosition())
|
||||
throw Exception("SEEK_CUR shift out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (!working_buffer.empty() && static_cast<size_t>(offset() + off) < working_buffer.size())
|
||||
{
|
||||
pos += off;
|
||||
return getPosition();
|
||||
}
|
||||
else
|
||||
start_pos = off + getPosition();
|
||||
}
|
||||
else if (whence == SEEK_SET)
|
||||
off_t new_pos;
|
||||
if (whence == SEEK_SET)
|
||||
{
|
||||
if (off < 0)
|
||||
throw Exception("SEEK_SET underflow: off = " + std::to_string(off), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (!working_buffer.empty() && static_cast<size_t>(off) >= start_pos
|
||||
&& static_cast<size_t>(off) < (start_pos + working_buffer.size()))
|
||||
{
|
||||
pos = working_buffer.begin() + (off - start_pos);
|
||||
return getPosition();
|
||||
}
|
||||
else
|
||||
start_pos = off;
|
||||
new_pos = off;
|
||||
}
|
||||
else if (whence == SEEK_CUR)
|
||||
{
|
||||
if (off < 0 && -off > getPosition())
|
||||
throw Exception("SEEK_CUR shift out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
new_pos = getPosition() + off;
|
||||
}
|
||||
else
|
||||
throw Exception("ReadBufferFromEncryptedFile::seek expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
throw Exception("ReadBufferFromFileEncrypted::seek expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
initialize();
|
||||
return start_pos;
|
||||
if ((offset - static_cast<off_t>(working_buffer.size()) <= new_pos) && (new_pos <= offset) && !need_seek)
|
||||
{
|
||||
/// Position is still inside buffer.
|
||||
pos = working_buffer.end() - offset + new_pos;
|
||||
assert(pos >= working_buffer.begin());
|
||||
assert(pos <= working_buffer.end());
|
||||
}
|
||||
else
|
||||
{
|
||||
need_seek = true;
|
||||
offset = new_pos;
|
||||
|
||||
/// No more reading from the current working buffer until next() is called.
|
||||
pos = working_buffer.end();
|
||||
assert(!hasPendingData());
|
||||
}
|
||||
|
||||
/// The encryptor always needs to know what the current offset is.
|
||||
encryptor.setOffset(new_pos);
|
||||
|
||||
return new_pos;
|
||||
}
|
||||
|
||||
off_t ReadBufferFromEncryptedFile::getPosition()
|
||||
{
|
||||
return offset - available();
|
||||
}
|
||||
|
||||
bool ReadBufferFromEncryptedFile::nextImpl()
|
||||
{
|
||||
if (need_seek)
|
||||
{
|
||||
off_t raw_offset = offset + InitVector::kSize;
|
||||
if (in->seek(raw_offset, SEEK_SET) != raw_offset)
|
||||
return false;
|
||||
need_seek = false;
|
||||
}
|
||||
|
||||
if (in->eof())
|
||||
return false;
|
||||
|
||||
if (initialized)
|
||||
start_pos += working_buffer.size();
|
||||
initialize();
|
||||
return true;
|
||||
}
|
||||
|
||||
void ReadBufferFromEncryptedFile::initialize()
|
||||
{
|
||||
size_t in_pos = start_pos + iv_offset;
|
||||
|
||||
String data;
|
||||
data.resize(buf_size);
|
||||
size_t data_size = 0;
|
||||
|
||||
in->seek(in_pos, SEEK_SET);
|
||||
while (data_size < buf_size && !in->eof())
|
||||
/// Read up to the size of `encrypted_buffer`.
|
||||
size_t bytes_read = 0;
|
||||
while (bytes_read < encrypted_buffer.size() && !in->eof())
|
||||
{
|
||||
auto size = in->read(data.data() + data_size, buf_size - data_size);
|
||||
data_size += size;
|
||||
in_pos += size;
|
||||
in->seek(in_pos, SEEK_SET);
|
||||
bytes_read += in->read(encrypted_buffer.data() + bytes_read, encrypted_buffer.size() - bytes_read);
|
||||
}
|
||||
|
||||
data.resize(data_size);
|
||||
working_buffer.resize(data_size);
|
||||
|
||||
decryptor.decrypt(data.data(), working_buffer.begin(), data_size, start_pos);
|
||||
/// The used cipher algorithms generate the same number of bytes in output as it were in input,
|
||||
/// so after deciphering the numbers of bytes will be still `bytes_read`.
|
||||
working_buffer.resize(bytes_read);
|
||||
encryptor.decrypt(encrypted_buffer.data(), bytes_read, working_buffer.begin());
|
||||
|
||||
pos = working_buffer.begin();
|
||||
initialized = true;
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,39 +12,33 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Reads data from the underlying read buffer and decrypts it.
|
||||
class ReadBufferFromEncryptedFile : public ReadBufferFromFileBase
|
||||
{
|
||||
public:
|
||||
ReadBufferFromEncryptedFile(
|
||||
size_t buf_size_,
|
||||
size_t buffer_size_,
|
||||
std::unique_ptr<ReadBufferFromFileBase> in_,
|
||||
const String & init_vector_,
|
||||
const FileEncryption::EncryptionKey & key_,
|
||||
const size_t iv_offset_);
|
||||
const String & key_,
|
||||
const FileEncryption::InitVector & init_vector_);
|
||||
|
||||
off_t seek(off_t off, int whence) override;
|
||||
|
||||
off_t getPosition() override { return start_pos + offset(); }
|
||||
off_t getPosition() override;
|
||||
|
||||
std::string getFileName() const override { return in->getFileName(); }
|
||||
|
||||
private:
|
||||
bool nextImpl() override;
|
||||
|
||||
void initialize();
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> in;
|
||||
size_t buf_size;
|
||||
|
||||
FileEncryption::Decryptor decryptor;
|
||||
bool initialized = false;
|
||||
off_t offset = 0;
|
||||
bool need_seek = false;
|
||||
|
||||
// current working_buffer.begin() offset from decrypted file
|
||||
size_t start_pos = 0;
|
||||
size_t iv_offset = 0;
|
||||
Memory<> encrypted_buffer;
|
||||
FileEncryption::Encryptor encryptor;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
||||
#endif
|
||||
|
@ -6,18 +6,21 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using InitVector = FileEncryption::InitVector;
|
||||
|
||||
WriteBufferFromEncryptedFile::WriteBufferFromEncryptedFile(
|
||||
size_t buf_size_,
|
||||
size_t buffer_size_,
|
||||
std::unique_ptr<WriteBufferFromFileBase> out_,
|
||||
const String & init_vector_,
|
||||
const FileEncryption::EncryptionKey & key_,
|
||||
const size_t & file_size)
|
||||
: WriteBufferFromFileBase(buf_size_, nullptr, 0)
|
||||
const String & key_,
|
||||
const InitVector & init_vector_,
|
||||
size_t old_file_size)
|
||||
: WriteBufferFromFileBase(buffer_size_, nullptr, 0)
|
||||
, out(std::move(out_))
|
||||
, flush_iv(!file_size)
|
||||
, iv(init_vector_)
|
||||
, encryptor(FileEncryption::Encryptor(init_vector_, key_, file_size))
|
||||
, flush_iv(!old_file_size)
|
||||
, encryptor(key_, init_vector_)
|
||||
{
|
||||
encryptor.setOffset(old_file_size);
|
||||
}
|
||||
|
||||
WriteBufferFromEncryptedFile::~WriteBufferFromEncryptedFile()
|
||||
@ -51,6 +54,11 @@ void WriteBufferFromEncryptedFile::finishImpl()
|
||||
{
|
||||
/// If buffer has pending data - write it.
|
||||
next();
|
||||
|
||||
/// Note that if there is no data to write an empty file will be written, even without the initialization vector
|
||||
/// (see nextImpl(): it writes the initialization vector only if there is some data ready to write).
|
||||
/// That's fine because DiskEncrypted allows files without initialization vectors when they're empty.
|
||||
|
||||
out->finalize();
|
||||
}
|
||||
|
||||
@ -58,6 +66,7 @@ void WriteBufferFromEncryptedFile::sync()
|
||||
{
|
||||
/// If buffer has pending data - write it.
|
||||
next();
|
||||
|
||||
out->sync();
|
||||
}
|
||||
|
||||
@ -68,12 +77,13 @@ void WriteBufferFromEncryptedFile::nextImpl()
|
||||
|
||||
if (flush_iv)
|
||||
{
|
||||
FileEncryption::writeIV(iv, *out);
|
||||
iv.write(*out);
|
||||
flush_iv = false;
|
||||
}
|
||||
|
||||
encryptor.encrypt(working_buffer.begin(), *out, offset());
|
||||
encryptor.encrypt(working_buffer.begin(), offset(), *out);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -12,15 +12,17 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Encrypts data and writes the encrypted data to the underlying write buffer.
|
||||
class WriteBufferFromEncryptedFile : public WriteBufferFromFileBase
|
||||
{
|
||||
public:
|
||||
/// `old_file_size` should be set to non-zero if we're going to append an existing file.
|
||||
WriteBufferFromEncryptedFile(
|
||||
size_t buf_size_,
|
||||
size_t buffer_size_,
|
||||
std::unique_ptr<WriteBufferFromFileBase> out_,
|
||||
const String & init_vector_,
|
||||
const FileEncryption::EncryptionKey & key_,
|
||||
const size_t & file_size);
|
||||
const String & key_,
|
||||
const FileEncryption::InitVector & init_vector_,
|
||||
size_t old_file_size = 0);
|
||||
~WriteBufferFromEncryptedFile() override;
|
||||
|
||||
void sync() override;
|
||||
@ -37,8 +39,9 @@ private:
|
||||
bool finished = false;
|
||||
std::unique_ptr<WriteBufferFromFileBase> out;
|
||||
|
||||
bool flush_iv;
|
||||
String iv;
|
||||
FileEncryption::InitVector iv;
|
||||
bool flush_iv = false;
|
||||
|
||||
FileEncryption::Encryptor encryptor;
|
||||
};
|
||||
|
||||
|
@ -15,25 +15,15 @@ struct InitVectorTestParam
|
||||
{
|
||||
const std::string_view comment;
|
||||
const String init;
|
||||
UInt128 adder;
|
||||
UInt128 setter;
|
||||
const String after_inc;
|
||||
UInt64 adder;
|
||||
const String after_add;
|
||||
const String after_set;
|
||||
};
|
||||
|
||||
|
||||
class InitVectorTest : public ::testing::TestWithParam<InitVectorTestParam> {};
|
||||
|
||||
|
||||
String string_ends_with(size_t size, String str)
|
||||
{
|
||||
String res(size, 0);
|
||||
res.replace(size - str.size(), str.size(), str);
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
static std::ostream & operator << (std::ostream & ostr, const InitVectorTestParam & param)
|
||||
{
|
||||
return ostr << param.comment;
|
||||
@ -44,20 +34,14 @@ TEST_P(InitVectorTest, InitVector)
|
||||
{
|
||||
const auto & param = GetParam();
|
||||
|
||||
auto iv = InitVector(param.init);
|
||||
ASSERT_EQ(param.init, iv.str());
|
||||
auto iv = InitVector::fromString(param.init);
|
||||
ASSERT_EQ(param.init, iv.toString());
|
||||
|
||||
iv.inc();
|
||||
ASSERT_EQ(param.after_inc, iv.str());
|
||||
++iv;
|
||||
ASSERT_EQ(param.after_inc, iv.toString());
|
||||
|
||||
iv.inc(param.adder);
|
||||
ASSERT_EQ(param.after_add, iv.str());
|
||||
|
||||
iv.set(param.setter);
|
||||
ASSERT_EQ(param.after_set, iv.str());
|
||||
|
||||
iv.set(0);
|
||||
ASSERT_EQ(param.init, iv.str());
|
||||
iv += param.adder;
|
||||
ASSERT_EQ(param.after_add, iv.toString());
|
||||
}
|
||||
|
||||
|
||||
@ -65,31 +49,32 @@ INSTANTIATE_TEST_SUITE_P(InitVectorInputs,
|
||||
InitVectorTest,
|
||||
::testing::ValuesIn(std::initializer_list<InitVectorTestParam>{
|
||||
{
|
||||
"Basic init vector test. Get zero-string, add 0, set 0",
|
||||
"Basic init vector test. Get zero-string, add 1, add 0",
|
||||
String(16, 0),
|
||||
String("\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01", 16),
|
||||
0,
|
||||
0,
|
||||
string_ends_with(16, "\x1"),
|
||||
string_ends_with(16, "\x1"),
|
||||
String(16, 0),
|
||||
String("\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01", 16),
|
||||
},
|
||||
{
|
||||
"Init vector test. Get zero-string, add 85, set 1024",
|
||||
"Init vector test. Get zero-string, add 1, add 85, add 1024",
|
||||
String(16, 0),
|
||||
String("\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01", 16),
|
||||
85,
|
||||
String("\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x56", 16),
|
||||
},
|
||||
{
|
||||
"Init vector test #2. Get zero-string, add 1, add 1024",
|
||||
String(16, 0),
|
||||
String("\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01", 16),
|
||||
1024,
|
||||
string_ends_with(16, "\x1"),
|
||||
string_ends_with(16, "\x56"),
|
||||
string_ends_with(16, String("\x4\0", 2)),
|
||||
String("\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x04\x01", 16)
|
||||
},
|
||||
{
|
||||
"Long init vector test",
|
||||
"\xa8\x65\x9c\x73\xf8\x5d\x83\xb4\x5c\xa6\x8c\x19\xf4\x77\x80\xe1",
|
||||
3349249125638641,
|
||||
1698923461902341,
|
||||
"\xa8\x65\x9c\x73\xf8\x5d\x83\xb4\x5c\xa6\x8c\x19\xf4\x77\x80\xe2",
|
||||
"\xa8\x65\x9c\x73\xf8\x5d\x83\xb4\x5c\xb2\x72\x39\xc8\xdd\x62\xd3",
|
||||
String("\xa8\x65\x9c\x73\xf8\x5d\x83\xb4\x5c\xac\x95\x43\x65\xea\x00\xe6", 16)
|
||||
String("\xa8\x65\x9c\x73\xf8\x5d\x83\xb4\x9c\xa6\x8c\x19\xf4\x77\x80\xe1", 16),
|
||||
String("\xa8\x65\x9c\x73\xf8\x5d\x83\xb4\x9c\xa6\x8c\x19\xf4\x77\x80\xe2", 16),
|
||||
9349249176525638641ULL,
|
||||
String("\xa8\x65\x9c\x73\xf8\x5d\x83\xb5\x1e\x65\xc0\xb1\x67\xe4\x0c\xd3", 16)
|
||||
},
|
||||
})
|
||||
);
|
||||
@ -97,53 +82,87 @@ INSTANTIATE_TEST_SUITE_P(InitVectorInputs,
|
||||
|
||||
TEST(FileEncryption, Encryption)
|
||||
{
|
||||
String iv(16, 0);
|
||||
EncryptionKey key("1234567812345678");
|
||||
String input = "abcd1234efgh5678ijkl";
|
||||
String expected = "\xfb\x8a\x9e\x66\x82\x72\x1b\xbe\x6b\x1d\xd8\x98\xc5\x8c\x63\xee\xcd\x36\x4a\x50";
|
||||
String key = "1234567812345678";
|
||||
InitVector iv;
|
||||
Encryptor encryptor{key, iv};
|
||||
|
||||
std::string_view input = "abcd1234efgh5678ijkl";
|
||||
std::string_view expected = "\xfb\x8a\x9e\x66\x82\x72\x1b\xbe\x6b\x1d\xd8\x98\xc5\x8c\x63\xee\xcd\x36\x4a\x50";
|
||||
|
||||
for (size_t i = 0; i < expected.size(); ++i)
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
encryptor.encrypt(&input[i], 1, buf);
|
||||
ASSERT_EQ(expected.substr(i, 1), buf.str());
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < expected.size(); ++i)
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
encryptor.setOffset(i);
|
||||
encryptor.encrypt(&input[i], 1, buf);
|
||||
ASSERT_EQ(expected.substr(i, 1), buf.str());
|
||||
}
|
||||
|
||||
String result(expected.size(), 0);
|
||||
for (size_t i = 0; i <= expected.size(); ++i)
|
||||
{
|
||||
auto buf = WriteBufferFromString(result);
|
||||
auto encryptor = Encryptor(iv, key, 0);
|
||||
encryptor.encrypt(input.data(), buf, i);
|
||||
ASSERT_EQ(expected.substr(0, i), result.substr(0, i));
|
||||
WriteBufferFromOwnString buf;
|
||||
encryptor.setOffset(0);
|
||||
encryptor.encrypt(input.data(), i, buf);
|
||||
ASSERT_EQ(expected.substr(0, i), buf.str());
|
||||
}
|
||||
|
||||
size_t offset = 25;
|
||||
String offset_expected = "\x6c\x67\xe4\xf5\x8f\x86\xb0\x19\xe5\xcd\x53\x59\xe0\xc6\x01\x5e\xc1\xfd\x60\x9d";
|
||||
std::string_view offset_expected = "\x6c\x67\xe4\xf5\x8f\x86\xb0\x19\xe5\xcd\x53\x59\xe0\xc6\x01\x5e\xc1\xfd\x60\x9d";
|
||||
for (size_t i = 0; i <= expected.size(); ++i)
|
||||
{
|
||||
auto buf = WriteBufferFromString(result);
|
||||
auto encryptor = Encryptor(iv, key, offset);
|
||||
encryptor.encrypt(input.data(), buf, i);
|
||||
ASSERT_EQ(offset_expected.substr(0, i), result.substr(0, i));
|
||||
WriteBufferFromOwnString buf;
|
||||
encryptor.setOffset(offset);
|
||||
encryptor.encrypt(input.data(), i, buf);
|
||||
ASSERT_EQ(offset_expected.substr(0, i), buf.str());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
TEST(FileEncryption, Decryption)
|
||||
{
|
||||
String iv(16, 0);
|
||||
EncryptionKey key("1234567812345678");
|
||||
String expected = "abcd1234efgh5678ijkl";
|
||||
String input = "\xfb\x8a\x9e\x66\x82\x72\x1b\xbe\x6b\x1d\xd8\x98\xc5\x8c\x63\xee\xcd\x36\x4a\x50";
|
||||
auto decryptor = Decryptor(iv, key);
|
||||
String result(expected.size(), 0);
|
||||
String key("1234567812345678");
|
||||
InitVector iv;
|
||||
Encryptor encryptor{key, iv};
|
||||
|
||||
std::string_view input = "\xfb\x8a\x9e\x66\x82\x72\x1b\xbe\x6b\x1d\xd8\x98\xc5\x8c\x63\xee\xcd\x36\x4a\x50";
|
||||
std::string_view expected = "abcd1234efgh5678ijkl";
|
||||
|
||||
for (size_t i = 0; i < expected.size(); ++i)
|
||||
{
|
||||
char c;
|
||||
encryptor.decrypt(&input[i], 1, &c);
|
||||
ASSERT_EQ(expected[i], c);
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < expected.size(); ++i)
|
||||
{
|
||||
char c;
|
||||
encryptor.setOffset(i);
|
||||
encryptor.decrypt(&input[i], 1, &c);
|
||||
ASSERT_EQ(expected[i], c);
|
||||
}
|
||||
|
||||
String buf(expected.size(), 0);
|
||||
for (size_t i = 0; i <= expected.size(); ++i)
|
||||
{
|
||||
decryptor.decrypt(input.data(), result.data(), i, 0);
|
||||
ASSERT_EQ(expected.substr(0, i), result.substr(0, i));
|
||||
encryptor.setOffset(0);
|
||||
encryptor.decrypt(input.data(), i, buf.data());
|
||||
ASSERT_EQ(expected.substr(0, i), buf.substr(0, i));
|
||||
}
|
||||
|
||||
size_t offset = 25;
|
||||
String offset_input = "\x6c\x67\xe4\xf5\x8f\x86\xb0\x19\xe5\xcd\x53\x59\xe0\xc6\x01\x5e\xc1\xfd\x60\x9d";
|
||||
for (size_t i = 0; i <= expected.size(); ++i)
|
||||
{
|
||||
decryptor.decrypt(offset_input.data(), result.data(), i, offset);
|
||||
ASSERT_EQ(expected.substr(0, i), result.substr(0, i));
|
||||
encryptor.setOffset(offset);
|
||||
encryptor.decrypt(offset_input.data(), i, buf.data());
|
||||
ASSERT_EQ(expected.substr(0, i), buf.substr(0, i));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -25,7 +25,7 @@
|
||||
<type>encrypted</type>
|
||||
<disk>disk_local</disk>
|
||||
<path>encrypted/</path>
|
||||
<key>abcdefghijklmnop</key>
|
||||
<key_hex>109105c600c12066f82f1a4dbb41a08e</key_hex>
|
||||
</disk_local_encrypted>
|
||||
</disks>
|
||||
<policies>
|
||||
|
Loading…
Reference in New Issue
Block a user