diff --git a/CHANGELOG.md b/CHANGELOG.md
index 1d11ddcfee7..1b36142cc9f 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
### Table of Contents
+**[ClickHouse release v23.12, 2023-12-28](#2312)**
**[ClickHouse release v23.11, 2023-12-06](#2311)**
**[ClickHouse release v23.10, 2023-11-02](#2310)**
**[ClickHouse release v23.9, 2023-09-28](#239)**
@@ -14,6 +15,146 @@
# 2023 Changelog
+### ClickHouse release 23.12, 2023-12-28
+
+#### Backward Incompatible Change
+* Fix check for non-deterministic functions in TTL expressions. Previously, you could create a TTL expression with non-deterministic functions in some cases, which could lead to undefined behavior later. This fixes [#37250](https://github.com/ClickHouse/ClickHouse/issues/37250). Disallow TTL expressions that don't depend on any columns of a table by default. It can be allowed back by `SET allow_suspicious_ttl_expressions = 1` or `SET compatibility = '23.11'`. Closes [#37286](https://github.com/ClickHouse/ClickHouse/issues/37286). [#51858](https://github.com/ClickHouse/ClickHouse/pull/51858) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* The MergeTree setting `clean_deleted_rows` is deprecated, it has no effect anymore. The `CLEANUP` keyword for the `OPTIMIZE` is not allowed by default (it can be unlocked with the `allow_experimental_replacing_merge_with_cleanup` setting). [#58267](https://github.com/ClickHouse/ClickHouse/pull/58267) ([Alexander Tokmakov](https://github.com/tavplubix)). This fixes [#57930](https://github.com/ClickHouse/ClickHouse/issues/57930). This closes [#54988](https://github.com/ClickHouse/ClickHouse/issues/54988). This closes [#54570](https://github.com/ClickHouse/ClickHouse/issues/54570). This closes [#50346](https://github.com/ClickHouse/ClickHouse/issues/50346). This closes [#47579](https://github.com/ClickHouse/ClickHouse/issues/47579). The feature has to be removed because it is not good. We have to remove it as quickly as possible, because there is no other option. [#57932](https://github.com/ClickHouse/ClickHouse/pull/57932) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+
+#### New Feature
+* Implement Refreshable Materialized Views, requested in [#33919](https://github.com/ClickHouse/ClickHouse/issues/33919). [#56946](https://github.com/ClickHouse/ClickHouse/pull/56946) ([Michael Kolupaev](https://github.com/al13n321), [Michael Guzov](https://github.com/koloshmet)).
+* Introduce `PASTE JOIN`, which allows users to join tables without `ON` clause simply by row numbers. Example: `SELECT * FROM (SELECT number AS a FROM numbers(2)) AS t1 PASTE JOIN (SELECT number AS a FROM numbers(2) ORDER BY a DESC) AS t2`. [#57995](https://github.com/ClickHouse/ClickHouse/pull/57995) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
+* The `ORDER BY` clause now supports specifying `ALL`, meaning that ClickHouse sorts by all columns in the `SELECT` clause. Example: `SELECT col1, col2 FROM tab WHERE [...] ORDER BY ALL`. [#57875](https://github.com/ClickHouse/ClickHouse/pull/57875) ([zhongyuankai](https://github.com/zhongyuankai)).
+* Added a new mutation command `ALTER TABLE
APPLY DELETED MASK`, which allows to enforce applying of mask written by lightweight delete and to remove rows marked as deleted from disk. [#57433](https://github.com/ClickHouse/ClickHouse/pull/57433) ([Anton Popov](https://github.com/CurtizJ)).
+* A handler `/binary` opens a visual viewer of symbols inside the ClickHouse binary. [#58211](https://github.com/ClickHouse/ClickHouse/pull/58211) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Added a new SQL function `sqid` to generate Sqids (https://sqids.org/), example: `SELECT sqid(125, 126)`. [#57512](https://github.com/ClickHouse/ClickHouse/pull/57512) ([Robert Schulze](https://github.com/rschu1ze)).
+* Add a new function `seriesPeriodDetectFFT` to detect series period using FFT. [#57574](https://github.com/ClickHouse/ClickHouse/pull/57574) ([Bhavna Jindal](https://github.com/bhavnajindal)).
+* Add an HTTP endpoint for checking if Keeper is ready to accept traffic. [#55876](https://github.com/ClickHouse/ClickHouse/pull/55876) ([Konstantin Bogdanov](https://github.com/thevar1able)).
+* Add 'union' mode for schema inference. In this mode the resulting table schema is the union of all files schemas (so schema is inferred from each file). The mode of schema inference is controlled by a setting `schema_inference_mode` with two possible values - `default` and `union`. Closes [#55428](https://github.com/ClickHouse/ClickHouse/issues/55428). [#55892](https://github.com/ClickHouse/ClickHouse/pull/55892) ([Kruglov Pavel](https://github.com/Avogar)).
+* Add new setting `input_format_csv_try_infer_numbers_from_strings` that allows to infer numbers from strings in CSV format. Closes [#56455](https://github.com/ClickHouse/ClickHouse/issues/56455). [#56859](https://github.com/ClickHouse/ClickHouse/pull/56859) ([Kruglov Pavel](https://github.com/Avogar)).
+* When the number of databases or tables exceeds a configurable threshold, show a warning to the user. [#57375](https://github.com/ClickHouse/ClickHouse/pull/57375) ([凌涛](https://github.com/lingtaolf)).
+* Dictionary with `HASHED_ARRAY` (and `COMPLEX_KEY_HASHED_ARRAY`) layout supports `SHARDS` similarly to `HASHED`. [#57544](https://github.com/ClickHouse/ClickHouse/pull/57544) ([vdimir](https://github.com/vdimir)).
+* Add asynchronous metrics for total primary key bytes and total allocated primary key bytes in memory. [#57551](https://github.com/ClickHouse/ClickHouse/pull/57551) ([Bharat Nallan](https://github.com/bharatnc)).
+* Add `SHA512_256` function. [#57645](https://github.com/ClickHouse/ClickHouse/pull/57645) ([Bharat Nallan](https://github.com/bharatnc)).
+* Add `FORMAT_BYTES` as an alias for `formatReadableSize`. [#57592](https://github.com/ClickHouse/ClickHouse/pull/57592) ([Bharat Nallan](https://github.com/bharatnc)).
+* Allow passing optional session token to the `s3` table function. [#57850](https://github.com/ClickHouse/ClickHouse/pull/57850) ([Shani Elharrar](https://github.com/shanielh)).
+* Introduce a new setting `http_make_head_request`. If it is turned off, the URL table engine will not do a HEAD request to determine the file size. This is needed to support inefficient, misconfigured, or not capable HTTP servers. [#54602](https://github.com/ClickHouse/ClickHouse/pull/54602) ([Fionera](https://github.com/fionera)).
+* It is now possible to refer to ALIAS column in index (non-primary-key) definitions (issue [#55650](https://github.com/ClickHouse/ClickHouse/issues/55650)). Example: `CREATE TABLE tab(col UInt32, col_alias ALIAS col + 1, INDEX idx (col_alias) TYPE minmax) ENGINE = MergeTree ORDER BY col;`. [#57546](https://github.com/ClickHouse/ClickHouse/pull/57546) ([Robert Schulze](https://github.com/rschu1ze)).
+* Added a new setting `readonly` which can be used to specify an S3 disk is read only. It can be useful to create a table on a disk of `s3_plain` type, while having read only access to the underlying S3 bucket. [#57977](https://github.com/ClickHouse/ClickHouse/pull/57977) ([Pengyuan Bian](https://github.com/bianpengyuan)).
+* The primary key analysis in MergeTree tables will now be applied to predicates that include the virtual column `_part_offset` (optionally with `_part`). This feature can serve as a special kind of a secondary index. [#58224](https://github.com/ClickHouse/ClickHouse/pull/58224) ([Amos Bird](https://github.com/amosbird)).
+
+#### Performance Improvement
+* Extract non-intersecting parts ranges from MergeTree table during FINAL processing. That way we can avoid additional FINAL logic for this non-intersecting parts ranges. In case when amount of duplicate values with same primary key is low, performance will be almost the same as without FINAL. Improve reading performance for MergeTree FINAL when `do_not_merge_across_partitions_select_final` setting is set. [#58120](https://github.com/ClickHouse/ClickHouse/pull/58120) ([Maksim Kita](https://github.com/kitaisreal)).
+* Made copy between s3 disks using a s3-server-side copy instead of copying through the buffer. Improves `BACKUP/RESTORE` operations and `clickhouse-disks copy` command. [#56744](https://github.com/ClickHouse/ClickHouse/pull/56744) ([MikhailBurdukov](https://github.com/MikhailBurdukov)).
+* Hash JOIN respects setting `max_joined_block_size_rows` and do not produce large blocks for `ALL JOIN`. [#56996](https://github.com/ClickHouse/ClickHouse/pull/56996) ([vdimir](https://github.com/vdimir)).
+* Release memory for aggregation earlier. This may avoid unnecessary external aggregation. [#57691](https://github.com/ClickHouse/ClickHouse/pull/57691) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
+* Improve performance of string serialization. [#57717](https://github.com/ClickHouse/ClickHouse/pull/57717) ([Maksim Kita](https://github.com/kitaisreal)).
+* Support trivial count optimization for `Merge`-engine tables. [#57867](https://github.com/ClickHouse/ClickHouse/pull/57867) ([skyoct](https://github.com/skyoct)).
+* Optimized aggregation in some cases. [#57872](https://github.com/ClickHouse/ClickHouse/pull/57872) ([Anton Popov](https://github.com/CurtizJ)).
+* The `hasAny` function can now take advantage of the full-text skipping indices. [#57878](https://github.com/ClickHouse/ClickHouse/pull/57878) ([Jpnock](https://github.com/Jpnock)).
+* Function `if(cond, then, else)` (and its alias `cond ? then : else`) were optimized to use branch-free evaluation. [#57885](https://github.com/ClickHouse/ClickHouse/pull/57885) ([zhanglistar](https://github.com/zhanglistar)).
+* MergeTree automatically derive `do_not_merge_across_partitions_select_final` setting if partition key expression contains only columns from primary key expression. [#58218](https://github.com/ClickHouse/ClickHouse/pull/58218) ([Maksim Kita](https://github.com/kitaisreal)).
+* Speedup `MIN` and `MAX` for native types. [#58231](https://github.com/ClickHouse/ClickHouse/pull/58231) ([Raúl Marín](https://github.com/Algunenano)).
+* Implement `SLRU` cache policy for filesystem cache. [#57076](https://github.com/ClickHouse/ClickHouse/pull/57076) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* The limit for the number of connections per endpoint for background fetches was raised from `15` to the value of `background_fetches_pool_size` setting. - MergeTree-level setting `replicated_max_parallel_fetches_for_host` became obsolete - MergeTree-level settings `replicated_fetches_http_connection_timeout`, `replicated_fetches_http_send_timeout` and `replicated_fetches_http_receive_timeout` are moved to the Server-level. - Setting `keep_alive_timeout` is added to the list of Server-level settings. [#57523](https://github.com/ClickHouse/ClickHouse/pull/57523) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
+* Make querying `system.filesystem_cache` not memory intensive. [#57687](https://github.com/ClickHouse/ClickHouse/pull/57687) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Reduce memory usage on strings deserialization. [#57787](https://github.com/ClickHouse/ClickHouse/pull/57787) ([Maksim Kita](https://github.com/kitaisreal)).
+* More efficient constructor for Enum - it makes sense when Enum has a boatload of values. [#57887](https://github.com/ClickHouse/ClickHouse/pull/57887) ([Duc Canh Le](https://github.com/canhld94)).
+* An improvement for reading from the filesystem cache: always use `pread` method. [#57970](https://github.com/ClickHouse/ClickHouse/pull/57970) ([Nikita Taranov](https://github.com/nickitat)).
+* Add optimization for AND notEquals chain in logical expression optimizer. This optimization is only available with the experimental Analyzer enabled. [#58214](https://github.com/ClickHouse/ClickHouse/pull/58214) ([Kevin Mingtarja](https://github.com/kevinmingtarja)).
+
+#### Improvement
+* Support for soft memory limit in Keeper. It will refuse requests if the memory usage is close to the maximum. [#57271](https://github.com/ClickHouse/ClickHouse/pull/57271) ([Han Fei](https://github.com/hanfei1991)). [#57699](https://github.com/ClickHouse/ClickHouse/pull/57699) ([Han Fei](https://github.com/hanfei1991)).
+* Make inserts into distributed tables handle updated cluster configuration properly. When the list of cluster nodes is dynamically updated, the Directory Monitor of the distribution table will update it. [#42826](https://github.com/ClickHouse/ClickHouse/pull/42826) ([zhongyuankai](https://github.com/zhongyuankai)).
+* Do not allow creating a replicated table with inconsistent merge parameters. [#56833](https://github.com/ClickHouse/ClickHouse/pull/56833) ([Duc Canh Le](https://github.com/canhld94)).
+* Show uncompressed size in `system.tables`. [#56618](https://github.com/ClickHouse/ClickHouse/issues/56618). [#57186](https://github.com/ClickHouse/ClickHouse/pull/57186) ([Chen Lixiang](https://github.com/chenlx0)).
+* Add `skip_unavailable_shards` as a setting for `Distributed` tables that is similar to the corresponding query-level setting. Closes [#43666](https://github.com/ClickHouse/ClickHouse/issues/43666). [#57218](https://github.com/ClickHouse/ClickHouse/pull/57218) ([Gagan Goel](https://github.com/tntnatbry)).
+* The function `substring` (aliases: `substr`, `mid`) can now be used with `Enum` types. Previously, the first function argument had to be a value of type `String` or `FixedString`. This improves compatibility with 3rd party tools such as Tableau via MySQL interface. [#57277](https://github.com/ClickHouse/ClickHouse/pull/57277) ([Serge Klochkov](https://github.com/slvrtrn)).
+* Function `format` now supports arbitrary argument types (instead of only `String` and `FixedString` arguments). This is important to calculate `SELECT format('The {0} to all questions is {1}', 'answer', 42)`. [#57549](https://github.com/ClickHouse/ClickHouse/pull/57549) ([Robert Schulze](https://github.com/rschu1ze)).
+* Allows to use the `date_trunc` function with a case-insensitive first argument. Both cases are now supported: `SELECT date_trunc('day', now())` and `SELECT date_trunc('DAY', now())`. [#57624](https://github.com/ClickHouse/ClickHouse/pull/57624) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
+* Better hints when a table doesn't exist. [#57342](https://github.com/ClickHouse/ClickHouse/pull/57342) ([Bharat Nallan](https://github.com/bharatnc)).
+* Allow to overwrite `max_partition_size_to_drop` and `max_table_size_to_drop` server settings in query time. [#57452](https://github.com/ClickHouse/ClickHouse/pull/57452) ([Jordi Villar](https://github.com/jrdi)).
+* Slightly better inference of unnamed tupes in JSON formats. [#57751](https://github.com/ClickHouse/ClickHouse/pull/57751) ([Kruglov Pavel](https://github.com/Avogar)).
+* Add support for read-only flag when connecting to Keeper (fixes [#53749](https://github.com/ClickHouse/ClickHouse/issues/53749)). [#57479](https://github.com/ClickHouse/ClickHouse/pull/57479) ([Mikhail Koviazin](https://github.com/mkmkme)).
+* Fix possible distributed sends stuck due to "No such file or directory" (during recovering a batch from disk). Fix possible issues with `error_count` from `system.distribution_queue` (in case of `distributed_directory_monitor_max_sleep_time_ms` >5min). Introduce profile event to track async INSERT failures - `DistributedAsyncInsertionFailures`. [#57480](https://github.com/ClickHouse/ClickHouse/pull/57480) ([Azat Khuzhin](https://github.com/azat)).
+* Support PostgreSQL generated columns and default column values in `MaterializedPostgreSQL` (experimental feature). Closes [#40449](https://github.com/ClickHouse/ClickHouse/issues/40449). [#57568](https://github.com/ClickHouse/ClickHouse/pull/57568) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Allow to apply some filesystem cache config settings changes without server restart. [#57578](https://github.com/ClickHouse/ClickHouse/pull/57578) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Properly handling PostgreSQL table structure with empty array. [#57618](https://github.com/ClickHouse/ClickHouse/pull/57618) ([Mike Kot](https://github.com/myrrc)).
+* Expose the total number of errors occurred since last server restart as a `ClickHouseErrorMetric_ALL` metric. [#57627](https://github.com/ClickHouse/ClickHouse/pull/57627) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
+* Allow nodes in the configuration file with `from_env`/`from_zk` reference and non empty element with replace=1. [#57628](https://github.com/ClickHouse/ClickHouse/pull/57628) ([Azat Khuzhin](https://github.com/azat)).
+* A table function `fuzzJSON` which allows generating a lot of malformed JSON for fuzzing. [#57646](https://github.com/ClickHouse/ClickHouse/pull/57646) ([Julia Kartseva](https://github.com/jkartseva)).
+* Allow IPv6 to UInt128 conversion and binary arithmetic. [#57707](https://github.com/ClickHouse/ClickHouse/pull/57707) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
+* Add a setting for `async inserts deduplication cache` - how long we wait for cache update. Deprecate setting `async_block_ids_cache_min_update_interval_ms`. Now cache is updated only in case of conflicts. [#57743](https://github.com/ClickHouse/ClickHouse/pull/57743) ([alesapin](https://github.com/alesapin)).
+* `sleep()` function now can be cancelled with `KILL QUERY`. [#57746](https://github.com/ClickHouse/ClickHouse/pull/57746) ([Vitaly Baranov](https://github.com/vitlibar)).
+* Forbid `CREATE TABLE ... AS SELECT` queries for `Replicated` table engines in the experimental `Replicated` database because they are not supported. Reference [#35408](https://github.com/ClickHouse/ClickHouse/issues/35408). [#57796](https://github.com/ClickHouse/ClickHouse/pull/57796) ([Nikolay Degterinsky](https://github.com/evillique)).
+* Fix and improve transforming queries for external databases, to recursively obtain all compatible predicates. [#57888](https://github.com/ClickHouse/ClickHouse/pull/57888) ([flynn](https://github.com/ucasfl)).
+* Support dynamic reloading of the filesystem cache size. Closes [#57866](https://github.com/ClickHouse/ClickHouse/issues/57866). [#57897](https://github.com/ClickHouse/ClickHouse/pull/57897) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Correctly support `system.stack_trace` for threads with blocked SIGRTMIN (these threads can exist in low-quality external libraries such as Apache rdkafka). [#57907](https://github.com/ClickHouse/ClickHouse/pull/57907) ([Azat Khuzhin](https://github.com/azat)). Aand also send signal to the threads only if it is not blocked to avoid waiting `storage_system_stack_trace_pipe_read_timeout_ms` when it does not make any sense. [#58136](https://github.com/ClickHouse/ClickHouse/pull/58136) ([Azat Khuzhin](https://github.com/azat)).
+* Tolerate keeper failures in the quorum inserts' check. [#57986](https://github.com/ClickHouse/ClickHouse/pull/57986) ([Raúl Marín](https://github.com/Algunenano)).
+* Add max/peak RSS (`MemoryResidentMax`) into system.asynchronous_metrics. [#58095](https://github.com/ClickHouse/ClickHouse/pull/58095) ([Azat Khuzhin](https://github.com/azat)).
+* This PR allows users to use s3-style links (`https://` and `s3://`) without mentioning region if it's not default. Also find the correct region if the user mentioned the wrong one. [#58148](https://github.com/ClickHouse/ClickHouse/pull/58148) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
+* `clickhouse-format --obfuscate` will know about Settings, MergeTreeSettings, and time zones and keep their names unchanged. [#58179](https://github.com/ClickHouse/ClickHouse/pull/58179) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Added explicit `finalize()` function in `ZipArchiveWriter`. Simplify too complicated code in `ZipArchiveWriter`. This fixes [#58074](https://github.com/ClickHouse/ClickHouse/issues/58074). [#58202](https://github.com/ClickHouse/ClickHouse/pull/58202) ([Vitaly Baranov](https://github.com/vitlibar)).
+* Make caches with the same path use the same cache objects. This behaviour existed before, but was broken in 23.4. If such caches with the same path have different set of cache settings, an exception will be thrown, that this is not allowed. [#58264](https://github.com/ClickHouse/ClickHouse/pull/58264) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Parallel replicas (experimental feature): friendly settings [#57542](https://github.com/ClickHouse/ClickHouse/pull/57542) ([Igor Nikonov](https://github.com/devcrafter)).
+* Parallel replicas (experimental feature): announcement response handling improvement [#57749](https://github.com/ClickHouse/ClickHouse/pull/57749) ([Igor Nikonov](https://github.com/devcrafter)).
+* Parallel replicas (experimental feature): give more respect to `min_number_of_marks` in `ParallelReplicasReadingCoordinator` [#57763](https://github.com/ClickHouse/ClickHouse/pull/57763) ([Nikita Taranov](https://github.com/nickitat)).
+* Parallel replicas (experimental feature): disable parallel replicas with IN (subquery) [#58133](https://github.com/ClickHouse/ClickHouse/pull/58133) ([Igor Nikonov](https://github.com/devcrafter)).
+* Parallel replicas (experimental feature): add profile event 'ParallelReplicasUsedCount' [#58173](https://github.com/ClickHouse/ClickHouse/pull/58173) ([Igor Nikonov](https://github.com/devcrafter)).
+* Non POST requests such as HEAD will be readonly similar to GET. [#58060](https://github.com/ClickHouse/ClickHouse/pull/58060) ([San](https://github.com/santrancisco)).
+* Add `bytes_uncompressed` column to `system.part_log` [#58167](https://github.com/ClickHouse/ClickHouse/pull/58167) ([Jordi Villar](https://github.com/jrdi)).
+* Add base backup name to `system.backups` and `system.backup_log` tables [#58178](https://github.com/ClickHouse/ClickHouse/pull/58178) ([Pradeep Chhetri](https://github.com/chhetripradeep)).
+* Add support for specifying query parameters in the command line in clickhouse-local [#58210](https://github.com/ClickHouse/ClickHouse/pull/58210) ([Pradeep Chhetri](https://github.com/chhetripradeep)).
+
+#### Build/Testing/Packaging Improvement
+* Randomize more settings [#39663](https://github.com/ClickHouse/ClickHouse/pull/39663) ([Anton Popov](https://github.com/CurtizJ)).
+* Randomize disabled optimizations in CI [#57315](https://github.com/ClickHouse/ClickHouse/pull/57315) ([Raúl Marín](https://github.com/Algunenano)).
+* Allow usage of Azure-related table engines/functions on macOS. [#51866](https://github.com/ClickHouse/ClickHouse/pull/51866) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* ClickHouse Fast Test now uses Musl instead of GLibc. [#57711](https://github.com/ClickHouse/ClickHouse/pull/57711) ([Alexey Milovidov](https://github.com/alexey-milovidov)). The fully-static Musl build is available to download from the CI.
+* Run ClickBench for every commit. This closes [#57708](https://github.com/ClickHouse/ClickHouse/issues/57708). [#57712](https://github.com/ClickHouse/ClickHouse/pull/57712) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Remove the usage of a harmful C/POSIX `select` function from external libraries. [#57467](https://github.com/ClickHouse/ClickHouse/pull/57467) ([Igor Nikonov](https://github.com/devcrafter)).
+* Settings only available in ClickHouse Cloud will be also present in the open-source ClickHouse build for convenience. [#57638](https://github.com/ClickHouse/ClickHouse/pull/57638) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
+
+#### Bug Fix (user-visible misbehavior in an official stable release)
+* Fixed a possibility of sorting order breakage in TTL GROUP BY [#49103](https://github.com/ClickHouse/ClickHouse/pull/49103) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
+* Fix: split `lttb` bucket strategy, first bucket and last bucket should only contain single point [#57003](https://github.com/ClickHouse/ClickHouse/pull/57003) ([FFish](https://github.com/wxybear)).
+* Fix possible deadlock in the `Template` format during sync after error [#57004](https://github.com/ClickHouse/ClickHouse/pull/57004) ([Kruglov Pavel](https://github.com/Avogar)).
+* Fix early stop while parsing a file with skipping lots of errors [#57006](https://github.com/ClickHouse/ClickHouse/pull/57006) ([Kruglov Pavel](https://github.com/Avogar)).
+* Prevent dictionary's ACL bypass via the `dictionary` table function [#57362](https://github.com/ClickHouse/ClickHouse/pull/57362) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
+* Fix another case of a "non-ready set" error found by Fuzzer. [#57423](https://github.com/ClickHouse/ClickHouse/pull/57423) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
+* Fix several issues regarding PostgreSQL `array_ndims` usage. [#57436](https://github.com/ClickHouse/ClickHouse/pull/57436) ([Ryan Jacobs](https://github.com/ryanmjacobs)).
+* Fix RWLock inconsistency after write lock timeout [#57454](https://github.com/ClickHouse/ClickHouse/pull/57454) ([Vitaly Baranov](https://github.com/vitlibar)). Fix RWLock inconsistency after write lock timeout (again) [#57733](https://github.com/ClickHouse/ClickHouse/pull/57733) ([Vitaly Baranov](https://github.com/vitlibar)).
+* Fix: don't exclude ephemeral column when building pushing to view chain [#57461](https://github.com/ClickHouse/ClickHouse/pull/57461) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
+* MaterializedPostgreSQL (experimental issue): fix issue [#41922](https://github.com/ClickHouse/ClickHouse/issues/41922), add test for [#41923](https://github.com/ClickHouse/ClickHouse/issues/41923) [#57515](https://github.com/ClickHouse/ClickHouse/pull/57515) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Ignore ON CLUSTER clause in grant/revoke queries for management of replicated access entities. [#57538](https://github.com/ClickHouse/ClickHouse/pull/57538) ([MikhailBurdukov](https://github.com/MikhailBurdukov)).
+* Fix crash in clickhouse-local [#57553](https://github.com/ClickHouse/ClickHouse/pull/57553) ([Nikolay Degterinsky](https://github.com/evillique)).
+* A fix for Hash JOIN. [#57564](https://github.com/ClickHouse/ClickHouse/pull/57564) ([vdimir](https://github.com/vdimir)).
+* Fix possible error in PostgreSQL source [#57567](https://github.com/ClickHouse/ClickHouse/pull/57567) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Fix type correction in Hash JOIN for nested LowCardinality. [#57614](https://github.com/ClickHouse/ClickHouse/pull/57614) ([vdimir](https://github.com/vdimir)).
+* Avoid hangs of `system.stack_trace` by correctly prohibiting parallel reading from it. [#57641](https://github.com/ClickHouse/ClickHouse/pull/57641) ([Azat Khuzhin](https://github.com/azat)).
+* Fix an error for aggregation of sparse columns with `any(...) RESPECT NULL` [#57710](https://github.com/ClickHouse/ClickHouse/pull/57710) ([Azat Khuzhin](https://github.com/azat)).
+* Fix unary operators parsing [#57713](https://github.com/ClickHouse/ClickHouse/pull/57713) ([Nikolay Degterinsky](https://github.com/evillique)).
+* Fix dependency loading for the experimental table engine `MaterializedPostgreSQL`. [#57754](https://github.com/ClickHouse/ClickHouse/pull/57754) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Fix retries for disconnected nodes for BACKUP/RESTORE ON CLUSTER [#57764](https://github.com/ClickHouse/ClickHouse/pull/57764) ([Vitaly Baranov](https://github.com/vitlibar)).
+* Fix result of external aggregation in case of partially materialized projection [#57790](https://github.com/ClickHouse/ClickHouse/pull/57790) ([Anton Popov](https://github.com/CurtizJ)).
+* Fix merge in aggregation functions with `*Map` combinator [#57795](https://github.com/ClickHouse/ClickHouse/pull/57795) ([Anton Popov](https://github.com/CurtizJ)).
+* Disable `system.kafka_consumers` because it has a bug. [#57822](https://github.com/ClickHouse/ClickHouse/pull/57822) ([Azat Khuzhin](https://github.com/azat)).
+* Fix LowCardinality keys support in Merge JOIN. [#57827](https://github.com/ClickHouse/ClickHouse/pull/57827) ([vdimir](https://github.com/vdimir)).
+* A fix for `InterpreterCreateQuery` related to the sample block. [#57855](https://github.com/ClickHouse/ClickHouse/pull/57855) ([Maksim Kita](https://github.com/kitaisreal)).
+* `addresses_expr` were ignored for named collections from PostgreSQL. [#57874](https://github.com/ClickHouse/ClickHouse/pull/57874) ([joelynch](https://github.com/joelynch)).
+* Fix invalid memory access in BLAKE3 (Rust) [#57876](https://github.com/ClickHouse/ClickHouse/pull/57876) ([Raúl Marín](https://github.com/Algunenano)). Then it was rewritten from Rust to C++ for better [memory-safety](https://www.memorysafety.org/). [#57994](https://github.com/ClickHouse/ClickHouse/pull/57994) ([Raúl Marín](https://github.com/Algunenano)).
+* Normalize function names in `CREATE INDEX` [#57906](https://github.com/ClickHouse/ClickHouse/pull/57906) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Fix handling of unavailable replicas before first request happened [#57933](https://github.com/ClickHouse/ClickHouse/pull/57933) ([Nikita Taranov](https://github.com/nickitat)).
+* Fix literal alias misclassification [#57988](https://github.com/ClickHouse/ClickHouse/pull/57988) ([Chen768959](https://github.com/Chen768959)).
+* Fix invalid preprocessing on Keeper [#58069](https://github.com/ClickHouse/ClickHouse/pull/58069) ([Antonio Andelic](https://github.com/antonio2368)).
+* Fix integer overflow in the `Poco` library, related to `UTF32Encoding` [#58073](https://github.com/ClickHouse/ClickHouse/pull/58073) ([Andrey Fedotov](https://github.com/anfedotoff)).
+* Fix parallel replicas (experimental feature) in presence of a scalar subquery with a big integer value [#58118](https://github.com/ClickHouse/ClickHouse/pull/58118) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix `accurateCastOrNull` for out-of-range `DateTime` [#58139](https://github.com/ClickHouse/ClickHouse/pull/58139) ([Andrey Zvonov](https://github.com/zvonand)).
+* Fix possible `PARAMETER_OUT_OF_BOUND` error during subcolumns reading from a wide part in MergeTree [#58175](https://github.com/ClickHouse/ClickHouse/pull/58175) ([Kruglov Pavel](https://github.com/Avogar)).
+* Fix a slow-down of CREATE VIEW with an enormous number of subqueries [#58220](https://github.com/ClickHouse/ClickHouse/pull/58220) ([Tao Wang](https://github.com/wangtZJU)).
+* Fix parallel parsing for JSONCompactEachRow [#58181](https://github.com/ClickHouse/ClickHouse/pull/58181) ([Alexey Milovidov](https://github.com/alexey-milovidov)). [#58250](https://github.com/ClickHouse/ClickHouse/pull/58250) ([Kruglov Pavel](https://github.com/Avogar)).
+
+
### ClickHouse release 23.11, 2023-12-06
#### Backward Incompatible Change
@@ -234,6 +375,7 @@
* Do not interpret the `send_timeout` set on the client side as the `receive_timeout` on the server side and vise-versa. [#56035](https://github.com/ClickHouse/ClickHouse/pull/56035) ([Azat Khuzhin](https://github.com/azat)).
* Comparison of time intervals with different units will throw an exception. This closes [#55942](https://github.com/ClickHouse/ClickHouse/issues/55942). You might have occasionally rely on the previous behavior when the underlying numeric values were compared regardless of the units. [#56090](https://github.com/ClickHouse/ClickHouse/pull/56090) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Rewrited the experimental `S3Queue` table engine completely: changed the way we keep information in zookeeper which allows to make less zookeeper requests, added caching of zookeeper state in cases when we know the state will not change, improved the polling from s3 process to make it less aggressive, changed the way ttl and max set for trached files is maintained, now it is a background process. Added `system.s3queue` and `system.s3queue_log` tables. Closes [#54998](https://github.com/ClickHouse/ClickHouse/issues/54998). [#54422](https://github.com/ClickHouse/ClickHouse/pull/54422) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Arbitrary paths on HTTP endpoint are no longer interpreted as a request to the `/query` endpoint. [#55521](https://github.com/ClickHouse/ClickHouse/pull/55521) ([Konstantin Bogdanov](https://github.com/thevar1able)).
#### New Feature
* Add function `arrayFold(accumulator, x1, ..., xn -> expression, initial, array1, ..., arrayn)` which applies a lambda function to multiple arrays of the same cardinality and collects the result in an accumulator. [#49794](https://github.com/ClickHouse/ClickHouse/pull/49794) ([Lirikl](https://github.com/Lirikl)).
diff --git a/LICENSE b/LICENSE
index 65c5df824c6..c653e59a8f3 100644
--- a/LICENSE
+++ b/LICENSE
@@ -1,4 +1,4 @@
-Copyright 2016-2023 ClickHouse, Inc.
+Copyright 2016-2024 ClickHouse, Inc.
Apache License
Version 2.0, January 2004
@@ -188,7 +188,7 @@ Copyright 2016-2023 ClickHouse, Inc.
same "printed page" as the copyright notice for easier
identification within third-party archives.
- Copyright 2016-2023 ClickHouse, Inc.
+ Copyright 2016-2024 ClickHouse, Inc.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
diff --git a/README.md b/README.md
index bf8ef0b4e98..c56b3c2fd0d 100644
--- a/README.md
+++ b/README.md
@@ -33,12 +33,7 @@ curl https://clickhouse.com/ | sh
## Upcoming Events
-* [**ClickHouse Meetup in Berlin**](https://www.meetup.com/clickhouse-berlin-user-group/events/296488501/) - Nov 30
-* [**ClickHouse Meetup in NYC**](https://www.meetup.com/clickhouse-new-york-user-group/events/296488779/) - Dec 11
-* [**ClickHouse Meetup in Sydney**](https://www.meetup.com/clickhouse-sydney-user-group/events/297638812/) - Dec 12
-* [**ClickHouse Meetup in Boston**](https://www.meetup.com/clickhouse-boston-user-group/events/296488840/) - Dec 12
-
-Also, keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler clickhouse com.
+Keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler clickhouse com.
## Recent Recordings
* **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments"
diff --git a/SECURITY.md b/SECURITY.md
index 7aaf9f3e5b9..a200e172a3b 100644
--- a/SECURITY.md
+++ b/SECURITY.md
@@ -13,9 +13,10 @@ The following versions of ClickHouse server are currently being supported with s
| Version | Supported |
|:-|:-|
+| 23.12 | ✔️ |
| 23.11 | ✔️ |
| 23.10 | ✔️ |
-| 23.9 | ✔️ |
+| 23.9 | ❌ |
| 23.8 | ✔️ |
| 23.7 | ❌ |
| 23.6 | ❌ |
diff --git a/base/poco/Util/src/XMLConfiguration.cpp b/base/poco/Util/src/XMLConfiguration.cpp
index e0d363cc870..648084aa28e 100644
--- a/base/poco/Util/src/XMLConfiguration.cpp
+++ b/base/poco/Util/src/XMLConfiguration.cpp
@@ -18,6 +18,7 @@
#ifndef POCO_UTIL_NO_XMLCONFIGURATION
+#include "Poco/String.h"
#include "Poco/SAX/InputSource.h"
#include "Poco/DOM/DOMParser.h"
#include "Poco/DOM/Element.h"
@@ -28,6 +29,8 @@
#include "Poco/NumberParser.h"
#include "Poco/NumberFormatter.h"
#include
+#include
+#include
namespace Poco {
@@ -275,8 +278,9 @@ void XMLConfiguration::enumerate(const std::string& key, Keys& range) const
{
if (pChild->nodeType() == Poco::XML::Node::ELEMENT_NODE)
{
- const std::string& nodeName = pChild->nodeName();
+ std::string nodeName = pChild->nodeName();
size_t& count = keys[nodeName];
+ replaceInPlace(nodeName, ".", "\\.");
if (count)
range.push_back(nodeName + "[" + NumberFormatter::format(count) + "]");
else
@@ -379,7 +383,21 @@ Poco::XML::Node* XMLConfiguration::findNode(std::string::const_iterator& it, con
{
while (it != end && *it == _delim) ++it;
std::string key;
- while (it != end && *it != _delim && *it != '[') key += *it++;
+ while (it != end)
+ {
+ if (*it == '\\' && std::distance(it, end) > 1)
+ {
+ // Skip backslash, copy only the char after it
+ std::advance(it, 1);
+ key += *it++;
+ continue;
+ }
+ if (*it == _delim)
+ break;
+ if (*it == '[')
+ break;
+ key += *it++;
+ }
return findNode(it, end, findElement(key, pNode, create), create);
}
}
diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt
index bc41819b717..e5a8c064808 100644
--- a/cmake/autogenerated_versions.txt
+++ b/cmake/autogenerated_versions.txt
@@ -2,11 +2,11 @@
# NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION,
# only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes.
-SET(VERSION_REVISION 54481)
-SET(VERSION_MAJOR 23)
-SET(VERSION_MINOR 12)
+SET(VERSION_REVISION 54482)
+SET(VERSION_MAJOR 24)
+SET(VERSION_MINOR 1)
SET(VERSION_PATCH 1)
-SET(VERSION_GITHASH 05bc8ef1e02b9c7332f08091775b255d191341bf)
-SET(VERSION_DESCRIBE v23.12.1.1-testing)
-SET(VERSION_STRING 23.12.1.1)
+SET(VERSION_GITHASH a2faa65b080a587026c86844f3a20c74d23a86f8)
+SET(VERSION_DESCRIBE v24.1.1.1-testing)
+SET(VERSION_STRING 24.1.1.1)
# end of autochange
diff --git a/cmake/target.cmake b/cmake/target.cmake
index 0d6993142b3..fb911ace7b5 100644
--- a/cmake/target.cmake
+++ b/cmake/target.cmake
@@ -12,6 +12,8 @@ elseif (CMAKE_SYSTEM_NAME MATCHES "FreeBSD")
elseif (CMAKE_SYSTEM_NAME MATCHES "Darwin")
set (OS_DARWIN 1)
add_definitions(-D OS_DARWIN)
+ # For MAP_ANON/MAP_ANONYMOUS
+ add_definitions(-D _DARWIN_C_SOURCE)
elseif (CMAKE_SYSTEM_NAME MATCHES "SunOS")
set (OS_SUNOS 1)
add_definitions(-D OS_SUNOS)
diff --git a/contrib/NuRaft b/contrib/NuRaft
index b7ea89b817a..2f5f52c4d8c 160000
--- a/contrib/NuRaft
+++ b/contrib/NuRaft
@@ -1 +1 @@
-Subproject commit b7ea89b817a18dc0eafc1f909d568869f02d2d04
+Subproject commit 2f5f52c4d8c87c2a3a3d101ca3a0194c9b77526f
diff --git a/contrib/mariadb-connector-c-cmake/CMakeLists.txt b/contrib/mariadb-connector-c-cmake/CMakeLists.txt
index 18d1510a57b..4257828890f 100644
--- a/contrib/mariadb-connector-c-cmake/CMakeLists.txt
+++ b/contrib/mariadb-connector-c-cmake/CMakeLists.txt
@@ -1,4 +1,4 @@
-if(OS_LINUX AND TARGET OpenSSL::SSL)
+if((OS_LINUX OR OS_DARWIN) AND TARGET OpenSSL::SSL)
option(ENABLE_MYSQL "Enable MySQL" ${ENABLE_LIBRARIES})
else ()
option(ENABLE_MYSQL "Enable MySQL" FALSE)
@@ -73,7 +73,7 @@ set(HAVE_SYS_TYPES_H 1)
set(HAVE_SYS_UN_H 1)
set(HAVE_UNISTD_H 1)
set(HAVE_UTIME_H 1)
-set(HAVE_UCONTEXT_H 1)
+set(HAVE_UCONTEXT_H 0)
set(HAVE_ALLOCA 1)
set(HAVE_DLERROR 0)
set(HAVE_DLOPEN 0)
@@ -116,9 +116,13 @@ CONFIGURE_FILE(${CC_SOURCE_DIR}/include/ma_config.h.in
CONFIGURE_FILE(${CC_SOURCE_DIR}/include/mariadb_version.h.in
${CC_BINARY_DIR}/include-public/mariadb_version.h)
-if(WITH_SSL)
+if (WITH_SSL)
set(SYSTEM_LIBS ${SYSTEM_LIBS} ${SSL_LIBRARIES})
-endif()
+endif ()
+
+if (OS_DARWIN)
+ set(SYSTEM_LIBS ${SYSTEM_LIBS} iconv)
+endif ()
function(REGISTER_PLUGIN)
@@ -227,15 +231,8 @@ ${CC_SOURCE_DIR}/libmariadb/secure/openssl_crypt.c
${CC_BINARY_DIR}/libmariadb/ma_client_plugin.c
)
-if(ICONV_INCLUDE_DIR)
- include_directories(BEFORE ${ICONV_INCLUDE_DIR})
-endif()
add_definitions(-DLIBICONV_PLUG)
-if(WITH_DYNCOL)
- set(LIBMARIADB_SOURCES ${LIBMARIADB_SOURCES} ${CC_SOURCE_DIR}/libmariadb/mariadb_dyncol.c)
-endif()
-
set(LIBMARIADB_SOURCES ${LIBMARIADB_SOURCES} ${CC_SOURCE_DIR}/libmariadb/mariadb_async.c ${CC_SOURCE_DIR}/libmariadb/ma_context.c)
diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile
index 59170af8edf..4b5e8cd3970 100644
--- a/docker/keeper/Dockerfile
+++ b/docker/keeper/Dockerfile
@@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
-ARG VERSION="23.11.3.23"
+ARG VERSION="23.12.2.59"
ARG PACKAGES="clickhouse-keeper"
ARG DIRECT_DOWNLOAD_URLS=""
diff --git a/docker/packager/README.md b/docker/packager/README.md
index 3a91f9a63f0..e0b7f38ea58 100644
--- a/docker/packager/README.md
+++ b/docker/packager/README.md
@@ -3,10 +3,10 @@ compilers and build settings. Correctly configured Docker daemon is single depen
Usage:
-Build deb package with `clang-14` in `debug` mode:
+Build deb package with `clang-17` in `debug` mode:
```
$ mkdir deb/test_output
-$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-14 --debug-build
+$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-17 --debug-build
$ ls -l deb/test_output
-rw-r--r-- 1 root root 3730 clickhouse-client_22.2.2+debug_all.deb
-rw-r--r-- 1 root root 84221888 clickhouse-common-static_22.2.2+debug_amd64.deb
@@ -17,11 +17,11 @@ $ ls -l deb/test_output
```
-Build ClickHouse binary with `clang-14` and `address` sanitizer in `relwithdebuginfo`
+Build ClickHouse binary with `clang-17` and `address` sanitizer in `relwithdebuginfo`
mode:
```
$ mkdir $HOME/some_clickhouse
-$ ./packager --output-dir=$HOME/some_clickhouse --package-type binary --compiler=clang-14 --sanitizer=address
+$ ./packager --output-dir=$HOME/some_clickhouse --package-type binary --compiler=clang-17 --sanitizer=address
$ ls -l $HOME/some_clickhouse
-rwxr-xr-x 1 root root 787061952 clickhouse
lrwxrwxrwx 1 root root 10 clickhouse-benchmark -> clickhouse
diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine
index b577775277e..452d8539a48 100644
--- a/docker/server/Dockerfile.alpine
+++ b/docker/server/Dockerfile.alpine
@@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
-ARG VERSION="23.11.3.23"
+ARG VERSION="23.12.2.59"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
ARG DIRECT_DOWNLOAD_URLS=""
diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu
index 6dc764bd0b9..0cefa3c14cb 100644
--- a/docker/server/Dockerfile.ubuntu
+++ b/docker/server/Dockerfile.ubuntu
@@ -30,7 +30,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
-ARG VERSION="23.11.3.23"
+ARG VERSION="23.12.2.59"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# set non-empty deb_location_url url to create a docker image
diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib
index 8f89c1b80dd..6f0dabb5207 100644
--- a/docker/test/stateless/stress_tests.lib
+++ b/docker/test/stateless/stress_tests.lib
@@ -236,6 +236,10 @@ function check_logs_for_critical_errors()
&& echo -e "S3_ERROR No such key thrown (see clickhouse-server.log or no_such_key_errors.txt)$FAIL$(trim_server_logs no_such_key_errors.txt)" >> /test_output/test_results.tsv \
|| echo -e "No lost s3 keys$OK" >> /test_output/test_results.tsv
+ rg -Fa "it is lost forever" /var/log/clickhouse-server/clickhouse-server*.log | grep 'SharedMergeTreePartCheckThread' > /dev/null \
+ && echo -e "Lost forever for SharedMergeTree$FAIL" >> /test_output/test_results.tsv \
+ || echo -e "No SharedMergeTree lost forever in clickhouse-server.log$OK" >> /test_output/test_results.tsv
+
# Remove file no_such_key_errors.txt if it's empty
[ -s /test_output/no_such_key_errors.txt ] || rm /test_output/no_such_key_errors.txt
diff --git a/docs/changelogs/v23.10.6.60-stable.md b/docs/changelogs/v23.10.6.60-stable.md
new file mode 100644
index 00000000000..5e1c126e729
--- /dev/null
+++ b/docs/changelogs/v23.10.6.60-stable.md
@@ -0,0 +1,51 @@
+---
+sidebar_position: 1
+sidebar_label: 2024
+---
+
+# 2024 Changelog
+
+### ClickHouse release v23.10.6.60-stable (68907bbe643) FIXME as compared to v23.10.5.20-stable (e84001e5c61)
+
+#### Improvement
+* Backported in [#58493](https://github.com/ClickHouse/ClickHouse/issues/58493): Fix transfer query to MySQL compatible query. Fixes [#57253](https://github.com/ClickHouse/ClickHouse/issues/57253). Fixes [#52654](https://github.com/ClickHouse/ClickHouse/issues/52654). Fixes [#56729](https://github.com/ClickHouse/ClickHouse/issues/56729). [#56456](https://github.com/ClickHouse/ClickHouse/pull/56456) ([flynn](https://github.com/ucasfl)).
+* Backported in [#57659](https://github.com/ClickHouse/ClickHouse/issues/57659): Handle sigabrt case when getting PostgreSQl table structure with empty array. [#57618](https://github.com/ClickHouse/ClickHouse/pull/57618) ([Mike Kot (Михаил Кот)](https://github.com/myrrc)).
+
+#### Build/Testing/Packaging Improvement
+* Backported in [#57586](https://github.com/ClickHouse/ClickHouse/issues/57586): Fix issue caught in https://github.com/docker-library/official-images/pull/15846. [#57571](https://github.com/ClickHouse/ClickHouse/pull/57571) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+
+#### Bug Fix (user-visible misbehavior in an official stable release)
+
+* Flatten only true Nested type if flatten_nested=1, not all Array(Tuple) [#56132](https://github.com/ClickHouse/ClickHouse/pull/56132) ([Kruglov Pavel](https://github.com/Avogar)).
+* Fix ALTER COLUMN with ALIAS [#56493](https://github.com/ClickHouse/ClickHouse/pull/56493) ([Nikolay Degterinsky](https://github.com/evillique)).
+* Prevent incompatible ALTER of projection columns [#56948](https://github.com/ClickHouse/ClickHouse/pull/56948) ([Amos Bird](https://github.com/amosbird)).
+* Fix segfault after ALTER UPDATE with Nullable MATERIALIZED column [#57147](https://github.com/ClickHouse/ClickHouse/pull/57147) ([Nikolay Degterinsky](https://github.com/evillique)).
+* Fix incorrect JOIN plan optimization with partially materialized normal projection [#57196](https://github.com/ClickHouse/ClickHouse/pull/57196) ([Amos Bird](https://github.com/amosbird)).
+* Fix `ReadonlyReplica` metric for all cases [#57267](https://github.com/ClickHouse/ClickHouse/pull/57267) ([Antonio Andelic](https://github.com/antonio2368)).
+* Background merges correctly use temporary data storage in the cache [#57275](https://github.com/ClickHouse/ClickHouse/pull/57275) ([vdimir](https://github.com/vdimir)).
+* MergeTree mutations reuse source part index granularity [#57352](https://github.com/ClickHouse/ClickHouse/pull/57352) ([Maksim Kita](https://github.com/kitaisreal)).
+* Fix function jsonMergePatch for partially const columns [#57379](https://github.com/ClickHouse/ClickHouse/pull/57379) ([Nikolay Degterinsky](https://github.com/evillique)).
+* Fix working with read buffers in StreamingFormatExecutor [#57438](https://github.com/ClickHouse/ClickHouse/pull/57438) ([Kruglov Pavel](https://github.com/Avogar)).
+* bugfix: correctly parse SYSTEM STOP LISTEN TCP SECURE [#57483](https://github.com/ClickHouse/ClickHouse/pull/57483) ([joelynch](https://github.com/joelynch)).
+* Ignore ON CLUSTER clause in grant/revoke queries for management of replicated access entities. [#57538](https://github.com/ClickHouse/ClickHouse/pull/57538) ([MikhailBurdukov](https://github.com/MikhailBurdukov)).
+* Disable system.kafka_consumers by default (due to possible live memory leak) [#57822](https://github.com/ClickHouse/ClickHouse/pull/57822) ([Azat Khuzhin](https://github.com/azat)).
+* Fix invalid memory access in BLAKE3 (Rust) [#57876](https://github.com/ClickHouse/ClickHouse/pull/57876) ([Raúl Marín](https://github.com/Algunenano)).
+* Normalize function names in CREATE INDEX [#57906](https://github.com/ClickHouse/ClickHouse/pull/57906) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Fix invalid preprocessing on Keeper [#58069](https://github.com/ClickHouse/ClickHouse/pull/58069) ([Antonio Andelic](https://github.com/antonio2368)).
+* Fix Integer overflow in Poco::UTF32Encoding [#58073](https://github.com/ClickHouse/ClickHouse/pull/58073) ([Andrey Fedotov](https://github.com/anfedotoff)).
+* Remove parallel parsing for JSONCompactEachRow [#58181](https://github.com/ClickHouse/ClickHouse/pull/58181) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix parallel parsing for JSONCompactEachRow [#58250](https://github.com/ClickHouse/ClickHouse/pull/58250) ([Kruglov Pavel](https://github.com/Avogar)).
+* Fix lost blobs after dropping a replica with broken detached parts [#58333](https://github.com/ClickHouse/ClickHouse/pull/58333) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* MergeTreePrefetchedReadPool disable for LIMIT only queries [#58505](https://github.com/ClickHouse/ClickHouse/pull/58505) ([Maksim Kita](https://github.com/kitaisreal)).
+
+#### NO CL CATEGORY
+
+* Backported in [#57916](https://github.com/ClickHouse/ClickHouse/issues/57916):. [#57909](https://github.com/ClickHouse/ClickHouse/pull/57909) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+
+#### NOT FOR CHANGELOG / INSIGNIFICANT
+
+* Pin alpine version of integration tests helper container [#57669](https://github.com/ClickHouse/ClickHouse/pull/57669) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* Remove heavy rust stable toolchain [#57905](https://github.com/ClickHouse/ClickHouse/pull/57905) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* Fix docker image for integration tests (fixes CI) [#57952](https://github.com/ClickHouse/ClickHouse/pull/57952) ([Azat Khuzhin](https://github.com/azat)).
+* Fix test_user_valid_until [#58409](https://github.com/ClickHouse/ClickHouse/pull/58409) ([Nikolay Degterinsky](https://github.com/evillique)).
+
diff --git a/docs/changelogs/v23.11.4.24-stable.md b/docs/changelogs/v23.11.4.24-stable.md
new file mode 100644
index 00000000000..40096285b06
--- /dev/null
+++ b/docs/changelogs/v23.11.4.24-stable.md
@@ -0,0 +1,26 @@
+---
+sidebar_position: 1
+sidebar_label: 2024
+---
+
+# 2024 Changelog
+
+### ClickHouse release v23.11.4.24-stable (e79d840d7fe) FIXME as compared to v23.11.3.23-stable (a14ab450b0e)
+
+#### Bug Fix (user-visible misbehavior in an official stable release)
+
+* Flatten only true Nested type if flatten_nested=1, not all Array(Tuple) [#56132](https://github.com/ClickHouse/ClickHouse/pull/56132) ([Kruglov Pavel](https://github.com/Avogar)).
+* Fix working with read buffers in StreamingFormatExecutor [#57438](https://github.com/ClickHouse/ClickHouse/pull/57438) ([Kruglov Pavel](https://github.com/Avogar)).
+* Disable system.kafka_consumers by default (due to possible live memory leak) [#57822](https://github.com/ClickHouse/ClickHouse/pull/57822) ([Azat Khuzhin](https://github.com/azat)).
+* Fix invalid preprocessing on Keeper [#58069](https://github.com/ClickHouse/ClickHouse/pull/58069) ([Antonio Andelic](https://github.com/antonio2368)).
+* Fix Integer overflow in Poco::UTF32Encoding [#58073](https://github.com/ClickHouse/ClickHouse/pull/58073) ([Andrey Fedotov](https://github.com/anfedotoff)).
+* Remove parallel parsing for JSONCompactEachRow [#58181](https://github.com/ClickHouse/ClickHouse/pull/58181) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix parallel parsing for JSONCompactEachRow [#58250](https://github.com/ClickHouse/ClickHouse/pull/58250) ([Kruglov Pavel](https://github.com/Avogar)).
+* Fix lost blobs after dropping a replica with broken detached parts [#58333](https://github.com/ClickHouse/ClickHouse/pull/58333) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* MergeTreePrefetchedReadPool disable for LIMIT only queries [#58505](https://github.com/ClickHouse/ClickHouse/pull/58505) ([Maksim Kita](https://github.com/kitaisreal)).
+
+#### NOT FOR CHANGELOG / INSIGNIFICANT
+
+* Handle another case for preprocessing in Keeper [#58308](https://github.com/ClickHouse/ClickHouse/pull/58308) ([Antonio Andelic](https://github.com/antonio2368)).
+* Fix test_user_valid_until [#58409](https://github.com/ClickHouse/ClickHouse/pull/58409) ([Nikolay Degterinsky](https://github.com/evillique)).
+
diff --git a/docs/changelogs/v23.12.1.1368-stable.md b/docs/changelogs/v23.12.1.1368-stable.md
new file mode 100644
index 00000000000..1a322ae9c0f
--- /dev/null
+++ b/docs/changelogs/v23.12.1.1368-stable.md
@@ -0,0 +1,327 @@
+---
+sidebar_position: 1
+sidebar_label: 2023
+---
+
+# 2023 Changelog
+
+### ClickHouse release v23.12.1.1368-stable (a2faa65b080) FIXME as compared to v23.11.1.2711-stable (05bc8ef1e02)
+
+#### Backward Incompatible Change
+* Fix check for non-deterministic functions in TTL expressions. Previously, you could create a TTL expression with non-deterministic functions in some cases, which could lead to undefined behavior later. This fixes [#37250](https://github.com/ClickHouse/ClickHouse/issues/37250). Disallow TTL expressions that don't depend on any columns of a table by default. It can be allowed back by `SET allow_suspicious_ttl_expressions = 1` or `SET compatibility = '23.11'`. Closes [#37286](https://github.com/ClickHouse/ClickHouse/issues/37286). [#51858](https://github.com/ClickHouse/ClickHouse/pull/51858) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Remove function `arrayFold` because it has a bug. This closes [#57816](https://github.com/ClickHouse/ClickHouse/issues/57816). This closes [#57458](https://github.com/ClickHouse/ClickHouse/issues/57458). [#57836](https://github.com/ClickHouse/ClickHouse/pull/57836) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Remove the feature of `is_deleted` row in ReplacingMergeTree and the `CLEANUP` modifier for the OPTIMIZE query. This fixes [#57930](https://github.com/ClickHouse/ClickHouse/issues/57930). This closes [#54988](https://github.com/ClickHouse/ClickHouse/issues/54988). This closes [#54570](https://github.com/ClickHouse/ClickHouse/issues/54570). This closes [#50346](https://github.com/ClickHouse/ClickHouse/issues/50346). This closes [#47579](https://github.com/ClickHouse/ClickHouse/issues/47579). The feature has to be removed because it is not good. We have to remove it as quickly as possible, because there is no other option. [#57932](https://github.com/ClickHouse/ClickHouse/pull/57932) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* The MergeTree setting `clean_deleted_rows` is deprecated, it has no effect anymore. The `CLEANUP` keyword for `OPTIMIZE` is not allowed by default (unless `allow_experimental_replacing_merge_with_cleanup` is enabled). [#58267](https://github.com/ClickHouse/ClickHouse/pull/58267) ([Alexander Tokmakov](https://github.com/tavplubix)).
+
+#### New Feature
+* Allow disabling of HEAD request before GET request. [#54602](https://github.com/ClickHouse/ClickHouse/pull/54602) ([Fionera](https://github.com/fionera)).
+* Add a HTTP endpoint for checking if Keeper is ready to accept traffic. [#55876](https://github.com/ClickHouse/ClickHouse/pull/55876) ([Konstantin Bogdanov](https://github.com/thevar1able)).
+* Add 'union' mode for schema inference. In this mode the resulting table schema is the union of all files schemas (so schema is inferred from each file). The mode of schema inference is controlled by a setting `schema_inference_mode` with 2 possible values - `default` and `union`. Closes [#55428](https://github.com/ClickHouse/ClickHouse/issues/55428). [#55892](https://github.com/ClickHouse/ClickHouse/pull/55892) ([Kruglov Pavel](https://github.com/Avogar)).
+* Add new setting `input_format_csv_try_infer_numbers_from_strings` that allows to infer numbers from strings in CSV format. Closes [#56455](https://github.com/ClickHouse/ClickHouse/issues/56455). [#56859](https://github.com/ClickHouse/ClickHouse/pull/56859) ([Kruglov Pavel](https://github.com/Avogar)).
+* Refreshable materialized views. [#56946](https://github.com/ClickHouse/ClickHouse/pull/56946) ([Michael Kolupaev](https://github.com/al13n321)).
+* Add more warnings on the number of databases, tables. [#57375](https://github.com/ClickHouse/ClickHouse/pull/57375) ([凌涛](https://github.com/lingtaolf)).
+* Added a new mutation command `ALTER TABLE APPLY DELETED MASK`, which allows to enforce applying of mask written by lightweight delete and to remove rows marked as deleted from disk. [#57433](https://github.com/ClickHouse/ClickHouse/pull/57433) ([Anton Popov](https://github.com/CurtizJ)).
+* Added a new SQL function `sqid` to generate Sqids (https://sqids.org/), example: `SELECT sqid(125, 126)`. [#57512](https://github.com/ClickHouse/ClickHouse/pull/57512) ([Robert Schulze](https://github.com/rschu1ze)).
+* Dictionary with `HASHED_ARRAY` (and `COMPLEX_KEY_HASHED_ARRAY`) layout supports `SHARDS` similarly to `HASHED`. [#57544](https://github.com/ClickHouse/ClickHouse/pull/57544) ([vdimir](https://github.com/vdimir)).
+* Add asynchronous metrics for total primary key bytes and total allocated primary key bytes in memory. [#57551](https://github.com/ClickHouse/ClickHouse/pull/57551) ([Bharat Nallan](https://github.com/bharatnc)).
+* Table system.dropped_tables_parts contains parts of system.dropped_tables tables (dropped but not yet removed tables). [#57555](https://github.com/ClickHouse/ClickHouse/pull/57555) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
+* Add `FORMAT_BYTES` as an alias for `formatReadableSize`. [#57592](https://github.com/ClickHouse/ClickHouse/pull/57592) ([Bharat Nallan](https://github.com/bharatnc)).
+* Add SHA512_256 function. [#57645](https://github.com/ClickHouse/ClickHouse/pull/57645) ([Bharat Nallan](https://github.com/bharatnc)).
+* Allow passing optional SESSION_TOKEN to `s3` table function. [#57850](https://github.com/ClickHouse/ClickHouse/pull/57850) ([Shani Elharrar](https://github.com/shanielh)).
+* Clause `ORDER BY` now supports specifying `ALL`, meaning that ClickHouse sorts by all columns in the `SELECT` clause. Example: `SELECT col1, col2 FROM tab WHERE [...] ORDER BY ALL`. [#57875](https://github.com/ClickHouse/ClickHouse/pull/57875) ([zhongyuankai](https://github.com/zhongyuankai)).
+* Added functions for punycode encoding/decoding: `punycodeEncode()` and `punycodeDecode()`. [#57969](https://github.com/ClickHouse/ClickHouse/pull/57969) ([Robert Schulze](https://github.com/rschu1ze)).
+* This PR reproduces the implementation of `PASTE JOIN`, which allows users to join tables without `ON` clause. Example: ``` SQL SELECT * FROM ( SELECT number AS a FROM numbers(2) ) AS t1 PASTE JOIN ( SELECT number AS a FROM numbers(2) ORDER BY a DESC ) AS t2. [#57995](https://github.com/ClickHouse/ClickHouse/pull/57995) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
+* A handler `/binary` opens a visual viewer of symbols inside the ClickHouse binary. [#58211](https://github.com/ClickHouse/ClickHouse/pull/58211) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+
+#### Performance Improvement
+* Made copy between s3 disks using a s3-server-side copy instead of copying through the buffer. Improves `BACKUP/RESTORE` operations and `clickhouse-disks copy` command. [#56744](https://github.com/ClickHouse/ClickHouse/pull/56744) ([MikhailBurdukov](https://github.com/MikhailBurdukov)).
+* HashJoin respects setting `max_joined_block_size_rows` and do not produce large blocks for `ALL JOIN`. [#56996](https://github.com/ClickHouse/ClickHouse/pull/56996) ([vdimir](https://github.com/vdimir)).
+* Release memory for aggregation earlier. This may avoid unnecessary external aggregation. [#57691](https://github.com/ClickHouse/ClickHouse/pull/57691) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
+* Improve performance of string serialization. [#57717](https://github.com/ClickHouse/ClickHouse/pull/57717) ([Maksim Kita](https://github.com/kitaisreal)).
+* Support trivial count optimization for `Merge`-engine tables. [#57867](https://github.com/ClickHouse/ClickHouse/pull/57867) ([skyoct](https://github.com/skyoct)).
+* Optimized aggregation in some cases. [#57872](https://github.com/ClickHouse/ClickHouse/pull/57872) ([Anton Popov](https://github.com/CurtizJ)).
+* The `hasAny()` function can now take advantage of the full-text skipping indices. [#57878](https://github.com/ClickHouse/ClickHouse/pull/57878) ([Jpnock](https://github.com/Jpnock)).
+* Function `if(cond, then, else)` (and its alias `cond ? : then : else`) were optimized to use branch-free evaluation. [#57885](https://github.com/ClickHouse/ClickHouse/pull/57885) ([zhanglistar](https://github.com/zhanglistar)).
+* Extract non intersecting parts ranges from MergeTree table during FINAL processing. That way we can avoid additional FINAL logic for this non intersecting parts ranges. In case when amount of duplicate values with same primary key is low, performance will be almost the same as without FINAL. Improve reading performance for MergeTree FINAL when `do_not_merge_across_partitions_select_final` setting is set. [#58120](https://github.com/ClickHouse/ClickHouse/pull/58120) ([Maksim Kita](https://github.com/kitaisreal)).
+* MergeTree automatically derive `do_not_merge_across_partitions_select_final` setting if partition key expression contains only columns from primary key expression. [#58218](https://github.com/ClickHouse/ClickHouse/pull/58218) ([Maksim Kita](https://github.com/kitaisreal)).
+* Speedup MIN and MAX for native types. [#58231](https://github.com/ClickHouse/ClickHouse/pull/58231) ([Raúl Marín](https://github.com/Algunenano)).
+
+#### Improvement
+* Make inserts into distributed tables handle updated cluster configuration properly. When the list of cluster nodes is dynamically updated, the Directory Monitor of the distribution table cannot sense the new node, and the Directory Monitor must be re-noded to sense it. [#42826](https://github.com/ClickHouse/ClickHouse/pull/42826) ([zhongyuankai](https://github.com/zhongyuankai)).
+* Replace --no-system-tables with loading virtual tables of system database lazily. [#55271](https://github.com/ClickHouse/ClickHouse/pull/55271) ([Azat Khuzhin](https://github.com/azat)).
+* Clickhouse-test print case sn, current time and case name in one test case. [#55710](https://github.com/ClickHouse/ClickHouse/pull/55710) ([guoxiaolong](https://github.com/guoxiaolongzte)).
+* Do not allow creating replicated table with inconsistent merge params. [#56833](https://github.com/ClickHouse/ClickHouse/pull/56833) ([Duc Canh Le](https://github.com/canhld94)).
+* Implement SLRU cache policy for filesystem cache. [#57076](https://github.com/ClickHouse/ClickHouse/pull/57076) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Show uncompressed size in `system.tables`, obtained from data parts' checksums [#56618](https://github.com/ClickHouse/ClickHouse/issues/56618). [#57186](https://github.com/ClickHouse/ClickHouse/pull/57186) ([Chen Lixiang](https://github.com/chenlx0)).
+* Add `skip_unavailable_shards` as a setting for `Distributed` tables that is similar to the corresponding query-level setting. Closes [#43666](https://github.com/ClickHouse/ClickHouse/issues/43666). [#57218](https://github.com/ClickHouse/ClickHouse/pull/57218) ([Gagan Goel](https://github.com/tntnatbry)).
+* Function `substring()` (aliases: `substr`, `mid`) can now be used with `Enum` types. Previously, the first function argument had to be a value of type `String` or `FixedString`. This improves compatibility with 3rd party tools such as Tableau via MySQL interface. [#57277](https://github.com/ClickHouse/ClickHouse/pull/57277) ([Serge Klochkov](https://github.com/slvrtrn)).
+* Better hints when a table doesn't exist. [#57342](https://github.com/ClickHouse/ClickHouse/pull/57342) ([Bharat Nallan](https://github.com/bharatnc)).
+* Allow to overwrite `max_partition_size_to_drop` and `max_table_size_to_drop` server settings in query time. [#57452](https://github.com/ClickHouse/ClickHouse/pull/57452) ([Jordi Villar](https://github.com/jrdi)).
+* Add support for read-only flag when connecting to the ZooKeeper server (fixes [#53749](https://github.com/ClickHouse/ClickHouse/issues/53749)). [#57479](https://github.com/ClickHouse/ClickHouse/pull/57479) ([Mikhail Koviazin](https://github.com/mkmkme)).
+* Fix possible distributed sends stuck due to "No such file or directory" (during recovering batch from disk). Fix possible issues with `error_count` from `system.distribution_queue` (in case of `distributed_directory_monitor_max_sleep_time_ms` >5min). Introduce profile event to track async INSERT failures - `DistributedAsyncInsertionFailures`. [#57480](https://github.com/ClickHouse/ClickHouse/pull/57480) ([Azat Khuzhin](https://github.com/azat)).
+* The limit for the number of connections per endpoint for background fetches was raised from `15` to the value of `background_fetches_pool_size` setting. - MergeTree-level setting `replicated_max_parallel_fetches_for_host` became obsolete - MergeTree-level settings `replicated_fetches_http_connection_timeout`, `replicated_fetches_http_send_timeout` and `replicated_fetches_http_receive_timeout` are moved to the Server-level. - Setting `keep_alive_timeout` is added to the list of Server-level settings. [#57523](https://github.com/ClickHouse/ClickHouse/pull/57523) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
+* It is now possible to refer to ALIAS column in index (non-primary-key) definitions (issue [#55650](https://github.com/ClickHouse/ClickHouse/issues/55650)). Example: `CREATE TABLE tab(col UInt32, col_alias ALIAS col + 1, INDEX idx (col_alias) TYPE minmax) ENGINE = MergeTree ORDER BY col;`. [#57546](https://github.com/ClickHouse/ClickHouse/pull/57546) ([Robert Schulze](https://github.com/rschu1ze)).
+* Function `format()` now supports arbitrary argument types (instead of only `String` and `FixedString` arguments). This is important to calculate `SELECT format('The {0} to all questions is {1}', 'answer', 42)`. [#57549](https://github.com/ClickHouse/ClickHouse/pull/57549) ([Robert Schulze](https://github.com/rschu1ze)).
+* Support PostgreSQL generated columns and default column values in `MaterializedPostgreSQL`. Closes [#40449](https://github.com/ClickHouse/ClickHouse/issues/40449). [#57568](https://github.com/ClickHouse/ClickHouse/pull/57568) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Allow to apply some filesystem cache config settings changes without server restart. [#57578](https://github.com/ClickHouse/ClickHouse/pull/57578) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Handle sigabrt case when getting PostgreSQl table structure with empty array. [#57618](https://github.com/ClickHouse/ClickHouse/pull/57618) ([Mike Kot (Михаил Кот)](https://github.com/myrrc)).
+* Allows to use the `date_trunc()` function with the first argument not depending on the case of it. Both cases are now supported: `SELECT date_trunc('day', now())` and `SELECT date_trunc('DAY', now())`. [#57624](https://github.com/ClickHouse/ClickHouse/pull/57624) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
+* Expose the total number of errors occurred since last server as a `ClickHouseErrorMetric_ALL` metric. [#57627](https://github.com/ClickHouse/ClickHouse/pull/57627) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
+* Allow nodes in config with from_env/from_zk and non empty element with replace=1. [#57628](https://github.com/ClickHouse/ClickHouse/pull/57628) ([Azat Khuzhin](https://github.com/azat)).
+* Generate malformed output that cannot be parsed as JSON. [#57646](https://github.com/ClickHouse/ClickHouse/pull/57646) ([Julia Kartseva](https://github.com/jkartseva)).
+* Consider lightweight deleted rows when selecting parts to merge if enabled. [#57648](https://github.com/ClickHouse/ClickHouse/pull/57648) ([Zhuo Qiu](https://github.com/jewelzqiu)).
+* Make querying system.filesystem_cache not memory intensive. [#57687](https://github.com/ClickHouse/ClickHouse/pull/57687) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Allow IPv6 to UInt128 conversion and binary arithmetic. [#57707](https://github.com/ClickHouse/ClickHouse/pull/57707) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
+* Support negative positional arguments. Closes [#57736](https://github.com/ClickHouse/ClickHouse/issues/57736). [#57741](https://github.com/ClickHouse/ClickHouse/pull/57741) ([flynn](https://github.com/ucasfl)).
+* Add a setting for `async inserts deduplication cache` -- how long we wait for cache update. Deprecate setting `async_block_ids_cache_min_update_interval_ms`. Now cache is updated only in case of conflicts. [#57743](https://github.com/ClickHouse/ClickHouse/pull/57743) ([alesapin](https://github.com/alesapin)).
+* `sleep()` function now can be cancelled with `KILL QUERY`. [#57746](https://github.com/ClickHouse/ClickHouse/pull/57746) ([Vitaly Baranov](https://github.com/vitlibar)).
+* Slightly better inference of unnamed tupes in JSON formats. [#57751](https://github.com/ClickHouse/ClickHouse/pull/57751) ([Kruglov Pavel](https://github.com/Avogar)).
+* Refactor UserDefinedSQL* classes to make it possible to add SQL UDF storages which are different from ZooKeeper and Disk. [#57752](https://github.com/ClickHouse/ClickHouse/pull/57752) ([Natasha Chizhonkova](https://github.com/chizhonkova)).
+* Forbid `CREATE TABLE ... AS SELECT` queries for Replicated table engines in Replicated database because they are broken. Reference [#35408](https://github.com/ClickHouse/ClickHouse/issues/35408). [#57796](https://github.com/ClickHouse/ClickHouse/pull/57796) ([Nikolay Degterinsky](https://github.com/evillique)).
+* Fix and improve transform query for external database, we should recursively obtain all compatible predicates. [#57888](https://github.com/ClickHouse/ClickHouse/pull/57888) ([flynn](https://github.com/ucasfl)).
+* Support dynamic reloading of filesystem cache size. Closes [#57866](https://github.com/ClickHouse/ClickHouse/issues/57866). [#57897](https://github.com/ClickHouse/ClickHouse/pull/57897) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Fix system.stack_trace for threads with blocked SIGRTMIN. [#57907](https://github.com/ClickHouse/ClickHouse/pull/57907) ([Azat Khuzhin](https://github.com/azat)).
+* Added a new setting `readonly` which can be used to specify a s3 disk is read only. It can be useful to create a table with read only `s3_plain` type disk. [#57977](https://github.com/ClickHouse/ClickHouse/pull/57977) ([Pengyuan Bian](https://github.com/bianpengyuan)).
+* Support keeper failures in quorum check. [#57986](https://github.com/ClickHouse/ClickHouse/pull/57986) ([Raúl Marín](https://github.com/Algunenano)).
+* Add max/peak RSS (`MemoryResidentMax`) into system.asynchronous_metrics. [#58095](https://github.com/ClickHouse/ClickHouse/pull/58095) ([Azat Khuzhin](https://github.com/azat)).
+* Fix system.stack_trace for threads with blocked SIGRTMIN (and also send signal to the threads only if it is not blocked to avoid waiting `storage_system_stack_trace_pipe_read_timeout_ms` when it does not make any sense). [#58136](https://github.com/ClickHouse/ClickHouse/pull/58136) ([Azat Khuzhin](https://github.com/azat)).
+* This PR allows users to use s3 links (`https://` and `s3://`) without mentioning region if it's not default. Also find the correct region if the user mentioned the wrong one. ### Documentation entry for user-facing changes. [#58148](https://github.com/ClickHouse/ClickHouse/pull/58148) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
+* `clickhouse-format --obfuscate` will know about Settings, MergeTreeSettings, and time zones and keep their names unchanged. [#58179](https://github.com/ClickHouse/ClickHouse/pull/58179) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Added explicit `finalize()` function in `ZipArchiveWriter`. Simplify too complicated code in `ZipArchiveWriter`. This PR fixes [#58074](https://github.com/ClickHouse/ClickHouse/issues/58074). [#58202](https://github.com/ClickHouse/ClickHouse/pull/58202) ([Vitaly Baranov](https://github.com/vitlibar)).
+* The primary key analysis in MergeTree tables will now be applied to predicates that include the virtual column `_part_offset` (optionally with `_part`). This feature can serve as a poor man's secondary index. [#58224](https://github.com/ClickHouse/ClickHouse/pull/58224) ([Amos Bird](https://github.com/amosbird)).
+* Make caches with the same path use the same cache objects. This behaviour existed before, but was broken in https://github.com/ClickHouse/ClickHouse/pull/48805 (in 23.4). If such caches with the same path have different set of cache settings, an exception will be thrown, that this is not allowed. [#58264](https://github.com/ClickHouse/ClickHouse/pull/58264) ([Kseniia Sumarokova](https://github.com/kssenii)).
+
+#### Build/Testing/Packaging Improvement
+* Allow usage of Azure-related table engines/functions on macOS. [#51866](https://github.com/ClickHouse/ClickHouse/pull/51866) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* ClickHouse Fast Test now uses Musl instead of GLibc. [#57711](https://github.com/ClickHouse/ClickHouse/pull/57711) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Run ClickBench for every commit. This closes [#57708](https://github.com/ClickHouse/ClickHouse/issues/57708). [#57712](https://github.com/ClickHouse/ClickHouse/pull/57712) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+
+#### Bug Fix (user-visible misbehavior in an official stable release)
+
+* Fixed a sorting order breakage in TTL GROUP BY [#49103](https://github.com/ClickHouse/ClickHouse/pull/49103) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
+* fix: split lttb bucket strategy, first bucket and last bucket should only contain single point [#57003](https://github.com/ClickHouse/ClickHouse/pull/57003) ([FFish](https://github.com/wxybear)).
+* Fix possible deadlock in Template format during sync after error [#57004](https://github.com/ClickHouse/ClickHouse/pull/57004) ([Kruglov Pavel](https://github.com/Avogar)).
+* Fix early stop while parsing file with skipping lots of errors [#57006](https://github.com/ClickHouse/ClickHouse/pull/57006) ([Kruglov Pavel](https://github.com/Avogar)).
+* Prevent dictionary's ACL bypass via dictionary() table function [#57362](https://github.com/ClickHouse/ClickHouse/pull/57362) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
+* Fix another case of non-ready set. [#57423](https://github.com/ClickHouse/ClickHouse/pull/57423) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
+* Fix several issues regarding PostgreSQL `array_ndims` usage. [#57436](https://github.com/ClickHouse/ClickHouse/pull/57436) ([Ryan Jacobs](https://github.com/ryanmjacobs)).
+* Fix RWLock inconsistency after write lock timeout [#57454](https://github.com/ClickHouse/ClickHouse/pull/57454) ([Vitaly Baranov](https://github.com/vitlibar)).
+* Fix: don't exclude ephemeral column when building pushing to view chain [#57461](https://github.com/ClickHouse/ClickHouse/pull/57461) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
+* MaterializedPostgreSQL: fix issue [#41922](https://github.com/ClickHouse/ClickHouse/issues/41922), add test for [#41923](https://github.com/ClickHouse/ClickHouse/issues/41923) [#57515](https://github.com/ClickHouse/ClickHouse/pull/57515) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Ignore ON CLUSTER clause in grant/revoke queries for management of replicated access entities. [#57538](https://github.com/ClickHouse/ClickHouse/pull/57538) ([MikhailBurdukov](https://github.com/MikhailBurdukov)).
+* Fix crash in clickhouse-local [#57553](https://github.com/ClickHouse/ClickHouse/pull/57553) ([Nikolay Degterinsky](https://github.com/evillique)).
+* Materialize block in HashJoin for Type::EMPTY [#57564](https://github.com/ClickHouse/ClickHouse/pull/57564) ([vdimir](https://github.com/vdimir)).
+* Fix possible segfault in PostgreSQLSource [#57567](https://github.com/ClickHouse/ClickHouse/pull/57567) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Fix type correction in HashJoin for nested low cardinality [#57614](https://github.com/ClickHouse/ClickHouse/pull/57614) ([vdimir](https://github.com/vdimir)).
+* Avoid hangs of system.stack_trace by correctly prohibit parallel read from it [#57641](https://github.com/ClickHouse/ClickHouse/pull/57641) ([Azat Khuzhin](https://github.com/azat)).
+* Fix SIGSEGV for aggregation of sparse columns with any() RESPECT NULL [#57710](https://github.com/ClickHouse/ClickHouse/pull/57710) ([Azat Khuzhin](https://github.com/azat)).
+* Fix unary operators parsing [#57713](https://github.com/ClickHouse/ClickHouse/pull/57713) ([Nikolay Degterinsky](https://github.com/evillique)).
+* Fix RWLock inconsistency after write lock timeout (again) [#57733](https://github.com/ClickHouse/ClickHouse/pull/57733) ([Vitaly Baranov](https://github.com/vitlibar)).
+* Table engine MaterializedPostgreSQL fix dependency loading [#57754](https://github.com/ClickHouse/ClickHouse/pull/57754) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Fix retries for disconnected nodes for BACKUP/RESTORE ON CLUSTER [#57764](https://github.com/ClickHouse/ClickHouse/pull/57764) ([Vitaly Baranov](https://github.com/vitlibar)).
+* Fix bug window functions: revert [#39631](https://github.com/ClickHouse/ClickHouse/issues/39631) [#57766](https://github.com/ClickHouse/ClickHouse/pull/57766) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix result of external aggregation in case of partially materialized projection [#57790](https://github.com/ClickHouse/ClickHouse/pull/57790) ([Anton Popov](https://github.com/CurtizJ)).
+* Fix merge in aggregation functions with `*Map` combinator [#57795](https://github.com/ClickHouse/ClickHouse/pull/57795) ([Anton Popov](https://github.com/CurtizJ)).
+* Disable system.kafka_consumers by default (due to possible live memory leak) [#57822](https://github.com/ClickHouse/ClickHouse/pull/57822) ([Azat Khuzhin](https://github.com/azat)).
+* Fix low-cardinality keys support in MergeJoin [#57827](https://github.com/ClickHouse/ClickHouse/pull/57827) ([vdimir](https://github.com/vdimir)).
+* Create consumers for Kafka tables on fly (but keep them for some period since last used) [#57829](https://github.com/ClickHouse/ClickHouse/pull/57829) ([Azat Khuzhin](https://github.com/azat)).
+* InterpreterCreateQuery sample block fix [#57855](https://github.com/ClickHouse/ClickHouse/pull/57855) ([Maksim Kita](https://github.com/kitaisreal)).
+* bugfix: addresses_expr ignored for psql named collections [#57874](https://github.com/ClickHouse/ClickHouse/pull/57874) ([joelynch](https://github.com/joelynch)).
+* Fix invalid memory access in BLAKE3 (Rust) [#57876](https://github.com/ClickHouse/ClickHouse/pull/57876) ([Raúl Marín](https://github.com/Algunenano)).
+* Resurrect `arrayFold()` [#57879](https://github.com/ClickHouse/ClickHouse/pull/57879) ([Robert Schulze](https://github.com/rschu1ze)).
+* Normalize function names in CREATE INDEX [#57906](https://github.com/ClickHouse/ClickHouse/pull/57906) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Fix handling of unavailable replicas before first request happened [#57933](https://github.com/ClickHouse/ClickHouse/pull/57933) ([Nikita Taranov](https://github.com/nickitat)).
+* Fix literal alias misclassification [#57988](https://github.com/ClickHouse/ClickHouse/pull/57988) ([Chen768959](https://github.com/Chen768959)).
+* Revert "Fix bug window functions: revert [#39631](https://github.com/ClickHouse/ClickHouse/issues/39631)" [#58031](https://github.com/ClickHouse/ClickHouse/pull/58031) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
+* Fix invalid preprocessing on Keeper [#58069](https://github.com/ClickHouse/ClickHouse/pull/58069) ([Antonio Andelic](https://github.com/antonio2368)).
+* Fix Integer overflow in Poco::UTF32Encoding [#58073](https://github.com/ClickHouse/ClickHouse/pull/58073) ([Andrey Fedotov](https://github.com/anfedotoff)).
+* Fix parallel replicas in presence of a scalar subquery with a big integer value [#58118](https://github.com/ClickHouse/ClickHouse/pull/58118) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix `accurateCastOrNull` for out-of-range DateTime [#58139](https://github.com/ClickHouse/ClickHouse/pull/58139) ([Andrey Zvonov](https://github.com/zvonand)).
+* Fix possible PARAMETER_OUT_OF_BOUND error during subcolumns reading from wide part in MergeTree [#58175](https://github.com/ClickHouse/ClickHouse/pull/58175) ([Kruglov Pavel](https://github.com/Avogar)).
+* Remove parallel parsing for JSONCompactEachRow [#58181](https://github.com/ClickHouse/ClickHouse/pull/58181) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* fix CREATE VIEW hang [#58220](https://github.com/ClickHouse/ClickHouse/pull/58220) ([Tao Wang](https://github.com/wangtZJU)).
+* Fix parallel parsing for JSONCompactEachRow [#58250](https://github.com/ClickHouse/ClickHouse/pull/58250) ([Kruglov Pavel](https://github.com/Avogar)).
+
+#### NO CL ENTRY
+
+* NO CL ENTRY: 'Revert "Revert "Update Sentry""'. [#57694](https://github.com/ClickHouse/ClickHouse/pull/57694) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* NO CL ENTRY: 'Revert "Fix RWLock inconsistency after write lock timeout"'. [#57730](https://github.com/ClickHouse/ClickHouse/pull/57730) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* NO CL ENTRY: 'Revert "improve CI with digest for docker, build and test jobs"'. [#57903](https://github.com/ClickHouse/ClickHouse/pull/57903) ([Max K.](https://github.com/mkaynov)).
+* NO CL ENTRY: 'Reapply "improve CI with digest for docker, build and test jobs"'. [#57904](https://github.com/ClickHouse/ClickHouse/pull/57904) ([Max K.](https://github.com/mkaynov)).
+* NO CL ENTRY: 'Revert "Merge pull request [#56573](https://github.com/ClickHouse/ClickHouse/issues/56573) from mkmkme/mkmkme/reload-config"'. [#57909](https://github.com/ClickHouse/ClickHouse/pull/57909) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* NO CL ENTRY: 'Revert "Add system.dropped_tables_parts table"'. [#58022](https://github.com/ClickHouse/ClickHouse/pull/58022) ([Antonio Andelic](https://github.com/antonio2368)).
+* NO CL ENTRY: 'Revert "Consider lightweight deleted rows when selecting parts to merge"'. [#58097](https://github.com/ClickHouse/ClickHouse/pull/58097) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* NO CL ENTRY: 'Revert "Fix leftover processes/hangs in tests"'. [#58207](https://github.com/ClickHouse/ClickHouse/pull/58207) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* NO CL ENTRY: 'Revert "Create consumers for Kafka tables on fly (but keep them for some period since last used)"'. [#58272](https://github.com/ClickHouse/ClickHouse/pull/58272) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* NO CL ENTRY: 'Revert "Implement punycode encoding/decoding"'. [#58277](https://github.com/ClickHouse/ClickHouse/pull/58277) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+
+#### NOT FOR CHANGELOG / INSIGNIFICANT
+
+* Randomize more settings [#39663](https://github.com/ClickHouse/ClickHouse/pull/39663) ([Anton Popov](https://github.com/CurtizJ)).
+* Add more tests for `compile_expressions` [#51113](https://github.com/ClickHouse/ClickHouse/pull/51113) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* [RFC] Correctly wait background threads [#52717](https://github.com/ClickHouse/ClickHouse/pull/52717) ([Azat Khuzhin](https://github.com/azat)).
+* improve CI with digest for docker, build and test jobs [#56317](https://github.com/ClickHouse/ClickHouse/pull/56317) ([Max K.](https://github.com/mkaynov)).
+* Prepare the introduction of more keeper faults [#56917](https://github.com/ClickHouse/ClickHouse/pull/56917) ([Raúl Marín](https://github.com/Algunenano)).
+* Analyzer: Fix assert in tryReplaceAndEqualsChainsWithConstant [#57139](https://github.com/ClickHouse/ClickHouse/pull/57139) ([vdimir](https://github.com/vdimir)).
+* Check what will happen if we build ClickHouse with Musl [#57180](https://github.com/ClickHouse/ClickHouse/pull/57180) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* support memory soft limit for keeper [#57271](https://github.com/ClickHouse/ClickHouse/pull/57271) ([Han Fei](https://github.com/hanfei1991)).
+* Randomize disabled optimizations in CI [#57315](https://github.com/ClickHouse/ClickHouse/pull/57315) ([Raúl Marín](https://github.com/Algunenano)).
+* Don't throw if noop when dropping database replica in batch [#57337](https://github.com/ClickHouse/ClickHouse/pull/57337) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Better JSON -> JSONEachRow fallback without catching exceptions [#57364](https://github.com/ClickHouse/ClickHouse/pull/57364) ([Kruglov Pavel](https://github.com/Avogar)).
+* Add tests for [#48496](https://github.com/ClickHouse/ClickHouse/issues/48496) [#57414](https://github.com/ClickHouse/ClickHouse/pull/57414) ([Raúl Marín](https://github.com/Algunenano)).
+* Add profile event for cache lookup in `ThreadPoolRemoteFSReader` [#57437](https://github.com/ClickHouse/ClickHouse/pull/57437) ([Nikita Taranov](https://github.com/nickitat)).
+* Remove select() usage [#57467](https://github.com/ClickHouse/ClickHouse/pull/57467) ([Igor Nikonov](https://github.com/devcrafter)).
+* Parallel replicas: friendly settings [#57542](https://github.com/ClickHouse/ClickHouse/pull/57542) ([Igor Nikonov](https://github.com/devcrafter)).
+* Fix formatting string prompt error [#57569](https://github.com/ClickHouse/ClickHouse/pull/57569) ([skyoct](https://github.com/skyoct)).
+* Tune CI scale up/down multipliers [#57572](https://github.com/ClickHouse/ClickHouse/pull/57572) ([Max K.](https://github.com/mkaynov)).
+* Revert "Revert "Implemented series period detect method using pocketfft lib"" [#57574](https://github.com/ClickHouse/ClickHouse/pull/57574) ([Bhavna Jindal](https://github.com/bhavnajindal)).
+* Correctly handle errors during opening query in editor in client [#57587](https://github.com/ClickHouse/ClickHouse/pull/57587) ([Azat Khuzhin](https://github.com/azat)).
+* Add a test for [#55251](https://github.com/ClickHouse/ClickHouse/issues/55251) [#57588](https://github.com/ClickHouse/ClickHouse/pull/57588) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Add a test for [#48039](https://github.com/ClickHouse/ClickHouse/issues/48039) [#57593](https://github.com/ClickHouse/ClickHouse/pull/57593) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Update CHANGELOG.md [#57594](https://github.com/ClickHouse/ClickHouse/pull/57594) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Update version after release [#57595](https://github.com/ClickHouse/ClickHouse/pull/57595) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Update version_date.tsv and changelogs after v23.11.1.2711-stable [#57597](https://github.com/ClickHouse/ClickHouse/pull/57597) ([robot-clickhouse](https://github.com/robot-clickhouse)).
+* Identify failed jobs in lambda and mark as steps=0 [#57600](https://github.com/ClickHouse/ClickHouse/pull/57600) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* Fix flaky test: distinct in order with analyzer [#57606](https://github.com/ClickHouse/ClickHouse/pull/57606) ([Igor Nikonov](https://github.com/devcrafter)).
+* CHJIT add assembly printer [#57610](https://github.com/ClickHouse/ClickHouse/pull/57610) ([Maksim Kita](https://github.com/kitaisreal)).
+* Fix parsing virtual hosted S3 URI in clickhouse_backupview script [#57612](https://github.com/ClickHouse/ClickHouse/pull/57612) ([Daniel Pozo Escalona](https://github.com/danipozo)).
+* Fix docs for `fileCluster` [#57613](https://github.com/ClickHouse/ClickHouse/pull/57613) ([Andrey Zvonov](https://github.com/zvonand)).
+* Analyzer: Fix logical error in MultiIfToIfPass [#57622](https://github.com/ClickHouse/ClickHouse/pull/57622) ([vdimir](https://github.com/vdimir)).
+* Throw more clear exception [#57626](https://github.com/ClickHouse/ClickHouse/pull/57626) ([alesapin](https://github.com/alesapin)).
+* Fix "logs and exception messages formatting", part 1 [#57630](https://github.com/ClickHouse/ClickHouse/pull/57630) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix "logs and exception messages formatting", part 2 [#57632](https://github.com/ClickHouse/ClickHouse/pull/57632) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix "logs and exception messages formatting", part 3 [#57633](https://github.com/ClickHouse/ClickHouse/pull/57633) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix "logs and exception messages formatting", part 4 [#57634](https://github.com/ClickHouse/ClickHouse/pull/57634) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Remove bad test (1) [#57636](https://github.com/ClickHouse/ClickHouse/pull/57636) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Remove bad test (2) [#57637](https://github.com/ClickHouse/ClickHouse/pull/57637) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* ClickHouse Cloud promotion [#57638](https://github.com/ClickHouse/ClickHouse/pull/57638) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
+* Remove bad test (3) [#57639](https://github.com/ClickHouse/ClickHouse/pull/57639) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Remove bad test (4) [#57640](https://github.com/ClickHouse/ClickHouse/pull/57640) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Random changes in random files [#57642](https://github.com/ClickHouse/ClickHouse/pull/57642) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
+* Merge half of [#51113](https://github.com/ClickHouse/ClickHouse/issues/51113) [#57643](https://github.com/ClickHouse/ClickHouse/pull/57643) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Analyzer: Fix JOIN ON true with join_use_nulls [#57662](https://github.com/ClickHouse/ClickHouse/pull/57662) ([vdimir](https://github.com/vdimir)).
+* Pin alpine version of integration tests helper container [#57669](https://github.com/ClickHouse/ClickHouse/pull/57669) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* Add support for system.stack_trace filtering optimizations for analyzer [#57682](https://github.com/ClickHouse/ClickHouse/pull/57682) ([Azat Khuzhin](https://github.com/azat)).
+* test for [#33308](https://github.com/ClickHouse/ClickHouse/issues/33308) [#57693](https://github.com/ClickHouse/ClickHouse/pull/57693) ([Denny Crane](https://github.com/den-crane)).
+* support keeper memory soft limit ratio [#57699](https://github.com/ClickHouse/ClickHouse/pull/57699) ([Han Fei](https://github.com/hanfei1991)).
+* Fix test_dictionaries_update_and_reload/test.py::test_reload_while_loading flakiness [#57714](https://github.com/ClickHouse/ClickHouse/pull/57714) ([Azat Khuzhin](https://github.com/azat)).
+* Tune autoscale to scale for single job in the queue [#57742](https://github.com/ClickHouse/ClickHouse/pull/57742) ([Max K.](https://github.com/mkaynov)).
+* Tune network memory for dockerhub proxy hosts [#57744](https://github.com/ClickHouse/ClickHouse/pull/57744) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* Parallel replicas: announcement response handling improvement [#57749](https://github.com/ClickHouse/ClickHouse/pull/57749) ([Igor Nikonov](https://github.com/devcrafter)).
+* Fix building Rust with Musl [#57756](https://github.com/ClickHouse/ClickHouse/pull/57756) ([Azat Khuzhin](https://github.com/azat)).
+* Fix flaky test_parallel_replicas_distributed_read_from_all [#57757](https://github.com/ClickHouse/ClickHouse/pull/57757) ([Igor Nikonov](https://github.com/devcrafter)).
+* Minor refactoring of toStartOfInterval() [#57761](https://github.com/ClickHouse/ClickHouse/pull/57761) ([Robert Schulze](https://github.com/rschu1ze)).
+* Don't run test 02919_skip_lots_of_parsing_errors on aarch64 [#57762](https://github.com/ClickHouse/ClickHouse/pull/57762) ([Kruglov Pavel](https://github.com/Avogar)).
+* More respect to `min_number_of_marks` in `ParallelReplicasReadingCoordinator` [#57763](https://github.com/ClickHouse/ClickHouse/pull/57763) ([Nikita Taranov](https://github.com/nickitat)).
+* SerializationString reduce memory usage [#57787](https://github.com/ClickHouse/ClickHouse/pull/57787) ([Maksim Kita](https://github.com/kitaisreal)).
+* Fix ThreadSanitizer data race in librdkafka [#57791](https://github.com/ClickHouse/ClickHouse/pull/57791) ([Ilya Golshtein](https://github.com/ilejn)).
+* Rename `system.async_loader` into `system.asynchronous_loader` [#57793](https://github.com/ClickHouse/ClickHouse/pull/57793) ([Sergei Trifonov](https://github.com/serxa)).
+* Set replica number to its position in cluster definition [#57800](https://github.com/ClickHouse/ClickHouse/pull/57800) ([Nikita Taranov](https://github.com/nickitat)).
+* fix clickhouse-client invocation in 02327_capnproto_protobuf_empty_messages [#57804](https://github.com/ClickHouse/ClickHouse/pull/57804) ([Mikhail Koviazin](https://github.com/mkmkme)).
+* Fix flaky test_parallel_replicas_over_distributed [#57809](https://github.com/ClickHouse/ClickHouse/pull/57809) ([Igor Nikonov](https://github.com/devcrafter)).
+* Revert [#57741](https://github.com/ClickHouse/ClickHouse/issues/57741) [#57811](https://github.com/ClickHouse/ClickHouse/pull/57811) ([Raúl Marín](https://github.com/Algunenano)).
+* Dumb down `substring()` tests [#57821](https://github.com/ClickHouse/ClickHouse/pull/57821) ([Robert Schulze](https://github.com/rschu1ze)).
+* Update version_date.tsv and changelogs after v23.11.2.11-stable [#57824](https://github.com/ClickHouse/ClickHouse/pull/57824) ([robot-clickhouse](https://github.com/robot-clickhouse)).
+* Fix 02906_force_optimize_projection_name [#57826](https://github.com/ClickHouse/ClickHouse/pull/57826) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
+* ClickBench: slightly better [#57831](https://github.com/ClickHouse/ClickHouse/pull/57831) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix 02932_kill_query_sleep flakiness [#57849](https://github.com/ClickHouse/ClickHouse/pull/57849) ([Azat Khuzhin](https://github.com/azat)).
+* Revert "Replace --no-system-tables with loading virtual tables of system database lazily" [#57851](https://github.com/ClickHouse/ClickHouse/pull/57851) ([Azat Khuzhin](https://github.com/azat)).
+* Fix memory leak in StorageHDFS [#57860](https://github.com/ClickHouse/ClickHouse/pull/57860) ([Andrey Zvonov](https://github.com/zvonand)).
+* Remove hardcoded clickhouse-client invocations from tests [#57861](https://github.com/ClickHouse/ClickHouse/pull/57861) ([Mikhail Koviazin](https://github.com/mkmkme)).
+* Follow up to [#57568](https://github.com/ClickHouse/ClickHouse/issues/57568) [#57863](https://github.com/ClickHouse/ClickHouse/pull/57863) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Fix assertion in HashJoin [#57873](https://github.com/ClickHouse/ClickHouse/pull/57873) ([vdimir](https://github.com/vdimir)).
+* More efficient constructor for SerializationEnum [#57887](https://github.com/ClickHouse/ClickHouse/pull/57887) ([Duc Canh Le](https://github.com/canhld94)).
+* Fix test_unset_skip_unavailable_shards [#57895](https://github.com/ClickHouse/ClickHouse/pull/57895) ([Raúl Marín](https://github.com/Algunenano)).
+* Add argument to fill the gap in cherry-pick [#57896](https://github.com/ClickHouse/ClickHouse/pull/57896) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* Delete debug logging in OutputFormatWithUTF8ValidationAdaptor [#57899](https://github.com/ClickHouse/ClickHouse/pull/57899) ([Kruglov Pavel](https://github.com/Avogar)).
+* Remove heavy rust stable toolchain [#57905](https://github.com/ClickHouse/ClickHouse/pull/57905) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* Improvements for 00002_log_and_exception_messages_formatting [#57910](https://github.com/ClickHouse/ClickHouse/pull/57910) ([Raúl Marín](https://github.com/Algunenano)).
+* Update CHANGELOG.md [#57911](https://github.com/ClickHouse/ClickHouse/pull/57911) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* remove cruft from TablesLoader [#57938](https://github.com/ClickHouse/ClickHouse/pull/57938) ([Bharat Nallan](https://github.com/bharatnc)).
+* Fix `/dashboard` work with passwords [#57948](https://github.com/ClickHouse/ClickHouse/pull/57948) ([Sergei Trifonov](https://github.com/serxa)).
+* Remove wrong test [#57950](https://github.com/ClickHouse/ClickHouse/pull/57950) ([Sergei Trifonov](https://github.com/serxa)).
+* Fix docker image for integration tests (fixes CI) [#57952](https://github.com/ClickHouse/ClickHouse/pull/57952) ([Azat Khuzhin](https://github.com/azat)).
+* Remove C++ templates (normalizeQuery) [#57963](https://github.com/ClickHouse/ClickHouse/pull/57963) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* A small fix for dashboard [#57964](https://github.com/ClickHouse/ClickHouse/pull/57964) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Always use `pread` for reading cache segments [#57970](https://github.com/ClickHouse/ClickHouse/pull/57970) ([Nikita Taranov](https://github.com/nickitat)).
+* Improve some tests [#57973](https://github.com/ClickHouse/ClickHouse/pull/57973) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Revert "Merge pull request [#57907](https://github.com/ClickHouse/ClickHouse/issues/57907) from azat/system.stack_trace-rt_tgsigqueueinfo" [#57974](https://github.com/ClickHouse/ClickHouse/pull/57974) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Add a test for [#49708](https://github.com/ClickHouse/ClickHouse/issues/49708) [#57979](https://github.com/ClickHouse/ClickHouse/pull/57979) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix style-check checkout head-ref [#57989](https://github.com/ClickHouse/ClickHouse/pull/57989) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* refine error message [#57991](https://github.com/ClickHouse/ClickHouse/pull/57991) ([Han Fei](https://github.com/hanfei1991)).
+* CI for docs only fix [#57992](https://github.com/ClickHouse/ClickHouse/pull/57992) ([Max K.](https://github.com/mkaynov)).
+* Replace rust's BLAKE3 with llvm's implementation [#57994](https://github.com/ClickHouse/ClickHouse/pull/57994) ([Raúl Marín](https://github.com/Algunenano)).
+* Better trivial count optimization for storage `Merge` [#57996](https://github.com/ClickHouse/ClickHouse/pull/57996) ([Anton Popov](https://github.com/CurtizJ)).
+* enhanced docs for `date_trunc()` [#58000](https://github.com/ClickHouse/ClickHouse/pull/58000) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
+* CI: add needs_changed_files flag for pr_info [#58003](https://github.com/ClickHouse/ClickHouse/pull/58003) ([Max K.](https://github.com/mkaynov)).
+* more messages in ci [#58007](https://github.com/ClickHouse/ClickHouse/pull/58007) ([Sema Checherinda](https://github.com/CheSema)).
+* Test parallel replicas with force_primary_key setting [#58010](https://github.com/ClickHouse/ClickHouse/pull/58010) ([Igor Nikonov](https://github.com/devcrafter)).
+* Update 00002_log_and_exception_messages_formatting.sql [#58012](https://github.com/ClickHouse/ClickHouse/pull/58012) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Fix rare race in external sort/aggregation with temporary data in cache [#58013](https://github.com/ClickHouse/ClickHouse/pull/58013) ([Anton Popov](https://github.com/CurtizJ)).
+* Fix segfault in FuzzJSON engine [#58015](https://github.com/ClickHouse/ClickHouse/pull/58015) ([Julia Kartseva](https://github.com/jkartseva)).
+* fix freebsd build [#58019](https://github.com/ClickHouse/ClickHouse/pull/58019) ([Julia Kartseva](https://github.com/jkartseva)).
+* Rename canUseParallelReplicas to canUseTaskBasedParallelReplicas [#58025](https://github.com/ClickHouse/ClickHouse/pull/58025) ([Raúl Marín](https://github.com/Algunenano)).
+* Remove fixed tests from analyzer_tech_debt.txt [#58028](https://github.com/ClickHouse/ClickHouse/pull/58028) ([Raúl Marín](https://github.com/Algunenano)).
+* More verbose errors on 00002_log_and_exception_messages_formatting [#58037](https://github.com/ClickHouse/ClickHouse/pull/58037) ([Raúl Marín](https://github.com/Algunenano)).
+* Make window insert result into constant [#58045](https://github.com/ClickHouse/ClickHouse/pull/58045) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
+* CI: Happy new year [#58046](https://github.com/ClickHouse/ClickHouse/pull/58046) ([Raúl Marín](https://github.com/Algunenano)).
+* Follow up for [#57691](https://github.com/ClickHouse/ClickHouse/issues/57691) [#58048](https://github.com/ClickHouse/ClickHouse/pull/58048) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
+* always run ast_fuzz and sqllancer [#58049](https://github.com/ClickHouse/ClickHouse/pull/58049) ([Max K.](https://github.com/mkaynov)).
+* Add GH status for PR formating [#58050](https://github.com/ClickHouse/ClickHouse/pull/58050) ([Max K.](https://github.com/mkaynov)).
+* Small improvement for SystemLogBase [#58051](https://github.com/ClickHouse/ClickHouse/pull/58051) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Bump Azure to v1.6.0 [#58052](https://github.com/ClickHouse/ClickHouse/pull/58052) ([Robert Schulze](https://github.com/rschu1ze)).
+* Correct values for randomization [#58058](https://github.com/ClickHouse/ClickHouse/pull/58058) ([Anton Popov](https://github.com/CurtizJ)).
+* Non post request should be readonly [#58060](https://github.com/ClickHouse/ClickHouse/pull/58060) ([San](https://github.com/santrancisco)).
+* Revert "Merge pull request [#55710](https://github.com/ClickHouse/ClickHouse/issues/55710) from guoxiaolongzte/clickhouse-test… [#58066](https://github.com/ClickHouse/ClickHouse/pull/58066) ([Raúl Marín](https://github.com/Algunenano)).
+* fix typo in the test 02479 [#58072](https://github.com/ClickHouse/ClickHouse/pull/58072) ([Sema Checherinda](https://github.com/CheSema)).
+* Bump Azure to 1.7.2 [#58075](https://github.com/ClickHouse/ClickHouse/pull/58075) ([Robert Schulze](https://github.com/rschu1ze)).
+* Fix flaky test `02567_and_consistency` [#58076](https://github.com/ClickHouse/ClickHouse/pull/58076) ([Anton Popov](https://github.com/CurtizJ)).
+* Fix Tests Bugfix Validate Check [#58078](https://github.com/ClickHouse/ClickHouse/pull/58078) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Fix for nightly job for digest-ci [#58079](https://github.com/ClickHouse/ClickHouse/pull/58079) ([Max K.](https://github.com/mkaynov)).
+* Test for parallel replicas with remote() [#58081](https://github.com/ClickHouse/ClickHouse/pull/58081) ([Igor Nikonov](https://github.com/devcrafter)).
+* Minor cosmetic changes [#58092](https://github.com/ClickHouse/ClickHouse/pull/58092) ([Raúl Marín](https://github.com/Algunenano)).
+* Reintroduce OPTIMIZE CLEANUP as no-op [#58100](https://github.com/ClickHouse/ClickHouse/pull/58100) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Add compatibility in the replication protocol for a removed feature [#58104](https://github.com/ClickHouse/ClickHouse/pull/58104) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Flaky 02922_analyzer_aggregate_nothing_type [#58105](https://github.com/ClickHouse/ClickHouse/pull/58105) ([Raúl Marín](https://github.com/Algunenano)).
+* Update version_date.tsv and changelogs after v23.11.3.23-stable [#58106](https://github.com/ClickHouse/ClickHouse/pull/58106) ([robot-clickhouse](https://github.com/robot-clickhouse)).
+* Limited CI on the master for docs only change [#58121](https://github.com/ClickHouse/ClickHouse/pull/58121) ([Max K.](https://github.com/mkaynov)).
+* style fix [#58125](https://github.com/ClickHouse/ClickHouse/pull/58125) ([Max K.](https://github.com/mkaynov)).
+* Support "do not test" label with ci.py [#58128](https://github.com/ClickHouse/ClickHouse/pull/58128) ([Max K.](https://github.com/mkaynov)).
+* Use the single images list for integration tests everywhere [#58130](https://github.com/ClickHouse/ClickHouse/pull/58130) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* Disable parallel replicas with IN (subquery) [#58133](https://github.com/ClickHouse/ClickHouse/pull/58133) ([Igor Nikonov](https://github.com/devcrafter)).
+* Fix clang-tidy [#58134](https://github.com/ClickHouse/ClickHouse/pull/58134) ([Raúl Marín](https://github.com/Algunenano)).
+* Run build report check job on build failures, fix [#58135](https://github.com/ClickHouse/ClickHouse/pull/58135) ([Max K.](https://github.com/mkaynov)).
+* Fix dashboard legend sorting and rows number [#58151](https://github.com/ClickHouse/ClickHouse/pull/58151) ([Sergei Trifonov](https://github.com/serxa)).
+* Remove retryStrategy assignments overwritten in ClientFactory::create() [#58163](https://github.com/ClickHouse/ClickHouse/pull/58163) ([Daniel Pozo Escalona](https://github.com/danipozo)).
+* Helper improvements [#58164](https://github.com/ClickHouse/ClickHouse/pull/58164) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* Pass through exceptions for reading from S3 [#58165](https://github.com/ClickHouse/ClickHouse/pull/58165) ([Azat Khuzhin](https://github.com/azat)).
+* [RFC] Adjust all std::ios implementations in poco to set failbit/badbit by default [#58166](https://github.com/ClickHouse/ClickHouse/pull/58166) ([Azat Khuzhin](https://github.com/azat)).
+* Add bytes_uncompressed to system.part_log [#58167](https://github.com/ClickHouse/ClickHouse/pull/58167) ([Jordi Villar](https://github.com/jrdi)).
+* Update docker/test/stateful/run.sh [#58168](https://github.com/ClickHouse/ClickHouse/pull/58168) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Update 00165_jit_aggregate_functions.sql [#58169](https://github.com/ClickHouse/ClickHouse/pull/58169) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Update clickhouse-test [#58170](https://github.com/ClickHouse/ClickHouse/pull/58170) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Profile event 'ParallelReplicasUsedCount' [#58173](https://github.com/ClickHouse/ClickHouse/pull/58173) ([Igor Nikonov](https://github.com/devcrafter)).
+* Fix flaky test `02719_aggregate_with_empty_string_key` [#58176](https://github.com/ClickHouse/ClickHouse/pull/58176) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix [#58171](https://github.com/ClickHouse/ClickHouse/issues/58171) [#58177](https://github.com/ClickHouse/ClickHouse/pull/58177) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Add base backup name to system.backups and system.backup_log tables [#58178](https://github.com/ClickHouse/ClickHouse/pull/58178) ([Pradeep Chhetri](https://github.com/chhetripradeep)).
+* Fix use-after-move [#58182](https://github.com/ClickHouse/ClickHouse/pull/58182) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Looking at strange code [#58196](https://github.com/ClickHouse/ClickHouse/pull/58196) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix all Exception with missing arguments [#58198](https://github.com/ClickHouse/ClickHouse/pull/58198) ([Azat Khuzhin](https://github.com/azat)).
+* Fix leftover processes/hangs in tests [#58200](https://github.com/ClickHouse/ClickHouse/pull/58200) ([Azat Khuzhin](https://github.com/azat)).
+* Fix DWARFBlockInputFormat failing on DWARF 5 unit address ranges [#58204](https://github.com/ClickHouse/ClickHouse/pull/58204) ([Michael Kolupaev](https://github.com/al13n321)).
+* Fix error in archive reader [#58206](https://github.com/ClickHouse/ClickHouse/pull/58206) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix DWARFBlockInputFormat using wrong base address sometimes [#58208](https://github.com/ClickHouse/ClickHouse/pull/58208) ([Michael Kolupaev](https://github.com/al13n321)).
+* Add support for specifying query parameters in the command line in clickhouse-local [#58210](https://github.com/ClickHouse/ClickHouse/pull/58210) ([Pradeep Chhetri](https://github.com/chhetripradeep)).
+* Fix leftover processes/hangs in tests (resubmit) [#58213](https://github.com/ClickHouse/ClickHouse/pull/58213) ([Azat Khuzhin](https://github.com/azat)).
+* Add optimization for AND notEquals chain in logical expression optimizer [#58214](https://github.com/ClickHouse/ClickHouse/pull/58214) ([Kevin Mingtarja](https://github.com/kevinmingtarja)).
+* Fix syntax and doc [#58221](https://github.com/ClickHouse/ClickHouse/pull/58221) ([San](https://github.com/santrancisco)).
+* Cleanup some known short messages [#58226](https://github.com/ClickHouse/ClickHouse/pull/58226) ([Raúl Marín](https://github.com/Algunenano)).
+* Some code refactoring (was an attempt to improve build time, but failed) [#58237](https://github.com/ClickHouse/ClickHouse/pull/58237) ([Azat Khuzhin](https://github.com/azat)).
+* Fix perf test README [#58245](https://github.com/ClickHouse/ClickHouse/pull/58245) ([Raúl Marín](https://github.com/Algunenano)).
+* [Analyzer] Add test for [#57086](https://github.com/ClickHouse/ClickHouse/issues/57086) [#58249](https://github.com/ClickHouse/ClickHouse/pull/58249) ([Raúl Marín](https://github.com/Algunenano)).
+* Reintroduce compatibility with `is_deleted` on a syntax level [#58251](https://github.com/ClickHouse/ClickHouse/pull/58251) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Avoid throwing ABORTED on normal situations [#58252](https://github.com/ClickHouse/ClickHouse/pull/58252) ([Raúl Marín](https://github.com/Algunenano)).
+* Remove mayBenefitFromIndexForIn [#58265](https://github.com/ClickHouse/ClickHouse/pull/58265) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
+* Allow a few retries when committing a part during shutdown [#58269](https://github.com/ClickHouse/ClickHouse/pull/58269) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Revert [#58267](https://github.com/ClickHouse/ClickHouse/issues/58267) [#58274](https://github.com/ClickHouse/ClickHouse/pull/58274) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+
diff --git a/docs/changelogs/v23.12.2.59-stable.md b/docs/changelogs/v23.12.2.59-stable.md
new file mode 100644
index 00000000000..6533f4e6b86
--- /dev/null
+++ b/docs/changelogs/v23.12.2.59-stable.md
@@ -0,0 +1,32 @@
+---
+sidebar_position: 1
+sidebar_label: 2024
+---
+
+# 2024 Changelog
+
+### ClickHouse release v23.12.2.59-stable (17ab210e761) FIXME as compared to v23.12.1.1368-stable (a2faa65b080)
+
+#### Backward Incompatible Change
+* Backported in [#58389](https://github.com/ClickHouse/ClickHouse/issues/58389): The MergeTree setting `clean_deleted_rows` is deprecated, it has no effect anymore. The `CLEANUP` keyword for `OPTIMIZE` is not allowed by default (unless `allow_experimental_replacing_merge_with_cleanup` is enabled). [#58316](https://github.com/ClickHouse/ClickHouse/pull/58316) ([Alexander Tokmakov](https://github.com/tavplubix)).
+
+#### Bug Fix (user-visible misbehavior in an official stable release)
+
+* Flatten only true Nested type if flatten_nested=1, not all Array(Tuple) [#56132](https://github.com/ClickHouse/ClickHouse/pull/56132) ([Kruglov Pavel](https://github.com/Avogar)).
+* Fix working with read buffers in StreamingFormatExecutor [#57438](https://github.com/ClickHouse/ClickHouse/pull/57438) ([Kruglov Pavel](https://github.com/Avogar)).
+* Fix lost blobs after dropping a replica with broken detached parts [#58333](https://github.com/ClickHouse/ClickHouse/pull/58333) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Fix segfault when graphite table does not have agg function [#58453](https://github.com/ClickHouse/ClickHouse/pull/58453) ([Duc Canh Le](https://github.com/canhld94)).
+* MergeTreePrefetchedReadPool disable for LIMIT only queries [#58505](https://github.com/ClickHouse/ClickHouse/pull/58505) ([Maksim Kita](https://github.com/kitaisreal)).
+
+#### NO CL ENTRY
+
+* NO CL ENTRY: 'Revert "Refreshable materialized views (takeover)"'. [#58296](https://github.com/ClickHouse/ClickHouse/pull/58296) ([Alexander Tokmakov](https://github.com/tavplubix)).
+
+#### NOT FOR CHANGELOG / INSIGNIFICANT
+
+* Fix an error in the release script - it didn't allow to make 23.12. [#58288](https://github.com/ClickHouse/ClickHouse/pull/58288) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Update version_date.tsv and changelogs after v23.12.1.1368-stable [#58290](https://github.com/ClickHouse/ClickHouse/pull/58290) ([robot-clickhouse](https://github.com/robot-clickhouse)).
+* Fix test_storage_s3_queue/test.py::test_drop_table [#58293](https://github.com/ClickHouse/ClickHouse/pull/58293) ([Kseniia Sumarokova](https://github.com/kssenii)).
+* Handle another case for preprocessing in Keeper [#58308](https://github.com/ClickHouse/ClickHouse/pull/58308) ([Antonio Andelic](https://github.com/antonio2368)).
+* Fix test_user_valid_until [#58409](https://github.com/ClickHouse/ClickHouse/pull/58409) ([Nikolay Degterinsky](https://github.com/evillique)).
+
diff --git a/docs/changelogs/v23.3.19.32-lts.md b/docs/changelogs/v23.3.19.32-lts.md
new file mode 100644
index 00000000000..4604c986fe6
--- /dev/null
+++ b/docs/changelogs/v23.3.19.32-lts.md
@@ -0,0 +1,36 @@
+---
+sidebar_position: 1
+sidebar_label: 2024
+---
+
+# 2024 Changelog
+
+### ClickHouse release v23.3.19.32-lts (c4d4ca8ec02) FIXME as compared to v23.3.18.15-lts (7228475d77a)
+
+#### Backward Incompatible Change
+* Backported in [#57840](https://github.com/ClickHouse/ClickHouse/issues/57840): Remove function `arrayFold` because it has a bug. This closes [#57816](https://github.com/ClickHouse/ClickHouse/issues/57816). This closes [#57458](https://github.com/ClickHouse/ClickHouse/issues/57458). [#57836](https://github.com/ClickHouse/ClickHouse/pull/57836) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+
+#### Improvement
+* Backported in [#58489](https://github.com/ClickHouse/ClickHouse/issues/58489): Fix transfer query to MySQL compatible query. Fixes [#57253](https://github.com/ClickHouse/ClickHouse/issues/57253). Fixes [#52654](https://github.com/ClickHouse/ClickHouse/issues/52654). Fixes [#56729](https://github.com/ClickHouse/ClickHouse/issues/56729). [#56456](https://github.com/ClickHouse/ClickHouse/pull/56456) ([flynn](https://github.com/ucasfl)).
+* Backported in [#57653](https://github.com/ClickHouse/ClickHouse/issues/57653): Handle sigabrt case when getting PostgreSQl table structure with empty array. [#57618](https://github.com/ClickHouse/ClickHouse/pull/57618) ([Mike Kot (Михаил Кот)](https://github.com/myrrc)).
+
+#### Build/Testing/Packaging Improvement
+* Backported in [#57580](https://github.com/ClickHouse/ClickHouse/issues/57580): Fix issue caught in https://github.com/docker-library/official-images/pull/15846. [#57571](https://github.com/ClickHouse/ClickHouse/pull/57571) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+
+#### Bug Fix (user-visible misbehavior in an official stable release)
+
+* Prevent incompatible ALTER of projection columns [#56948](https://github.com/ClickHouse/ClickHouse/pull/56948) ([Amos Bird](https://github.com/amosbird)).
+* Fix segfault after ALTER UPDATE with Nullable MATERIALIZED column [#57147](https://github.com/ClickHouse/ClickHouse/pull/57147) ([Nikolay Degterinsky](https://github.com/evillique)).
+* Fix incorrect JOIN plan optimization with partially materialized normal projection [#57196](https://github.com/ClickHouse/ClickHouse/pull/57196) ([Amos Bird](https://github.com/amosbird)).
+* MergeTree mutations reuse source part index granularity [#57352](https://github.com/ClickHouse/ClickHouse/pull/57352) ([Maksim Kita](https://github.com/kitaisreal)).
+* Fix invalid memory access in BLAKE3 (Rust) [#57876](https://github.com/ClickHouse/ClickHouse/pull/57876) ([Raúl Marín](https://github.com/Algunenano)).
+* Normalize function names in CREATE INDEX [#57906](https://github.com/ClickHouse/ClickHouse/pull/57906) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Fix invalid preprocessing on Keeper [#58069](https://github.com/ClickHouse/ClickHouse/pull/58069) ([Antonio Andelic](https://github.com/antonio2368)).
+* Fix Integer overflow in Poco::UTF32Encoding [#58073](https://github.com/ClickHouse/ClickHouse/pull/58073) ([Andrey Fedotov](https://github.com/anfedotoff)).
+* Remove parallel parsing for JSONCompactEachRow [#58181](https://github.com/ClickHouse/ClickHouse/pull/58181) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+
+#### NOT FOR CHANGELOG / INSIGNIFICANT
+
+* Pin alpine version of integration tests helper container [#57669](https://github.com/ClickHouse/ClickHouse/pull/57669) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* Fix docker image for integration tests (fixes CI) [#57952](https://github.com/ClickHouse/ClickHouse/pull/57952) ([Azat Khuzhin](https://github.com/azat)).
+
diff --git a/docs/changelogs/v23.8.9.54-lts.md b/docs/changelogs/v23.8.9.54-lts.md
new file mode 100644
index 00000000000..00607c60c39
--- /dev/null
+++ b/docs/changelogs/v23.8.9.54-lts.md
@@ -0,0 +1,47 @@
+---
+sidebar_position: 1
+sidebar_label: 2024
+---
+
+# 2024 Changelog
+
+### ClickHouse release v23.8.9.54-lts (192a1d231fa) FIXME as compared to v23.8.8.20-lts (5e012a03bf2)
+
+#### Improvement
+* Backported in [#57668](https://github.com/ClickHouse/ClickHouse/issues/57668): Output valid JSON/XML on excetpion during HTTP query execution. Add setting `http_write_exception_in_output_format` to enable/disable this behaviour (enabled by default). [#52853](https://github.com/ClickHouse/ClickHouse/pull/52853) ([Kruglov Pavel](https://github.com/Avogar)).
+* Backported in [#58491](https://github.com/ClickHouse/ClickHouse/issues/58491): Fix transfer query to MySQL compatible query. Fixes [#57253](https://github.com/ClickHouse/ClickHouse/issues/57253). Fixes [#52654](https://github.com/ClickHouse/ClickHouse/issues/52654). Fixes [#56729](https://github.com/ClickHouse/ClickHouse/issues/56729). [#56456](https://github.com/ClickHouse/ClickHouse/pull/56456) ([flynn](https://github.com/ucasfl)).
+* Backported in [#57238](https://github.com/ClickHouse/ClickHouse/issues/57238): Fetching a part waits when that part is fully committed on remote replica. It is better not send part in PreActive state. In case of zero copy this is mandatory restriction. [#56808](https://github.com/ClickHouse/ClickHouse/pull/56808) ([Sema Checherinda](https://github.com/CheSema)).
+* Backported in [#57655](https://github.com/ClickHouse/ClickHouse/issues/57655): Handle sigabrt case when getting PostgreSQl table structure with empty array. [#57618](https://github.com/ClickHouse/ClickHouse/pull/57618) ([Mike Kot (Михаил Кот)](https://github.com/myrrc)).
+
+#### Build/Testing/Packaging Improvement
+* Backported in [#57582](https://github.com/ClickHouse/ClickHouse/issues/57582): Fix issue caught in https://github.com/docker-library/official-images/pull/15846. [#57571](https://github.com/ClickHouse/ClickHouse/pull/57571) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+
+#### Bug Fix (user-visible misbehavior in an official stable release)
+
+* Flatten only true Nested type if flatten_nested=1, not all Array(Tuple) [#56132](https://github.com/ClickHouse/ClickHouse/pull/56132) ([Kruglov Pavel](https://github.com/Avogar)).
+* Fix ALTER COLUMN with ALIAS [#56493](https://github.com/ClickHouse/ClickHouse/pull/56493) ([Nikolay Degterinsky](https://github.com/evillique)).
+* Prevent incompatible ALTER of projection columns [#56948](https://github.com/ClickHouse/ClickHouse/pull/56948) ([Amos Bird](https://github.com/amosbird)).
+* Fix segfault after ALTER UPDATE with Nullable MATERIALIZED column [#57147](https://github.com/ClickHouse/ClickHouse/pull/57147) ([Nikolay Degterinsky](https://github.com/evillique)).
+* Fix incorrect JOIN plan optimization with partially materialized normal projection [#57196](https://github.com/ClickHouse/ClickHouse/pull/57196) ([Amos Bird](https://github.com/amosbird)).
+* Fix `ReadonlyReplica` metric for all cases [#57267](https://github.com/ClickHouse/ClickHouse/pull/57267) ([Antonio Andelic](https://github.com/antonio2368)).
+* Fix working with read buffers in StreamingFormatExecutor [#57438](https://github.com/ClickHouse/ClickHouse/pull/57438) ([Kruglov Pavel](https://github.com/Avogar)).
+* bugfix: correctly parse SYSTEM STOP LISTEN TCP SECURE [#57483](https://github.com/ClickHouse/ClickHouse/pull/57483) ([joelynch](https://github.com/joelynch)).
+* Ignore ON CLUSTER clause in grant/revoke queries for management of replicated access entities. [#57538](https://github.com/ClickHouse/ClickHouse/pull/57538) ([MikhailBurdukov](https://github.com/MikhailBurdukov)).
+* Disable system.kafka_consumers by default (due to possible live memory leak) [#57822](https://github.com/ClickHouse/ClickHouse/pull/57822) ([Azat Khuzhin](https://github.com/azat)).
+* Fix invalid memory access in BLAKE3 (Rust) [#57876](https://github.com/ClickHouse/ClickHouse/pull/57876) ([Raúl Marín](https://github.com/Algunenano)).
+* Normalize function names in CREATE INDEX [#57906](https://github.com/ClickHouse/ClickHouse/pull/57906) ([Alexander Tokmakov](https://github.com/tavplubix)).
+* Fix invalid preprocessing on Keeper [#58069](https://github.com/ClickHouse/ClickHouse/pull/58069) ([Antonio Andelic](https://github.com/antonio2368)).
+* Fix Integer overflow in Poco::UTF32Encoding [#58073](https://github.com/ClickHouse/ClickHouse/pull/58073) ([Andrey Fedotov](https://github.com/anfedotoff)).
+* Remove parallel parsing for JSONCompactEachRow [#58181](https://github.com/ClickHouse/ClickHouse/pull/58181) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
+* Fix parallel parsing for JSONCompactEachRow [#58250](https://github.com/ClickHouse/ClickHouse/pull/58250) ([Kruglov Pavel](https://github.com/Avogar)).
+
+#### NO CL ENTRY
+
+* NO CL ENTRY: 'Update PeekableWriteBuffer.cpp'. [#57701](https://github.com/ClickHouse/ClickHouse/pull/57701) ([Kruglov Pavel](https://github.com/Avogar)).
+
+#### NOT FOR CHANGELOG / INSIGNIFICANT
+
+* Pin alpine version of integration tests helper container [#57669](https://github.com/ClickHouse/ClickHouse/pull/57669) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* Remove heavy rust stable toolchain [#57905](https://github.com/ClickHouse/ClickHouse/pull/57905) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
+* Fix docker image for integration tests (fixes CI) [#57952](https://github.com/ClickHouse/ClickHouse/pull/57952) ([Azat Khuzhin](https://github.com/azat)).
+
diff --git a/docs/en/development/build-cross-s390x.md b/docs/en/development/build-cross-s390x.md
index 088dd6f2679..b7cda515d77 100644
--- a/docs/en/development/build-cross-s390x.md
+++ b/docs/en/development/build-cross-s390x.md
@@ -1,206 +1,206 @@
----
-slug: /en/development/build-cross-s390x
-sidebar_position: 69
-title: How to Build, Run and Debug ClickHouse on Linux for s390x (zLinux)
-sidebar_label: Build on Linux for s390x (zLinux)
----
-
-As of writing (2023/3/10) building for s390x considered to be experimental. Not all features can be enabled, has broken features and is currently under active development.
-
-
-## Building
-
-As s390x does not support boringssl, it uses OpenSSL and has two related build options.
-- By default, the s390x build will dynamically link to OpenSSL libraries. It will build OpenSSL shared objects, so it's not necessary to install OpenSSL beforehand. (This option is recommended in all cases.)
-- Another option is to build OpenSSL in-tree. In this case two build flags need to be supplied to cmake
-```bash
--DENABLE_OPENSSL_DYNAMIC=0 -DENABLE_OPENSSL=1
-```
-
-These instructions assume that the host machine is x86_64 and has all the tooling required to build natively based on the [build instructions](../development/build.md). It also assumes that the host is Ubuntu 22.04 but the following instructions should also work on Ubuntu 20.04.
-
-In addition to installing the tooling used to build natively, the following additional packages need to be installed:
-
-```bash
-apt-get install binutils-s390x-linux-gnu libc6-dev-s390x-cross gcc-s390x-linux-gnu binfmt-support qemu-user-static
-```
-
-If you wish to cross compile rust code install the rust cross compile target for s390x:
-```bash
-rustup target add s390x-unknown-linux-gnu
-```
-
-To build for s390x:
-```bash
-cmake -DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-s390x.cmake ..
-ninja
-```
-
-## Running
-
-Once built, the binary can be run with, eg.:
-
-```bash
-qemu-s390x-static -L /usr/s390x-linux-gnu ./clickhouse
-```
-
-## Debugging
-
-Install LLDB:
-
-```bash
-apt-get install lldb-15
-```
-
-To Debug a s390x executable, run clickhouse using QEMU in debug mode:
-
-```bash
-qemu-s390x-static -g 31338 -L /usr/s390x-linux-gnu ./clickhouse
-```
-
-In another shell run LLDB and attach, replace `` and `` with the values corresponding to your environment.
-```bash
-lldb-15
-(lldb) target create ./clickhouse
-Current executable set to '//ClickHouse//programs/clickhouse' (s390x).
-(lldb) settings set target.source-map //ClickHouse
-(lldb) gdb-remote 31338
-Process 1 stopped
-* thread #1, stop reason = signal SIGTRAP
- frame #0: 0x0000004020e74cd0
--> 0x4020e74cd0: lgr %r2, %r15
- 0x4020e74cd4: aghi %r15, -160
- 0x4020e74cd8: xc 0(8,%r15), 0(%r15)
- 0x4020e74cde: brasl %r14, 275429939040
-(lldb) b main
-Breakpoint 1: 9 locations.
-(lldb) c
-Process 1 resuming
-Process 1 stopped
-* thread #1, stop reason = breakpoint 1.1
- frame #0: 0x0000004005cd9fc0 clickhouse`main(argc_=1, argv_=0x0000004020e594a8) at main.cpp:450:17
- 447 #if !defined(FUZZING_MODE)
- 448 int main(int argc_, char ** argv_)
- 449 {
--> 450 inside_main = true;
- 451 SCOPE_EXIT({ inside_main = false; });
- 452
- 453 /// PHDR cache is required for query profiler to work reliably
-```
-
-## Visual Studio Code integration
-
-- [CodeLLDB](https://github.com/vadimcn/vscode-lldb) extension is required for visual debugging.
-- [Command Variable](https://github.com/rioj7/command-variable) extension can help dynamic launches if using [CMake Variants](https://github.com/microsoft/vscode-cmake-tools/blob/main/docs/variants.md).
-- Make sure to set the backend to your LLVM installation eg. `"lldb.library": "/usr/lib/x86_64-linux-gnu/liblldb-15.so"`
-- Make sure to run the clickhouse executable in debug mode prior to launch. (It is also possible to create a `preLaunchTask` that automates this)
-
-### Example configurations
-#### cmake-variants.yaml
-```yaml
-buildType:
- default: relwithdebinfo
- choices:
- debug:
- short: Debug
- long: Emit debug information
- buildType: Debug
- release:
- short: Release
- long: Optimize generated code
- buildType: Release
- relwithdebinfo:
- short: RelWithDebInfo
- long: Release with Debug Info
- buildType: RelWithDebInfo
- tsan:
- short: MinSizeRel
- long: Minimum Size Release
- buildType: MinSizeRel
-
-toolchain:
- default: default
- description: Select toolchain
- choices:
- default:
- short: x86_64
- long: x86_64
- s390x:
- short: s390x
- long: s390x
- settings:
- CMAKE_TOOLCHAIN_FILE: cmake/linux/toolchain-s390x.cmake
-```
-
-#### launch.json
-```json
-{
- "version": "0.2.0",
- "configurations": [
- {
- "type": "lldb",
- "request": "custom",
- "name": "(lldb) Launch s390x with qemu",
- "targetCreateCommands": ["target create ${command:cmake.launchTargetPath}"],
- "processCreateCommands": ["gdb-remote 2159"],
- "preLaunchTask": "Run ClickHouse"
- }
- ]
-}
-```
-
-#### settings.json
-This would also put different builds under different subfolders of the `build` folder.
-```json
-{
- "cmake.buildDirectory": "${workspaceFolder}/build/${buildKitVendor}-${buildKitVersion}-${variant:toolchain}-${variant:buildType}",
- "lldb.library": "/usr/lib/x86_64-linux-gnu/liblldb-15.so"
-}
-```
-
-#### run-debug.sh
-```sh
-#! /bin/sh
-echo 'Starting debugger session'
-cd $1
-qemu-s390x-static -g 2159 -L /usr/s390x-linux-gnu $2 $3 $4
-```
-
-#### tasks.json
-Defines a task to run the compiled executable in `server` mode under a `tmp` folder next to the binaries, with configuration from under `programs/server/config.xml`.
-```json
-{
- "version": "2.0.0",
- "tasks": [
- {
- "label": "Run ClickHouse",
- "type": "shell",
- "isBackground": true,
- "command": "${workspaceFolder}/.vscode/run-debug.sh",
- "args": [
- "${command:cmake.launchTargetDirectory}/tmp",
- "${command:cmake.launchTargetPath}",
- "server",
- "--config-file=${workspaceFolder}/programs/server/config.xml"
- ],
- "problemMatcher": [
- {
- "pattern": [
- {
- "regexp": ".",
- "file": 1,
- "location": 2,
- "message": 3
- }
- ],
- "background": {
- "activeOnStart": true,
- "beginsPattern": "^Starting debugger session",
- "endsPattern": ".*"
- }
- }
- ]
- }
- ]
-}
-```
+---
+slug: /en/development/build-cross-s390x
+sidebar_position: 69
+title: How to Build, Run and Debug ClickHouse on Linux for s390x (zLinux)
+sidebar_label: Build on Linux for s390x (zLinux)
+---
+
+As of writing (2023/3/10) building for s390x considered to be experimental. Not all features can be enabled, has broken features and is currently under active development.
+
+
+## Building
+
+As s390x does not support boringssl, it uses OpenSSL and has two related build options.
+- By default, the s390x build will dynamically link to OpenSSL libraries. It will build OpenSSL shared objects, so it's not necessary to install OpenSSL beforehand. (This option is recommended in all cases.)
+- Another option is to build OpenSSL in-tree. In this case two build flags need to be supplied to cmake
+```bash
+-DENABLE_OPENSSL_DYNAMIC=0 -DENABLE_OPENSSL=1
+```
+
+These instructions assume that the host machine is x86_64 and has all the tooling required to build natively based on the [build instructions](../development/build.md). It also assumes that the host is Ubuntu 22.04 but the following instructions should also work on Ubuntu 20.04.
+
+In addition to installing the tooling used to build natively, the following additional packages need to be installed:
+
+```bash
+apt-get install binutils-s390x-linux-gnu libc6-dev-s390x-cross gcc-s390x-linux-gnu binfmt-support qemu-user-static
+```
+
+If you wish to cross compile rust code install the rust cross compile target for s390x:
+```bash
+rustup target add s390x-unknown-linux-gnu
+```
+
+To build for s390x:
+```bash
+cmake -DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-s390x.cmake ..
+ninja
+```
+
+## Running
+
+Once built, the binary can be run with, eg.:
+
+```bash
+qemu-s390x-static -L /usr/s390x-linux-gnu ./clickhouse
+```
+
+## Debugging
+
+Install LLDB:
+
+```bash
+apt-get install lldb-15
+```
+
+To Debug a s390x executable, run clickhouse using QEMU in debug mode:
+
+```bash
+qemu-s390x-static -g 31338 -L /usr/s390x-linux-gnu ./clickhouse
+```
+
+In another shell run LLDB and attach, replace `` and `` with the values corresponding to your environment.
+```bash
+lldb-15
+(lldb) target create ./clickhouse
+Current executable set to '//ClickHouse//programs/clickhouse' (s390x).
+(lldb) settings set target.source-map //ClickHouse
+(lldb) gdb-remote 31338
+Process 1 stopped
+* thread #1, stop reason = signal SIGTRAP
+ frame #0: 0x0000004020e74cd0
+-> 0x4020e74cd0: lgr %r2, %r15
+ 0x4020e74cd4: aghi %r15, -160
+ 0x4020e74cd8: xc 0(8,%r15), 0(%r15)
+ 0x4020e74cde: brasl %r14, 275429939040
+(lldb) b main
+Breakpoint 1: 9 locations.
+(lldb) c
+Process 1 resuming
+Process 1 stopped
+* thread #1, stop reason = breakpoint 1.1
+ frame #0: 0x0000004005cd9fc0 clickhouse`main(argc_=1, argv_=0x0000004020e594a8) at main.cpp:450:17
+ 447 #if !defined(FUZZING_MODE)
+ 448 int main(int argc_, char ** argv_)
+ 449 {
+-> 450 inside_main = true;
+ 451 SCOPE_EXIT({ inside_main = false; });
+ 452
+ 453 /// PHDR cache is required for query profiler to work reliably
+```
+
+## Visual Studio Code integration
+
+- [CodeLLDB](https://github.com/vadimcn/vscode-lldb) extension is required for visual debugging.
+- [Command Variable](https://github.com/rioj7/command-variable) extension can help dynamic launches if using [CMake Variants](https://github.com/microsoft/vscode-cmake-tools/blob/main/docs/variants.md).
+- Make sure to set the backend to your LLVM installation eg. `"lldb.library": "/usr/lib/x86_64-linux-gnu/liblldb-15.so"`
+- Make sure to run the clickhouse executable in debug mode prior to launch. (It is also possible to create a `preLaunchTask` that automates this)
+
+### Example configurations
+#### cmake-variants.yaml
+```yaml
+buildType:
+ default: relwithdebinfo
+ choices:
+ debug:
+ short: Debug
+ long: Emit debug information
+ buildType: Debug
+ release:
+ short: Release
+ long: Optimize generated code
+ buildType: Release
+ relwithdebinfo:
+ short: RelWithDebInfo
+ long: Release with Debug Info
+ buildType: RelWithDebInfo
+ tsan:
+ short: MinSizeRel
+ long: Minimum Size Release
+ buildType: MinSizeRel
+
+toolchain:
+ default: default
+ description: Select toolchain
+ choices:
+ default:
+ short: x86_64
+ long: x86_64
+ s390x:
+ short: s390x
+ long: s390x
+ settings:
+ CMAKE_TOOLCHAIN_FILE: cmake/linux/toolchain-s390x.cmake
+```
+
+#### launch.json
+```json
+{
+ "version": "0.2.0",
+ "configurations": [
+ {
+ "type": "lldb",
+ "request": "custom",
+ "name": "(lldb) Launch s390x with qemu",
+ "targetCreateCommands": ["target create ${command:cmake.launchTargetPath}"],
+ "processCreateCommands": ["gdb-remote 2159"],
+ "preLaunchTask": "Run ClickHouse"
+ }
+ ]
+}
+```
+
+#### settings.json
+This would also put different builds under different subfolders of the `build` folder.
+```json
+{
+ "cmake.buildDirectory": "${workspaceFolder}/build/${buildKitVendor}-${buildKitVersion}-${variant:toolchain}-${variant:buildType}",
+ "lldb.library": "/usr/lib/x86_64-linux-gnu/liblldb-15.so"
+}
+```
+
+#### run-debug.sh
+```sh
+#! /bin/sh
+echo 'Starting debugger session'
+cd $1
+qemu-s390x-static -g 2159 -L /usr/s390x-linux-gnu $2 $3 $4
+```
+
+#### tasks.json
+Defines a task to run the compiled executable in `server` mode under a `tmp` folder next to the binaries, with configuration from under `programs/server/config.xml`.
+```json
+{
+ "version": "2.0.0",
+ "tasks": [
+ {
+ "label": "Run ClickHouse",
+ "type": "shell",
+ "isBackground": true,
+ "command": "${workspaceFolder}/.vscode/run-debug.sh",
+ "args": [
+ "${command:cmake.launchTargetDirectory}/tmp",
+ "${command:cmake.launchTargetPath}",
+ "server",
+ "--config-file=${workspaceFolder}/programs/server/config.xml"
+ ],
+ "problemMatcher": [
+ {
+ "pattern": [
+ {
+ "regexp": ".",
+ "file": 1,
+ "location": 2,
+ "message": 3
+ }
+ ],
+ "background": {
+ "activeOnStart": true,
+ "beginsPattern": "^Starting debugger session",
+ "endsPattern": ".*"
+ }
+ }
+ ]
+ }
+ ]
+}
+```
diff --git a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md
index 9467da33398..6de818c130f 100644
--- a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md
+++ b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md
@@ -25,7 +25,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
[ORDER BY expr]
[PRIMARY KEY expr]
[SAMPLE BY expr]
-[SETTINGS name=value, ...]
+[SETTINGS name=value, clean_deleted_rows=value, ...]
```
For a description of request parameters, see [statement description](../../../sql-reference/statements/create/table.md).
@@ -88,6 +88,53 @@ SELECT * FROM mySecondReplacingMT FINAL;
└─────┴─────────┴─────────────────────┘
```
+### is_deleted
+
+`is_deleted` — Name of a column used during a merge to determine whether the data in this row represents the state or is to be deleted; `1` is a “deleted“ row, `0` is a “state“ row.
+
+ Column data type — `UInt8`.
+
+:::note
+`is_deleted` can only be enabled when `ver` is used.
+
+The row is deleted when `OPTIMIZE ... FINAL CLEANUP` or `OPTIMIZE ... FINAL` is used, or if the engine setting `clean_deleted_rows` has been set to `Always`.
+
+No matter the operation on the data, the version must be increased. If two inserted rows have the same version number, the last inserted row is the one kept.
+
+:::
+
+Example:
+```sql
+-- with ver and is_deleted
+CREATE OR REPLACE TABLE myThirdReplacingMT
+(
+ `key` Int64,
+ `someCol` String,
+ `eventTime` DateTime,
+ `is_deleted` UInt8
+)
+ENGINE = ReplacingMergeTree(eventTime, is_deleted)
+ORDER BY key;
+
+INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 01:01:01', 0);
+INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 01:01:01', 1);
+
+select * from myThirdReplacingMT final;
+
+0 rows in set. Elapsed: 0.003 sec.
+
+-- delete rows with is_deleted
+OPTIMIZE TABLE myThirdReplacingMT FINAL CLEANUP;
+
+INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 00:00:00', 0);
+
+select * from myThirdReplacingMT final;
+
+┌─key─┬─someCol─┬───────────eventTime─┬─is_deleted─┐
+│ 1 │ first │ 2020-01-01 00:00:00 │ 0 │
+└─────┴─────────┴─────────────────────┴────────────┘
+```
+
## Query clauses
When creating a `ReplacingMergeTree` table the same [clauses](../../../engines/table-engines/mergetree-family/mergetree.md) are required, as when creating a `MergeTree` table.
diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md
index e24c6e5a6dc..a6bbb20cc8d 100644
--- a/docs/en/getting-started/example-datasets/youtube-dislikes.md
+++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md
@@ -25,8 +25,7 @@ The steps below will easily work on a local install of ClickHouse too. The only
1. Let's see what the data looks like. The `s3cluster` table function returns a table, so we can `DESCRIBE` the result:
```sql
-DESCRIBE s3Cluster(
- 'default',
+DESCRIBE s3(
'https://clickhouse-public-datasets.s3.amazonaws.com/youtube/original/files/*.zst',
'JSONLines'
);
@@ -35,29 +34,29 @@ DESCRIBE s3Cluster(
ClickHouse infers the following schema from the JSON file:
```response
-┌─name────────────────┬─type─────────────────────────────────┐
-│ id │ Nullable(String) │
-│ fetch_date │ Nullable(Int64) │
-│ upload_date │ Nullable(String) │
-│ title │ Nullable(String) │
-│ uploader_id │ Nullable(String) │
-│ uploader │ Nullable(String) │
-│ uploader_sub_count │ Nullable(Int64) │
-│ is_age_limit │ Nullable(Bool) │
-│ view_count │ Nullable(Int64) │
-│ like_count │ Nullable(Int64) │
-│ dislike_count │ Nullable(Int64) │
-│ is_crawlable │ Nullable(Bool) │
-│ is_live_content │ Nullable(Bool) │
-│ has_subtitles │ Nullable(Bool) │
-│ is_ads_enabled │ Nullable(Bool) │
-│ is_comments_enabled │ Nullable(Bool) │
-│ description │ Nullable(String) │
-│ rich_metadata │ Array(Map(String, Nullable(String))) │
-│ super_titles │ Array(Map(String, Nullable(String))) │
-│ uploader_badges │ Nullable(String) │
-│ video_badges │ Nullable(String) │
-└─────────────────────┴──────────────────────────────────────┘
+┌─name────────────────┬─type───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ id │ Nullable(String) │ │ │ │ │ │
+│ fetch_date │ Nullable(String) │ │ │ │ │ │
+│ upload_date │ Nullable(String) │ │ │ │ │ │
+│ title │ Nullable(String) │ │ │ │ │ │
+│ uploader_id │ Nullable(String) │ │ │ │ │ │
+│ uploader │ Nullable(String) │ │ │ │ │ │
+│ uploader_sub_count │ Nullable(Int64) │ │ │ │ │ │
+│ is_age_limit │ Nullable(Bool) │ │ │ │ │ │
+│ view_count │ Nullable(Int64) │ │ │ │ │ │
+│ like_count │ Nullable(Int64) │ │ │ │ │ │
+│ dislike_count │ Nullable(Int64) │ │ │ │ │ │
+│ is_crawlable │ Nullable(Bool) │ │ │ │ │ │
+│ is_live_content │ Nullable(Bool) │ │ │ │ │ │
+│ has_subtitles │ Nullable(Bool) │ │ │ │ │ │
+│ is_ads_enabled │ Nullable(Bool) │ │ │ │ │ │
+│ is_comments_enabled │ Nullable(Bool) │ │ │ │ │ │
+│ description │ Nullable(String) │ │ │ │ │ │
+│ rich_metadata │ Array(Tuple(call Nullable(String), content Nullable(String), subtitle Nullable(String), title Nullable(String), url Nullable(String))) │ │ │ │ │ │
+│ super_titles │ Array(Tuple(text Nullable(String), url Nullable(String))) │ │ │ │ │ │
+│ uploader_badges │ Nullable(String) │ │ │ │ │ │
+│ video_badges │ Nullable(String) │ │ │ │ │ │
+└─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
```
2. Based on the inferred schema, we cleaned up the data types and added a primary key. Define the following table:
@@ -82,13 +81,13 @@ CREATE TABLE youtube
`is_ads_enabled` Bool,
`is_comments_enabled` Bool,
`description` String,
- `rich_metadata` Array(Map(String, String)),
- `super_titles` Array(Map(String, String)),
+ `rich_metadata` Array(Tuple(call String, content String, subtitle String, title String, url String)),
+ `super_titles` Array(Tuple(text String, url String)),
`uploader_badges` String,
`video_badges` String
)
ENGINE = MergeTree
-ORDER BY (uploader, upload_date);
+ORDER BY (uploader, upload_date)
```
3. The following command streams the records from the S3 files into the `youtube` table.
diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md
index 836b1f2f637..ed67af48af7 100644
--- a/docs/en/interfaces/formats.md
+++ b/docs/en/interfaces/formats.md
@@ -1262,6 +1262,7 @@ SELECT * FROM json_each_row_nested
- [input_format_import_nested_json](/docs/en/operations/settings/settings-formats.md/#input_format_import_nested_json) - map nested JSON data to nested tables (it works for JSONEachRow format). Default value - `false`.
- [input_format_json_read_bools_as_numbers](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_bools_as_numbers) - allow to parse bools as numbers in JSON input formats. Default value - `true`.
+- [input_format_json_read_bools_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_bools_as_strings) - allow to parse bools as strings in JSON input formats. Default value - `true`.
- [input_format_json_read_numbers_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_numbers_as_strings) - allow to parse numbers as strings in JSON input formats. Default value - `true`.
- [input_format_json_read_arrays_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_arrays_as_strings) - allow to parse JSON arrays as strings in JSON input formats. Default value - `true`.
- [input_format_json_read_objects_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_objects_as_strings) - allow to parse JSON objects as strings in JSON input formats. Default value - `true`.
diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md
index ef858796936..4db1d53987a 100644
--- a/docs/en/interfaces/schema-inference.md
+++ b/docs/en/interfaces/schema-inference.md
@@ -614,6 +614,26 @@ DESC format(JSONEachRow, $$
└───────┴─────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
```
+##### input_format_json_read_bools_as_strings
+
+Enabling this setting allows reading Bool values as strings.
+
+This setting is enabled by default.
+
+**Example:**
+
+```sql
+SET input_format_json_read_bools_as_strings = 1;
+DESC format(JSONEachRow, $$
+ {"value" : true}
+ {"value" : "Hello, World"}
+ $$)
+```
+```response
+┌─name──┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
+│ value │ Nullable(String) │ │ │ │ │ │
+└───────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
+```
##### input_format_json_read_arrays_as_strings
Enabling this setting allows reading JSON array values as strings.
diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md
index 0b858038caf..50c5ff4457f 100644
--- a/docs/en/operations/query-cache.md
+++ b/docs/en/operations/query-cache.md
@@ -29,8 +29,8 @@ Transactionally inconsistent caching is traditionally provided by client tools o
the same caching logic and configuration is often duplicated. With ClickHouse's query cache, the caching logic moves to the server side.
This reduces maintenance effort and avoids redundancy.
-:::security consideration
-The cached query result is tied to the user executing it. Authorization checks are performed when the query is executed. This means that if there are any alterations to the user's role or permissions between the time the query is cached and when the cache is accessed, the result will not reflect these changes. We recommend using different users to distinguish between different levels of access, instead of actively toggling roles for a single user between queries, as this practice may lead to unexpected query results.
+:::note
+Security consideration: The cached query result is tied to the user executing it. Authorization checks are performed when the query is executed. This means that if there are any alterations to the user's role or permissions between the time the query is cached and when the cache is accessed, the result will not reflect these changes. We recommend using different users to distinguish between different levels of access, instead of actively toggling roles for a single user between queries, as this practice may lead to unexpected query results.
:::
## Configuration Settings and Usage
diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md
index da049554c67..c7e461d15ae 100644
--- a/docs/en/operations/settings/merge-tree-settings.md
+++ b/docs/en/operations/settings/merge-tree-settings.md
@@ -852,6 +852,16 @@ If the file name for column is too long (more than `max_file_name_length` bytes)
The maximal length of the file name to keep it as is without hashing. Takes effect only if setting `replace_long_file_name_to_hash` is enabled. The value of this setting does not include the length of file extension. So, it is recommended to set it below the maximum filename length (usually 255 bytes) with some gap to avoid filesystem errors. Default value: 127.
+## clean_deleted_rows
+
+Enable/disable automatic deletion of rows flagged as `is_deleted` when perform `OPTIMIZE ... FINAL` on a table using the ReplacingMergeTree engine. When disabled, the `CLEANUP` keyword has to be added to the `OPTIMIZE ... FINAL` to have the same behaviour.
+
+Possible values:
+
+- `Always` or `Never`.
+
+Default value: `Never`
+
## allow_experimental_block_number_column
Persists virtual column `_block_number` on merges.
diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md
index 3d76bd9df73..43a73844b79 100644
--- a/docs/en/operations/settings/settings-formats.md
+++ b/docs/en/operations/settings/settings-formats.md
@@ -377,6 +377,12 @@ Allow parsing bools as numbers in JSON input formats.
Enabled by default.
+## input_format_json_read_bools_as_strings {#input_format_json_read_bools_as_strings}
+
+Allow parsing bools as strings in JSON input formats.
+
+Enabled by default.
+
## input_format_json_read_numbers_as_strings {#input_format_json_read_numbers_as_strings}
Allow parsing numbers as strings in JSON input formats.
diff --git a/docs/en/operations/settings/settings-users.md b/docs/en/operations/settings/settings-users.md
index 1f41eafd02e..96477f777a9 100644
--- a/docs/en/operations/settings/settings-users.md
+++ b/docs/en/operations/settings/settings-users.md
@@ -4,7 +4,7 @@ sidebar_position: 63
sidebar_label: User Settings
---
-# User Settings
+# Users and Roles Settings
The `users` section of the `user.xml` configuration file contains user settings.
@@ -187,3 +187,34 @@ The following configuration forces that user `user1` can only see the rows of `t
```
The `filter` can be any expression resulting in a [UInt8](../../sql-reference/data-types/int-uint.md)-type value. It usually contains comparisons and logical operators. Rows from `database_name.table1` where filter results to 0 are not returned for this user. The filtering is incompatible with `PREWHERE` operations and disables `WHERE→PREWHERE` optimization.
+
+## Roles
+
+You can create any predefined roles using the `roles` section of the `user.xml` configuration file.
+
+Structure of the `roles` section:
+
+```xml
+
+
+
+ GRANT SHOW ON *.*
+ REVOKE SHOW ON system.*
+ GRANT CREATE ON *.* WITH GRANT OPTION
+
+
+
+```
+
+These roles can also be granted to users from the `users` section:
+
+```xml
+
+
+ ...
+
+ GRANT test_role
+
+
+
+```
diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md
index 6e087467bb9..d4ee8106320 100644
--- a/docs/en/operations/settings/settings.md
+++ b/docs/en/operations/settings/settings.md
@@ -3847,6 +3847,8 @@ Possible values:
- `none` — Is similar to throw, but distributed DDL query returns no result set.
- `null_status_on_timeout` — Returns `NULL` as execution status in some rows of result set instead of throwing `TIMEOUT_EXCEEDED` if query is not finished on the corresponding hosts.
- `never_throw` — Do not throw `TIMEOUT_EXCEEDED` and do not rethrow exceptions if query has failed on some hosts.
+- `null_status_on_timeout_only_active` — similar to `null_status_on_timeout`, but doesn't wait for inactive replicas of the `Replicated` database
+- `throw_only_active` — similar to `throw`, but doesn't wait for inactive replicas of the `Replicated` database
Default value: `throw`.
diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md
index 796f65a9d30..b3ef1128c42 100644
--- a/docs/en/operations/storing-data.md
+++ b/docs/en/operations/storing-data.md
@@ -196,7 +196,7 @@ These settings should be defined in the disk configuration section.
- `max_elements` - a limit for a number of cache files. Default: `10000000`.
-- `load_metadata_threads` - number of threads being used to load cache metadata on starting time. Default: `1`.
+- `load_metadata_threads` - number of threads being used to load cache metadata on starting time. Default: `16`.
File Cache **query/profile settings**:
diff --git a/docs/en/operations/system-tables/asynchronous_metrics.md b/docs/en/operations/system-tables/asynchronous_metrics.md
index 90c5a7d2e7a..fe8f963b1ec 100644
--- a/docs/en/operations/system-tables/asynchronous_metrics.md
+++ b/docs/en/operations/system-tables/asynchronous_metrics.md
@@ -551,6 +551,14 @@ Total amount of bytes (compressed, including data and indices) stored in all tab
Total amount of data parts in all tables of MergeTree family. Numbers larger than 10 000 will negatively affect the server startup time and it may indicate unreasonable choice of the partition key.
+### TotalPrimaryKeyBytesInMemory
+
+The total amount of memory (in bytes) used by primary key values (only takes active parts into account).
+
+### TotalPrimaryKeyBytesInMemoryAllocated
+
+The total amount of memory (in bytes) reserved for primary key values (only takes active parts into account).
+
### TotalRowsOfMergeTreeTables
Total amount of rows (records) stored in all tables of MergeTree family.
diff --git a/docs/en/operations/system-tables/database_engines.md b/docs/en/operations/system-tables/database_engines.md
new file mode 100644
index 00000000000..09f0687af65
--- /dev/null
+++ b/docs/en/operations/system-tables/database_engines.md
@@ -0,0 +1,26 @@
+---
+slug: /en/operations/system-tables/database_engines
+---
+# database_engines
+
+Contains the list of database engines supported by the server.
+
+This table contains the following columns (the column type is shown in brackets):
+
+- `name` (String) — The name of database engine.
+
+Example:
+
+``` sql
+SELECT *
+FROM system.database_engines
+WHERE name in ('Atomic', 'Lazy', 'Ordinary')
+```
+
+``` text
+┌─name─────┐
+│ Ordinary │
+│ Atomic │
+│ Lazy │
+└──────────┘
+```
diff --git a/docs/en/operations/system-tables/dropped_tables_parts.md b/docs/en/operations/system-tables/dropped_tables_parts.md
new file mode 100644
index 00000000000..095f35287fe
--- /dev/null
+++ b/docs/en/operations/system-tables/dropped_tables_parts.md
@@ -0,0 +1,14 @@
+---
+slug: /en/operations/system-tables/dropped_tables_parts
+---
+# dropped_tables_parts {#system_tables-dropped_tables_parts}
+
+Contains information about parts of [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) dropped tables from [system.dropped_tables](./dropped_tables.md)
+
+The schema of this table is the same as [system.parts](./parts.md)
+
+**See Also**
+
+- [MergeTree family](../../engines/table-engines/mergetree-family/mergetree.md)
+- [system.parts](./parts.md)
+- [system.dropped_tables](./dropped_tables.md)
diff --git a/docs/en/operations/system-tables/errors.md b/docs/en/operations/system-tables/errors.md
index 01762962152..4582ea631b3 100644
--- a/docs/en/operations/system-tables/errors.md
+++ b/docs/en/operations/system-tables/errors.md
@@ -9,11 +9,15 @@ Columns:
- `name` ([String](../../sql-reference/data-types/string.md)) — name of the error (`errorCodeToName`).
- `code` ([Int32](../../sql-reference/data-types/int-uint.md)) — code number of the error.
-- `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — the number of times this error has been happened.
-- `last_error_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — time when the last error happened.
+- `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — the number of times this error happened.
+- `last_error_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — the time when the last error happened.
- `last_error_message` ([String](../../sql-reference/data-types/string.md)) — message for the last error.
-- `last_error_trace` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — A [stack trace](https://en.wikipedia.org/wiki/Stack_trace) which represents a list of physical addresses where the called methods are stored.
-- `remote` ([UInt8](../../sql-reference/data-types/int-uint.md)) — remote exception (i.e. received during one of the distributed query).
+- `last_error_trace` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — A [stack trace](https://en.wikipedia.org/wiki/Stack_trace) that represents a list of physical addresses where the called methods are stored.
+- `remote` ([UInt8](../../sql-reference/data-types/int-uint.md)) — remote exception (i.e. received during one of the distributed queries).
+
+:::note
+Counters for some errors may increase during successful query execution. It's not recommended to use this table for server monitoring purposes unless you are sure that corresponding error can not be a false positive.
+:::
**Example**
diff --git a/docs/en/operations/system-tables/server_settings.md b/docs/en/operations/system-tables/server_settings.md
index 7efe605ccef..417c3460a53 100644
--- a/docs/en/operations/system-tables/server_settings.md
+++ b/docs/en/operations/system-tables/server_settings.md
@@ -14,6 +14,11 @@ Columns:
- `changed` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether a setting was specified in `config.xml`
- `description` ([String](../../sql-reference/data-types/string.md)) — Short server setting description.
- `type` ([String](../../sql-reference/data-types/string.md)) — Server setting value type.
+- `changeable_without_restart` ([Enum8](../../sql-reference/data-types/enum.md)) — Whether the setting can be changed at server runtime. Values:
+ - `'No' `
+ - `'IncreaseOnly'`
+ - `'DecreaseOnly'`
+ - `'Yes'`
- `is_obsolete` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) - Shows whether a setting is obsolete.
**Example**
@@ -27,22 +32,21 @@ WHERE name LIKE '%thread_pool%'
```
``` text
-┌─name────────────────────────────────────────_─value─_─default─_─changed─_─description──────────────────────────────────────────────────────────────────────────────────────────────────────
-───────────────────────────────────_─type───_─is_obsolete─┐
-│ max_thread_pool_size │ 10000 │ 10000 │ 1 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │ 0 │
-│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │ 0 │
-│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │ 0 │
-│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │ 0 │
-│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │ 0 │
-│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │ 0 │
-│ max_active_parts_loading_thread_pool_size │ 64 │ 64 │ 0 │ The number of threads to load active set of data parts (Active ones) at startup. │ UInt64 │ 0 │
-│ max_outdated_parts_loading_thread_pool_size │ 32 │ 32 │ 0 │ The number of threads to load inactive set of data parts (Outdated ones) at startup. │ UInt64 │ 0 │
-│ max_parts_cleaning_thread_pool_size │ 128 │ 128 │ 0 │ The number of threads for concurrent removal of inactive data parts. │ UInt64 │ 0 │
-│ max_backups_io_thread_pool_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that would be used for IO operations for BACKUP queries │ UInt64 │ 0 │
-│ max_backups_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for backups IO thread pool. │ UInt64 │ 0 │
-│ backups_io_thread_pool_queue_size │ 0 │ 0 │ 0 │ Queue size for backups IO thread pool. │ UInt64 │ 0 │
-└─────────────────────────────────────────────┴───────┴─────────┴─────────┴──────────────────────────────────────────────────────────────────────────────────────────────────────────────────
-───────────────────────────────────┴────────┴─────────────┘
+┌─name────────────────────────────────────────┬─value─┬─default─┬─changed─┬─description─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─type───┬─changeable_without_restart─┬─is_obsolete─┐
+│ max_thread_pool_size │ 10000 │ 10000 │ 0 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │ No │ 0 │
+│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │ No │ 0 │
+│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │ No │ 0 │
+│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │ No │ 0 │
+│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │ No │ 0 │
+│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │ No │ 0 │
+│ max_active_parts_loading_thread_pool_size │ 64 │ 64 │ 0 │ The number of threads to load active set of data parts (Active ones) at startup. │ UInt64 │ No │ 0 │
+│ max_outdated_parts_loading_thread_pool_size │ 32 │ 32 │ 0 │ The number of threads to load inactive set of data parts (Outdated ones) at startup. │ UInt64 │ No │ 0 │
+│ max_parts_cleaning_thread_pool_size │ 128 │ 128 │ 0 │ The number of threads for concurrent removal of inactive data parts. │ UInt64 │ No │ 0 │
+│ max_backups_io_thread_pool_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that would be used for IO operations for BACKUP queries │ UInt64 │ No │ 0 │
+│ max_backups_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for backups IO thread pool. │ UInt64 │ No │ 0 │
+│ backups_io_thread_pool_queue_size │ 0 │ 0 │ 0 │ Queue size for backups IO thread pool. │ UInt64 │ No │ 0 │
+└─────────────────────────────────────────────┴───────┴─────────┴─────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴────────┴────────────────────────────┴─────────────┘
+
```
Using of `WHERE changed` can be useful, for example, when you want to check
diff --git a/docs/en/operations/system-tables/view_refreshes.md b/docs/en/operations/system-tables/view_refreshes.md
new file mode 100644
index 00000000000..12377507b39
--- /dev/null
+++ b/docs/en/operations/system-tables/view_refreshes.md
@@ -0,0 +1,43 @@
+---
+slug: /en/operations/system-tables/view_refreshes
+---
+# view_refreshes
+
+Information about [Refreshable Materialized Views](../../sql-reference/statements/create/view.md#refreshable-materialized-view). Contains all refreshable materialized views, regardless of whether there's a refresh in progress or not.
+
+
+Columns:
+
+- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database the table is in.
+- `view` ([String](../../sql-reference/data-types/string.md)) — Table name.
+- `status` ([String](../../sql-reference/data-types/string.md)) — Current state of the refresh.
+- `last_refresh_result` ([String](../../sql-reference/data-types/string.md)) — Outcome of the latest refresh attempt.
+- `last_refresh_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Time of the last refresh attempt. `NULL` if no refresh attempts happened since server startup or table creation.
+- `last_success_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Time of the last successful refresh. `NULL` if no successful refreshes happened since server startup or table creation.
+- `duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md)) — How long the last refresh attempt took.
+- `next_refresh_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Time at which the next refresh is scheduled to start.
+- `remaining_dependencies` ([Array(String)](../../sql-reference/data-types/array.md)) — If the view has [refresh dependencies](../../sql-reference/statements/create/view.md#refresh-dependencies), this array contains the subset of those dependencies that are not satisfied for the current refresh yet. If `status = 'WaitingForDependencies'`, a refresh is ready to start as soon as these dependencies are fulfilled.
+- `exception` ([String](../../sql-reference/data-types/string.md)) — if `last_refresh_result = 'Exception'`, i.e. the last refresh attempt failed, this column contains the corresponding error message and stack trace.
+- `refresh_count` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of successful refreshes since last server restart or table creation.
+- `progress` ([Float64](../../sql-reference/data-types/float.md)) — Progress of the current refresh, between 0 and 1.
+- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of rows read by the current refresh so far.
+- `total_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Estimated total number of rows that need to be read by the current refresh.
+
+(There are additional columns related to current refresh progress, but they are currently unreliable.)
+
+**Example**
+
+```sql
+SELECT
+ database,
+ view,
+ status,
+ last_refresh_result,
+ last_refresh_time,
+ next_refresh_time
+FROM system.view_refreshes
+
+┌─database─┬─view───────────────────────┬─status────┬─last_refresh_result─┬───last_refresh_time─┬───next_refresh_time─┐
+│ default │ hello_documentation_reader │ Scheduled │ Finished │ 2023-12-01 01:24:00 │ 2023-12-01 01:25:00 │
+└──────────┴────────────────────────────┴───────────┴─────────────────────┴─────────────────────┴─────────────────────┘
+```
diff --git a/docs/en/operations/utilities/clickhouse-format.md b/docs/en/operations/utilities/clickhouse-format.md
index 101310cc65e..3e4295598aa 100644
--- a/docs/en/operations/utilities/clickhouse-format.md
+++ b/docs/en/operations/utilities/clickhouse-format.md
@@ -27,7 +27,7 @@ $ clickhouse-format --query "select number from numbers(10) where number%2 order
Result:
-```sql
+```bash
SELECT number
FROM numbers(10)
WHERE number % 2
@@ -49,22 +49,20 @@ SELECT sum(number) FROM numbers(5)
3. Multiqueries:
```bash
-$ clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);"
+$ clickhouse-format -n <<< "SELECT min(number) FROM numbers(5); SELECT max(number) FROM numbers(5);"
```
Result:
-```sql
-SELECT *
-FROM
-(
- SELECT 1 AS x
- UNION ALL
- SELECT 1
- UNION DISTINCT
- SELECT 3
-)
+```
+SELECT min(number)
+FROM numbers(5)
;
+
+SELECT max(number)
+FROM numbers(5)
+;
+
```
4. Obfuscating:
@@ -75,7 +73,7 @@ $ clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWE
Result:
-```sql
+```
SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END;
```
@@ -87,7 +85,7 @@ $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWE
Result:
-```sql
+```
SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END;
```
@@ -99,7 +97,7 @@ $ clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELE
Result:
-```sql
+```
SELECT * \
FROM \
( \
diff --git a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md
index e21dad5b2f5..62edc221858 100644
--- a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md
+++ b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md
@@ -1,62 +1,64 @@
----
-slug: /en/sql-reference/aggregate-functions/reference/sparkbar
-sidebar_position: 311
-sidebar_label: sparkbar
----
-
-# sparkbar
-
-The function plots a frequency histogram for values `x` and the repetition rate `y` of these values over the interval `[min_x, max_x]`.
-Repetitions for all `x` falling into the same bucket are averaged, so data should be pre-aggregated.
-Negative repetitions are ignored.
-
-If no interval is specified, then the minimum `x` is used as the interval start, and the maximum `x` — as the interval end.
-Otherwise, values outside the interval are ignored.
-
-**Syntax**
-
-``` sql
-sparkbar(buckets[, min_x, max_x])(x, y)
-```
-
-**Parameters**
-
-- `buckets` — The number of segments. Type: [Integer](../../../sql-reference/data-types/int-uint.md).
-- `min_x` — The interval start. Optional parameter.
-- `max_x` — The interval end. Optional parameter.
-
-**Arguments**
-
-- `x` — The field with values.
-- `y` — The field with the frequency of values.
-
-**Returned value**
-
-- The frequency histogram.
-
-**Example**
-
-Query:
-
-``` sql
-CREATE TABLE spark_bar_data (`value` Int64, `event_date` Date) ENGINE = MergeTree ORDER BY event_date;
-
-INSERT INTO spark_bar_data VALUES (1,'2020-01-01'), (3,'2020-01-02'), (4,'2020-01-02'), (-3,'2020-01-02'), (5,'2020-01-03'), (2,'2020-01-04'), (3,'2020-01-05'), (7,'2020-01-06'), (6,'2020-01-07'), (8,'2020-01-08'), (2,'2020-01-11');
-
-SELECT sparkbar(9)(event_date,cnt) FROM (SELECT sum(value) as cnt, event_date FROM spark_bar_data GROUP BY event_date);
-
-SELECT sparkbar(9, toDate('2020-01-01'), toDate('2020-01-10'))(event_date,cnt) FROM (SELECT sum(value) as cnt, event_date FROM spark_bar_data GROUP BY event_date);
-```
-
-Result:
-
-``` text
-┌─sparkbar(9)(event_date, cnt)─┐
-│ ▂▅▂▃▆█ ▂ │
-└──────────────────────────────┘
-
-┌─sparkbar(9, toDate('2020-01-01'), toDate('2020-01-10'))(event_date, cnt)─┐
-│ ▂▅▂▃▇▆█ │
-└──────────────────────────────────────────────────────────────────────────┘
-```
-
+---
+slug: /en/sql-reference/aggregate-functions/reference/sparkbar
+sidebar_position: 311
+sidebar_label: sparkbar
+---
+
+# sparkbar
+
+The function plots a frequency histogram for values `x` and the repetition rate `y` of these values over the interval `[min_x, max_x]`.
+Repetitions for all `x` falling into the same bucket are averaged, so data should be pre-aggregated.
+Negative repetitions are ignored.
+
+If no interval is specified, then the minimum `x` is used as the interval start, and the maximum `x` — as the interval end.
+Otherwise, values outside the interval are ignored.
+
+**Syntax**
+
+``` sql
+sparkbar(buckets[, min_x, max_x])(x, y)
+```
+
+**Parameters**
+
+- `buckets` — The number of segments. Type: [Integer](../../../sql-reference/data-types/int-uint.md).
+- `min_x` — The interval start. Optional parameter.
+- `max_x` — The interval end. Optional parameter.
+
+**Arguments**
+
+- `x` — The field with values.
+- `y` — The field with the frequency of values.
+
+**Returned value**
+
+- The frequency histogram.
+
+**Example**
+
+Query:
+
+``` sql
+CREATE TABLE spark_bar_data (`value` Int64, `event_date` Date) ENGINE = MergeTree ORDER BY event_date;
+
+INSERT INTO spark_bar_data VALUES (1,'2020-01-01'), (3,'2020-01-02'), (4,'2020-01-02'), (-3,'2020-01-02'), (5,'2020-01-03'), (2,'2020-01-04'), (3,'2020-01-05'), (7,'2020-01-06'), (6,'2020-01-07'), (8,'2020-01-08'), (2,'2020-01-11');
+
+SELECT sparkbar(9)(event_date,cnt) FROM (SELECT sum(value) as cnt, event_date FROM spark_bar_data GROUP BY event_date);
+
+SELECT sparkbar(9, toDate('2020-01-01'), toDate('2020-01-10'))(event_date,cnt) FROM (SELECT sum(value) as cnt, event_date FROM spark_bar_data GROUP BY event_date);
+```
+
+Result:
+
+``` text
+┌─sparkbar(9)(event_date, cnt)─┐
+│ ▂▅▂▃▆█ ▂ │
+└──────────────────────────────┘
+
+┌─sparkbar(9, toDate('2020-01-01'), toDate('2020-01-10'))(event_date, cnt)─┐
+│ ▂▅▂▃▇▆█ │
+└──────────────────────────────────────────────────────────────────────────┘
+```
+
+The alias for this function is sparkBar.
+
diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md
index 0261589b968..5622097537e 100644
--- a/docs/en/sql-reference/functions/date-time-functions.md
+++ b/docs/en/sql-reference/functions/date-time-functions.md
@@ -1483,7 +1483,9 @@ For mode values with a meaning of “with 4 or more days this year,” weeks are
- Otherwise, it is the last week of the previous year, and the next week is week 1.
-For mode values with a meaning of “contains January 1”, the week contains January 1 is week 1. It does not matter how many days in the new year the week contained, even if it contained only one day.
+For mode values with a meaning of “contains January 1”, the week contains January 1 is week 1.
+It does not matter how many days in the new year the week contained, even if it contained only one day.
+I.e. if the last week of December contains January 1 of the next year, it will be week 1 of the next year.
**Syntax**
diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md
index a23849c13aa..2c6a468af0e 100644
--- a/docs/en/sql-reference/functions/hash-functions.md
+++ b/docs/en/sql-reference/functions/hash-functions.md
@@ -1779,7 +1779,9 @@ Result:
## sqid
-Transforms numbers into YouTube-like short URL hash called [Sqid](https://sqids.org/).
+Transforms numbers into a [Sqid](https://sqids.org/) which is a YouTube-like ID string.
+The output alphabet is `abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789`.
+Do not use this function for hashing - the generated IDs can be decoded back into numbers.
**Syntax**
diff --git a/docs/en/sql-reference/functions/ip-address-functions.md b/docs/en/sql-reference/functions/ip-address-functions.md
index 33c788a632e..be20e02d77e 100644
--- a/docs/en/sql-reference/functions/ip-address-functions.md
+++ b/docs/en/sql-reference/functions/ip-address-functions.md
@@ -501,41 +501,3 @@ Result:
│ 0 │
└────────────────────────────────────────────────────────────────────┘
```
-
-## reverseDNSQuery
-
-Performs a reverse DNS query to get the PTR records associated with the IP address.
-
-**Syntax**
-
-``` sql
-reverseDNSQuery(address)
-```
-
-This function performs reverse DNS resolutions on both IPv4 and IPv6.
-
-**Arguments**
-
-- `address` — An IPv4 or IPv6 address. [String](../../sql-reference/data-types/string.md).
-
-**Returned value**
-
-- Associated domains (PTR records).
-
-Type: Type: [Array(String)](../../sql-reference/data-types/array.md).
-
-**Example**
-
-Query:
-
-``` sql
-SELECT reverseDNSQuery('192.168.0.2');
-```
-
-Result:
-
-``` text
-┌─reverseDNSQuery('192.168.0.2')────────────┐
-│ ['test2.example.com','test3.example.com'] │
-└───────────────────────────────────────────┘
-```
diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md
index 84839c2489c..3ede66cf316 100644
--- a/docs/en/sql-reference/functions/rounding-functions.md
+++ b/docs/en/sql-reference/functions/rounding-functions.md
@@ -53,7 +53,7 @@ The rounded number of the same type as the input number.
**Example of use with Float**
``` sql
-SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3
+SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3;
```
``` text
@@ -67,7 +67,22 @@ SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3
**Example of use with Decimal**
``` sql
-SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3
+SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3;
+```
+
+``` text
+┌───x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐
+│ 0 │ 0 │
+│ 0.5 │ 1 │
+│ 1 │ 1 │
+└─────┴──────────────────────────────────────────────────┘
+```
+
+If you want to keep the trailing zeros, you need to enable `output_format_decimal_trailing_zeros`
+
+``` sql
+SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3 settings output_format_decimal_trailing_zeros=1;
+
```
``` text
diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md
index 5bad7d53e62..3461ab28bf9 100644
--- a/docs/en/sql-reference/functions/string-functions.md
+++ b/docs/en/sql-reference/functions/string-functions.md
@@ -1385,8 +1385,8 @@ Result:
## punycodeEncode
-Returns the [Punycode](https://en.wikipedia.org/wiki/Punycode) of a string.
-The string must be UTF8-encoded, otherwise results are undefined.
+Returns the [Punycode](https://en.wikipedia.org/wiki/Punycode) representation of a string.
+The string must be UTF8-encoded, otherwise the behavior is undefined.
**Syntax**
@@ -1419,6 +1419,7 @@ Result:
## punycodeDecode
Returns the UTF8-encoded plaintext of a [Punycode](https://en.wikipedia.org/wiki/Punycode)-encoded string.
+If no valid Punycode-encoded string is given, an exception is thrown.
**Syntax**
@@ -1443,11 +1444,87 @@ select punycodeDecode('Mnchen-3ya');
Result:
```result
-┌─punycodeEncode('Mnchen-3ya')─┐
+┌─punycodeDecode('Mnchen-3ya')─┐
│ München │
└──────────────────────────────┘
```
+## tryPunycodeDecode
+
+Like `punycodeDecode` but returns an empty string if no valid Punycode-encoded string is given.
+
+## idnaEncode
+
+Returns the the ASCII representation (ToASCII algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism.
+The input string must be UTF-encoded and translatable to an ASCII string, otherwise an exception is thrown.
+Note: No percent decoding or trimming of tabs, spaces or control characters is performed.
+
+**Syntax**
+
+```sql
+idnaEncode(val)
+```
+
+**Arguments**
+
+- `val` - Input value. [String](../data-types/string.md)
+
+**Returned value**
+
+- A ASCII representation according to the IDNA mechanism of the input value. [String](../data-types/string.md)
+
+**Example**
+
+``` sql
+select idnaEncode('straße.münchen.de');
+```
+
+Result:
+
+```result
+┌─idnaEncode('straße.münchen.de')─────┐
+│ xn--strae-oqa.xn--mnchen-3ya.de │
+└─────────────────────────────────────┘
+```
+
+## tryIdnaEncode
+
+Like `idnaEncode` but returns an empty string in case of an error instead of throwing an exception.
+
+## idnaDecode
+
+Returns the the Unicode (UTF-8) representation (ToUnicode algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism.
+In case of an error (e.g. because the input is invalid), the input string is returned.
+Note that repeated application of `idnaEncode()` and `idnaDecode()` does not necessarily return the original string due to case normalization.
+
+**Syntax**
+
+```sql
+idnaDecode(val)
+```
+
+**Arguments**
+
+- `val` - Input value. [String](../data-types/string.md)
+
+**Returned value**
+
+- A Unicode (UTF-8) representation according to the IDNA mechanism of the input value. [String](../data-types/string.md)
+
+**Example**
+
+``` sql
+select idnaDecode('xn--strae-oqa.xn--mnchen-3ya.de');
+```
+
+Result:
+
+```result
+┌─idnaDecode('xn--strae-oqa.xn--mnchen-3ya.de')─┐
+│ straße.münchen.de │
+└───────────────────────────────────────────────┘
+```
+
## byteHammingDistance
Calculates the [hamming distance](https://en.wikipedia.org/wiki/Hamming_distance) between two byte strings.
diff --git a/docs/en/sql-reference/statements/alter/view.md b/docs/en/sql-reference/statements/alter/view.md
index 5c5bf0355f6..517e64e3e5b 100644
--- a/docs/en/sql-reference/statements/alter/view.md
+++ b/docs/en/sql-reference/statements/alter/view.md
@@ -6,28 +6,28 @@ sidebar_label: VIEW
# ALTER TABLE … MODIFY QUERY Statement
-You can modify `SELECT` query that was specified when a [materialized view](../create/view.md#materialized) was created with the `ALTER TABLE … MODIFY QUERY` statement without interrupting ingestion process.
+You can modify `SELECT` query that was specified when a [materialized view](../create/view.md#materialized) was created with the `ALTER TABLE … MODIFY QUERY` statement without interrupting ingestion process.
-The `allow_experimental_alter_materialized_view_structure` setting must be enabled.
+The `allow_experimental_alter_materialized_view_structure` setting must be enabled.
This command is created to change materialized view created with `TO [db.]name` clause. It does not change the structure of the underling storage table and it does not change the columns' definition of the materialized view, because of this the application of this command is very limited for materialized views are created without `TO [db.]name` clause.
**Example with TO table**
```sql
-CREATE TABLE events (ts DateTime, event_type String)
+CREATE TABLE events (ts DateTime, event_type String)
ENGINE = MergeTree ORDER BY (event_type, ts);
-CREATE TABLE events_by_day (ts DateTime, event_type String, events_cnt UInt64)
+CREATE TABLE events_by_day (ts DateTime, event_type String, events_cnt UInt64)
ENGINE = SummingMergeTree ORDER BY (event_type, ts);
-CREATE MATERIALIZED VIEW mv TO events_by_day AS
+CREATE MATERIALIZED VIEW mv TO events_by_day AS
SELECT toStartOfDay(ts) ts, event_type, count() events_cnt
FROM events
-GROUP BY ts, event_type;
+GROUP BY ts, event_type;
-INSERT INTO events
-SELECT Date '2020-01-01' + interval number * 900 second,
+INSERT INTO events
+SELECT Date '2020-01-01' + interval number * 900 second,
['imp', 'click'][number%2+1]
FROM numbers(100);
@@ -43,23 +43,23 @@ ORDER BY ts, event_type;
│ 2020-01-02 00:00:00 │ imp │ 2 │
└─────────────────────┴────────────┴─────────────────┘
--- Let's add the new measurment `cost`
+-- Let's add the new measurment `cost`
-- and the new dimension `browser`.
-ALTER TABLE events
+ALTER TABLE events
ADD COLUMN browser String,
ADD COLUMN cost Float64;
-- Column do not have to match in a materialized view and TO
-- (destination table), so the next alter does not break insertion.
-ALTER TABLE events_by_day
+ALTER TABLE events_by_day
ADD COLUMN cost Float64,
ADD COLUMN browser String after event_type,
MODIFY ORDER BY (event_type, ts, browser);
-INSERT INTO events
-SELECT Date '2020-01-02' + interval number * 900 second,
+INSERT INTO events
+SELECT Date '2020-01-02' + interval number * 900 second,
['imp', 'click'][number%2+1],
['firefox', 'safary', 'chrome'][number%3+1],
10/(number+1)%33
@@ -82,16 +82,16 @@ ORDER BY ts, event_type;
└─────────────────────┴────────────┴─────────┴────────────┴──────┘
SET allow_experimental_alter_materialized_view_structure=1;
-
-ALTER TABLE mv MODIFY QUERY
+
+ALTER TABLE mv MODIFY QUERY
SELECT toStartOfDay(ts) ts, event_type, browser,
count() events_cnt,
sum(cost) cost
FROM events
GROUP BY ts, event_type, browser;
-INSERT INTO events
-SELECT Date '2020-01-03' + interval number * 900 second,
+INSERT INTO events
+SELECT Date '2020-01-03' + interval number * 900 second,
['imp', 'click'][number%2+1],
['firefox', 'safary', 'chrome'][number%3+1],
10/(number+1)%33
@@ -138,7 +138,7 @@ PRIMARY KEY (event_type, ts)
ORDER BY (event_type, ts, browser)
SETTINGS index_granularity = 8192
--- !!! The columns' definition is unchanged but it does not matter, we are not quering
+-- !!! The columns' definition is unchanged but it does not matter, we are not quering
-- MATERIALIZED VIEW, we are quering TO (storage) table.
-- SELECT section is updated.
@@ -169,7 +169,7 @@ The application is very limited because you can only change the `SELECT` section
```sql
CREATE TABLE src_table (`a` UInt32) ENGINE = MergeTree ORDER BY a;
-CREATE MATERIALIZED VIEW mv (`a` UInt32) ENGINE = MergeTree ORDER BY a AS SELECT a FROM src_table;
+CREATE MATERIALIZED VIEW mv (`a` UInt32) ENGINE = MergeTree ORDER BY a AS SELECT a FROM src_table;
INSERT INTO src_table (a) VALUES (1), (2);
SELECT * FROM mv;
```
@@ -199,3 +199,7 @@ SELECT * FROM mv;
## ALTER LIVE VIEW Statement
`ALTER LIVE VIEW ... REFRESH` statement refreshes a [Live view](../create/view.md#live-view). See [Force Live View Refresh](../create/view.md#live-view-alter-refresh).
+
+## ALTER TABLE … MODIFY REFRESH Statement
+
+`ALTER TABLE ... MODIFY REFRESH` statement changes refresh parameters of a [Refreshable Materialized View](../create/view.md#refreshable-materialized-view). See [Changing Refresh Parameters](../create/view.md#changing-refresh-parameters).
diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md
index 56828745048..f6158acd9a4 100644
--- a/docs/en/sql-reference/statements/create/view.md
+++ b/docs/en/sql-reference/statements/create/view.md
@@ -37,6 +37,7 @@ SELECT a, b, c FROM (SELECT ...)
```
## Parameterized View
+
Parametrized views are similar to normal views, but can be created with parameters which are not resolved immediately. These views can be used with table functions, which specify the name of the view as function name and the parameter values as its arguments.
``` sql
@@ -66,7 +67,7 @@ When creating a materialized view with `TO [db].[table]`, you can't also use `PO
A materialized view is implemented as follows: when inserting data to the table specified in `SELECT`, part of the inserted data is converted by this `SELECT` query, and the result is inserted in the view.
-:::note
+:::note
Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in the `SELECT` query result, ClickHouse uses a default value, even if the column is not [Nullable](../../data-types/nullable.md). A safe practice would be to add aliases for every column when using Materialized views.
Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view.
@@ -96,9 +97,116 @@ This feature is deprecated and will be removed in the future.
For your convenience, the old documentation is located [here](https://pastila.nl/?00f32652/fdf07272a7b54bda7e13b919264e449f.md)
+## Refreshable Materialized View {#refreshable-materialized-view}
+
+```sql
+CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name
+REFRESH EVERY|AFTER interval [OFFSET interval]
+RANDOMIZE FOR interval
+DEPENDS ON [db.]name [, [db.]name [, ...]]
+[TO[db.]name] [(columns)] [ENGINE = engine] [EMPTY]
+AS SELECT ...
+```
+where `interval` is a sequence of simple intervals:
+```sql
+number SECOND|MINUTE|HOUR|DAY|WEEK|MONTH|YEAR
+```
+
+Periodically runs the corresponding query and stores its result in a table, atomically replacing the table's previous contents.
+
+Differences from regular non-refreshable materialized views:
+ * No insert trigger. I.e. when new data is inserted into the table specified in SELECT, it's *not* automatically pushed to the refreshable materialized view. The periodic refresh runs the entire query and replaces the entire table.
+ * No restrictions on the SELECT query. Table functions (e.g. `url()`), views, UNION, JOIN, are all allowed.
+
+:::note
+Refreshable materialized views are a work in progress. Setting `allow_experimental_refreshable_materialized_view = 1` is required for creating one. Current limitations:
+ * not compatible with Replicated database or table engines,
+ * require [Atomic database engine](../../../engines/database-engines/atomic.md),
+ * no retries for failed refresh - we just skip to the next scheduled refresh time,
+ * no limit on number of concurrent refreshes.
+:::
+
+### Refresh Schedule
+
+Example refresh schedules:
+```sql
+REFRESH EVERY 1 DAY -- every day, at midnight (UTC)
+REFRESH EVERY 1 MONTH -- on 1st day of every month, at midnight
+REFRESH EVERY 1 MONTH OFFSET 5 DAY 2 HOUR -- on 6th day of every month, at 2:00 am
+REFRESH EVERY 2 WEEK OFFSET 5 DAY 15 HOUR 10 MINUTE -- every other Saturday, at 3:10 pm
+REFRESH EVERY 30 MINUTE -- at 00:00, 00:30, 01:00, 01:30, etc
+REFRESH AFTER 30 MINUTE -- 30 minutes after the previous refresh completes, no alignment with time of day
+-- REFRESH AFTER 1 HOUR OFFSET 1 MINUTE -- syntax errror, OFFSET is not allowed with AFTER
+```
+
+`RANDOMIZE FOR` randomly adjusts the time of each refresh, e.g.:
+```sql
+REFRESH EVERY 1 DAY OFFSET 2 HOUR RANDOMIZE FOR 1 HOUR -- every day at random time between 01:30 and 02:30
+```
+
+At most one refresh may be running at a time, for a given view. E.g. if a view with `REFRESH EVERY 1 MINUTE` takes 2 minutes to refresh, it'll just be refreshing every 2 minutes. If it then becomes faster and starts refreshing in 10 seconds, it'll go back to refreshing every minute. (In particular, it won't refresh every 10 seconds to catch up with a backlog of missed refreshes - there's no such backlog.)
+
+Additionally, a refresh is started immediately after the materialized view is created, unless `EMPTY` is specified in the `CREATE` query. If `EMPTY` is specified, the first refresh happens according to schedule.
+
+### Dependencies {#refresh-dependencies}
+
+`DEPENDS ON` synchronizes refreshes of different tables. By way of example, suppose there's a chain of two refreshable materialized views:
+```sql
+CREATE MATERIALIZED VIEW source REFRESH EVERY 1 DAY AS SELECT * FROM url(...)
+CREATE MATERIALIZED VIEW destination REFRESH EVERY 1 DAY AS SELECT ... FROM source
+```
+Without `DEPENDS ON`, both views will start a refresh at midnight, and `destination` typically will see yesterday's data in `source`. If we add dependency:
+```
+CREATE MATERIALIZED VIEW destination REFRESH EVERY 1 DAY DEPENDS ON source AS SELECT ... FROM source
+```
+then `destination`'s refresh will start only after `source`'s refresh finished for that day, so `destination` will be based on fresh data.
+
+Alternatively, the same result can be achieved with:
+```
+CREATE MATERIALIZED VIEW destination REFRESH AFTER 1 HOUR DEPENDS ON source AS SELECT ... FROM source
+```
+where `1 HOUR` can be any duration less than `source`'s refresh period. The dependent table won't be refreshed more frequently than any of its dependencies. This is a valid way to set up a chain of refreshable views without specifying the real refresh period more than once.
+
+A few more examples:
+ * `REFRESH EVERY 1 DAY OFFSET 10 MINUTE` (`destination`) depends on `REFRESH EVERY 1 DAY` (`source`)
+ If `source` refresh takes more than 10 minutes, `destination` will wait for it.
+ * `REFRESH EVERY 1 DAY OFFSET 1 HOUR` depends on `REFRESH EVERY 1 DAY OFFSET 23 HOUR`
+ Similar to the above, even though the corresponding refreshes happen on different calendar days.
+ `destination`'s refresh on day X+1 will wait for `source`'s refresh on day X (if it takes more than 2 hours).
+ * `REFRESH EVERY 2 HOUR` depends on `REFRESH EVERY 1 HOUR`
+ The 2 HOUR refresh happens after the 1 HOUR refresh for every other hour, e.g. after the midnight
+ refresh, then after the 2am refresh, etc.
+ * `REFRESH EVERY 1 MINUTE` depends on `REFRESH EVERY 2 HOUR`
+ `REFRESH AFTER 1 MINUTE` depends on `REFRESH EVERY 2 HOUR`
+ `REFRESH AFTER 1 MINUTE` depends on `REFRESH AFTER 2 HOUR`
+ `destination` is refreshed once after every `source` refresh, i.e. every 2 hours. The `1 MINUTE` is effectively ignored.
+ * `REFRESH AFTER 1 HOUR` depends on `REFRESH AFTER 1 HOUR`
+ Currently this is not recommended.
+
+:::note
+`DEPENDS ON` only works between refreshable materialized views. Listing a regular table in the `DEPENDS ON` list will prevent the view from ever refreshing (dependencies can be removed with `ALTER`, see below).
+:::
+
+### Changing Refresh Parameters {#changing-refresh-parameters}
+
+To change refresh parameters:
+```
+ALTER TABLE [db.]name MODIFY REFRESH EVERY|AFTER ... [RANDOMIZE FOR ...] [DEPENDS ON ...]
+```
+
+:::note
+This replaces refresh schedule *and* dependencies. If the table had a `DEPENDS ON`, doing a `MODIFY REFRESH` without `DEPENDS ON` will remove the dependencies.
+:::
+
+### Other operations
+
+The status of all refreshable materialized views is available in table [`system.view_refreshes`](../../../operations/system-tables/view_refreshes.md). In particular, it contains refresh progress (if running), last and next refresh time, exception message if a refresh failed.
+
+To manually stop, start, trigger, or cancel refreshes use [`SYSTEM STOP|START|REFRESH|CANCEL VIEW`](../system.md#refreshable-materialized-views).
+
## Window View [Experimental]
-:::info
+:::info
This is an experimental feature that may change in backwards-incompatible ways in the future releases. Enable usage of window views and `WATCH` query using [allow_experimental_window_view](../../../operations/settings/settings.md#allow-experimental-window-view) setting. Input the command `set allow_experimental_window_view = 1`.
:::
diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md
index 695801983b7..0fdbbeac235 100644
--- a/docs/en/sql-reference/statements/system.md
+++ b/docs/en/sql-reference/statements/system.md
@@ -449,7 +449,7 @@ SYSTEM SYNC FILE CACHE [ON CLUSTER cluster_name]
```
-### SYSTEM STOP LISTEN
+## SYSTEM STOP LISTEN
Closes the socket and gracefully terminates the existing connections to the server on the specified port with the specified protocol.
@@ -464,7 +464,7 @@ SYSTEM STOP LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QU
- If `QUERIES DEFAULT [EXCEPT .. [,..]]` modifier is specified, all default protocols are stopped, unless specified with `EXCEPT` clause.
- If `QUERIES CUSTOM [EXCEPT .. [,..]]` modifier is specified, all custom protocols are stopped, unless specified with `EXCEPT` clause.
-### SYSTEM START LISTEN
+## SYSTEM START LISTEN
Allows new connections to be established on the specified protocols.
@@ -473,3 +473,47 @@ However, if the server on the specified port and protocol was not stopped using
```sql
SYSTEM START LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QUERIES CUSTOM | TCP | TCP WITH PROXY | TCP SECURE | HTTP | HTTPS | MYSQL | GRPC | POSTGRESQL | PROMETHEUS | CUSTOM 'protocol']
```
+
+## Managing Refreshable Materialized Views {#refreshable-materialized-views}
+
+Commands to control background tasks performed by [Refreshable Materialized Views](../../sql-reference/statements/create/view.md#refreshable-materialized-view)
+
+Keep an eye on [`system.view_refreshes`](../../operations/system-tables/view_refreshes.md) while using them.
+
+### SYSTEM REFRESH VIEW
+
+Trigger an immediate out-of-schedule refresh of a given view.
+
+```sql
+SYSTEM REFRESH VIEW [db.]name
+```
+
+### SYSTEM STOP VIEW, SYSTEM STOP VIEWS
+
+Disable periodic refreshing of the given view or all refreshable views. If a refresh is in progress, cancel it too.
+
+```sql
+SYSTEM STOP VIEW [db.]name
+```
+```sql
+SYSTEM STOP VIEWS
+```
+
+### SYSTEM START VIEW, SYSTEM START VIEWS
+
+Enable periodic refreshing for the given view or all refreshable views. No immediate refresh is triggered.
+
+```sql
+SYSTEM START VIEW [db.]name
+```
+```sql
+SYSTEM START VIEWS
+```
+
+### SYSTEM CANCEL VIEW
+
+If there's a refresh in progress for the given view, interrupt and cancel it. Otherwise do nothing.
+
+```sql
+SYSTEM CANCEL VIEW [db.]name
+```
diff --git a/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md
index c17e7982b98..e8089b2c42b 100644
--- a/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md
+++ b/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md
@@ -86,6 +86,59 @@ SELECT * FROM mySecondReplacingMT FINAL;
│ 1 │ first │ 2020-01-01 01:01:01 │
└─────┴─────────┴─────────────────────┘
```
+### is_deleted
+
+`is_deleted` — Имя столбца, который используется во время слияния для обозначения того, нужно ли отображать строку или она подлежит удалению; `1` - для удаления строки, `0` - для отображения строки.
+
+ Тип данных столбца — `UInt8`.
+
+:::note
+`is_deleted` может быть использован, если `ver` используется.
+
+Строка удаляется в следующих случаях:
+
+ - при использовании инструкции `OPTIMIZE ... FINAL CLEANUP`
+ - при использовании инструкции `OPTIMIZE ... FINAL`
+ - параметр движка `clean_deleted_rows` установлен в значение `Always` (по умолчанию - `Never`)
+ - есть новые версии строки
+
+Не рекомендуется выполнять `FINAL CLEANUP` или использовать параметр движка `clean_deleted_rows` со значением `Always`, это может привести к неожиданным результатам, например удаленные строки могут вновь появиться.
+
+Вне зависимости от производимых изменений над данными, версия должна увеличиваться. Если у двух строк одна и та же версия, то остается только последняя вставленная строка.
+:::
+
+Пример:
+
+```sql
+-- with ver and is_deleted
+CREATE OR REPLACE TABLE myThirdReplacingMT
+(
+ `key` Int64,
+ `someCol` String,
+ `eventTime` DateTime,
+ `is_deleted` UInt8
+)
+ENGINE = ReplacingMergeTree(eventTime, is_deleted)
+ORDER BY key;
+
+INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 01:01:01', 0);
+INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 01:01:01', 1);
+
+select * from myThirdReplacingMT final;
+
+0 rows in set. Elapsed: 0.003 sec.
+
+-- delete rows with is_deleted
+OPTIMIZE TABLE myThirdReplacingMT FINAL CLEANUP;
+
+INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 00:00:00', 0);
+
+select * from myThirdReplacingMT final;
+
+┌─key─┬─someCol─┬───────────eventTime─┬─is_deleted─┐
+│ 1 │ first │ 2020-01-01 00:00:00 │ 0 │
+└─────┴─────────┴─────────────────────┴────────────┘
+```
## Секции запроса
diff --git a/docs/ru/getting-started/example-datasets/github-events.mdx b/docs/ru/getting-started/example-datasets/github-events.mdx
index c6e58a9f5a4..84f445074af 100644
--- a/docs/ru/getting-started/example-datasets/github-events.mdx
+++ b/docs/ru/getting-started/example-datasets/github-events.mdx
@@ -1,9 +1,9 @@
---
slug: /ru/getting-started/example-datasets/github-events
sidebar_label: GitHub Events
-title: "GitHub Events Dataset"
+title: "Набор данных о событиях на GitHub"
---
-import Content from '@site/docs/en/getting-started/example-datasets/github-events.md';
+Набор данных о событиях на GitHub с 2011 года по 6 декабря 2020 года содержит 3,1 млрд записей. Объём исходных данных — 75 ГБ, для загрузки в Clickhouse потребуется около 200 ГБ свободного пространства хранения (при использовании метода сжатия lz4).
-
+Полное описание набора, инструкции по загрузке и запросы к нему опубликованы на https://ghe.clickhouse.tech/
diff --git a/docs/ru/operations/utilities/clickhouse-format.md b/docs/ru/operations/utilities/clickhouse-format.md
index af66930b368..9c4b7304940 100644
--- a/docs/ru/operations/utilities/clickhouse-format.md
+++ b/docs/ru/operations/utilities/clickhouse-format.md
@@ -1,115 +1,115 @@
----
+---
slug: /ru/operations/utilities/clickhouse-format
-sidebar_position: 65
-sidebar_label: clickhouse-format
----
-
-# clickhouse-format {#clickhouse-format}
-
-Позволяет форматировать входящие запросы.
-
-Ключи:
-
-- `--help` или`-h` — выводит описание ключей.
-- `--query` — форматирует запрос любой длины и сложности.
-- `--hilite` — добавляет подсветку синтаксиса с экранированием символов.
-- `--oneline` — форматирование в одну строку.
-- `--quiet` или `-q` — проверяет синтаксис без вывода результата.
-- `--multiquery` or `-n` — поддерживает несколько запросов в одной строке.
-- `--obfuscate` — обфусцирует вместо форматирования.
-- `--seed <строка>` — задает строку, которая определяет результат обфускации.
-- `--backslash` — добавляет обратный слеш в конце каждой строки отформатированного запроса. Удобно использовать если многострочный запрос скопирован из интернета или другого источника и его нужно выполнить из командной строки.
-
-## Примеры {#examples}
-
-1. Форматирование запроса:
-
-```bash
-$ clickhouse-format --query "select number from numbers(10) where number%2 order by number desc;"
-```
-
-Результат:
-
-```text
-SELECT number
-FROM numbers(10)
-WHERE number % 2
-ORDER BY number DESC
-```
-
-2. Подсветка синтаксиса и форматирование в одну строку:
-
-```bash
-$ clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);"
-```
-
-Результат:
-
-```sql
-SELECT sum(number) FROM numbers(5)
-```
-
-3. Несколько запросов в одной строке:
-
-```bash
-$ clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);"
-```
-
-Результат:
-
-```text
-SELECT *
-FROM
-(
- SELECT 1 AS x
- UNION ALL
- SELECT 1
- UNION DISTINCT
- SELECT 3
-)
-;
-```
-
-4. Обфускация:
-
-```bash
-$ clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;"
-```
-
-Результат:
-
-```text
-SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END;
-```
-
-Тот же запрос с другой инициализацией обфускатора:
-
-```bash
-$ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;"
-```
-
-Результат:
-
-```text
-SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END;
-```
-
-5. Добавление обратного слеша:
-
-```bash
-$ clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);"
-```
-
-Результат:
-
-```text
-SELECT * \
-FROM \
-( \
- SELECT 1 AS x \
- UNION ALL \
- SELECT 1 \
- UNION DISTINCT \
- SELECT 3 \
-)
-```
+sidebar_position: 65
+sidebar_label: clickhouse-format
+---
+
+# clickhouse-format {#clickhouse-format}
+
+Позволяет форматировать входящие запросы.
+
+Ключи:
+
+- `--help` или`-h` — выводит описание ключей.
+- `--query` — форматирует запрос любой длины и сложности.
+- `--hilite` — добавляет подсветку синтаксиса с экранированием символов.
+- `--oneline` — форматирование в одну строку.
+- `--quiet` или `-q` — проверяет синтаксис без вывода результата.
+- `--multiquery` or `-n` — поддерживает несколько запросов в одной строке.
+- `--obfuscate` — обфусцирует вместо форматирования.
+- `--seed <строка>` — задает строку, которая определяет результат обфускации.
+- `--backslash` — добавляет обратный слеш в конце каждой строки отформатированного запроса. Удобно использовать если многострочный запрос скопирован из интернета или другого источника и его нужно выполнить из командной строки.
+
+## Примеры {#examples}
+
+1. Форматирование запроса:
+
+```bash
+$ clickhouse-format --query "select number from numbers(10) where number%2 order by number desc;"
+```
+
+Результат:
+
+```text
+SELECT number
+FROM numbers(10)
+WHERE number % 2
+ORDER BY number DESC
+```
+
+2. Подсветка синтаксиса и форматирование в одну строку:
+
+```bash
+$ clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);"
+```
+
+Результат:
+
+```sql
+SELECT sum(number) FROM numbers(5)
+```
+
+3. Несколько запросов в одной строке:
+
+```bash
+$ clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);"
+```
+
+Результат:
+
+```text
+SELECT *
+FROM
+(
+ SELECT 1 AS x
+ UNION ALL
+ SELECT 1
+ UNION DISTINCT
+ SELECT 3
+)
+;
+```
+
+4. Обфускация:
+
+```bash
+$ clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;"
+```
+
+Результат:
+
+```text
+SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END;
+```
+
+Тот же запрос с другой инициализацией обфускатора:
+
+```bash
+$ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;"
+```
+
+Результат:
+
+```text
+SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END;
+```
+
+5. Добавление обратного слеша:
+
+```bash
+$ clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);"
+```
+
+Результат:
+
+```text
+SELECT * \
+FROM \
+( \
+ SELECT 1 AS x \
+ UNION ALL \
+ SELECT 1 \
+ UNION DISTINCT \
+ SELECT 3 \
+)
+```
diff --git a/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md b/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md
index 958a4bd3504..3b36ee04095 100644
--- a/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md
+++ b/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md
@@ -1,62 +1,62 @@
----
-slug: /ru/sql-reference/aggregate-functions/reference/sparkbar
-sidebar_position: 311
-sidebar_label: sparkbar
----
-
-# sparkbar {#sparkbar}
-
-Функция строит гистограмму частот по заданным значениям `x` и частоте повторения этих значений `y` на интервале `[min_x, max_x]`. Повторения для всех `x`, попавших в один бакет, усредняются, поэтому данные должны быть предварительно агрегированы. Отрицательные повторения игнорируются.
-
-Если интервал для построения не указан, то в качестве нижней границы интервала будет взято минимальное значение `x`, а в качестве верхней границы — максимальное значение `x`.
-Значения `x` вне указанного интервала игнорируются.
-
-
-**Синтаксис**
-
-``` sql
-sparkbar(width[, min_x, max_x])(x, y)
-```
-
-**Параметры**
-
-- `width` — Количество столбцов гистограммы. Тип: [Integer](../../../sql-reference/data-types/int-uint.md).
-
-- `min_x` — Начало интервала. Необязательный параметр.
-- `max_x` — Конец интервала. Необязательный параметр.
-
-**Аргументы**
-
-- `x` — Поле со значениями.
-- `y` — Поле с частотой повторения значений.
-
-
-**Возвращаемые значения**
-
-- Гистограмма частот.
-
-**Пример**
-
-Запрос:
-
-``` sql
-CREATE TABLE spark_bar_data (`value` Int64, `event_date` Date) ENGINE = MergeTree ORDER BY event_date;
-
-INSERT INTO spark_bar_data VALUES (1,'2020-01-01'), (3,'2020-01-02'), (4,'2020-01-02'), (-3,'2020-01-02'), (5,'2020-01-03'), (2,'2020-01-04'), (3,'2020-01-05'), (7,'2020-01-06'), (6,'2020-01-07'), (8,'2020-01-08'), (2,'2020-01-11');
-
-SELECT sparkbar(9)(event_date,cnt) FROM (SELECT sum(value) as cnt, event_date FROM spark_bar_data GROUP BY event_date);
-
-SELECT sparkbar(9, toDate('2020-01-01'), toDate('2020-01-10'))(event_date,cnt) FROM (SELECT sum(value) as cnt, event_date FROM spark_bar_data GROUP BY event_date);
-```
-
-Результат:
-
-``` text
-┌─sparkbar(9)(event_date, cnt)─┐
-│ ▂▅▂▃▆█ ▂ │
-└──────────────────────────────┘
-
-┌─sparkbar(9, toDate('2020-01-01'), toDate('2020-01-10'))(event_date, cnt)─┐
-│ ▂▅▂▃▇▆█ │
-└──────────────────────────────────────────────────────────────────────────┘
-```
+---
+slug: /ru/sql-reference/aggregate-functions/reference/sparkbar
+sidebar_position: 311
+sidebar_label: sparkbar
+---
+
+# sparkbar {#sparkbar}
+
+Функция строит гистограмму частот по заданным значениям `x` и частоте повторения этих значений `y` на интервале `[min_x, max_x]`. Повторения для всех `x`, попавших в один бакет, усредняются, поэтому данные должны быть предварительно агрегированы. Отрицательные повторения игнорируются.
+
+Если интервал для построения не указан, то в качестве нижней границы интервала будет взято минимальное значение `x`, а в качестве верхней границы — максимальное значение `x`.
+Значения `x` вне указанного интервала игнорируются.
+
+
+**Синтаксис**
+
+``` sql
+sparkbar(width[, min_x, max_x])(x, y)
+```
+
+**Параметры**
+
+- `width` — Количество столбцов гистограммы. Тип: [Integer](../../../sql-reference/data-types/int-uint.md).
+
+- `min_x` — Начало интервала. Необязательный параметр.
+- `max_x` — Конец интервала. Необязательный параметр.
+
+**Аргументы**
+
+- `x` — Поле со значениями.
+- `y` — Поле с частотой повторения значений.
+
+
+**Возвращаемые значения**
+
+- Гистограмма частот.
+
+**Пример**
+
+Запрос:
+
+``` sql
+CREATE TABLE spark_bar_data (`value` Int64, `event_date` Date) ENGINE = MergeTree ORDER BY event_date;
+
+INSERT INTO spark_bar_data VALUES (1,'2020-01-01'), (3,'2020-01-02'), (4,'2020-01-02'), (-3,'2020-01-02'), (5,'2020-01-03'), (2,'2020-01-04'), (3,'2020-01-05'), (7,'2020-01-06'), (6,'2020-01-07'), (8,'2020-01-08'), (2,'2020-01-11');
+
+SELECT sparkbar(9)(event_date,cnt) FROM (SELECT sum(value) as cnt, event_date FROM spark_bar_data GROUP BY event_date);
+
+SELECT sparkbar(9, toDate('2020-01-01'), toDate('2020-01-10'))(event_date,cnt) FROM (SELECT sum(value) as cnt, event_date FROM spark_bar_data GROUP BY event_date);
+```
+
+Результат:
+
+``` text
+┌─sparkbar(9)(event_date, cnt)─┐
+│ ▂▅▂▃▆█ ▂ │
+└──────────────────────────────┘
+
+┌─sparkbar(9, toDate('2020-01-01'), toDate('2020-01-10'))(event_date, cnt)─┐
+│ ▂▅▂▃▇▆█ │
+└──────────────────────────────────────────────────────────────────────────┘
+```
diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md
index fa5728a097d..cbbb456aa80 100644
--- a/docs/ru/sql-reference/functions/date-time-functions.md
+++ b/docs/ru/sql-reference/functions/date-time-functions.md
@@ -578,7 +578,9 @@ SELECT
- В противном случае это последняя неделя предыдущего года, а следующая неделя - неделя 1.
-Для режимов со значением «содержит 1 января», неделя 1 – это неделя содержащая 1 января. Не имеет значения, сколько дней в новом году содержала неделя, даже если она содержала только один день.
+Для режимов со значением «содержит 1 января», неделя 1 – это неделя, содержащая 1 января.
+Не имеет значения, сколько дней нового года содержит эта неделя, даже если она содержит только один день.
+Так, если последняя неделя декабря содержит 1 января следующего года, то она считается неделей 1 следующего года.
**Пример**
diff --git a/docs/zh/engines/table-engines/mergetree-family/mergetree.md b/docs/zh/engines/table-engines/mergetree-family/mergetree.md
index f9a97bba001..bfa69338657 100644
--- a/docs/zh/engines/table-engines/mergetree-family/mergetree.md
+++ b/docs/zh/engines/table-engines/mergetree-family/mergetree.md
@@ -1,822 +1,822 @@
----
-slug: /zh/engines/table-engines/mergetree-family/mergetree
----
-# MergeTree {#table_engines-mergetree}
-
-Clickhouse 中最强大的表引擎当属 `MergeTree` (合并树)引擎及该系列(`*MergeTree`)中的其他引擎。
-
-`MergeTree` 系列的引擎被设计用于插入极大量的数据到一张表当中。数据可以以数据片段的形式一个接着一个的快速写入,数据片段在后台按照一定的规则进行合并。相比在插入时不断修改(重写)已存储的数据,这种策略会高效很多。
-
-主要特点:
-
-- 存储的数据按主键排序。
-
- 这使得您能够创建一个小型的稀疏索引来加快数据检索。
-
-- 如果指定了 [分区键](custom-partitioning-key.md) 的话,可以使用分区。
-
- 在相同数据集和相同结果集的情况下 ClickHouse 中某些带分区的操作会比普通操作更快。查询中指定了分区键时 ClickHouse 会自动截取分区数据。这也有效增加了查询性能。
-
-- 支持数据副本。
-
- `ReplicatedMergeTree` 系列的表提供了数据副本功能。更多信息,请参阅 [数据副本](replication.md) 一节。
-
-- 支持数据采样。
-
- 需要的话,您可以给表设置一个采样方法。
-
-:::info
-[合并](../special/merge.md#merge) 引擎并不属于 `*MergeTree` 系列。
-:::
-
-## 建表 {#table_engine-mergetree-creating-a-table}
-
-``` sql
-CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
-(
- name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1],
- name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2],
- ...
- INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1,
- INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2
-) ENGINE = MergeTree()
-ORDER BY expr
-[PARTITION BY expr]
-[PRIMARY KEY expr]
-[SAMPLE BY expr]
-[TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'], ...]
-[SETTINGS name=value, ...]
-```
-
-对于以上参数的描述,可参考 [CREATE 语句 的描述](../../../engines/table-engines/mergetree-family/mergetree.md) 。
-
-
-
-**子句**
-
-- `ENGINE` - 引擎名和参数。 `ENGINE = MergeTree()`. `MergeTree` 引擎没有参数。
-
-- `ORDER BY` — 排序键。
-
- 可以是一组列的元组或任意的表达式。 例如: `ORDER BY (CounterID, EventDate)` 。
-
- 如果没有使用 `PRIMARY KEY` 显式指定的主键,ClickHouse 会使用排序键作为主键。
-
- 如果不需要排序,可以使用 `ORDER BY tuple()`. 参考 [选择主键](https://clickhouse.com/docs/en/engines/table-engines/mergetree-family/mergetree/#selecting-the-primary-key)
-
-- `PARTITION BY` — [分区键](custom-partitioning-key.md) ,可选项。
-
- 大多数情况下,不需要使用分区键。即使需要使用,也不需要使用比月更细粒度的分区键。分区不会加快查询(这与 ORDER BY 表达式不同)。永远也别使用过细粒度的分区键。不要使用客户端指定分区标识符或分区字段名称来对数据进行分区(而是将分区字段标识或名称作为 ORDER BY 表达式的第一列来指定分区)。
-
- 要按月分区,可以使用表达式 `toYYYYMM(date_column)` ,这里的 `date_column` 是一个 [Date](../../../engines/table-engines/mergetree-family/mergetree.md) 类型的列。分区名的格式会是 `"YYYYMM"` 。
-
-- `PRIMARY KEY` - 如果要 [选择与排序键不同的主键](#choosing-a-primary-key-that-differs-from-the-sorting-key),在这里指定,可选项。
-
- 默认情况下主键跟排序键(由 `ORDER BY` 子句指定)相同。
- 因此,大部分情况下不需要再专门指定一个 `PRIMARY KEY` 子句。
-
-- `SAMPLE BY` - 用于抽样的表达式,可选项。
-
- 如果要用抽样表达式,主键中必须包含这个表达式。例如:
- `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))` 。
-
-- `TTL` - 指定行存储的持续时间并定义数据片段在硬盘和卷上的移动逻辑的规则列表,可选项。
-
- 表达式中必须存在至少一个 `Date` 或 `DateTime` 类型的列,比如:
-
- `TTL date + INTERVAl 1 DAY`
-
- 规则的类型 `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'`指定了当满足条件(到达指定时间)时所要执行的动作:移除过期的行,还是将数据片段(如果数据片段中的所有行都满足表达式的话)移动到指定的磁盘(`TO DISK 'xxx'`) 或 卷(`TO VOLUME 'xxx'`)。默认的规则是移除(`DELETE`)。可以在列表中指定多个规则,但最多只能有一个`DELETE`的规则。
-
- 更多细节,请查看 [表和列的 TTL](#table_engine-mergetree-ttl)
-
-- `SETTINGS` — 控制 `MergeTree` 行为的额外参数,可选项:
-
- - `index_granularity` — 索引粒度。索引中相邻的『标记』间的数据行数。默认值8192 。参考[数据存储](#mergetree-data-storage)。
- - `index_granularity_bytes` — 索引粒度,以字节为单位,默认值: 10Mb。如果想要仅按数据行数限制索引粒度, 请设置为0(不建议)。
- - `min_index_granularity_bytes` - 允许的最小数据粒度,默认值:1024b。该选项用于防止误操作,添加了一个非常低索引粒度的表。参考[数据存储](#mergetree-data-storage)
- - `enable_mixed_granularity_parts` — 是否启用通过 `index_granularity_bytes` 控制索引粒度的大小。在19.11版本之前, 只有 `index_granularity` 配置能够用于限制索引粒度的大小。当从具有很大的行(几十上百兆字节)的表中查询数据时候,`index_granularity_bytes` 配置能够提升ClickHouse的性能。如果您的表里有很大的行,可以开启这项配置来提升`SELECT` 查询的性能。
- - `use_minimalistic_part_header_in_zookeeper` — ZooKeeper中数据片段存储方式 。如果`use_minimalistic_part_header_in_zookeeper=1` ,ZooKeeper 会存储更少的数据。更多信息参考[服务配置参数]([Server Settings | ClickHouse Documentation](https://clickhouse.com/docs/zh/operations/server-configuration-parameters/settings/))这章中的 [设置描述](../../../operations/server-configuration-parameters/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) 。
- - `min_merge_bytes_to_use_direct_io` — 使用直接 I/O 来操作磁盘的合并操作时要求的最小数据量。合并数据片段时,ClickHouse 会计算要被合并的所有数据的总存储空间。如果大小超过了 `min_merge_bytes_to_use_direct_io` 设置的字节数,则 ClickHouse 将使用直接 I/O 接口(`O_DIRECT` 选项)对磁盘读写。如果设置 `min_merge_bytes_to_use_direct_io = 0` ,则会禁用直接 I/O。默认值:`10 * 1024 * 1024 * 1024` 字节。
-
- - `merge_with_ttl_timeout` — TTL合并频率的最小间隔时间,单位:秒。默认值: 86400 (1 天)。
- - `write_final_mark` — 是否启用在数据片段尾部写入最终索引标记。默认值: 1(不要关闭)。
- - `merge_max_block_size` — 在块中进行合并操作时的最大行数限制。默认值:8192
- - `storage_policy` — 存储策略。 参见 [使用具有多个块的设备进行数据存储](#table_engine-mergetree-multiple-volumes).
- - `min_bytes_for_wide_part`,`min_rows_for_wide_part` 在数据片段中可以使用`Wide`格式进行存储的最小字节数/行数。您可以不设置、只设置一个,或全都设置。参考:[数据存储](#mergetree-data-storage)
- - `max_parts_in_total` - 所有分区中最大块的数量(意义不明)
- - `max_compress_block_size` - 在数据压缩写入表前,未压缩数据块的最大大小。您可以在全局设置中设置该值(参见[max_compress_block_size](https://clickhouse.com/docs/zh/operations/settings/settings/#max-compress-block-size))。建表时指定该值会覆盖全局设置。
- - `min_compress_block_size` - 在数据压缩写入表前,未压缩数据块的最小大小。您可以在全局设置中设置该值(参见[min_compress_block_size](https://clickhouse.com/docs/zh/operations/settings/settings/#min-compress-block-size))。建表时指定该值会覆盖全局设置。
- - `max_partitions_to_read` - 一次查询中可访问的分区最大数。您可以在全局设置中设置该值(参见[max_partitions_to_read](https://clickhouse.com/docs/zh/operations/settings/settings/#max_partitions_to_read))。
-
-**示例配置**
-
-``` sql
-ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192
-```
-
-在这个例子中,我们设置了按月进行分区。
-
-同时我们设置了一个按用户 ID 哈希的抽样表达式。这使得您可以对该表中每个 `CounterID` 和 `EventDate` 的数据伪随机分布。如果您在查询时指定了 [SAMPLE](../../../engines/table-engines/mergetree-family/mergetree.md#select-sample-clause) 子句。 ClickHouse会返回对于用户子集的一个均匀的伪随机数据采样。
-
-`index_granularity` 可省略因为 8192 是默认设置 。
-
-
-已弃用的建表方法
-
-:::attention "注意"
-不要在新版项目中使用该方法,可能的话,请将旧项目切换到上述方法。
-:::
-
-``` sql
-CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
-(
- name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1],
- name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2],
- ...
-) ENGINE [=] MergeTree(date-column [, sampling_expression], (primary, key), index_granularity)
-```
-
-**MergeTree() 参数**
-
-- `date-column` — 类型为 [日期](../../../engines/table-engines/mergetree-family/mergetree.md) 的列名。ClickHouse 会自动依据这个列按月创建分区。分区名格式为 `"YYYYMM"` 。
-- `sampling_expression` — 采样表达式。
-- `(primary, key)` — 主键。类型 — [元组()](../../../engines/table-engines/mergetree-family/mergetree.md)
-- `index_granularity` — 索引粒度。即索引中相邻『标记』间的数据行数。设为 8192 可以适用大部分场景。
-
-**示例**
-
- MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID)), 8192)
-
-对于主要的配置方法,这里 `MergeTree` 引擎跟前面的例子一样,可以以同样的方式配置。
-
-
-## 数据存储 {#mergetree-data-storage}
-
-表由按主键排序的数据片段(DATA PART)组成。
-
-当数据被插入到表中时,会创建多个数据片段并按主键的字典序排序。例如,主键是 `(CounterID, Date)` 时,片段中数据首先按 `CounterID` 排序,具有相同 `CounterID` 的部分按 `Date` 排序。
-
-不同分区的数据会被分成不同的片段,ClickHouse 在后台合并数据片段以便更高效存储。不同分区的数据片段不会进行合并。合并机制并不保证具有相同主键的行全都合并到同一个数据片段中。
-
-数据片段可以以 `Wide` 或 `Compact` 格式存储。在 `Wide` 格式下,每一列都会在文件系统中存储为单独的文件,在 `Compact` 格式下所有列都存储在一个文件中。`Compact` 格式可以提高插入量少插入频率频繁时的性能。
-
-数据存储格式由 `min_bytes_for_wide_part` 和 `min_rows_for_wide_part` 表引擎参数控制。如果数据片段中的字节数或行数少于相应的设置值,数据片段会以 `Compact` 格式存储,否则会以 `Wide` 格式存储。
-
-每个数据片段被逻辑的分割成颗粒(granules)。颗粒是 ClickHouse 中进行数据查询时的最小不可分割数据集。ClickHouse 不会对行或值进行拆分,所以每个颗粒总是包含整数个行。每个颗粒的第一行通过该行的主键值进行标记,
-ClickHouse 会为每个数据片段创建一个索引文件来存储这些标记。对于每列,无论它是否包含在主键当中,ClickHouse 都会存储类似标记。这些标记让您可以在列文件中直接找到数据。
-
-颗粒的大小通过表引擎参数 `index_granularity` 和 `index_granularity_bytes` 控制。颗粒的行数的在 `[1, index_granularity]` 范围中,这取决于行的大小。如果单行的大小超过了 `index_granularity_bytes` 设置的值,那么一个颗粒的大小会超过 `index_granularity_bytes`。在这种情况下,颗粒的大小等于该行的大小。
-
-## 主键和索引在查询中的表现 {#primary-keys-and-indexes-in-queries}
-
-我们以 `(CounterID, Date)` 以主键。排序好的索引的图示会是下面这样:
-
-``` text
- 全部数据 : [-------------------------------------------------------------------------]
- CounterID: [aaaaaaaaaaaaaaaaaabbbbcdeeeeeeeeeeeeefgggggggghhhhhhhhhiiiiiiiiikllllllll]
- Date: [1111111222222233331233211111222222333211111112122222223111112223311122333]
- 标记: | | | | | | | | | | |
- a,1 a,2 a,3 b,3 e,2 e,3 g,1 h,2 i,1 i,3 l,3
- 标记号: 0 1 2 3 4 5 6 7 8 9 10
-```
-
-如果指定查询如下:
-
-- `CounterID in ('a', 'h')`,服务器会读取标记号在 `[0, 3)` 和 `[6, 8)` 区间中的数据。
-- `CounterID IN ('a', 'h') AND Date = 3`,服务器会读取标记号在 `[1, 3)` 和 `[7, 8)` 区间中的数据。
-- `Date = 3`,服务器会读取标记号在 `[1, 10]` 区间中的数据。
-
-上面例子可以看出使用索引通常会比全表描述要高效。
-
-稀疏索引会引起额外的数据读取。当读取主键单个区间范围的数据时,每个数据块中最多会多读 `index_granularity * 2` 行额外的数据。
-
-稀疏索引使得您可以处理极大量的行,因为大多数情况下,这些索引常驻于内存。
-
-ClickHouse 不要求主键唯一,所以您可以插入多条具有相同主键的行。
-
-您可以在`PRIMARY KEY`与`ORDER BY`条件中使用`可为空的`类型的表达式,但强烈建议不要这么做。为了启用这项功能,请打开[allow_nullable_key](../../../operations/settings/index.md#allow-nullable-key),[NULLS_LAST](../../../sql-reference/statements/select/order-by.md#sorting-of-special-values)规则也适用于`ORDER BY`条件中有NULL值的情况下。
-
-### 主键的选择 {#zhu-jian-de-xuan-ze}
-
-主键中列的数量并没有明确的限制。依据数据结构,您可以在主键包含多些或少些列。这样可以:
-
- - 改善索引的性能。
-
- - 如果当前主键是 `(a, b)` ,在下列情况下添加另一个 `c` 列会提升性能:
-
- - 查询会使用 `c` 列作为条件
- - 很长的数据范围( `index_granularity` 的数倍)里 `(a, b)` 都是相同的值,并且这样的情况很普遍。换言之,就是加入另一列后,可以让您的查询略过很长的数据范围。
-
- - 改善数据压缩。
-
- ClickHouse 以主键排序片段数据,所以,数据的一致性越高,压缩越好。
-
- - 在[CollapsingMergeTree](collapsingmergetree.md#table_engine-collapsingmergetree) 和 [SummingMergeTree](summingmergetree.md) 引擎里进行数据合并时会提供额外的处理逻辑。
-
- 在这种情况下,指定与主键不同的 *排序键* 也是有意义的。
-
-长的主键会对插入性能和内存消耗有负面影响,但主键中额外的列并不影响 `SELECT` 查询的性能。
-
-可以使用 `ORDER BY tuple()` 语法创建没有主键的表。在这种情况下 ClickHouse 根据数据插入的顺序存储。如果在使用 `INSERT ... SELECT` 时希望保持数据的排序,请设置 [max_insert_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads)。
-
-想要根据初始顺序进行数据查询,使用 [单线程查询](../../../operations/settings/settings.md#settings-max_threads)
-
-### 选择与排序键不同的主键 {#choosing-a-primary-key-that-differs-from-the-sorting-key}
-
-Clickhouse可以做到指定一个跟排序键不一样的主键,此时排序键用于在数据片段中进行排序,主键用于在索引文件中进行标记的写入。这种情况下,主键表达式元组必须是排序键表达式元组的前缀(即主键为(a,b),排序列必须为(a,b,******))。
-
-当使用 [SummingMergeTree](summingmergetree.md) 和 [AggregatingMergeTree](aggregatingmergetree.md) 引擎时,这个特性非常有用。通常在使用这类引擎时,表里的列分两种:*维度* 和 *度量* 。典型的查询会通过任意的 `GROUP BY` 对度量列进行聚合并通过维度列进行过滤。由于 SummingMergeTree 和 AggregatingMergeTree 会对排序键相同的行进行聚合,所以把所有的维度放进排序键是很自然的做法。但这将导致排序键中包含大量的列,并且排序键会伴随着新添加的维度不断的更新。
-
-在这种情况下合理的做法是,只保留少量的列在主键当中用于提升扫描效率,将维度列添加到排序键中。
-
-对排序键进行 [ALTER](../../../sql-reference/statements/alter.md) 是轻量级的操作,因为当一个新列同时被加入到表里和排序键里时,已存在的数据片段并不需要修改。由于旧的排序键是新排序键的前缀,并且新添加的列中没有数据,因此在表修改时的数据对于新旧的排序键来说都是有序的。
-
-### 索引和分区在查询中的应用 {#use-of-indexes-and-partitions-in-queries}
-
-对于 `SELECT` 查询,ClickHouse 分析是否可以使用索引。如果 `WHERE/PREWHERE` 子句具有下面这些表达式(作为完整WHERE条件的一部分或全部)则可以使用索引:进行相等/不相等的比较;对主键列或分区列进行`IN`运算、有固定前缀的`LIKE`运算(如name like 'test%')、函数运算(部分函数适用),还有对上述表达式进行逻辑运算。
-
-
-因此,在索引键的一个或多个区间上快速地执行查询是可能的。下面例子中,指定标签;指定标签和日期范围;指定标签和日期;指定多个标签和日期范围等执行查询,都会非常快。
-
-当引擎配置如下时:
-
-``` sql
- ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate) SETTINGS index_granularity=8192
-```
-
-这种情况下,这些查询:
-
-``` sql
-SELECT count() FROM table WHERE EventDate = toDate(now()) AND CounterID = 34
-SELECT count() FROM table WHERE EventDate = toDate(now()) AND (CounterID = 34 OR CounterID = 42)
-SELECT count() FROM table WHERE ((EventDate >= toDate('2014-01-01') AND EventDate <= toDate('2014-01-31')) OR EventDate = toDate('2014-05-01')) AND CounterID IN (101500, 731962, 160656) AND (CounterID = 101500 OR EventDate != toDate('2014-05-01'))
-```
-
-ClickHouse 会依据主键索引剪掉不符合的数据,依据按月分区的分区键剪掉那些不包含符合数据的分区。
-
-上文的查询显示,即使索引用于复杂表达式,因为读表操作经过优化,所以使用索引不会比完整扫描慢。
-
-下面这个例子中,不会使用索引。
-
-``` sql
-SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%'
-```
-
-要检查 ClickHouse 执行一个查询时能否使用索引,可设置 [force_index_by_date](../../../operations/settings/settings.md#settings-force_index_by_date) 和 [force_primary_key](../../../operations/settings/settings.md) 。
-
-使用按月分区的分区列允许只读取包含适当日期区间的数据块,这种情况下,数据块会包含很多天(最多整月)的数据。在块中,数据按主键排序,主键第一列可能不包含日期。因此,仅使用日期而没有用主键字段作为条件的查询将会导致需要读取超过这个指定日期以外的数据。
-
-### 部分单调主键的使用
-
-考虑这样的场景,比如一个月中的天数。它们在一个月的范围内形成一个[单调序列](https://zh.wikipedia.org/wiki/单调函数) ,但如果扩展到更大的时间范围它们就不再单调了。这就是一个部分单调序列。如果用户使用部分单调的主键创建表,ClickHouse同样会创建一个稀疏索引。当用户从这类表中查询数据时,ClickHouse 会对查询条件进行分析。如果用户希望获取两个索引标记之间的数据并且这两个标记在一个月以内,ClickHouse 可以在这种特殊情况下使用到索引,因为它可以计算出查询参数与索引标记之间的距离。
-
-如果查询参数范围内的主键不是单调序列,那么 ClickHouse 无法使用索引。在这种情况下,ClickHouse 会进行全表扫描。
-
-ClickHouse 在任何主键代表一个部分单调序列的情况下都会使用这个逻辑。
-
-### 跳数索引 {#tiao-shu-suo-yin-fen-duan-hui-zong-suo-yin-shi-yan-xing-de}
-
-此索引在 `CREATE` 语句的列部分里定义。
-
-``` sql
-INDEX index_name expr TYPE type(...) GRANULARITY granularity_value
-```
-
-`*MergeTree` 系列的表可以指定跳数索引。
-跳数索引是指数据片段按照粒度(建表时指定的`index_granularity`)分割成小块后,将上述SQL的granularity_value数量的小块组合成一个大的块,对这些大块写入索引信息,这样有助于使用`where`筛选时跳过大量不必要的数据,减少`SELECT`需要读取的数据量。
-
-**示例**
-
-``` sql
-CREATE TABLE table_name
-(
- u64 UInt64,
- i32 Int32,
- s String,
- ...
- INDEX a (u64 * i32, s) TYPE minmax GRANULARITY 3,
- INDEX b (u64 * length(s)) TYPE set(1000) GRANULARITY 4
-) ENGINE = MergeTree()
-...
-```
-
-上例中的索引能让 ClickHouse 执行下面这些查询时减少读取数据量。
-
-``` sql
-SELECT count() FROM table WHERE s < 'z'
-SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
-```
-
-#### 可用的索引类型 {#table_engine-mergetree-data_skipping-indexes}
-
-- `minmax`
- 存储指定表达式的极值(如果表达式是 `tuple` ,则存储 `tuple` 中每个元素的极值),这些信息用于跳过数据块,类似主键。
-
-- `set(max_rows)`
- 存储指定表达式的不重复值(不超过 `max_rows` 个,`max_rows=0` 则表示『无限制』)。这些信息可用于检查数据块是否满足 `WHERE` 条件。
-
-- `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
- 存储一个包含数据块中所有 n元短语(ngram) 的 [布隆过滤器](https://en.wikipedia.org/wiki/Bloom_filter) 。只可用在字符串上。
- 可用于优化 `equals` , `like` 和 `in` 表达式的性能。
- - `n` – 短语长度。
- - `size_of_bloom_filter_in_bytes` – 布隆过滤器大小,字节为单位。(因为压缩得好,可以指定比较大的值,如 256 或 512)。
- - `number_of_hash_functions` – 布隆过滤器中使用的哈希函数的个数。
- - `random_seed` – 哈希函数的随机种子。
-
-- `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
- 跟 `ngrambf_v1` 类似,但是存储的是token而不是ngrams。Token是由非字母数字的符号分割的序列。
-
-- `bloom_filter(bloom_filter([false_positive])` – 为指定的列存储布隆过滤器
-
- 可选参数`false_positive`用来指定从布隆过滤器收到错误响应的几率。取值范围是 (0,1),默认值:0.025
-
- 支持的数据类型:`Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`。
-
- 以下函数会用到这个索引: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md)
-
-``` sql
-INDEX sample_index (u64 * length(s)) TYPE minmax GRANULARITY 4
-INDEX sample_index2 (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARITY 4
-INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY 4
-```
-
-#### 函数支持 {#functions-support}
-
-WHERE 子句中的条件可以包含对某列数据进行运算的函数表达式,如果列是索引的一部分,ClickHouse会在执行函数时尝试使用索引。不同的函数对索引的支持是不同的。
-
-`set` 索引会对所有函数生效,其他索引对函数的生效情况见下表
-
-| 函数 (操作符) / 索引 | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter |
-| ------------------------------------------------------------ | ----------- | ------ | ---------- | ---------- | ------------ |
-| [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#equals) | ✔ | ✔ | ✔ | ✔ | ✔ |
-| [notEquals(!=, <>)](../../../sql-reference/functions/comparison-functions.md#notequals) | ✔ | ✔ | ✔ | ✔ | ✔ |
-| [like](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✔ | ✔ |
-| [notLike](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✗ | ✗ | ✗ |
-| [startsWith](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ |
-| [endsWith](../../../sql-reference/functions/string-functions.md#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ |
-| [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ |
-| [in](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ |
-| [notIn](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ |
-| [less (\<)](../../../sql-reference/functions/comparison-functions.md#less) | ✔ | ✔ | ✗ | ✗ | ✗ |
-| [greater (\>)](../../../sql-reference/functions/comparison-functions.md#greater) | ✔ | ✔ | ✗ | ✗ | ✗ |
-| [lessOrEquals (\<=)](../../../sql-reference/functions/comparison-functions.md#lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ |
-| [greaterOrEquals (\>=)](../../../sql-reference/functions/comparison-functions.md#greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ |
-| [empty](../../../sql-reference/functions/array-functions.md#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ |
-| [notEmpty](../../../sql-reference/functions/array-functions.md#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ |
-| [has](../../../sql-reference/functions/array-functions.md#function-has) | ✗ | ✗ | ✔ | ✔ | ✔ | ✔ |
-| [hasAny](../../../sql-reference/functions/array-functions.md#function-hasAny) | ✗ | ✗ | ✔ | ✔ | ✔ | ✗ |
-| [hasAll](../../../sql-reference/functions/array-functions.md#function-hasAll) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ |
-| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ |
-
-常量参数小于 ngram 大小的函数不能使用 `ngrambf_v1` 进行查询优化。
-
-:::note
-布隆过滤器可能会包含不符合条件的匹配,所以 `ngrambf_v1`, `tokenbf_v1` 和 `bloom_filter` 索引不能用于结果返回为假的函数,例如:
-
-- 可以用来优化的场景
- - `s LIKE '%test%'`
- - `NOT s NOT LIKE '%test%'`
- - `s = 1`
- - `NOT s != 1`
- - `startsWith(s, 'test')`
-- 不能用来优化的场景
- - `NOT s LIKE '%test%'`
- - `s NOT LIKE '%test%'`
- - `NOT s = 1`
- - `s != 1`
- - `NOT startsWith(s, 'test')`
-:::
-
-## 并发数据访问 {#concurrent-data-access}
-
-对于表的并发访问,我们使用多版本机制。换言之,当一张表同时被读和更新时,数据从当前查询到的一组片段中读取。没有冗长的的锁。插入不会阻碍读取。
-
-对表的读操作是自动并行的。
-
-## 列和表的 TTL {#table_engine-mergetree-ttl}
-
-TTL用于设置值的生命周期,它既可以为整张表设置,也可以为每个列字段单独设置。表级别的 TTL 还会指定数据在磁盘和卷上自动转移的逻辑。
-
-TTL 表达式的计算结果必须是 [日期](../../../engines/table-engines/mergetree-family/mergetree.md) 或 [日期时间](../../../engines/table-engines/mergetree-family/mergetree.md) 类型的字段。
-
-示例:
-
-``` sql
-TTL time_column
-TTL time_column + interval
-```
-
-要定义`interval`, 需要使用 [时间间隔](../../../engines/table-engines/mergetree-family/mergetree.md#operators-datetime) 操作符。
-
-``` sql
-TTL date_time + INTERVAL 1 MONTH
-TTL date_time + INTERVAL 15 HOUR
-```
-
-### 列 TTL {#mergetree-column-ttl}
-
-当列中的值过期时, ClickHouse会将它们替换成该列数据类型的默认值。如果数据片段中列的所有值均已过期,则ClickHouse 会从文件系统中的数据片段中删除此列。
-
-`TTL`子句不能被用于主键字段。
-
-**示例:**
-
-创建表时指定 `TTL`
-
-``` sql
-CREATE TABLE example_table
-(
- d DateTime,
- a Int TTL d + INTERVAL 1 MONTH,
- b Int TTL d + INTERVAL 1 MONTH,
- c String
-)
-ENGINE = MergeTree
-PARTITION BY toYYYYMM(d)
-ORDER BY d;
-```
-
-为表中已存在的列字段添加 `TTL`
-
-``` sql
-ALTER TABLE example_table
- MODIFY COLUMN
- c String TTL d + INTERVAL 1 DAY;
-```
-
-修改列字段的 `TTL`
-
-``` sql
-ALTER TABLE example_table
- MODIFY COLUMN
- c String TTL d + INTERVAL 1 MONTH;
-```
-
-### 表 TTL {#mergetree-table-ttl}
-
-表可以设置一个用于移除过期行的表达式,以及多个用于在磁盘或卷上自动转移数据片段的表达式。当表中的行过期时,ClickHouse 会删除所有对应的行。对于数据片段的转移特性,必须所有的行都满足转移条件。
-
-``` sql
-TTL expr
- [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'][, DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'] ...
- [WHERE conditions]
- [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ]
-
-```
-
-TTL 规则的类型紧跟在每个 TTL 表达式后面,它会影响满足表达式时(到达指定时间时)应当执行的操作:
-
-- `DELETE` - 删除过期的行(默认操作);
-- `TO DISK 'aaa'` - 将数据片段移动到磁盘 `aaa`;
-- `TO VOLUME 'bbb'` - 将数据片段移动到卷 `bbb`.
-- `GROUP BY` - 聚合过期的行
-
-使用`WHERE`从句,您可以指定哪些过期的行会被删除或聚合(不适用于移动)。`GROUP BY`表达式必须是表主键的前缀。如果某列不是`GROUP BY`表达式的一部分,也没有在SET从句显示引用,结果行中相应列的值是随机的(就好像使用了`any`函数)。
-
-**示例**:
-
-创建时指定 TTL
-
-``` sql
-CREATE TABLE example_table
-(
- d DateTime,
- a Int
-)
-ENGINE = MergeTree
-PARTITION BY toYYYYMM(d)
-ORDER BY d
-TTL d + INTERVAL 1 MONTH DELETE,
- d + INTERVAL 1 WEEK TO VOLUME 'aaa',
- d + INTERVAL 2 WEEK TO DISK 'bbb';
-```
-
-修改表的 `TTL`
-
-``` sql
-ALTER TABLE example_table
- MODIFY TTL d + INTERVAL 1 DAY;
-```
-
-创建一张表,设置一个月后数据过期,这些过期的行中日期为星期一的删除:
-
-``` sql
-CREATE TABLE table_with_where
-(
- d DateTime,
- a Int
-)
-ENGINE = MergeTree
-PARTITION BY toYYYYMM(d)
-ORDER BY d
-TTL d + INTERVAL 1 MONTH DELETE WHERE toDayOfWeek(d) = 1;
-```
-
-创建一张表,设置过期的列会被聚合。列`x`包含每组行中的最大值,`y`为最小值,`d`为可能任意值。
-
-``` sql
-CREATE TABLE table_for_aggregation
-(
- d DateTime,
- k1 Int,
- k2 Int,
- x Int,
- y Int
-)
-ENGINE = MergeTree
-ORDER BY (k1, k2)
-TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y);
-```
-
-**删除数据**
-
-ClickHouse 在数据片段合并时会删除掉过期的数据。
-
-当ClickHouse发现数据过期时, 它将会执行一个计划外的合并。要控制这类合并的频率, 您可以设置 `merge_with_ttl_timeout`。如果该值被设置的太低, 它将引发大量计划外的合并,这可能会消耗大量资源。
-
-如果在两次合并的时间间隔中执行 `SELECT` 查询, 则可能会得到过期的数据。为了避免这种情况,可以在 `SELECT` 之前使用 [OPTIMIZE](../../../engines/table-engines/mergetree-family/mergetree.md#misc_operations-optimize) 。
-
-## 使用多个块设备进行数据存储 {#table_engine-mergetree-multiple-volumes}
-
-### 介绍 {#introduction}
-
-MergeTree 系列表引擎可以将数据存储在多个块设备上。这对某些可以潜在被划分为“冷”“热”的表来说是很有用的。最新数据被定期的查询但只需要很小的空间。相反,详尽的历史数据很少被用到。如果有多块磁盘可用,那么“热”的数据可以放置在快速的磁盘上(比如 NVMe 固态硬盘或内存),“冷”的数据可以放在相对较慢的磁盘上(比如机械硬盘)。
-
-数据片段是 `MergeTree` 引擎表的最小可移动单元。属于同一个数据片段的数据被存储在同一块磁盘上。数据片段会在后台自动的在磁盘间移动,也可以通过 [ALTER](../../../sql-reference/statements/alter.md#alter_move-partition) 查询来移动。
-
-### 术语 {#terms}
-
-- 磁盘 — 挂载到文件系统的块设备
-- 默认磁盘 — 在服务器设置中通过 [path](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) 参数指定的数据存储
-- 卷 — 相同磁盘的顺序列表 (类似于 [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures))
-- 存储策略 — 卷的集合及他们之间的数据移动规则
-
- 以上名称的信息在Clickhouse中系统表[system.storage_policies](https://clickhouse.com/docs/zh/operations/system-tables/storage_policies/#system_tables-storage_policies)和[system.disks](https://clickhouse.com/docs/zh/operations/system-tables/disks/#system_tables-disks)体现。为了应用存储策略,可以在建表时使用`storage_policy`设置。
-
-### 配置 {#table_engine-mergetree-multiple-volumes_configure}
-
-磁盘、卷和存储策略应当在主配置文件 `config.xml` 或 `config.d` 目录中的独立文件中的 `` 标签内定义。
-
-配置结构:
-
-``` xml
-
-
-
- /mnt/fast_ssd/clickhouse/
-
-
- /mnt/hdd1/clickhouse/
- 10485760
-
-
- /mnt/hdd2/clickhouse/
- 10485760
-
-
- ...
-
-
- ...
-
-```
-
-标签:
-
-- `` — 磁盘名,名称必须与其他磁盘不同.
-- `path` — 服务器将用来存储数据 (`data` 和 `shadow` 目录) 的路径, 应当以 ‘/’ 结尾.
-- `keep_free_space_bytes` — 需要保留的剩余磁盘空间.
-
-磁盘定义的顺序无关紧要。
-
-存储策略配置:
-
-``` xml
-
- ...
-
-
-
-
- disk_name_from_disks_configuration
- 1073741824
-
-
-
-
-
-
- 0.2
-
-
-
-
-
-
-
- ...
-
-```
-
-标签:
-
-- `policy_name_N` — 策略名称,不能重复。
-- `volume_name_N` — 卷名称,不能重复。
-- `disk` — 卷中的磁盘。
-- `max_data_part_size_bytes` — 卷中的磁盘可以存储的数据片段的最大大小。
-- `move_factor` — 当可用空间少于这个因子时,数据将自动的向下一个卷(如果有的话)移动 (默认值为 0.1)。
-
-配置示例:
-
-``` xml
-
- ...
-
-
-
-
- disk1
- disk2
-
-
-
-
-
-
-
- fast_ssd
- 1073741824
-
-
- disk1
-
-
- 0.2
-
-
-
-
-
- jbod1
-
-
- external
-
-
-
-
- ...
-
-```
-
-在给出的例子中, `hdd_in_order` 策略实现了 [循环制](https://zh.wikipedia.org/wiki/循环制) 方法。因此这个策略只定义了一个卷(`single`),数据片段会以循环的顺序全部存储到它的磁盘上。当有多个类似的磁盘挂载到系统上,但没有配置 RAID 时,这种策略非常有用。请注意一个每个独立的磁盘驱动都并不可靠,您可能需要用3份或更多的复制份数来补偿它。
-
-如果在系统中有不同类型的磁盘可用,可以使用 `moving_from_ssd_to_hdd`。`hot` 卷由 SSD 磁盘(`fast_ssd`)组成,这个卷上可以存储的数据片段的最大大小为 1GB。所有大于 1GB 的数据片段都会被直接存储到 `cold` 卷上,`cold` 卷包含一个名为 `disk1` 的 HDD 磁盘。
-同样,一旦 `fast_ssd` 被填充超过 80%,数据会通过后台进程向 `disk1` 进行转移。
-
-存储策略中卷的枚举顺序是很重要的。因为当一个卷被充满时,数据会向下一个卷转移。磁盘的枚举顺序同样重要,因为数据是依次存储在磁盘上的。
-
-在创建表时,可以应用存储策略:
-
-``` sql
-CREATE TABLE table_with_non_default_policy (
- EventDate Date,
- OrderID UInt64,
- BannerID UInt64,
- SearchPhrase String
-) ENGINE = MergeTree
-ORDER BY (OrderID, BannerID)
-PARTITION BY toYYYYMM(EventDate)
-SETTINGS storage_policy = 'moving_from_ssd_to_hdd'
-```
-
-`default` 存储策略意味着只使用一个卷,这个卷只包含一个在 `` 中定义的磁盘。您可以使用[ALTER TABLE ... MODIFY SETTING]来修改存储策略,新的存储策略应该包含所有以前的磁盘和卷,并使用相同的名称。
-
-可以通过 [background_move_pool_size](../../../operations/server-configuration-parameters/settings.md#background_move_pool_size) 设置调整执行后台任务的线程数。
-
-### 详细说明 {#details}
-
-对于 `MergeTree` 表,数据通过以下不同的方式写入到磁盘当中:
-
-- 插入(`INSERT`查询)
-- 后台合并和[数据变异](../../../sql-reference/statements/alter.md#alter-mutations)
-- 从另一个副本下载
-- [ALTER TABLE … FREEZE PARTITION](../../../sql-reference/statements/alter.md#alter_freeze-partition) 冻结分区
-
-除了数据变异和冻结分区以外的情况下,数据按照以下逻辑存储到卷或磁盘上:
-
-1. 首个卷(按定义顺序)拥有足够的磁盘空间存储数据片段(`unreserved_space > current_part_size`)并且允许存储给定数据片段的大小(`max_data_part_size_bytes > current_part_size`)
-2. 在这个数据卷内,紧挨着先前存储数据的那块磁盘之后的磁盘,拥有比数据片段大的剩余空间。(`unreserved_space - keep_free_space_bytes > current_part_size`)
-
-更进一步,数据变异和分区冻结使用的是 [硬链接](https://en.wikipedia.org/wiki/Hard_link)。不同磁盘之间的硬链接是不支持的,所以在这种情况下数据片段都会被存储到原来的那一块磁盘上。
-
-在后台,数据片段基于剩余空间(`move_factor`参数)根据卷在配置文件中定义的顺序进行转移。数据永远不会从最后一个移出也不会从第一个移入。可以通过系统表 [system.part_log](../../../operations/system-tables/part_log.md#system_tables-part-log) (字段 `type = MOVE_PART`) 和 [system.parts](../../../operations/system-tables/parts.md#system_tables-parts) (字段 `path` 和 `disk`) 来监控后台的移动情况。具体细节可以通过服务器日志查看。
-
-用户可以通过 [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter.md#alter_move-partition) 强制移动一个数据片段或分区到另外一个卷,所有后台移动的限制都会被考虑在内。这个查询会自行启动,无需等待后台操作完成。如果没有足够的可用空间或任何必须条件没有被满足,用户会收到报错信息。
-
-数据移动不会妨碍到数据复制。也就是说,同一张表的不同副本可以指定不同的存储策略。
-
-在后台合并和数据变异之后,旧的数据片段会在一定时间后被移除 (`old_parts_lifetime`)。在这期间,他们不能被移动到其他的卷或磁盘。也就是说,直到数据片段被完全移除,它们仍然会被磁盘占用空间计算在内。
-
-## 使用S3进行数据存储 {#using-s3-data-storage}
-
-`MergeTree`系列表引擎允许使用[S3](https://aws.amazon.com/s3/)存储数据,需要修改磁盘类型为`S3`。
-
-示例配置:
-
-``` xml
-
- ...
-
-
- s3
- https://storage.yandexcloud.net/my-bucket/root-path/
- your_access_key_id
- your_secret_access_key
-
- your_base64_encoded_customer_key
-
- http://proxy1
- http://proxy2
-
- 10000
- 5000
- 10
- 4
- 1000
- /var/lib/clickhouse/disks/s3/
- false
-
-
- ...
-
-```
-
-必须的参数:
-
-- `endpoint` - S3的结点URL,以`path`或`virtual hosted`[格式](https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html)书写。
-- `access_key_id` - S3的Access Key ID。
-- `secret_access_key` - S3的Secret Access Key。
-
-可选参数:
-
-- `region` - S3的区域名称
-- `use_environment_credentials` - 从环境变量AWS_ACCESS_KEY_ID、AWS_SECRET_ACCESS_KEY和AWS_SESSION_TOKEN中读取认证参数。默认值为`false`。
-- `use_insecure_imds_request` - 如果设置为`true`,S3客户端在认证时会使用不安全的IMDS请求。默认值为`false`。
-- `proxy` - 访问S3结点URL时代理设置。每一个`uri`项的值都应该是合法的代理URL。
-- `connect_timeout_ms` - Socket连接超时时间,默认值为`10000`,即10秒。
-- `request_timeout_ms` - 请求超时时间,默认值为`5000`,即5秒。
-- `retry_attempts` - 请求失败后的重试次数,默认值为10。
-- `single_read_retries` - 读过程中连接丢失后重试次数,默认值为4。
-- `min_bytes_for_seek` - 使用查找操作,而不是顺序读操作的最小字节数,默认值为1000。
-- `metadata_path` - 本地存放S3元数据文件的路径,默认值为`/var/lib/clickhouse/disks//`
-- `skip_access_check` - 如果为`true`,Clickhouse启动时不检查磁盘是否可用。默认为`false`。
-- `server_side_encryption_customer_key_base64` - 如果指定该项的值,请求时会加上为了访问SSE-C加密数据而必须的头信息。
-
-S3磁盘也可以设置冷热存储:
-```xml
-
- ...
-
-
- s3
- https://storage.yandexcloud.net/my-bucket/root-path/
- your_access_key_id
- your_secret_access_key
-
-
-
-
-
-
- s3
-
-
-
-
-
-
- default
-
-
- s3
-
-
- 0.2
-
-
- ...
-
-```
-
-指定了`cold`选项后,本地磁盘剩余空间如果小于`move_factor * disk_size`,或有TTL设置时,数据就会定时迁移至S3了。
-
-## 虚拟列 {#virtual-columns}
-
-- `_part` - 分区名称。
-- `_part_index` - 作为请求的结果,按顺序排列的分区数。
-- `_partition_id` — 分区名称。
-- `_part_uuid` - 唯一部分标识符(如果 MergeTree 设置`assign_part_uuids` 已启用)。
-- `_partition_value` — `partition by` 表达式的值(元组)。
-- `_sample_factor` - 采样因子(来自请求)。
+---
+slug: /zh/engines/table-engines/mergetree-family/mergetree
+---
+# MergeTree {#table_engines-mergetree}
+
+Clickhouse 中最强大的表引擎当属 `MergeTree` (合并树)引擎及该系列(`*MergeTree`)中的其他引擎。
+
+`MergeTree` 系列的引擎被设计用于插入极大量的数据到一张表当中。数据可以以数据片段的形式一个接着一个的快速写入,数据片段在后台按照一定的规则进行合并。相比在插入时不断修改(重写)已存储的数据,这种策略会高效很多。
+
+主要特点:
+
+- 存储的数据按主键排序。
+
+ 这使得您能够创建一个小型的稀疏索引来加快数据检索。
+
+- 如果指定了 [分区键](custom-partitioning-key.md) 的话,可以使用分区。
+
+ 在相同数据集和相同结果集的情况下 ClickHouse 中某些带分区的操作会比普通操作更快。查询中指定了分区键时 ClickHouse 会自动截取分区数据。这也有效增加了查询性能。
+
+- 支持数据副本。
+
+ `ReplicatedMergeTree` 系列的表提供了数据副本功能。更多信息,请参阅 [数据副本](replication.md) 一节。
+
+- 支持数据采样。
+
+ 需要的话,您可以给表设置一个采样方法。
+
+:::info
+[合并](../special/merge.md#merge) 引擎并不属于 `*MergeTree` 系列。
+:::
+
+## 建表 {#table_engine-mergetree-creating-a-table}
+
+``` sql
+CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
+(
+ name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1],
+ name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2],
+ ...
+ INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1,
+ INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2
+) ENGINE = MergeTree()
+ORDER BY expr
+[PARTITION BY expr]
+[PRIMARY KEY expr]
+[SAMPLE BY expr]
+[TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'], ...]
+[SETTINGS name=value, ...]
+```
+
+对于以上参数的描述,可参考 [CREATE 语句 的描述](../../../engines/table-engines/mergetree-family/mergetree.md) 。
+
+
+
+**子句**
+
+- `ENGINE` - 引擎名和参数。 `ENGINE = MergeTree()`. `MergeTree` 引擎没有参数。
+
+- `ORDER BY` — 排序键。
+
+ 可以是一组列的元组或任意的表达式。 例如: `ORDER BY (CounterID, EventDate)` 。
+
+ 如果没有使用 `PRIMARY KEY` 显式指定的主键,ClickHouse 会使用排序键作为主键。
+
+ 如果不需要排序,可以使用 `ORDER BY tuple()`. 参考 [选择主键](https://clickhouse.com/docs/en/engines/table-engines/mergetree-family/mergetree/#selecting-the-primary-key)
+
+- `PARTITION BY` — [分区键](custom-partitioning-key.md) ,可选项。
+
+ 大多数情况下,不需要使用分区键。即使需要使用,也不需要使用比月更细粒度的分区键。分区不会加快查询(这与 ORDER BY 表达式不同)。永远也别使用过细粒度的分区键。不要使用客户端指定分区标识符或分区字段名称来对数据进行分区(而是将分区字段标识或名称作为 ORDER BY 表达式的第一列来指定分区)。
+
+ 要按月分区,可以使用表达式 `toYYYYMM(date_column)` ,这里的 `date_column` 是一个 [Date](../../../engines/table-engines/mergetree-family/mergetree.md) 类型的列。分区名的格式会是 `"YYYYMM"` 。
+
+- `PRIMARY KEY` - 如果要 [选择与排序键不同的主键](#choosing-a-primary-key-that-differs-from-the-sorting-key),在这里指定,可选项。
+
+ 默认情况下主键跟排序键(由 `ORDER BY` 子句指定)相同。
+ 因此,大部分情况下不需要再专门指定一个 `PRIMARY KEY` 子句。
+
+- `SAMPLE BY` - 用于抽样的表达式,可选项。
+
+ 如果要用抽样表达式,主键中必须包含这个表达式。例如:
+ `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))` 。
+
+- `TTL` - 指定行存储的持续时间并定义数据片段在硬盘和卷上的移动逻辑的规则列表,可选项。
+
+ 表达式中必须存在至少一个 `Date` 或 `DateTime` 类型的列,比如:
+
+ `TTL date + INTERVAl 1 DAY`
+
+ 规则的类型 `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'`指定了当满足条件(到达指定时间)时所要执行的动作:移除过期的行,还是将数据片段(如果数据片段中的所有行都满足表达式的话)移动到指定的磁盘(`TO DISK 'xxx'`) 或 卷(`TO VOLUME 'xxx'`)。默认的规则是移除(`DELETE`)。可以在列表中指定多个规则,但最多只能有一个`DELETE`的规则。
+
+ 更多细节,请查看 [表和列的 TTL](#table_engine-mergetree-ttl)
+
+- `SETTINGS` — 控制 `MergeTree` 行为的额外参数,可选项:
+
+ - `index_granularity` — 索引粒度。索引中相邻的『标记』间的数据行数。默认值8192 。参考[数据存储](#mergetree-data-storage)。
+ - `index_granularity_bytes` — 索引粒度,以字节为单位,默认值: 10Mb。如果想要仅按数据行数限制索引粒度, 请设置为0(不建议)。
+ - `min_index_granularity_bytes` - 允许的最小数据粒度,默认值:1024b。该选项用于防止误操作,添加了一个非常低索引粒度的表。参考[数据存储](#mergetree-data-storage)
+ - `enable_mixed_granularity_parts` — 是否启用通过 `index_granularity_bytes` 控制索引粒度的大小。在19.11版本之前, 只有 `index_granularity` 配置能够用于限制索引粒度的大小。当从具有很大的行(几十上百兆字节)的表中查询数据时候,`index_granularity_bytes` 配置能够提升ClickHouse的性能。如果您的表里有很大的行,可以开启这项配置来提升`SELECT` 查询的性能。
+ - `use_minimalistic_part_header_in_zookeeper` — ZooKeeper中数据片段存储方式 。如果`use_minimalistic_part_header_in_zookeeper=1` ,ZooKeeper 会存储更少的数据。更多信息参考[服务配置参数]([Server Settings | ClickHouse Documentation](https://clickhouse.com/docs/zh/operations/server-configuration-parameters/settings/))这章中的 [设置描述](../../../operations/server-configuration-parameters/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) 。
+ - `min_merge_bytes_to_use_direct_io` — 使用直接 I/O 来操作磁盘的合并操作时要求的最小数据量。合并数据片段时,ClickHouse 会计算要被合并的所有数据的总存储空间。如果大小超过了 `min_merge_bytes_to_use_direct_io` 设置的字节数,则 ClickHouse 将使用直接 I/O 接口(`O_DIRECT` 选项)对磁盘读写。如果设置 `min_merge_bytes_to_use_direct_io = 0` ,则会禁用直接 I/O。默认值:`10 * 1024 * 1024 * 1024` 字节。
+
+ - `merge_with_ttl_timeout` — TTL合并频率的最小间隔时间,单位:秒。默认值: 86400 (1 天)。
+ - `write_final_mark` — 是否启用在数据片段尾部写入最终索引标记。默认值: 1(不要关闭)。
+ - `merge_max_block_size` — 在块中进行合并操作时的最大行数限制。默认值:8192
+ - `storage_policy` — 存储策略。 参见 [使用具有多个块的设备进行数据存储](#table_engine-mergetree-multiple-volumes).
+ - `min_bytes_for_wide_part`,`min_rows_for_wide_part` 在数据片段中可以使用`Wide`格式进行存储的最小字节数/行数。您可以不设置、只设置一个,或全都设置。参考:[数据存储](#mergetree-data-storage)
+ - `max_parts_in_total` - 所有分区中最大块的数量(意义不明)
+ - `max_compress_block_size` - 在数据压缩写入表前,未压缩数据块的最大大小。您可以在全局设置中设置该值(参见[max_compress_block_size](https://clickhouse.com/docs/zh/operations/settings/settings/#max-compress-block-size))。建表时指定该值会覆盖全局设置。
+ - `min_compress_block_size` - 在数据压缩写入表前,未压缩数据块的最小大小。您可以在全局设置中设置该值(参见[min_compress_block_size](https://clickhouse.com/docs/zh/operations/settings/settings/#min-compress-block-size))。建表时指定该值会覆盖全局设置。
+ - `max_partitions_to_read` - 一次查询中可访问的分区最大数。您可以在全局设置中设置该值(参见[max_partitions_to_read](https://clickhouse.com/docs/zh/operations/settings/settings/#max_partitions_to_read))。
+
+**示例配置**
+
+``` sql
+ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192
+```
+
+在这个例子中,我们设置了按月进行分区。
+
+同时我们设置了一个按用户 ID 哈希的抽样表达式。这使得您可以对该表中每个 `CounterID` 和 `EventDate` 的数据伪随机分布。如果您在查询时指定了 [SAMPLE](../../../engines/table-engines/mergetree-family/mergetree.md#select-sample-clause) 子句。 ClickHouse会返回对于用户子集的一个均匀的伪随机数据采样。
+
+`index_granularity` 可省略因为 8192 是默认设置 。
+
+
+已弃用的建表方法
+
+:::attention "注意"
+不要在新版项目中使用该方法,可能的话,请将旧项目切换到上述方法。
+:::
+
+``` sql
+CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
+(
+ name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1],
+ name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2],
+ ...
+) ENGINE [=] MergeTree(date-column [, sampling_expression], (primary, key), index_granularity)
+```
+
+**MergeTree() 参数**
+
+- `date-column` — 类型为 [日期](../../../engines/table-engines/mergetree-family/mergetree.md) 的列名。ClickHouse 会自动依据这个列按月创建分区。分区名格式为 `"YYYYMM"` 。
+- `sampling_expression` — 采样表达式。
+- `(primary, key)` — 主键。类型 — [元组()](../../../engines/table-engines/mergetree-family/mergetree.md)
+- `index_granularity` — 索引粒度。即索引中相邻『标记』间的数据行数。设为 8192 可以适用大部分场景。
+
+**示例**
+
+ MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID)), 8192)
+
+对于主要的配置方法,这里 `MergeTree` 引擎跟前面的例子一样,可以以同样的方式配置。
+
+
+## 数据存储 {#mergetree-data-storage}
+
+表由按主键排序的数据片段(DATA PART)组成。
+
+当数据被插入到表中时,会创建多个数据片段并按主键的字典序排序。例如,主键是 `(CounterID, Date)` 时,片段中数据首先按 `CounterID` 排序,具有相同 `CounterID` 的部分按 `Date` 排序。
+
+不同分区的数据会被分成不同的片段,ClickHouse 在后台合并数据片段以便更高效存储。不同分区的数据片段不会进行合并。合并机制并不保证具有相同主键的行全都合并到同一个数据片段中。
+
+数据片段可以以 `Wide` 或 `Compact` 格式存储。在 `Wide` 格式下,每一列都会在文件系统中存储为单独的文件,在 `Compact` 格式下所有列都存储在一个文件中。`Compact` 格式可以提高插入量少插入频率频繁时的性能。
+
+数据存储格式由 `min_bytes_for_wide_part` 和 `min_rows_for_wide_part` 表引擎参数控制。如果数据片段中的字节数或行数少于相应的设置值,数据片段会以 `Compact` 格式存储,否则会以 `Wide` 格式存储。
+
+每个数据片段被逻辑的分割成颗粒(granules)。颗粒是 ClickHouse 中进行数据查询时的最小不可分割数据集。ClickHouse 不会对行或值进行拆分,所以每个颗粒总是包含整数个行。每个颗粒的第一行通过该行的主键值进行标记,
+ClickHouse 会为每个数据片段创建一个索引文件来存储这些标记。对于每列,无论它是否包含在主键当中,ClickHouse 都会存储类似标记。这些标记让您可以在列文件中直接找到数据。
+
+颗粒的大小通过表引擎参数 `index_granularity` 和 `index_granularity_bytes` 控制。颗粒的行数的在 `[1, index_granularity]` 范围中,这取决于行的大小。如果单行的大小超过了 `index_granularity_bytes` 设置的值,那么一个颗粒的大小会超过 `index_granularity_bytes`。在这种情况下,颗粒的大小等于该行的大小。
+
+## 主键和索引在查询中的表现 {#primary-keys-and-indexes-in-queries}
+
+我们以 `(CounterID, Date)` 以主键。排序好的索引的图示会是下面这样:
+
+``` text
+ 全部数据 : [-------------------------------------------------------------------------]
+ CounterID: [aaaaaaaaaaaaaaaaaabbbbcdeeeeeeeeeeeeefgggggggghhhhhhhhhiiiiiiiiikllllllll]
+ Date: [1111111222222233331233211111222222333211111112122222223111112223311122333]
+ 标记: | | | | | | | | | | |
+ a,1 a,2 a,3 b,3 e,2 e,3 g,1 h,2 i,1 i,3 l,3
+ 标记号: 0 1 2 3 4 5 6 7 8 9 10
+```
+
+如果指定查询如下:
+
+- `CounterID in ('a', 'h')`,服务器会读取标记号在 `[0, 3)` 和 `[6, 8)` 区间中的数据。
+- `CounterID IN ('a', 'h') AND Date = 3`,服务器会读取标记号在 `[1, 3)` 和 `[7, 8)` 区间中的数据。
+- `Date = 3`,服务器会读取标记号在 `[1, 10]` 区间中的数据。
+
+上面例子可以看出使用索引通常会比全表描述要高效。
+
+稀疏索引会引起额外的数据读取。当读取主键单个区间范围的数据时,每个数据块中最多会多读 `index_granularity * 2` 行额外的数据。
+
+稀疏索引使得您可以处理极大量的行,因为大多数情况下,这些索引常驻于内存。
+
+ClickHouse 不要求主键唯一,所以您可以插入多条具有相同主键的行。
+
+您可以在`PRIMARY KEY`与`ORDER BY`条件中使用`可为空的`类型的表达式,但强烈建议不要这么做。为了启用这项功能,请打开[allow_nullable_key](../../../operations/settings/index.md#allow-nullable-key),[NULLS_LAST](../../../sql-reference/statements/select/order-by.md#sorting-of-special-values)规则也适用于`ORDER BY`条件中有NULL值的情况下。
+
+### 主键的选择 {#zhu-jian-de-xuan-ze}
+
+主键中列的数量并没有明确的限制。依据数据结构,您可以在主键包含多些或少些列。这样可以:
+
+ - 改善索引的性能。
+
+ - 如果当前主键是 `(a, b)` ,在下列情况下添加另一个 `c` 列会提升性能:
+
+ - 查询会使用 `c` 列作为条件
+ - 很长的数据范围( `index_granularity` 的数倍)里 `(a, b)` 都是相同的值,并且这样的情况很普遍。换言之,就是加入另一列后,可以让您的查询略过很长的数据范围。
+
+ - 改善数据压缩。
+
+ ClickHouse 以主键排序片段数据,所以,数据的一致性越高,压缩越好。
+
+ - 在[CollapsingMergeTree](collapsingmergetree.md#table_engine-collapsingmergetree) 和 [SummingMergeTree](summingmergetree.md) 引擎里进行数据合并时会提供额外的处理逻辑。
+
+ 在这种情况下,指定与主键不同的 *排序键* 也是有意义的。
+
+长的主键会对插入性能和内存消耗有负面影响,但主键中额外的列并不影响 `SELECT` 查询的性能。
+
+可以使用 `ORDER BY tuple()` 语法创建没有主键的表。在这种情况下 ClickHouse 根据数据插入的顺序存储。如果在使用 `INSERT ... SELECT` 时希望保持数据的排序,请设置 [max_insert_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads)。
+
+想要根据初始顺序进行数据查询,使用 [单线程查询](../../../operations/settings/settings.md#settings-max_threads)
+
+### 选择与排序键不同的主键 {#choosing-a-primary-key-that-differs-from-the-sorting-key}
+
+Clickhouse可以做到指定一个跟排序键不一样的主键,此时排序键用于在数据片段中进行排序,主键用于在索引文件中进行标记的写入。这种情况下,主键表达式元组必须是排序键表达式元组的前缀(即主键为(a,b),排序列必须为(a,b,******))。
+
+当使用 [SummingMergeTree](summingmergetree.md) 和 [AggregatingMergeTree](aggregatingmergetree.md) 引擎时,这个特性非常有用。通常在使用这类引擎时,表里的列分两种:*维度* 和 *度量* 。典型的查询会通过任意的 `GROUP BY` 对度量列进行聚合并通过维度列进行过滤。由于 SummingMergeTree 和 AggregatingMergeTree 会对排序键相同的行进行聚合,所以把所有的维度放进排序键是很自然的做法。但这将导致排序键中包含大量的列,并且排序键会伴随着新添加的维度不断的更新。
+
+在这种情况下合理的做法是,只保留少量的列在主键当中用于提升扫描效率,将维度列添加到排序键中。
+
+对排序键进行 [ALTER](../../../sql-reference/statements/alter.md) 是轻量级的操作,因为当一个新列同时被加入到表里和排序键里时,已存在的数据片段并不需要修改。由于旧的排序键是新排序键的前缀,并且新添加的列中没有数据,因此在表修改时的数据对于新旧的排序键来说都是有序的。
+
+### 索引和分区在查询中的应用 {#use-of-indexes-and-partitions-in-queries}
+
+对于 `SELECT` 查询,ClickHouse 分析是否可以使用索引。如果 `WHERE/PREWHERE` 子句具有下面这些表达式(作为完整WHERE条件的一部分或全部)则可以使用索引:进行相等/不相等的比较;对主键列或分区列进行`IN`运算、有固定前缀的`LIKE`运算(如name like 'test%')、函数运算(部分函数适用),还有对上述表达式进行逻辑运算。
+
+
+因此,在索引键的一个或多个区间上快速地执行查询是可能的。下面例子中,指定标签;指定标签和日期范围;指定标签和日期;指定多个标签和日期范围等执行查询,都会非常快。
+
+当引擎配置如下时:
+
+``` sql
+ ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate) SETTINGS index_granularity=8192
+```
+
+这种情况下,这些查询:
+
+``` sql
+SELECT count() FROM table WHERE EventDate = toDate(now()) AND CounterID = 34
+SELECT count() FROM table WHERE EventDate = toDate(now()) AND (CounterID = 34 OR CounterID = 42)
+SELECT count() FROM table WHERE ((EventDate >= toDate('2014-01-01') AND EventDate <= toDate('2014-01-31')) OR EventDate = toDate('2014-05-01')) AND CounterID IN (101500, 731962, 160656) AND (CounterID = 101500 OR EventDate != toDate('2014-05-01'))
+```
+
+ClickHouse 会依据主键索引剪掉不符合的数据,依据按月分区的分区键剪掉那些不包含符合数据的分区。
+
+上文的查询显示,即使索引用于复杂表达式,因为读表操作经过优化,所以使用索引不会比完整扫描慢。
+
+下面这个例子中,不会使用索引。
+
+``` sql
+SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%'
+```
+
+要检查 ClickHouse 执行一个查询时能否使用索引,可设置 [force_index_by_date](../../../operations/settings/settings.md#settings-force_index_by_date) 和 [force_primary_key](../../../operations/settings/settings.md) 。
+
+使用按月分区的分区列允许只读取包含适当日期区间的数据块,这种情况下,数据块会包含很多天(最多整月)的数据。在块中,数据按主键排序,主键第一列可能不包含日期。因此,仅使用日期而没有用主键字段作为条件的查询将会导致需要读取超过这个指定日期以外的数据。
+
+### 部分单调主键的使用
+
+考虑这样的场景,比如一个月中的天数。它们在一个月的范围内形成一个[单调序列](https://zh.wikipedia.org/wiki/单调函数) ,但如果扩展到更大的时间范围它们就不再单调了。这就是一个部分单调序列。如果用户使用部分单调的主键创建表,ClickHouse同样会创建一个稀疏索引。当用户从这类表中查询数据时,ClickHouse 会对查询条件进行分析。如果用户希望获取两个索引标记之间的数据并且这两个标记在一个月以内,ClickHouse 可以在这种特殊情况下使用到索引,因为它可以计算出查询参数与索引标记之间的距离。
+
+如果查询参数范围内的主键不是单调序列,那么 ClickHouse 无法使用索引。在这种情况下,ClickHouse 会进行全表扫描。
+
+ClickHouse 在任何主键代表一个部分单调序列的情况下都会使用这个逻辑。
+
+### 跳数索引 {#tiao-shu-suo-yin-fen-duan-hui-zong-suo-yin-shi-yan-xing-de}
+
+此索引在 `CREATE` 语句的列部分里定义。
+
+``` sql
+INDEX index_name expr TYPE type(...) GRANULARITY granularity_value
+```
+
+`*MergeTree` 系列的表可以指定跳数索引。
+跳数索引是指数据片段按照粒度(建表时指定的`index_granularity`)分割成小块后,将上述SQL的granularity_value数量的小块组合成一个大的块,对这些大块写入索引信息,这样有助于使用`where`筛选时跳过大量不必要的数据,减少`SELECT`需要读取的数据量。
+
+**示例**
+
+``` sql
+CREATE TABLE table_name
+(
+ u64 UInt64,
+ i32 Int32,
+ s String,
+ ...
+ INDEX a (u64 * i32, s) TYPE minmax GRANULARITY 3,
+ INDEX b (u64 * length(s)) TYPE set(1000) GRANULARITY 4
+) ENGINE = MergeTree()
+...
+```
+
+上例中的索引能让 ClickHouse 执行下面这些查询时减少读取数据量。
+
+``` sql
+SELECT count() FROM table WHERE s < 'z'
+SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
+```
+
+#### 可用的索引类型 {#table_engine-mergetree-data_skipping-indexes}
+
+- `minmax`
+ 存储指定表达式的极值(如果表达式是 `tuple` ,则存储 `tuple` 中每个元素的极值),这些信息用于跳过数据块,类似主键。
+
+- `set(max_rows)`
+ 存储指定表达式的不重复值(不超过 `max_rows` 个,`max_rows=0` 则表示『无限制』)。这些信息可用于检查数据块是否满足 `WHERE` 条件。
+
+- `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
+ 存储一个包含数据块中所有 n元短语(ngram) 的 [布隆过滤器](https://en.wikipedia.org/wiki/Bloom_filter) 。只可用在字符串上。
+ 可用于优化 `equals` , `like` 和 `in` 表达式的性能。
+ - `n` – 短语长度。
+ - `size_of_bloom_filter_in_bytes` – 布隆过滤器大小,字节为单位。(因为压缩得好,可以指定比较大的值,如 256 或 512)。
+ - `number_of_hash_functions` – 布隆过滤器中使用的哈希函数的个数。
+ - `random_seed` – 哈希函数的随机种子。
+
+- `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
+ 跟 `ngrambf_v1` 类似,但是存储的是token而不是ngrams。Token是由非字母数字的符号分割的序列。
+
+- `bloom_filter(bloom_filter([false_positive])` – 为指定的列存储布隆过滤器
+
+ 可选参数`false_positive`用来指定从布隆过滤器收到错误响应的几率。取值范围是 (0,1),默认值:0.025
+
+ 支持的数据类型:`Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`。
+
+ 以下函数会用到这个索引: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md)
+
+``` sql
+INDEX sample_index (u64 * length(s)) TYPE minmax GRANULARITY 4
+INDEX sample_index2 (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARITY 4
+INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY 4
+```
+
+#### 函数支持 {#functions-support}
+
+WHERE 子句中的条件可以包含对某列数据进行运算的函数表达式,如果列是索引的一部分,ClickHouse会在执行函数时尝试使用索引。不同的函数对索引的支持是不同的。
+
+`set` 索引会对所有函数生效,其他索引对函数的生效情况见下表
+
+| 函数 (操作符) / 索引 | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter |
+| ------------------------------------------------------------ | ----------- | ------ | ---------- | ---------- | ------------ |
+| [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#equals) | ✔ | ✔ | ✔ | ✔ | ✔ |
+| [notEquals(!=, <>)](../../../sql-reference/functions/comparison-functions.md#notequals) | ✔ | ✔ | ✔ | ✔ | ✔ |
+| [like](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✔ | ✔ |
+| [notLike](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✗ | ✗ | ✗ |
+| [startsWith](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ |
+| [endsWith](../../../sql-reference/functions/string-functions.md#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ |
+| [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ |
+| [in](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ |
+| [notIn](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ |
+| [less (\<)](../../../sql-reference/functions/comparison-functions.md#less) | ✔ | ✔ | ✗ | ✗ | ✗ |
+| [greater (\>)](../../../sql-reference/functions/comparison-functions.md#greater) | ✔ | ✔ | ✗ | ✗ | ✗ |
+| [lessOrEquals (\<=)](../../../sql-reference/functions/comparison-functions.md#lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ |
+| [greaterOrEquals (\>=)](../../../sql-reference/functions/comparison-functions.md#greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ |
+| [empty](../../../sql-reference/functions/array-functions.md#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ |
+| [notEmpty](../../../sql-reference/functions/array-functions.md#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ |
+| [has](../../../sql-reference/functions/array-functions.md#function-has) | ✗ | ✗ | ✔ | ✔ | ✔ | ✔ |
+| [hasAny](../../../sql-reference/functions/array-functions.md#function-hasAny) | ✗ | ✗ | ✔ | ✔ | ✔ | ✗ |
+| [hasAll](../../../sql-reference/functions/array-functions.md#function-hasAll) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ |
+| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ |
+
+常量参数小于 ngram 大小的函数不能使用 `ngrambf_v1` 进行查询优化。
+
+:::note
+布隆过滤器可能会包含不符合条件的匹配,所以 `ngrambf_v1`, `tokenbf_v1` 和 `bloom_filter` 索引不能用于结果返回为假的函数,例如:
+
+- 可以用来优化的场景
+ - `s LIKE '%test%'`
+ - `NOT s NOT LIKE '%test%'`
+ - `s = 1`
+ - `NOT s != 1`
+ - `startsWith(s, 'test')`
+- 不能用来优化的场景
+ - `NOT s LIKE '%test%'`
+ - `s NOT LIKE '%test%'`
+ - `NOT s = 1`
+ - `s != 1`
+ - `NOT startsWith(s, 'test')`
+:::
+
+## 并发数据访问 {#concurrent-data-access}
+
+对于表的并发访问,我们使用多版本机制。换言之,当一张表同时被读和更新时,数据从当前查询到的一组片段中读取。没有冗长的的锁。插入不会阻碍读取。
+
+对表的读操作是自动并行的。
+
+## 列和表的 TTL {#table_engine-mergetree-ttl}
+
+TTL用于设置值的生命周期,它既可以为整张表设置,也可以为每个列字段单独设置。表级别的 TTL 还会指定数据在磁盘和卷上自动转移的逻辑。
+
+TTL 表达式的计算结果必须是 [日期](../../../engines/table-engines/mergetree-family/mergetree.md) 或 [日期时间](../../../engines/table-engines/mergetree-family/mergetree.md) 类型的字段。
+
+示例:
+
+``` sql
+TTL time_column
+TTL time_column + interval
+```
+
+要定义`interval`, 需要使用 [时间间隔](../../../engines/table-engines/mergetree-family/mergetree.md#operators-datetime) 操作符。
+
+``` sql
+TTL date_time + INTERVAL 1 MONTH
+TTL date_time + INTERVAL 15 HOUR
+```
+
+### 列 TTL {#mergetree-column-ttl}
+
+当列中的值过期时, ClickHouse会将它们替换成该列数据类型的默认值。如果数据片段中列的所有值均已过期,则ClickHouse 会从文件系统中的数据片段中删除此列。
+
+`TTL`子句不能被用于主键字段。
+
+**示例:**
+
+创建表时指定 `TTL`
+
+``` sql
+CREATE TABLE example_table
+(
+ d DateTime,
+ a Int TTL d + INTERVAL 1 MONTH,
+ b Int TTL d + INTERVAL 1 MONTH,
+ c String
+)
+ENGINE = MergeTree
+PARTITION BY toYYYYMM(d)
+ORDER BY d;
+```
+
+为表中已存在的列字段添加 `TTL`
+
+``` sql
+ALTER TABLE example_table
+ MODIFY COLUMN
+ c String TTL d + INTERVAL 1 DAY;
+```
+
+修改列字段的 `TTL`
+
+``` sql
+ALTER TABLE example_table
+ MODIFY COLUMN
+ c String TTL d + INTERVAL 1 MONTH;
+```
+
+### 表 TTL {#mergetree-table-ttl}
+
+表可以设置一个用于移除过期行的表达式,以及多个用于在磁盘或卷上自动转移数据片段的表达式。当表中的行过期时,ClickHouse 会删除所有对应的行。对于数据片段的转移特性,必须所有的行都满足转移条件。
+
+``` sql
+TTL expr
+ [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'][, DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'] ...
+ [WHERE conditions]
+ [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ]
+
+```
+
+TTL 规则的类型紧跟在每个 TTL 表达式后面,它会影响满足表达式时(到达指定时间时)应当执行的操作:
+
+- `DELETE` - 删除过期的行(默认操作);
+- `TO DISK 'aaa'` - 将数据片段移动到磁盘 `aaa`;
+- `TO VOLUME 'bbb'` - 将数据片段移动到卷 `bbb`.
+- `GROUP BY` - 聚合过期的行
+
+使用`WHERE`从句,您可以指定哪些过期的行会被删除或聚合(不适用于移动)。`GROUP BY`表达式必须是表主键的前缀。如果某列不是`GROUP BY`表达式的一部分,也没有在SET从句显示引用,结果行中相应列的值是随机的(就好像使用了`any`函数)。
+
+**示例**:
+
+创建时指定 TTL
+
+``` sql
+CREATE TABLE example_table
+(
+ d DateTime,
+ a Int
+)
+ENGINE = MergeTree
+PARTITION BY toYYYYMM(d)
+ORDER BY d
+TTL d + INTERVAL 1 MONTH DELETE,
+ d + INTERVAL 1 WEEK TO VOLUME 'aaa',
+ d + INTERVAL 2 WEEK TO DISK 'bbb';
+```
+
+修改表的 `TTL`
+
+``` sql
+ALTER TABLE example_table
+ MODIFY TTL d + INTERVAL 1 DAY;
+```
+
+创建一张表,设置一个月后数据过期,这些过期的行中日期为星期一的删除:
+
+``` sql
+CREATE TABLE table_with_where
+(
+ d DateTime,
+ a Int
+)
+ENGINE = MergeTree
+PARTITION BY toYYYYMM(d)
+ORDER BY d
+TTL d + INTERVAL 1 MONTH DELETE WHERE toDayOfWeek(d) = 1;
+```
+
+创建一张表,设置过期的列会被聚合。列`x`包含每组行中的最大值,`y`为最小值,`d`为可能任意值。
+
+``` sql
+CREATE TABLE table_for_aggregation
+(
+ d DateTime,
+ k1 Int,
+ k2 Int,
+ x Int,
+ y Int
+)
+ENGINE = MergeTree
+ORDER BY (k1, k2)
+TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y);
+```
+
+**删除数据**
+
+ClickHouse 在数据片段合并时会删除掉过期的数据。
+
+当ClickHouse发现数据过期时, 它将会执行一个计划外的合并。要控制这类合并的频率, 您可以设置 `merge_with_ttl_timeout`。如果该值被设置的太低, 它将引发大量计划外的合并,这可能会消耗大量资源。
+
+如果在两次合并的时间间隔中执行 `SELECT` 查询, 则可能会得到过期的数据。为了避免这种情况,可以在 `SELECT` 之前使用 [OPTIMIZE](../../../engines/table-engines/mergetree-family/mergetree.md#misc_operations-optimize) 。
+
+## 使用多个块设备进行数据存储 {#table_engine-mergetree-multiple-volumes}
+
+### 介绍 {#introduction}
+
+MergeTree 系列表引擎可以将数据存储在多个块设备上。这对某些可以潜在被划分为“冷”“热”的表来说是很有用的。最新数据被定期的查询但只需要很小的空间。相反,详尽的历史数据很少被用到。如果有多块磁盘可用,那么“热”的数据可以放置在快速的磁盘上(比如 NVMe 固态硬盘或内存),“冷”的数据可以放在相对较慢的磁盘上(比如机械硬盘)。
+
+数据片段是 `MergeTree` 引擎表的最小可移动单元。属于同一个数据片段的数据被存储在同一块磁盘上。数据片段会在后台自动的在磁盘间移动,也可以通过 [ALTER](../../../sql-reference/statements/alter.md#alter_move-partition) 查询来移动。
+
+### 术语 {#terms}
+
+- 磁盘 — 挂载到文件系统的块设备
+- 默认磁盘 — 在服务器设置中通过 [path](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) 参数指定的数据存储
+- 卷 — 相同磁盘的顺序列表 (类似于 [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures))
+- 存储策略 — 卷的集合及他们之间的数据移动规则
+
+ 以上名称的信息在Clickhouse中系统表[system.storage_policies](https://clickhouse.com/docs/zh/operations/system-tables/storage_policies/#system_tables-storage_policies)和[system.disks](https://clickhouse.com/docs/zh/operations/system-tables/disks/#system_tables-disks)体现。为了应用存储策略,可以在建表时使用`storage_policy`设置。
+
+### 配置 {#table_engine-mergetree-multiple-volumes_configure}
+
+磁盘、卷和存储策略应当在主配置文件 `config.xml` 或 `config.d` 目录中的独立文件中的 `` 标签内定义。
+
+配置结构:
+
+``` xml
+
+
+
+ /mnt/fast_ssd/clickhouse/
+
+
+ /mnt/hdd1/clickhouse/
+ 10485760
+
+
+ /mnt/hdd2/clickhouse/
+ 10485760
+
+
+ ...
+
+
+ ...
+
+```
+
+标签:
+
+- `` — 磁盘名,名称必须与其他磁盘不同.
+- `path` — 服务器将用来存储数据 (`data` 和 `shadow` 目录) 的路径, 应当以 ‘/’ 结尾.
+- `keep_free_space_bytes` — 需要保留的剩余磁盘空间.
+
+磁盘定义的顺序无关紧要。
+
+存储策略配置:
+
+``` xml
+
+ ...
+
+
+
+
+ disk_name_from_disks_configuration
+ 1073741824
+
+
+
+
+
+
+ 0.2
+
+
+
+
+
+
+
+ ...
+
+```
+
+标签:
+
+- `policy_name_N` — 策略名称,不能重复。
+- `volume_name_N` — 卷名称,不能重复。
+- `disk` — 卷中的磁盘。
+- `max_data_part_size_bytes` — 卷中的磁盘可以存储的数据片段的最大大小。
+- `move_factor` — 当可用空间少于这个因子时,数据将自动的向下一个卷(如果有的话)移动 (默认值为 0.1)。
+
+配置示例:
+
+``` xml
+
+ ...
+
+
+
+
+ disk1
+ disk2
+
+
+
+
+
+
+
+ fast_ssd
+ 1073741824
+
+
+ disk1
+
+
+ 0.2
+
+
+
+
+
+ jbod1
+
+
+ external
+
+
+
+
+ ...
+
+```
+
+在给出的例子中, `hdd_in_order` 策略实现了 [循环制](https://zh.wikipedia.org/wiki/循环制) 方法。因此这个策略只定义了一个卷(`single`),数据片段会以循环的顺序全部存储到它的磁盘上。当有多个类似的磁盘挂载到系统上,但没有配置 RAID 时,这种策略非常有用。请注意一个每个独立的磁盘驱动都并不可靠,您可能需要用3份或更多的复制份数来补偿它。
+
+如果在系统中有不同类型的磁盘可用,可以使用 `moving_from_ssd_to_hdd`。`hot` 卷由 SSD 磁盘(`fast_ssd`)组成,这个卷上可以存储的数据片段的最大大小为 1GB。所有大于 1GB 的数据片段都会被直接存储到 `cold` 卷上,`cold` 卷包含一个名为 `disk1` 的 HDD 磁盘。
+同样,一旦 `fast_ssd` 被填充超过 80%,数据会通过后台进程向 `disk1` 进行转移。
+
+存储策略中卷的枚举顺序是很重要的。因为当一个卷被充满时,数据会向下一个卷转移。磁盘的枚举顺序同样重要,因为数据是依次存储在磁盘上的。
+
+在创建表时,可以应用存储策略:
+
+``` sql
+CREATE TABLE table_with_non_default_policy (
+ EventDate Date,
+ OrderID UInt64,
+ BannerID UInt64,
+ SearchPhrase String
+) ENGINE = MergeTree
+ORDER BY (OrderID, BannerID)
+PARTITION BY toYYYYMM(EventDate)
+SETTINGS storage_policy = 'moving_from_ssd_to_hdd'
+```
+
+`default` 存储策略意味着只使用一个卷,这个卷只包含一个在 `` 中定义的磁盘。您可以使用[ALTER TABLE ... MODIFY SETTING]来修改存储策略,新的存储策略应该包含所有以前的磁盘和卷,并使用相同的名称。
+
+可以通过 [background_move_pool_size](../../../operations/server-configuration-parameters/settings.md#background_move_pool_size) 设置调整执行后台任务的线程数。
+
+### 详细说明 {#details}
+
+对于 `MergeTree` 表,数据通过以下不同的方式写入到磁盘当中:
+
+- 插入(`INSERT`查询)
+- 后台合并和[数据变异](../../../sql-reference/statements/alter.md#alter-mutations)
+- 从另一个副本下载
+- [ALTER TABLE … FREEZE PARTITION](../../../sql-reference/statements/alter.md#alter_freeze-partition) 冻结分区
+
+除了数据变异和冻结分区以外的情况下,数据按照以下逻辑存储到卷或磁盘上:
+
+1. 首个卷(按定义顺序)拥有足够的磁盘空间存储数据片段(`unreserved_space > current_part_size`)并且允许存储给定数据片段的大小(`max_data_part_size_bytes > current_part_size`)
+2. 在这个数据卷内,紧挨着先前存储数据的那块磁盘之后的磁盘,拥有比数据片段大的剩余空间。(`unreserved_space - keep_free_space_bytes > current_part_size`)
+
+更进一步,数据变异和分区冻结使用的是 [硬链接](https://en.wikipedia.org/wiki/Hard_link)。不同磁盘之间的硬链接是不支持的,所以在这种情况下数据片段都会被存储到原来的那一块磁盘上。
+
+在后台,数据片段基于剩余空间(`move_factor`参数)根据卷在配置文件中定义的顺序进行转移。数据永远不会从最后一个移出也不会从第一个移入。可以通过系统表 [system.part_log](../../../operations/system-tables/part_log.md#system_tables-part-log) (字段 `type = MOVE_PART`) 和 [system.parts](../../../operations/system-tables/parts.md#system_tables-parts) (字段 `path` 和 `disk`) 来监控后台的移动情况。具体细节可以通过服务器日志查看。
+
+用户可以通过 [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter.md#alter_move-partition) 强制移动一个数据片段或分区到另外一个卷,所有后台移动的限制都会被考虑在内。这个查询会自行启动,无需等待后台操作完成。如果没有足够的可用空间或任何必须条件没有被满足,用户会收到报错信息。
+
+数据移动不会妨碍到数据复制。也就是说,同一张表的不同副本可以指定不同的存储策略。
+
+在后台合并和数据变异之后,旧的数据片段会在一定时间后被移除 (`old_parts_lifetime`)。在这期间,他们不能被移动到其他的卷或磁盘。也就是说,直到数据片段被完全移除,它们仍然会被磁盘占用空间计算在内。
+
+## 使用S3进行数据存储 {#using-s3-data-storage}
+
+`MergeTree`系列表引擎允许使用[S3](https://aws.amazon.com/s3/)存储数据,需要修改磁盘类型为`S3`。
+
+示例配置:
+
+``` xml
+
+ ...
+
+
+ s3
+ https://storage.yandexcloud.net/my-bucket/root-path/
+ your_access_key_id
+ your_secret_access_key
+
+ your_base64_encoded_customer_key
+
+ http://proxy1
+ http://proxy2
+
+ 10000
+ 5000
+ 10
+ 4
+ 1000
+ /var/lib/clickhouse/disks/s3/
+ false
+
+
+ ...
+
+```
+
+必须的参数:
+
+- `endpoint` - S3的结点URL,以`path`或`virtual hosted`[格式](https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html)书写。
+- `access_key_id` - S3的Access Key ID。
+- `secret_access_key` - S3的Secret Access Key。
+
+可选参数:
+
+- `region` - S3的区域名称
+- `use_environment_credentials` - 从环境变量AWS_ACCESS_KEY_ID、AWS_SECRET_ACCESS_KEY和AWS_SESSION_TOKEN中读取认证参数。默认值为`false`。
+- `use_insecure_imds_request` - 如果设置为`true`,S3客户端在认证时会使用不安全的IMDS请求。默认值为`false`。
+- `proxy` - 访问S3结点URL时代理设置。每一个`uri`项的值都应该是合法的代理URL。
+- `connect_timeout_ms` - Socket连接超时时间,默认值为`10000`,即10秒。
+- `request_timeout_ms` - 请求超时时间,默认值为`5000`,即5秒。
+- `retry_attempts` - 请求失败后的重试次数,默认值为10。
+- `single_read_retries` - 读过程中连接丢失后重试次数,默认值为4。
+- `min_bytes_for_seek` - 使用查找操作,而不是顺序读操作的最小字节数,默认值为1000。
+- `metadata_path` - 本地存放S3元数据文件的路径,默认值为`/var/lib/clickhouse/disks//`
+- `skip_access_check` - 如果为`true`,Clickhouse启动时不检查磁盘是否可用。默认为`false`。
+- `server_side_encryption_customer_key_base64` - 如果指定该项的值,请求时会加上为了访问SSE-C加密数据而必须的头信息。
+
+S3磁盘也可以设置冷热存储:
+```xml
+
+ ...
+
+
+ s3
+ https://storage.yandexcloud.net/my-bucket/root-path/
+ your_access_key_id
+ your_secret_access_key
+
+
+
+
+
+
+ s3
+
+
+
+
+
+
+ default
+
+
+ s3
+
+
+ 0.2
+
+
+ ...
+
+```
+
+指定了`cold`选项后,本地磁盘剩余空间如果小于`move_factor * disk_size`,或有TTL设置时,数据就会定时迁移至S3了。
+
+## 虚拟列 {#virtual-columns}
+
+- `_part` - 分区名称。
+- `_part_index` - 作为请求的结果,按顺序排列的分区数。
+- `_partition_id` — 分区名称。
+- `_part_uuid` - 唯一部分标识符(如果 MergeTree 设置`assign_part_uuids` 已启用)。
+- `_partition_value` — `partition by` 表达式的值(元组)。
+- `_sample_factor` - 采样因子(来自请求)。
diff --git a/docs/zh/faq/general/dbms-naming.md b/docs/zh/faq/general/dbms-naming.md
index e732c2f054e..f24b3134093 100644
--- a/docs/zh/faq/general/dbms-naming.md
+++ b/docs/zh/faq/general/dbms-naming.md
@@ -1,18 +1,18 @@
----
+---
slug: /zh/faq/general/dbms-naming
-title: "\u201CClickHouse\u201D 有什么含义?"
-toc_hidden: true
-sidebar_position: 10
----
-
-# “ClickHouse” 有什么含义? {#what-does-clickhouse-mean}
-
-它是“**点击**流”和“数据**仓库**”的组合。它来自于Yandex最初的用例。在Metrica网站上,ClickHouse本应该保存人们在互联网上的所有点击记录,现在它仍然在做这项工作。你可以在[ClickHouse history](../../introduction/history.md)页面上阅读更多关于这个用例的信息。
-
-这个由两部分组成的意思有两个结果:
-
-- 唯一正确的写“Click**H**ouse”的方式是用大写H。
-- 如果需要缩写,请使用“**CH**”。由于一些历史原因,缩写CK在中国也很流行,主要是因为中文中最早的一个关于ClickHouse的演讲使用了这种形式。
-
-!!! info “有趣的事实”
- 多年后ClickHouse闻名于世, 这种命名方法:结合各有深意的两个词被赞扬为最好的数据库命名方式, 卡内基梅隆大学数据库副教授[Andy Pavlo做的研究](https://www.cs.cmu.edu/~pavlo/blog/2020/03/on-naming-a-database-management-system.html) 。ClickHouse与Postgres共同获得“史上最佳数据库名”奖。
+title: "\u201CClickHouse\u201D 有什么含义?"
+toc_hidden: true
+sidebar_position: 10
+---
+
+# “ClickHouse” 有什么含义? {#what-does-clickhouse-mean}
+
+它是“**点击**流”和“数据**仓库**”的组合。它来自于Yandex最初的用例。在Metrica网站上,ClickHouse本应该保存人们在互联网上的所有点击记录,现在它仍然在做这项工作。你可以在[ClickHouse history](../../introduction/history.md)页面上阅读更多关于这个用例的信息。
+
+这个由两部分组成的意思有两个结果:
+
+- 唯一正确的写“Click**H**ouse”的方式是用大写H。
+- 如果需要缩写,请使用“**CH**”。由于一些历史原因,缩写CK在中国也很流行,主要是因为中文中最早的一个关于ClickHouse的演讲使用了这种形式。
+
+!!! info “有趣的事实”
+ 多年后ClickHouse闻名于世, 这种命名方法:结合各有深意的两个词被赞扬为最好的数据库命名方式, 卡内基梅隆大学数据库副教授[Andy Pavlo做的研究](https://www.cs.cmu.edu/~pavlo/blog/2020/03/on-naming-a-database-management-system.html) 。ClickHouse与Postgres共同获得“史上最佳数据库名”奖。
diff --git a/docs/zh/faq/general/how-do-i-contribute-code-to-clickhouse.md b/docs/zh/faq/general/how-do-i-contribute-code-to-clickhouse.md
index daa7abf525f..16f48baf7ef 100644
--- a/docs/zh/faq/general/how-do-i-contribute-code-to-clickhouse.md
+++ b/docs/zh/faq/general/how-do-i-contribute-code-to-clickhouse.md
@@ -1,18 +1,18 @@
----
+---
slug: /zh/faq/general/how-do-i-contribute-code-to-clickhouse
-title: 我如何为ClickHouse贡献代码?
-toc_hidden: true
-sidebar_position: 120
----
-
-# 我如何为ClickHouse贡献代码? {#how-do-i-contribute-code-to-clickhouse}
-
-ClickHouse是一个开源项目[在GitHub上开发](https://github.com/ClickHouse/ClickHouse)。
-
-按照惯例,贡献指南发布在源代码库根目录的 [CONTRIBUTING.md](https://github.com/ClickHouse/ClickHouse/blob/master/CONTRIBUTING.md)文件中。
-
-如果你想对ClickHouse提出实质性的改变建议,可以考虑[在GitHub上发布一个问题](https://github.com/ClickHouse/ClickHouse/issues/new/choose),解释一下你想做什么,先与维护人员和社区讨论一下。[此类RFC问题的例子](https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aissue+is%3Aopen+rfc)。
-
-如果您的贡献与安全相关,也请查看[我们的安全政策](https://github.com/ClickHouse/ClickHouse/security/policy/)。
-
-
+title: 我如何为ClickHouse贡献代码?
+toc_hidden: true
+sidebar_position: 120
+---
+
+# 我如何为ClickHouse贡献代码? {#how-do-i-contribute-code-to-clickhouse}
+
+ClickHouse是一个开源项目[在GitHub上开发](https://github.com/ClickHouse/ClickHouse)。
+
+按照惯例,贡献指南发布在源代码库根目录的 [CONTRIBUTING.md](https://github.com/ClickHouse/ClickHouse/blob/master/CONTRIBUTING.md)文件中。
+
+如果你想对ClickHouse提出实质性的改变建议,可以考虑[在GitHub上发布一个问题](https://github.com/ClickHouse/ClickHouse/issues/new/choose),解释一下你想做什么,先与维护人员和社区讨论一下。[此类RFC问题的例子](https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aissue+is%3Aopen+rfc)。
+
+如果您的贡献与安全相关,也请查看[我们的安全政策](https://github.com/ClickHouse/ClickHouse/security/policy/)。
+
+
diff --git a/docs/zh/faq/integration/index.md b/docs/zh/faq/integration/index.md
index 3a3f97761f3..b0ca2d05c05 100644
--- a/docs/zh/faq/integration/index.md
+++ b/docs/zh/faq/integration/index.md
@@ -1,22 +1,22 @@
----
-slug: /zh/faq/integration/
-title: 关于集成ClickHouse和其他系统的问题
-toc_hidden_folder: true
-sidebar_position: 4
-sidebar_label: Integration
----
-
-# 关于集成ClickHouse和其他系统的问题 {#question-about-integrating-clickhouse-and-other-systems}
-
-问题:
-
-- [如何从 ClickHouse 导出数据到一个文件?](../../faq/integration/file-export.md)
-- [如何导入JSON到ClickHouse?](../../faq/integration/json-import.md)
-- [如果我用ODBC链接Oracle数据库出现编码问题该怎么办?](../../faq/integration/oracle-odbc.md)
-
-
-
-!!! info "没看到你要找的东西吗?"
- 查看[其他faq类别](../../faq/index.md)或浏览左边栏中的主要文档文章。
-
+---
+slug: /zh/faq/integration/
+title: 关于集成ClickHouse和其他系统的问题
+toc_hidden_folder: true
+sidebar_position: 4
+sidebar_label: Integration
+---
+
+# 关于集成ClickHouse和其他系统的问题 {#question-about-integrating-clickhouse-and-other-systems}
+
+问题:
+
+- [如何从 ClickHouse 导出数据到一个文件?](../../faq/integration/file-export.md)
+- [如何导入JSON到ClickHouse?](../../faq/integration/json-import.md)
+- [如果我用ODBC链接Oracle数据库出现编码问题该怎么办?](../../faq/integration/oracle-odbc.md)
+
+
+
+!!! info "没看到你要找的东西吗?"
+ 查看[其他faq类别](../../faq/index.md)或浏览左边栏中的主要文档文章。
+
{## [原文](https://clickhouse.com/docs/en/faq/integration/) ##}
\ No newline at end of file
diff --git a/docs/zh/faq/operations/index.md b/docs/zh/faq/operations/index.md
index 153eda6199a..1fe84655ada 100644
--- a/docs/zh/faq/operations/index.md
+++ b/docs/zh/faq/operations/index.md
@@ -1,21 +1,21 @@
----
-slug: /zh/faq/operations/
-title: 关于操作ClickHouse服务器和集群的问题
-toc_hidden_folder: true
-sidebar_position: 3
-sidebar_label: Operations
----
-
-# 关于操作ClickHouse服务器和集群的问题 {#question-about-operating-clickhouse-servers-and-clusters}
-
-问题:
-
-- [如果想在生产环境部署,需要用哪个版本的 ClickHouse 呢?](../../faq/operations/production.md)
-- [是否可能从 ClickHouse 数据表中删除所有旧的数据记录?](../../faq/operations/delete-old-data.md)
-- [ClickHouse支持多区域复制吗?](../../faq/operations/multi-region-replication.md)
-
-
-!!! info "没看到你要找的东西吗?"
- 查看[其他faq类别](../../faq/index.md)或浏览左边栏中的主要文档文章。
-
-{## [原文](https://clickhouse.com/docs/en/faq/production/) ##}
+---
+slug: /zh/faq/operations/
+title: 关于操作ClickHouse服务器和集群的问题
+toc_hidden_folder: true
+sidebar_position: 3
+sidebar_label: Operations
+---
+
+# 关于操作ClickHouse服务器和集群的问题 {#question-about-operating-clickhouse-servers-and-clusters}
+
+问题:
+
+- [如果想在生产环境部署,需要用哪个版本的 ClickHouse 呢?](../../faq/operations/production.md)
+- [是否可能从 ClickHouse 数据表中删除所有旧的数据记录?](../../faq/operations/delete-old-data.md)
+- [ClickHouse支持多区域复制吗?](../../faq/operations/multi-region-replication.md)
+
+
+!!! info "没看到你要找的东西吗?"
+ 查看[其他faq类别](../../faq/index.md)或浏览左边栏中的主要文档文章。
+
+{## [原文](https://clickhouse.com/docs/en/faq/production/) ##}
diff --git a/docs/zh/faq/operations/multi-region-replication.md b/docs/zh/faq/operations/multi-region-replication.md
index 05f856a9ea7..14df8b72eff 100644
--- a/docs/zh/faq/operations/multi-region-replication.md
+++ b/docs/zh/faq/operations/multi-region-replication.md
@@ -1,15 +1,15 @@
----
+---
slug: /zh/faq/operations/multi-region-replication
-title: ClickHouse支持多区域复制吗?
-toc_hidden: true
-sidebar_position: 30
----
-
-# ClickHouse支持多区域复制吗? {#does-clickhouse-support-multi-region-replication}
-
-简短的回答是“是的”。然而,我们建议将所有区域/数据中心之间的延迟保持在两位数字范围内,否则,在通过分布式共识协议时,写性能将受到影响。例如,美国海岸之间的复制可能会很好,但美国和欧洲之间就不行。
-
-在配置方面,这与单区域复制没有区别,只是使用位于不同位置的主机作为副本。
-
-更多信息,请参见[关于数据复制的完整文章](../../engines/table-engines/mergetree-family/replication.md)。
-
+title: ClickHouse支持多区域复制吗?
+toc_hidden: true
+sidebar_position: 30
+---
+
+# ClickHouse支持多区域复制吗? {#does-clickhouse-support-multi-region-replication}
+
+简短的回答是“是的”。然而,我们建议将所有区域/数据中心之间的延迟保持在两位数字范围内,否则,在通过分布式共识协议时,写性能将受到影响。例如,美国海岸之间的复制可能会很好,但美国和欧洲之间就不行。
+
+在配置方面,这与单区域复制没有区别,只是使用位于不同位置的主机作为副本。
+
+更多信息,请参见[关于数据复制的完整文章](../../engines/table-engines/mergetree-family/replication.md)。
+
diff --git a/programs/bash-completion/completions/ch b/programs/bash-completion/completions/ch
new file mode 120000
index 00000000000..7101fd9ed04
--- /dev/null
+++ b/programs/bash-completion/completions/ch
@@ -0,0 +1 @@
+clickhouse
\ No newline at end of file
diff --git a/programs/bash-completion/completions/chc b/programs/bash-completion/completions/chc
new file mode 100644
index 00000000000..0e34cd4eab2
--- /dev/null
+++ b/programs/bash-completion/completions/chc
@@ -0,0 +1,2 @@
+[[ -v $_CLICKHOUSE_COMPLETION_LOADED ]] || source "$(dirname "${BASH_SOURCE[0]}")/clickhouse-bootstrap"
+_complete_clickhouse_generic chc
diff --git a/programs/bash-completion/completions/chl b/programs/bash-completion/completions/chl
new file mode 100644
index 00000000000..6d0338bf122
--- /dev/null
+++ b/programs/bash-completion/completions/chl
@@ -0,0 +1,2 @@
+[[ -v $_CLICKHOUSE_COMPLETION_LOADED ]] || source "$(dirname "${BASH_SOURCE[0]}")/clickhouse-bootstrap"
+_complete_clickhouse_generic chl
diff --git a/programs/bash-completion/completions/clickhouse b/programs/bash-completion/completions/clickhouse
index fc55398dcf1..ff0a60c60be 100644
--- a/programs/bash-completion/completions/clickhouse
+++ b/programs/bash-completion/completions/clickhouse
@@ -31,3 +31,4 @@ function _complete_for_clickhouse_entrypoint_bin()
}
_complete_clickhouse_generic clickhouse _complete_for_clickhouse_entrypoint_bin
+_complete_clickhouse_generic ch _complete_for_clickhouse_entrypoint_bin
diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp
index 1dfdcb3c745..59505d08f5c 100644
--- a/programs/copier/ClusterCopier.cpp
+++ b/programs/copier/ClusterCopier.cpp
@@ -1559,7 +1559,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
QueryPipeline input;
QueryPipeline output;
{
- BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute();
+ BlockIO io_insert = InterpreterFactory::instance().get(query_insert_ast, context_insert)->execute();
InterpreterSelectWithUnionQuery select(query_select_ast, context_select, SelectQueryOptions{});
QueryPlan plan;
@@ -1944,7 +1944,7 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts,
auto local_context = Context::createCopy(context);
local_context->setSettings(task_cluster->settings_pull);
- auto pipeline = InterpreterFactory::get(query_ast, local_context)->execute().pipeline;
+ auto pipeline = InterpreterFactory::instance().get(query_ast, local_context)->execute().pipeline;
PullingPipelineExecutor executor(pipeline);
Block block;
executor.pull(block);
@@ -1989,7 +1989,7 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi
auto local_context = Context::createCopy(context);
local_context->setSettings(task_cluster->settings_pull);
- auto pipeline = InterpreterFactory::get(query_ast, local_context)->execute().pipeline;
+ auto pipeline = InterpreterFactory::instance().get(query_ast, local_context)->execute().pipeline;
PullingPipelineExecutor executor(pipeline);
Block result;
executor.pull(result);
diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp
index 8f24d13d379..53f79888573 100644
--- a/programs/copier/ClusterCopierApp.cpp
+++ b/programs/copier/ClusterCopierApp.cpp
@@ -2,7 +2,9 @@
#include
#include
#include
+#include
#include
+#include
#include
#include
#include
@@ -156,9 +158,11 @@ void ClusterCopierApp::mainImpl()
context->setApplicationType(Context::ApplicationType::LOCAL);
context->setPath(process_path + "/");
+ registerInterpreters();
registerFunctions();
registerAggregateFunctions();
registerTableFunctions();
+ registerDatabases();
registerStorages();
registerDictionaries();
registerDisks(/* global_skip_access_check= */ true);
diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp
index 8e7f38b6a1e..ecf02283ab7 100644
--- a/programs/format/Format.cpp
+++ b/programs/format/Format.cpp
@@ -7,6 +7,7 @@
#include
#include
#include
+#include
#include
#include
#include
@@ -17,6 +18,7 @@
#include
#include
+#include
#include
#include
#include
@@ -127,9 +129,11 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
hash_func.update(options["seed"].as());
}
+ registerInterpreters();
registerFunctions();
registerAggregateFunctions();
registerTableFunctions();
+ registerDatabases();
registerStorages();
registerFormats();
diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp
index 48d26233d94..109884ec899 100644
--- a/programs/keeper/Keeper.cpp
+++ b/programs/keeper/Keeper.cpp
@@ -335,7 +335,7 @@ try
else if (std::filesystem::is_directory(std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination"))
{
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG,
- "By default 'keeper.storage_path' could be assigned to {}, but the directory {} already exists. Please specify 'keeper.storage_path' in the keeper configuration explicitly",
+ "By default 'keeper_server.storage_path' could be assigned to {}, but the directory {} already exists. Please specify 'keeper_server.storage_path' in the keeper configuration explicitly",
KEEPER_DEFAULT_PATH, String{std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination"});
}
else
diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp
index b425258b2d3..c9841277b6d 100644
--- a/programs/local/LocalServer.cpp
+++ b/programs/local/LocalServer.cpp
@@ -10,6 +10,7 @@
#include
#include
#include
+#include
#include
#include
#include
@@ -19,6 +20,7 @@
#include
#include
#include
+#include
#include
#include
#include
@@ -485,10 +487,12 @@ try
Poco::ErrorHandler::set(&error_handler);
}
+ registerInterpreters();
/// Don't initialize DateLUT
registerFunctions();
registerAggregateFunctions();
registerTableFunctions();
+ registerDatabases();
registerStorages();
registerDictionaries();
registerDisks(/* global_skip_access_check= */ true);
@@ -726,12 +730,7 @@ void LocalServer::processConfig()
/// We load temporary database first, because projections need it.
DatabaseCatalog::instance().initializeAndLoadTemporaryDatabase();
- /** Init dummy default DB
- * NOTE: We force using isolated default database to avoid conflicts with default database from server environment
- * Otherwise, metadata of temporary File(format, EXPLICIT_PATH) tables will pollute metadata/ directory;
- * if such tables will not be dropped, clickhouse-server will not be able to load them due to security reasons.
- */
- std::string default_database = config().getString("default_database", "_local");
+ std::string default_database = config().getString("default_database", "default");
DatabaseCatalog::instance().attachDatabase(default_database, createClickHouseLocalDatabaseOverlay(default_database, global_context));
global_context->setCurrentDatabase(default_database);
@@ -744,7 +743,7 @@ void LocalServer::processConfig()
LOG_DEBUG(log, "Loading metadata from {}", path);
auto startup_system_tasks = loadMetadataSystem(global_context);
- attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE));
+ attachSystemTablesServer(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE), false);
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA));
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE));
waitLoad(TablesLoaderForegroundPoolId, startup_system_tasks);
@@ -763,7 +762,7 @@ void LocalServer::processConfig()
}
else if (!config().has("no-system-tables"))
{
- attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE));
+ attachSystemTablesServer(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE), false);
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA));
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE));
}
diff --git a/programs/main.cpp b/programs/main.cpp
index 959984d565d..7d07112de66 100644
--- a/programs/main.cpp
+++ b/programs/main.cpp
@@ -158,7 +158,6 @@ std::pair clickhouse_applications[] =
std::pair clickhouse_short_names[] =
{
#if ENABLE_CLICKHOUSE_LOCAL
- {"ch", "local"},
{"chl", "local"},
#endif
#if ENABLE_CLICKHOUSE_CLIENT
@@ -502,6 +501,17 @@ int main(int argc_, char ** argv_)
}
}
+ /// Interpret binary without argument or with arguments starts with dash
+ /// ('-') as clickhouse-local for better usability:
+ ///
+ /// clickhouse # dumps help
+ /// clickhouse -q 'select 1' # use local
+ /// clickhouse # spawn local
+ /// clickhouse local # spawn local
+ ///
+ if (main_func == printHelp && !argv.empty() && (argv.size() == 1 || argv[1][0] == '-'))
+ main_func = mainEntryClickHouseLocal;
+
return main_func(static_cast(argv.size()), argv.data());
}
#endif
diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp
index 481510d681f..80d4b42890d 100644
--- a/programs/server/Server.cpp
+++ b/programs/server/Server.cpp
@@ -58,6 +58,7 @@
#include
#include
#include
+#include
#include
#include
#include
@@ -72,6 +73,7 @@
#include
#include
#include
+#include
#include
#include
#include
@@ -645,9 +647,11 @@ try
}
#endif
+ registerInterpreters();
registerFunctions();
registerAggregateFunctions();
registerTableFunctions();
+ registerDatabases();
registerStorages();
registerDictionaries();
registerDisks(/* global_skip_access_check= */ false);
@@ -1258,11 +1262,11 @@ try
{
Settings::checkNoSettingNamesAtTopLevel(*config, config_path);
- ServerSettings server_settings_;
- server_settings_.loadSettingsFromConfig(*config);
+ ServerSettings new_server_settings;
+ new_server_settings.loadSettingsFromConfig(*config);
- size_t max_server_memory_usage = server_settings_.max_server_memory_usage;
- double max_server_memory_usage_to_ram_ratio = server_settings_.max_server_memory_usage_to_ram_ratio;
+ size_t max_server_memory_usage = new_server_settings.max_server_memory_usage;
+ double max_server_memory_usage_to_ram_ratio = new_server_settings.max_server_memory_usage_to_ram_ratio;
size_t current_physical_server_memory = getMemoryAmount(); /// With cgroups, the amount of memory available to the server can be changed dynamically.
size_t default_max_server_memory_usage = static_cast(current_physical_server_memory * max_server_memory_usage_to_ram_ratio);
@@ -1292,9 +1296,9 @@ try
total_memory_tracker.setDescription("(total)");
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
- size_t merges_mutations_memory_usage_soft_limit = server_settings_.merges_mutations_memory_usage_soft_limit;
+ size_t merges_mutations_memory_usage_soft_limit = new_server_settings.merges_mutations_memory_usage_soft_limit;
- size_t default_merges_mutations_server_memory_usage = static_cast(current_physical_server_memory * server_settings_.merges_mutations_memory_usage_to_ram_ratio);
+ size_t default_merges_mutations_server_memory_usage = static_cast(current_physical_server_memory * new_server_settings.merges_mutations_memory_usage_to_ram_ratio);
if (merges_mutations_memory_usage_soft_limit == 0)
{
merges_mutations_memory_usage_soft_limit = default_merges_mutations_server_memory_usage;
@@ -1302,7 +1306,7 @@ try
" ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio)",
formatReadableSizeWithBinarySuffix(merges_mutations_memory_usage_soft_limit),
formatReadableSizeWithBinarySuffix(current_physical_server_memory),
- server_settings_.merges_mutations_memory_usage_to_ram_ratio);
+ new_server_settings.merges_mutations_memory_usage_to_ram_ratio);
}
else if (merges_mutations_memory_usage_soft_limit > default_merges_mutations_server_memory_usage)
{
@@ -1311,7 +1315,7 @@ try
" ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio)",
formatReadableSizeWithBinarySuffix(merges_mutations_memory_usage_soft_limit),
formatReadableSizeWithBinarySuffix(current_physical_server_memory),
- server_settings_.merges_mutations_memory_usage_to_ram_ratio);
+ new_server_settings.merges_mutations_memory_usage_to_ram_ratio);
}
LOG_INFO(log, "Merges and mutations memory limit is set to {}",
@@ -1320,7 +1324,7 @@ try
background_memory_tracker.setDescription("(background)");
background_memory_tracker.setMetric(CurrentMetrics::MergesMutationsMemoryTracking);
- total_memory_tracker.setAllowUseJemallocMemory(server_settings_.allow_use_jemalloc_memory);
+ total_memory_tracker.setAllowUseJemallocMemory(new_server_settings.allow_use_jemalloc_memory);
auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker();
total_memory_tracker.setOvercommitTracker(global_overcommit_tracker);
@@ -1344,26 +1348,26 @@ try
global_context->setRemoteHostFilter(*config);
global_context->setHTTPHeaderFilter(*config);
- global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop);
- global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop);
- global_context->setMaxTableNumToWarn(server_settings_.max_table_num_to_warn);
- global_context->setMaxDatabaseNumToWarn(server_settings_.max_database_num_to_warn);
- global_context->setMaxPartNumToWarn(server_settings_.max_part_num_to_warn);
+ global_context->setMaxTableSizeToDrop(new_server_settings.max_table_size_to_drop);
+ global_context->setMaxPartitionSizeToDrop(new_server_settings.max_partition_size_to_drop);
+ global_context->setMaxTableNumToWarn(new_server_settings.max_table_num_to_warn);
+ global_context->setMaxDatabaseNumToWarn(new_server_settings.max_database_num_to_warn);
+ global_context->setMaxPartNumToWarn(new_server_settings.max_part_num_to_warn);
ConcurrencyControl::SlotCount concurrent_threads_soft_limit = ConcurrencyControl::Unlimited;
- if (server_settings_.concurrent_threads_soft_limit_num > 0 && server_settings_.concurrent_threads_soft_limit_num < concurrent_threads_soft_limit)
- concurrent_threads_soft_limit = server_settings_.concurrent_threads_soft_limit_num;
- if (server_settings_.concurrent_threads_soft_limit_ratio_to_cores > 0)
+ if (new_server_settings.concurrent_threads_soft_limit_num > 0 && new_server_settings.concurrent_threads_soft_limit_num < concurrent_threads_soft_limit)
+ concurrent_threads_soft_limit = new_server_settings.concurrent_threads_soft_limit_num;
+ if (new_server_settings.concurrent_threads_soft_limit_ratio_to_cores > 0)
{
- auto value = server_settings_.concurrent_threads_soft_limit_ratio_to_cores * std::thread::hardware_concurrency();
+ auto value = new_server_settings.concurrent_threads_soft_limit_ratio_to_cores * std::thread::hardware_concurrency();
if (value > 0 && value < concurrent_threads_soft_limit)
concurrent_threads_soft_limit = value;
}
ConcurrencyControl::instance().setMaxConcurrency(concurrent_threads_soft_limit);
- global_context->getProcessList().setMaxSize(server_settings_.max_concurrent_queries);
- global_context->getProcessList().setMaxInsertQueriesAmount(server_settings_.max_concurrent_insert_queries);
- global_context->getProcessList().setMaxSelectQueriesAmount(server_settings_.max_concurrent_select_queries);
+ global_context->getProcessList().setMaxSize(new_server_settings.max_concurrent_queries);
+ global_context->getProcessList().setMaxInsertQueriesAmount(new_server_settings.max_concurrent_insert_queries);
+ global_context->getProcessList().setMaxSelectQueriesAmount(new_server_settings.max_concurrent_select_queries);
if (config->has("keeper_server"))
global_context->updateKeeperConfiguration(*config);
@@ -1374,68 +1378,68 @@ try
/// This is done for backward compatibility.
if (global_context->areBackgroundExecutorsInitialized())
{
- auto new_pool_size = server_settings_.background_pool_size;
- auto new_ratio = server_settings_.background_merges_mutations_concurrency_ratio;
+ auto new_pool_size = new_server_settings.background_pool_size;
+ auto new_ratio = new_server_settings.background_merges_mutations_concurrency_ratio;
global_context->getMergeMutateExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, static_cast(new_pool_size * new_ratio));
- global_context->getMergeMutateExecutor()->updateSchedulingPolicy(server_settings_.background_merges_mutations_scheduling_policy.toString());
+ global_context->getMergeMutateExecutor()->updateSchedulingPolicy(new_server_settings.background_merges_mutations_scheduling_policy.toString());
}
if (global_context->areBackgroundExecutorsInitialized())
{
- auto new_pool_size = server_settings_.background_move_pool_size;
+ auto new_pool_size = new_server_settings.background_move_pool_size;
global_context->getMovesExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size);
}
if (global_context->areBackgroundExecutorsInitialized())
{
- auto new_pool_size = server_settings_.background_fetches_pool_size;
+ auto new_pool_size = new_server_settings.background_fetches_pool_size;
global_context->getFetchesExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size);
}
if (global_context->areBackgroundExecutorsInitialized())
{
- auto new_pool_size = server_settings_.background_common_pool_size;
+ auto new_pool_size = new_server_settings.background_common_pool_size;
global_context->getCommonExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size);
}
- global_context->getBufferFlushSchedulePool().increaseThreadsCount(server_settings_.background_buffer_flush_schedule_pool_size);
- global_context->getSchedulePool().increaseThreadsCount(server_settings_.background_schedule_pool_size);
- global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings_.background_message_broker_schedule_pool_size);
- global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings_.background_distributed_schedule_pool_size);
+ global_context->getBufferFlushSchedulePool().increaseThreadsCount(new_server_settings.background_buffer_flush_schedule_pool_size);
+ global_context->getSchedulePool().increaseThreadsCount(new_server_settings.background_schedule_pool_size);
+ global_context->getMessageBrokerSchedulePool().increaseThreadsCount(new_server_settings.background_message_broker_schedule_pool_size);
+ global_context->getDistributedSchedulePool().increaseThreadsCount(new_server_settings.background_distributed_schedule_pool_size);
- global_context->getAsyncLoader().setMaxThreads(TablesLoaderForegroundPoolId, server_settings_.tables_loader_foreground_pool_size);
- global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundLoadPoolId, server_settings_.tables_loader_background_pool_size);
- global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundStartupPoolId, server_settings_.tables_loader_background_pool_size);
+ global_context->getAsyncLoader().setMaxThreads(TablesLoaderForegroundPoolId, new_server_settings.tables_loader_foreground_pool_size);
+ global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundLoadPoolId, new_server_settings.tables_loader_background_pool_size);
+ global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundStartupPoolId, new_server_settings.tables_loader_background_pool_size);
getIOThreadPool().reloadConfiguration(
- server_settings.max_io_thread_pool_size,
- server_settings.max_io_thread_pool_free_size,
- server_settings.io_thread_pool_queue_size);
+ new_server_settings.max_io_thread_pool_size,
+ new_server_settings.max_io_thread_pool_free_size,
+ new_server_settings.io_thread_pool_queue_size);
getBackupsIOThreadPool().reloadConfiguration(
- server_settings.max_backups_io_thread_pool_size,
- server_settings.max_backups_io_thread_pool_free_size,
- server_settings.backups_io_thread_pool_queue_size);
+ new_server_settings.max_backups_io_thread_pool_size,
+ new_server_settings.max_backups_io_thread_pool_free_size,
+ new_server_settings.backups_io_thread_pool_queue_size);
getActivePartsLoadingThreadPool().reloadConfiguration(
- server_settings.max_active_parts_loading_thread_pool_size,
+ new_server_settings.max_active_parts_loading_thread_pool_size,
0, // We don't need any threads once all the parts will be loaded
- server_settings.max_active_parts_loading_thread_pool_size);
+ new_server_settings.max_active_parts_loading_thread_pool_size);
getOutdatedPartsLoadingThreadPool().reloadConfiguration(
- server_settings.max_outdated_parts_loading_thread_pool_size,
+ new_server_settings.max_outdated_parts_loading_thread_pool_size,
0, // We don't need any threads once all the parts will be loaded
- server_settings.max_outdated_parts_loading_thread_pool_size);
+ new_server_settings.max_outdated_parts_loading_thread_pool_size);
/// It could grow if we need to synchronously wait until all the data parts will be loaded.
getOutdatedPartsLoadingThreadPool().setMaxTurboThreads(
- server_settings.max_active_parts_loading_thread_pool_size
+ new_server_settings.max_active_parts_loading_thread_pool_size
);
getPartsCleaningThreadPool().reloadConfiguration(
- server_settings.max_parts_cleaning_thread_pool_size,
+ new_server_settings.max_parts_cleaning_thread_pool_size,
0, // We don't need any threads one all the parts will be deleted
- server_settings.max_parts_cleaning_thread_pool_size);
+ new_server_settings.max_parts_cleaning_thread_pool_size);
if (config->has("resources"))
{
diff --git a/programs/server/config.d/graphite_alternative.xml b/programs/server/config.d/graphite_alternative.xml
deleted file mode 120000
index 400b9e75f1f..00000000000
--- a/programs/server/config.d/graphite_alternative.xml
+++ /dev/null
@@ -1 +0,0 @@
-../../../tests/config/config.d/graphite_alternative.xml
\ No newline at end of file
diff --git a/programs/server/config.xml b/programs/server/config.xml
index 52a1c528040..e1428b17084 100644
--- a/programs/server/config.xml
+++ b/programs/server/config.xml
@@ -713,11 +713,11 @@
For example, if there two users A, B and a row policy is defined only for A, then
if this setting is true the user B will see all rows, and if this setting is false the user B will see no rows.
By default this setting is false for compatibility with earlier access configurations. -->
- false
+ true
- false
+ true
- false
+ true
- false
+ true
- false
+ true
600
@@ -1379,6 +1379,9 @@
+
+
+
diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html
index 79ca983ae7f..04fdfb2d3ca 100644
--- a/programs/server/dashboard.html
+++ b/programs/server/dashboard.html
@@ -965,12 +965,10 @@ document.getElementById('mass-editor-textarea').addEventListener('input', e => {
function legendAsTooltipPlugin({ className, style = { background: "var(--legend-background)" } } = {}) {
let legendEl;
- let showTop = false;
- const showLimit = 5;
+ let multiline;
function init(u, opts) {
legendEl = u.root.querySelector(".u-legend");
-
legendEl.classList.remove("u-inline");
className && legendEl.classList.add(className);
@@ -986,18 +984,19 @@ function legendAsTooltipPlugin({ className, style = { background: "var(--legend-
...style
});
+ const nodes = legendEl.querySelectorAll("th");
+ for (let i = 0; i < nodes.length; i++)
+ nodes[i]._order = i;
+
if (opts.series.length == 2) {
- const nodes = legendEl.querySelectorAll("th");
+ multiline = false;
for (let i = 0; i < nodes.length; i++)
nodes[i].style.display = "none";
} else {
+ multiline = true;
legendEl.querySelector("th").remove();
legendEl.querySelector("td").setAttribute('colspan', '2');
legendEl.querySelector("td").style.textAlign = 'center';
- }
-
- if (opts.series.length - 1 > showLimit) {
- showTop = true;
let footer = legendEl.insertRow().insertCell();
footer.setAttribute('colspan', '2');
footer.style.textAlign = 'center';
@@ -1024,18 +1023,20 @@ function legendAsTooltipPlugin({ className, style = { background: "var(--legend-
left -= legendEl.clientWidth / 2;
top -= legendEl.clientHeight / 2;
legendEl.style.transform = "translate(" + left + "px, " + top + "px)";
- if (showTop) {
+
+ if (multiline) {
let nodes = nodeListToArray(legendEl.querySelectorAll("tr"));
let header = nodes.shift();
let footer = nodes.pop();
- nodes.forEach(function (node) { node._sort_key = +node.querySelector("td").textContent; });
- nodes.sort((a, b) => +b._sort_key - +a._sort_key);
+ let showLimit = Math.floor(u.height / 30);
+ nodes.forEach(function (node) { node._sort_key = nodes.length > showLimit ? +node.querySelector("td").textContent.replace(/,/g,'') : node._order; });
+ nodes.sort((a, b) => b._sort_key - a._sort_key);
nodes.forEach(function (node) { node.parentNode.appendChild(node); });
for (let i = 0; i < nodes.length; i++) {
nodes[i].style.display = i < showLimit ? null : "none";
- delete nodes[i]._sort_key;
}
footer.parentNode.appendChild(footer);
+ footer.style.display = nodes.length > showLimit ? null : "none";
}
}
diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h
index 45d427a7c55..463be6a3aea 100644
--- a/src/Access/Common/AccessType.h
+++ b/src/Access/Common/AccessType.h
@@ -82,7 +82,8 @@ enum class AccessType
\
M(ALTER_VIEW_REFRESH, "ALTER LIVE VIEW REFRESH, REFRESH VIEW", VIEW, ALTER_VIEW) \
M(ALTER_VIEW_MODIFY_QUERY, "ALTER TABLE MODIFY QUERY", VIEW, ALTER_VIEW) \
- M(ALTER_VIEW, "", GROUP, ALTER) /* allows to execute ALTER VIEW REFRESH, ALTER VIEW MODIFY QUERY;
+ M(ALTER_VIEW_MODIFY_REFRESH, "ALTER TABLE MODIFY QUERY", VIEW, ALTER_VIEW) \
+ M(ALTER_VIEW, "", GROUP, ALTER) /* allows to execute ALTER VIEW REFRESH, ALTER VIEW MODIFY QUERY, ALTER VIEW MODIFY REFRESH;
implicitly enabled by the grant ALTER_TABLE */\
\
M(ALTER, "", GROUP, ALL) /* allows to execute ALTER {TABLE|LIVE VIEW} */\
@@ -177,6 +178,7 @@ enum class AccessType
M(SYSTEM_MOVES, "SYSTEM STOP MOVES, SYSTEM START MOVES, STOP MOVES, START MOVES", TABLE, SYSTEM) \
M(SYSTEM_PULLING_REPLICATION_LOG, "SYSTEM STOP PULLING REPLICATION LOG, SYSTEM START PULLING REPLICATION LOG", TABLE, SYSTEM) \
M(SYSTEM_CLEANUP, "SYSTEM STOP CLEANUP, SYSTEM START CLEANUP", TABLE, SYSTEM) \
+ M(SYSTEM_VIEWS, "SYSTEM REFRESH VIEW, SYSTEM START VIEWS, SYSTEM STOP VIEWS, SYSTEM START VIEW, SYSTEM STOP VIEW, SYSTEM CANCEL VIEW, REFRESH VIEW, START VIEWS, STOP VIEWS, START VIEW, STOP VIEW, CANCEL VIEW", VIEW, SYSTEM) \
M(SYSTEM_DISTRIBUTED_SENDS, "SYSTEM STOP DISTRIBUTED SENDS, SYSTEM START DISTRIBUTED SENDS, STOP DISTRIBUTED SENDS, START DISTRIBUTED SENDS", TABLE, SYSTEM_SENDS) \
M(SYSTEM_REPLICATED_SENDS, "SYSTEM STOP REPLICATED SENDS, SYSTEM START REPLICATED SENDS, STOP REPLICATED SENDS, START REPLICATED SENDS", TABLE, SYSTEM_SENDS) \
M(SYSTEM_SENDS, "SYSTEM STOP SENDS, SYSTEM START SENDS, STOP SENDS, START SENDS", GROUP, SYSTEM) \
diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp
index 90fddd0085d..567b131c00e 100644
--- a/src/Access/ContextAccess.cpp
+++ b/src/Access/ContextAccess.cpp
@@ -155,6 +155,7 @@ namespace
"formats",
"privileges",
"data_type_families",
+ "database_engines",
"table_engines",
"table_functions",
"aggregate_function_combinators",
diff --git a/src/Access/SettingsProfilesCache.cpp b/src/Access/SettingsProfilesCache.cpp
index f03e68ba455..275b3aeb6b5 100644
--- a/src/Access/SettingsProfilesCache.cpp
+++ b/src/Access/SettingsProfilesCache.cpp
@@ -140,8 +140,7 @@ void SettingsProfilesCache::mergeSettingsAndConstraintsFor(EnabledSettings & ena
auto info = std::make_shared(access_control);
- info->profiles = merged_settings.toProfileIDs();
- substituteProfiles(merged_settings, info->profiles_with_implicit, info->names_of_profiles);
+ substituteProfiles(merged_settings, info->profiles, info->profiles_with_implicit, info->names_of_profiles);
info->settings = merged_settings.toSettingsChanges();
info->constraints = merged_settings.toSettingsConstraints(access_control);
@@ -152,9 +151,12 @@ void SettingsProfilesCache::mergeSettingsAndConstraintsFor(EnabledSettings & ena
void SettingsProfilesCache::substituteProfiles(
SettingsProfileElements & elements,
+ std::vector & profiles,
std::vector & substituted_profiles,
std::unordered_map & names_of_substituted_profiles) const
{
+ profiles = elements.toProfileIDs();
+
/// We should substitute profiles in reversive order because the same profile can occur
/// in `elements` multiple times (with some other settings in between) and in this case
/// the last occurrence should override all the previous ones.
@@ -184,6 +186,11 @@ void SettingsProfilesCache::substituteProfiles(
names_of_substituted_profiles.emplace(profile_id, profile->getName());
}
std::reverse(substituted_profiles.begin(), substituted_profiles.end());
+
+ std::erase_if(profiles, [&substituted_profiles_set](const UUID & profile_id)
+ {
+ return !substituted_profiles_set.contains(profile_id);
+ });
}
std::shared_ptr SettingsProfilesCache::getEnabledSettings(
@@ -225,13 +232,13 @@ std::shared_ptr SettingsProfilesCache::getSettingsPr
if (auto pos = this->profile_infos_cache.get(profile_id))
return *pos;
- SettingsProfileElements elements = all_profiles[profile_id]->elements;
+ SettingsProfileElements elements;
+ auto & element = elements.emplace_back();
+ element.parent_profile = profile_id;
auto info = std::make_shared(access_control);
- info->profiles.push_back(profile_id);
- info->profiles_with_implicit.push_back(profile_id);
- substituteProfiles(elements, info->profiles_with_implicit, info->names_of_profiles);
+ substituteProfiles(elements, info->profiles, info->profiles_with_implicit, info->names_of_profiles);
info->settings = elements.toSettingsChanges();
info->constraints.merge(elements.toSettingsConstraints(access_control));
diff --git a/src/Access/SettingsProfilesCache.h b/src/Access/SettingsProfilesCache.h
index 28914596ccc..afc3c3e13a5 100644
--- a/src/Access/SettingsProfilesCache.h
+++ b/src/Access/SettingsProfilesCache.h
@@ -37,7 +37,11 @@ private:
void profileRemoved(const UUID & profile_id);
void mergeSettingsAndConstraints();
void mergeSettingsAndConstraintsFor(EnabledSettings & enabled) const;
- void substituteProfiles(SettingsProfileElements & elements, std::vector & substituted_profiles, std::unordered_map & names_of_substituted_profiles) const;
+
+ void substituteProfiles(SettingsProfileElements & elements,
+ std::vector & profiles,
+ std::vector & substituted_profiles,
+ std::unordered_map & names_of_substituted_profiles) const;
const AccessControl & access_control;
std::unordered_map all_profiles;
diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp
index b5a15513a89..a7594503992 100644
--- a/src/Access/tests/gtest_access_rights_ops.cpp
+++ b/src/Access/tests/gtest_access_rights_ops.cpp
@@ -51,7 +51,7 @@ TEST(AccessRights, Union)
"CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, "
"TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, "
"SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, "
- "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, "
+ "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, "
"SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, "
"SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*, GRANT NAMED COLLECTION ADMIN ON db1");
}
diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp
index b95471df90a..6c6397e35d5 100644
--- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp
+++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp
@@ -20,6 +20,7 @@
#include
#include
+#include
#include
diff --git a/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp
index 850a7c688ad..d5abdbc12fb 100644
--- a/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp
+++ b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp
@@ -14,8 +14,9 @@
#include
#include
#include
-#include
#include
+#include
+#include
#include
#include
@@ -48,7 +49,7 @@ struct LargestTriangleThreeBucketsData : public StatisticalSamplex and this->y in ascending order of this->x using index
std::vector index(this->x.size());
- std::iota(index.begin(), index.end(), 0);
+ iota(index.data(), index.size(), size_t(0));
::sort(index.begin(), index.end(), [&](size_t i1, size_t i2) { return this->x[i1] < this->x[i2]; });
SampleX temp_x{};
diff --git a/src/AggregateFunctions/AggregateFunctionMax.cpp b/src/AggregateFunctions/AggregateFunctionMax.cpp
index 813129e42ec..e9cd651b8db 100644
--- a/src/AggregateFunctions/AggregateFunctionMax.cpp
+++ b/src/AggregateFunctions/AggregateFunctionMax.cpp
@@ -1,7 +1,8 @@
#include
-#include
#include
-
+#include
+#include
+#include
namespace DB
{
@@ -10,10 +11,211 @@ struct Settings;
namespace
{
+template
+class AggregateFunctionsSingleValueMax final : public AggregateFunctionsSingleValue
+{
+ using Parent = AggregateFunctionsSingleValue;
+
+public:
+ explicit AggregateFunctionsSingleValueMax(const DataTypePtr & type) : Parent(type) { }
+
+ /// Specializations for native numeric types
+ void addBatchSinglePlace(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** __restrict columns,
+ Arena * arena,
+ ssize_t if_argument_pos) const override;
+
+ void addBatchSinglePlaceNotNull(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** __restrict columns,
+ const UInt8 * __restrict null_map,
+ Arena * arena,
+ ssize_t if_argument_pos) const override;
+};
+
+// NOLINTBEGIN(bugprone-macro-parentheses)
+#define SPECIALIZE(TYPE) \
+template <> \
+void AggregateFunctionsSingleValueMax>>::addBatchSinglePlace( \
+ size_t row_begin, \
+ size_t row_end, \
+ AggregateDataPtr __restrict place, \
+ const IColumn ** __restrict columns, \
+ Arena *, \
+ ssize_t if_argument_pos) const \
+{ \
+ const auto & column = assert_cast>::ColVecType &>(*columns[0]); \
+ std::optional opt; \
+ if (if_argument_pos >= 0) \
+ { \
+ const auto & flags = assert_cast(*columns[if_argument_pos]).getData(); \
+ opt = findExtremeMaxIf(column.getData().data(), flags.data(), row_begin, row_end); \
+ } \
+ else \
+ opt = findExtremeMax(column.getData().data(), row_begin, row_end); \
+ if (opt.has_value()) \
+ this->data(place).changeIfGreater(opt.value()); \
+}
+// NOLINTEND(bugprone-macro-parentheses)
+
+FOR_BASIC_NUMERIC_TYPES(SPECIALIZE)
+#undef SPECIALIZE
+
+template
+void AggregateFunctionsSingleValueMax::addBatchSinglePlace(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** __restrict columns,
+ Arena * arena,
+ ssize_t if_argument_pos) const
+{
+ if constexpr (!is_any_of)
+ {
+ /// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's
+ /// faster than doing a permutation
+ return Parent::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos);
+ }
+
+ constexpr int nan_direction_hint = 1;
+ auto const & column = *columns[0];
+ if (if_argument_pos >= 0)
+ {
+ size_t index = row_begin;
+ const auto & if_flags = assert_cast(*columns[if_argument_pos]).getData();
+ while (if_flags[index] == 0 && index < row_end)
+ index++;
+ if (index >= row_end)
+ return;
+
+ for (size_t i = index + 1; i < row_end; i++)
+ {
+ if ((if_flags[i] != 0) && (column.compareAt(i, index, column, nan_direction_hint) > 0))
+ index = i;
+ }
+ this->data(place).changeIfGreater(column, index, arena);
+ }
+ else
+ {
+ if (row_begin >= row_end)
+ return;
+
+ /// TODO: Introduce row_begin and row_end to getPermutation
+ if (row_begin != 0 || row_end != column.size())
+ {
+ size_t index = row_begin;
+ for (size_t i = index + 1; i < row_end; i++)
+ {
+ if (column.compareAt(i, index, column, nan_direction_hint) > 0)
+ index = i;
+ }
+ this->data(place).changeIfGreater(column, index, arena);
+ }
+ else
+ {
+ constexpr IColumn::PermutationSortDirection direction = IColumn::PermutationSortDirection::Descending;
+ constexpr IColumn::PermutationSortStability stability = IColumn::PermutationSortStability::Unstable;
+ IColumn::Permutation permutation;
+ constexpr UInt64 limit = 1;
+ column.getPermutation(direction, stability, limit, nan_direction_hint, permutation);
+ this->data(place).changeIfGreater(column, permutation[0], arena);
+ }
+ }
+}
+
+// NOLINTBEGIN(bugprone-macro-parentheses)
+#define SPECIALIZE(TYPE) \
+template <> \
+void AggregateFunctionsSingleValueMax>>::addBatchSinglePlaceNotNull( \
+ size_t row_begin, \
+ size_t row_end, \
+ AggregateDataPtr __restrict place, \
+ const IColumn ** __restrict columns, \
+ const UInt8 * __restrict null_map, \
+ Arena *, \
+ ssize_t if_argument_pos) const \
+{ \
+ const auto & column = assert_cast>::ColVecType &>(*columns[0]); \
+ std::optional opt; \
+ if (if_argument_pos >= 0) \
+ { \
+ const auto * if_flags = assert_cast(*columns[if_argument_pos]).getData().data(); \
+ auto final_flags = std::make_unique(row_end); \
+ for (size_t i = row_begin; i < row_end; ++i) \
+ final_flags[i] = (!null_map[i]) & !!if_flags[i]; \
+ opt = findExtremeMaxIf(column.getData().data(), final_flags.get(), row_begin, row_end); \
+ } \
+ else \
+ opt = findExtremeMaxNotNull(column.getData().data(), null_map, row_begin, row_end); \
+ if (opt.has_value()) \
+ this->data(place).changeIfGreater(opt.value()); \
+}
+// NOLINTEND(bugprone-macro-parentheses)
+
+FOR_BASIC_NUMERIC_TYPES(SPECIALIZE)
+#undef SPECIALIZE
+
+template
+void AggregateFunctionsSingleValueMax::addBatchSinglePlaceNotNull(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** __restrict columns,
+ const UInt8 * __restrict null_map,
+ Arena * arena,
+ ssize_t if_argument_pos) const
+{
+ if constexpr (!is_any_of)
+ {
+ /// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's
+ /// faster than doing a permutation
+ return Parent::addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, if_argument_pos);
+ }
+
+ constexpr int nan_direction_hint = 1;
+ auto const & column = *columns[0];
+ if (if_argument_pos >= 0)
+ {
+ size_t index = row_begin;
+ const auto & if_flags = assert_cast(*columns[if_argument_pos]).getData();
+ while ((if_flags[index] == 0 || null_map[index] != 0) && (index < row_end))
+ index++;
+ if (index >= row_end)
+ return;
+
+ for (size_t i = index + 1; i < row_end; i++)
+ {
+ if ((if_flags[i] != 0) && (null_map[i] == 0) && (column.compareAt(i, index, column, nan_direction_hint) > 0))
+ index = i;
+ }
+ this->data(place).changeIfGreater(column, index, arena);
+ }
+ else
+ {
+ size_t index = row_begin;
+ while ((null_map[index] != 0) && (index < row_end))
+ index++;
+ if (index >= row_end)
+ return;
+
+ for (size_t i = index + 1; i < row_end; i++)
+ {
+ if ((null_map[i] == 0) && (column.compareAt(i, index, column, nan_direction_hint) > 0))
+ index = i;
+ }
+ this->data(place).changeIfGreater(column, index, arena);
+ }
+}
+
AggregateFunctionPtr createAggregateFunctionMax(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
- return AggregateFunctionPtr(createAggregateFunctionSingleValue(name, argument_types, parameters, settings));
+ return AggregateFunctionPtr(createAggregateFunctionSingleValue(name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionArgMax(
diff --git a/src/AggregateFunctions/AggregateFunctionMin.cpp b/src/AggregateFunctions/AggregateFunctionMin.cpp
index ac3e05121f7..d767bd5c563 100644
--- a/src/AggregateFunctions/AggregateFunctionMin.cpp
+++ b/src/AggregateFunctions/AggregateFunctionMin.cpp
@@ -1,6 +1,8 @@
#include
-#include
#include
+#include
+#include
+#include
namespace DB
@@ -10,10 +12,212 @@ struct Settings;
namespace
{
+template
+class AggregateFunctionsSingleValueMin final : public AggregateFunctionsSingleValue
+{
+ using Parent = AggregateFunctionsSingleValue;
+
+public:
+ explicit AggregateFunctionsSingleValueMin(const DataTypePtr & type) : Parent(type) { }
+
+ /// Specializations for native numeric types
+ void addBatchSinglePlace(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** __restrict columns,
+ Arena * arena,
+ ssize_t if_argument_pos) const override;
+
+ void addBatchSinglePlaceNotNull(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** __restrict columns,
+ const UInt8 * __restrict null_map,
+ Arena * arena,
+ ssize_t if_argument_pos) const override;
+};
+
+// NOLINTBEGIN(bugprone-macro-parentheses)
+#define SPECIALIZE(TYPE) \
+ template <> \
+ void AggregateFunctionsSingleValueMin>>::addBatchSinglePlace( \
+ size_t row_begin, \
+ size_t row_end, \
+ AggregateDataPtr __restrict place, \
+ const IColumn ** __restrict columns, \
+ Arena *, \
+ ssize_t if_argument_pos) const \
+ { \
+ const auto & column = assert_cast>::ColVecType &>(*columns[0]); \
+ std::optional opt; \
+ if (if_argument_pos >= 0) \
+ { \
+ const auto & flags = assert_cast(*columns[if_argument_pos]).getData(); \
+ opt = findExtremeMinIf(column.getData().data(), flags.data(), row_begin, row_end); \
+ } \
+ else \
+ opt = findExtremeMin(column.getData().data(), row_begin, row_end); \
+ if (opt.has_value()) \
+ this->data(place).changeIfLess(opt.value()); \
+ }
+// NOLINTEND(bugprone-macro-parentheses)
+
+FOR_BASIC_NUMERIC_TYPES(SPECIALIZE)
+#undef SPECIALIZE
+
+template
+void AggregateFunctionsSingleValueMin::addBatchSinglePlace(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** __restrict columns,
+ Arena * arena,
+ ssize_t if_argument_pos) const
+{
+ if constexpr (!is_any_of)
+ {
+ /// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's
+ /// faster than doing a permutation
+ return Parent::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos);
+ }
+
+ constexpr int nan_direction_hint = 1;
+ auto const & column = *columns[0];
+ if (if_argument_pos >= 0)
+ {
+ size_t index = row_begin;
+ const auto & if_flags = assert_cast(*columns[if_argument_pos]).getData();
+ while (if_flags[index] == 0 && index < row_end)
+ index++;
+ if (index >= row_end)
+ return;
+
+ for (size_t i = index + 1; i < row_end; i++)
+ {
+ if ((if_flags[i] != 0) && (column.compareAt(i, index, column, nan_direction_hint) < 0))
+ index = i;
+ }
+ this->data(place).changeIfLess(column, index, arena);
+ }
+ else
+ {
+ if (row_begin >= row_end)
+ return;
+
+ /// TODO: Introduce row_begin and row_end to getPermutation
+ if (row_begin != 0 || row_end != column.size())
+ {
+ size_t index = row_begin;
+ for (size_t i = index + 1; i < row_end; i++)
+ {
+ if (column.compareAt(i, index, column, nan_direction_hint) < 0)
+ index = i;
+ }
+ this->data(place).changeIfLess(column, index, arena);
+ }
+ else
+ {
+ constexpr IColumn::PermutationSortDirection direction = IColumn::PermutationSortDirection::Ascending;
+ constexpr IColumn::PermutationSortStability stability = IColumn::PermutationSortStability::Unstable;
+ IColumn::Permutation permutation;
+ constexpr UInt64 limit = 1;
+ column.getPermutation(direction, stability, limit, nan_direction_hint, permutation);
+ this->data(place).changeIfLess(column, permutation[0], arena);
+ }
+ }
+}
+
+// NOLINTBEGIN(bugprone-macro-parentheses)
+#define SPECIALIZE(TYPE) \
+ template <> \
+ void AggregateFunctionsSingleValueMin>>::addBatchSinglePlaceNotNull( \
+ size_t row_begin, \
+ size_t row_end, \
+ AggregateDataPtr __restrict place, \
+ const IColumn ** __restrict columns, \
+ const UInt8 * __restrict null_map, \
+ Arena *, \
+ ssize_t if_argument_pos) const \
+ { \
+ const auto & column = assert_cast>::ColVecType &>(*columns[0]); \
+ std::optional opt; \
+ if (if_argument_pos >= 0) \
+ { \
+ const auto * if_flags = assert_cast(*columns[if_argument_pos]).getData().data(); \
+ auto final_flags = std::make_unique(row_end); \
+ for (size_t i = row_begin; i < row_end; ++i) \
+ final_flags[i] = (!null_map[i]) & !!if_flags[i]; \
+ opt = findExtremeMinIf(column.getData().data(), final_flags.get(), row_begin, row_end); \
+ } \
+ else \
+ opt = findExtremeMinNotNull(column.getData().data(), null_map, row_begin, row_end); \
+ if (opt.has_value()) \
+ this->data(place).changeIfLess(opt.value()); \
+ }
+// NOLINTEND(bugprone-macro-parentheses)
+
+FOR_BASIC_NUMERIC_TYPES(SPECIALIZE)
+#undef SPECIALIZE
+
+template
+void AggregateFunctionsSingleValueMin::addBatchSinglePlaceNotNull(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** __restrict columns,
+ const UInt8 * __restrict null_map,
+ Arena * arena,
+ ssize_t if_argument_pos) const
+{
+ if constexpr (!is_any_of)
+ {
+ /// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's
+ /// faster than doing a permutation
+ return Parent::addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, if_argument_pos);
+ }
+
+ constexpr int nan_direction_hint = 1;
+ auto const & column = *columns[0];
+ if (if_argument_pos >= 0)
+ {
+ size_t index = row_begin;
+ const auto & if_flags = assert_cast(*columns[if_argument_pos]).getData();
+ while ((if_flags[index] == 0 || null_map[index] != 0) && (index < row_end))
+ index++;
+ if (index >= row_end)
+ return;
+
+ for (size_t i = index + 1; i < row_end; i++)
+ {
+ if ((if_flags[i] != 0) && (null_map[index] == 0) && (column.compareAt(i, index, column, nan_direction_hint) < 0))
+ index = i;
+ }
+ this->data(place).changeIfLess(column, index, arena);
+ }
+ else
+ {
+ size_t index = row_begin;
+ while ((null_map[index] != 0) && (index < row_end))
+ index++;
+ if (index >= row_end)
+ return;
+
+ for (size_t i = index + 1; i < row_end; i++)
+ {
+ if ((null_map[i] == 0) && (column.compareAt(i, index, column, nan_direction_hint) < 0))
+ index = i;
+ }
+ this->data(place).changeIfLess(column, index, arena);
+ }
+}
+
AggregateFunctionPtr createAggregateFunctionMin(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
- return AggregateFunctionPtr(createAggregateFunctionSingleValue(name, argument_types, parameters, settings));
+ return AggregateFunctionPtr(createAggregateFunctionSingleValue(
+ name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionArgMin(
diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h
index ef1de76df79..dec70861543 100644
--- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h
+++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h
@@ -43,14 +43,12 @@ namespace ErrorCodes
template
struct SingleValueDataFixed
{
-private:
using Self = SingleValueDataFixed;
using ColVecType = ColumnVectorOrDecimal;
bool has_value = false; /// We need to remember if at least one value has been passed. This is necessary for AggregateFunctionIf.
T value = T{};
-public:
static constexpr bool result_is_nullable = false;
static constexpr bool should_skip_null_arguments = true;
static constexpr bool is_any = false;
@@ -157,6 +155,15 @@ public:
return false;
}
+ void changeIfLess(T from)
+ {
+ if (!has() || from < value)
+ {
+ has_value = true;
+ value = from;
+ }
+ }
+
bool changeIfGreater(const IColumn & column, size_t row_num, Arena * arena)
{
if (!has() || assert_cast(column).getData()[row_num] > value)
@@ -179,6 +186,15 @@ public:
return false;
}
+ void changeIfGreater(T & from)
+ {
+ if (!has() || from > value)
+ {
+ has_value = true;
+ value = from;
+ }
+ }
+
bool isEqualTo(const Self & to) const
{
return has() && to.value == value;
@@ -448,7 +464,6 @@ public:
}
#endif
-
};
struct Compatibility
@@ -950,6 +965,7 @@ template
struct AggregateFunctionMinData : Data
{
using Self = AggregateFunctionMinData;
+ using Impl = Data;
bool changeIfBetter(const IColumn & column, size_t row_num, Arena * arena) { return this->changeIfLess(column, row_num, arena); }
bool changeIfBetter(const Self & to, Arena * arena) { return this->changeIfLess(to, arena); }
@@ -978,6 +994,7 @@ template
struct AggregateFunctionMaxData : Data
{
using Self = AggregateFunctionMaxData;
+ using Impl = Data;
bool changeIfBetter(const IColumn & column, size_t row_num, Arena * arena) { return this->changeIfGreater(column, row_num, arena); }
bool changeIfBetter(const Self & to, Arena * arena) { return this->changeIfGreater(to, arena); }
@@ -1214,7 +1231,7 @@ struct AggregateFunctionAnyHeavyData : Data
template
-class AggregateFunctionsSingleValue final : public IAggregateFunctionDataHelper>
+class AggregateFunctionsSingleValue : public IAggregateFunctionDataHelper>
{
static constexpr bool is_any = Data::is_any;
@@ -1230,8 +1247,11 @@ public:
|| StringRef(Data::name()) == StringRef("max"))
{
if (!type->isComparable())
- throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of aggregate function {} "
- "because the values of that data type are not comparable", type->getName(), getName());
+ throw Exception(
+ ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
+ "Illegal type {} of argument of aggregate function {} because the values of that data type are not comparable",
+ type->getName(),
+ Data::name());
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionSparkbar.cpp b/src/AggregateFunctions/AggregateFunctionSparkbar.cpp
index 7ff9df03824..b6e538520a8 100644
--- a/src/AggregateFunctions/AggregateFunctionSparkbar.cpp
+++ b/src/AggregateFunctions/AggregateFunctionSparkbar.cpp
@@ -378,6 +378,7 @@ AggregateFunctionPtr createAggregateFunctionSparkbar(const std::string & name, c
void registerAggregateFunctionSparkbar(AggregateFunctionFactory & factory)
{
factory.registerFunction("sparkbar", createAggregateFunctionSparkbar);
+ factory.registerAlias("sparkBar", "sparkbar");
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionSum.h b/src/AggregateFunctions/AggregateFunctionSum.h
index b3006f2ce82..5781ab69c6b 100644
--- a/src/AggregateFunctions/AggregateFunctionSum.h
+++ b/src/AggregateFunctions/AggregateFunctionSum.h
@@ -504,7 +504,7 @@ public:
const auto * if_flags = assert_cast(*columns[if_argument_pos]).getData().data();
auto final_flags = std::make_unique(row_end);
for (size_t i = row_begin; i < row_end; ++i)
- final_flags[i] = (!null_map[i]) & if_flags[i];
+ final_flags[i] = (!null_map[i]) & !!if_flags[i];
this->data(place).addManyConditional(column.getData().data(), final_flags.get(), row_begin, row_end);
}
diff --git a/src/AggregateFunctions/QuantilesCommon.h b/src/AggregateFunctions/QuantilesCommon.h
index 3dda0119485..afbca84b827 100644
--- a/src/AggregateFunctions/QuantilesCommon.h
+++ b/src/AggregateFunctions/QuantilesCommon.h
@@ -6,6 +6,7 @@
#include
#include
+#include
namespace DB
@@ -63,10 +64,9 @@ struct QuantileLevels
if (isNaN(levels[i]) || levels[i] < 0 || levels[i] > 1)
throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Quantile level is out of range [0..1]");
-
- permutation[i] = i;
}
+ iota(permutation.data(), size, Permutation::value_type(0));
::sort(permutation.begin(), permutation.end(), [this] (size_t a, size_t b) { return levels[a] < levels[b]; });
}
};
diff --git a/src/AggregateFunctions/StatCommon.h b/src/AggregateFunctions/StatCommon.h
index 23054e25189..8b1395ea95c 100644
--- a/src/AggregateFunctions/StatCommon.h
+++ b/src/AggregateFunctions/StatCommon.h
@@ -7,6 +7,7 @@
#include
#include
+#include
#include
#include
@@ -30,7 +31,7 @@ std::pair computeRanksAndTieCorrection(const Values & value
const size_t size = values.size();
/// Save initial positions, than sort indices according to the values.
std::vector indexes(size);
- std::iota(indexes.begin(), indexes.end(), 0);
+ iota(indexes.data(), indexes.size(), size_t(0));
std::sort(indexes.begin(), indexes.end(),
[&] (size_t lhs, size_t rhs) { return values[lhs] < values[rhs]; });
diff --git a/src/AggregateFunctions/examples/quantile-t-digest.cpp b/src/AggregateFunctions/examples/quantile-t-digest.cpp
index b4e58e6203c..5360304b311 100644
--- a/src/AggregateFunctions/examples/quantile-t-digest.cpp
+++ b/src/AggregateFunctions/examples/quantile-t-digest.cpp
@@ -1,6 +1,7 @@
#include
#include
#include
+#include
int main(int, char **)
{
diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h
index 922eaabe75c..b07aa2d31b0 100644
--- a/src/Analyzer/IQueryTreeNode.h
+++ b/src/Analyzer/IQueryTreeNode.h
@@ -143,9 +143,17 @@ public:
return alias;
}
+ const String & getOriginalAlias() const
+ {
+ return original_alias.empty() ? alias : original_alias;
+ }
+
/// Set node alias
void setAlias(String alias_value)
{
+ if (original_alias.empty())
+ original_alias = std::move(alias);
+
alias = std::move(alias_value);
}
@@ -276,6 +284,9 @@ protected:
private:
String alias;
+ /// An alias from query. Alias can be replaced by query passes,
+ /// but we need to keep the original one to support additional_table_filters.
+ String original_alias;
ASTPtr original_ast;
};
diff --git a/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp b/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp
index 4e0562a2fe8..117e649ac88 100644
--- a/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp
+++ b/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp
@@ -1,6 +1,8 @@
#include
#include
+#include
+#include
#include
@@ -52,6 +54,13 @@ public:
if (!isTuple(rhs_argument_result_type))
return;
+ if (function_node->getResultType()->equals(DataTypeNullable(std::make_shared())))
+ /** The function `equals` can return Nullable(Nothing), e.g., in the case of (a, b) == (NULL, 1).
+ * On the other hand, `AND` returns Nullable(UInt8), so we would need to convert types.
+ * It's better to just skip this trivial case.
+ */
+ return;
+
auto lhs_argument_node_type = lhs_argument->getNodeType();
auto rhs_argument_node_type = rhs_argument->getNodeType();
diff --git a/src/Analyzer/Passes/FuseFunctionsPass.cpp b/src/Analyzer/Passes/FuseFunctionsPass.cpp
index e77b3ddcb20..443e13b7d9d 100644
--- a/src/Analyzer/Passes/FuseFunctionsPass.cpp
+++ b/src/Analyzer/Passes/FuseFunctionsPass.cpp
@@ -1,5 +1,6 @@
#include
+#include
#include
#include
#include
@@ -184,7 +185,7 @@ FunctionNodePtr createFusedQuantilesNode(std::vector & nodes
{
/// Sort nodes and parameters in ascending order of quantile level
std::vector permutation(nodes.size());
- std::iota(permutation.begin(), permutation.end(), 0);
+ iota(permutation.data(), permutation.size(), size_t(0));
std::sort(permutation.begin(), permutation.end(), [&](size_t i, size_t j) { return parameters[i].get() < parameters[j].get(); });
std::vector new_nodes;
diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp
index 371c0a07511..59b3b036698 100644
--- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp
+++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp
@@ -250,13 +250,13 @@ public:
if (function_node->getFunctionName() == "and")
{
- tryReplaceAndEqualsChainsWithConstant(node);
+ tryOptimizeAndEqualsNotEqualsChain(node);
return;
}
}
private:
- void tryReplaceAndEqualsChainsWithConstant(QueryTreeNodePtr & node)
+ void tryOptimizeAndEqualsNotEqualsChain(QueryTreeNodePtr & node)
{
auto & function_node = node->as();
assert(function_node.getFunctionName() == "and");
@@ -266,53 +266,132 @@ private:
QueryTreeNodes and_operands;
- QueryTreeNodePtrWithHashMap node_to_constants;
+ QueryTreeNodePtrWithHashMap equals_node_to_constants;
+ QueryTreeNodePtrWithHashMap not_equals_node_to_constants;
+ QueryTreeNodePtrWithHashMap node_to_not_equals_functions;
for (const auto & argument : function_node.getArguments())
{
auto * argument_function = argument->as();
- if (!argument_function || argument_function->getFunctionName() != "equals")
+ const auto valid_functions = std::unordered_set{"equals", "notEquals"};
+ if (!argument_function || !valid_functions.contains(argument_function->getFunctionName()))
{
and_operands.push_back(argument);
continue;
}
- const auto & equals_arguments = argument_function->getArguments().getNodes();
- const auto & lhs = equals_arguments[0];
- const auto & rhs = equals_arguments[1];
+ const auto function_name = argument_function->getFunctionName();
+ const auto & function_arguments = argument_function->getArguments().getNodes();
+ const auto & lhs = function_arguments[0];
+ const auto & rhs = function_arguments[1];
- const auto has_and_with_different_constant = [&](const QueryTreeNodePtr & expression, const ConstantNode * constant)
+ if (function_name == "equals")
{
- if (auto it = node_to_constants.find(expression); it != node_to_constants.end())
+ const auto has_and_with_different_constant = [&](const QueryTreeNodePtr & expression, const ConstantNode * constant)
{
- if (!it->second->isEqual(*constant))
- return true;
+ if (auto it = equals_node_to_constants.find(expression); it != equals_node_to_constants.end())
+ {
+ if (!it->second->isEqual(*constant))
+ return true;
+ }
+ else
+ {
+ equals_node_to_constants.emplace(expression, constant);
+ and_operands.push_back(argument);
+ }
+
+ return false;
+ };
+
+ bool collapse_to_false = false;
+
+ if (const auto * lhs_literal = lhs->as())
+ collapse_to_false = has_and_with_different_constant(rhs, lhs_literal);
+ else if (const auto * rhs_literal = rhs->as())
+ collapse_to_false = has_and_with_different_constant(lhs, rhs_literal);
+ else
+ and_operands.push_back(argument);
+
+ if (collapse_to_false)
+ {
+ auto false_value = std::make_shared(0u, function_node.getResultType());
+ auto false_node = std::make_shared(std::move(false_value));
+ node = std::move(false_node);
+ return;
+ }
+ }
+ else if (function_name == "notEquals")
+ {
+ /// collect all inequality checks (x <> value)
+
+ const auto add_not_equals_function_if_not_present = [&](const auto & expression_node, const ConstantNode * constant)
+ {
+ auto & constant_set = not_equals_node_to_constants[expression_node];
+ if (!constant_set.contains(constant))
+ {
+ constant_set.insert(constant);
+ node_to_not_equals_functions[expression_node].push_back(argument);
+ }
+ };
+
+ if (const auto * lhs_literal = lhs->as();
+ lhs_literal && !lhs_literal->getValue().isNull())
+ add_not_equals_function_if_not_present(rhs, lhs_literal);
+ else if (const auto * rhs_literal = rhs->as();
+ rhs_literal && !rhs_literal->getValue().isNull())
+ add_not_equals_function_if_not_present(lhs, rhs_literal);
+ else
+ and_operands.push_back(argument);
+ }
+ else
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function name: '{}'", function_name);
+ }
+
+ auto not_in_function_resolver = FunctionFactory::instance().get("notIn", getContext());
+
+ for (auto & [expression, not_equals_functions] : node_to_not_equals_functions)
+ {
+ const auto & settings = getSettings();
+ if (not_equals_functions.size() < settings.optimize_min_inequality_conjunction_chain_length && !expression.node->getResultType()->lowCardinality())
+ {
+ std::move(not_equals_functions.begin(), not_equals_functions.end(), std::back_inserter(and_operands));
+ continue;
+ }
+
+ Tuple args;
+ args.reserve(not_equals_functions.size());
+ /// first we create tuple from RHS of notEquals functions
+ for (const auto & not_equals : not_equals_functions)
+ {
+ const auto * not_equals_function = not_equals->as();
+ assert(not_equals_function && not_equals_function->getFunctionName() == "notEquals");
+
+ const auto & not_equals_arguments = not_equals_function->getArguments().getNodes();
+ if (const auto * rhs_literal = not_equals_arguments[1]->as())
+ {
+ args.push_back(rhs_literal->getValue());
}
else
{
- node_to_constants.emplace(expression, constant);
- and_operands.push_back(argument);
+ const auto * lhs_literal = not_equals_arguments[0]->as();
+ assert(lhs_literal);
+ args.push_back(lhs_literal->getValue());
}
-
- return false;
- };
-
- bool collapse_to_false = false;
-
- if (const auto * lhs_literal = lhs->as())
- collapse_to_false = has_and_with_different_constant(rhs, lhs_literal);
- else if (const auto * rhs_literal = rhs->as())
- collapse_to_false = has_and_with_different_constant(lhs, rhs_literal);
- else
- and_operands.push_back(argument);
-
- if (collapse_to_false)
- {
- auto false_value = std::make_shared(0u, function_node.getResultType());
- auto false_node = std::make_shared(std::move(false_value));
- node = std::move(false_node);
- return;
}
+
+ auto rhs_node = std::make_shared(std::move(args));
+
+ auto not_in_function = std::make_shared("notIn");
+
+ QueryTreeNodes not_in_arguments;
+ not_in_arguments.reserve(2);
+ not_in_arguments.push_back(expression.node);
+ not_in_arguments.push_back(std::move(rhs_node));
+
+ not_in_function->getArguments().getNodes() = std::move(not_in_arguments);
+ not_in_function->resolveAsFunction(not_in_function_resolver);
+
+ and_operands.push_back(std::move(not_in_function));
}
if (and_operands.size() == function_node.getArguments().getNodes().size())
@@ -322,9 +401,6 @@ private:
{
/// AND operator can have UInt8 or bool as its type.
/// bool is used if a bool constant is at least one operand.
- /// Because we reduce the number of operands here by eliminating the same equality checks,
- /// the only situation we can end up here is we had AND check where all the equality checks are the same so we know the type is UInt8.
- /// Otherwise, we will have > 1 operands and we don't have to do anything.
auto operand_type = and_operands[0]->getResultType();
auto function_type = function_node.getResultType();
diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h
index 80062f38eac..658f6d767c4 100644
--- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h
+++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h
@@ -68,7 +68,25 @@ namespace DB
* WHERE a = 1 AND b = 'test';
* -------------------------------
*
- * 5. Remove unnecessary IS NULL checks in JOIN ON clause
+ * 5. Replaces chains of inequality functions inside an AND with a single NOT IN operator.
+ * The replacement is done if:
+ * - one of the operands of the inequality function is a constant
+ * - length of chain is at least 'optimize_min_inequality_conjunction_chain_length' long OR the expression has type of LowCardinality
+ *
+ * E.g. (optimize_min_inequality_conjunction_chain_length = 2)
+ * -------------------------------
+ * SELECT *
+ * FROM table
+ * WHERE a <> 1 AND a <> 2;
+ *
+ * will be transformed into
+ *
+ * SELECT *
+ * FROM TABLE
+ * WHERE a NOT IN (1, 2);
+ * -------------------------------
+ *
+ * 6. Remove unnecessary IS NULL checks in JOIN ON clause
* - equality check with explicit IS NULL check replaced with <=> operator
* -------------------------------
* SELECT * FROM t1 JOIN t2 ON a = b OR (a IS NULL AND b IS NULL)
@@ -85,7 +103,11 @@ class LogicalExpressionOptimizerPass final : public IQueryTreePass
public:
String getName() override { return "LogicalExpressionOptimizer"; }
- String getDescription() override { return "Transform equality chain to a single IN function or a constant if possible"; }
+ String getDescription() override
+ {
+ return "Transforms chains of logical expressions if possible, i.e. "
+ "replace chains of equality functions inside an OR with a single IN operator";
+ }
void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override;
};
diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp
index 3290d918a8b..4ad9581b5b6 100644
--- a/src/Analyzer/Passes/QueryAnalysisPass.cpp
+++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp
@@ -52,6 +52,7 @@
#include
+#include
#include
#include
#include
@@ -1198,7 +1199,7 @@ private:
static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope);
- static void replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope);
+ void replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope);
static void convertLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope);
@@ -2168,7 +2169,12 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_
scope.scope_node->formatASTForErrorMessage());
--positional_argument_number;
- *node_to_replace = projection_nodes[positional_argument_number];
+ *node_to_replace = projection_nodes[positional_argument_number]->clone();
+ if (auto it = resolved_expressions.find(projection_nodes[positional_argument_number]);
+ it != resolved_expressions.end())
+ {
+ resolved_expressions[*node_to_replace] = it->second;
+ }
}
}
@@ -7366,6 +7372,7 @@ void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context
{
QueryAnalyzer analyzer;
analyzer.resolve(query_tree_node, table_expression, context);
+ createUniqueTableAliases(query_tree_node, table_expression, context);
}
}
diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp
index f75022220e7..53fcf534f64 100644
--- a/src/Analyzer/Utils.cpp
+++ b/src/Analyzer/Utils.cpp
@@ -326,7 +326,7 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q
}
}
-QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node)
+QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join)
{
QueryTreeNodes result;
@@ -357,6 +357,8 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node)
{
auto & array_join_node = node_to_process->as();
nodes_to_process.push_front(array_join_node.getTableExpression());
+ if (add_array_join)
+ result.push_back(std::move(node_to_process));
break;
}
case QueryTreeNodeType::JOIN:
diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h
index e3316f5ad6b..d3eb6ba3cc2 100644
--- a/src/Analyzer/Utils.h
+++ b/src/Analyzer/Utils.h
@@ -51,7 +51,7 @@ std::optional tryExtractConstantFromConditionNode(const QueryTreeNodePtr &
void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression, const IQueryTreeNode::ConvertToASTOptions & convert_to_ast_options);
/// Extract table, table function, query, union from join tree
-QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node);
+QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join = false);
/// Extract left table expression from join tree
QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node);
diff --git a/src/Analyzer/createUniqueTableAliases.cpp b/src/Analyzer/createUniqueTableAliases.cpp
new file mode 100644
index 00000000000..8f850fe8dec
--- /dev/null
+++ b/src/Analyzer/createUniqueTableAliases.cpp
@@ -0,0 +1,141 @@
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+
+namespace DB
+{
+
+namespace
+{
+
+class CreateUniqueTableAliasesVisitor : public InDepthQueryTreeVisitorWithContext
+{
+public:
+ using Base = InDepthQueryTreeVisitorWithContext;
+
+ explicit CreateUniqueTableAliasesVisitor(const ContextPtr & context)
+ : Base(context)
+ {
+ // Insert a fake node on top of the stack.
+ scope_nodes_stack.push_back(std::make_shared(Names{}, nullptr));
+ }
+
+ void enterImpl(QueryTreeNodePtr & node)
+ {
+ auto node_type = node->getNodeType();
+
+ switch (node_type)
+ {
+ case QueryTreeNodeType::QUERY:
+ [[fallthrough]];
+ case QueryTreeNodeType::UNION:
+ {
+ /// Queries like `(SELECT 1) as t` have invalid syntax. To avoid creating such queries (e.g. in StorageDistributed)
+ /// we need to remove aliases for top level queries.
+ /// N.B. Subquery depth starts count from 1, so the following condition checks if it's a top level.
+ if (getSubqueryDepth() == 1)
+ {
+ node->removeAlias();
+ break;
+ }
+ [[fallthrough]];
+ }
+ case QueryTreeNodeType::TABLE:
+ [[fallthrough]];
+ case QueryTreeNodeType::TABLE_FUNCTION:
+ [[fallthrough]];
+ case QueryTreeNodeType::ARRAY_JOIN:
+ {
+ auto & alias = table_expression_to_alias[node];
+ if (alias.empty())
+ {
+ scope_to_nodes_with_aliases[scope_nodes_stack.back()].push_back(node);
+ alias = fmt::format("__table{}", ++next_id);
+ node->setAlias(alias);
+ }
+ break;
+ }
+ default:
+ break;
+ }
+
+ switch (node_type)
+ {
+ case QueryTreeNodeType::QUERY:
+ [[fallthrough]];
+ case QueryTreeNodeType::UNION:
+ [[fallthrough]];
+ case QueryTreeNodeType::LAMBDA:
+ scope_nodes_stack.push_back(node);
+ break;
+ default:
+ break;
+ }
+ }
+
+ void leaveImpl(QueryTreeNodePtr & node)
+ {
+ if (scope_nodes_stack.back() == node)
+ {
+ if (auto it = scope_to_nodes_with_aliases.find(scope_nodes_stack.back());
+ it != scope_to_nodes_with_aliases.end())
+ {
+ for (const auto & node_with_alias : it->second)
+ {
+ table_expression_to_alias.erase(node_with_alias);
+ }
+ scope_to_nodes_with_aliases.erase(it);
+ }
+ scope_nodes_stack.pop_back();
+ }
+
+ /// Here we revisit subquery for IN function. Reasons:
+ /// * For remote query execution, query tree may be traversed a few times.
+ /// In such a case, it is possible to get AST like
+ /// `IN ((SELECT ... FROM table AS __table4) AS __table1)` which result in
+ /// `Multiple expressions for the alias` exception
+ /// * Tables in subqueries could have different aliases => different three hashes,
+ /// which is important to be able to find a set in PreparedSets
+ /// See 01253_subquery_in_aggregate_function_JustStranger.
+ ///
+ /// So, we revisit this subquery to make aliases stable.
+ /// This should be safe cause columns from IN subquery can't be used in main query anyway.
+ if (node->getNodeType() == QueryTreeNodeType::FUNCTION)
+ {
+ auto * function_node = node->as();
+ if (isNameOfInFunction(function_node->getFunctionName()))
+ {
+ auto arg = function_node->getArguments().getNodes().back();
+ /// Avoid aliasing IN `table`
+ if (arg->getNodeType() != QueryTreeNodeType::TABLE)
+ CreateUniqueTableAliasesVisitor(getContext()).visit(function_node->getArguments().getNodes().back());
+ }
+ }
+ }
+
+private:
+ size_t next_id = 0;
+
+ // Stack of nodes which create scopes: QUERY, UNION and LAMBDA.
+ QueryTreeNodes scope_nodes_stack;
+
+ std::unordered_map scope_to_nodes_with_aliases;
+
+ // We need to use raw pointer as a key, not a QueryTreeNodePtrWithHash.
+ std::unordered_map table_expression_to_alias;
+};
+
+}
+
+
+void createUniqueTableAliases(QueryTreeNodePtr & node, const QueryTreeNodePtr & /*table_expression*/, const ContextPtr & context)
+{
+ CreateUniqueTableAliasesVisitor(context).visit(node);
+}
+
+}
diff --git a/src/Analyzer/createUniqueTableAliases.h b/src/Analyzer/createUniqueTableAliases.h
new file mode 100644
index 00000000000..d57a198498c
--- /dev/null
+++ b/src/Analyzer/createUniqueTableAliases.h
@@ -0,0 +1,18 @@
+#pragma once
+
+#include