```
create table kafka
(
a UInt32,
a_str String Alias toString(a)
) engine = Kafka;
create table data
(
a UInt32;
a_str String
) engine = MergeTree
order by tuple();
create materialized view data_mv to data
(
a UInt32,
a_str String
) as
select a, a_str from kafka;
```
Alias type works as expected in comparison with MATERIALIZED/EPHEMERAL
or column with default expression.
Ref: https://github.com/ClickHouse/ClickHouse/pull/47138
Co-authored-by: Azat Khuzhin <a3at.mail@gmail.com>
https://docs.aws.amazon.com/AmazonS3/latest/userguide/UsingKMSEncryption.html
Similar to the server_side_encryption_customer_key_base64 option for
configuring SSE-C with S3, add the following settings to configure
SSE-KMS on a per-endpoint/disk basis:
- server_side_encryption_kms_key_id
- server_side_encryption_kms_encryption_context
- server_side_encryption_kms_bucket_key_enabled
The S3 table engine supports specifying extra HTTP headers in S3
requests to certain endpoints, via the "headers" setting. This commit
adds the same setting to S3 disk config.
After #36425 there was a lot of confusions/problems with configuring pools - when the message was confusing, and settings need to be ajusted in several places.
See some examples in #44251, #43351, #47900, #46515.
The commit includes the following changes:
1) Introduced a unified mechanism for reading pool sizes from the configuration file(s). Previously, pool sizes were read from the Context.cpp with fallbacks to profiles, whereas main_config_reloader in Server.cpp read them directly without fallbacks.
2) Corrected the data type for background_merges_mutations_concurrency_ratio. It should be float instead of int.
3) Refactored the default values for settings. Previously, they were defined in multiple places throughout the codebase, but they are now defined in one place (or two, to be exact: Settings.h and ServerSettings.h).
4) Improved documentation, including the correct message in system.settings.
Additionally make the code more conform with #46550.
The Kafka table engine allows global configuration and per-Kafka-topic
configuration. The latter uses syntax <kafka_TOPIC>, e.g. for topic
"football":
<kafka_football>
<retry_backoff_ms>250</retry_backoff_ms>
<fetch_min_bytes>100000</fetch_min_bytes>
</kafka_football>
Some users had to find out the hard way that such configuration doesn't
take effect if the topic name contains a period, e.g. "sports.football".
The reason is that ClickHouse configuration framework already uses
periods as level separators to descend the configuration hierarchy.
(Besides that, per-topic configuration at the same level as global
configuration could be considered ugly.)
Note that Kafka topics may contain characters "a-zA-Z0-9._-" (*) and
a tree-like topic organization using periods is quite common in
practice.
This PR deprecates the existing per-topic configuration syntax (but
continues to support it for backward compat) and introduces a new
per-topic configuration syntax below the global Kafka configuration of
the form:
<kafka>
<topic name="football">
<retry_backoff_ms>250</retry_backoff_ms>
<fetch_min_bytes>100000</fetch_min_bytes>
</topic>
</kafka>
The period restriction doesn't apply to XML attributes, so <topic
name="sports.football"> will work. Also, everything Kafka-related is
below <kafka>.
Considered but rejected alternatives:
- Extending Poco ConfigurationView with custom separators (e.g."/"
instead of "."). Won't work easily because ConfigurationView only
builds a path but defers descending the configuration tree to the
normal configuration classes.
- Reloading the configuration file in StorageKafka (instead of reading
the loaded file) but with a custom separator. This mode is supported
by XML configuration. Too ugly and error-prone since the true
configuration is composed from multiple configuration files.
(*) https://stackoverflow.com/a/37067544