mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Fix synchronization while updating from the config of an encrypted disk.
This commit is contained in:
parent
71d20659c9
commit
a05184d83a
@ -16,6 +16,7 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int INCORRECT_DISK_INDEX;
|
||||
extern const int DATA_ENCRYPTION_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -37,80 +38,121 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
struct DiskEncryptedSettings
|
||||
std::unique_ptr<const DiskEncryptedSettings> parseDiskEncryptedSettings(
|
||||
const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const DisksMap & map)
|
||||
{
|
||||
DiskPtr wrapped_disk;
|
||||
String path_on_wrapped_disk;
|
||||
std::unordered_map<UInt64, String> keys;
|
||||
UInt64 current_key_id;
|
||||
Algorithm current_algorithm;
|
||||
|
||||
DiskEncryptedSettings(
|
||||
const String & disk_name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const DisksMap & map)
|
||||
try
|
||||
{
|
||||
try
|
||||
{
|
||||
current_algorithm = DEFAULT_ENCRYPTION_ALGORITHM;
|
||||
if (config.has(config_prefix + ".algorithm"))
|
||||
parseFromString(current_algorithm, config.getString(config_prefix + ".algorithm"));
|
||||
auto res = std::make_unique<DiskEncryptedSettings>();
|
||||
res->current_algorithm = DEFAULT_ENCRYPTION_ALGORITHM;
|
||||
if (config.has(config_prefix + ".algorithm"))
|
||||
parseFromString(res->current_algorithm, config.getString(config_prefix + ".algorithm"));
|
||||
|
||||
Strings config_keys;
|
||||
config.keys(config_prefix, config_keys);
|
||||
for (const std::string & config_key : config_keys)
|
||||
Strings config_keys;
|
||||
config.keys(config_prefix, config_keys);
|
||||
for (const std::string & config_key : config_keys)
|
||||
{
|
||||
String key;
|
||||
UInt64 key_id;
|
||||
|
||||
if ((config_key == "key") || config_key.starts_with("key["))
|
||||
{
|
||||
String key;
|
||||
UInt64 key_id;
|
||||
|
||||
if ((config_key == "key") || config_key.starts_with("key["))
|
||||
{
|
||||
key = config.getString(config_prefix + "." + config_key, "");
|
||||
key_id = config.getUInt64(config_prefix + "." + config_key + "[@id]", 0);
|
||||
}
|
||||
else if ((config_key == "key_hex") || config_key.starts_with("key_hex["))
|
||||
{
|
||||
key = unhexKey(config.getString(config_prefix + "." + config_key, ""));
|
||||
key_id = config.getUInt64(config_prefix + "." + config_key + "[@id]", 0);
|
||||
}
|
||||
else
|
||||
continue;
|
||||
|
||||
auto it = keys.find(key_id);
|
||||
if (it != keys.end())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple keys have the same ID {}", key_id);
|
||||
keys[key_id] = key;
|
||||
key = config.getString(config_prefix + "." + config_key, "");
|
||||
key_id = config.getUInt64(config_prefix + "." + config_key + "[@id]", 0);
|
||||
}
|
||||
else if ((config_key == "key_hex") || config_key.starts_with("key_hex["))
|
||||
{
|
||||
key = unhexKey(config.getString(config_prefix + "." + config_key, ""));
|
||||
key_id = config.getUInt64(config_prefix + "." + config_key + "[@id]", 0);
|
||||
}
|
||||
else
|
||||
continue;
|
||||
|
||||
if (keys.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No keys, an encrypted disk needs keys to work", current_key_id);
|
||||
|
||||
current_key_id = config.getUInt64(config_prefix + ".current_key_id", 0);
|
||||
if (!keys.contains(current_key_id))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key with ID {} not found", current_key_id);
|
||||
FileEncryption::checkKeySize(current_algorithm, keys[current_key_id].size());
|
||||
|
||||
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");
|
||||
|
||||
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 {}",
|
||||
wrapped_disk_name);
|
||||
wrapped_disk = wrapped_disk_it->second;
|
||||
|
||||
path_on_wrapped_disk = config.getString(config_prefix + ".path", "");
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("Disk " + disk_name);
|
||||
throw;
|
||||
if (res->keys.contains(key_id))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple keys have the same ID {}", key_id);
|
||||
res->keys[key_id] = key;
|
||||
}
|
||||
|
||||
if (res->keys.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No keys, an encrypted disk needs keys to work");
|
||||
|
||||
res->current_key_id = config.getUInt64(config_prefix + ".current_key_id", 0);
|
||||
if (!res->keys.contains(res->current_key_id))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found a key with the current ID {}", res->current_key_id);
|
||||
FileEncryption::checkKeySize(res->current_algorithm, res->keys[res->current_key_id].size());
|
||||
|
||||
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. Encrypted disk is a wrapper over another disk");
|
||||
|
||||
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 {}",
|
||||
wrapped_disk_name);
|
||||
res->wrapped_disk = wrapped_disk_it->second;
|
||||
|
||||
res->disk_path = config.getString(config_prefix + ".path", "");
|
||||
if (!res->disk_path.empty() && (res->disk_path.back() != '/'))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk path must ends with '/', but '{}' doesn't.", quoteString(res->disk_path));
|
||||
|
||||
return res;
|
||||
}
|
||||
};
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("Disk " + name);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
FileEncryption::Header readHeader(ReadBufferFromFileBase & read_buffer)
|
||||
{
|
||||
try
|
||||
{
|
||||
FileEncryption::Header header;
|
||||
header.read(read_buffer);
|
||||
return header;
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("While reading the header of encrypted file " + quoteString(read_buffer.getFileName()));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
String getCurrentKey(const String & path, const DiskEncryptedSettings & settings)
|
||||
{
|
||||
auto it = settings.keys.find(settings.current_key_id);
|
||||
if (it == settings.keys.end())
|
||||
throw Exception(
|
||||
ErrorCodes::DATA_ENCRYPTION_ERROR,
|
||||
"Not found a key with the current ID {} required to cipher file {}",
|
||||
settings.current_key_id,
|
||||
quoteString(path));
|
||||
|
||||
return it->second;
|
||||
}
|
||||
|
||||
String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings)
|
||||
{
|
||||
auto it = settings.keys.find(header.key_id);
|
||||
if (it == settings.keys.end())
|
||||
throw Exception(
|
||||
ErrorCodes::DATA_ENCRYPTION_ERROR,
|
||||
"Not found a key with ID {} required to decipher file {}",
|
||||
header.key_id,
|
||||
quoteString(path));
|
||||
|
||||
String key = it->second;
|
||||
if (calculateKeyHash(key) != header.key_hash)
|
||||
throw Exception(
|
||||
ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong key with ID {}, could not decipher file {}", header.key_id, quoteString(path));
|
||||
|
||||
return key;
|
||||
}
|
||||
|
||||
bool inline isSameDiskType(const IDisk & one, const IDisk & another)
|
||||
{
|
||||
@ -144,6 +186,22 @@ private:
|
||||
std::unique_ptr<IReservation> reservation;
|
||||
};
|
||||
|
||||
DiskEncrypted::DiskEncrypted(
|
||||
const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_)
|
||||
: DiskEncrypted(name_, parseDiskEncryptedSettings(name_, config_, config_prefix_, map_))
|
||||
{
|
||||
}
|
||||
|
||||
DiskEncrypted::DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_)
|
||||
: DiskDecorator(settings_->wrapped_disk)
|
||||
, name(name_)
|
||||
, disk_path(settings_->disk_path)
|
||||
, disk_absolute_path(settings_->wrapped_disk->getPath() + settings_->disk_path)
|
||||
, current_settings(std::move(settings_))
|
||||
{
|
||||
delegate->createDirectories(disk_path);
|
||||
}
|
||||
|
||||
ReservationPtr DiskEncrypted::reserve(UInt64 bytes)
|
||||
{
|
||||
auto reservation = delegate->reserve(bytes);
|
||||
@ -152,58 +210,23 @@ ReservationPtr DiskEncrypted::reserve(UInt64 bytes)
|
||||
return std::make_unique<DiskEncryptedReservation>(std::static_pointer_cast<DiskEncrypted>(shared_from_this()), std::move(reservation));
|
||||
}
|
||||
|
||||
DiskEncrypted::DiskEncrypted(
|
||||
const String & name_,
|
||||
DiskPtr wrapped_disk_,
|
||||
const String & path_on_wrapped_disk_,
|
||||
const std::unordered_map<UInt64, String> & keys_,
|
||||
UInt64 current_key_id_,
|
||||
FileEncryption::Algorithm current_algorithm_)
|
||||
: DiskDecorator(wrapped_disk_)
|
||||
, name(name_)
|
||||
, disk_path(path_on_wrapped_disk_)
|
||||
, keys(keys_)
|
||||
, current_key_id(current_key_id_)
|
||||
, current_algorithm(current_algorithm_)
|
||||
{
|
||||
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::BAD_ARGUMENTS);
|
||||
|
||||
delegate->createDirectories(disk_path);
|
||||
}
|
||||
|
||||
|
||||
String DiskEncrypted::getKey(UInt64 key_id) const
|
||||
{
|
||||
auto it = keys.find(key_id);
|
||||
if (it == keys.end())
|
||||
throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Key with ID {} not found", key_id);
|
||||
return it->second;
|
||||
}
|
||||
|
||||
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 (auto * to_disk_enc = typeid_cast<DiskEncrypted *>(to_disk.get()))
|
||||
{
|
||||
if (keys == to_encrypted_disk->keys)
|
||||
auto from_settings = current_settings.get();
|
||||
auto to_settings = to_disk_enc->current_settings.get();
|
||||
if (from_settings->keys == to_settings->keys)
|
||||
{
|
||||
/// Keys are the same so we can simply copy the encrypted file.
|
||||
delegate->copy(wrappedPath(from_path), to_encrypted_disk->delegate, to_encrypted_disk->wrappedPath(to_path));
|
||||
auto wrapped_from_path = wrappedPath(from_path);
|
||||
auto to_delegate = to_disk_enc->delegate;
|
||||
auto wrapped_to_path = to_disk_enc->wrappedPath(to_path);
|
||||
delegate->copy(wrapped_from_path, to_delegate, wrapped_to_path);
|
||||
return;
|
||||
}
|
||||
}
|
||||
@ -221,62 +244,43 @@ std::unique_ptr<ReadBufferFromFileBase> DiskEncrypted::readFile(
|
||||
size_t mmap_threshold,
|
||||
MMappedFileCache * mmap_cache) const
|
||||
{
|
||||
try
|
||||
{
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
auto buffer = delegate->readFile(wrapped_path, buf_size, estimated_size, aio_threshold, mmap_threshold, mmap_cache);
|
||||
FileEncryption::Header header;
|
||||
header.read(*buffer);
|
||||
String key = getKey(header.key_id);
|
||||
if (calculateKeyHash(key) != header.key_hash)
|
||||
throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong key, could not read file");
|
||||
return std::make_unique<ReadBufferFromEncryptedFile>(buf_size, std::move(buffer), key, header);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("File " + quoteString(path));
|
||||
throw;
|
||||
}
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
auto buffer = delegate->readFile(wrapped_path, buf_size, estimated_size, aio_threshold, mmap_threshold, mmap_cache);
|
||||
auto settings = current_settings.get();
|
||||
FileEncryption::Header header = readHeader(*buffer);
|
||||
String key = getKey(path, header, *settings);
|
||||
return std::make_unique<ReadBufferFromEncryptedFile>(buf_size, std::move(buffer), key, header);
|
||||
}
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DiskEncrypted::writeFile(const String & path, size_t buf_size, WriteMode mode)
|
||||
{
|
||||
try
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
FileEncryption::Header header;
|
||||
String key;
|
||||
UInt64 old_file_size = 0;
|
||||
auto settings = current_settings.get();
|
||||
if (mode == WriteMode::Append && exists(path))
|
||||
{
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
FileEncryption::Header header;
|
||||
String key;
|
||||
UInt64 old_file_size = 0;
|
||||
if (mode == WriteMode::Append && exists(path))
|
||||
old_file_size = getFileSize(path);
|
||||
if (old_file_size)
|
||||
{
|
||||
old_file_size = getFileSize(path);
|
||||
if (old_file_size)
|
||||
{
|
||||
/// Append mode: we continue to use the same header.
|
||||
auto read_buffer = delegate->readFile(wrapped_path, FileEncryption::Header::kSize);
|
||||
header.read(*read_buffer);
|
||||
key = getKey(header.key_id);
|
||||
if (calculateKeyHash(key) != header.key_hash)
|
||||
throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong key, could not append file");
|
||||
}
|
||||
/// Append mode: we continue to use the same header.
|
||||
auto read_buffer = delegate->readFile(wrapped_path, FileEncryption::Header::kSize);
|
||||
header = readHeader(*read_buffer);
|
||||
key = getKey(path, header, *settings);
|
||||
}
|
||||
if (!old_file_size)
|
||||
{
|
||||
/// Rewrite mode: we generate a new header.
|
||||
key = getKey(current_key_id);
|
||||
header.algorithm = current_algorithm;
|
||||
header.key_id = current_key_id;
|
||||
header.key_hash = calculateKeyHash(key);
|
||||
header.init_vector = InitVector::random();
|
||||
}
|
||||
auto buffer = delegate->writeFile(wrapped_path, buf_size, mode);
|
||||
return std::make_unique<WriteBufferFromEncryptedFile>(buf_size, std::move(buffer), key, header, old_file_size);
|
||||
}
|
||||
catch (Exception & e)
|
||||
if (!old_file_size)
|
||||
{
|
||||
e.addMessage("File " + quoteString(path));
|
||||
throw;
|
||||
/// Rewrite mode: we generate a new header.
|
||||
key = getCurrentKey(path, *settings);
|
||||
header.algorithm = settings->current_algorithm;
|
||||
header.key_id = settings->current_key_id;
|
||||
header.key_hash = calculateKeyHash(key);
|
||||
header.init_vector = InitVector::random();
|
||||
}
|
||||
auto buffer = delegate->writeFile(wrapped_path, buf_size, mode);
|
||||
return std::make_unique<WriteBufferFromEncryptedFile>(buf_size, std::move(buffer), key, header, old_file_size);
|
||||
}
|
||||
|
||||
|
||||
@ -303,15 +307,16 @@ void DiskEncrypted::applyNewSettings(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
ContextPtr /*context*/,
|
||||
const String & config_prefix,
|
||||
const DisksMap & map)
|
||||
const DisksMap & disk_map)
|
||||
{
|
||||
DiskEncryptedSettings settings{name, config, config_prefix, map};
|
||||
delegate = settings.wrapped_disk;
|
||||
disk_path = settings.path_on_wrapped_disk;
|
||||
keys = settings.keys;
|
||||
current_key_id = settings.current_key_id;
|
||||
current_algorithm = settings.current_algorithm;
|
||||
initialize();
|
||||
auto new_settings = parseDiskEncryptedSettings(name, config, config_prefix, disk_map);
|
||||
if (new_settings->wrapped_disk != delegate)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Сhanging wrapped disk on the fly is not supported. Disk {}", name);
|
||||
|
||||
if (new_settings->disk_path != disk_path)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Сhanging disk path on the fly is not supported. Disk {}", name);
|
||||
|
||||
current_settings.set(std::move(new_settings));
|
||||
}
|
||||
|
||||
void registerDiskEncrypted(DiskFactory & factory)
|
||||
@ -322,14 +327,7 @@ void registerDiskEncrypted(DiskFactory & factory)
|
||||
ContextPtr /*context*/,
|
||||
const DisksMap & map) -> DiskPtr
|
||||
{
|
||||
DiskEncryptedSettings settings{name, config, config_prefix, map};
|
||||
return std::make_shared<DiskEncrypted>(
|
||||
name,
|
||||
settings.wrapped_disk,
|
||||
settings.path_on_wrapped_disk,
|
||||
settings.keys,
|
||||
settings.current_key_id,
|
||||
settings.current_algorithm);
|
||||
return std::make_shared<DiskEncrypted>(name, config, config_prefix, map);
|
||||
};
|
||||
factory.registerDiskType("encrypted", creator);
|
||||
}
|
||||
|
@ -7,6 +7,7 @@
|
||||
#if USE_SSL
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/DiskDecorator.h>
|
||||
#include <Common/MultiVersion.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -15,19 +16,23 @@ class ReadBufferFromFileBase;
|
||||
class WriteBufferFromFileBase;
|
||||
namespace FileEncryption { enum class Algorithm; }
|
||||
|
||||
struct DiskEncryptedSettings
|
||||
{
|
||||
DiskPtr wrapped_disk;
|
||||
String disk_path;
|
||||
std::unordered_map<UInt64, String> keys;
|
||||
UInt64 current_key_id;
|
||||
FileEncryption::Algorithm current_algorithm;
|
||||
};
|
||||
|
||||
/// Encrypted disk ciphers all written files on the fly and writes the encrypted files to an underlying (normal) disk.
|
||||
/// And when we read files from an encrypted disk it deciphers them automatically,
|
||||
/// so we can work with a encrypted disk like it's a normal disk.
|
||||
class DiskEncrypted : public DiskDecorator
|
||||
{
|
||||
public:
|
||||
DiskEncrypted(
|
||||
const String & name_,
|
||||
DiskPtr wrapped_disk_,
|
||||
const String & path_on_wrapped_disk_,
|
||||
const std::unordered_map<UInt64, String> & keys_,
|
||||
UInt64 current_key_id_,
|
||||
FileEncryption::Algorithm current_algorithm_);
|
||||
DiskEncrypted(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_);
|
||||
DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_);
|
||||
|
||||
const String & getName() const override { return name; }
|
||||
const String & getPath() const override { return disk_absolute_path; }
|
||||
@ -215,8 +220,6 @@ public:
|
||||
SyncGuardPtr getDirectorySyncGuard(const String & path) const override;
|
||||
|
||||
private:
|
||||
void initialize();
|
||||
|
||||
String wrappedPath(const String & path) const
|
||||
{
|
||||
// if path starts_with disk_path -> got already wrapped path
|
||||
@ -225,14 +228,10 @@ private:
|
||||
return disk_path + path;
|
||||
}
|
||||
|
||||
String getKey(UInt64 key_id) const;
|
||||
|
||||
String name;
|
||||
String disk_path;
|
||||
String disk_absolute_path;
|
||||
std::unordered_map<UInt64, String> keys;
|
||||
UInt64 current_key_id;
|
||||
FileEncryption::Algorithm current_algorithm;
|
||||
const String name;
|
||||
const String disk_path;
|
||||
const String disk_absolute_path;
|
||||
MultiVersion<DiskEncryptedSettings> current_settings;
|
||||
};
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user