Add a codec AES_128_GCM_SIV for encrypting columns on disk (#19896)

* Add a codec Encrypted() for encrypting columns on disk

While this is implemented as a compression codec, it does not actually compress data. It instead encrypts data on disk. The key is obtained by executing a user-specified command at the server startup, or if it's not specified the codec refuses to process any data. For now the only supported cipher is 'AES-128-GCM-SIV'.
This commit is contained in:
PHO 2021-07-30 18:12:33 +09:00 committed by GitHub
parent cae5502d51
commit 6425dd001a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 453 additions and 4 deletions

View File

@ -299,6 +299,7 @@ function run_tests
01318_decrypt # Depends on OpenSSL 01318_decrypt # Depends on OpenSSL
01663_aes_msan # Depends on OpenSSL 01663_aes_msan # Depends on OpenSSL
01667_aes_args_check # Depends on OpenSSL 01667_aes_args_check # Depends on OpenSSL
01683_codec_encrypted # Depends on OpenSSL
01776_decrypt_aead_size_check # Depends on OpenSSL 01776_decrypt_aead_size_check # Depends on OpenSSL
01811_filter_by_null # Depends on OpenSSL 01811_filter_by_null # Depends on OpenSSL
01281_unsucceeded_insert_select_queries_counter 01281_unsucceeded_insert_select_queries_counter

View File

@ -69,6 +69,28 @@ If no conditions met for a data part, ClickHouse uses the `lz4` compression.
</compression> </compression>
``` ```
## encryption {#server-settings-encryption}
Configures a command to obtain a key to be used by [encryption codecs](../../sql-reference/statements/create/table.md#create-query-encryption-codecs). The command, or a shell script, is expected to write a Base64-encoded key of any length to the stdout.
**Example**
For Linux with systemd:
```xml
<encryption>
<key_command>/usr/bin/systemd-ask-password --id="clickhouse-server" --timeout=0 "Enter the ClickHouse encryption passphrase:" | base64</key_command>
</encryption>
```
For other systems:
```xml
<encryption>
<key_command><![CDATA[IFS=; echo -n >/dev/tty "Enter the ClickHouse encryption passphrase: "; stty=`stty -F /dev/tty -g`; stty -F /dev/tty -echo; read k </dev/tty; stty -F /dev/tty "$stty"; echo -n $k | base64]]></key_command>
</encryption>
```
## custom_settings_prefixes {#custom_settings_prefixes} ## 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. List of prefixes for [custom settings](../../operations/settings/index.md#custom_settings). The prefixes must be separated with commas.

View File

@ -254,6 +254,20 @@ CREATE TABLE codec_example
ENGINE = MergeTree() ENGINE = MergeTree()
``` ```
### Encryption Codecs {#create-query-encryption-codecs}
These codecs don't actually compress data, but instead encrypt data on disk. These are only available when an encryption key is specified by [encryption](../../../operations/server-configuration-parameters/settings.md#server-settings-encryption) settings. Note that encryption only makes sense at the end of codec pipelines, because encrypted data usually can't be compressed in any meaningful way.
Encryption codecs:
- `Encrypted('AES-128-GCM-SIV')` — Encrypts data with AES-128 in [RFC 8452](https://tools.ietf.org/html/rfc8452) GCM-SIV mode. This codec uses a fixed nonce and encryption is therefore deterministic. This makes it compatible with deduplicating engines such as [ReplicatedMergeTree](../../../engines/table-engines/mergetree-family/replication.md) but has a weakness: when the same data block is encrypted twice, the resulting ciphertext will be exactly the same so an adversary who can read the disk can see this equivalence (although only the equivalence).
!!! attention "Attention"
Most engines including the "*MergeTree" family create index files on disk without applying codecs. This means plaintext will appear on disk if an encrypted column is indexed.
!!! attention "Attention"
If you perform a SELECT query mentioning a specific value in an encrypted column (such as in its WHERE clause), the value may appear in [system.query_log](../../../operations/system-tables/query_log.md). You may want to disable the logging.
## Temporary Tables {#temporary-tables} ## Temporary Tables {#temporary-tables}
ClickHouse supports temporary tables which have the following characteristics: ClickHouse supports temporary tables which have the following characteristics:

View File

@ -26,6 +26,7 @@
#include <Common/DNSResolver.h> #include <Common/DNSResolver.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <Common/Macros.h> #include <Common/Macros.h>
#include <Common/ShellCommand.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Common/ZooKeeper/ZooKeeper.h> #include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/ZooKeeperNodeCache.h> #include <Common/ZooKeeper/ZooKeeperNodeCache.h>
@ -39,6 +40,7 @@
#include <Common/remapExecutable.h> #include <Common/remapExecutable.h>
#include <Common/TLDListsHolder.h> #include <Common/TLDListsHolder.h>
#include <IO/HTTPCommon.h> #include <IO/HTTPCommon.h>
#include <IO/ReadHelpers.h>
#include <IO/UseSSL.h> #include <IO/UseSSL.h>
#include <Interpreters/AsynchronousMetrics.h> #include <Interpreters/AsynchronousMetrics.h>
#include <Interpreters/DDLWorker.h> #include <Interpreters/DDLWorker.h>
@ -95,6 +97,9 @@
#endif #endif
#if USE_SSL #if USE_SSL
# if USE_INTERNAL_SSL_LIBRARY
# include <Compression/CompressionCodecEncrypted.h>
# endif
# include <Poco/Net/Context.h> # include <Poco/Net/Context.h>
# include <Poco/Net/SecureServerSocket.h> # include <Poco/Net/SecureServerSocket.h>
#endif #endif
@ -107,6 +112,10 @@
# include <Server/KeeperTCPHandlerFactory.h> # include <Server/KeeperTCPHandlerFactory.h>
#endif #endif
#if USE_BASE64
# include <turbob64.h>
#endif
#if USE_JEMALLOC #if USE_JEMALLOC
# include <jemalloc/jemalloc.h> # include <jemalloc/jemalloc.h>
#endif #endif
@ -242,6 +251,7 @@ namespace ErrorCodes
extern const int SUPPORT_IS_DISABLED; extern const int SUPPORT_IS_DISABLED;
extern const int ARGUMENT_OUT_OF_BOUND; extern const int ARGUMENT_OUT_OF_BOUND;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG; extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int INCORRECT_DATA;
extern const int INVALID_CONFIG_PARAMETER; extern const int INVALID_CONFIG_PARAMETER;
extern const int SYSTEM_ERROR; extern const int SYSTEM_ERROR;
extern const int FAILED_TO_GETPWUID; extern const int FAILED_TO_GETPWUID;
@ -445,6 +455,39 @@ 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<char> buf(b64_key.size());
const size_t key_size = tb64dec(reinterpret_cast<const unsigned char *>(b64_key.data()), b64_key.size(),
reinterpret_cast<unsigned char *>(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() [[noreturn]] void forceShutdown()
{ {
@ -916,6 +959,10 @@ if (ThreadFuzzer::instance().isEffective())
global_context->getMergeTreeSettings().sanityCheck(settings); global_context->getMergeTreeSettings().sanityCheck(settings);
global_context->getReplicatedMergeTreeSettings().sanityCheck(settings); global_context->getReplicatedMergeTreeSettings().sanityCheck(settings);
/// Set up encryption.
if (config().has("encryption.key_command"))
loadEncryptionKey(config().getString("encryption.key_command"), log);
Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0); Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0);
Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024)); Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024));

View File

@ -1002,6 +1002,16 @@
</compression> </compression>
--> -->
<!-- Configuration of encryption. The server executes a command to
obtain an encryption key at startup if such a command is
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. -->
<encryption>
<!-- <key_command>/usr/bin/systemd-ask-password &#45;&#45;id="clickhouse-server" &#45;&#45;timeout=0 "Enter the ClickHouse encryption passphrase:" | base64</key_command> -->
<!-- <key_command><![CDATA[IFS=; echo -n >/dev/tty "Enter the ClickHouse encryption passphrase: "; stty=`stty -F /dev/tty -g`; stty -F /dev/tty -echo; read k </dev/tty; stty -F /dev/tty "$stty"; echo -n $k | base64]]></key_command> -->
</encryption>
<!-- Allow to execute distributed DDL queries (CREATE, DROP, ALTER, RENAME) on cluster. <!-- Allow to execute distributed DDL queries (CREATE, DROP, ALTER, RENAME) on cluster.
Works only if ZooKeeper is enabled. Comment it if such functionality isn't required. --> Works only if ZooKeeper is enabled. Comment it if such functionality isn't required. -->
<distributed_ddl> <distributed_ddl>

View File

@ -2,8 +2,10 @@
// .h autogenerated by cmake! // .h autogenerated by cmake!
#cmakedefine01 USE_BASE64
#cmakedefine01 USE_RE2_ST #cmakedefine01 USE_RE2_ST
#cmakedefine01 USE_SSL #cmakedefine01 USE_SSL
#cmakedefine01 USE_INTERNAL_SSL_LIBRARY
#cmakedefine01 USE_HDFS #cmakedefine01 USE_HDFS
#cmakedefine01 USE_INTERNAL_HDFS3_LIBRARY #cmakedefine01 USE_INTERNAL_HDFS3_LIBRARY
#cmakedefine01 USE_AWS_S3 #cmakedefine01 USE_AWS_S3

View File

@ -0,0 +1,212 @@
#include <Common/config.h>
#include <Compression/CompressionFactory.h>
#if USE_SSL && USE_INTERNAL_SSL_LIBRARY
#include <Compression/CompressionCodecEncrypted.h>
#include <Parsers/ASTLiteral.h>
#include <cassert>
#include <openssl/digest.h>
#include <openssl/err.h>
#include <openssl/hkdf.h>
#include <string_view>
namespace DB
{
namespace ErrorCodes
{
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;
}
void CompressionCodecEncrypted::setMasterKey(const std::string_view & master_key)
{
keys.emplace(master_key);
}
CompressionCodecEncrypted::KeyHolder::KeyHolder(const std::string_view & master_key)
{
// 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<const uint8_t*>(keygen_key.data()), keygen_key.size(),
16 /* tag size */, nullptr);
if (!ok)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
}
CompressionCodecEncrypted::KeyHolder::~KeyHolder()
{
EVP_AEAD_CTX_cleanup(&ctx);
}
const CompressionCodecEncrypted::KeyHolder & CompressionCodecEncrypted::getKeys()
{
if (keys)
return *keys;
else
throw Exception("There is no configuration for encryption in the server config",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
}
CompressionCodecEncrypted::CompressionCodecEncrypted(const std::string_view & cipher)
{
setCodecDescription("Encrypted", {std::make_shared<ASTLiteral>(cipher)});
}
uint8_t CompressionCodecEncrypted::getMethodByte() const
{
return static_cast<uint8_t>(CompressionMethodByte::Encrypted);
}
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).
return uncompressed_size + 16;
}
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 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;
}
void CompressionCodecEncrypted::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size [[maybe_unused]]) const
{
// 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);
}
std::string CompressionCodecEncrypted::lastErrorString()
{
std::array<char, 1024> buffer;
ERR_error_string_n(ERR_get_error(), buffer.data(), buffer.size());
return std::string(buffer.data());
}
std::string CompressionCodecEncrypted::deriveKey(const std::string_view & master_key)
{
std::string_view salt(""); // No salt: derive keys in a deterministic manner.
std::string_view info("Codec Encrypted('AES-128-GCM-SIV') key generation key");
std::array<char, 32> result;
const int ok = HKDF(reinterpret_cast<uint8_t *>(result.data()), result.size(),
EVP_sha256(),
reinterpret_cast<const uint8_t *>(master_key.data()), master_key.size(),
reinterpret_cast<const uint8_t *>(salt.data()), salt.size(),
reinterpret_cast<const uint8_t *>(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<uint8_t *>(ciphertext_and_tag),
&out_len, plaintext.size() + 16,
reinterpret_cast<const uint8_t *>(nonce.data()), nonce.size(),
reinterpret_cast<const uint8_t *>(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<uint8_t *>(plaintext),
&out_len, ciphertext.size(),
reinterpret_cast<const uint8_t *>(nonce.data()), nonce.size(),
reinterpret_cast<const uint8_t *>(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->children.size() != 1)
throw Exception("Codec Encrypted() must have 1 parameter, given " +
std::to_string(arguments->children.size()),
ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
const auto children = arguments->children;
const auto * literal = children[0]->as<ASTLiteral>();
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<String>();
if (cipher == "AES-128-GCM-SIV")
return std::make_shared<CompressionCodecEncrypted>(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<CompressionCodecEncrypted>("AES-128-GCM-SIV");
}
});
}
}
#else /* USE_SSL && USE_INTERNAL_SSL_LIBRARY */
namespace DB
{
void registerCodecEncrypted(CompressionCodecFactory &)
{
}
}
#endif /* USE_SSL && USE_INTERNAL_SSL_LIBRARY */

View File

@ -0,0 +1,104 @@
#pragma once
// This depends on BoringSSL-specific API, notably <openssl/aead.h>.
#include <Common/config.h>
#if USE_SSL && USE_INTERNAL_SSL_LIBRARY
#include <Compression/ICompressionCodec.h>
#include <boost/noncopyable.hpp>
#include <openssl/aead.h>
#include <optional>
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
{
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);
CompressionCodecEncrypted(const std::string_view & cipher);
uint8_t getMethodByte() const override;
void updateHash(SipHash & hash) const override;
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;
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
{
KeyHolder(const std::string_view & master_key);
~KeyHolder();
std::string keygen_key;
EVP_AEAD_CTX ctx;
};
static const KeyHolder & getKeys();
static inline std::optional<KeyHolder> keys;
};
}
#endif /* USE_SSL && USE_INTERNAL_SSL_LIBRARY */

