Add comments for encrypted disks; some improvements in code.

This commit is contained in:
Vitaly Baranov 2021-07-11 22:26:39 +03:00
parent c6177bd0cc
commit 9cc258327e
13 changed files with 616 additions and 492 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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