From c23fe5baf683f097b7123c9d32b668f6eaa015b5 Mon Sep 17 00:00:00 2001
From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com>
Date: Mon, 13 Sep 2021 11:25:36 +0300
Subject: [PATCH] Improve codec for encr 19896 (#27476)
* change syntax of encrypted command
* commit all encrypted changes
* correct encryption
* correct config for test
* add tests and correct code style and typos
* correct test
* fix unbundled build
* add log warning messages
* improve code according to review comments
* correct nonce
* correct errors found by fuzzing
* improve codec AES_128_GCM_SIV. Add AES_256_GCM_SIV. Add sections for last in tests. Improve documentation
* Update CompressionCodecEncrypted.h
* Update 01683_codec_encrypted.sql
* correct compression factory after changes in master
* correct behavior with wrong key in data
* correct fuzzer
* add connection for fuzzer with fix for compression_encrypted
* refactor code
* add load from config with throwing errors on server start
* fix typos and check style
* Update Server.cpp
* correct loading and reading
* refactor code. fix uninitialized value
* refactor code
* move defines from server to cpp file
* correct build
* remove repeated code
* correct namespace
* fix code style
---
.../settings.md | 78 +-
programs/server/Server.cpp | 44 +-
programs/server/config.xml | 26 +-
src/Compression/CompressionCodecEncrypted.cpp | 664 +++++++++++++-----
src/Compression/CompressionCodecEncrypted.h | 201 +++---
.../CompressionFactoryAdditions.cpp | 20 +-
src/Compression/CompressionInfo.h | 19 +-
src/Compression/ICompressionCodec.h | 2 +-
.../fuzzers/encrypted_decompress_fuzzer.cpp | 298 +++++++-
src/Compression/ya.make | 5 +
tests/config/config.d/encryption.xml | 12 +-
.../test_codec_encrypted/__init__.py | 0
.../integration/test_codec_encrypted/test.py | 73 ++
.../01683_codec_encrypted.reference | 1 +
.../0_stateless/01683_codec_encrypted.sql | 9 +-
15 files changed, 1102 insertions(+), 350 deletions(-)
create mode 100644 tests/integration/test_codec_encrypted/__init__.py
create mode 100644 tests/integration/test_codec_encrypted/test.py
diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md
index be19f476978..8ad450738b7 100644
--- a/docs/en/operations/server-configuration-parameters/settings.md
+++ b/docs/en/operations/server-configuration-parameters/settings.md
@@ -69,29 +69,85 @@ If no conditions met for a data part, ClickHouse uses the `lz4` compression.
```
-
+
+Load key from environment variable:
+
+```xml
+
+
+
+
+
+```
+
+Where current_key_id sets the current key for encryption, and all specified keys can be used for decryption.
+
+All this methods can be applied for multiple keys:
+
+```xml
+
+
+ 00112233445566778899aabbccddeeff
+
+ 1
+
+
+```
+
+Where `current_key_id` shows current key for encryption.
+
+Also user can add nonce that must be 12 bytes long (by default encryption and decryption will use nonce consisting of zero bytes):
+
+```xml
+
+
+ 0123456789101
+
+
+```
+
+Or it can be set in hex:
+
+```xml
+
+
+ abcdefabcdef
+
+
+```
+
+Everything above can be applied for `aes_256_gcm_siv` (but key must be 32 bytes length).
+
## custom_settings_prefixes {#custom_settings_prefixes}
List of prefixes for [custom settings](../../operations/settings/index.md#custom_settings). The prefixes must be separated with commas.
diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp
index 0c834174519..2d09495c338 100644
--- a/programs/server/Server.cpp
+++ b/programs/server/Server.cpp
@@ -79,6 +79,7 @@
#include
#include
#include
+#include
#if !defined(ARCADIA_BUILD)
# include "config_core.h"
@@ -251,7 +252,6 @@ namespace ErrorCodes
extern const int SUPPORT_IS_DISABLED;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
- extern const int INCORRECT_DATA;
extern const int INVALID_CONFIG_PARAMETER;
extern const int SYSTEM_ERROR;
extern const int FAILED_TO_GETPWUID;
@@ -456,40 +456,6 @@ void checkForUsersNotInMainConfig(
}
}
-static void loadEncryptionKey(const std::string & key_command [[maybe_unused]], Poco::Logger * log)
-{
-#if USE_BASE64 && USE_SSL && USE_INTERNAL_SSL_LIBRARY
-
- auto process = ShellCommand::execute(key_command);
-
- std::string b64_key;
- readStringUntilEOF(b64_key, process->out);
- process->wait();
-
- // turbob64 doesn't like whitespace characters in input. Strip
- // them before decoding.
- std::erase_if(b64_key, [](char c)
- {
- return c == ' ' || c == '\t' || c == '\r' || c == '\n';
- });
-
- std::vector buf(b64_key.size());
- const size_t key_size = tb64dec(reinterpret_cast(b64_key.data()), b64_key.size(),
- reinterpret_cast(buf.data()));
- if (!key_size)
- throw Exception("Failed to decode encryption key", ErrorCodes::INCORRECT_DATA);
- else if (key_size < 16)
- LOG_WARNING(log, "The encryption key should be at least 16 octets long.");
-
- const std::string_view key = std::string_view(buf.data(), key_size);
- CompressionCodecEncrypted::setMasterKey(key);
-
-#else
- LOG_WARNING(log, "Server was built without Base64 or SSL support. Encryption is disabled.");
-#endif
-}
-
-
[[noreturn]] void forceShutdown()
{
#if defined(THREAD_SANITIZER) && defined(OS_LINUX)
@@ -904,6 +870,8 @@ if (ThreadFuzzer::instance().isEffective())
global_context->updateStorageConfiguration(*config);
global_context->updateInterserverCredentials(*config);
+
+ CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, "encryption_codecs");
},
/* already_loaded = */ false); /// Reload it right now (initial loading)
@@ -976,9 +944,9 @@ if (ThreadFuzzer::instance().isEffective())
global_context->getMergeTreeSettings().sanityCheck(settings);
global_context->getReplicatedMergeTreeSettings().sanityCheck(settings);
- /// Set up encryption.
- if (config().has("encryption.key_command"))
- loadEncryptionKey(config().getString("encryption.key_command"), log);
+
+ /// try set up encryption. There are some errors in config, error will be printed and server wouldn't start.
+ CompressionCodecEncrypted::Configuration::instance().load(config(), "encryption_codecs");
Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0);
diff --git a/programs/server/config.xml b/programs/server/config.xml
index b8ef17458be..6c98ac740fe 100644
--- a/programs/server/config.xml
+++ b/programs/server/config.xml
@@ -1026,10 +1026,28 @@
defined, or encryption codecs will be disabled otherwise. The
command is executed through /bin/sh and is expected to write
a Base64-encoded key to the stdout. -->
-
-
-
-
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp
index 7d667fbc1b7..d35411d6266 100644
--- a/src/Compression/CompressionCodecEncrypted.cpp
+++ b/src/Compression/CompressionCodecEncrypted.cpp
@@ -1,215 +1,553 @@
#if !defined(ARCADIA_BUILD)
# include
#endif
+#include "Common/Exception.h"
+#include "common/types.h"
+#include "IO/VarInt.h"
#include
-#if USE_SSL && USE_INTERNAL_SSL_LIBRARY
-
#include
+#include
+#include
+#include
+
+// This depends on BoringSSL-specific API, notably .
+#if USE_SSL && USE_INTERNAL_SSL_LIBRARY
#include
-#include
#include // Y_IGNORE
#include
-#include // Y_IGNORE
-#include
+#include
+#include // Y_IGNORE
+#endif
+// Common part for both parts (with SSL and without)
namespace DB
{
- namespace ErrorCodes
+
+namespace ErrorCodes
+{
+ extern const int OPENSSL_ERROR;
+}
+
+namespace
+{
+
+/// Get string name for method. Return empty string for undefined Method
+String getMethodName(EncryptionMethod Method)
+{
+ if (Method == AES_128_GCM_SIV)
{
- extern const int ILLEGAL_CODEC_PARAMETER;
- extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
- extern const int NO_ELEMENTS_IN_CONFIG;
- extern const int OPENSSL_ERROR;
+ return "AES_128_GCM_SIV";
}
-
- void CompressionCodecEncrypted::setMasterKey(const std::string_view & master_key)
+ else if (Method == AES_256_GCM_SIV)
{
- keys.emplace(master_key);
+ return "AES_256_GCM_SIV";
}
-
- CompressionCodecEncrypted::KeyHolder::KeyHolder(const std::string_view & master_key)
+ else
{
- // Derive a key from it.
- keygen_key = deriveKey(master_key);
-
- // EVP_AEAD_CTX is not stateful so we can create an
- // instance now.
- EVP_AEAD_CTX_zero(&ctx);
- const int ok = EVP_AEAD_CTX_init(&ctx, EVP_aead_aes_128_gcm(),
- reinterpret_cast(keygen_key.data()), keygen_key.size(),
- 16 /* tag size */, nullptr);
- if (!ok)
- throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
+ return "";
}
+}
- CompressionCodecEncrypted::KeyHolder::~KeyHolder()
+/// Get method code (used for codec, to understand which one we are using)
+uint8_t getMethodCode(EncryptionMethod Method)
+{
+ if (Method == AES_128_GCM_SIV)
{
- EVP_AEAD_CTX_cleanup(&ctx);
+ return uint8_t(CompressionMethodByte::AES_128_GCM_SIV);
}
-
- const CompressionCodecEncrypted::KeyHolder & CompressionCodecEncrypted::getKeys()
+ else if (Method == AES_256_GCM_SIV)
{
- if (keys)
- return *keys;
- else
- throw Exception("There is no configuration for encryption in the server config",
- ErrorCodes::NO_ELEMENTS_IN_CONFIG);
+ return uint8_t(CompressionMethodByte::AES_256_GCM_SIV);
}
-
- CompressionCodecEncrypted::CompressionCodecEncrypted(const std::string_view & cipher)
+ else
{
- setCodecDescription("Encrypted", {std::make_shared(cipher)});
+ throw Exception("Wrong encryption Method. Got " + getMethodName(Method), ErrorCodes::BAD_ARGUMENTS);
}
+}
- uint8_t CompressionCodecEncrypted::getMethodByte() const
+} // end of namespace
+
+} // end of namespace DB
+
+#if USE_SSL && USE_INTERNAL_SSL_LIBRARY
+namespace DB
+{
+
+namespace ErrorCodes
+{
+ extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
+ extern const int LOGICAL_ERROR;
+ extern const int BAD_ARGUMENTS;
+ extern const int INCORRECT_DATA;
+}
+
+namespace
+{
+constexpr size_t tag_size = 16; /// AES-GCM-SIV always uses a tag of 16 bytes length
+constexpr size_t key_id_max_size = 8; /// Max size of varint.
+constexpr size_t nonce_max_size = 13; /// Nonce size and one byte to show if nonce in in text
+
+/// Get encryption/decryption algorithms.
+auto getMethod(EncryptionMethod Method)
+{
+ if (Method == AES_128_GCM_SIV)
{
- return static_cast(CompressionMethodByte::Encrypted);
+ return EVP_aead_aes_128_gcm_siv;
}
-
- void CompressionCodecEncrypted::updateHash(SipHash & hash) const
+ else if (Method == AES_256_GCM_SIV)
{
- getCodecDesc()->updateTreeHash(hash);
+ return EVP_aead_aes_256_gcm_siv;
}
-
- UInt32 CompressionCodecEncrypted::getMaxCompressedDataSize(UInt32 uncompressed_size) const
+ else
{
- // The GCM mode is a stream cipher. No paddings are
- // involved. There will be a tag at the end of ciphertext (16
- // octets).
- return uncompressed_size + 16;
+ throw Exception("Wrong encryption Method. Got " + getMethodName(Method), ErrorCodes::BAD_ARGUMENTS);
}
+}
- UInt32 CompressionCodecEncrypted::doCompressData(const char * source, UInt32 source_size, char * dest) const
+/// Find out key size for each algorithm
+UInt64 methodKeySize(EncryptionMethod Method)
+{
+ if (Method == AES_128_GCM_SIV)
{
- // Generate an IV out of the data block and the key-generation
- // key. It is completely deterministic, but does not leak any
- // information about the data block except for equivalence of
- // identical blocks (under the same master key). The IV will
- // be used as an authentication tag. The ciphertext and the
- // tag will be written directly in the dest buffer.
- const std::string_view plaintext = std::string_view(source, source_size);
-
- encrypt(plaintext, dest);
- return source_size + 16;
+ return 16;
}
-
- void CompressionCodecEncrypted::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size [[maybe_unused]]) const
+ else if (Method == AES_256_GCM_SIV)
{
- // Extract the IV from the encrypted data block. Decrypt the
- // block with the extracted IV, and compare the tag. Throw an
- // exception if tags don't match.
- const std::string_view ciphertext_and_tag = std::string_view(source, source_size);
- assert(ciphertext_and_tag.size() == uncompressed_size + 16);
-
- decrypt(ciphertext_and_tag, dest);
+ return 32;
}
-
- std::string CompressionCodecEncrypted::lastErrorString()
+ else
{
- std::array buffer{};
- ERR_error_string_n(ERR_get_error(), buffer.data(), buffer.size());
- return std::string(buffer.data());
+ throw Exception("Wrong encryption Method. Got " + getMethodName(Method), ErrorCodes::BAD_ARGUMENTS);
}
+}
- std::string CompressionCodecEncrypted::deriveKey(const std::string_view & master_key)
+std::string lastErrorString()
+{
+ std::array buffer;
+ ERR_error_string_n(ERR_get_error(), buffer.data(), buffer.size());
+ return std::string(buffer.data());
+}
+
+/// Encrypt plaintext with particular algorithm and put result into ciphertext_and_tag.
+/// This function get key and nonce and encrypt text with their help.
+/// If something went wrong (can't init context or can't encrypt data) it throws exception.
+/// It returns length of encrypted text.
+size_t encrypt(const std::string_view & plaintext, char * ciphertext_and_tag, EncryptionMethod method, const String & key, const String & nonce)
+{
+ /// Init context for encryption, using key.
+ EVP_AEAD_CTX encrypt_ctx;
+ EVP_AEAD_CTX_zero(&encrypt_ctx);
+ const int ok_init = EVP_AEAD_CTX_init(&encrypt_ctx, getMethod(method)(),
+ reinterpret_cast(key.data()), key.size(),
+ 16 /* tag size */, nullptr);
+ if (!ok_init)
+ throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
+
+ /// encrypt data using context and given nonce.
+ size_t out_len;
+ const int ok_open = EVP_AEAD_CTX_seal(&encrypt_ctx,
+ reinterpret_cast(ciphertext_and_tag),
+ &out_len, plaintext.size() + 16,
+ reinterpret_cast(nonce.data()), nonce.size(),
+ reinterpret_cast(plaintext.data()), plaintext.size(),
+ nullptr, 0);
+ if (!ok_open)
+ throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
+
+ return out_len;
+}
+
+/// Encrypt plaintext with particular algorithm and put result into ciphertext_and_tag.
+/// This function get key and nonce and encrypt text with their help.
+/// If something went wrong (can't init context or can't encrypt data) it throws exception.
+/// It returns length of encrypted text.
+size_t decrypt(const std::string_view & ciphertext, char * plaintext, EncryptionMethod method, const String & key, const String & nonce)
+{
+ /// Init context for decryption with given key.
+ EVP_AEAD_CTX decrypt_ctx;
+ EVP_AEAD_CTX_zero(&decrypt_ctx);
+
+ const int ok_init = EVP_AEAD_CTX_init(&decrypt_ctx, getMethod(method)(),
+ reinterpret_cast(key.data()), key.size(),
+ 16 /* tag size */, nullptr);
+ if (!ok_init)
+ throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
+
+ /// decrypt data using given nonce
+ size_t out_len;
+ const int ok_open = EVP_AEAD_CTX_open(&decrypt_ctx,
+ reinterpret_cast(plaintext),
+ &out_len, ciphertext.size(),
+ reinterpret_cast(nonce.data()), nonce.size(),
+ reinterpret_cast(ciphertext.data()), ciphertext.size(),
+ nullptr, 0);
+ if (!ok_open)
+ throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
+
+ return out_len;
+}
+
+/// Register codec in factory
+void registerEncryptionCodec(CompressionCodecFactory & factory, EncryptionMethod Method)
+{
+ const auto method_code = getMethodCode(Method); /// Codec need to know its code
+ factory.registerCompressionCodec(getMethodName(Method), method_code, [&, Method](const ASTPtr & arguments) -> CompressionCodecPtr
{
- /// No salt: derive keys in a deterministic manner.
- std::string_view salt(""); // NOLINT
- std::string_view info("Codec Encrypted('AES-128-GCM-SIV') key generation key");
- std::array result;
-
- const int ok = HKDF(reinterpret_cast(result.data()), result.size(),
- EVP_sha256(),
- reinterpret_cast(master_key.data()), master_key.size(),
- reinterpret_cast(salt.data()), salt.size(),
- reinterpret_cast(info.data()), info.size());
- if (!ok)
- throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
-
- return std::string(result.data(), 16);
- }
-
- void CompressionCodecEncrypted::encrypt(const std::string_view & plaintext, char * ciphertext_and_tag)
- {
- // Fixed nonce. Yes this is unrecommended, but we have to live
- // with it.
- std::string_view nonce("\0\0\0\0\0\0\0\0\0\0\0\0", 12);
-
- size_t out_len;
- const int ok = EVP_AEAD_CTX_seal(&getKeys().ctx,
- reinterpret_cast(ciphertext_and_tag),
- &out_len, plaintext.size() + 16,
- reinterpret_cast(nonce.data()), nonce.size(),
- reinterpret_cast(plaintext.data()), plaintext.size(),
- nullptr, 0);
- if (!ok)
- throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
-
- assert(out_len == plaintext.size() + 16);
- }
-
- void CompressionCodecEncrypted::decrypt(const std::string_view & ciphertext, char * plaintext)
- {
- std::string_view nonce("\0\0\0\0\0\0\0\0\0\0\0\0", 12);
-
- size_t out_len;
- const int ok = EVP_AEAD_CTX_open(&getKeys().ctx,
- reinterpret_cast(plaintext),
- &out_len, ciphertext.size(),
- reinterpret_cast(nonce.data()), nonce.size(),
- reinterpret_cast(ciphertext.data()), ciphertext.size(),
- nullptr, 0);
- if (!ok)
- throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
-
- assert(out_len == ciphertext.size() - 16);
- }
-
- void registerCodecEncrypted(CompressionCodecFactory & factory)
- {
- const auto method_code = uint8_t(CompressionMethodByte::Encrypted);
- factory.registerCompressionCodec("Encrypted", method_code, [&](const ASTPtr & arguments) -> CompressionCodecPtr
+ if (arguments)
{
- if (arguments)
- {
- if (arguments->children.size() != 1)
- throw Exception("Codec Encrypted() must have 1 parameter, given " +
- std::to_string(arguments->children.size()),
- ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
+ if (!arguments->children.empty())
+ throw Exception("Codec " + getMethodName(Method) + " must not have parameters, given " +
+ std::to_string(arguments->children.size()),
+ ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
+ }
+ return std::make_shared(Method);
+ });
+}
- const auto children = arguments->children;
- const auto * literal = children[0]->as();
- if (!literal)
- throw Exception("Wrong argument for codec Encrypted(). Expected a string literal",
- ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
-
- const String cipher = literal->value.safeGet();
- if (cipher == "AES-128-GCM-SIV")
- return std::make_shared(cipher);
- else
- throw Exception("Cipher '" + cipher + "' is not supported",
- ErrorCodes::ILLEGAL_CODEC_PARAMETER);
- }
- else
- {
- /* The factory is asking us to construct the codec
- * only from the method code. How can that be
- * possible? For now we only support a single cipher
- * so it's not really a problem, but if we were to
- * support more ciphers it would be catastrophic. */
- return std::make_shared("AES-128-GCM-SIV");
- }
- });
+String unhexKey(const String & hex)
+{
+ try
+ {
+ return boost::algorithm::unhex(hex);
}
+ catch (const std::exception &)
+ {
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read key_hex, check for valid characters [0-9a-fA-F] and length");
+ }
+}
+
+/// Firstly, write a byte, which shows if the nonce will be put in text (if it was defined in config)
+/// Secondly, write nonce in text (this step depends from first step)
+/// return new position to write
+inline char* writeNonce(const String& nonce, char* dest)
+{
+ /// If nonce consists of nul bytes, it shouldn't be in dest. Zero byte is the only byte that should be written.
+ /// Otherwise, 1 is written and data from nonce is copied
+ if (nonce != String("\0\0\0\0\0\0\0\0\0\0\0\0", 12))
+ {
+ *dest = 1;
+ ++dest;
+ size_t copied_symbols = nonce.copy(dest, nonce.size());
+ if (copied_symbols != nonce.size())
+ throw Exception(ErrorCodes::INCORRECT_DATA, "Can't copy nonce into destination. Count of copied symbols {}, need to copy {}", copied_symbols, nonce.size());
+ dest += copied_symbols;
+ return dest;
+ }
+ else
+ {
+ *dest = 0;
+ return ++dest;
+ }
+}
+
+/// Firstly, read a byte, which shows if the nonce will be put in text (if it was defined in config)
+/// Secondly, read nonce in text (this step depends from first step)
+/// return new position to read
+inline const char* readNonce(String& nonce, const char* source)
+{
+ /// If first is zero byte: move source and set zero-bytes nonce
+ if (!*source)
+ {
+ nonce = {"\0\0\0\0\0\0\0\0\0\0\0\0", 12};
+ return ++source;
+ }
+ /// Move to next byte. Nonce will begin from there
+ ++source;
+
+ /// Otherwise, use data from source in nonce
+ nonce = {source, 12};
+ source += 12;
+ return source;
+}
+
+}
+
+CompressionCodecEncrypted::Configuration & CompressionCodecEncrypted::Configuration::instance()
+{
+ static CompressionCodecEncrypted::Configuration ret;
+ return ret;
+}
+
+void CompressionCodecEncrypted::Configuration::loadImpl(
+ const Poco::Util::AbstractConfiguration & config, const String & config_prefix, EncryptionMethod method, std::unique_ptr & new_params)
+{
+ // if method is not smaller than MAX_ENCRYPTION_METHOD it is incorrect
+ if (method >= MAX_ENCRYPTION_METHOD)
+ throw Exception("Wrong argument for loading configurations.", ErrorCodes::BAD_ARGUMENTS);
+
+ /// Scan all keys in config and add them into storage. If key is in hex, transform it.
+ /// Remember key ID for each key, because it will be used in encryption/decryption
+ 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["))
+ {
+ 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;
+
+ /// For each key its id should be unique.
+ if (new_params->keys_storage[method].contains(key_id))
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple keys have the same ID {}", key_id);
+
+ /// Check size of key. Its length depends on encryption algorithm.
+ if (key.size() != methodKeySize(method))
+ throw Exception(
+ ErrorCodes::BAD_ARGUMENTS,
+ "Got an encryption key with unexpected size {}, the size should be {}",
+ key.size(), methodKeySize(method));
+
+ new_params->keys_storage[method][key_id] = key;
+ }
+
+ /// Check that we have at least one key for this method (otherwise it is incorrect to use it).
+ if (new_params->keys_storage[method].empty())
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "No keys, an encryption needs keys to work");
+
+ /// Try to find which key will be used for encryption. If there is no current_key,
+ /// first key will be used for encryption (its index equals to zero).
+ new_params->current_key_id[method] = config.getUInt64(config_prefix + ".current_key_id", 0);
+
+ /// Check that we have current key. Otherwise config is incorrect.
+ if (!new_params->keys_storage[method].contains(new_params->current_key_id[method]))
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found a key with the current ID {}", new_params->current_key_id[method]);
+
+ /// Read nonce (in hex or in string). Its length should be 12 bytes.
+ if (config.has(config_prefix + ".nonce_hex"))
+ new_params->nonce[method] = unhexKey(config.getString(config_prefix + ".nonce_hex"));
+ else
+ new_params->nonce[method] = config.getString(config_prefix + ".nonce", "");
+
+ if (new_params->nonce[method].size() != 12 && !new_params->nonce[method].empty())
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "Got nonce with unexpected size {}, the size should be 12", new_params->nonce[method].size());
+}
+
+bool CompressionCodecEncrypted::Configuration::tryLoad(const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
+{
+ /// Try to create new parameters and fill them from config.
+ /// If there will be some errors, print their message to notify user that
+ /// something went wrong and new parameters are not available
+ try
+ {
+ load(config, config_prefix);
+ }
+ catch (...)
+ {
+ tryLogCurrentException(__PRETTY_FUNCTION__);
+ return false;
+ }
+ return true;
+}
+
+void CompressionCodecEncrypted::Configuration::load(const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
+{
+ /// Try to create new parameters and fill them from config.
+ /// If there will be some errors, throw error
+ std::unique_ptr new_params(new Params);
+ if (config.has(config_prefix + ".aes_128_gcm_siv"))
+ {
+ loadImpl(config, config_prefix + ".aes_128_gcm_siv", AES_128_GCM_SIV, new_params);
+ }
+ if (config.has(config_prefix + ".aes_256_gcm_siv"))
+ {
+ loadImpl(config, config_prefix + ".aes_256_gcm_siv", AES_256_GCM_SIV, new_params);
+ }
+
+ params.set(std::move(new_params));
+}
+
+void CompressionCodecEncrypted::Configuration::getCurrentKeyAndNonce(EncryptionMethod method, UInt64 & current_key_id, String ¤t_key, String & nonce) const
+{
+ /// It parameters were not set, throw exception
+ if (!params.get())
+ throw Exception("Empty params in CompressionCodecEncrypted configuration", ErrorCodes::BAD_ARGUMENTS);
+
+ /// Save parameters in variable, because they can always change.
+ /// As this function not atomic, we should be certain that we get information from one particular version for correct work.
+ const auto current_params = params.get();
+ current_key_id = current_params->current_key_id[method];
+
+ /// As parameters can be created empty, we need to check that this key is available.
+ if (current_params->keys_storage[method].contains(current_key_id))
+ current_key = current_params->keys_storage[method].at(current_key_id);
+ else
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no current_key {} in config. Please, put it in config and reload.", current_key_id);
+
+ /// If there is no nonce in config, we need to generate particular one,
+ /// because all encryptions should have nonce and random nonce generation will lead to cases
+ /// when nonce after config reload (nonce is not defined in config) will differ from previously generated one.
+ /// This will lead to data loss.
+ nonce = current_params->nonce[method];
+ if (nonce.empty())
+ nonce = {"\0\0\0\0\0\0\0\0\0\0\0\0", 12};
+}
+
+String CompressionCodecEncrypted::Configuration::getKey(EncryptionMethod method, const UInt64 & key_id) const
+{
+ String key;
+ /// See description of previous finction, logic is the same.
+ if (!params.get())
+ throw Exception("Empty params in CompressionCodecEncrypted configuration", ErrorCodes::BAD_ARGUMENTS);
+
+ const auto current_params = params.get();
+
+ /// check if there is current key in storage
+ if (current_params->keys_storage[method].contains(key_id))
+ key = current_params->keys_storage[method].at(key_id);
+ else
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no key {} in config", key_id);
+
+ return key;
+}
+
+
+CompressionCodecEncrypted::CompressionCodecEncrypted(EncryptionMethod Method): encryption_method(Method)
+{
+ setCodecDescription(getMethodName(encryption_method));
+}
+
+uint8_t CompressionCodecEncrypted::getMethodByte() const
+{
+ return getMethodCode(encryption_method);
+}
+
+void CompressionCodecEncrypted::updateHash(SipHash & hash) const
+{
+ getCodecDesc()->updateTreeHash(hash);
+}
+
+UInt32 CompressionCodecEncrypted::getMaxCompressedDataSize(UInt32 uncompressed_size) const
+{
+ // The GCM mode is a stream cipher. No paddings are
+ // involved. There will be a tag at the end of ciphertext (16
+ // octets). Also it has not more than 8 bytes for key_id in the beginning
+ // KeyID is followed by byte, that shows if nonce was set in config (and also will be put into data)
+ // and 12 bytes nonce or this byte will be equal to zero and no nonce will follow it.
+ return uncompressed_size + tag_size + key_id_max_size + nonce_max_size;
+}
+
+UInt32 CompressionCodecEncrypted::doCompressData(const char * source, UInt32 source_size, char * dest) const
+{
+ // Generate an IV out of the data block and the key-generation
+ // key. It is completely deterministic, but does not leak any
+ // information about the data block except for equivalence of
+ // identical blocks (under the same key).
+
+ const std::string_view plaintext = std::string_view(source, source_size);
+
+ /// Get key and nonce for encryption
+ UInt64 current_key_id;
+ String current_key, nonce;
+ Configuration::instance().getCurrentKeyAndNonce(encryption_method, current_key_id, current_key, nonce);
+
+ /// Write current key id to support multiple keys.
+ /// (key id in the beginning will help to decrypt data after changing current key)
+ char* ciphertext_with_nonce = writeVarUInt(current_key_id, dest);
+ size_t keyid_size = ciphertext_with_nonce - dest;
+
+ /// write nonce in data. This will help to read data even after changing nonce in config
+ /// If there were no nonce in data, one zero byte will be written
+ char* ciphertext = writeNonce(nonce, ciphertext_with_nonce);
+ UInt64 nonce_size = ciphertext - ciphertext_with_nonce;
+
+ // The IV will be used as an authentication tag. The ciphertext and the
+ // tag will be written directly in the dest buffer.
+ size_t out_len = encrypt(plaintext, ciphertext, encryption_method, current_key, nonce);
+
+ /// Length of encrypted text should be equal to text length plus tag_size (which was added by algorithm).
+ if (out_len != source_size + tag_size)
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't encrypt data, length after encryption {} is wrong, expected {}", out_len, source_size + tag_size);
+
+ return out_len + keyid_size + nonce_size;
+}
+
+void CompressionCodecEncrypted::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const
+{
+ /// The key is needed for decrypting. That's why it is read at the beginning of process.
+ UInt64 key_id;
+ const char * ciphertext_with_nonce = readVarUInt(key_id, source, source_size);
+
+ /// Size of text should be decreased by key_size, because key_size bytes were not participating in encryption process.
+ size_t keyid_size = ciphertext_with_nonce - source;
+ String nonce;
+ String key = Configuration::instance().getKey(encryption_method, key_id);
+
+ /// try to read nonce from file (if it was set while encrypting)
+ const char * ciphertext = readNonce(nonce, ciphertext_with_nonce);
+
+ /// Size of text should be decreased by nonce_size, because nonce_size bytes were not participating in encryption process.
+ UInt64 nonce_size = ciphertext - ciphertext_with_nonce;
+
+ /// Count text size (nonce and key_id was read from source)
+ size_t ciphertext_size = source_size - keyid_size - nonce_size;
+ if (ciphertext_size != uncompressed_size + tag_size)
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't decrypt data, uncompressed_size {} is wrong, expected {}", uncompressed_size, ciphertext_size - tag_size);
+
+
+ size_t out_len = decrypt({ciphertext, ciphertext_size}, dest, encryption_method, key, nonce);
+ if (out_len != ciphertext_size - tag_size)
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't decrypt data, out length after decryption {} is wrong, expected {}", out_len, ciphertext_size - tag_size);
+}
+
}
#else /* USE_SSL && USE_INTERNAL_SSL_LIBRARY */
namespace DB
{
- void registerCodecEncrypted(CompressionCodecFactory &)
- {
- }
+
+namespace
+{
+
+/// Register codec in factory
+void registerEncryptionCodec(CompressionCodecFactory & factory, EncryptionMethod Method)
+{
+ auto throw_no_ssl = [](const ASTPtr &) -> CompressionCodecPtr { throw Exception(ErrorCodes::OPENSSL_ERROR, "Server was built without SSL support. Encryption is disabled."); };
+ const auto method_code = getMethodCode(Method); /// Codec need to know its code
+ factory.registerCompressionCodec(getMethodName(Method), method_code, throw_no_ssl);
+}
+
+}
+
+CompressionCodecEncrypted::Configuration & CompressionCodecEncrypted::Configuration::instance()
+{
+ static CompressionCodecEncrypted::Configuration ret;
+ return ret;
+}
+
+/// if encryption is disabled.
+bool CompressionCodecEncrypted::Configuration::tryLoad(const Poco::Util::AbstractConfiguration & config [[maybe_unused]], const String & config_prefix [[maybe_unused]])
+{
+ return false;
+}
+
+/// if encryption is disabled, print warning about this.
+void CompressionCodecEncrypted::Configuration::load(const Poco::Util::AbstractConfiguration & config [[maybe_unused]], const String & config_prefix [[maybe_unused]])
+{
+ LOG_WARNING(&Poco::Logger::get("CompressionCodecEncrypted"), "Server was built without SSL support. Encryption is disabled.");
+}
+
}
#endif /* USE_SSL && USE_INTERNAL_SSL_LIBRARY */
+
+namespace DB
+{
+/// Register codecs for all algorithms
+void registerCodecEncrypted(CompressionCodecFactory & factory)
+{
+ registerEncryptionCodec(factory, AES_128_GCM_SIV);
+ registerEncryptionCodec(factory, AES_256_GCM_SIV);
+}
+}
diff --git a/src/Compression/CompressionCodecEncrypted.h b/src/Compression/CompressionCodecEncrypted.h
index cef9b2e6072..7869e4d7194 100644
--- a/src/Compression/CompressionCodecEncrypted.h
+++ b/src/Compression/CompressionCodecEncrypted.h
@@ -1,109 +1,138 @@
#pragma once
-// This depends on BoringSSL-specific API, notably .
-#include
-#if USE_SSL && USE_INTERNAL_SSL_LIBRARY && !defined(ARCADIA_BUILD)
-
+#if !defined(ARCADIA_BUILD)
+#include
+#include
+#include
#include
-#include
-#include // Y_IGNORE
-#include
+#include
+#include
namespace DB
{
- /** This codec encrypts and decrypts blocks with AES-128 in
- * GCM-SIV mode (RFC-8452), which is the only cipher currently
- * supported. Although it is implemented as a compression codec
- * it doesn't actually compress data. In fact encrypted data will
- * no longer be compressible in any meaningful way. This means if
- * you want to apply both compression and encryption to your
- * columns, you need to put this codec at the end of the chain
- * like "column Int32 Codec(Delta, LZ4,
- * Encrypted('AES-128-GCM-SIV'))".
- *
- * The key is obtained by executing a command specified in the
- * configuration file at startup, and if it doesn't specify a
- * command the codec refuses to process any data. The command is
- * expected to write a Base64-encoded key of any length, and we
- * apply HKDF-SHA-256 to derive a 128-bit key-generation key
- * (only the first half of the result is used). We then encrypt
- * blocks in AES-128-GCM-SIV with a universally fixed nonce (12
- * repeated NUL characters).
- *
- * This construct has a weakness due to the nonce being fixed at
- * all times: when the same data block is encrypted twice, the
- * resulting ciphertext will be exactly the same. We have to live
- * with this weakness because ciphertext must be deterministic,
- * as otherwise our engines like ReplicatedMergeTree cannot
- * deduplicate data blocks.
- */
- class CompressionCodecEncrypted : public ICompressionCodec
+
+/// Now we have two algorithms.
+enum EncryptionMethod
+{
+ AES_128_GCM_SIV,
+ AES_256_GCM_SIV,
+ MAX_ENCRYPTION_METHOD
+};
+
+/** This codec encrypts and decrypts blocks with AES-128 in
+ * GCM-SIV mode (RFC-8452), which is the only cipher currently
+ * supported. Although it is implemented as a compression codec
+ * it doesn't actually compress data. In fact encrypted data will
+ * no longer be compressible in any meaningful way. This means if
+ * you want to apply both compression and encryption to your
+ * columns, you need to put this codec at the end of the chain
+ * like "column Int32 Codec(Delta, LZ4,
+ * AES_128_GCM_SIV)".
+ *
+ * The key is obtained by executing a command specified in the
+ * configuration file at startup, and if it doesn't specify a
+ * command the codec refuses to process any data. The command is
+ * expected to write a Base64-encoded key of any length, and we
+ * apply HKDF-SHA-256 to derive a 128-bit key-generation key
+ * (only the first half of the result is used). We then encrypt
+ * blocks in AES-128-GCM-SIV with a universally fixed nonce (12
+ * repeated NUL characters).
+ *
+ * This construct has a weakness due to the nonce being fixed at
+ * all times: when the same data block is encrypted twice, the
+ * resulting ciphertext will be exactly the same. We have to live
+ * with this weakness because ciphertext must be deterministic,
+ * as otherwise our engines like ReplicatedMergeTree cannot
+ * deduplicate data blocks.
+ */
+class CompressionCodecEncrypted : public ICompressionCodec
+{
+public:
+ /** If a key is available, the server is supposed to
+ * invoke this static method at the startup. The codec will
+ * refuse to compress or decompress any data until that. The
+ * key can be an arbitrary octet string, but it is
+ * recommended that the key is at least 16 octets long.
+ *
+ * Note that the key is currently not guarded by a
+ * mutex. This method should be invoked no more than once.
+ */
+ explicit CompressionCodecEncrypted(EncryptionMethod Method);
+
+ /**
+ * This is utility class. It holds information about encryption configuration.
+ */
+ class Configuration
{
public:
- /** If a master key is available, the server is supposed to
- * invoke this static method at the startup. The codec will
- * refuse to compress or decompress any data until that. The
- * key can be an arbitrary octet string, but it is
- * recommended that the key is at least 16 octets long.
- *
- * Note that the master key is currently not guarded by a
- * mutex. This method should be invoked no more than once.
- */
- static void setMasterKey(const std::string_view & master_key);
+ /// Configuration should be singleton. Instance method
+ static Configuration & instance();
- explicit CompressionCodecEncrypted(const std::string_view & cipher);
+ /// Try to load data from config.
+ bool tryLoad(const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
- uint8_t getMethodByte() const override;
- void updateHash(SipHash & hash) const override;
+ /// Load data and throw exception if something went wrong.
+ void load(const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
- bool isCompression() const override
- {
- return false;
- }
-
- bool isGenericCompression() const override
- {
- return false;
- }
-
- bool isPostProcessing() const override
- {
- return true;
- }
-
- protected:
- UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
-
- UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
- void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
+ /// Get current key and nonce (they will be set in variables, which are pass in this function).
+ /// All data sets at the same time to prevent situations,
+ /// when config changes and key and nonce are read from different versions
+ /// If nonce is empty, it will return 12 null bytes.
+ void getCurrentKeyAndNonce(EncryptionMethod method, UInt64 & current_key_id, String & current_key, String & nonce) const;
+ /// Same as getCurrentKeyAndNonce. It is used to get key. (need for correct decryption, that is why nonce is not necessary)
+ String getKey(EncryptionMethod method, const UInt64 & key_id) const;
private:
- static std::string lastErrorString();
- static std::string deriveKey(const std::string_view & master_key);
- static void encrypt(const std::string_view & plaintext, char * ciphertext_and_tag);
- static void decrypt(const std::string_view & ciphertext_and_tag, char * plaintext);
-
- /** A private class that holds keys derived from the master
- * key.
- */
- struct KeyHolder : private boost::noncopyable
+ /// struct Params consists of:
+ /// 1) hash-table of keys and their ids
+ /// 2) current key for encryption
+ /// 3) nonce for encryption
+ /// All this parameters have MAX_ENCRYPTION_METHOD count of versions,
+ /// because all algorithms can be described in config and used for different tables.
+ struct Params
{
- explicit KeyHolder(const std::string_view & master_key);
- ~KeyHolder();
-
- std::string keygen_key;
- EVP_AEAD_CTX ctx;
+ std::unordered_map keys_storage[MAX_ENCRYPTION_METHOD];
+ UInt64 current_key_id[MAX_ENCRYPTION_METHOD] = {0, 0};
+ String nonce[MAX_ENCRYPTION_METHOD];
};
- static const KeyHolder & getKeys();
+ // used to read data from config and create Params
+ static void loadImpl(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, EncryptionMethod method, std::unique_ptr& new_params);
- static inline std::optional keys;
+ MultiVersion params;
};
- inline CompressionCodecPtr getCompressionCodecEncrypted(const std::string_view & master_key)
+ uint8_t getMethodByte() const override;
+ void updateHash(SipHash & hash) const override;
+
+ bool isCompression() const override
{
- return std::make_shared(master_key);
+ return false;
}
+
+ bool isGenericCompression() const override
+ {
+ return false;
+ }
+
+ bool isEncryption() const override
+ {
+ return true;
+ }
+protected:
+ UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
+
+ /// Encrypt data with chosen method.
+ /// Throws exception if encryption is impossible or size of encrypted text is incorrect
+ UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
+
+ /// Decrypt data with chosen method
+ /// Throws exception if decryption is impossible or size of decrypted text is incorrect
+ void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
+private:
+ EncryptionMethod encryption_method;
+};
+
}
-#endif /* USE_SSL && USE_INTERNAL_SSL_LIBRARY */
+#endif /* NOT Arcadia_build */
diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp
index bff294d1ea5..9e0353e6711 100644
--- a/src/Compression/CompressionFactoryAdditions.cpp
+++ b/src/Compression/CompressionFactoryAdditions.cpp
@@ -62,7 +62,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
bool is_compression = false;
bool has_none = false;
std::optional generic_compression_codec_pos;
- std::set post_processing_codecs;
+ std::set encryption_codecs;
bool can_substitute_codec_arguments = true;
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
@@ -141,8 +141,8 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
if (!generic_compression_codec_pos && result_codec->isGenericCompression())
generic_compression_codec_pos = i;
- if (result_codec->isPostProcessing())
- post_processing_codecs.insert(i);
+ if (result_codec->isEncryption())
+ encryption_codecs.insert(i);
}
String codec_description = queryToString(codecs_descriptions);
@@ -157,8 +157,8 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
/// Allow to explicitly specify single NONE codec if user don't want any compression.
/// But applying other transformations solely without compression (e.g. Delta) does not make sense.
- /// It's okay to apply post-processing codecs solely without anything else.
- if (!is_compression && !has_none && post_processing_codecs.size() != codecs_descriptions->children.size())
+ /// It's okay to apply encryption codecs solely without anything else.
+ if (!is_compression && !has_none && encryption_codecs.size() != codecs_descriptions->children.size())
throw Exception(
"Compression codec " + codec_description
+ " does not compress anything."
@@ -168,10 +168,10 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
ErrorCodes::BAD_ARGUMENTS);
- /// It does not make sense to apply any non-post-processing codecs
- /// after post-processing one.
- if (!post_processing_codecs.empty() &&
- *post_processing_codecs.begin() != codecs_descriptions->children.size() - post_processing_codecs.size())
+ /// It does not make sense to apply any non-encryption codecs
+ /// after encryption one.
+ if (!encryption_codecs.empty() &&
+ *encryption_codecs.begin() != codecs_descriptions->children.size() - encryption_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any non-post-processing codecs after"
" post-processing ones. (Note: you can enable setting 'allow_suspicious_codecs'"
@@ -180,7 +180,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
/// It does not make sense to apply any transformations after generic compression algorithm
/// So, generic compression can be only one and only at the end.
if (generic_compression_codec_pos &&
- *generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - post_processing_codecs.size())
+ *generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - encryption_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any transformations after generic compression algorithm."
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS);
diff --git a/src/Compression/CompressionInfo.h b/src/Compression/CompressionInfo.h
index 869c7110d62..bbe8315f3ea 100644
--- a/src/Compression/CompressionInfo.h
+++ b/src/Compression/CompressionInfo.h
@@ -35,15 +35,16 @@ namespace DB
enum class CompressionMethodByte : uint8_t
{
- NONE = 0x02,
- LZ4 = 0x82,
- ZSTD = 0x90,
- Multiple = 0x91,
- Delta = 0x92,
- T64 = 0x93,
- DoubleDelta = 0x94,
- Gorilla = 0x95,
- Encrypted = 0x96,
+ NONE = 0x02,
+ LZ4 = 0x82,
+ ZSTD = 0x90,
+ Multiple = 0x91,
+ Delta = 0x92,
+ T64 = 0x93,
+ DoubleDelta = 0x94,
+ Gorilla = 0x95,
+ AES_128_GCM_SIV = 0x96,
+ AES_256_GCM_SIV = 0x97
};
}
diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h
index 3f603087f2b..7a401186be3 100644
--- a/src/Compression/ICompressionCodec.h
+++ b/src/Compression/ICompressionCodec.h
@@ -76,7 +76,7 @@ public:
virtual bool isGenericCompression() const = 0;
/// If it is a post-processing codec such as encryption. Usually it does not make sense to apply non-post-processing codecs after this.
- virtual bool isPostProcessing() const { return false; }
+ virtual bool isEncryption() const { return false; }
/// It is a codec available only for evaluation purposes and not meant to be used in production.
/// It will not be allowed to use unless the user will turn off the safety switch.
diff --git a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp
index 6211fd4b9cc..9e4d4a3241f 100644
--- a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp
+++ b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp
@@ -1,49 +1,301 @@
+#include
+#include
#include
+#include
#include
#include
#include
#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include "Common/Exception.h"
-namespace DB
+inline DB::CompressionCodecPtr getCompressionCodecEncrypted(DB::EncryptionMethod Method)
{
- CompressionCodecPtr getCompressionCodecEncrypted(const std::string_view & master_key);
+ return std::make_shared(Method);
}
-constexpr size_t key_size = 20;
-
-struct AuxiliaryRandomData
+namespace
{
- char key[key_size];
- size_t decompressed_size;
+
+using namespace Poco;
+using namespace Poco::XML;
+
+/*
+Fuzzing data consists of:
+ first byte:
+ 1) length of nonce is in data (for correct work with wrong data from user)
+ 2) length of key is in data (for correct work with wrong data from user)
+ 3) is 128 turn on
+ 4) multiple keys for 128
+ 5) is 256 turn on
+ 6) multiple keys for 256
+ 7) nonce is set
+ 8) current_key is set
+
+ read_key() will cosist of following steps:
+ if (2):
+ read 4 byte to know size
+ if multiple_keys:
+ read id
+ else:
+ size is chosen correctly according to algorithm
+
+ read size_bytes as a key
+
+ next bytes will have information in this order:
+ if (3):
+ if (4):
+ read count
+ for i in range(count):
+ read_key()
+ else:
+ read_key()
+ if (7):
+ read_nonce (simillar to read_key)
+ if (8):
+ set current_key
+
+ same for AES_256_GCM_SIV with (5) and (6) instead of (3) and (4)
+
+ This class read data and generate xml documentation.
+*/
+class XMLGenerator
+{
+public:
+ XMLGenerator(const uint8_t * data, size_t& size);
+
+ /// Try to generate config from input data using algorithm, which is described before class declaration
+ void generate();
+
+ /// Size of part, which was used on generating config
+ size_t keySize() const;
+
+ /// Get config
+ const Poco::AutoPtr& getResult() const;
+
+ /// If something happened in generator, it will be true
+ bool hasError() const;
+private:
+ /// generate algorithm section with key and nonce
+ bool generateAlgorithmKeys(AutoPtr& document_root, std::string name,
+ uint8_t mask_for_algo, uint8_t mask_for_multiple_keys);
+
+ /// move on count bytes stream and increase counter
+ /// returns false if some errors occuried
+ bool next(ssize_t count=1);
+
+ /// Create a key from data
+ ssize_t generateKey(std::string name, bool multiple=false);
+
+ const uint8_t * data;
+
+ size_t start_size;
+ size_t keys_size;
+
+ AutoPtr xml_document;
+ AutoPtr algo;
+ AutoPtr conf;
+
+ uint8_t first_byte;
+
+ bool error;
};
+XMLGenerator::XMLGenerator(const uint8_t * Data, size_t& Size): data(Data), start_size(Size),
+ conf(new Poco::Util::XMLConfiguration()), error(false) {}
+
+size_t XMLGenerator::keySize() const { return keys_size; }
+
+const Poco::AutoPtr& XMLGenerator::getResult() const { return conf; }
+
+bool XMLGenerator::hasError() const { return error; }
+
+bool XMLGenerator::next(ssize_t count)
+{
+ /// If negative step - something went wrong
+ if (count == -1)
+ {
+ error = true;
+ return false;
+ }
+
+ /// move data and increase counter
+ keys_size += count;
+
+ /// If get after eof
+ if (keys_size >= start_size)
+ {
+ error = true;
+ return false;
+ }
+ data += count;
+
+ return true;
+}
+
+/*
+key
+or
+key
+*/
+ssize_t XMLGenerator::generateKey(std::string name, bool multiple)
+{
+ /// set traditional key size for algorithms
+ uint64_t size = 0;
+ if (name == "aes_128_gcm_siv")
+ size = 16;
+ if (name == "aes_256_gcm_siv")
+ size = 32;
+
+ /// try to read size from data
+ if (first_byte & 0x40)
+ {
+ size = *(reinterpret_cast(data));
+ if (!next(8))
+ return -1;
+ }
+ /// if it is not defined, leave
+ if (!size)
+ return -1;
+
+ AutoPtr key_holder;
+ if (multiple)
+ {
+ /// multiple keys have ids.
+ uint64_t id = *(reinterpret_cast(data));
+ if (!next(8))
+ return -1;
+
+ key_holder = xml_document->createElement("key[id=" + std::to_string(id) + "]");
+
+ }
+ else
+ {
+ key_holder = xml_document->createElement("key");
+ }
+ AutoPtr key(xml_document->createTextNode(std::string(data, data + size)));
+ key_holder->appendChild(key);
+ algo->appendChild(key_holder);
+
+ if (!next(size))
+ return -1;
+ return size;
+}
+
+bool XMLGenerator::generateAlgorithmKeys(
+ AutoPtr& document_root, std::string name, uint8_t mask_for_algo, uint8_t mask_for_multiple_keys)
+{
+ /// check if algorithm is enabled, then add multiple keys or single key
+ if (first_byte & mask_for_algo)
+ {
+ algo = xml_document->createElement(name);
+ document_root->appendChild(algo);
+
+ if (first_byte & mask_for_multiple_keys)
+ {
+ uint64_t count = *(reinterpret_cast(data));
+ if (!next(8))
+ return false;
+
+ for (size_t i = 0; i < count; ++i)
+ {
+ if (!next(generateKey(name)))
+ return false;
+ }
+ }
+ else
+ {
+ if (!next(generateKey(name)))
+ return false;
+ }
+ }
+
+ /// add nonce
+ if (first_byte & 0x02)
+ {
+ uint64_t nonce_size = 12;
+ if (first_byte & 0x80)
+ {
+ nonce_size = *(reinterpret_cast(data));
+ if (!next(8))
+ return false;
+ }
+
+ AutoPtr nonce_holder(xml_document->createElement("nonce"));
+ AutoPtr nonce(xml_document->createTextNode(std::string(data, data + nonce_size)));
+ nonce_holder->appendChild(nonce);
+ algo->appendChild(nonce_holder);
+ }
+
+ /// add current key id
+ if (first_byte & 0x01)
+ {
+ uint64_t current_key = *(reinterpret_cast(data));
+ if (!next(8))
+ return false;
+
+ AutoPtr cur_key_holder(xml_document->createElement("nonce"));
+ AutoPtr cur_key(xml_document->createTextNode(std::to_string(current_key)));
+ cur_key_holder->appendChild(cur_key);
+ algo->appendChild(cur_key_holder);
+ }
+
+ return true;
+}
+
+void XMLGenerator::generate()
+{
+ AutoPtr document_root(xml_document->createElement("encryption_codecs"));
+ xml_document->appendChild(document_root);
+
+ /// read first byte for parsing
+ first_byte = *data;
+ if (!next())
+ return;
+
+ if (!generateAlgorithmKeys(document_root, "aes_128_gmc_siv", 0x20, 0x10))
+ return;
+ if (!generateAlgorithmKeys(document_root, "aes_256_gmc_siv", 0x08, 0x04))
+ return;
+
+ conf->load(xml_document);
+}
+
+}
+
+
extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size)
try
{
- if (size < sizeof(AuxiliaryRandomData))
+ XMLGenerator generator(data, size);
+
+ generator.generate();
+ if (generator.hasError())
return 0;
- const auto * p = reinterpret_cast(data);
+ auto config = generator.getResult();
+ auto codec_128 = getCompressionCodecEncrypted(DB::AES_128_GCM_SIV);
+ auto codec_256 = getCompressionCodecEncrypted(DB::AES_256_GCM_SIV);
+ DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, "");
- std::string key = std::string(p->key, key_size);
- auto codec = DB::getCompressionCodecEncrypted(key);
+ size_t data_size = size - generator.keySize();
- size_t output_buffer_size = p->decompressed_size % 65536;
- size -= sizeof(AuxiliaryRandomData);
- data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t);
-
- std::string input = std::string(reinterpret_cast(data), size);
- fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size);
-
- if (output_buffer_size < size)
- return 0;
+ std::string input = std::string(reinterpret_cast(data), data_size);
+ fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, data_size, input.size() - 31);
DB::Memory<> memory;
- memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer());
-
- codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size);
+ memory.resize(input.size() + codec_128->getAdditionalSizeAtTheEndOfBuffer());
+ codec_128->doDecompressData(input.data(), input.size(), memory.data(), input.size() - 31);
+ memory.resize(input.size() + codec_128->getAdditionalSizeAtTheEndOfBuffer());
+ codec_256->doDecompressData(input.data(), input.size(), memory.data(), input.size() - 31);
return 0;
}
catch (...)
diff --git a/src/Compression/ya.make b/src/Compression/ya.make
index 8ba48696225..420ada412e0 100644
--- a/src/Compression/ya.make
+++ b/src/Compression/ya.make
@@ -35,6 +35,11 @@ SRCS(
CompressionFactoryAdditions.cpp
ICompressionCodec.cpp
LZ4_decompress_faster.cpp
+ fuzzers/compressed_buffer_fuzzer.cpp
+ fuzzers/delta_decompress_fuzzer.cpp
+ fuzzers/double_delta_decompress_fuzzer.cpp
+ fuzzers/encrypted_decompress_fuzzer.cpp
+ fuzzers/lz4_decompress_fuzzer.cpp
getCompressionCodecForFile.cpp
)
diff --git a/tests/config/config.d/encryption.xml b/tests/config/config.d/encryption.xml
index 216021ae744..b1b84cf46c8 100644
--- a/tests/config/config.d/encryption.xml
+++ b/tests/config/config.d/encryption.xml
@@ -1,6 +1,10 @@
-
-
- echo "U29tZSBmaXhlZCBrZXkgdGhhdCBpcyBhdCBsZWFzdCAxNiBieXRlcyBsb25n"
-
+
+
+ a32902703dab1cedd7ff7287067787ca
+
+
+ a32902703dab1cedd7ff7287067787caa32902703dab1cedd7ff7287067787ca
+
+
diff --git a/tests/integration/test_codec_encrypted/__init__.py b/tests/integration/test_codec_encrypted/__init__.py
new file mode 100644
index 00000000000..e69de29bb2d
diff --git a/tests/integration/test_codec_encrypted/test.py b/tests/integration/test_codec_encrypted/test.py
new file mode 100644
index 00000000000..5d47948ac3c
--- /dev/null
+++ b/tests/integration/test_codec_encrypted/test.py
@@ -0,0 +1,73 @@
+import pytest
+from helpers.cluster import ClickHouseCluster
+from helpers.client import QueryRuntimeException
+from helpers.test_tools import assert_eq_with_retry
+
+cluster = ClickHouseCluster(__file__)
+
+node = cluster.add_instance('node')
+
+@pytest.fixture(scope="module")
+def start_cluster():
+ try:
+ cluster.start()
+ yield cluster
+
+ finally:
+ cluster.shutdown()
+
+def make_storage_with_key(id):
+ node.exec_in_container(["bash", "-c" , """cat > /etc/clickhouse-server/config.d/storage_keys_config.xml << EOF
+
+
+
+
+ 83e84e9a4eb11535c0670dc62d808ee0
+ abcdefghijklmnop
+ {cur_id}
+
+
+ 83e84e9a4eb11535c0670dc62d808ee083e84e9a4eb11535c0670dc62d808ee0
+ abcdefghijklmnopabcdefghijklmnop
+ {cur_id}
+
+
+
+EOF""".format(cur_id=id)])
+ node.query("SYSTEM RELOAD CONFIG")
+
+def test_different_keys(start_cluster):
+ make_storage_with_key(0)
+ node.query("""
+ CREATE TABLE encrypted_test_128 (
+ id Int64,
+ data String Codec(AES_128_GCM_SIV)
+ ) ENGINE=MergeTree()
+ ORDER BY id
+ """)
+
+ node.query("""
+ CREATE TABLE encrypted_test_256 (
+ id Int64,
+ data String Codec(AES_256_GCM_SIV)
+ ) ENGINE=MergeTree()
+ ORDER BY id
+ """)
+
+ node.query("INSERT INTO encrypted_test_128 VALUES (0,'data'),(1,'data')")
+ select_query = "SELECT * FROM encrypted_test_128 ORDER BY id FORMAT Values"
+ assert node.query(select_query) == "(0,'data'),(1,'data')"
+
+ make_storage_with_key(1)
+ node.query("INSERT INTO encrypted_test_128 VALUES (3,'text'),(4,'text')")
+ select_query = "SELECT * FROM encrypted_test_128 ORDER BY id FORMAT Values"
+ assert node.query(select_query) == "(0,'data'),(1,'data'),(3,'text'),(4,'text')"
+
+ node.query("INSERT INTO encrypted_test_256 VALUES (0,'data'),(1,'data')")
+ select_query = "SELECT * FROM encrypted_test_256 ORDER BY id FORMAT Values"
+ assert node.query(select_query) == "(0,'data'),(1,'data')"
+
+ make_storage_with_key(1)
+ node.query("INSERT INTO encrypted_test_256 VALUES (3,'text'),(4,'text')")
+ select_query = "SELECT * FROM encrypted_test_256 ORDER BY id FORMAT Values"
+ assert node.query(select_query) == "(0,'data'),(1,'data'),(3,'text'),(4,'text')"
diff --git a/tests/queries/0_stateless/01683_codec_encrypted.reference b/tests/queries/0_stateless/01683_codec_encrypted.reference
index 0d30be781e5..4bed8ab92b8 100644
--- a/tests/queries/0_stateless/01683_codec_encrypted.reference
+++ b/tests/queries/0_stateless/01683_codec_encrypted.reference
@@ -1 +1,2 @@
1 Some plaintext
+1 Some plaintext
diff --git a/tests/queries/0_stateless/01683_codec_encrypted.sql b/tests/queries/0_stateless/01683_codec_encrypted.sql
index e780819ebd3..d3ba1bb5e69 100644
--- a/tests/queries/0_stateless/01683_codec_encrypted.sql
+++ b/tests/queries/0_stateless/01683_codec_encrypted.sql
@@ -2,7 +2,14 @@
-- Tag no-fasttest: Depends on OpenSSL
DROP TABLE IF EXISTS encryption_test;
-CREATE TABLE encryption_test (i Int, s String Codec(Encrypted('AES-128-GCM-SIV'))) ENGINE = MergeTree ORDER BY i;
+CREATE TABLE encryption_test (i Int, s String Codec(AES_128_GCM_SIV)) ENGINE = MergeTree ORDER BY i;
+
+INSERT INTO encryption_test VALUES (1, 'Some plaintext');
+SELECT * FROM encryption_test;
+
+DROP TABLE encryption_test;
+
+CREATE TABLE encryption_test (i Int, s String Codec(AES_256_GCM_SIV)) ENGINE = MergeTree ORDER BY i;
INSERT INTO encryption_test VALUES (1, 'Some plaintext');
SELECT * FROM encryption_test;