View File

@ -79,6 +79,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
bool is_compression = false; bool is_compression = false;
bool has_none = false; bool has_none = false;
std::optional<size_t> generic_compression_codec_pos; std::optional<size_t> generic_compression_codec_pos;
std::set<size_t> post_processing_codecs;
bool can_substitute_codec_arguments = true; bool can_substitute_codec_arguments = true;
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i) for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
@ -156,6 +157,9 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
if (!generic_compression_codec_pos && result_codec->isGenericCompression()) if (!generic_compression_codec_pos && result_codec->isGenericCompression())
generic_compression_codec_pos = i; generic_compression_codec_pos = i;
if (result_codec->isPostProcessing())
post_processing_codecs.insert(i);
} }
String codec_description = queryToString(codecs_descriptions); String codec_description = queryToString(codecs_descriptions);
@ -170,7 +174,8 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
/// Allow to explicitly specify single NONE codec if user don't want any compression. /// 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. /// But applying other transformations solely without compression (e.g. Delta) does not make sense.
if (!is_compression && !has_none) /// 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())
throw Exception( throw Exception(
"Compression codec " + codec_description "Compression codec " + codec_description
+ " does not compress anything." + " does not compress anything."
@ -180,9 +185,19 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
ErrorCodes::BAD_ARGUMENTS); 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())
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'"
" to skip this check).", ErrorCodes::BAD_ARGUMENTS);
/// It does not make sense to apply any transformations after generic compression algorithm /// 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. /// 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) if (generic_compression_codec_pos &&
*generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - post_processing_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless," 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." " 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); " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS);
@ -337,6 +352,7 @@ void registerCodecDelta(CompressionCodecFactory & factory);
void registerCodecT64(CompressionCodecFactory & factory); void registerCodecT64(CompressionCodecFactory & factory);
void registerCodecDoubleDelta(CompressionCodecFactory & factory); void registerCodecDoubleDelta(CompressionCodecFactory & factory);
void registerCodecGorilla(CompressionCodecFactory & factory); void registerCodecGorilla(CompressionCodecFactory & factory);
void registerCodecEncrypted(CompressionCodecFactory & factory);
void registerCodecMultiple(CompressionCodecFactory & factory); void registerCodecMultiple(CompressionCodecFactory & factory);
CompressionCodecFactory::CompressionCodecFactory() CompressionCodecFactory::CompressionCodecFactory()
@ -349,6 +365,7 @@ CompressionCodecFactory::CompressionCodecFactory()
registerCodecT64(*this); registerCodecT64(*this);
registerCodecDoubleDelta(*this); registerCodecDoubleDelta(*this);
registerCodecGorilla(*this); registerCodecGorilla(*this);
registerCodecEncrypted(*this);
registerCodecMultiple(*this); registerCodecMultiple(*this);
default_codec = get("LZ4", {}); default_codec = get("LZ4", {});

View File

@ -43,6 +43,7 @@ enum class CompressionMethodByte : uint8_t
T64 = 0x93, T64 = 0x93,
DoubleDelta = 0x94, DoubleDelta = 0x94,
Gorilla = 0x95, Gorilla = 0x95,
Encrypted = 0x96,
}; };
} }

View File

@ -73,6 +73,9 @@ public:
/// Is it a generic compression algorithm like lz4, zstd. Usually it does not make sense to apply generic compression more than single time. /// Is it a generic compression algorithm like lz4, zstd. Usually it does not make sense to apply generic compression more than single time.
virtual bool isGenericCompression() const = 0; 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; }
/// It is a codec available only for evaluation purposes and not meant to be used in production. /// 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. /// It will not be allowed to use unless the user will turn off the safety switch.
virtual bool isExperimental() const { return false; } virtual bool isExperimental() const { return false; }

View File

@ -24,6 +24,7 @@ SRCS(
CompressedWriteBuffer.cpp CompressedWriteBuffer.cpp
CompressionCodecDelta.cpp CompressionCodecDelta.cpp
CompressionCodecDoubleDelta.cpp CompressionCodecDoubleDelta.cpp
CompressionCodecEncrypted.cpp
CompressionCodecGorilla.cpp CompressionCodecGorilla.cpp
CompressionCodecLZ4.cpp CompressionCodecLZ4.cpp
CompressionCodecMultiple.cpp CompressionCodecMultiple.cpp

View File

@ -0,0 +1,6 @@
<yandex>
<encryption>
<!-- "Some fixed key that is at least 16 bytes long" in Base64 -->
<key_command>echo "U29tZSBmaXhlZCBrZXkgdGhhdCBpcyBhdCBsZWFzdCAxNiBieXRlcyBsb25n"</key_command>
</encryption>
</yandex>

View File

@ -34,6 +34,7 @@ ln -sf $SRC_PATH/config.d/logging_no_rotate.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/top_level_domains_path.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_path.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/encryption.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/zookeeper_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/zookeeper_log.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/

View File

@ -0,0 +1 @@
1 Some plaintext

View File

@ -0,0 +1,7 @@
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;
INSERT INTO encryption_test VALUES (1, 'Some plaintext');
SELECT * FROM encryption_test;
DROP TABLE encryption_test;