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 +#include + +class IQueryTreeNode; +using QueryTreeNodePtr = std::shared_ptr; + +namespace DB +{ + +/* + * For each table expression in the Query Tree generate and add a unique alias. + * If table expression had an alias in initial query tree, override it. + */ +void createUniqueTableAliases(QueryTreeNodePtr & node, const QueryTreeNodePtr & table_expression, const ContextPtr & context); + +} diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index cedcecfd35c..2eba3440be9 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -154,14 +154,14 @@ BackupCoordinationStageSync::State BackupCoordinationStageSync::readCurrentState /// If the "alive" node doesn't exist then we don't have connection to the corresponding host. /// This node is ephemeral so probably it will be recreated soon. We use zookeeper retries to wait. /// In worst case when we won't manage to see the alive node for a long time we will just abort the backup. - String message; + const auto * const suffix = retries_ctl.isLastRetry() ? "" : ", will retry"; if (started) - message = fmt::format("Lost connection to host {}", host); + retries_ctl.setUserError(Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, + "Lost connection to host {}{}", host, suffix)); else - message = fmt::format("No connection to host {} yet", host); - if (!retries_ctl.isLastRetry()) - message += ", will retry"; - retries_ctl.setUserError(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, message); + retries_ctl.setUserError(Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, + "No connection to host {} yet{}", host, suffix)); + state.disconnected_host = host; return state; } diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 967abe09b3f..d143d813a2f 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -68,12 +68,16 @@ namespace client_configuration.connectTimeoutMs = 10 * 1000; /// Requests in backups can be extremely long, set to one hour client_configuration.requestTimeoutMs = 60 * 60 * 1000; - client_configuration.retryStrategy = std::make_shared(request_settings.retry_attempts); + + S3::ClientSettings client_settings{ + .use_virtual_addressing = s3_uri.is_virtual_hosted_style, + .disable_checksum = local_settings.s3_disable_checksum, + .gcs_issue_compose_request = context->getConfigRef().getBool("s3.gcs_issue_compose_request", false), + }; return S3::ClientFactory::instance().create( client_configuration, - s3_uri.is_virtual_hosted_style, - local_settings.s3_disable_checksum, + client_settings, credentials.GetAWSAccessKeyId(), credentials.GetAWSSecretKey(), settings.auth_settings.server_side_encryption_customer_key_base64, diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index a87f8520138..8c4bb7e414c 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -610,7 +610,6 @@ void BackupsWorker::doBackup( void BackupsWorker::buildFileInfosForBackupEntries(const BackupPtr & backup, const BackupEntries & backup_entries, const ReadSettings & read_settings, std::shared_ptr backup_coordination) { - LOG_TRACE(log, "{}", Stage::BUILDING_FILE_INFOS); backup_coordination->setStage(Stage::BUILDING_FILE_INFOS, ""); backup_coordination->waitForStage(Stage::BUILDING_FILE_INFOS); backup_coordination->addFileInfos(::DB::buildFileInfosForBackupEntries(backup_entries, backup->getBaseBackup(), read_settings, getThreadPool(ThreadPoolId::BACKUP_MAKE_FILES_LIST))); diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 4e580e493a7..a33773f19ab 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -573,11 +573,12 @@ void RestorerFromBackup::createDatabase(const String & database_name) const create_database_query->if_not_exists = (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists); LOG_TRACE(log, "Creating database {}: {}", backQuoteIfNeed(database_name), serializeAST(*create_database_query)); - + auto query_context = Context::createCopy(context); + query_context->setSetting("allow_deprecated_database_ordinary", 1); try { /// Execute CREATE DATABASE query. - InterpreterCreateQuery interpreter{create_database_query, context}; + InterpreterCreateQuery interpreter{create_database_query, query_context}; interpreter.setInternal(true); interpreter.execute(); } diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 6063c701708..86cb9acd056 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -226,6 +226,7 @@ add_object_library(clickhouse_storages_statistics Storages/Statistics) add_object_library(clickhouse_storages_liveview Storages/LiveView) add_object_library(clickhouse_storages_windowview Storages/WindowView) add_object_library(clickhouse_storages_s3queue Storages/S3Queue) +add_object_library(clickhouse_storages_materializedview Storages/MaterializedView) add_object_library(clickhouse_client Client) add_object_library(clickhouse_bridge BridgeHelper) add_object_library(clickhouse_server Server) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 0ec5db6c69d..2018015b46d 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -1,18 +1,19 @@ #include #include #include -#include -#include +#include #include #include -#include -#include -#include +#include #include -#include #include -#include +#include #include +#include +#include +#include +#include +#include namespace DB @@ -626,8 +627,7 @@ void ColumnAggregateFunction::getPermutation(PermutationSortDirection /*directio { size_t s = data.size(); res.resize(s); - for (size_t i = 0; i < s; ++i) - res[i] = i; + iota(res.data(), s, IColumn::Permutation::value_type(0)); } void ColumnAggregateFunction::updatePermutation(PermutationSortDirection, PermutationSortStability, diff --git a/src/Columns/ColumnCompressed.cpp b/src/Columns/ColumnCompressed.cpp index 9fb7b108501..3bdc514d6d8 100644 --- a/src/Columns/ColumnCompressed.cpp +++ b/src/Columns/ColumnCompressed.cpp @@ -1,4 +1,5 @@ #include +#include #pragma clang diagnostic ignored "-Wold-style-cast" diff --git a/src/Columns/ColumnConst.cpp b/src/Columns/ColumnConst.cpp index 10e960ea244..9aa0f5cfa49 100644 --- a/src/Columns/ColumnConst.cpp +++ b/src/Columns/ColumnConst.cpp @@ -2,9 +2,10 @@ #include #include -#include -#include #include +#include +#include +#include #include @@ -128,8 +129,7 @@ void ColumnConst::getPermutation(PermutationSortDirection /*direction*/, Permuta size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const { res.resize(s); - for (size_t i = 0; i < s; ++i) - res[i] = i; + iota(res.data(), s, IColumn::Permutation::value_type(0)); } void ColumnConst::updatePermutation(PermutationSortDirection /*direction*/, PermutationSortStability /*stability*/, diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index baccfc69147..20fc5d8e1fe 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -1,10 +1,11 @@ -#include #include -#include -#include -#include +#include #include #include +#include +#include +#include +#include #include @@ -163,8 +164,7 @@ void ColumnDecimal::getPermutation(IColumn::PermutationSortDirection directio if (limit >= data_size) limit = 0; - for (size_t i = 0; i < data_size; ++i) - res[i] = i; + iota(res.data(), data_size, IColumn::Permutation::value_type(0)); if constexpr (is_arithmetic_v && !is_big_int_v) { @@ -183,8 +183,7 @@ void ColumnDecimal::getPermutation(IColumn::PermutationSortDirection directio /// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters. if (data_size >= 256 && data_size <= std::numeric_limits::max() && use_radix_sort) { - for (size_t i = 0; i < data_size; ++i) - res[i] = i; + iota(res.data(), data_size, IColumn::Permutation::value_type(0)); bool try_sort = false; diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index c21e88744e0..efcdc4e4419 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -5,6 +5,7 @@ #include #include + namespace DB { namespace ErrorCodes @@ -16,7 +17,7 @@ class IFunctionBase; using FunctionBasePtr = std::shared_ptr; /** A column containing a lambda expression. - * Behaves like a constant-column. Contains an expression, but not input or output data. + * Contains an expression and captured columns, but not input arguments. */ class ColumnFunction final : public COWHelper { @@ -207,8 +208,6 @@ private: bool is_function_compiled; void appendArgument(const ColumnWithTypeAndName & column); - - void addOffsetsForReplication(const IColumn::Offsets & offsets); }; const ColumnFunction * checkAndGetShortCircuitArgument(const ColumnPtr & column); diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 2052ec3c968..f7176568a1b 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -838,7 +839,7 @@ MutableColumnPtr ColumnObject::cloneResized(size_t new_size) const void ColumnObject::getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation & res) const { res.resize(num_rows); - std::iota(res.begin(), res.end(), 0); + iota(res.data(), res.size(), size_t(0)); } void ColumnObject::compareColumn(const IColumn & rhs, size_t rhs_row_num, diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 057c0cd7112..02e6e9e56b4 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -1,11 +1,12 @@ -#include -#include #include +#include #include -#include -#include -#include +#include #include +#include +#include +#include +#include #include #include @@ -499,8 +500,7 @@ void ColumnSparse::getPermutationImpl(IColumn::PermutationSortDirection directio res.resize(_size); if (offsets->empty()) { - for (size_t i = 0; i < _size; ++i) - res[i] = i; + iota(res.data(), _size, IColumn::Permutation::value_type(0)); return; } diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index d8992125be4..356bb0493d2 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -1,16 +1,17 @@ #include -#include -#include #include +#include #include -#include +#include #include #include +#include +#include #include #include +#include #include -#include namespace DB @@ -378,8 +379,7 @@ void ColumnTuple::getPermutationImpl(IColumn::PermutationSortDirection direction { size_t rows = size(); res.resize(rows); - for (size_t i = 0; i < rows; ++i) - res[i] = i; + iota(res.data(), rows, IColumn::Permutation::value_type(0)); if (limit >= rows) limit = 0; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 37e62c76596..b1cf449dfde 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -1,24 +1,25 @@ #include "ColumnVector.h" -#include #include +#include #include #include -#include #include +#include +#include +#include +#include +#include #include #include #include #include #include #include -#include #include +#include #include -#include -#include -#include -#include +#include #include #include @@ -244,8 +245,7 @@ void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction if (limit >= data_size) limit = 0; - for (size_t i = 0; i < data_size; ++i) - res[i] = i; + iota(res.data(), data_size, IColumn::Permutation::value_type(0)); if constexpr (is_arithmetic_v && !is_big_int_v) { diff --git a/src/Columns/IColumnDummy.cpp b/src/Columns/IColumnDummy.cpp index 42b66e1156c..7c237536f94 100644 --- a/src/Columns/IColumnDummy.cpp +++ b/src/Columns/IColumnDummy.cpp @@ -1,6 +1,8 @@ -#include -#include #include +#include +#include +#include +#include namespace DB @@ -86,8 +88,7 @@ void IColumnDummy::getPermutation(IColumn::PermutationSortDirection /*direction* size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const { res.resize(s); - for (size_t i = 0; i < s; ++i) - res[i] = i; + iota(res.data(), s, IColumn::Permutation::value_type(0)); } ColumnPtr IColumnDummy::replicate(const Offsets & offsets) const diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index 0eab9452813..8e0bf0014f2 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -6,10 +6,11 @@ * implementation. */ -#include -#include -#include #include +#include +#include +#include +#include namespace DB @@ -299,8 +300,7 @@ void IColumn::getPermutationImpl( if (limit >= data_size) limit = 0; - for (size_t i = 0; i < data_size; ++i) - res[i] = i; + iota(res.data(), data_size, Permutation::value_type(0)); if (limit) { diff --git a/src/Columns/tests/gtest_column_sparse.cpp b/src/Columns/tests/gtest_column_sparse.cpp index c3450ff91b4..02b15a2f5c4 100644 --- a/src/Columns/tests/gtest_column_sparse.cpp +++ b/src/Columns/tests/gtest_column_sparse.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -191,7 +192,7 @@ TEST(ColumnSparse, Permute) auto [sparse_src, full_src] = createColumns(n, k); IColumn::Permutation perm(n); - std::iota(perm.begin(), perm.end(), 0); + iota(perm.data(), perm.size(), size_t(0)); std::shuffle(perm.begin(), perm.end(), rng); auto sparse_dst = sparse_src->permute(perm, limit); diff --git a/src/Columns/tests/gtest_column_stable_permutation.cpp b/src/Columns/tests/gtest_column_stable_permutation.cpp index df898cffa04..0dabd4d1fc2 100644 --- a/src/Columns/tests/gtest_column_stable_permutation.cpp +++ b/src/Columns/tests/gtest_column_stable_permutation.cpp @@ -9,7 +9,6 @@ #include #include #include - #include #include #include @@ -17,6 +16,7 @@ #include #include #include +#include using namespace DB; @@ -32,8 +32,7 @@ void stableGetColumnPermutation( size_t size = column.size(); out_permutation.resize(size); - for (size_t i = 0; i < size; ++i) - out_permutation[i] = i; + iota(out_permutation.data(), size, IColumn::Permutation::value_type(0)); std::stable_sort( out_permutation.begin(), @@ -146,10 +145,7 @@ void assertColumnPermutations(ColumnCreateFunc column_create_func, ValueTransfor std::vector> ranges(ranges_size); std::vector ranges_permutations(ranges_size); - for (size_t i = 0; i < ranges_size; ++i) - { - ranges_permutations[i] = i; - } + iota(ranges_permutations.data(), ranges_size, IColumn::Permutation::value_type(0)); IColumn::Permutation actual_permutation; IColumn::Permutation expected_permutation; diff --git a/src/Columns/tests/gtest_column_vector.cpp b/src/Columns/tests/gtest_column_vector.cpp index 14bf36434b6..b71d4a095ab 100644 --- a/src/Columns/tests/gtest_column_vector.cpp +++ b/src/Columns/tests/gtest_column_vector.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include using namespace DB; diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 2e00b157621..c4137920395 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -1,9 +1,190 @@ -#include "Allocator.h" +#include +#include +#include +#include +#include + +#include +#include + +#include +#include /// MADV_POPULATE_WRITE + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_ALLOCATE_MEMORY; + extern const int LOGICAL_ERROR; +} + +} + +namespace +{ + +using namespace DB; + +#if defined(MADV_POPULATE_WRITE) +/// Address passed to madvise is required to be aligned to the page boundary. +auto adjustToPageSize(void * buf, size_t len, size_t page_size) +{ + const uintptr_t address_numeric = reinterpret_cast(buf); + const size_t next_page_start = ((address_numeric + page_size - 1) / page_size) * page_size; + return std::make_pair(reinterpret_cast(next_page_start), len - (next_page_start - address_numeric)); +} +#endif + +void prefaultPages([[maybe_unused]] void * buf_, [[maybe_unused]] size_t len_) +{ +#if defined(MADV_POPULATE_WRITE) + if (len_ < POPULATE_THRESHOLD) + return; + + static const size_t page_size = ::getPageSize(); + if (len_ < page_size) /// Rounded address should be still within [buf, buf + len). + return; + + auto [buf, len] = adjustToPageSize(buf_, len_, page_size); + if (auto res = ::madvise(buf, len, MADV_POPULATE_WRITE); res < 0) + LOG_TRACE( + LogFrequencyLimiter(&Poco::Logger::get("Allocator"), 1), + "Attempt to populate pages failed: {} (EINVAL is expected for kernels < 5.14)", + errnoToString(res)); +#endif +} + +template +void * allocNoTrack(size_t size, size_t alignment) +{ + void * buf; + if (alignment <= MALLOC_MIN_ALIGNMENT) + { + if constexpr (clear_memory) + buf = ::calloc(size, 1); + else + buf = ::malloc(size); + + if (nullptr == buf) + throw DB::ErrnoException(DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot malloc {}.", ReadableSize(size)); + } + else + { + buf = nullptr; + int res = posix_memalign(&buf, alignment, size); + + if (0 != res) + throw DB::ErrnoException( + DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)); + + if constexpr (clear_memory) + memset(buf, 0, size); + } + + if constexpr (populate) + prefaultPages(buf, size); + + return buf; +} + +void freeNoTrack(void * buf) +{ + ::free(buf); +} + +void checkSize(size_t size) +{ + /// More obvious exception in case of possible overflow (instead of just "Cannot mmap"). + if (size >= 0x8000000000000000ULL) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Too large size ({}) passed to allocator. It indicates an error.", size); +} + +} /// Constant is chosen almost arbitrarily, what I observed is 128KB is too small, 1MB is almost indistinguishable from 64MB and 1GB is too large. extern const size_t POPULATE_THRESHOLD = 16 * 1024 * 1024; +template +void * Allocator::alloc(size_t size, size_t alignment) +{ + checkSize(size); + auto trace = CurrentMemoryTracker::alloc(size); + void * ptr = allocNoTrack(size, alignment); + trace.onAlloc(ptr, size); + return ptr; +} + + +template +void Allocator::free(void * buf, size_t size) +{ + try + { + checkSize(size); + freeNoTrack(buf); + auto trace = CurrentMemoryTracker::free(size); + trace.onFree(buf, size); + } + catch (...) + { + DB::tryLogCurrentException("Allocator::free"); + throw; + } +} + +template +void * Allocator::realloc(void * buf, size_t old_size, size_t new_size, size_t alignment) +{ + checkSize(new_size); + + if (old_size == new_size) + { + /// nothing to do. + /// BTW, it's not possible to change alignment while doing realloc. + } + else if (alignment <= MALLOC_MIN_ALIGNMENT) + { + /// Resize malloc'd memory region with no special alignment requirement. + auto trace_free = CurrentMemoryTracker::free(old_size); + auto trace_alloc = CurrentMemoryTracker::alloc(new_size); + trace_free.onFree(buf, old_size); + + void * new_buf = ::realloc(buf, new_size); + if (nullptr == new_buf) + { + throw DB::ErrnoException( + DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, + "Allocator: Cannot realloc from {} to {}", + ReadableSize(old_size), + ReadableSize(new_size)); + } + + buf = new_buf; + trace_alloc.onAlloc(buf, new_size); + + if constexpr (clear_memory) + if (new_size > old_size) + memset(reinterpret_cast(buf) + old_size, 0, new_size - old_size); + } + else + { + /// Big allocs that requires a copy. MemoryTracker is called inside 'alloc', 'free' methods. + void * new_buf = alloc(new_size, alignment); + memcpy(new_buf, buf, std::min(old_size, new_size)); + free(buf, old_size); + buf = new_buf; + } + + if constexpr (populate) + prefaultPages(buf, new_size); + + return buf; +} + + template class Allocator; template class Allocator; template class Allocator; diff --git a/src/Common/Allocator.h b/src/Common/Allocator.h index 269e23f3719..b865dacc2e9 100644 --- a/src/Common/Allocator.h +++ b/src/Common/Allocator.h @@ -8,47 +8,19 @@ #define ALLOCATOR_ASLR 1 #endif -#include -#include - #if !defined(OS_DARWIN) && !defined(OS_FREEBSD) #include #endif -#include -#include -#include - #include -#include - -#include -#include -#include -#include - #include - -#include -#include -#include +#include extern const size_t POPULATE_THRESHOLD; static constexpr size_t MALLOC_MIN_ALIGNMENT = 8; -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_ALLOCATE_MEMORY; - extern const int LOGICAL_ERROR; -} - -} - /** Previously there was a code which tried to use manual mmap and mremap (clickhouse_mremap.h) for large allocations/reallocations (64MB+). * Most modern allocators (including jemalloc) don't use mremap, so the idea was to take advantage from mremap system call for large reallocs. * Actually jemalloc had support for mremap, but it was intentionally removed from codebase https://github.com/jemalloc/jemalloc/commit/e2deab7a751c8080c2b2cdcfd7b11887332be1bb. @@ -69,83 +41,16 @@ class Allocator { public: /// Allocate memory range. - void * alloc(size_t size, size_t alignment = 0) - { - checkSize(size); - auto trace = CurrentMemoryTracker::alloc(size); - void * ptr = allocNoTrack(size, alignment); - trace.onAlloc(ptr, size); - return ptr; - } + void * alloc(size_t size, size_t alignment = 0); /// Free memory range. - void free(void * buf, size_t size) - { - try - { - checkSize(size); - freeNoTrack(buf); - auto trace = CurrentMemoryTracker::free(size); - trace.onFree(buf, size); - } - catch (...) - { - DB::tryLogCurrentException("Allocator::free"); - throw; - } - } + void free(void * buf, size_t size); /** Enlarge memory range. * Data from old range is moved to the beginning of new range. * Address of memory range could change. */ - void * realloc(void * buf, size_t old_size, size_t new_size, size_t alignment = 0) - { - checkSize(new_size); - - if (old_size == new_size) - { - /// nothing to do. - /// BTW, it's not possible to change alignment while doing realloc. - } - else if (alignment <= MALLOC_MIN_ALIGNMENT) - { - /// Resize malloc'd memory region with no special alignment requirement. - auto trace_free = CurrentMemoryTracker::free(old_size); - auto trace_alloc = CurrentMemoryTracker::alloc(new_size); - trace_free.onFree(buf, old_size); - - void * new_buf = ::realloc(buf, new_size); - if (nullptr == new_buf) - { - throw DB::ErrnoException( - DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, - "Allocator: Cannot realloc from {} to {}", - ReadableSize(old_size), - ReadableSize(new_size)); - } - - buf = new_buf; - trace_alloc.onAlloc(buf, new_size); - - if constexpr (clear_memory) - if (new_size > old_size) - memset(reinterpret_cast(buf) + old_size, 0, new_size - old_size); - } - else - { - /// Big allocs that requires a copy. MemoryTracker is called inside 'alloc', 'free' methods. - void * new_buf = alloc(new_size, alignment); - memcpy(new_buf, buf, std::min(old_size, new_size)); - free(buf, old_size); - buf = new_buf; - } - - if constexpr (populate) - prefaultPages(buf, new_size); - - return buf; - } + void * realloc(void * buf, size_t old_size, size_t new_size, size_t alignment = 0); protected: static constexpr size_t getStackThreshold() @@ -156,76 +61,6 @@ protected: static constexpr bool clear_memory = clear_memory_; private: - void * allocNoTrack(size_t size, size_t alignment) - { - void * buf; - if (alignment <= MALLOC_MIN_ALIGNMENT) - { - if constexpr (clear_memory) - buf = ::calloc(size, 1); - else - buf = ::malloc(size); - - if (nullptr == buf) - throw DB::ErrnoException(DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot malloc {}.", ReadableSize(size)); - } - else - { - buf = nullptr; - int res = posix_memalign(&buf, alignment, size); - - if (0 != res) - throw DB::ErrnoException( - DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)); - - if constexpr (clear_memory) - memset(buf, 0, size); - } - - if constexpr (populate) - prefaultPages(buf, size); - - return buf; - } - - void freeNoTrack(void * buf) - { - ::free(buf); - } - - void checkSize(size_t size) - { - /// More obvious exception in case of possible overflow (instead of just "Cannot mmap"). - if (size >= 0x8000000000000000ULL) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Too large size ({}) passed to allocator. It indicates an error.", size); - } - - /// Address passed to madvise is required to be aligned to the page boundary. - auto adjustToPageSize(void * buf, size_t len, size_t page_size) - { - const uintptr_t address_numeric = reinterpret_cast(buf); - const size_t next_page_start = ((address_numeric + page_size - 1) / page_size) * page_size; - return std::make_pair(reinterpret_cast(next_page_start), len - (next_page_start - address_numeric)); - } - - void prefaultPages([[maybe_unused]] void * buf_, [[maybe_unused]] size_t len_) - { -#if defined(MADV_POPULATE_WRITE) - if (len_ < POPULATE_THRESHOLD) - return; - - static const size_t page_size = ::getPageSize(); - if (len_ < page_size) /// Rounded address should be still within [buf, buf + len). - return; - - auto [buf, len] = adjustToPageSize(buf_, len_, page_size); - if (auto res = ::madvise(buf, len, MADV_POPULATE_WRITE); res < 0) - LOG_TRACE( - LogFrequencyLimiter(&Poco::Logger::get("Allocator"), 1), - "Attempt to populate pages failed: {} (EINVAL is expected for kernels < 5.14)", - errnoToString(res)); -#endif - } }; diff --git a/src/Common/Arena.h b/src/Common/Arena.h index 7604091442e..917bef0d6e8 100644 --- a/src/Common/Arena.h +++ b/src/Common/Arena.h @@ -8,6 +8,7 @@ #include #include #include +#include #if __has_include() && defined(ADDRESS_SANITIZER) # include diff --git a/src/Common/ArenaWithFreeLists.h b/src/Common/ArenaWithFreeLists.h index 76760a20320..80b4a00241d 100644 --- a/src/Common/ArenaWithFreeLists.h +++ b/src/Common/ArenaWithFreeLists.h @@ -1,5 +1,6 @@ #pragma once +#include #include #if __has_include() && defined(ADDRESS_SANITIZER) # include diff --git a/src/Common/ArrayCache.h b/src/Common/ArrayCache.h index b6dde039227..cb15759e1ba 100644 --- a/src/Common/ArrayCache.h +++ b/src/Common/ArrayCache.h @@ -19,11 +19,6 @@ #include #include -/// Required for older Darwin builds, that lack definition of MAP_ANONYMOUS -#ifndef MAP_ANONYMOUS -#define MAP_ANONYMOUS MAP_ANON -#endif - namespace DB { diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index e8deb459b24..31cf1962251 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -8,6 +9,7 @@ #include #include #include +#include #include #include diff --git a/src/Common/BitHelpers.h b/src/Common/BitHelpers.h index 79c612d47e4..bb81d271140 100644 --- a/src/Common/BitHelpers.h +++ b/src/Common/BitHelpers.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include diff --git a/src/Common/CalendarTimeInterval.cpp b/src/Common/CalendarTimeInterval.cpp new file mode 100644 index 00000000000..b218e1d3c7c --- /dev/null +++ b/src/Common/CalendarTimeInterval.cpp @@ -0,0 +1,144 @@ +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +CalendarTimeInterval::CalendarTimeInterval(const CalendarTimeInterval::Intervals & intervals) +{ + for (auto [kind, val] : intervals) + { + switch (kind.kind) + { + case IntervalKind::Nanosecond: + case IntervalKind::Microsecond: + case IntervalKind::Millisecond: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sub-second intervals are not supported here"); + + case IntervalKind::Second: + case IntervalKind::Minute: + case IntervalKind::Hour: + case IntervalKind::Day: + case IntervalKind::Week: + seconds += val * kind.toAvgSeconds(); + break; + + case IntervalKind::Month: + months += val; + break; + case IntervalKind::Quarter: + months += val * 3; + break; + case IntervalKind::Year: + months += val * 12; + break; + } + } +} + +CalendarTimeInterval::Intervals CalendarTimeInterval::toIntervals() const +{ + Intervals res; + auto greedy = [&](UInt64 x, std::initializer_list> kinds) + { + for (auto [kind, count] : kinds) + { + UInt64 k = x / count; + if (k == 0) + continue; + x -= k * count; + res.emplace_back(kind, k); + } + chassert(x == 0); + }; + greedy(months, {{IntervalKind::Year, 12}, {IntervalKind::Month, 1}}); + greedy(seconds, {{IntervalKind::Week, 3600*24*7}, {IntervalKind::Day, 3600*24}, {IntervalKind::Hour, 3600}, {IntervalKind::Minute, 60}, {IntervalKind::Second, 1}}); + return res; +} + +UInt64 CalendarTimeInterval::minSeconds() const +{ + return 3600*24 * (months/12 * 365 + months%12 * 28) + seconds; +} + +UInt64 CalendarTimeInterval::maxSeconds() const +{ + return 3600*24 * (months/12 * 366 + months%12 * 31) + seconds; +} + +void CalendarTimeInterval::assertSingleUnit() const +{ + if (seconds && months) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interval shouldn't contain both calendar units and clock units (e.g. months and days)"); +} + +void CalendarTimeInterval::assertPositive() const +{ + if (!seconds && !months) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interval must be positive"); +} + +/// Number of whole months between 1970-01-01 and `t`. +static Int64 toAbsoluteMonth(std::chrono::system_clock::time_point t) +{ + std::chrono::year_month_day ymd(std::chrono::floor(t)); + return (Int64(int(ymd.year())) - 1970) * 12 + Int64(unsigned(ymd.month()) - 1); +} + +static std::chrono::sys_seconds startOfAbsoluteMonth(Int64 absolute_month) +{ + Int64 year = absolute_month >= 0 ? absolute_month/12 : -((-absolute_month+11)/12); + Int64 month = absolute_month - year*12; + chassert(month >= 0 && month < 12); + std::chrono::year_month_day ymd( + std::chrono::year(int(year + 1970)), + std::chrono::month(unsigned(month + 1)), + std::chrono::day(1)); + return std::chrono::sys_days(ymd); +} + +std::chrono::sys_seconds CalendarTimeInterval::advance(std::chrono::system_clock::time_point tp) const +{ + auto t = std::chrono::sys_seconds(std::chrono::floor(tp)); + if (months) + { + auto m = toAbsoluteMonth(t); + auto s = t - startOfAbsoluteMonth(m); + t = startOfAbsoluteMonth(m + Int64(months)) + s; + } + return t + std::chrono::seconds(Int64(seconds)); +} + +std::chrono::sys_seconds CalendarTimeInterval::floor(std::chrono::system_clock::time_point tp) const +{ + assertSingleUnit(); + assertPositive(); + + if (months) + return startOfAbsoluteMonth(toAbsoluteMonth(tp) / months * months); + else + { + constexpr std::chrono::seconds epoch(-3600*24*3); + auto t = std::chrono::sys_seconds(std::chrono::floor(tp)); + /// We want to align with weeks, but 1970-01-01 is a Thursday, so align with 1969-12-29 instead. + return std::chrono::sys_seconds((t.time_since_epoch() - epoch) / seconds * seconds + epoch); + } +} + +bool CalendarTimeInterval::operator==(const CalendarTimeInterval & rhs) const +{ + return std::tie(months, seconds) == std::tie(rhs.months, rhs.seconds); +} + +bool CalendarTimeInterval::operator!=(const CalendarTimeInterval & rhs) const +{ + return !(*this == rhs); +} + +} diff --git a/src/Common/CalendarTimeInterval.h b/src/Common/CalendarTimeInterval.h new file mode 100644 index 00000000000..d5acc6ee2f2 --- /dev/null +++ b/src/Common/CalendarTimeInterval.h @@ -0,0 +1,63 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Represents a duration of calendar time, e.g.: +/// * 2 weeks + 5 minutes + and 21 seconds (aka 605121 seconds), +/// * 1 (calendar) month - not equivalent to any number of seconds! +/// * 3 years + 2 weeks (aka 36 months + 604800 seconds). +/// +/// Be careful with calendar arithmetic: it's missing many familiar properties of numbers. +/// E.g. x + y - y is not always equal to x (October 31 + 1 month - 1 month = November 1). +struct CalendarTimeInterval +{ + UInt64 seconds = 0; + UInt64 months = 0; + + using Intervals = std::vector>; + + CalendarTimeInterval() = default; + + /// Year, Quarter, Month are converted to months. + /// Week, Day, Hour, Minute, Second are converted to seconds. + /// Millisecond, Microsecond, Nanosecond throw exception. + explicit CalendarTimeInterval(const Intervals & intervals); + + /// E.g. for {36 months, 604801 seconds} returns {3 years, 2 weeks, 1 second}. + Intervals toIntervals() const; + + /// Approximate shortest and longest duration in seconds. E.g. a month is [28, 31] days. + UInt64 minSeconds() const; + UInt64 maxSeconds() const; + + /// Checks that the interval has only months or only seconds, throws otherwise. + void assertSingleUnit() const; + void assertPositive() const; + + /// Add this interval to the timestamp. First months, then seconds. + /// Gets weird near month boundaries: October 31 + 1 month = December 1. + std::chrono::sys_seconds advance(std::chrono::system_clock::time_point t) const; + + /// Rounds the timestamp down to the nearest timestamp "aligned" with this interval. + /// The interval must satisfy assertSingleUnit() and assertPositive(). + /// * For months, rounds to the start of a month whose abosolute index is divisible by `months`. + /// The month index is 0-based starting from January 1970. + /// E.g. if the interval is 1 month, rounds down to the start of the month. + /// * For seconds, rounds to a timestamp x such that (x - December 29 1969 (Monday)) is divisible + /// by this interval. + /// E.g. if the interval is 1 week, rounds down to the start of the week (Monday). + /// + /// Guarantees: + /// * advance(floor(x)) > x + /// * floor(advance(floor(x))) = advance(floor(x)) + std::chrono::sys_seconds floor(std::chrono::system_clock::time_point t) const; + + bool operator==(const CalendarTimeInterval & rhs) const; + bool operator!=(const CalendarTimeInterval & rhs) const; +}; + +} diff --git a/src/Common/ColumnsHashingImpl.h b/src/Common/ColumnsHashingImpl.h index 3240510ea9b..7116160e94c 100644 --- a/src/Common/ColumnsHashingImpl.h +++ b/src/Common/ColumnsHashingImpl.h @@ -31,6 +31,17 @@ public: using HashMethodContextPtr = std::shared_ptr; +struct LastElementCacheStats +{ + UInt64 hits = 0; + UInt64 misses = 0; + + void update(size_t num_tries, size_t num_misses) + { + hits += num_tries - num_misses; + misses += num_misses; + } +}; namespace columns_hashing_impl { @@ -39,14 +50,19 @@ template struct LastElementCache { static constexpr bool consecutive_keys_optimization = consecutive_keys_optimization_; + Value value; bool empty = true; bool found = false; + UInt64 misses = 0; - bool check(const Value & value_) { return !empty && value == value_; } + bool check(const Value & value_) const { return value == value_; } template - bool check(const Key & key) { return !empty && value.first == key; } + bool check(const Key & key) const { return value.first == key; } + + bool hasOnlyOneValue() const { return found && misses == 1; } + UInt64 getMisses() const { return misses; } }; template @@ -166,6 +182,7 @@ public: return EmplaceResult(!has_null_key); } } + auto key_holder = static_cast(*this).getKeyHolder(row, pool); return emplaceImpl(key_holder, data); } @@ -183,6 +200,7 @@ public: return FindResult(data.hasNullKeyData(), 0); } } + auto key_holder = static_cast(*this).getKeyHolder(row, pool); return findKeyImpl(keyHolderGetKey(key_holder), data); } @@ -194,6 +212,30 @@ public: return data.hash(keyHolderGetKey(key_holder)); } + ALWAYS_INLINE void resetCache() + { + if constexpr (consecutive_keys_optimization) + { + cache.empty = true; + cache.found = false; + cache.misses = 0; + } + } + + ALWAYS_INLINE bool hasOnlyOneValueSinceLastReset() const + { + if constexpr (consecutive_keys_optimization) + return cache.hasOnlyOneValue(); + return false; + } + + ALWAYS_INLINE UInt64 getCacheMissesSinceLastReset() const + { + if constexpr (consecutive_keys_optimization) + return cache.getMisses(); + return 0; + } + ALWAYS_INLINE bool isNullAt(size_t row) const { if constexpr (nullable) @@ -225,17 +267,15 @@ protected: else cache.value = Value(); } - if constexpr (nullable) - { + if constexpr (nullable) null_map = &checkAndGetColumn(column)->getNullMapColumn(); - } } template ALWAYS_INLINE EmplaceResult emplaceImpl(KeyHolder & key_holder, Data & data) { - if constexpr (Cache::consecutive_keys_optimization) + if constexpr (consecutive_keys_optimization) { if (cache.found && cache.check(keyHolderGetKey(key_holder))) { @@ -266,6 +306,7 @@ protected: { cache.found = true; cache.empty = false; + ++cache.misses; if constexpr (has_mapped) { @@ -288,12 +329,12 @@ protected: template ALWAYS_INLINE FindResult findKeyImpl(Key key, Data & data) { - if constexpr (Cache::consecutive_keys_optimization) + if constexpr (consecutive_keys_optimization) { /// It's possible to support such combination, but code will became more complex. /// Now there's not place where we need this options enabled together static_assert(!FindResult::has_offset, "`consecutive_keys_optimization` and `has_offset` are conflicting options"); - if (cache.check(key)) + if (likely(!cache.empty) && cache.check(key)) { if constexpr (has_mapped) return FindResult(&cache.value.second, cache.found, 0); @@ -308,6 +349,7 @@ protected: { cache.found = it != nullptr; cache.empty = false; + ++cache.misses; if constexpr (has_mapped) { @@ -325,9 +367,8 @@ protected: size_t offset = 0; if constexpr (FindResult::has_offset) - { offset = it ? data.offsetInternal(it) : 0; - } + if constexpr (has_mapped) return FindResult(it ? &it->getMapped() : nullptr, it != nullptr, offset); else diff --git a/src/Common/CounterInFile.h b/src/Common/CounterInFile.h index 993ed97966a..854bf7cc675 100644 --- a/src/Common/CounterInFile.h +++ b/src/Common/CounterInFile.h @@ -88,7 +88,7 @@ public: { /// A more understandable error message. if (e.code() == DB::ErrorCodes::CANNOT_READ_ALL_DATA || e.code() == DB::ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) - throw DB::ParsingException(e.code(), "File {} is empty. You must fill it manually with appropriate value.", path); + throw DB::Exception(e.code(), "File {} is empty. You must fill it manually with appropriate value.", path); else throw; } diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 38b14e4b0b4..2613e9ec116 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -253,6 +253,8 @@ M(MergeTreeAllRangesAnnouncementsSent, "The current number of announcement being sent in flight from the remote server to the initiator server about the set of data parts (for MergeTree tables). Measured on the remote server side.") \ M(CreatedTimersInQueryProfiler, "Number of Created thread local timers in QueryProfiler") \ M(ActiveTimersInQueryProfiler, "Number of Active thread local timers in QueryProfiler") \ + M(RefreshableViews, "Number materialized views with periodic refreshing (REFRESH)") \ + M(RefreshingViews, "Number of materialized views currently executing a refresh") \ #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 9222a27afdf..577a83e40b9 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -589,6 +589,7 @@ M(707, GCP_ERROR) \ M(708, ILLEGAL_STATISTIC) \ M(709, CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT) \ + M(710, FAULT_INJECTED) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index d5f1984a5ff..e1f010cc740 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -616,48 +616,4 @@ ExecutionStatus ExecutionStatus::fromText(const std::string & data) return status; } -ParsingException::ParsingException() = default; -ParsingException::ParsingException(const std::string & msg, int code) - : Exception(msg, code) -{ -} - -/// We use additional field formatted_message_ to make this method const. -std::string ParsingException::displayText() const -{ - try - { - formatted_message = message(); - bool need_newline = false; - if (!file_name.empty()) - { - formatted_message += fmt::format(": (in file/uri {})", file_name); - need_newline = true; - } - - if (line_number != -1) - { - formatted_message += fmt::format(": (at row {})", line_number); - need_newline = true; - } - - if (need_newline) - formatted_message += "\n"; - } - catch (...) {} // NOLINT(bugprone-empty-catch) - - if (!formatted_message.empty()) - { - std::string result = name(); - result.append(": "); - result.append(formatted_message); - return result; - } - else - { - return Exception::displayText(); - } -} - - } diff --git a/src/Common/Exception.h b/src/Common/Exception.h index aabc848b230..6f30fde3876 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -235,43 +235,6 @@ private: const char * className() const noexcept override { return "DB::ErrnoException"; } }; - -/// Special class of exceptions, used mostly in ParallelParsingInputFormat for -/// more convenient calculation of problem line number. -class ParsingException : public Exception -{ - ParsingException(const std::string & msg, int code); -public: - ParsingException(); - - // Format message with fmt::format, like the logging functions. - template - ParsingException(int code, FormatStringHelper fmt, Args &&... args) : Exception(fmt::format(fmt.fmt_str, std::forward(args)...), code) - { - message_format_string = fmt.message_format_string; - } - - std::string displayText() const override; - - ssize_t getLineNumber() const { return line_number; } - void setLineNumber(int line_number_) { line_number = line_number_;} - - String getFileName() const { return file_name; } - void setFileName(const String & file_name_) { file_name = file_name_; } - - Exception * clone() const override { return new ParsingException(*this); } - void rethrow() const override { throw *this; } // NOLINT - -private: - ssize_t line_number{-1}; - String file_name; - mutable std::string formatted_message; - - const char * name() const noexcept override { return "DB::ParsingException"; } - const char * className() const noexcept override { return "DB::ParsingException"; } -}; - - using Exceptions = std::vector; /** Try to write an exception to the log (and forget about it). diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index 9665788dac2..f29aee0cdcc 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -34,6 +34,8 @@ static struct InitFiu #define APPLY_FOR_FAILPOINTS(ONCE, REGULAR, PAUSEABLE_ONCE, PAUSEABLE) \ ONCE(replicated_merge_tree_commit_zk_fail_after_op) \ + ONCE(replicated_queue_fail_next_entry) \ + REGULAR(replicated_queue_unfail_entries) \ ONCE(replicated_merge_tree_insert_quorum_fail_0) \ REGULAR(replicated_merge_tree_commit_zk_fail_when_recovering_from_hw_fault) \ REGULAR(use_delayed_remote_source) \ diff --git a/src/Common/FiberStack.h b/src/Common/FiberStack.h index 067b0aa7a63..9d135f27306 100644 --- a/src/Common/FiberStack.h +++ b/src/Common/FiberStack.h @@ -13,6 +13,11 @@ #include #endif +/// Required for older Darwin builds, that lack definition of MAP_ANONYMOUS +#ifndef MAP_ANONYMOUS +#define MAP_ANONYMOUS MAP_ANON +#endif + namespace DB::ErrorCodes { extern const int CANNOT_ALLOCATE_MEMORY; diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index 6893286f196..0f45d0ac169 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -71,6 +71,8 @@ struct IntervalKind /// Returns false if the conversion did not succeed. /// For example, `IntervalKind::tryParseString('second', result)` returns `result` equals `IntervalKind::Kind::Second`. static bool tryParseString(const std::string & kind, IntervalKind::Kind & result); + + auto operator<=>(const IntervalKind & other) const { return kind <=> other.kind; } }; /// NOLINTNEXTLINE diff --git a/src/Common/NamePrompter.h b/src/Common/NamePrompter.h index 97c345414bb..cc72554657f 100644 --- a/src/Common/NamePrompter.h +++ b/src/Common/NamePrompter.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index ab1a430cebb..92803af93a9 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include diff --git a/src/Common/PODArray.cpp b/src/Common/PODArray.cpp index d21dc40867d..dd1fed08cb5 100644 --- a/src/Common/PODArray.cpp +++ b/src/Common/PODArray.cpp @@ -1,8 +1,46 @@ +#include #include + namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_MPROTECT; + extern const int CANNOT_ALLOCATE_MEMORY; +} + +namespace PODArrayDetails +{ + +#ifndef NDEBUG +void protectMemoryRegion(void * addr, size_t len, int prot) +{ + if (0 != mprotect(addr, len, prot)) + throw ErrnoException(ErrorCodes::CANNOT_MPROTECT, "Cannot mprotect memory region"); +} +#endif + +size_t byte_size(size_t num_elements, size_t element_size) +{ + size_t amount; + if (__builtin_mul_overflow(num_elements, element_size, &amount)) + throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Amount of memory requested to allocate is more than allowed"); + return amount; +} + +size_t minimum_memory_for_elements(size_t num_elements, size_t element_size, size_t pad_left, size_t pad_right) +{ + size_t amount; + if (__builtin_add_overflow(byte_size(num_elements, element_size), pad_left + pad_right, &amount)) + throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Amount of memory requested to allocate is more than allowed"); + return amount; +} + +} + + /// Used for left padding of PODArray when empty const char empty_pod_array[empty_pod_array_size]{}; @@ -25,4 +63,5 @@ template class PODArray, 0, 0>; template class PODArray, 0, 0>; template class PODArray, 0, 0>; template class PODArray, 0, 0>; + } diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index 77cecf694f3..6a048d1c6c0 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -1,27 +1,21 @@ #pragma once +#include +#include +#include +#include +#include +#include #include #include #include #include #include -#include - -#include -#include - -#include -#include -#include -#include - #ifndef NDEBUG - #include +#include #endif -#include - /** Whether we can use memcpy instead of a loop with assignment to T from U. * It is Ok if types are the same. And if types are integral and of the same size, * example: char, signed char, unsigned char. @@ -35,12 +29,6 @@ constexpr bool memcpy_can_be_used_for_assignment = std::is_same_v namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_MPROTECT; - extern const int CANNOT_ALLOCATE_MEMORY; -} - /** A dynamic array for POD types. * Designed for a small number of large arrays (rather than a lot of small ones). * To be more precise - for use in ColumnVector. @@ -77,6 +65,19 @@ namespace ErrorCodes static constexpr size_t empty_pod_array_size = 1024; extern const char empty_pod_array[empty_pod_array_size]; +namespace PODArrayDetails +{ + +void protectMemoryRegion(void * addr, size_t len, int prot); + +/// The amount of memory occupied by the num_elements of the elements. +size_t byte_size(size_t num_elements, size_t element_size); /// NOLINT + +/// Minimum amount of memory to allocate for num_elements, including padding. +size_t minimum_memory_for_elements(size_t num_elements, size_t element_size, size_t pad_left, size_t pad_right); /// NOLINT + +}; + /** Base class that depend only on size of element, not on element itself. * You can static_cast to this class if you want to insert some data regardless to the actual type T. */ @@ -102,27 +103,9 @@ protected: char * c_end = null; char * c_end_of_storage = null; /// Does not include pad_right. - /// The amount of memory occupied by the num_elements of the elements. - static size_t byte_size(size_t num_elements) /// NOLINT - { - size_t amount; - if (__builtin_mul_overflow(num_elements, ELEMENT_SIZE, &amount)) - throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Amount of memory requested to allocate is more than allowed"); - return amount; - } - - /// Minimum amount of memory to allocate for num_elements, including padding. - static size_t minimum_memory_for_elements(size_t num_elements) - { - size_t amount; - if (__builtin_add_overflow(byte_size(num_elements), pad_left + pad_right, &amount)) - throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Amount of memory requested to allocate is more than allowed"); - return amount; - } - void alloc_for_num_elements(size_t num_elements) /// NOLINT { - alloc(minimum_memory_for_elements(num_elements)); + alloc(PODArrayDetails::minimum_memory_for_elements(num_elements, ELEMENT_SIZE, pad_left, pad_right)); } template @@ -188,7 +171,7 @@ protected: // The allocated memory should be multiplication of ELEMENT_SIZE to hold the element, otherwise, // memory issue such as corruption could appear in edge case. realloc(std::max(integerRoundUp(initial_bytes, ELEMENT_SIZE), - minimum_memory_for_elements(1)), + PODArrayDetails::minimum_memory_for_elements(1, ELEMENT_SIZE, pad_left, pad_right)), std::forward(allocator_params)...); } else @@ -208,8 +191,7 @@ protected: if (right_rounded_down > left_rounded_up) { size_t length = right_rounded_down - left_rounded_up; - if (0 != mprotect(left_rounded_up, length, prot)) - throw ErrnoException(ErrorCodes::CANNOT_MPROTECT, "Cannot mprotect memory region"); + PODArrayDetails::protectMemoryRegion(left_rounded_up, length, prot); } } @@ -232,14 +214,14 @@ public: void reserve(size_t n, TAllocatorParams &&... allocator_params) { if (n > capacity()) - realloc(roundUpToPowerOfTwoOrZero(minimum_memory_for_elements(n)), std::forward(allocator_params)...); + realloc(roundUpToPowerOfTwoOrZero(PODArrayDetails::minimum_memory_for_elements(n, ELEMENT_SIZE, pad_left, pad_right)), std::forward(allocator_params)...); } template void reserve_exact(size_t n, TAllocatorParams &&... allocator_params) /// NOLINT { if (n > capacity()) - realloc(minimum_memory_for_elements(n), std::forward(allocator_params)...); + realloc(PODArrayDetails::minimum_memory_for_elements(n, ELEMENT_SIZE, pad_left, pad_right), std::forward(allocator_params)...); } template @@ -258,7 +240,7 @@ public: void resize_assume_reserved(const size_t n) /// NOLINT { - c_end = c_start + byte_size(n); + c_end = c_start + PODArrayDetails::byte_size(n, ELEMENT_SIZE); } const char * raw_data() const /// NOLINT @@ -339,7 +321,7 @@ public: explicit PODArray(size_t n) { this->alloc_for_num_elements(n); - this->c_end += this->byte_size(n); + this->c_end += PODArrayDetails::byte_size(n, sizeof(T)); } PODArray(size_t n, const T & x) @@ -411,9 +393,9 @@ public: if (n > old_size) { this->reserve(n); - memset(this->c_end, 0, this->byte_size(n - old_size)); + memset(this->c_end, 0, PODArrayDetails::byte_size(n - old_size, sizeof(T))); } - this->c_end = this->c_start + this->byte_size(n); + this->c_end = this->c_start + PODArrayDetails::byte_size(n, sizeof(T)); } void resize_fill(size_t n, const T & value) /// NOLINT @@ -424,7 +406,7 @@ public: this->reserve(n); std::fill(t_end(), t_end() + n - old_size, value); } - this->c_end = this->c_start + this->byte_size(n); + this->c_end = this->c_start + PODArrayDetails::byte_size(n, sizeof(T)); } template @@ -487,7 +469,7 @@ public: if (required_capacity > this->capacity()) this->reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward(allocator_params)...); - size_t bytes_to_copy = this->byte_size(from_end - from_begin); + size_t bytes_to_copy = PODArrayDetails::byte_size(from_end - from_begin, sizeof(T)); if (bytes_to_copy) { memcpy(this->c_end, reinterpret_cast(rhs.begin() + from_begin), bytes_to_copy); @@ -502,7 +484,7 @@ public: static_assert(pad_right_ >= PADDING_FOR_SIMD - 1); static_assert(sizeof(T) == sizeof(*from_begin)); insertPrepare(from_begin, from_end, std::forward(allocator_params)...); - size_t bytes_to_copy = this->byte_size(from_end - from_begin); + size_t bytes_to_copy = PODArrayDetails::byte_size(from_end - from_begin, sizeof(T)); memcpySmallAllowReadWriteOverflow15(this->c_end, reinterpret_cast(&*from_begin), bytes_to_copy); this->c_end += bytes_to_copy; } @@ -513,11 +495,11 @@ public: { static_assert(memcpy_can_be_used_for_assignment, std::decay_t>); - size_t bytes_to_copy = this->byte_size(from_end - from_begin); + size_t bytes_to_copy = PODArrayDetails::byte_size(from_end - from_begin, sizeof(T)); if (!bytes_to_copy) return; - size_t bytes_to_move = this->byte_size(end() - it); + size_t bytes_to_move = PODArrayDetails::byte_size(end() - it, sizeof(T)); insertPrepare(from_begin, from_end); @@ -545,10 +527,10 @@ public: if (required_capacity > this->capacity()) this->reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward(allocator_params)...); - size_t bytes_to_copy = this->byte_size(copy_size); + size_t bytes_to_copy = PODArrayDetails::byte_size(copy_size, sizeof(T)); if (bytes_to_copy) { - auto begin = this->c_start + this->byte_size(start_index); + auto begin = this->c_start + PODArrayDetails::byte_size(start_index, sizeof(T)); memcpy(this->c_end, reinterpret_cast(&*begin), bytes_to_copy); this->c_end += bytes_to_copy; } @@ -560,7 +542,7 @@ public: static_assert(memcpy_can_be_used_for_assignment, std::decay_t>); this->assertNotIntersects(from_begin, from_end); - size_t bytes_to_copy = this->byte_size(from_end - from_begin); + size_t bytes_to_copy = PODArrayDetails::byte_size(from_end - from_begin, sizeof(T)); if (bytes_to_copy) { memcpy(this->c_end, reinterpret_cast(&*from_begin), bytes_to_copy); @@ -593,13 +575,13 @@ public: /// arr1 takes ownership of the heap memory of arr2. arr1.c_start = arr2.c_start; arr1.c_end_of_storage = arr1.c_start + heap_allocated - arr2.pad_right - arr2.pad_left; - arr1.c_end = arr1.c_start + this->byte_size(heap_size); + arr1.c_end = arr1.c_start + PODArrayDetails::byte_size(heap_size, sizeof(T)); /// Allocate stack space for arr2. arr2.alloc(stack_allocated, std::forward(allocator_params)...); /// Copy the stack content. - memcpy(arr2.c_start, stack_c_start, this->byte_size(stack_size)); - arr2.c_end = arr2.c_start + this->byte_size(stack_size); + memcpy(arr2.c_start, stack_c_start, PODArrayDetails::byte_size(stack_size, sizeof(T))); + arr2.c_end = arr2.c_start + PODArrayDetails::byte_size(stack_size, sizeof(T)); }; auto do_move = [&](PODArray & src, PODArray & dest) @@ -608,8 +590,8 @@ public: { dest.dealloc(); dest.alloc(src.allocated_bytes(), std::forward(allocator_params)...); - memcpy(dest.c_start, src.c_start, this->byte_size(src.size())); - dest.c_end = dest.c_start + this->byte_size(src.size()); + memcpy(dest.c_start, src.c_start, PODArrayDetails::byte_size(src.size(), sizeof(T))); + dest.c_end = dest.c_start + PODArrayDetails::byte_size(src.size(), sizeof(T)); src.c_start = Base::null; src.c_end = Base::null; @@ -666,8 +648,8 @@ public: this->c_end_of_storage = this->c_start + rhs_allocated - Base::pad_right - Base::pad_left; rhs.c_end_of_storage = rhs.c_start + lhs_allocated - Base::pad_right - Base::pad_left; - this->c_end = this->c_start + this->byte_size(rhs_size); - rhs.c_end = rhs.c_start + this->byte_size(lhs_size); + this->c_end = this->c_start + PODArrayDetails::byte_size(rhs_size, sizeof(T)); + rhs.c_end = rhs.c_start + PODArrayDetails::byte_size(lhs_size, sizeof(T)); } else if (this->isAllocatedFromStack() && !rhs.isAllocatedFromStack()) { @@ -702,7 +684,7 @@ public: if (required_capacity > this->capacity()) this->reserve_exact(required_capacity, std::forward(allocator_params)...); - size_t bytes_to_copy = this->byte_size(required_capacity); + size_t bytes_to_copy = PODArrayDetails::byte_size(required_capacity, sizeof(T)); if (bytes_to_copy) memcpy(this->c_start, reinterpret_cast(&*from_begin), bytes_to_copy); diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 4bdf6288a1c..119e0d99143 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -288,6 +288,18 @@ The server successfully detected this situation and will download merged part fr M(OSReadChars, "Number of bytes read from filesystem, including page cache.") \ M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \ \ + M(ParallelReplicasHandleRequestMicroseconds, "Time spent processing requests for marks from replicas") \ + M(ParallelReplicasHandleAnnouncementMicroseconds, "Time spent processing replicas announcements") \ + \ + M(ParallelReplicasReadAssignedMarks, "Sum across all replicas of how many of scheduled marks were assigned by consistent hash") \ + M(ParallelReplicasReadUnassignedMarks, "Sum across all replicas of how many unassigned marks were scheduled") \ + M(ParallelReplicasReadAssignedForStealingMarks, "Sum across all replicas of how many of scheduled marks were assigned for stealing by consistent hash") \ + \ + M(ParallelReplicasStealingByHashMicroseconds, "Time spent collecting segments meant for stealing by hash") \ + M(ParallelReplicasProcessingPartsMicroseconds, "Time spent processing data parts") \ + M(ParallelReplicasStealingLeftoversMicroseconds, "Time spent collecting orphaned segments") \ + M(ParallelReplicasCollectingOwnedSegmentsMicroseconds, "Time spent collecting segments meant by hash") \ + \ M(PerfCpuCycles, "Total cycles. Be wary of what happens during CPU frequency scaling.") \ M(PerfInstructions, "Retired instructions. Be careful, these can be affected by various issues, most notably hardware interrupt counts.") \ M(PerfCacheReferences, "Cache accesses. Usually, this indicates Last Level Cache accesses, but this may vary depending on your CPU. This may include prefetches and coherency messages; again this depends on the design of your CPU.") \ diff --git a/src/Common/TargetSpecific.h b/src/Common/TargetSpecific.h index fd6a57090b8..68f6d39c3ff 100644 --- a/src/Common/TargetSpecific.h +++ b/src/Common/TargetSpecific.h @@ -348,6 +348,25 @@ DECLARE_AVX512VBMI2_SPECIFIC_CODE( #if ENABLE_MULTITARGET_CODE && defined(__GNUC__) && defined(__x86_64__) +/// NOLINTNEXTLINE +#define MULTITARGET_FUNCTION_AVX2_SSE42(FUNCTION_HEADER, name, FUNCTION_BODY) \ + FUNCTION_HEADER \ + \ + AVX2_FUNCTION_SPECIFIC_ATTRIBUTE \ + name##AVX2 \ + FUNCTION_BODY \ + \ + FUNCTION_HEADER \ + \ + SSE42_FUNCTION_SPECIFIC_ATTRIBUTE \ + name##SSE42 \ + FUNCTION_BODY \ + \ + FUNCTION_HEADER \ + \ + name \ + FUNCTION_BODY \ + /// NOLINTNEXTLINE #define MULTITARGET_FUNCTION_AVX512BW_AVX512F_AVX2_SSE42(FUNCTION_HEADER, name, FUNCTION_BODY) \ FUNCTION_HEADER \ @@ -381,6 +400,14 @@ DECLARE_AVX512VBMI2_SPECIFIC_CODE( #else + /// NOLINTNEXTLINE +#define MULTITARGET_FUNCTION_AVX2_SSE42(FUNCTION_HEADER, name, FUNCTION_BODY) \ + FUNCTION_HEADER \ + \ + name \ + FUNCTION_BODY \ + + /// NOLINTNEXTLINE #define MULTITARGET_FUNCTION_AVX512BW_AVX512F_AVX2_SSE42(FUNCTION_HEADER, name, FUNCTION_BODY) \ FUNCTION_HEADER \ diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 565affb0c65..3c2e6228421 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -28,6 +28,40 @@ namespace CurrentMetrics extern const Metric GlobalThreadScheduled; } +class JobWithPriority +{ +public: + using Job = std::function; + + Job job; + Priority priority; + CurrentMetrics::Increment metric_increment; + DB::OpenTelemetry::TracingContextOnThread thread_trace_context; + + /// Call stacks of all jobs' schedulings leading to this one + std::vector frame_pointers; + bool enable_job_stack_trace = false; + + JobWithPriority( + Job job_, Priority priority_, CurrentMetrics::Metric metric, + const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_, + bool capture_frame_pointers) + : job(job_), priority(priority_), metric_increment(metric), + thread_trace_context(thread_trace_context_), enable_job_stack_trace(capture_frame_pointers) + { + if (!capture_frame_pointers) + return; + /// Save all previous jobs call stacks and append with current + frame_pointers = DB::Exception::thread_frame_pointers; + frame_pointers.push_back(StackTrace().getFramePointers()); + } + + bool operator<(const JobWithPriority & rhs) const + { + return priority > rhs.priority; // Reversed for `priority_queue` max-heap to yield minimum value (i.e. highest priority) first + } +}; + static constexpr auto DEFAULT_THREAD_NAME = "ThreadPool"; template diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 3117509ab8f..31e4eabf63b 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -20,9 +20,10 @@ #include #include #include -#include #include +class JobWithPriority; + /** Very simple thread pool similar to boost::threadpool. * Advantages: * - catches exceptions and rethrows on wait. @@ -128,37 +129,6 @@ private: bool threads_remove_themselves = true; const bool shutdown_on_exception = true; - struct JobWithPriority - { - Job job; - Priority priority; - CurrentMetrics::Increment metric_increment; - DB::OpenTelemetry::TracingContextOnThread thread_trace_context; - - /// Call stacks of all jobs' schedulings leading to this one - std::vector frame_pointers; - bool enable_job_stack_trace = false; - - JobWithPriority( - Job job_, Priority priority_, CurrentMetrics::Metric metric, - const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_, - bool capture_frame_pointers) - : job(job_), priority(priority_), metric_increment(metric), - thread_trace_context(thread_trace_context_), enable_job_stack_trace(capture_frame_pointers) - { - if (!capture_frame_pointers) - return; - /// Save all previous jobs call stacks and append with current - frame_pointers = DB::Exception::thread_frame_pointers; - frame_pointers.push_back(StackTrace().getFramePointers()); - } - - bool operator<(const JobWithPriority & rhs) const - { - return priority > rhs.priority; // Reversed for `priority_queue` max-heap to yield minimum value (i.e. highest priority) first - } - }; - boost::heap::priority_queue jobs; std::list threads; std::exception_ptr first_exception; diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 101a56cd620..c99823b2dfa 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 9ec7208d3eb..d732b900d37 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1,4 +1,5 @@ -#include "Common/ZooKeeper/ZooKeeperConstants.h" +#include +#include #include #include diff --git a/src/Common/examples/shell_command_inout.cpp b/src/Common/examples/shell_command_inout.cpp index 615700cd042..a646dfba311 100644 --- a/src/Common/examples/shell_command_inout.cpp +++ b/src/Common/examples/shell_command_inout.cpp @@ -6,6 +6,7 @@ #include #include #include +#include /** This example shows how we can proxy stdin to ShellCommand and obtain stdout in streaming fashion. */ diff --git a/src/Common/findExtreme.cpp b/src/Common/findExtreme.cpp new file mode 100644 index 00000000000..032ac75b79b --- /dev/null +++ b/src/Common/findExtreme.cpp @@ -0,0 +1,143 @@ +#include +#include +#include + +namespace DB +{ + +template +struct MinComparator +{ + static ALWAYS_INLINE inline const T & cmp(const T & a, const T & b) { return std::min(a, b); } +}; + +template +struct MaxComparator +{ + static ALWAYS_INLINE inline const T & cmp(const T & a, const T & b) { return std::max(a, b); } +}; + +MULTITARGET_FUNCTION_AVX2_SSE42( + MULTITARGET_FUNCTION_HEADER(template static std::optional NO_INLINE), + findExtremeImpl, + MULTITARGET_FUNCTION_BODY((const T * __restrict ptr, const UInt8 * __restrict condition_map [[maybe_unused]], size_t row_begin, size_t row_end) /// NOLINT + { + size_t count = row_end - row_begin; + ptr += row_begin; + if constexpr (!add_all_elements) + condition_map += row_begin; + + T ret{}; + size_t i = 0; + for (; i < count; i++) + { + if (add_all_elements || !condition_map[i] == add_if_cond_zero) + { + ret = ptr[i]; + break; + } + } + if (i >= count) + return std::nullopt; + + /// Unroll the loop manually for floating point, since the compiler doesn't do it without fastmath + /// as it might change the return value + if constexpr (std::is_floating_point_v) + { + constexpr size_t unroll_block = 512 / sizeof(T); /// Chosen via benchmarks with AVX2 so YMMV + size_t unrolled_end = i + (((count - i) / unroll_block) * unroll_block); + + if (i < unrolled_end) + { + T partial_min[unroll_block]; + for (size_t unroll_it = 0; unroll_it < unroll_block; unroll_it++) + partial_min[unroll_it] = ret; + + while (i < unrolled_end) + { + for (size_t unroll_it = 0; unroll_it < unroll_block; unroll_it++) + { + if (add_all_elements || !condition_map[i + unroll_it] == add_if_cond_zero) + partial_min[unroll_it] = ComparatorClass::cmp(partial_min[unroll_it], ptr[i + unroll_it]); + } + i += unroll_block; + } + for (size_t unroll_it = 0; unroll_it < unroll_block; unroll_it++) + ret = ComparatorClass::cmp(ret, partial_min[unroll_it]); + } + } + + for (; i < count; i++) + { + if (add_all_elements || !condition_map[i] == add_if_cond_zero) + ret = ComparatorClass::cmp(ret, ptr[i]); + } + + return ret; + } +)) + +/// Given a vector of T finds the extreme (MIN or MAX) value +template +static std::optional +findExtreme(const T * __restrict ptr, const UInt8 * __restrict condition_map [[maybe_unused]], size_t start, size_t end) +{ +#if USE_MULTITARGET_CODE + /// We see no benefit from using AVX512BW or AVX512F (over AVX2), so we only declare SSE and AVX2 + if (isArchSupported(TargetArch::AVX2)) + return findExtremeImplAVX2(ptr, condition_map, start, end); + + if (isArchSupported(TargetArch::SSE42)) + return findExtremeImplSSE42(ptr, condition_map, start, end); +#endif + return findExtremeImpl(ptr, condition_map, start, end); +} + +template +std::optional findExtremeMin(const T * __restrict ptr, size_t start, size_t end) +{ + return findExtreme, true, false>(ptr, nullptr, start, end); +} + +template +std::optional findExtremeMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end) +{ + return findExtreme, false, true>(ptr, condition_map, start, end); +} + +template +std::optional findExtremeMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end) +{ + return findExtreme, false, false>(ptr, condition_map, start, end); +} + +template +std::optional findExtremeMax(const T * __restrict ptr, size_t start, size_t end) +{ + return findExtreme, true, false>(ptr, nullptr, start, end); +} + +template +std::optional findExtremeMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end) +{ + return findExtreme, false, true>(ptr, condition_map, start, end); +} + +template +std::optional findExtremeMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end) +{ + return findExtreme, false, false>(ptr, condition_map, start, end); +} + + +#define INSTANTIATION(T) \ + template std::optional findExtremeMin(const T * __restrict ptr, size_t start, size_t end); \ + template std::optional findExtremeMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ + template std::optional findExtremeMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ + template std::optional findExtremeMax(const T * __restrict ptr, size_t start, size_t end); \ + template std::optional findExtremeMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ + template std::optional findExtremeMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); + +FOR_BASIC_NUMERIC_TYPES(INSTANTIATION) +#undef INSTANTIATION +} diff --git a/src/Common/findExtreme.h b/src/Common/findExtreme.h new file mode 100644 index 00000000000..b38c24697c0 --- /dev/null +++ b/src/Common/findExtreme.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include +#include + +#include +#include + +namespace DB +{ +template +concept is_any_native_number = (is_any_of); + +template +std::optional findExtremeMin(const T * __restrict ptr, size_t start, size_t end); + +template +std::optional findExtremeMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); + +template +std::optional findExtremeMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); + +template +std::optional findExtremeMax(const T * __restrict ptr, size_t start, size_t end); + +template +std::optional findExtremeMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); + +template +std::optional findExtremeMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); + +#define EXTERN_INSTANTIATION(T) \ + extern template std::optional findExtremeMin(const T * __restrict ptr, size_t start, size_t end); \ + extern template std::optional findExtremeMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ + extern template std::optional findExtremeMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ + extern template std::optional findExtremeMax(const T * __restrict ptr, size_t start, size_t end); \ + extern template std::optional findExtremeMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \ + extern template std::optional findExtremeMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); + + FOR_BASIC_NUMERIC_TYPES(EXTERN_INSTANTIATION) +#undef EXTERN_INSTANTIATION + +} diff --git a/src/Common/iota.cpp b/src/Common/iota.cpp new file mode 100644 index 00000000000..98f18eb195b --- /dev/null +++ b/src/Common/iota.cpp @@ -0,0 +1,36 @@ +#include +#include +#include + +namespace DB +{ + +MULTITARGET_FUNCTION_AVX2_SSE42( + MULTITARGET_FUNCTION_HEADER(template void NO_INLINE), + iotaImpl, MULTITARGET_FUNCTION_BODY((T * begin, size_t count, T first_value) /// NOLINT + { + for (size_t i = 0; i < count; i++) + *(begin + i) = static_cast(first_value + i); + }) +) + +template +void iota(T * begin, size_t count, T first_value) +{ +#if USE_MULTITARGET_CODE + if (isArchSupported(TargetArch::AVX2)) + return iotaImplAVX2(begin, count, first_value); + + if (isArchSupported(TargetArch::SSE42)) + return iotaImplSSE42(begin, count, first_value); +#endif + return iotaImpl(begin, count, first_value); +} + +template void iota(UInt8 * begin, size_t count, UInt8 first_value); +template void iota(UInt32 * begin, size_t count, UInt32 first_value); +template void iota(UInt64 * begin, size_t count, UInt64 first_value); +#if defined(OS_DARWIN) +template void iota(size_t * begin, size_t count, size_t first_value); +#endif +} diff --git a/src/Common/iota.h b/src/Common/iota.h new file mode 100644 index 00000000000..7910274d15d --- /dev/null +++ b/src/Common/iota.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include + +/// This is a replacement for std::iota to use dynamic dispatch +/// Note that is only defined for containers with contiguous memory only + +namespace DB +{ + +/// Make sure to add any new type to the extern declaration at the end of the file and instantiate it in iota.cpp + +template +concept iota_supported_types = (is_any_of< + T, + UInt8, + UInt32, + UInt64 +#if defined(OS_DARWIN) + , + size_t +#endif + >); + +template void iota(T * begin, size_t count, T first_value); + +extern template void iota(UInt8 * begin, size_t count, UInt8 first_value); +extern template void iota(UInt32 * begin, size_t count, UInt32 first_value); +extern template void iota(UInt64 * begin, size_t count, UInt64 first_value); +#if defined(OS_DARWIN) +extern template void iota(size_t * begin, size_t count, size_t first_value); +#endif +} diff --git a/src/Common/levenshteinDistance.cpp b/src/Common/levenshteinDistance.cpp index 9eb6c0f9050..3ab80af94bb 100644 --- a/src/Common/levenshteinDistance.cpp +++ b/src/Common/levenshteinDistance.cpp @@ -1,5 +1,6 @@ -#include #include +#include +#include namespace DB { @@ -11,8 +12,7 @@ size_t levenshteinDistance(const String & lhs, const String & rhs) PODArrayWithStackMemory row(n + 1); - for (size_t i = 1; i <= n; ++i) - row[i] = i; + iota(row.data() + 1, n, size_t(1)); for (size_t j = 1; j <= m; ++j) { diff --git a/src/Common/memcpySmall.h b/src/Common/memcpySmall.h index 5eaa1203f05..0c2aee96250 100644 --- a/src/Common/memcpySmall.h +++ b/src/Common/memcpySmall.h @@ -1,6 +1,7 @@ #pragma once #include +#include /// ssize_t #ifdef __SSE2__ # include diff --git a/src/Common/setThreadName.cpp b/src/Common/setThreadName.cpp index e14abb247f3..aae80272206 100644 --- a/src/Common/setThreadName.cpp +++ b/src/Common/setThreadName.cpp @@ -28,25 +28,31 @@ namespace ErrorCodes static thread_local char thread_name[THREAD_NAME_SIZE]{}; -void setThreadName(const char * name) +void setThreadName(const char * name, bool truncate) { - if (strlen(name) > THREAD_NAME_SIZE - 1) + size_t name_len = strlen(name); + if (!truncate && name_len > THREAD_NAME_SIZE - 1) throw DB::Exception(DB::ErrorCodes::PTHREAD_ERROR, "Thread name cannot be longer than 15 bytes"); + size_t name_capped_len = std::min(1 + name_len, THREAD_NAME_SIZE - 1); + char name_capped[THREAD_NAME_SIZE]; + memcpy(name_capped, name, name_capped_len); + name_capped[name_capped_len] = '\0'; + #if defined(OS_FREEBSD) - pthread_set_name_np(pthread_self(), name); + pthread_set_name_np(pthread_self(), name_capped); if ((false)) #elif defined(OS_DARWIN) - if (0 != pthread_setname_np(name)) + if (0 != pthread_setname_np(name_capped)) #elif defined(OS_SUNOS) - if (0 != pthread_setname_np(pthread_self(), name)) + if (0 != pthread_setname_np(pthread_self(), name_capped)) #else - if (0 != prctl(PR_SET_NAME, name, 0, 0, 0)) + if (0 != prctl(PR_SET_NAME, name_capped, 0, 0, 0)) #endif if (errno != ENOSYS && errno != EPERM) /// It's ok if the syscall is unsupported or not allowed in some environments. throw DB::ErrnoException(DB::ErrorCodes::PTHREAD_ERROR, "Cannot set thread name with prctl(PR_SET_NAME, ...)"); - memcpy(thread_name, name, std::min(1 + strlen(name), THREAD_NAME_SIZE - 1)); + memcpy(thread_name, name_capped, name_capped_len); } const char * getThreadName() diff --git a/src/Common/setThreadName.h b/src/Common/setThreadName.h index 1834ea9696f..fdb2717925f 100644 --- a/src/Common/setThreadName.h +++ b/src/Common/setThreadName.h @@ -4,7 +4,9 @@ /** Sets the thread name (maximum length is 15 bytes), * which will be visible in ps, gdb, /proc, * for convenience of observation and debugging. + * + * @param truncate - if true, will truncate to 15 automatically, otherwise throw */ -void setThreadName(const char * name); +void setThreadName(const char * name, bool truncate = false); const char * getThreadName(); diff --git a/src/Common/tests/gtest_config_dot.cpp b/src/Common/tests/gtest_config_dot.cpp new file mode 100644 index 00000000000..d88d896677b --- /dev/null +++ b/src/Common/tests/gtest_config_dot.cpp @@ -0,0 +1,30 @@ +#include +#include +#include +#include + +#include + + +using namespace DB; + +TEST(Common, ConfigWithDotInKeys) +{ + std::string xml(R"CONFIG( + 1 +)CONFIG"); + + Poco::XML::DOMParser dom_parser; + Poco::AutoPtr document = dom_parser.parseString(xml); + Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); + + /// directly + EXPECT_EQ(ConfigHelper::getBool(*config, "foo.bar", false, false), false); + EXPECT_EQ(ConfigHelper::getBool(*config, "foo\\.bar", false, false), true); + + /// via keys() + Poco::Util::AbstractConfiguration::Keys keys; + config->keys("", keys); + ASSERT_EQ(1, keys.size()); + ASSERT_EQ("foo\\.bar", keys[0]); +} diff --git a/src/Common/tests/gtest_hash_table.cpp b/src/Common/tests/gtest_hash_table.cpp index 72941126cfd..ae432de7766 100644 --- a/src/Common/tests/gtest_hash_table.cpp +++ b/src/Common/tests/gtest_hash_table.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -20,7 +21,7 @@ namespace std::vector getVectorWithNumbersUpToN(size_t n) { std::vector res(n); - std::iota(res.begin(), res.end(), 0); + iota(res.data(), res.size(), UInt64(0)); return res; } diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 25d809c9726..ee0356adde5 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -11,6 +11,7 @@ #include "libaccel_config.h" #include #include +#include #include diff --git a/src/Compression/examples/compressed_buffer.cpp b/src/Compression/examples/compressed_buffer.cpp index aef2cf4ab90..74646ff0f28 100644 --- a/src/Compression/examples/compressed_buffer.cpp +++ b/src/Compression/examples/compressed_buffer.cpp @@ -23,7 +23,7 @@ int main(int, char **) Stopwatch stopwatch; { - DB::WriteBufferFromFile buf("test1", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); + DB::WriteBufferFromFile buf("test1", DB::DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); DB::CompressedWriteBuffer compressed_buf(buf); stopwatch.restart(); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 638ee11858a..26ee3668ef6 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -4,6 +4,7 @@ #include "config.h" #include +#include #include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -196,13 +198,9 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server nuraft::raft_server::commit_in_bg(); } - void commitLogs(uint64_t index_to_commit, bool initial_commit_exec) + std::unique_lock lockRaft() { - leader_commit_index_.store(index_to_commit); - quick_commit_index_ = index_to_commit; - lagging_sm_target_index_ = index_to_commit; - - commit_in_bg_exec(0, initial_commit_exec); + return std::unique_lock(lock_); } using nuraft::raft_server::raft_server; @@ -518,6 +516,7 @@ void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession & RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForSessions & requests_for_sessions) { std::vector> entries; + entries.reserve(requests_for_sessions.size()); for (const auto & request_for_session : requests_for_sessions) entries.push_back(getZooKeeperLogEntry(request_for_session)); @@ -630,73 +629,74 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ { const auto preprocess_logs = [&] { + auto lock = raft_instance->lockRaft(); + + if (keeper_context->local_logs_preprocessed) + return; + keeper_context->local_logs_preprocessed = true; auto log_store = state_manager->load_log_store(); - if (last_log_idx_on_disk > 0 && last_log_idx_on_disk > state_machine->last_commit_index()) - { - auto log_entries = log_store->log_entries(state_machine->last_commit_index() + 1, last_log_idx_on_disk + 1); + auto log_entries = log_store->log_entries(state_machine->last_commit_index() + 1, log_store->next_slot()); - size_t preprocessed = 0; - LOG_INFO(log, "Preprocessing {} log entries", log_entries->size()); - auto idx = state_machine->last_commit_index() + 1; - for (const auto & entry : *log_entries) - { - if (entry && entry->get_val_type() == nuraft::log_val_type::app_log) - state_machine->pre_commit(idx, entry->get_buf()); - - ++idx; - ++preprocessed; - - if (preprocessed % 50000 == 0) - LOG_TRACE(log, "Preprocessed {}/{} entries", preprocessed, log_entries->size()); - } - LOG_INFO(log, "Preprocessing done"); - } - else + if (log_entries->empty()) { LOG_INFO(log, "All local log entries preprocessed"); + return; } + + size_t preprocessed = 0; + LOG_INFO(log, "Preprocessing {} log entries", log_entries->size()); + auto idx = state_machine->last_commit_index() + 1; + for (const auto & entry : *log_entries) + { + if (entry && entry->get_val_type() == nuraft::log_val_type::app_log) + state_machine->pre_commit(idx, entry->get_buf()); + + ++idx; + ++preprocessed; + + if (preprocessed % 50000 == 0) + LOG_TRACE(log, "Preprocessed {}/{} entries", preprocessed, log_entries->size()); + } + LOG_INFO(log, "Preprocessing done"); }; switch (type) { - case nuraft::cb_func::InitialBatchCommited: + case nuraft::cb_func::PreAppendLogLeader: { - preprocess_logs(); - break; + /// we cannot preprocess anything new as leader because we don't have up-to-date in-memory state + /// until we preprocess all stored logs + return nuraft::cb_func::ReturnCode::ReturnNull; } case nuraft::cb_func::GotAppendEntryReqFromLeader: { - auto & req = *static_cast(param->ctx); - - if (req.get_commit_idx() == 0 || req.log_entries().empty()) - break; - - auto last_committed_index = state_machine->last_commit_index(); - // Actual log number. - auto index_to_commit = std::min({last_log_idx_on_disk, req.get_last_log_idx(), req.get_commit_idx()}); - - if (index_to_commit > last_committed_index) - { - LOG_TRACE(log, "Trying to commit local log entries, committing upto {}", index_to_commit); - raft_instance->commitLogs(index_to_commit, true); - /// after we manually committed all the local logs we can, we assert that all of the local logs are either - /// committed or preprocessed - if (!keeper_context->local_logs_preprocessed) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Local logs are not preprocessed"); - } - else if (last_log_idx_on_disk <= last_committed_index) - { - keeper_context->local_logs_preprocessed = true; - } - else if - ( - index_to_commit == 0 || - (index_to_commit == last_committed_index && last_log_idx_on_disk > index_to_commit) /// we need to rollback all the logs so we preprocess all of them - ) + /// maybe we got snapshot installed + if (state_machine->last_commit_index() >= last_log_idx_on_disk) { preprocess_logs(); + break; } + + auto & req = *static_cast(param->ctx); + + if (req.log_entries().empty()) + break; + + if (req.get_last_log_idx() < last_log_idx_on_disk) + last_log_idx_on_disk = req.get_last_log_idx(); + /// we don't want to accept too many new logs before we preprocess all the local logs + /// because the next log index is decreased on each failure we need to also accept requests when it's near last_log_idx_on_disk + /// so the counter is reset on the leader side + else if (raft_instance->get_target_committed_log_idx() >= last_log_idx_on_disk && req.get_last_log_idx() > last_log_idx_on_disk) + return nuraft::cb_func::ReturnNull; + + break; + } + case nuraft::cb_func::StateMachineExecution: + { + if (state_machine->last_commit_index() >= last_log_idx_on_disk) + preprocess_logs(); break; } default: diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index a245ccc16df..910615bf6ef 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -98,10 +99,15 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo client_configuration.endpointOverride = new_uri.endpoint; + S3::ClientSettings client_settings{ + .use_virtual_addressing = new_uri.is_virtual_hosted_style, + .disable_checksum = false, + .gcs_issue_compose_request = false, + }; + auto client = S3::ClientFactory::instance().create( client_configuration, - new_uri.is_virtual_hosted_style, - /* disable_checksum= */ false, + client_settings, credentials.GetAWSAccessKeyId(), credentials.GetAWSSecretKey(), auth_settings.server_side_encryption_customer_key_base64, diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h index ae3ff1553b0..d092a8d4440 100644 --- a/src/Coordination/LoggerWrapper.h +++ b/src/Coordination/LoggerWrapper.h @@ -13,6 +13,7 @@ private: static inline const std::unordered_map LEVELS = { + {LogsLevel::test, Poco::Message::Priority::PRIO_TEST}, {LogsLevel::trace, Poco::Message::Priority::PRIO_TRACE}, {LogsLevel::debug, Poco::Message::Priority::PRIO_DEBUG}, {LogsLevel::information, Poco::Message::PRIO_INFORMATION}, diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 2b5fd3424c0..dd19f0b9967 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1000,7 +1000,7 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); DB::WriteBufferFromFile plain_buf( - "./logs/changelog_11_15.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + "./logs/changelog_11_15.bin" + params.extension, DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(0); DB::KeeperLogStore changelog_reader( @@ -1073,7 +1073,7 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); DB::WriteBufferFromFile plain_buf( - "./logs/changelog_1_20.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + "./logs/changelog_1_20.bin" + params.extension, DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(30); DB::KeeperLogStore changelog_reader( @@ -1130,7 +1130,7 @@ TEST_F(CoordinationTest, ChangelogTestReadAfterBrokenTruncate3) EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin")); DB::WriteBufferFromFile plain_buf( - "./logs/changelog_1_20.bin", DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + "./logs/changelog_1_20.bin", DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(plain_buf.size() - 30); DB::KeeperLogStore changelog_reader( @@ -1733,7 +1733,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotBroken) /// Let's corrupt file DB::WriteBufferFromFile plain_buf( - "./snapshots/snapshot_50.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + "./snapshots/snapshot_50.bin" + params.extension, DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(34); plain_buf.sync(); @@ -2770,7 +2770,7 @@ TEST_P(CoordinationTest, TestDurableState) { SCOPED_TRACE("Read from corrupted file"); state_manager.reset(); - DB::WriteBufferFromFile write_buf("./state", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY); + DB::WriteBufferFromFile write_buf("./state", DB::DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY); write_buf.seek(20, SEEK_SET); DB::writeIntBinary(31, write_buf); write_buf.sync(); @@ -2787,7 +2787,7 @@ TEST_P(CoordinationTest, TestDurableState) SCOPED_TRACE("Read from file with invalid size"); state_manager.reset(); - DB::WriteBufferFromFile write_buf("./state", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY); + DB::WriteBufferFromFile write_buf("./state", DB::DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY); DB::writeIntBinary(20, write_buf); write_buf.sync(); write_buf.close(); diff --git a/src/Core/BackgroundSchedulePool.cpp b/src/Core/BackgroundSchedulePool.cpp index ec1ae047d05..fa892bc3c84 100644 --- a/src/Core/BackgroundSchedulePool.cpp +++ b/src/Core/BackgroundSchedulePool.cpp @@ -31,7 +31,7 @@ bool BackgroundSchedulePoolTaskInfo::schedule() return true; } -bool BackgroundSchedulePoolTaskInfo::scheduleAfter(size_t milliseconds, bool overwrite) +bool BackgroundSchedulePoolTaskInfo::scheduleAfter(size_t milliseconds, bool overwrite, bool only_if_scheduled) { std::lock_guard lock(schedule_mutex); @@ -39,6 +39,8 @@ bool BackgroundSchedulePoolTaskInfo::scheduleAfter(size_t milliseconds, bool ove return false; if (delayed && !overwrite) return false; + if (!delayed && only_if_scheduled) + return false; pool.scheduleDelayedTask(shared_from_this(), milliseconds, lock); return true; diff --git a/src/Core/BackgroundSchedulePool.h b/src/Core/BackgroundSchedulePool.h index e97b02e976f..eca93353283 100644 --- a/src/Core/BackgroundSchedulePool.h +++ b/src/Core/BackgroundSchedulePool.h @@ -106,8 +106,10 @@ public: bool schedule(); /// Schedule for execution after specified delay. - /// If overwrite is set then the task will be re-scheduled (if it was already scheduled, i.e. delayed == true). - bool scheduleAfter(size_t milliseconds, bool overwrite = true); + /// If overwrite is set, and the task is already scheduled with a delay (delayed == true), + /// the task will be re-scheduled with the new delay. + /// If only_if_scheduled is set, don't do anything unless the task is already scheduled with a delay. + bool scheduleAfter(size_t milliseconds, bool overwrite = true, bool only_if_scheduled = false); /// Further attempts to schedule become no-op. Will wait till the end of the current execution of the task. void deactivate(); diff --git a/src/Core/Defines.h b/src/Core/Defines.h index e2ffc2b7d7a..a3ab76c0b93 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -3,66 +3,70 @@ #include #include -#define DBMS_DEFAULT_PORT 9000 -#define DBMS_DEFAULT_SECURE_PORT 9440 -#define DBMS_DEFAULT_CONNECT_TIMEOUT_SEC 10 -#define DBMS_DEFAULT_SEND_TIMEOUT_SEC 300 -#define DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC 300 +namespace DB +{ + +static constexpr auto DBMS_DEFAULT_PORT = 9000; +static constexpr auto DBMS_DEFAULT_SECURE_PORT = 9440; +static constexpr auto DBMS_DEFAULT_CONNECT_TIMEOUT_SEC = 10; +static constexpr auto DBMS_DEFAULT_SEND_TIMEOUT_SEC = 300; +static constexpr auto DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC = 300; /// Timeout for synchronous request-result protocol call (like Ping or TablesStatus). -#define DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC 5 -#define DBMS_DEFAULT_POLL_INTERVAL 10 +static constexpr auto DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC = 5; +static constexpr auto DBMS_DEFAULT_POLL_INTERVAL = 10; /// The size of the I/O buffer by default. -#define DBMS_DEFAULT_BUFFER_SIZE 1048576ULL +static constexpr auto DBMS_DEFAULT_BUFFER_SIZE = 1048576ULL; -#define PADDING_FOR_SIMD 64 +static constexpr auto PADDING_FOR_SIMD = 64; /** Which blocks by default read the data (by number of rows). * Smaller values give better cache locality, less consumption of RAM, but more overhead to process the query. */ -#define DEFAULT_BLOCK_SIZE 65409 /// 65536 - PADDING_FOR_SIMD - (PADDING_FOR_SIMD - 1) bytes padding that we usually have in arrays +static constexpr auto DEFAULT_BLOCK_SIZE + = 65409; /// 65536 - PADDING_FOR_SIMD - (PADDING_FOR_SIMD - 1) bytes padding that we usually have in = arrays /** Which blocks should be formed for insertion into the table, if we control the formation of blocks. * (Sometimes the blocks are inserted exactly such blocks that have been read / transmitted from the outside, and this parameter does not affect their size.) * More than DEFAULT_BLOCK_SIZE, because in some tables a block of data on the disk is created for each block (quite a big thing), * and if the parts were small, then it would be costly then to combine them. */ -#define DEFAULT_INSERT_BLOCK_SIZE \ - 1048449 /// 1048576 - PADDING_FOR_SIMD - (PADDING_FOR_SIMD - 1) bytes padding that we usually have in arrays +static constexpr auto DEFAULT_INSERT_BLOCK_SIZE + = 1048449; /// 1048576 - PADDING_FOR_SIMD - (PADDING_FOR_SIMD - 1) bytes padding that we usually have in arrays -#define DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC 60 -#define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160) +static constexpr auto DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC = 60; +static constexpr auto SHOW_CHARS_ON_SYNTAX_ERROR = ptrdiff_t(160); /// each period reduces the error counter by 2 times /// too short a period can cause errors to disappear immediately after creation. -#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD 60 +static constexpr auto DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD = 60; /// replica error max cap, this is to prevent replica from accumulating too many errors and taking to long to recover. -#define DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT 1000 +static constexpr auto DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT = 1000; /// The boundary on which the blocks for asynchronous file operations should be aligned. -#define DEFAULT_AIO_FILE_BLOCK_SIZE 4096 +static constexpr auto DEFAULT_AIO_FILE_BLOCK_SIZE = 4096; -#define DEFAULT_HTTP_READ_BUFFER_TIMEOUT 30 -#define DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT 1 +static constexpr auto DEFAULT_HTTP_READ_BUFFER_TIMEOUT = 30; +static constexpr auto DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT = 1; /// Maximum number of http-connections between two endpoints /// the number is unmotivated -#define DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT 15 +static constexpr auto DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT = 15; -#define DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT 30 +static constexpr auto DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT = 30; -#define DBMS_DEFAULT_PATH "/var/lib/clickhouse/" +static constexpr auto DBMS_DEFAULT_PATH = "/var/lib/clickhouse/"; /// Actually, there may be multiple acquisitions of different locks for a given table within one query. /// Check with IStorage class for the list of possible locks -#define DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC 120 +static constexpr auto DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC = 120; /// Default limit on recursion depth of recursive descend parser. -#define DBMS_DEFAULT_MAX_PARSER_DEPTH 1000 +static constexpr auto DBMS_DEFAULT_MAX_PARSER_DEPTH = 1000; /// Default limit on query size. -#define DBMS_DEFAULT_MAX_QUERY_SIZE 262144 +static constexpr auto DBMS_DEFAULT_MAX_QUERY_SIZE = 262144; /// Max depth of hierarchical dictionary -#define DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH 1000 +static constexpr auto DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH = 1000; /// Default maximum (total and entry) sizes and policies of various caches static constexpr auto DEFAULT_UNCOMPRESSED_CACHE_POLICY = "SLRU"; @@ -95,7 +99,9 @@ static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS = 30'000'000uz; /// /// Look at compiler-rt/lib/sanitizer_common/sanitizer_stacktrace.h #if !defined(SANITIZER) -#define QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS 1000000000 +static constexpr auto QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS = 1000000000; #else -#define QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS 0 +static constexpr auto QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS = 0; #endif + +} diff --git a/src/Core/Field.h b/src/Core/Field.h index e77217abc03..6afa98ed9c0 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -122,7 +122,7 @@ struct CustomType bool isSecret() const { return impl->isSecret(); } const char * getTypeName() const { return impl->getTypeName(); } String toString(bool show_secrets = true) const { return impl->toString(show_secrets); } - const CustomTypeImpl & getImpl() { return *impl; } + const CustomTypeImpl & getImpl() const { return *impl; } bool operator < (const CustomType & rhs) const { return *impl < *rhs.impl; } bool operator <= (const CustomType & rhs) const { return *impl <= *rhs.impl; } @@ -292,7 +292,7 @@ concept not_field_or_bool_or_stringlike /** 32 is enough. Round number is used for alignment and for better arithmetic inside std::vector. * NOTE: Actually, sizeof(std::string) is 32 when using libc++, so Field is 40 bytes. */ -#define DBMS_MIN_FIELD_SIZE 32 +static constexpr auto DBMS_MIN_FIELD_SIZE = 32; /** Discriminated union of several types. diff --git a/src/Core/InterpolateDescription.cpp b/src/Core/InterpolateDescription.cpp index e7b74716b79..d828c2e85e9 100644 --- a/src/Core/InterpolateDescription.cpp +++ b/src/Core/InterpolateDescription.cpp @@ -3,10 +3,16 @@ #include #include #include +#include +#include +#include +#include +#include +#include + namespace DB { - InterpolateDescription::InterpolateDescription(ActionsDAGPtr actions_, const Aliases & aliases) : actions(actions_) { @@ -28,5 +34,4 @@ namespace DB result_columns_order.push_back(name); } } - } diff --git a/src/Core/InterpolateDescription.h b/src/Core/InterpolateDescription.h index 8aabce1470e..62d7120508b 100644 --- a/src/Core/InterpolateDescription.h +++ b/src/Core/InterpolateDescription.h @@ -2,20 +2,18 @@ #include #include -#include #include -#include -#include -#include -#include -#include -#include -#include +#include +#include namespace DB { +class ActionsDAG; +using ActionsDAGPtr = std::shared_ptr; +using Aliases = std::unordered_map; + /// Interpolate description struct InterpolateDescription { diff --git a/src/Core/MySQL/MySQLCharset.cpp b/src/Core/MySQL/MySQLCharset.cpp index 0acf3f130a6..d8e68565f3d 100644 --- a/src/Core/MySQL/MySQLCharset.cpp +++ b/src/Core/MySQL/MySQLCharset.cpp @@ -5,13 +5,16 @@ #if USE_ICU #include -#define CHUNK_SIZE 1024 -static const char * TARGET_CHARSET = "utf8"; #endif namespace DB { +#if USE_ICU +static constexpr auto CHUNK_SIZE = 1024; +static constexpr auto TARGET_CHARSET = "utf8"; +#endif + namespace ErrorCodes { extern const int UNKNOWN_EXCEPTION; diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 0e2e5b3dc60..058c6fdc903 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -1,77 +1,80 @@ #pragma once -#define DBMS_MIN_REVISION_WITH_CLIENT_INFO 54032 -#define DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE 54058 -#define DBMS_MIN_REVISION_WITH_QUOTA_KEY_IN_CLIENT_INFO 54060 -#define DBMS_MIN_REVISION_WITH_TABLES_STATUS 54226 -#define DBMS_MIN_REVISION_WITH_TIME_ZONE_PARAMETER_IN_DATETIME_DATA_TYPE 54337 -#define DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME 54372 -#define DBMS_MIN_REVISION_WITH_VERSION_PATCH 54401 -#define DBMS_MIN_REVISION_WITH_SERVER_LOGS 54406 +namespace DB +{ + +static constexpr auto DBMS_MIN_REVISION_WITH_CLIENT_INFO = 54032; +static constexpr auto DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE = 54058; +static constexpr auto DBMS_MIN_REVISION_WITH_QUOTA_KEY_IN_CLIENT_INFO = 54060; +static constexpr auto DBMS_MIN_REVISION_WITH_TABLES_STATUS = 54226; +static constexpr auto DBMS_MIN_REVISION_WITH_TIME_ZONE_PARAMETER_IN_DATETIME_DATA_TYPE = 54337; +static constexpr auto DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME = 54372; +static constexpr auto DBMS_MIN_REVISION_WITH_VERSION_PATCH = 54401; +static constexpr auto DBMS_MIN_REVISION_WITH_SERVER_LOGS = 54406; /// Minimum revision with exactly the same set of aggregation methods and rules to select them. /// Two-level (bucketed) aggregation is incompatible if servers are inconsistent in these rules /// (keys will be placed in different buckets and result will not be fully aggregated). -#define DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 54448 -#define DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 21 -#define DBMS_MIN_MINOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 4 -#define DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA 54410 +static constexpr auto DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD = 54448; +static constexpr auto DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD = 21; +static constexpr auto DBMS_MIN_MINOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD = 4; +static constexpr auto DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA = 54410; -#define DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE 54405 -#define DBMS_MIN_REVISION_WITH_CLIENT_WRITE_INFO 54420 +static constexpr auto DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE = 54405; +static constexpr auto DBMS_MIN_REVISION_WITH_CLIENT_WRITE_INFO = 54420; /// Minimum revision supporting SettingsBinaryFormat::STRINGS. -#define DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS 54429 -#define DBMS_MIN_REVISION_WITH_SCALARS 54429 +static constexpr auto DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS = 54429; +static constexpr auto DBMS_MIN_REVISION_WITH_SCALARS = 54429; /// Minimum revision supporting OpenTelemetry -#define DBMS_MIN_REVISION_WITH_OPENTELEMETRY 54442 +static constexpr auto DBMS_MIN_REVISION_WITH_OPENTELEMETRY = 54442; -#define DBMS_MIN_REVISION_WITH_AGGREGATE_FUNCTIONS_VERSIONING 54452 +static constexpr auto DBMS_MIN_REVISION_WITH_AGGREGATE_FUNCTIONS_VERSIONING = 54452; -#define DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION 1 +static constexpr auto DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION = 1; -#define DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION 3 -#define DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS 54453 +static constexpr auto DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION = 3; +static constexpr auto DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS = 54453; -#define DBMS_MERGE_TREE_PART_INFO_VERSION 1 +static constexpr auto DBMS_MERGE_TREE_PART_INFO_VERSION = 1; -#define DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET 54441 +static constexpr auto DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET = 54441; -#define DBMS_MIN_REVISION_WITH_X_FORWARDED_FOR_IN_CLIENT_INFO 54443 -#define DBMS_MIN_REVISION_WITH_REFERER_IN_CLIENT_INFO 54447 +static constexpr auto DBMS_MIN_REVISION_WITH_X_FORWARDED_FOR_IN_CLIENT_INFO = 54443; +static constexpr auto DBMS_MIN_REVISION_WITH_REFERER_IN_CLIENT_INFO = 54447; -#define DBMS_MIN_PROTOCOL_VERSION_WITH_DISTRIBUTED_DEPTH 54448 +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_DISTRIBUTED_DEPTH = 54448; -#define DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS 54451 +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS = 54451; -#define DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION 54454 +static constexpr auto DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION = 54454; -#define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME = 54449; -#define DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS_IN_INSERT 54456 +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS_IN_INSERT = 54456; -#define DBMS_MIN_PROTOCOL_VERSION_WITH_VIEW_IF_PERMITTED 54457 +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_VIEW_IF_PERMITTED = 54457; -#define DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM 54458 +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM = 54458; -#define DBMS_MIN_PROTOCOL_VERSION_WITH_QUOTA_KEY 54458 +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_QUOTA_KEY = 54458; -#define DBMS_MIN_PROTOCOL_VERSION_WITH_PARAMETERS 54459 +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_PARAMETERS = 54459; /// The server will send query elapsed run time in the Progress packet. -#define DBMS_MIN_PROTOCOL_VERSION_WITH_SERVER_QUERY_TIME_IN_PROGRESS 54460 +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_SERVER_QUERY_TIME_IN_PROGRESS = 54460; -#define DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES 54461 +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES = 54461; -#define DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 54462 +static constexpr auto DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 = 54462; -#define DBMS_MIN_PROTOCOL_VERSION_WITH_TOTAL_BYTES_IN_PROGRESS 54463 +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_TOTAL_BYTES_IN_PROGRESS = 54463; -#define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54464 +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES = 54464; -#define DBMS_MIN_REVISION_WITH_SPARSE_SERIALIZATION 54465 +static constexpr auto DBMS_MIN_REVISION_WITH_SPARSE_SERIALIZATION = 54465; -#define DBMS_MIN_REVISION_WITH_SSH_AUTHENTICATION 54466 +static constexpr auto DBMS_MIN_REVISION_WITH_SSH_AUTHENTICATION = 54466; /// Version of ClickHouse TCP protocol. /// @@ -80,4 +83,6 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54466 +static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54466; + +} diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 85e3d33f80b..2a9fa8e744c 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -26,6 +26,8 @@ namespace DB M(UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0) \ M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0) \ M(UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0) \ + M(UInt64, max_mutations_bandwidth_for_server, 0, "The maximum read speed of all mutations on server in bytes per second. Zero means unlimited.", 0) \ + M(UInt64, max_merges_bandwidth_for_server, 0, "The maximum read speed of all merges on server in bytes per second. Zero means unlimited.", 0) \ M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \ @@ -111,6 +113,8 @@ namespace DB M(Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0) \ M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \ + /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp + DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) struct ServerSettings : public BaseSettings diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b75004a3396..58b7cbab4c9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -185,6 +185,7 @@ class IColumn; M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \ + M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \ \ M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards. Shard is marked as unavailable when: 1) The shard cannot be reached due to a connection failure. 2) Shard is unresolvable through DNS. 3) Table does not exist on the shard.", 0) \ \ @@ -219,6 +220,7 @@ class IColumn; M(Bool, mysql_map_fixed_string_to_text_in_show_columns, false, "If enabled, FixedString type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise.", 0) \ \ M(UInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ", 0) \ + M(UInt64, optimize_min_inequality_conjunction_chain_length, 3, "The minimum length of the expression `expr <> x1 AND ... expr <> xN` for optimization ", 0) \ \ M(UInt64, min_bytes_to_use_direct_io, 0, "The minimum number of bytes for reading the data with O_DIRECT option during SELECT queries execution. 0 - disabled.", 0) \ M(UInt64, min_bytes_to_use_mmap_io, 0, "The minimum number of bytes for reading the data with mmap option during SELECT queries execution. 0 - disabled.", 0) \ @@ -583,6 +585,9 @@ class IColumn; M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \ M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ + M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped taraget table during pushing to views", 0) \ + M(Bool, allow_experimental_refreshable_materialized_view, false, "Allow refreshable materialized views (CREATE MATERIALIZED VIEW REFRESH ...).", 0) \ + M(Bool, stop_refreshable_materialized_views_on_startup, false, "On server startup, prevent scheduling of refreshable materialized views, as if with SYSTEM STOP VIEWS. You can manually start them with SYSTEM START VIEWS or SYSTEM START VIEW afterwards. Also applies to newly created views. Has no effect on non-refreshable materialized views.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ @@ -658,6 +663,7 @@ class IColumn; M(Bool, allow_aggregate_partitions_independently, false, "Enable independent aggregation of partitions on separate threads when partition key suits group by key. Beneficial when number of partitions close to number of cores and partitions have roughly the same size", 0) \ M(Bool, force_aggregate_partitions_independently, false, "Force the use of optimization when it is applicable, but heuristics decided not to use it", 0) \ M(UInt64, max_number_of_partitions_for_independent_aggregation, 128, "Maximal number of partitions in table to apply optimization", 0) \ + M(Float, min_hit_rate_to_use_consecutive_keys_optimization, 0.5, "Minimal hit rate of a cache which is used for consecutive keys optimization in aggregation to keep it enabled", 0) \ /** Experimental feature for moving data between shards. */ \ \ M(Bool, allow_experimental_query_deduplication, false, "Experimental data deduplication for SELECT queries based on part UUIDs", 0) \ @@ -703,7 +709,6 @@ class IColumn; M(Bool, query_plan_execute_functions_after_sorting, true, "Allow to re-order functions after sorting", 0) \ M(Bool, query_plan_reuse_storage_ordering_for_window_functions, true, "Allow to use the storage sorting for window functions", 0) \ M(Bool, query_plan_lift_up_union, true, "Allow to move UNIONs up so that more parts of the query plan can be optimized", 0) \ - M(Bool, query_plan_optimize_primary_key, true, "Analyze primary key using query plan (instead of AST)", 0) \ M(Bool, query_plan_read_in_order, true, "Use query plan for read-in-order optimization", 0) \ M(Bool, query_plan_aggregation_in_order, true, "Use query plan for aggregation-in-order optimization", 0) \ M(Bool, query_plan_remove_redundant_sorting, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries", 0) \ @@ -839,7 +844,7 @@ class IColumn; M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0) \ M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \ - M(Bool, print_pretty_type_names, false, "Print pretty type names in DESCRIBE query and toTypeName() function", 0) \ + M(Bool, print_pretty_type_names, true, "Print pretty type names in DESCRIBE query and toTypeName() function", 0) \ M(Bool, create_table_empty_primary_key_by_default, false, "Allow to create *MergeTree tables with empty primary key when ORDER BY and PRIMARY KEY not specified", 0) \ M(Bool, allow_named_collection_override_by_default, true, "Allow named collections' fields override by default.", 0)\ M(Bool, allow_experimental_shared_merge_tree, false, "Only available in ClickHouse Cloud", 0) \ @@ -912,6 +917,7 @@ class IColumn; MAKE_OBSOLETE(M, Bool, optimize_move_functions_out_of_any, false) \ MAKE_OBSOLETE(M, Bool, allow_experimental_undrop_table_query, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_s3queue, true) \ + MAKE_OBSOLETE(M, Bool, query_plan_optimize_primary_key, true) \ /** The section above is for obsolete settings. Do not add anything there. */ @@ -977,6 +983,7 @@ class IColumn; M(SchemaInferenceMode, schema_inference_mode, "default", "Mode of schema inference. 'default' - assume that all files have the same schema and schema can be inferred from any file, 'union' - files can have different schemas and the resulting schema should be the a union of schemas of all files", 0) \ M(Bool, schema_inference_make_columns_nullable, true, "If set to true, all inferred types will be Nullable in schema inference for formats without information about nullability.", 0) \ M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \ + M(Bool, input_format_json_read_bools_as_strings, true, "Allow to parse bools as strings in JSON input formats", 0) \ M(Bool, input_format_json_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference", 0) \ M(Bool, input_format_json_validate_types_from_metadata, true, "For JSON/JSONCompact/JSONColumnsWithMetadata input formats this controls whether format parser should check if data types from input metadata match data types of the corresponding columns from the table", 0) \ M(Bool, input_format_json_read_numbers_as_strings, true, "Allow to parse numbers as strings in JSON input formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index aad57ffebb7..fdee1fd5b13 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -81,6 +81,8 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, + {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}}}, {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index c35e69977ed..0c84c1cc7d2 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -98,6 +98,8 @@ IMPLEMENT_SETTING_AUTO_ENUM(DefaultDatabaseEngine, ErrorCodes::BAD_ARGUMENTS) IMPLEMENT_SETTING_AUTO_ENUM(DefaultTableEngine, ErrorCodes::BAD_ARGUMENTS) +IMPLEMENT_SETTING_AUTO_ENUM(CleanDeletedRows, ErrorCodes::BAD_ARGUMENTS) + IMPLEMENT_SETTING_MULTI_ENUM(MySQLDataTypesSupport, ErrorCodes::UNKNOWN_MYSQL_DATATYPES_SUPPORT_LEVEL, {{"decimal", MySQLDataTypesSupport::DECIMAL}, {"datetime64", MySQLDataTypesSupport::DATETIME64}, @@ -113,6 +115,8 @@ IMPLEMENT_SETTING_ENUM(DistributedDDLOutputMode, ErrorCodes::BAD_ARGUMENTS, {{"none", DistributedDDLOutputMode::NONE}, {"throw", DistributedDDLOutputMode::THROW}, {"null_status_on_timeout", DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT}, + {"throw_only_active", DistributedDDLOutputMode::THROW_ONLY_ACTIVE}, + {"null_status_on_timeout_only_active", DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE}, {"never_throw", DistributedDDLOutputMode::NEVER_THROW}}) IMPLEMENT_SETTING_ENUM(StreamingHandleErrorMode, ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 2e71c96b954..246cdf6f684 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -140,6 +140,14 @@ enum class DefaultTableEngine DECLARE_SETTING_ENUM(DefaultTableEngine) +enum class CleanDeletedRows +{ + Never = 0, /// Disable. + Always, +}; + +DECLARE_SETTING_ENUM(CleanDeletedRows) + enum class MySQLDataTypesSupport { DECIMAL, // convert MySQL's decimal and number to ClickHouse Decimal when applicable @@ -165,6 +173,8 @@ enum class DistributedDDLOutputMode THROW, NULL_STATUS_ON_TIMEOUT, NEVER_THROW, + THROW_ONLY_ACTIVE, + NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE, }; DECLARE_SETTING_ENUM(DistributedDDLOutputMode) diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index acd26ca338b..1f246af74d3 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -85,10 +85,7 @@ std::string DataTypeMap::doGetName() const std::string DataTypeMap::doGetPrettyName(size_t indent) const { WriteBufferFromOwnString s; - s << "Map(\n" - << fourSpaceIndent(indent + 1) << key_type->getPrettyName(indent + 1) << ",\n" - << fourSpaceIndent(indent + 1) << value_type->getPrettyName(indent + 1) << '\n' - << fourSpaceIndent(indent) << ')'; + s << "Map(" << key_type->getPrettyName(indent) << ", " << value_type->getPrettyName(indent) << ')'; return s.str(); } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index fd2e5e6a784..db8a14c537a 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -98,21 +98,38 @@ std::string DataTypeTuple::doGetPrettyName(size_t indent) const { size_t size = elems.size(); WriteBufferFromOwnString s; - s << "Tuple(\n"; - for (size_t i = 0; i != size; ++i) + /// If the Tuple is named, we will output it in multiple lines with indentation. + if (have_explicit_names) { - if (i != 0) - s << ",\n"; + s << "Tuple(\n"; - s << fourSpaceIndent(indent + 1); - if (have_explicit_names) - s << backQuoteIfNeed(names[i]) << ' '; + for (size_t i = 0; i != size; ++i) + { + if (i != 0) + s << ",\n"; - s << elems[i]->getPrettyName(indent + 1); + s << fourSpaceIndent(indent + 1) + << backQuoteIfNeed(names[i]) << ' ' + << elems[i]->getPrettyName(indent + 1); + } + + s << ')'; + } + else + { + s << "Tuple("; + + for (size_t i = 0; i != size; ++i) + { + if (i != 0) + s << ", "; + s << elems[i]->getPrettyName(indent); + } + + s << ')'; } - s << '\n' << fourSpaceIndent(indent) << ')'; return s.str(); } diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index efac2454a03..a7cc1b21389 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -77,10 +77,11 @@ static Block flattenImpl(const Block & block, bool flatten_named_tuple) for (const auto & elem : block) { - if (const DataTypeArray * type_arr = typeid_cast(elem.type.get())) + if (isNested(elem.type)) { - const DataTypeTuple * type_tuple = typeid_cast(type_arr->getNestedType().get()); - if (type_tuple && type_tuple->haveExplicitNames()) + const DataTypeArray * type_arr = assert_cast(elem.type.get()); + const DataTypeTuple * type_tuple = assert_cast(type_arr->getNestedType().get()); + if (type_tuple->haveExplicitNames()) { const DataTypes & element_types = type_tuple->getElements(); const Strings & names = type_tuple->getElementNames(); @@ -149,7 +150,7 @@ Block flatten(const Block & block) } -Block flattenArrayOfTuples(const Block & block) +Block flattenNested(const Block & block) { return flattenImpl(block, false); } diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index e009ceb18fe..85c29d2c08f 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -20,13 +20,13 @@ namespace Nested /// Flat a column of nested type into columns /// 1) For named tuples,t Tuple(x .., y ..., ...), replace it with t.x ..., t.y ... , ... - /// 2) For an Array with named Tuple element column, a Array(Tuple(x ..., y ..., ...)), replace it with multiple Array Columns, a.x ..., a.y ..., ... + /// 2) For an Nested column, a Array(Tuple(x ..., y ..., ...)), replace it with multiple Array Columns, a.x ..., a.y ..., ... Block flatten(const Block & block); - /// Same as flatten but only for Array with named Tuple element column. - Block flattenArrayOfTuples(const Block & block); + /// Same as flatten but only for Nested column. + Block flattenNested(const Block & block); - /// Collect Array columns in a form of `column_name.element_name` to single Array(Tuple(...)) column. + /// Collect Array columns in a form of `column_name.element_name` to single Nested column. NamesAndTypesList collect(const NamesAndTypesList & names_and_types); /// Convert old-style nested (single arrays with same prefix, `n.a`, `n.b`...) to subcolumns of data type Nested. diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index c804f58c567..0d99b741a23 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -348,6 +348,8 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams( { auto mutable_column = column->assumeMutable(); ColumnArray & column_array = typeid_cast(*mutable_column); + size_t prev_last_offset = column_array.getOffsets().back(); + settings.path.push_back(Substream::ArraySizes); if (auto cached_column = getFromSubstreamsCache(cache, settings.path)) @@ -371,9 +373,9 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams( /// Number of values corresponding with `offset_values` must be read. size_t last_offset = offset_values.back(); - if (last_offset < nested_column->size()) + if (last_offset < prev_last_offset) throw Exception(ErrorCodes::LOGICAL_ERROR, "Nested column is longer than last offset"); - size_t nested_limit = last_offset - nested_column->size(); + size_t nested_limit = last_offset - prev_last_offset; if (unlikely(nested_limit > MAX_ARRAYS_SIZE)) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array sizes are too large: {}", nested_limit); @@ -388,7 +390,7 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams( /// Check consistency between offsets and elements subcolumns. /// But if elements column is empty - it's ok for columns of Nested types that was added by ALTER. if (!nested_column->empty() && nested_column->size() != last_offset) - throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all array values: read just {} of {}", + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all array values: read just {} of {}", toString(nested_column->size()), toString(last_offset)); column = std::move(mutable_column); @@ -443,7 +445,7 @@ static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && r if (*istr.position() == ',') ++istr.position(); else - throw ParsingException(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, + throw Exception(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Cannot read array from text, expected comma or end of array, found '{}'", *istr.position()); } diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 15203bdc9fa..d9efc6fff10 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -359,7 +359,7 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col nested_column.popBack(1); if (null_representation.find('\t') != std::string::npos || null_representation.find('\n') != std::string::npos) - throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "TSV custom null representation " + throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "TSV custom null representation " "containing '\\t' or '\\n' may not work correctly for large input."); WriteBufferFromOwnString parsed_value; @@ -367,7 +367,7 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col nested_serialization->serializeTextEscaped(nested_column, nested_column.size() - 1, parsed_value, settings); else nested_serialization->serializeTextRaw(nested_column, nested_column.size() - 1, parsed_value, settings); - throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing \"{}{}\" as Nullable" + throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing \"{}{}\" as Nullable" " at position {}: got \"{}\", which was deserialized as \"{}\". " "It seems that input data is ill-formatted.", std::string(pos, buf.buffer().end()), @@ -452,7 +452,7 @@ ReturnType SerializationNullable::deserializeTextQuotedImpl(IColumn & column, Re /// It can happen only if there is an unquoted string instead of a number. /// We also should delete incorrectly deserialized value from nested column. nested_column.popBack(1); - throw DB::ParsingException( + throw DB::Exception( ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing Nullable: got an unquoted string {} instead of a number", String(buf.position(), std::min(10ul, buf.available()))); @@ -589,12 +589,12 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB if (null_representation.find(settings.csv.delimiter) != std::string::npos || null_representation.find('\r') != std::string::npos || null_representation.find('\n') != std::string::npos) - throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "CSV custom null representation containing " + throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "CSV custom null representation containing " "format_csv_delimiter, '\\r' or '\\n' may not work correctly for large input."); WriteBufferFromOwnString parsed_value; nested_serialization->serializeTextCSV(nested_column, nested_column.size() - 1, parsed_value, settings); - throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing \"{}{}\" as Nullable" + throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing \"{}{}\" as Nullable" " at position {}: got \"{}\", which was deserialized as \"{}\". " "It seems that input data is ill-formatted.", std::string(pos, buf.buffer().end()), diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 788ff429088..b2b083fd466 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -335,6 +335,22 @@ void SerializationString::deserializeTextJSON(IColumn & column, ReadBuffer & ist { read(column, [&](ColumnString::Chars & data) { readJSONArrayInto(data, istr); }); } + else if (settings.json.read_bools_as_strings && !istr.eof() && (*istr.position() == 't' || *istr.position() == 'f')) + { + String str_value; + if (*istr.position() == 't') + { + assertString("true", istr); + str_value = "true"; + } + else if (*istr.position() == 'f') + { + assertString("false", istr); + str_value = "false"; + } + + read(column, [&](ColumnString::Chars & data) { data.insert(str_value.begin(), str_value.end()); }); + } else if (settings.json.read_numbers_as_strings && !istr.eof() && *istr.position() != '"') { String field; diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 1daa6351c23..8a5ba5f033f 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -622,4 +623,16 @@ void DatabaseAtomic::checkDetachedTableNotInUse(const UUID & uuid) assertDetachedTableNotInUse(uuid); } +void registerDatabaseAtomic(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + return make_shared( + args.database_name, + args.metadata_path, + args.uuid, + args.context); + }; + factory.registerDatabase("Atomic", create_fn); +} } diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index 3a3dea1d38e..e2e0d52cd88 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -140,4 +141,14 @@ void DatabaseDictionary::shutdown() { } +void registerDatabaseDictionary(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + return make_shared( + args.database_name, + args.context); + }; + factory.registerDatabase("Dictionary", create_fn); +} } diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index a967ecf67c6..fc8073eac3b 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -1,60 +1,15 @@ -#include - #include -#include -#include -#include -#include -#include -#include + +#include #include #include -#include #include #include #include #include -#include -#include #include #include - -#include "config.h" - -#if USE_MYSQL -# include -# include -# include -# include -# include -# include -# include -# include -#endif - -#if USE_MYSQL || USE_LIBPQXX -#include -#include -#endif - -#if USE_LIBPQXX -#include -#include -#include -#include -#endif - -#if USE_SQLITE -#include -#endif - -#if USE_AWS_S3 -#include -#endif - -#if USE_HDFS -#include -#endif +#include namespace fs = std::filesystem; @@ -67,7 +22,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int UNKNOWN_DATABASE_ENGINE; extern const int CANNOT_CREATE_DATABASE; - extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; } void cckMetadataPathForOrdinary(const ASTCreateQuery & create, const String & metadata_path) @@ -103,8 +58,54 @@ void cckMetadataPathForOrdinary(const ASTCreateQuery & create, const String & me } +/// validate validates the database engine that's specified in the create query for +/// engine arguments, settings and table overrides. +void validate(const ASTCreateQuery & create_query) + +{ + auto * storage = create_query.storage; + + /// Check engine may have arguments + static const std::unordered_set engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL", + "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3", "HDFS"}; + + const String & engine_name = storage->engine->name; + bool engine_may_have_arguments = engines_with_arguments.contains(engine_name); + + if (storage->engine->arguments && !engine_may_have_arguments) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have arguments", engine_name); + + /// Check engine may have settings + bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializedPostgreSQL"; + bool has_unexpected_element = storage->engine->parameters || storage->partition_by || + storage->primary_key || storage->order_by || + storage->sample_by; + if (has_unexpected_element || (!may_have_settings && storage->settings)) + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_AST, + "Database engine `{}` cannot have parameters, primary_key, order_by, sample_by, settings", engine_name); + + /// Check engine with table overrides + static const std::unordered_set engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"}; + if (create_query.table_overrides && !engines_with_table_overrides.contains(engine_name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have table overrides", engine_name); +} + DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context) { + const auto engine_name = create.storage->engine->name; + /// check if the database engine is a valid one before proceeding + if (!database_engines.contains(engine_name)) + { + auto hints = getHints(engine_name); + if (!hints.empty()) + throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine {}. Maybe you meant: {}", engine_name, toString(hints)); + else + throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", create.storage->engine->name); + } + + /// if the engine is found (i.e. registered with the factory instance), then validate if the + /// supplied engine arguments, settings and table overrides are valid for the engine. + validate(create); cckMetadataPathForOrdinary(create, metadata_path); DatabasePtr impl = getImpl(create, metadata_path, context); @@ -119,383 +120,42 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m return impl; } -template -static inline ValueType safeGetLiteralValue(const ASTPtr &ast, const String &engine_name) +void DatabaseFactory::registerDatabase(const std::string & name, CreatorFn creator_fn) { - if (!ast || !ast->as()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} requested literal argument.", engine_name); + if (!database_engines.emplace(name, std::move(creator_fn)).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "DatabaseFactory: the database engine name '{}' is not unique", name); +} - return ast->as()->value.safeGet(); +DatabaseFactory & DatabaseFactory::instance() +{ + static DatabaseFactory db_fact; + return db_fact; } DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context) { - auto * engine_define = create.storage; + auto * storage = create.storage; const String & database_name = create.getDatabase(); - const String & engine_name = engine_define->engine->name; - const UUID & uuid = create.uuid; - - static const std::unordered_set database_engines{"Ordinary", "Atomic", "Memory", - "Dictionary", "Lazy", "Replicated", "MySQL", "MaterializeMySQL", "MaterializedMySQL", - "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3", "HDFS"}; - - if (!database_engines.contains(engine_name)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine name `{}` does not exist", engine_name); - - static const std::unordered_set engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL", - "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3", "HDFS"}; - - static const std::unordered_set engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"}; - bool engine_may_have_arguments = engines_with_arguments.contains(engine_name); - - if (engine_define->engine->arguments && !engine_may_have_arguments) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have arguments", engine_name); - - bool has_unexpected_element = engine_define->engine->parameters || engine_define->partition_by || - engine_define->primary_key || engine_define->order_by || - engine_define->sample_by; - bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializedPostgreSQL"; - - if (has_unexpected_element || (!may_have_settings && engine_define->settings)) - throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_AST, - "Database engine `{}` cannot have parameters, primary_key, order_by, sample_by, settings", engine_name); - - if (create.table_overrides && !engines_with_table_overrides.contains(engine_name)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have table overrides", engine_name); - - if (engine_name == "Ordinary") - { - if (!create.attach && !context->getSettingsRef().allow_deprecated_database_ordinary) - throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, - "Ordinary database engine is deprecated (see also allow_deprecated_database_ordinary setting)"); - - return std::make_shared(database_name, metadata_path, context); - } - - if (engine_name == "Atomic") - return std::make_shared(database_name, metadata_path, uuid, context); - else if (engine_name == "Memory") - return std::make_shared(database_name, context); - else if (engine_name == "Dictionary") - return std::make_shared(database_name, context); - -#if USE_MYSQL - - else if (engine_name == "MySQL" || engine_name == "MaterializeMySQL" || engine_name == "MaterializedMySQL") - { - const ASTFunction * engine = engine_define->engine; - if (!engine->arguments) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name); - - StorageMySQL::Configuration configuration; - ASTs & arguments = engine->arguments->children; - auto mysql_settings = std::make_unique(); - - if (auto named_collection = tryGetNamedCollectionWithOverrides(arguments, context)) - { - configuration = StorageMySQL::processNamedCollectionResult(*named_collection, *mysql_settings, context, false); - } - else - { - if (arguments.size() != 4) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "MySQL database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments."); - - - arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context); - const auto & host_port = safeGetLiteralValue(arguments[0], engine_name); - - if (engine_name == "MySQL") - { - size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; - configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306); - } - else - { - const auto & [remote_host, remote_port] = parseAddress(host_port, 3306); - configuration.host = remote_host; - configuration.port = remote_port; - } - - configuration.database = safeGetLiteralValue(arguments[1], engine_name); - configuration.username = safeGetLiteralValue(arguments[2], engine_name); - configuration.password = safeGetLiteralValue(arguments[3], engine_name); - } - - try - { - if (engine_name == "MySQL") - { - mysql_settings->loadFromQueryContext(context, *engine_define); - if (engine_define->settings) - mysql_settings->loadFromQuery(*engine_define); - - auto mysql_pool = createMySQLPoolWithFailover(configuration, *mysql_settings); - - return std::make_shared( - context, database_name, metadata_path, engine_define, configuration.database, - std::move(mysql_settings), std::move(mysql_pool), create.attach); - } - - MySQLClient client(configuration.host, configuration.port, configuration.username, configuration.password); - auto mysql_pool = mysqlxx::Pool(configuration.database, configuration.host, configuration.username, configuration.password, configuration.port); - - auto materialize_mode_settings = std::make_unique(); - - if (engine_define->settings) - materialize_mode_settings->loadFromQuery(*engine_define); - - if (uuid == UUIDHelpers::Nil) - { - auto print_create_ast = create.clone(); - print_create_ast->as()->attach = false; - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "The MaterializedMySQL database engine no longer supports Ordinary databases. To re-create the database, delete " - "the old one by executing \"rm -rf {}{{,.sql}}\", then re-create the database with the following query: {}", - metadata_path, - queryToString(print_create_ast)); - } - - return std::make_shared( - context, database_name, metadata_path, uuid, configuration.database, std::move(mysql_pool), - std::move(client), std::move(materialize_mode_settings)); - } - catch (...) - { - const auto & exception_message = getCurrentExceptionMessage(true); - throw Exception(ErrorCodes::CANNOT_CREATE_DATABASE, "Cannot create MySQL database, because {}", exception_message); - } - } -#endif - - else if (engine_name == "Lazy") - { - const ASTFunction * engine = engine_define->engine; - - if (!engine->arguments || engine->arguments->children.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Lazy database require cache_expiration_time_seconds argument"); - - const auto & arguments = engine->arguments->children; - - const auto cache_expiration_time_seconds = safeGetLiteralValue(arguments[0], "Lazy"); - return std::make_shared(database_name, metadata_path, cache_expiration_time_seconds, context); - } - - else if (engine_name == "Replicated") - { - const ASTFunction * engine = engine_define->engine; - - if (!engine->arguments || engine->arguments->children.size() != 3) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replicated database requires 3 arguments: zookeeper path, shard name and replica name"); - - auto & arguments = engine->arguments->children; - for (auto & engine_arg : arguments) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); - - String zookeeper_path = safeGetLiteralValue(arguments[0], "Replicated"); - String shard_name = safeGetLiteralValue(arguments[1], "Replicated"); - String replica_name = safeGetLiteralValue(arguments[2], "Replicated"); - - zookeeper_path = context->getMacros()->expand(zookeeper_path); - shard_name = context->getMacros()->expand(shard_name); - replica_name = context->getMacros()->expand(replica_name); - - DatabaseReplicatedSettings database_replicated_settings{}; - if (engine_define->settings) - database_replicated_settings.loadFromQuery(*engine_define); - - return std::make_shared(database_name, metadata_path, uuid, - zookeeper_path, shard_name, replica_name, - std::move(database_replicated_settings), context); - } - -#if USE_LIBPQXX - - else if (engine_name == "PostgreSQL") - { - const ASTFunction * engine = engine_define->engine; - if (!engine->arguments) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name); - - ASTs & engine_args = engine->arguments->children; - auto use_table_cache = false; - StoragePostgreSQL::Configuration configuration; - - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) - { - configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection, context, false); - use_table_cache = named_collection->getOrDefault("use_table_cache", 0); - } - else - { - if (engine_args.size() < 4 || engine_args.size() > 6) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "PostgreSQL Database require `host:port`, `database_name`, `username`, `password`" - "[, `schema` = "", `use_table_cache` = 0"); - - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); - - const auto & host_port = safeGetLiteralValue(engine_args[0], engine_name); - size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; - - configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432); - configuration.database = safeGetLiteralValue(engine_args[1], engine_name); - configuration.username = safeGetLiteralValue(engine_args[2], engine_name); - configuration.password = safeGetLiteralValue(engine_args[3], engine_name); - - bool is_deprecated_syntax = false; - if (engine_args.size() >= 5) - { - auto arg_value = engine_args[4]->as()->value; - if (arg_value.getType() == Field::Types::Which::String) - { - configuration.schema = safeGetLiteralValue(engine_args[4], engine_name); - } - else - { - use_table_cache = safeGetLiteralValue(engine_args[4], engine_name); - LOG_WARNING(&Poco::Logger::get("DatabaseFactory"), "A deprecated syntax of PostgreSQL database engine is used"); - is_deprecated_syntax = true; - } - } - - if (!is_deprecated_syntax && engine_args.size() >= 6) - use_table_cache = safeGetLiteralValue(engine_args[5], engine_name); - } - - const auto & settings = context->getSettingsRef(); - auto pool = std::make_shared( - configuration, - settings.postgresql_connection_pool_size, - settings.postgresql_connection_pool_wait_timeout, - POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, - settings.postgresql_connection_pool_auto_close_connection); - - return std::make_shared( - context, metadata_path, engine_define, database_name, configuration, pool, use_table_cache); - } - else if (engine_name == "MaterializedPostgreSQL") - { - const ASTFunction * engine = engine_define->engine; - if (!engine->arguments) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name); - - ASTs & engine_args = engine->arguments->children; - StoragePostgreSQL::Configuration configuration; - - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) - { - configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection, context, false); - } - else - { - if (engine_args.size() != 4) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "MaterializedPostgreSQL Database require `host:port`, `database_name`, `username`, `password`."); - - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); - - auto parsed_host_port = parseAddress(safeGetLiteralValue(engine_args[0], engine_name), 5432); - - configuration.host = parsed_host_port.first; - configuration.port = parsed_host_port.second; - configuration.database = safeGetLiteralValue(engine_args[1], engine_name); - configuration.username = safeGetLiteralValue(engine_args[2], engine_name); - configuration.password = safeGetLiteralValue(engine_args[3], engine_name); - } - - auto connection_info = postgres::formatConnectionString( - configuration.database, configuration.host, configuration.port, configuration.username, configuration.password); - - auto postgresql_replica_settings = std::make_unique(); - if (engine_define->settings) - postgresql_replica_settings->loadFromQuery(*engine_define); - - return std::make_shared( - context, metadata_path, uuid, create.attach, - database_name, configuration.database, connection_info, - std::move(postgresql_replica_settings)); - } - - -#endif - -#if USE_SQLITE - else if (engine_name == "SQLite") - { - const ASTFunction * engine = engine_define->engine; - - if (!engine->arguments || engine->arguments->children.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "SQLite database requires 1 argument: database path"); - - const auto & arguments = engine->arguments->children; - - String database_path = safeGetLiteralValue(arguments[0], "SQLite"); - - return std::make_shared(context, engine_define, create.attach, database_path); - } -#endif - - else if (engine_name == "Filesystem") - { - const ASTFunction * engine = engine_define->engine; - - /// If init_path is empty, then the current path will be used - std::string init_path; - - if (engine->arguments && !engine->arguments->children.empty()) - { - if (engine->arguments->children.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filesystem database requires at most 1 argument: filesystem_path"); - - const auto & arguments = engine->arguments->children; - init_path = safeGetLiteralValue(arguments[0], engine_name); - } - - return std::make_shared(database_name, init_path, context); - } - -#if USE_AWS_S3 - else if (engine_name == "S3") - { - const ASTFunction * engine = engine_define->engine; - - DatabaseS3::Configuration config; - - if (engine->arguments && !engine->arguments->children.empty()) - { - ASTs & engine_args = engine->arguments->children; - config = DatabaseS3::parseArguments(engine_args, context); - } - - return std::make_shared(database_name, config, context); - } -#endif - -#if USE_HDFS - else if (engine_name == "HDFS") - { - const ASTFunction * engine = engine_define->engine; - - /// If source_url is empty, then table name must contain full url - std::string source_url; - - if (engine->arguments && !engine->arguments->children.empty()) - { - if (engine->arguments->children.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS database requires at most 1 argument: source_url"); - - const auto & arguments = engine->arguments->children; - source_url = safeGetLiteralValue(arguments[0], engine_name); - } - - return std::make_shared(database_name, source_url, context); - } -#endif - - throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", engine_name); + const String & engine_name = storage->engine->name; + + bool has_engine_args = false; + if (storage->engine->arguments) + has_engine_args = true; + + ASTs empty_engine_args; + Arguments arguments{ + .engine_name = engine_name, + .engine_args = has_engine_args ? storage->engine->arguments->children : empty_engine_args, + .create_query = create, + .database_name = database_name, + .metadata_path = metadata_path, + .uuid = create.uuid, + .context = context}; + + // creator_fn creates and returns a DatabasePtr with the supplied arguments + auto creator_fn = database_engines.at(engine_name); + + return creator_fn(arguments); } } diff --git a/src/Databases/DatabaseFactory.h b/src/Databases/DatabaseFactory.h index cb631cd76d0..6b92963f46e 100644 --- a/src/Databases/DatabaseFactory.h +++ b/src/Databases/DatabaseFactory.h @@ -1,19 +1,70 @@ #pragma once +#include #include #include +#include +#include namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + class ASTCreateQuery; -class DatabaseFactory +template +static inline ValueType safeGetLiteralValue(const ASTPtr &ast, const String &engine_name) +{ + if (!ast || !ast->as()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} requested literal argument.", engine_name); + + return ast->as()->value.safeGet(); +} + +class DatabaseFactory : private boost::noncopyable, public IHints<> { public: - static DatabasePtr get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context); - static DatabasePtr getImpl(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context); + static DatabaseFactory & instance(); + + struct Arguments + { + const String & engine_name; + ASTs & engine_args; + ASTStorage * storage; + const ASTCreateQuery & create_query; + const String & database_name; + const String & metadata_path; + const UUID & uuid; + ContextPtr & context; + }; + + DatabasePtr get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context); + + using CreatorFn = std::function; + + using DatabaseEngines = std::unordered_map; + + void registerDatabase(const std::string & name, CreatorFn creator_fn); + + const DatabaseEngines & getDatabaseEngines() const { return database_engines; } + + std::vector getAllRegisteredNames() const override + { + std::vector result; + auto getter = [](const auto & pair) { return pair.first; }; + std::transform(database_engines.begin(), database_engines.end(), std::back_inserter(result), getter); + return result; + } + +private: + DatabaseEngines database_engines; + + DatabasePtr getImpl(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context); }; } diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index ca1b5b27a59..5564f1d07cf 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -237,4 +238,28 @@ DatabaseTablesIteratorPtr DatabaseFilesystem::getTablesIterator(ContextPtr, cons return std::make_unique(Tables{}, getDatabaseName()); } +void registerDatabaseFilesystem(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + auto * engine_define = args.create_query.storage; + const ASTFunction * engine = engine_define->engine; + const String & engine_name = engine_define->engine->name; + + /// If init_path is empty, then the current path will be used + std::string init_path; + + if (engine->arguments && !engine->arguments->children.empty()) + { + if (engine->arguments->children.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filesystem database requires at most 1 argument: filesystem_path"); + + const auto & arguments = engine->arguments->children; + init_path = safeGetLiteralValue(arguments[0], engine_name); + } + + return std::make_shared(args.database_name, init_path, args.context); + }; + factory.registerDatabase("Filesystem", create_fn); +} } diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index 750d79c8493..6810f655116 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -2,6 +2,7 @@ #if USE_HDFS +#include #include #include @@ -237,6 +238,30 @@ DatabaseTablesIteratorPtr DatabaseHDFS::getTablesIterator(ContextPtr, const Filt return std::make_unique(Tables{}, getDatabaseName()); } +void registerDatabaseHDFS(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + auto * engine_define = args.create_query.storage; + const ASTFunction * engine = engine_define->engine; + const String & engine_name = engine_define->engine->name; + + /// If source_url is empty, then table name must contain full url + std::string source_url; + + if (engine->arguments && !engine->arguments->children.empty()) + { + if (engine->arguments->children.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS database requires at most 1 argument: source_url"); + + const auto & arguments = engine->arguments->children; + source_url = safeGetLiteralValue(arguments[0], engine_name); + } + + return std::make_shared(args.database_name, source_url, args.context); + }; + factory.registerDatabase("HDFS", create_fn); +} } // DB #endif diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index c6249c68933..fcd832e7cc2 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -7,6 +8,7 @@ #include #include #include +#include #include #include @@ -34,6 +36,7 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int UNSUPPORTED_METHOD; extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } @@ -354,4 +357,26 @@ const StoragePtr & DatabaseLazyIterator::table() const return current_storage; } +void registerDatabaseLazy(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + auto * engine_define = args.create_query.storage; + const ASTFunction * engine = engine_define->engine; + + if (!engine->arguments || engine->arguments->children.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Lazy database require cache_expiration_time_seconds argument"); + + const auto & arguments = engine->arguments->children; + + const auto cache_expiration_time_seconds = safeGetLiteralValue(arguments[0], "Lazy"); + + return make_shared( + args.database_name, + args.metadata_path, + cache_expiration_time_seconds, + args.context); + }; + factory.registerDatabase("Lazy", create_fn); +} } diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 2a7a2ad8ccc..794eebbc399 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -209,4 +210,15 @@ std::vector> DatabaseMemory::getTablesForBackup(co return res; } +void registerDatabaseMemory(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + return make_shared( + args.database_name, + args.context); + }; + factory.registerDatabase("Memory", create_fn); +} + } diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 9a9dcf22c88..8973b533720 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -37,6 +38,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int UNKNOWN_DATABASE_ENGINE; } static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; @@ -321,4 +323,19 @@ void DatabaseOrdinary::commitAlterTable(const StorageID &, const String & table_ } } +void registerDatabaseOrdinary(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + if (!args.create_query.attach && !args.context->getSettingsRef().allow_deprecated_database_ordinary) + throw Exception( + ErrorCodes::UNKNOWN_DATABASE_ENGINE, + "Ordinary database engine is deprecated (see also allow_deprecated_database_ordinary setting)"); + return make_shared( + args.database_name, + args.metadata_path, + args.context); + }; + factory.registerDatabase("Ordinary", create_fn); +} } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 952c0689a0d..d484b223706 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -1054,7 +1055,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep for (auto & [_, intermediate, to] : replicated_tables_to_rename) rename_table(intermediate, to); - LOG_DEBUG(log, "Renames completed succesessfully"); + LOG_DEBUG(log, "Renames completed successfully"); for (const auto & id : dropped_tables) DatabaseCatalog::instance().waitTableFinallyDropped(id); @@ -1652,4 +1653,41 @@ bool DatabaseReplicated::shouldReplicateQuery(const ContextPtr & query_context, return true; } +void registerDatabaseReplicated(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + auto * engine_define = args.create_query.storage; + const ASTFunction * engine = engine_define->engine; + + if (!engine->arguments || engine->arguments->children.size() != 3) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replicated database requires 3 arguments: zookeeper path, shard name and replica name"); + + auto & arguments = engine->arguments->children; + for (auto & engine_arg : arguments) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.context); + + String zookeeper_path = safeGetLiteralValue(arguments[0], "Replicated"); + String shard_name = safeGetLiteralValue(arguments[1], "Replicated"); + String replica_name = safeGetLiteralValue(arguments[2], "Replicated"); + + zookeeper_path = args.context->getMacros()->expand(zookeeper_path); + shard_name = args.context->getMacros()->expand(shard_name); + replica_name = args.context->getMacros()->expand(replica_name); + + DatabaseReplicatedSettings database_replicated_settings{}; + if (engine_define->settings) + database_replicated_settings.loadFromQuery(*engine_define); + + return std::make_shared( + args.database_name, + args.metadata_path, + args.uuid, + zookeeper_path, + shard_name, + replica_name, + std::move(database_replicated_settings), args.context); + }; + factory.registerDatabase("Replicated", create_fn); +} } diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index b92b4a971c1..1721b0e9e97 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -2,6 +2,7 @@ #if USE_AWS_S3 +#include #include #include @@ -307,6 +308,24 @@ DatabaseTablesIteratorPtr DatabaseS3::getTablesIterator(ContextPtr, const Filter return std::make_unique(Tables{}, getDatabaseName()); } -} +void registerDatabaseS3(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + auto * engine_define = args.create_query.storage; + const ASTFunction * engine = engine_define->engine; + DatabaseS3::Configuration config; + + if (engine->arguments && !engine->arguments->children.empty()) + { + ASTs & engine_args = engine->arguments->children; + config = DatabaseS3::parseArguments(engine_args, args.context); + } + + return std::make_shared(args.database_name, config, args.context); + }; + factory.registerDatabase("S3", create_fn); +} +} #endif diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index afc09fbe62a..bda48737621 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -65,6 +65,11 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo query->replace(ast_create_query.select, metadata.select.select_query); } + if (metadata.refresh) + { + query->replace(ast_create_query.refresh_strategy, metadata.refresh); + } + /// MaterializedView, Dictionary are types of CREATE query without storage. if (ast_create_query.storage) { diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index a31e74cc7ae..cbb080a0baa 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -2,13 +2,20 @@ #if USE_MYSQL +# include +# include # include -# include +# include +# include # include # include # include +# include +# include +# include # include +# include # include # include # include @@ -21,6 +28,7 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; + extern const int BAD_ARGUMENTS; } DatabaseMaterializedMySQL::DatabaseMaterializedMySQL( @@ -179,6 +187,86 @@ void DatabaseMaterializedMySQL::stopReplication() started_up = false; } +void registerDatabaseMaterializedMySQL(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + auto * engine_define = args.create_query.storage; + const ASTFunction * engine = engine_define->engine; + const String & engine_name = engine_define->engine->name; + + if (!engine->arguments) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name); + StorageMySQL::Configuration configuration; + ASTs & arguments = engine->arguments->children; + auto mysql_settings = std::make_unique(); + + if (auto named_collection = tryGetNamedCollectionWithOverrides(arguments, args.context)) + { + configuration = StorageMySQL::processNamedCollectionResult(*named_collection, *mysql_settings, args.context, false); + } + else + { + if (arguments.size() != 4) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "MySQL database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments."); + + + arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], args.context); + const auto & host_port = safeGetLiteralValue(arguments[0], engine_name); + + if (engine_name == "MySQL") + { + size_t max_addresses = args.context->getSettingsRef().glob_expansion_max_elements; + configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306); + } + else + { + const auto & [remote_host, remote_port] = parseAddress(host_port, 3306); + configuration.host = remote_host; + configuration.port = remote_port; + } + + configuration.database = safeGetLiteralValue(arguments[1], engine_name); + configuration.username = safeGetLiteralValue(arguments[2], engine_name); + configuration.password = safeGetLiteralValue(arguments[3], engine_name); + } + MySQLClient client(configuration.host, configuration.port, configuration.username, configuration.password); + auto mysql_pool + = mysqlxx::Pool(configuration.database, configuration.host, configuration.username, configuration.password, configuration.port); + + auto materialize_mode_settings = std::make_unique(); + + if (engine_define->settings) + materialize_mode_settings->loadFromQuery(*engine_define); + + if (args.uuid == UUIDHelpers::Nil) + { + auto print_create_ast = args.create_query.clone(); + print_create_ast->as()->attach = false; + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "The MaterializedMySQL database engine no longer supports Ordinary databases. To re-create the database, delete " + "the old one by executing \"rm -rf {}{{,.sql}}\", then re-create the database with the following query: {}", + args.metadata_path, + queryToString(print_create_ast)); + } + + return make_shared( + args.context, + args.database_name, + args.metadata_path, + args.uuid, + configuration.database, + std::move(mysql_pool), + std::move(client), + std::move(materialize_mode_settings)); + }; + factory.registerDatabase("MaterializeMySQL", create_fn); + factory.registerDatabase("MaterializedMySQL", create_fn); +} + } #endif diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 7d2ed7a9662..96a5c3a18ce 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -2,6 +2,7 @@ #if USE_MYSQL # include +# include # include # include # include @@ -14,6 +15,7 @@ # include # include # include +# include # include # include # include @@ -21,8 +23,11 @@ # include # include # include +# include +# include # include # include +# include # include # include # include @@ -41,6 +46,8 @@ namespace ErrorCodes extern const int TABLE_IS_DROPPED; extern const int TABLE_ALREADY_EXISTS; extern const int UNEXPECTED_AST_STRUCTURE; + extern const int CANNOT_CREATE_DATABASE; + extern const int BAD_ARGUMENTS; } constexpr static const auto suffix = ".remove_flag"; @@ -504,6 +511,77 @@ void DatabaseMySQL::createTable(ContextPtr local_context, const String & table_n attachTable(local_context, table_name, storage, {}); } +void registerDatabaseMySQL(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + auto * engine_define = args.create_query.storage; + const ASTFunction * engine = engine_define->engine; + const String & engine_name = engine_define->engine->name; + if (!engine->arguments) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name); + + StorageMySQL::Configuration configuration; + ASTs & arguments = engine->arguments->children; + auto mysql_settings = std::make_unique(); + + if (auto named_collection = tryGetNamedCollectionWithOverrides(arguments, args.context)) + { + configuration = StorageMySQL::processNamedCollectionResult(*named_collection, *mysql_settings, args.context, false); + } + else + { + if (arguments.size() != 4) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "MySQL database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments."); + + + arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], args.context); + const auto & host_port = safeGetLiteralValue(arguments[0], engine_name); + + if (engine_name == "MySQL") + { + size_t max_addresses = args.context->getSettingsRef().glob_expansion_max_elements; + configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306); + } + else + { + const auto & [remote_host, remote_port] = parseAddress(host_port, 3306); + configuration.host = remote_host; + configuration.port = remote_port; + } + + configuration.database = safeGetLiteralValue(arguments[1], engine_name); + configuration.username = safeGetLiteralValue(arguments[2], engine_name); + configuration.password = safeGetLiteralValue(arguments[3], engine_name); + } + mysql_settings->loadFromQueryContext(args.context, *engine_define); + if (engine_define->settings) + mysql_settings->loadFromQuery(*engine_define); + + auto mysql_pool = createMySQLPoolWithFailover(configuration, *mysql_settings); + + try + { + return make_shared( + args.context, + args.database_name, + args.metadata_path, + engine_define, + configuration.database, + std::move(mysql_settings), + std::move(mysql_pool), + args.create_query.attach); + } + catch (...) + { + const auto & exception_message = getCurrentExceptionMessage(true); + throw Exception(ErrorCodes::CANNOT_CREATE_DATABASE, "Cannot create MySQL database, because {}", exception_message); + } + }; + factory.registerDatabase("MySQL", create_fn); +} } #endif diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 78be0611631..a659821e179 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -8,23 +8,25 @@ #include #include #include +#include +#include #include #include #include #include #include +#include +#include #include #include #include #include #include -#include #include #include +#include #include #include -#include -#include namespace DB { @@ -471,6 +473,59 @@ DatabaseTablesIteratorPtr DatabaseMaterializedPostgreSQL::getTablesIterator( return DatabaseAtomic::getTablesIterator(StorageMaterializedPostgreSQL::makeNestedTableContext(local_context), filter_by_table_name); } +void registerDatabaseMaterializedPostgreSQL(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + auto * engine_define = args.create_query.storage; + const ASTFunction * engine = engine_define->engine; + ASTs & engine_args = engine->arguments->children; + const String & engine_name = engine_define->engine->name; + + if (!engine->arguments) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name); + + StoragePostgreSQL::Configuration configuration; + + if (!engine->arguments) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name); + + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, args.context)) + { + configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection, args.context, false); + } + else + { + if (engine_args.size() != 4) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "MaterializedPostgreSQL Database require `host:port`, `database_name`, `username`, `password`."); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.context); + + auto parsed_host_port = parseAddress(safeGetLiteralValue(engine_args[0], engine_name), 5432); + + configuration.host = parsed_host_port.first; + configuration.port = parsed_host_port.second; + configuration.database = safeGetLiteralValue(engine_args[1], engine_name); + configuration.username = safeGetLiteralValue(engine_args[2], engine_name); + configuration.password = safeGetLiteralValue(engine_args[3], engine_name); + } + + auto connection_info = postgres::formatConnectionString( + configuration.database, configuration.host, configuration.port, configuration.username, configuration.password); + + auto postgresql_replica_settings = std::make_unique(); + if (engine_define->settings) + postgresql_replica_settings->loadFromQuery(*engine_define); + + return std::make_shared( + args.context, args.metadata_path, args.uuid, args.create_query.attach, + args.database_name, configuration.database, connection_info, + std::move(postgresql_replica_settings)); + }; + factory.registerDatabase("MaterializedPostgreSQL", create_fn); +} } #endif diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 24f04c16029..1fe5c078581 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -6,14 +6,18 @@ #include #include +#include #include #include +#include #include #include #include #include #include #include +#include +#include #include #include #include @@ -478,6 +482,83 @@ ASTPtr DatabasePostgreSQL::getColumnDeclaration(const DataTypePtr & data_type) c return std::make_shared(data_type->getName()); } +void registerDatabasePostgreSQL(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + auto * engine_define = args.create_query.storage; + const ASTFunction * engine = engine_define->engine; + ASTs & engine_args = engine->arguments->children; + const String & engine_name = engine_define->engine->name; + + if (!engine->arguments) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name); + + auto use_table_cache = false; + StoragePostgreSQL::Configuration configuration; + + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, args.context)) + { + configuration = StoragePostgreSQL::processNamedCollectionResult(*named_collection, args.context, false); + use_table_cache = named_collection->getOrDefault("use_table_cache", 0); + } + else + { + if (engine_args.size() < 4 || engine_args.size() > 6) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "PostgreSQL Database require `host:port`, `database_name`, `username`, `password`" + "[, `schema` = "", `use_table_cache` = 0"); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.context); + + const auto & host_port = safeGetLiteralValue(engine_args[0], engine_name); + size_t max_addresses = args.context->getSettingsRef().glob_expansion_max_elements; + + configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432); + configuration.database = safeGetLiteralValue(engine_args[1], engine_name); + configuration.username = safeGetLiteralValue(engine_args[2], engine_name); + configuration.password = safeGetLiteralValue(engine_args[3], engine_name); + + bool is_deprecated_syntax = false; + if (engine_args.size() >= 5) + { + auto arg_value = engine_args[4]->as()->value; + if (arg_value.getType() == Field::Types::Which::String) + { + configuration.schema = safeGetLiteralValue(engine_args[4], engine_name); + } + else + { + use_table_cache = safeGetLiteralValue(engine_args[4], engine_name); + LOG_WARNING(&Poco::Logger::get("DatabaseFactory"), "A deprecated syntax of PostgreSQL database engine is used"); + is_deprecated_syntax = true; + } + } + + if (!is_deprecated_syntax && engine_args.size() >= 6) + use_table_cache = safeGetLiteralValue(engine_args[5], engine_name); + } + + const auto & settings = args.context->getSettingsRef(); + auto pool = std::make_shared( + configuration, + settings.postgresql_connection_pool_size, + settings.postgresql_connection_pool_wait_timeout, + POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, + settings.postgresql_connection_pool_auto_close_connection); + + return std::make_shared( + args.context, + args.metadata_path, + engine_define, + args.database_name, + configuration, + pool, + use_table_cache); + }; + factory.registerDatabase("PostgreSQL", create_fn); +} } #endif diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index eb7f72b61aa..469ca52890a 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -207,7 +207,6 @@ PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList( columns.push_back(NameAndTypePair(column_name, data_type)); auto attgenerated = std::get<6>(row); - LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: attgenerated: {}", attgenerated); attributes.emplace( column_name, diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index d031fd8e420..605a354bd7e 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -5,11 +5,11 @@ #include #include #include +#include #include #include #include #include -#include #include #include @@ -21,6 +21,7 @@ namespace ErrorCodes { extern const int SQLITE_ENGINE_ERROR; extern const int UNKNOWN_TABLE; + extern const int BAD_ARGUMENTS; } DatabaseSQLite::DatabaseSQLite( @@ -201,6 +202,24 @@ ASTPtr DatabaseSQLite::getCreateTableQueryImpl(const String & table_name, Contex return create_table_query; } +void registerDatabaseSQLite(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + auto * engine_define = args.create_query.storage; + const ASTFunction * engine = engine_define->engine; + + if (!engine->arguments || engine->arguments->children.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "SQLite database requires 1 argument: database path"); + + const auto & arguments = engine->arguments->children; + + String database_path = safeGetLiteralValue(arguments[0], "SQLite"); + + return std::make_shared(args.context, engine_define, args.create_query.attach, database_path); + }; + factory.registerDatabase("SQLite", create_fn); +} } #endif diff --git a/src/Databases/TablesDependencyGraph.h b/src/Databases/TablesDependencyGraph.h index e71d5ecc5fc..50be3bbf969 100644 --- a/src/Databases/TablesDependencyGraph.h +++ b/src/Databases/TablesDependencyGraph.h @@ -60,7 +60,7 @@ public: /// Removes all dependencies of "table_id", returns those dependencies. std::vector removeDependencies(const StorageID & table_id, bool remove_isolated_tables = false); - /// Removes a table from the graph and removes all references to in from the graph (both from its dependencies and dependents). + /// Removes a table from the graph and removes all references to it from the graph (both from its dependencies and dependents). bool removeTable(const StorageID & table_id); /// Removes tables from the graph by a specified filter. diff --git a/src/Databases/registerDatabases.cpp b/src/Databases/registerDatabases.cpp new file mode 100644 index 00000000000..4f7c229bdf4 --- /dev/null +++ b/src/Databases/registerDatabases.cpp @@ -0,0 +1,72 @@ +#include +#include + + +namespace DB +{ + +void registerDatabaseAtomic(DatabaseFactory & factory); +void registerDatabaseOrdinary(DatabaseFactory & factory); +void registerDatabaseDictionary(DatabaseFactory & factory); +void registerDatabaseMemory(DatabaseFactory & factory); +void registerDatabaseLazy(DatabaseFactory & factory); +void registerDatabaseFilesystem(DatabaseFactory & factory); +void registerDatabaseReplicated(DatabaseFactory & factory); + +#if USE_MYSQL +void registerDatabaseMySQL(DatabaseFactory & factory); +void registerDatabaseMaterializedMySQL(DatabaseFactory & factory); +#endif + +#if USE_LIBPQXX +void registerDatabasePostgreSQL(DatabaseFactory & factory); + +void registerDatabaseMaterializedPostgreSQL(DatabaseFactory & factory); +#endif + +#if USE_SQLITE +void registerDatabaseSQLite(DatabaseFactory & factory); +#endif + +#if USE_AWS_S3 +void registerDatabaseS3(DatabaseFactory & factory); +#endif + +#if USE_HDFS +void registerDatabaseHDFS(DatabaseFactory & factory); +#endif + +void registerDatabases() +{ + auto & factory = DatabaseFactory::instance(); + registerDatabaseAtomic(factory); + registerDatabaseOrdinary(factory); + registerDatabaseDictionary(factory); + registerDatabaseMemory(factory); + registerDatabaseLazy(factory); + registerDatabaseFilesystem(factory); + registerDatabaseReplicated(factory); + +#if USE_MYSQL + registerDatabaseMySQL(factory); + registerDatabaseMaterializedMySQL(factory); +#endif + +#if USE_LIBPQXX + registerDatabasePostgreSQL(factory); + registerDatabaseMaterializedPostgreSQL(factory); +#endif + +#if USE_SQLITE + registerDatabaseSQLite(factory); +#endif + +#if USE_AWS_S3 + registerDatabaseS3(factory); +#endif + +#if USE_HDFS + registerDatabaseHDFS(factory); +#endif +} +} diff --git a/src/Databases/registerDatabases.h b/src/Databases/registerDatabases.h new file mode 100644 index 00000000000..dbf1bbb6e64 --- /dev/null +++ b/src/Databases/registerDatabases.h @@ -0,0 +1,6 @@ +#pragma once + +namespace DB +{ +void registerDatabases(); +} diff --git a/src/Dictionaries/Embedded/GeoDictionariesLoader.cpp b/src/Dictionaries/Embedded/GeoDictionariesLoader.cpp index 93612491b20..bfbbf774148 100644 --- a/src/Dictionaries/Embedded/GeoDictionariesLoader.cpp +++ b/src/Dictionaries/Embedded/GeoDictionariesLoader.cpp @@ -4,6 +4,9 @@ #include "GeodataProviders/HierarchiesProvider.h" #include "GeodataProviders/NamesProvider.h" +namespace DB +{ + std::unique_ptr GeoDictionariesLoader::reloadRegionsHierarchies(const Poco::Util::AbstractConfiguration & config) { static constexpr auto config_key = "path_to_regions_hierarchy_file"; @@ -27,3 +30,5 @@ std::unique_ptr GeoDictionariesLoader::reloadRegionsNames(const Po auto data_provider = std::make_unique(directory); return std::make_unique(std::move(data_provider)); } + +} diff --git a/src/Dictionaries/Embedded/GeoDictionariesLoader.h b/src/Dictionaries/Embedded/GeoDictionariesLoader.h index d09e69cf561..f795456985e 100644 --- a/src/Dictionaries/Embedded/GeoDictionariesLoader.h +++ b/src/Dictionaries/Embedded/GeoDictionariesLoader.h @@ -6,6 +6,9 @@ #include +namespace DB +{ + // Default implementation of geo dictionaries loader used by native server application class GeoDictionariesLoader { @@ -13,3 +16,5 @@ public: static std::unique_ptr reloadRegionsHierarchies(const Poco::Util::AbstractConfiguration & config); static std::unique_ptr reloadRegionsNames(const Poco::Util::AbstractConfiguration & config); }; + +} diff --git a/src/Dictionaries/Embedded/GeodataProviders/Entries.h b/src/Dictionaries/Embedded/GeodataProviders/Entries.h index 942c2f5adbc..6b27c5ae19e 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/Entries.h +++ b/src/Dictionaries/Embedded/GeodataProviders/Entries.h @@ -3,6 +3,9 @@ #include #include "Types.h" +namespace DB +{ + struct RegionEntry { RegionID id; @@ -17,3 +20,5 @@ struct RegionNameEntry RegionID id; std::string name; }; + +} diff --git a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp index 210459da0be..5d8781d6f23 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp @@ -9,6 +9,9 @@ namespace fs = std::filesystem; +namespace DB +{ + bool RegionsHierarchyDataSource::isModified() const { return updates_tracker.isModified(); @@ -17,7 +20,7 @@ bool RegionsHierarchyDataSource::isModified() const IRegionsHierarchyReaderPtr RegionsHierarchyDataSource::createReader() { updates_tracker.fixCurrentVersion(); - auto file_reader = std::make_shared(path); + auto file_reader = std::make_shared(path); return std::make_unique(std::move(file_reader)); } @@ -73,3 +76,5 @@ IRegionsHierarchyDataSourcePtr RegionsHierarchiesDataProvider::getHierarchySourc throw Poco::Exception("Regions hierarchy '" + name + "' not found"); } + +} diff --git a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.h b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.h index c2e36f59e1e..6ded62dbf83 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.h +++ b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.h @@ -5,6 +5,8 @@ #include #include +namespace DB +{ // Represents local file with regions hierarchy dump class RegionsHierarchyDataSource : public IRegionsHierarchyDataSource @@ -50,3 +52,5 @@ public: private: void discoverFilesWithCustomHierarchies(); }; + +} diff --git a/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.cpp b/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.cpp index 68bd6142416..d9ac19f4d67 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.cpp @@ -3,6 +3,8 @@ #include #include +namespace DB +{ bool RegionsHierarchyFormatReader::readNext(RegionEntry & entry) { @@ -15,11 +17,11 @@ bool RegionsHierarchyFormatReader::readNext(RegionEntry & entry) Int32 read_parent_id = 0; Int8 read_type = 0; - DB::readIntText(read_region_id, *input); - DB::assertChar('\t', *input); - DB::readIntText(read_parent_id, *input); - DB::assertChar('\t', *input); - DB::readIntText(read_type, *input); + readIntText(read_region_id, *input); + assertChar('\t', *input); + readIntText(read_parent_id, *input); + assertChar('\t', *input); + readIntText(read_type, *input); /** Then there can be a newline (old version) * or tab, the region's population, line feed (new version). @@ -29,11 +31,11 @@ bool RegionsHierarchyFormatReader::readNext(RegionEntry & entry) { ++input->position(); UInt64 population_big = 0; - DB::readIntText(population_big, *input); + readIntText(population_big, *input); population = population_big > std::numeric_limits::max() ? std::numeric_limits::max() : static_cast(population_big); } - DB::assertChar('\n', *input); + assertChar('\n', *input); if (read_region_id <= 0 || read_type < 0) continue; @@ -55,3 +57,5 @@ bool RegionsHierarchyFormatReader::readNext(RegionEntry & entry) return false; } + +} diff --git a/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.h b/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.h index 64f393ada62..ebd8fca4ff9 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.h +++ b/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.h @@ -3,15 +3,19 @@ #include #include "IHierarchiesProvider.h" +namespace DB +{ // Reads regions hierarchy in geoexport format class RegionsHierarchyFormatReader : public IRegionsHierarchyReader { private: - DB::ReadBufferPtr input; + ReadBufferPtr input; public: - explicit RegionsHierarchyFormatReader(DB::ReadBufferPtr input_) : input(std::move(input_)) {} + explicit RegionsHierarchyFormatReader(ReadBufferPtr input_) : input(std::move(input_)) {} bool readNext(RegionEntry & entry) override; }; + +} diff --git a/src/Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h b/src/Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h index f7d51135440..68ab0fdca2d 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h +++ b/src/Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h @@ -5,6 +5,8 @@ #include #include "Entries.h" +namespace DB +{ // Iterates over all regions in data source class IRegionsHierarchyReader @@ -46,3 +48,5 @@ public: }; using IRegionsHierarchiesDataProviderPtr = std::shared_ptr; + +} diff --git a/src/Dictionaries/Embedded/GeodataProviders/INamesProvider.h b/src/Dictionaries/Embedded/GeodataProviders/INamesProvider.h index 679c14d546b..6cd7d78f6d5 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/INamesProvider.h +++ b/src/Dictionaries/Embedded/GeodataProviders/INamesProvider.h @@ -3,6 +3,8 @@ #include #include "Entries.h" +namespace DB +{ // Iterates over all name entries in data source class ILanguageRegionsNamesReader @@ -49,3 +51,5 @@ public: }; using IRegionsNamesDataProviderPtr = std::unique_ptr; + +} diff --git a/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.cpp b/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.cpp index 9d0c57f18eb..99216507c10 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.cpp @@ -2,6 +2,8 @@ #include +namespace DB +{ bool LanguageRegionsNamesFormatReader::readNext(RegionNameEntry & entry) { @@ -10,10 +12,10 @@ bool LanguageRegionsNamesFormatReader::readNext(RegionNameEntry & entry) Int32 read_region_id; std::string region_name; - DB::readIntText(read_region_id, *input); - DB::assertChar('\t', *input); - DB::readString(region_name, *input); - DB::assertChar('\n', *input); + readIntText(read_region_id, *input); + assertChar('\t', *input); + readString(region_name, *input); + assertChar('\n', *input); if (read_region_id <= 0) continue; @@ -25,3 +27,5 @@ bool LanguageRegionsNamesFormatReader::readNext(RegionNameEntry & entry) return false; } + +} diff --git a/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.h b/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.h index 49d324d434e..50b2abd47c1 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.h +++ b/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.h @@ -3,15 +3,19 @@ #include #include "INamesProvider.h" +namespace DB +{ // Reads regions names list in geoexport format class LanguageRegionsNamesFormatReader : public ILanguageRegionsNamesReader { private: - DB::ReadBufferPtr input; + ReadBufferPtr input; public: - explicit LanguageRegionsNamesFormatReader(DB::ReadBufferPtr input_) : input(std::move(input_)) {} + explicit LanguageRegionsNamesFormatReader(ReadBufferPtr input_) : input(std::move(input_)) {} bool readNext(RegionNameEntry & entry) override; }; + +} diff --git a/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp b/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp index 5f79fda070f..e6a8d308e87 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp @@ -6,6 +6,9 @@ namespace fs = std::filesystem; +namespace DB +{ + bool LanguageRegionsNamesDataSource::isModified() const { return updates_tracker.isModified(); @@ -19,7 +22,7 @@ size_t LanguageRegionsNamesDataSource::estimateTotalSize() const ILanguageRegionsNamesReaderPtr LanguageRegionsNamesDataSource::createReader() { updates_tracker.fixCurrentVersion(); - auto file_reader = std::make_shared(path); + auto file_reader = std::make_shared(path); return std::make_unique(std::move(file_reader)); } @@ -51,3 +54,5 @@ std::string RegionsNamesDataProvider::getDataFilePath(const std::string & langua { return directory + "/regions_names_" + language + ".txt"; } + +} diff --git a/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.h b/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.h index 2d49cceab86..8ba1f33d2c4 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.h +++ b/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.h @@ -3,6 +3,8 @@ #include #include "INamesProvider.h" +namespace DB +{ // Represents local file with list of regions ids / names class LanguageRegionsNamesDataSource : public ILanguageRegionsNamesDataSource @@ -46,3 +48,5 @@ public: private: std::string getDataFilePath(const std::string & language) const; }; + +} diff --git a/src/Dictionaries/Embedded/GeodataProviders/Types.h b/src/Dictionaries/Embedded/GeodataProviders/Types.h index e63f6bae716..0fd6a01051a 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/Types.h +++ b/src/Dictionaries/Embedded/GeodataProviders/Types.h @@ -2,6 +2,8 @@ #include +namespace DB +{ using RegionID = UInt32; using RegionDepth = UInt8; @@ -16,3 +18,5 @@ enum class RegionType : Int8 Area = 5, City = 6, }; + +} diff --git a/src/Dictionaries/Embedded/RegionsHierarchies.cpp b/src/Dictionaries/Embedded/RegionsHierarchies.cpp index be828b8b281..c3c62bcc83c 100644 --- a/src/Dictionaries/Embedded/RegionsHierarchies.cpp +++ b/src/Dictionaries/Embedded/RegionsHierarchies.cpp @@ -3,6 +3,8 @@ #include #include +namespace DB +{ RegionsHierarchies::RegionsHierarchies(IRegionsHierarchiesDataProviderPtr data_provider) { @@ -19,3 +21,5 @@ RegionsHierarchies::RegionsHierarchies(IRegionsHierarchiesDataProviderPtr data_p reload(); } + +} diff --git a/src/Dictionaries/Embedded/RegionsHierarchies.h b/src/Dictionaries/Embedded/RegionsHierarchies.h index 925b7b490ff..996c1177b6e 100644 --- a/src/Dictionaries/Embedded/RegionsHierarchies.h +++ b/src/Dictionaries/Embedded/RegionsHierarchies.h @@ -5,6 +5,8 @@ #include "GeodataProviders/IHierarchiesProvider.h" #include "RegionsHierarchy.h" +namespace DB +{ /** Contains several hierarchies of regions. * Used to support several different perspectives on the ownership of regions by countries. @@ -37,3 +39,5 @@ public: return it->second; } }; + +} diff --git a/src/Dictionaries/Embedded/RegionsHierarchy.cpp b/src/Dictionaries/Embedded/RegionsHierarchy.cpp index c266bf7efb8..23f4c250a23 100644 --- a/src/Dictionaries/Embedded/RegionsHierarchy.cpp +++ b/src/Dictionaries/Embedded/RegionsHierarchy.cpp @@ -12,7 +12,7 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_DATA; -} + extern const int LOGICAL_ERROR; } @@ -54,9 +54,8 @@ void RegionsHierarchy::reload() if (region_entry.id > max_region_id) { if (region_entry.id > max_size) - throw DB::Exception(DB::ErrorCodes::INCORRECT_DATA, - "Region id is too large: {}, should be not more than {}", - DB::toString(region_entry.id), DB::toString(max_size)); + throw Exception( + ErrorCodes::INCORRECT_DATA, "Region id is too large: {}, should be not more than {}", region_entry.id, max_size); max_region_id = region_entry.id; @@ -112,16 +111,18 @@ void RegionsHierarchy::reload() ++depth; if (depth == std::numeric_limits::max()) - throw Poco::Exception( - "Logical error in regions hierarchy: region " + DB::toString(current) + " possible is inside infinite loop"); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Logical error in regions hierarchy: region {} possible is inside infinite loop", current); current = new_parents[current]; if (current == 0) break; if (current > max_region_id) - throw Poco::Exception( - "Logical error in regions hierarchy: region " + DB::toString(current) + " (specified as parent) doesn't exist"); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Logical error in regions hierarchy: region {} (specified as parent) doesn't exist", + current); if (types[current] == RegionType::City) new_city[i] = current; @@ -156,3 +157,5 @@ void RegionsHierarchy::reload() populations.swap(new_populations); depths.swap(new_depths); } + +} diff --git a/src/Dictionaries/Embedded/RegionsHierarchy.h b/src/Dictionaries/Embedded/RegionsHierarchy.h index 508bca0d1e1..5d9aacb9512 100644 --- a/src/Dictionaries/Embedded/RegionsHierarchy.h +++ b/src/Dictionaries/Embedded/RegionsHierarchy.h @@ -6,6 +6,8 @@ #include "GeodataProviders/IHierarchiesProvider.h" #include +namespace DB +{ class IRegionsHierarchyDataProvider; @@ -129,3 +131,5 @@ public: return populations[region]; } }; + +} diff --git a/src/Dictionaries/Embedded/RegionsNames.cpp b/src/Dictionaries/Embedded/RegionsNames.cpp index 93ca9e6dbc9..847dfe99b10 100644 --- a/src/Dictionaries/Embedded/RegionsNames.cpp +++ b/src/Dictionaries/Embedded/RegionsNames.cpp @@ -10,12 +10,12 @@ namespace DB { + namespace ErrorCodes { extern const int INCORRECT_DATA; + extern const int LOGICAL_ERROR; } -} - RegionsNames::RegionsNames(IRegionsNamesDataProviderPtr data_provider) { @@ -30,7 +30,7 @@ RegionsNames::RegionsNames(IRegionsNamesDataProviderPtr data_provider) std::string RegionsNames::dumpSupportedLanguagesNames() { - DB::WriteBufferFromOwnString out; + WriteBufferFromOwnString out; for (size_t i = 0; i < total_languages; ++i) { if (i > 0) @@ -74,7 +74,8 @@ void RegionsNames::reload() size_t old_size = new_chars.size(); if (new_chars.capacity() < old_size + name_entry.name.length() + 1) - throw Poco::Exception("Logical error. Maybe size estimate of " + names_source->getSourceName() + " is wrong."); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Logical error. Maybe size estimate of {} is wrong", names_source->getSourceName()); new_chars.resize(old_size + name_entry.name.length() + 1); memcpy(new_chars.data() + old_size, name_entry.name.c_str(), name_entry.name.length() + 1); @@ -84,9 +85,8 @@ void RegionsNames::reload() max_region_id = name_entry.id; if (name_entry.id > max_size) - throw DB::Exception(DB::ErrorCodes::INCORRECT_DATA, - "Region id is too large: {}, should be not more than {}", - DB::toString(name_entry.id), DB::toString(max_size)); + throw Exception( + ErrorCodes::INCORRECT_DATA, "Region id is too large: {}, should be not more than {}", name_entry.id, max_size); } while (name_entry.id >= new_names_refs.size()) @@ -102,3 +102,5 @@ void RegionsNames::reload() for (size_t language_id = 0; language_id < total_languages; ++language_id) names_refs[language_id].resize(max_region_id + 1, StringRef("", 0)); } + +} diff --git a/src/Dictionaries/Embedded/RegionsNames.h b/src/Dictionaries/Embedded/RegionsNames.h index 1e0ea3f0923..0053c74745a 100644 --- a/src/Dictionaries/Embedded/RegionsNames.h +++ b/src/Dictionaries/Embedded/RegionsNames.h @@ -7,6 +7,8 @@ #include #include "GeodataProviders/INamesProvider.h" +namespace DB +{ /** A class that allows you to recognize by region id its text name in one of the supported languages. * @@ -111,3 +113,5 @@ public: void reload(); }; + +} diff --git a/src/Dictionaries/HashedDictionaryParallelLoader.h b/src/Dictionaries/HashedDictionaryParallelLoader.h index 907a987555e..ec892af7e36 100644 --- a/src/Dictionaries/HashedDictionaryParallelLoader.h +++ b/src/Dictionaries/HashedDictionaryParallelLoader.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -53,7 +54,7 @@ public: LOG_TRACE(dictionary.log, "Will load the dictionary using {} threads (with {} backlog)", shards, backlog); shards_slots.resize(shards); - std::iota(shards_slots.begin(), shards_slots.end(), 0); + iota(shards_slots.data(), shards_slots.size(), UInt64(0)); for (size_t shard = 0; shard < shards; ++shard) { diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index df3ae439b00..6f800bd921d 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -5,6 +5,7 @@ #include +#include #include #include #include @@ -507,7 +508,7 @@ const IColumn * unrollSimplePolygons(const ColumnPtr & column, Offset & offset) if (!ptr_polygons) throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column containing arrays of points"); offset.ring_offsets.assign(ptr_polygons->getOffsets()); - std::iota(offset.polygon_offsets.begin(), offset.polygon_offsets.end(), 1); + iota(offset.polygon_offsets.data(), offset.polygon_offsets.size(), IColumn::Offsets::value_type(1)); offset.multi_polygon_offsets.assign(offset.polygon_offsets); return ptr_polygons->getDataPtr().get(); diff --git a/src/Dictionaries/PolygonDictionaryUtils.h b/src/Dictionaries/PolygonDictionaryUtils.h index 0238ef0b2b9..63d97e9dabd 100644 --- a/src/Dictionaries/PolygonDictionaryUtils.h +++ b/src/Dictionaries/PolygonDictionaryUtils.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -184,7 +185,7 @@ public: { setBoundingBox(); std::vector order(polygons.size()); - std::iota(order.begin(), order.end(), 0); + iota(order.data(), order.size(), size_t(0)); root = makeCell(min_x, min_y, max_x, max_y, order); } diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 8e21b95ce68..5e77ff61789 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 9f4b59a6443..c3baf3fdbda 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index fdf82430812..c8b3aeaca28 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -22,11 +22,13 @@ struct S3ObjectStorageSettings const S3Settings::RequestSettings & request_settings_, uint64_t min_bytes_for_seek_, int32_t list_object_keys_size_, - int32_t objects_chunk_size_to_delete_) + int32_t objects_chunk_size_to_delete_, + bool read_only_) : request_settings(request_settings_) , min_bytes_for_seek(min_bytes_for_seek_) , list_object_keys_size(list_object_keys_size_) , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) + , read_only(read_only_) {} S3Settings::RequestSettings request_settings; @@ -34,6 +36,7 @@ struct S3ObjectStorageSettings uint64_t min_bytes_for_seek; int32_t list_object_keys_size; int32_t objects_chunk_size_to_delete; + bool read_only; }; @@ -166,6 +169,8 @@ public: ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + bool isReadOnly() const override { return s3_settings.get()->read_only; } + private: void setNewSettings(std::unique_ptr && s3_settings_); diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 2ddde4021b3..8ea559be5ba 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -1,4 +1,5 @@ #include +#include "IO/S3/Client.h" #if USE_AWS_S3 @@ -34,7 +35,8 @@ std::unique_ptr getSettings(const Poco::Util::AbstractC request_settings, config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".list_object_keys_size", 1000), - config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000)); + config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), + config.getBool(config_prefix + ".readonly", false)); } std::unique_ptr getClient( @@ -92,14 +94,15 @@ std::unique_ptr getClient( HTTPHeaderEntries headers = S3::getHTTPHeaders(config_prefix, config); S3::ServerSideEncryptionKMSConfig sse_kms_config = S3::getSSEKMSConfig(config_prefix, config); - client_configuration.retryStrategy - = std::make_shared( - config.getUInt64(config_prefix + ".retry_attempts", settings.request_settings.retry_attempts)); + S3::ClientSettings client_settings{ + .use_virtual_addressing = uri.is_virtual_hosted_style, + .disable_checksum = local_settings.s3_disable_checksum, + .gcs_issue_compose_request = config.getBool("s3.gcs_issue_compose_request", false), + }; return S3::ClientFactory::instance().create( client_configuration, - uri.is_virtual_hosted_style, - local_settings.s3_disable_checksum, + client_settings, config.getString(config_prefix + ".access_key_id", ""), config.getString(config_prefix + ".secret_access_key", ""), config.getString(config_prefix + ".server_side_encryption_customer_key_base64", ""), diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 9cc7cb3b89e..a7e9fb8e99f 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -450,10 +450,11 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo break; case FormatSettings::EscapingRule::JSON: result += fmt::format( - ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, read_objects_as_strings={}, read_numbers_as_strings={}, " + ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, read_bools_as_strings={}, read_objects_as_strings={}, read_numbers_as_strings={}, " "read_arrays_as_strings={}, try_infer_objects_as_tuples={}, infer_incomplete_types_as_strings={}, try_infer_objects={}", settings.json.try_infer_numbers_from_strings, settings.json.read_bools_as_numbers, + settings.json.read_bools_as_strings, settings.json.read_objects_as_strings, settings.json.read_numbers_as_strings, settings.json.read_arrays_as_strings, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 15743365d7d..0344ed54ae3 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -111,6 +111,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; format_settings.json.quote_decimals = settings.output_format_json_quote_decimals; format_settings.json.read_bools_as_numbers = settings.input_format_json_read_bools_as_numbers; + format_settings.json.read_bools_as_strings = settings.input_format_json_read_bools_as_strings; format_settings.json.read_numbers_as_strings = settings.input_format_json_read_numbers_as_strings; format_settings.json.read_objects_as_strings = settings.input_format_json_read_objects_as_strings; format_settings.json.read_arrays_as_strings = settings.input_format_json_read_arrays_as_strings; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 8d5c044a311..5982d30f6a7 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -204,6 +204,7 @@ struct FormatSettings bool ignore_unknown_keys_in_named_tuple = false; bool serialize_as_strings = false; bool read_bools_as_numbers = true; + bool read_bools_as_strings = true; bool read_numbers_as_strings = true; bool read_objects_as_strings = true; bool read_arrays_as_strings = true; diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index b8b9a9ecb0d..779f38032d8 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -43,7 +43,7 @@ namespace JSONUtils { const auto current_object_size = memory.size() + static_cast(pos - in.position()); if (min_bytes != 0 && current_object_size > 10 * min_bytes) - throw ParsingException(ErrorCodes::INCORRECT_DATA, + throw Exception(ErrorCodes::INCORRECT_DATA, "Size of JSON object at position {} is extremely large. Expected not greater than {} bytes, but current is {} bytes per row. " "Increase the value setting 'min_chunk_bytes_for_parallel_parsing' or check your data manually, " "most likely JSON is malformed", in.count(), min_bytes, current_object_size); diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index 4c25460eb63..8286b24d0a6 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -120,7 +120,7 @@ Block NativeReader::read() if (istr.eof()) { if (use_index) - throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Input doesn't contain all data for index."); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Input doesn't contain all data for index."); return res; } diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index e2ba188d015..f065d2f0f4d 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -377,6 +377,22 @@ namespace type_indexes.erase(TypeIndex::UInt8); } + /// If we have Bool and String types convert all numbers to String. + /// It's applied only when setting input_format_json_read_bools_as_strings is enabled. + void transformJSONBoolsAndStringsToString(DataTypes & data_types, TypeIndexesSet & type_indexes) + { + if (!type_indexes.contains(TypeIndex::String) || !type_indexes.contains(TypeIndex::UInt8)) + return; + + for (auto & type : data_types) + { + if (isBool(type)) + type = std::make_shared(); + } + + type_indexes.erase(TypeIndex::UInt8); + } + /// If we have type Nothing/Nullable(Nothing) and some other non Nothing types, /// convert all Nothing/Nullable(Nothing) types to the first non Nothing. /// For example, when we have [Nothing, Array(Int64)] it will convert it to [Array(Int64), Array(Int64)] @@ -628,6 +644,10 @@ namespace if (settings.json.read_bools_as_numbers) transformBoolsAndNumbersToNumbers(data_types, type_indexes); + /// Convert Bool to String if needed. + if (settings.json.read_bools_as_strings) + transformJSONBoolsAndStringsToString(data_types, type_indexes); + if (settings.json.try_infer_objects_as_tuples) mergeJSONPaths(data_types, type_indexes, settings, json_info); }; diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index 7c7ccac8b01..cc9cf380693 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -14,6 +14,7 @@ void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); void registerFileSegmentationEngineRegexp(FormatFactory & factory); void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); void registerFileSegmentationEngineJSONAsObject(FormatFactory & factory); +void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory); #if USE_HIVE void registerFileSegmentationEngineHiveText(FormatFactory & factory); #endif @@ -160,6 +161,7 @@ void registerFormats() registerFileSegmentationEngineJSONEachRow(factory); registerFileSegmentationEngineJSONAsString(factory); registerFileSegmentationEngineJSONAsObject(factory); + registerFileSegmentationEngineJSONCompactEachRow(factory); #if USE_HIVE registerFileSegmentationEngineHiveText(factory); #endif diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 5a06c363065..eed75788fcd 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -47,7 +47,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index 75c91a2e964..fb5109eaa88 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -159,7 +159,6 @@ private: class FunctionCapture : public IFunctionBase { public: - using Capture = ExecutableFunctionCapture::Capture; using CapturePtr = ExecutableFunctionCapture::CapturePtr; FunctionCapture( @@ -201,10 +200,10 @@ public: FunctionCaptureOverloadResolver( ExpressionActionsPtr expression_actions_, - const Names & captured_names_, - const NamesAndTypesList & lambda_arguments_, - const DataTypePtr & function_return_type_, - const String & expression_return_name_) + const Names & captured_names, + const NamesAndTypesList & lambda_arguments, + const DataTypePtr & function_return_type, + const String & expression_return_name) : expression_actions(std::move(expression_actions_)) { /// Check that expression does not contain unusual actions that will break columns structure. @@ -219,9 +218,9 @@ public: arguments_map[arg.name] = arg.type; DataTypes captured_types; - captured_types.reserve(captured_names_.size()); + captured_types.reserve(captured_names.size()); - for (const auto & captured_name : captured_names_) + for (const auto & captured_name : captured_names) { auto it = arguments_map.find(captured_name); if (it == arguments_map.end()) @@ -232,21 +231,21 @@ public: } DataTypes argument_types; - argument_types.reserve(lambda_arguments_.size()); - for (const auto & lambda_argument : lambda_arguments_) + argument_types.reserve(lambda_arguments.size()); + for (const auto & lambda_argument : lambda_arguments) argument_types.push_back(lambda_argument.type); - return_type = std::make_shared(argument_types, function_return_type_); + return_type = std::make_shared(argument_types, function_return_type); name = "Capture[" + toString(captured_types) + "](" + toString(argument_types) + ") -> " - + function_return_type_->getName(); + + function_return_type->getName(); capture = std::make_shared(Capture{ - .captured_names = captured_names_, + .captured_names = captured_names, .captured_types = std::move(captured_types), - .lambda_arguments = lambda_arguments_, - .return_name = expression_return_name_, - .return_type = function_return_type_, + .lambda_arguments = lambda_arguments, + .return_name = expression_return_name, + .return_type = function_return_type, }); } diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index 3098d02630a..a5e819179d6 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #ifdef __SSE4_2__ # include @@ -246,8 +247,7 @@ struct ByteEditDistanceImpl ResultType insertion = 0; ResultType deletion = 0; - for (size_t i = 0; i <= haystack_size; ++i) - distances0[i] = i; + iota(distances0.data(), haystack_size + 1, ResultType(0)); for (size_t pos_needle = 0; pos_needle < needle_size; ++pos_needle) { diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index a7ab80f697a..9773673c63c 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -74,6 +74,8 @@ public: size_t getNumberOfArguments() const override { return 0; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + /// Called if at least one function argument is a lambda expression. /// For argument-lambda expressions, it defines the types of arguments of these expressions. void getLambdaArgumentTypes(DataTypes & arguments) const override @@ -370,10 +372,10 @@ public: /// Put all the necessary columns multiplied by the sizes of arrays into the columns. auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(column_first_array->getOffsets())); - auto * replicated_column_function = typeid_cast(replicated_column_function_ptr.get()); - replicated_column_function->appendArguments(arrays); + auto & replicated_column_function = typeid_cast(*replicated_column_function_ptr); + replicated_column_function.appendArguments(arrays); - auto lambda_result = replicated_column_function->reduce(); + auto lambda_result = replicated_column_function.reduce(); /// Convert LowCardinality(T) -> T and Const(LowCardinality(T)) -> Const(T), /// because we removed LowCardinality from return type of lambda expression. diff --git a/src/Functions/array/arrayRandomSample.cpp b/src/Functions/array/arrayRandomSample.cpp index 1e28e089a2a..40344efb077 100644 --- a/src/Functions/array/arrayRandomSample.cpp +++ b/src/Functions/array/arrayRandomSample.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -80,7 +81,7 @@ public: const size_t cur_samples = std::min(num_elements, samples); indices.resize(num_elements); - std::iota(indices.begin(), indices.end(), prev_array_offset); + iota(indices.data(), indices.size(), prev_array_offset); std::shuffle(indices.begin(), indices.end(), rng); for (UInt64 i = 0; i < cur_samples; i++) diff --git a/src/Functions/array/arrayShuffle.cpp b/src/Functions/array/arrayShuffle.cpp index faa5ae47b29..10cb51d27d2 100644 --- a/src/Functions/array/arrayShuffle.cpp +++ b/src/Functions/array/arrayShuffle.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -150,7 +151,7 @@ ColumnPtr FunctionArrayShuffleImpl::executeGeneric(const ColumnArray & a size_t size = offsets.size(); size_t nested_size = array.getData().size(); IColumn::Permutation permutation(nested_size); - std::iota(std::begin(permutation), std::end(permutation), 0); + iota(permutation.data(), permutation.size(), IColumn::Permutation::value_type(0)); ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) diff --git a/src/Functions/array/arraySort.cpp b/src/Functions/array/arraySort.cpp index a853289e8cc..184b1f82280 100644 --- a/src/Functions/array/arraySort.cpp +++ b/src/Functions/array/arraySort.cpp @@ -1,5 +1,6 @@ -#include #include +#include +#include namespace DB { @@ -55,9 +56,7 @@ ColumnPtr ArraySortImpl::execute( size_t size = offsets.size(); size_t nested_size = array.getData().size(); IColumn::Permutation permutation(nested_size); - - for (size_t i = 0; i < nested_size; ++i) - permutation[i] = i; + iota(permutation.data(), nested_size, IColumn::Permutation::value_type(0)); ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index c9c9020f068..f75e6eb4fc8 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -412,14 +412,14 @@ private: }; -/** TimeDiff(t1, t2) +/** timeDiff(t1, t2) * t1 and t2 can be Date or DateTime */ class FunctionTimeDiff : public IFunction { using ColumnDateTime64 = ColumnDecimal; public: - static constexpr auto name = "TimeDiff"; + static constexpr auto name = "timeDiff"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override diff --git a/src/Functions/format.cpp b/src/Functions/format.cpp index 41b6d65023b..036ff9f0c57 100644 --- a/src/Functions/format.cpp +++ b/src/Functions/format.cpp @@ -39,6 +39,7 @@ public: size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override diff --git a/src/Functions/formatReadableDecimalSize.cpp b/src/Functions/formatReadableDecimalSize.cpp index b6fd0de8f7b..1aa5abc526e 100644 --- a/src/Functions/formatReadableDecimalSize.cpp +++ b/src/Functions/formatReadableDecimalSize.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Functions/formatReadableQuantity.cpp b/src/Functions/formatReadableQuantity.cpp index 682fac88969..483e8a77a0b 100644 --- a/src/Functions/formatReadableQuantity.cpp +++ b/src/Functions/formatReadableQuantity.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Functions/formatReadableSize.cpp b/src/Functions/formatReadableSize.cpp index 22505907fa7..5c11603e9d7 100644 --- a/src/Functions/formatReadableSize.cpp +++ b/src/Functions/formatReadableSize.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Functions/geoToS2.cpp b/src/Functions/geoToS2.cpp index 8d065b01c34..f27cd26fd9d 100644 --- a/src/Functions/geoToS2.cpp +++ b/src/Functions/geoToS2.cpp @@ -101,19 +101,35 @@ public: const Float64 lon = data_col_lon[row]; const Float64 lat = data_col_lat[row]; - if (isNaN(lon) || isNaN(lat)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments must not be NaN"); + if (isNaN(lon)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument for longitude in function {}. It must not be NaN", getName()); + if (!isFinite(lon)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal argument for longitude in function {}. It must not be infinite", + getName()); - if (!(isFinite(lon) && isFinite(lat))) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Arguments must not be infinite"); + if (isNaN(lat)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument for latitude in function {}. It must not be NaN", getName()); + if (!isFinite(lat)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal argument for latitude in function {}. It must not be infinite", + getName()); - /// S2 acceptes point as (latitude, longitude) + /// S2 accepts point as (latitude, longitude) S2LatLng lat_lng = S2LatLng::FromDegrees(lat, lon); if (!lat_lng.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" - "and the longitude is between -180 and 180 degrees inclusive."); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Point ({}, {}) is invalid in function {}. For valid point the latitude is between -90 and 90 degrees inclusive" + "and the longitude is between -180 and 180 degrees inclusive.", + lon, + lat, + getName()); S2CellId id(lat_lng); diff --git a/src/Functions/h3ToString.cpp b/src/Functions/h3ToString.cpp index 897329ed9ec..f8a10d5252b 100644 --- a/src/Functions/h3ToString.cpp +++ b/src/Functions/h3ToString.cpp @@ -84,7 +84,7 @@ public: const UInt64 hindex = data[row]; if (!isValidCell(hindex)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid H3 index: {}", hindex); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid H3 index: {} in function {}", hindex, getName()); h3ToString(hindex, pos, H3_INDEX_STRING_LENGTH); diff --git a/src/Functions/idna.cpp b/src/Functions/idna.cpp new file mode 100644 index 00000000000..a73347400c6 --- /dev/null +++ b/src/Functions/idna.cpp @@ -0,0 +1,202 @@ +#include "config.h" + +#if USE_IDNA + +#include +#include +#include + +#ifdef __clang__ +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wnewline-eof" +#endif +# include +# include +# include +#ifdef __clang__ +# pragma clang diagnostic pop +#endif + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; +} + +/// Implementation of +/// - idnaEncode(), tryIdnaEncode() and idnaDecode(), see https://en.wikipedia.org/wiki/Internationalized_domain_name#ToASCII_and_ToUnicode +/// and [3] https://www.unicode.org/reports/tr46/#ToUnicode + +enum class ErrorHandling +{ + Throw, /// Throw exception + Empty /// Return empty string +}; + + +/// Translates a UTF-8 string (typically an Internationalized Domain Name for Applications, IDNA) to an ASCII-encoded equivalent. The +/// encoding is performed per domain component and based on Punycode with ASCII Compatible Encoding (ACE) prefix "xn--". +/// Example: "straße.münchen.de" --> "xn--strae-oqa.xn--mnchen-3ya.de" +/// Note: doesn't do percent decoding. Doesn't trim tabs, spaces or control characters. Expects non-empty inputs. +template +struct IdnaEncode +{ + static void vector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + const size_t rows = offsets.size(); + res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII + res_offsets.reserve(rows); + + size_t prev_offset = 0; + std::string ascii; + for (size_t row = 0; row < rows; ++row) + { + const char * value = reinterpret_cast(&data[prev_offset]); + const size_t value_length = offsets[row] - prev_offset - 1; + + std::string_view value_view(value, value_length); + if (!value_view.empty()) /// to_ascii() expects non-empty input + { + ascii = ada::idna::to_ascii(value_view); + const bool ok = !ascii.empty(); + if (!ok) + { + if constexpr (error_handling == ErrorHandling::Throw) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' cannot be converted to ASCII", value_view); + } + else + { + static_assert(error_handling == ErrorHandling::Empty); + ascii.clear(); + } + } + } + + res_data.insert(ascii.c_str(), ascii.c_str() + ascii.size() + 1); + res_offsets.push_back(res_data.size()); + + prev_offset = offsets[row]; + + ascii.clear(); + } + } + + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Arguments of type FixedString are not allowed"); + } +}; + +/// Translates an ASII-encoded IDNA string back to its UTF-8 representation. +struct IdnaDecode +{ + /// As per the specification, invalid inputs are returned as is, i.e. there is no special error handling. + static void vector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + const size_t rows = offsets.size(); + res_data.reserve(data.size()); /// just a guess, assuming the input is all-ASCII + res_offsets.reserve(rows); + + size_t prev_offset = 0; + std::string unicode; + for (size_t row = 0; row < rows; ++row) + { + const char * ascii = reinterpret_cast(&data[prev_offset]); + const size_t ascii_length = offsets[row] - prev_offset - 1; + std::string_view ascii_view(ascii, ascii_length); + + unicode = ada::idna::to_unicode(ascii_view); + + res_data.insert(unicode.c_str(), unicode.c_str() + unicode.size() + 1); + res_offsets.push_back(res_data.size()); + + prev_offset = offsets[row]; + + unicode.clear(); + } + } + + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Arguments of type FixedString are not allowed"); + } +}; + +struct NameIdnaEncode { static constexpr auto name = "idnaEncode"; }; +struct NameTryIdnaEncode { static constexpr auto name = "tryIdnaEncode"; }; +struct NameIdnaDecode { static constexpr auto name = "idnaDecode"; }; + +using FunctionIdnaEncode = FunctionStringToString, NameIdnaEncode>; +using FunctionTryIdnaEncode = FunctionStringToString, NameTryIdnaEncode>; +using FunctionIdnaDecode = FunctionStringToString; + +REGISTER_FUNCTION(Idna) +{ + factory.registerFunction(FunctionDocumentation{ + .description=R"( +Computes an ASCII representation of an Internationalized Domain Name. Throws an exception in case of error.)", + .syntax="idnaEncode(str)", + .arguments={{"str", "Input string"}}, + .returned_value="An ASCII-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT idnaEncode('straße.münchen.de') AS ascii;", + R"( +┌─ascii───────────────────────────┐ +│ xn--strae-oqa.xn--mnchen-3ya.de │ +└─────────────────────────────────┘ + )" + }} + }); + + factory.registerFunction(FunctionDocumentation{ + .description=R"( +Computes a ASCII representation of an Internationalized Domain Name. Returns an empty string in case of error)", + .syntax="punycodeEncode(str)", + .arguments={{"str", "Input string"}}, + .returned_value="An ASCII-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT idnaEncodeOrNull('München') AS ascii;", + R"( +┌─ascii───────────────────────────┐ +│ xn--strae-oqa.xn--mnchen-3ya.de │ +└─────────────────────────────────┘ + )" + }} + }); + + factory.registerFunction(FunctionDocumentation{ + .description=R"( +Computes the Unicode representation of ASCII-encoded Internationalized Domain Name.)", + .syntax="idnaDecode(str)", + .arguments={{"str", "Input string"}}, + .returned_value="An Unicode-encoded domain name [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT idnaDecode('xn--strae-oqa.xn--mnchen-3ya.de') AS unicode;", + R"( +┌─unicode───────────┐ +│ straße.münchen.de │ +└───────────────────┘ + )" + }} + }); +} + +} + +#endif + diff --git a/src/Functions/punycode.cpp b/src/Functions/punycode.cpp index c11409f0d1a..159189744bd 100644 --- a/src/Functions/punycode.cpp +++ b/src/Functions/punycode.cpp @@ -2,6 +2,7 @@ #if USE_IDNA +#include #include #include @@ -21,11 +22,22 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; + extern const int NOT_IMPLEMENTED; } -struct PunycodeEncodeImpl +/// Implementation of +/// - punycodeEncode(), punycodeDecode() and tryPunycodeDecode(), see https://en.wikipedia.org/wiki/Punycode + +enum class ErrorHandling { + Throw, /// Throw exception + Empty /// Return empty string +}; + + +struct PunycodeEncode +{ + /// Encoding-as-punycode can only fail if the input isn't valid UTF8. In that case, return undefined output, i.e. garbage-in, garbage-out. static void vector( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, @@ -46,11 +58,13 @@ struct PunycodeEncodeImpl const size_t value_utf32_length = ada::idna::utf32_length_from_utf8(value, value_length); value_utf32.resize(value_utf32_length); - ada::idna::utf8_to_utf32(value, value_length, value_utf32.data()); + const size_t codepoints = ada::idna::utf8_to_utf32(value, value_length, value_utf32.data()); + if (codepoints == 0) + value_utf32.clear(); /// input was empty or no valid UTF-8 const bool ok = ada::idna::utf32_to_punycode(value_utf32, value_puny); if (!ok) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Internal error during Punycode encoding"); + value_puny.clear(); res_data.insert(value_puny.c_str(), value_puny.c_str() + value_puny.size() + 1); res_offsets.push_back(res_data.size()); @@ -64,11 +78,13 @@ struct PunycodeEncodeImpl [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by punycodeEncode function"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Arguments of type FixedString are not allowed"); } }; -struct PunycodeDecodeImpl + +template +struct PunycodeDecode { static void vector( const ColumnString::Chars & data, @@ -91,7 +107,17 @@ struct PunycodeDecodeImpl const std::string_view value_punycode(value, value_length); const bool ok = ada::idna::punycode_to_utf32(value_punycode, value_utf32); if (!ok) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Internal error during Punycode decoding"); + { + if constexpr (error_handling == ErrorHandling::Throw) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' is not a valid Punycode-encoded string", value_punycode); + } + else + { + static_assert(error_handling == ErrorHandling::Empty); + value_utf32.clear(); + } + } const size_t utf8_length = ada::idna::utf8_length_from_utf32(value_utf32.data(), value_utf32.size()); value_utf8.resize(utf8_length); @@ -109,23 +135,21 @@ struct PunycodeDecodeImpl [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by punycodeDecode function"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Arguments of type FixedString are not allowed"); } }; -struct NamePunycodeEncode -{ - static constexpr auto name = "punycodeEncode"; -}; +struct NamePunycodeEncode { static constexpr auto name = "punycodeEncode"; }; +struct NamePunycodeDecode { static constexpr auto name = "punycodeDecode"; }; +struct NameTryPunycodeDecode { static constexpr auto name = "tryPunycodeDecode"; }; -struct NamePunycodeDecode -{ - static constexpr auto name = "punycodeDecode"; -}; +using FunctionPunycodeEncode = FunctionStringToString; +using FunctionPunycodeDecode = FunctionStringToString, NamePunycodeDecode>; +using FunctionTryPunycodeDecode = FunctionStringToString, NameTryPunycodeDecode>; REGISTER_FUNCTION(Punycode) { - factory.registerFunction>(FunctionDocumentation{ + factory.registerFunction(FunctionDocumentation{ .description=R"( Computes a Punycode representation of a string.)", .syntax="punycodeEncode(str)", @@ -142,9 +166,9 @@ Computes a Punycode representation of a string.)", }} }); - factory.registerFunction>(FunctionDocumentation{ + factory.registerFunction(FunctionDocumentation{ .description=R"( -Computes a Punycode representation of a string.)", +Computes a Punycode representation of a string. Throws an exception if the input is not valid Punycode.)", .syntax="punycodeDecode(str)", .arguments={{"str", "A Punycode-encoded string"}}, .returned_value="The plaintext representation [String](/docs/en/sql-reference/data-types/string.md).", @@ -154,6 +178,23 @@ Computes a Punycode representation of a string.)", R"( ┌─plain───┐ │ München │ +└─────────┘ + )" + }} + }); + + factory.registerFunction(FunctionDocumentation{ + .description=R"( +Computes a Punycode representation of a string. Returns an empty string if the input is not valid Punycode.)", + .syntax="punycodeDecode(str)", + .arguments={{"str", "A Punycode-encoded string"}}, + .returned_value="The plaintext representation [String](/docs/en/sql-reference/data-types/string.md).", + .examples={ + {"simple", + "SELECT tryPunycodeDecode('Mnchen-3ya') AS plain;", + R"( +┌─plain───┐ +│ München │ └─────────┘ )" }} diff --git a/src/Functions/randDistribution.cpp b/src/Functions/randDistribution.cpp index db101486de8..4e616ada697 100644 --- a/src/Functions/randDistribution.cpp +++ b/src/Functions/randDistribution.cpp @@ -1,7 +1,8 @@ #include #include #include -#include "Common/Exception.h" +#include +#include #include #include #include diff --git a/src/Functions/reverseDNSQuery.cpp b/src/Functions/reverseDNSQuery.cpp deleted file mode 100644 index b4d963a6a15..00000000000 --- a/src/Functions/reverseDNSQuery.cpp +++ /dev/null @@ -1,118 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; - extern const int FUNCTION_NOT_ALLOWED; -} - -class ReverseDNSQuery : public IFunction -{ -public: - static constexpr auto name = "reverseDNSQuery"; - static constexpr auto allow_function_config_name = "allow_reverse_dns_query_function"; - - static FunctionPtr create(ContextPtr) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & data_type, size_t input_rows_count) const override - { - if (!Context::getGlobalContextInstance()->getConfigRef().getBool(allow_function_config_name, false)) - { - throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Function {} is not allowed because {} is not set", name, allow_function_config_name); - } - - if (arguments.empty()) - { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least one argument", name); - } - - auto res_type = getReturnTypeImpl({data_type}); - - if (input_rows_count == 0u) - { - return res_type->createColumnConstWithDefaultValue(input_rows_count); - } - - if (!isString(arguments[0].type)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} requires the input column to be of type String", name); - } - - auto input_column = arguments[0].column; - - auto ip_address = Poco::Net::IPAddress(input_column->getDataAt(0).toString()); - - auto ptr_records = DNSResolver::instance().reverseResolve(ip_address); - - if (ptr_records.empty()) - return res_type->createColumnConstWithDefaultValue(input_rows_count); - - Array res; - - for (const auto & ptr_record : ptr_records) - { - res.push_back(ptr_record); - } - - return res_type->createColumnConst(input_rows_count, res); - } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override - { - return false; - } - - size_t getNumberOfArguments() const override - { - return 1u; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(std::make_shared()); - } - -}; - - -REGISTER_FUNCTION(ReverseDNSQuery) -{ - factory.registerFunction( - FunctionDocumentation{ - .description = R"(Performs a reverse DNS query to get the PTR records associated with the IP address)", - .syntax = "reverseDNSQuery(address)", - .arguments = {{"address", "An IPv4 or IPv6 address. [String](../../sql-reference/data-types/string.md)"}}, - .returned_value = "Associated domains (PTR records). [String](../../sql-reference/data-types/string.md).", - .examples = {{"", - "SELECT reverseDNSQuery('192.168.0.2');", -R"( -┌─reverseDNSQuery('192.168.0.2')────────────┐ -│ ['test2.example.com','test3.example.com'] │ -└───────────────────────────────────────────┘ -)"}} - } - ); -} - -} diff --git a/src/Functions/rowNumberInBlock.cpp b/src/Functions/rowNumberInBlock.cpp index e5fe2aeb178..25c9e9c56f3 100644 --- a/src/Functions/rowNumberInBlock.cpp +++ b/src/Functions/rowNumberInBlock.cpp @@ -56,8 +56,7 @@ public: auto column = ColumnUInt64::create(); auto & data = column->getData(); data.resize(input_rows_count); - for (size_t i = 0; i < input_rows_count; ++i) - data[i] = i; + iota(data.data(), input_rows_count, UInt64(0)); return column; } diff --git a/src/Functions/s2CapContains.cpp b/src/Functions/s2CapContains.cpp index 9dfbc05a6a0..72e9da69a7d 100644 --- a/src/Functions/s2CapContains.cpp +++ b/src/Functions/s2CapContains.cpp @@ -131,16 +131,16 @@ public: const auto point = S2CellId(data_point[row]); if (isNaN(degrees)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Radius of the cap must not be nan"); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Radius of the cap must not be nan in function {}", getName()); if (std::isinf(degrees)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Radius of the cap must not be infinite"); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Radius of the cap must not be infinite in function {}", getName()); if (!center.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Center is not valid"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Center (id {}) is not valid in function {}", data_center[row], getName()); if (!point.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Point is not valid"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Point (id {}) is not valid in function {}", data_point[row], getName()); S1Angle angle = S1Angle::Degrees(degrees); S2Cap cap(center.ToPoint(), angle); diff --git a/src/Functions/s2CellsIntersect.cpp b/src/Functions/s2CellsIntersect.cpp index 1fac5fd6e60..320f3c964a2 100644 --- a/src/Functions/s2CellsIntersect.cpp +++ b/src/Functions/s2CellsIntersect.cpp @@ -100,10 +100,12 @@ public: const UInt64 id_second = data_id_second[row]; auto first_cell = S2CellId(id_first); - auto second_cell = S2CellId(id_second); + if (!first_cell.is_valid()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "First cell (id {}) is not valid in function {}", id_first, getName()); - if (!first_cell.is_valid() || !second_cell.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cell is not valid"); + auto second_cell = S2CellId(id_second); + if (!second_cell.is_valid()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second cell (id {}) is not valid in function {}", id_second, getName()); dst_data.emplace_back(S2CellId(id_first).intersects(S2CellId(id_second))); } diff --git a/src/Functions/s2GetNeighbors.cpp b/src/Functions/s2GetNeighbors.cpp index b200f61315b..a6371b9ff68 100644 --- a/src/Functions/s2GetNeighbors.cpp +++ b/src/Functions/s2GetNeighbors.cpp @@ -94,7 +94,7 @@ public: S2CellId cell_id(id); if (!cell_id.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cell is not valid"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cell (id {}) is not valid in function {}", id, getName()); S2CellId neighbors[4]; cell_id.GetEdgeNeighbors(neighbors); diff --git a/src/Functions/FunctionSqid.cpp b/src/Functions/sqid.cpp similarity index 94% rename from src/Functions/FunctionSqid.cpp rename to src/Functions/sqid.cpp index 4517bba963e..363a3f8ac13 100644 --- a/src/Functions/FunctionSqid.cpp +++ b/src/Functions/sqid.cpp @@ -57,9 +57,10 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - size_t num_args = arguments.size(); auto col_res = ColumnString::create(); + col_res->reserve(input_rows_count); + const size_t num_args = arguments.size(); std::vector numbers(num_args); for (size_t i = 0; i < input_rows_count; ++i) { @@ -83,7 +84,7 @@ REGISTER_FUNCTION(Sqid) { factory.registerFunction(FunctionDocumentation{ .description=R"( -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.)", .syntax="sqid(number1, ...)", .arguments={{"number1, ...", "Arbitrarily many UInt8, UInt16, UInt32 or UInt64 arguments"}}, .returned_value="A hash id [String](/docs/en/sql-reference/data-types/string.md).", diff --git a/src/Functions/stringToH3.cpp b/src/Functions/stringToH3.cpp index d8728b346d0..94418efdfdf 100644 --- a/src/Functions/stringToH3.cpp +++ b/src/Functions/stringToH3.cpp @@ -88,7 +88,7 @@ private: if (res_data[row_num] == 0) { - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid H3 index: {}", h3index_str); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid H3 index: {} in function {}", h3index_str, name); } h3index_source.next(); diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index f1d2b60f1f4..3c9654740f4 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -91,19 +91,6 @@ namespace const auto type_arr_from_nested = type_arr_from->getNestedType(); - auto src = tryGetLeastSupertype(DataTypes{type_x, type_arr_from_nested}); - if (!src - /// Compatibility with previous versions, that allowed even UInt64 with Int64, - /// regardless of ambiguous conversions. - && !isNativeNumber(type_x) && !isNativeNumber(type_arr_from_nested)) - { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "First argument and elements of array " - "of the second argument of function {} must have compatible types", - getName()); - } - const DataTypeArray * type_arr_to = checkAndGetDataType(arguments[2].get()); if (!type_arr_to) @@ -766,15 +753,18 @@ namespace } } + WhichDataType which(from_type); + /// Note: Doesn't check the duplicates in the `from` array. /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 - if (WhichDataType which(from_type); isNativeNumber(which) || which.isDecimal32() || which.isDecimal64()) + if (isNativeNumber(which) || which.isDecimal32() || which.isDecimal64() || which.isEnum()) { cache.table_num_to_idx = std::make_unique(); auto & table = *cache.table_num_to_idx; for (size_t i = 0; i < size; ++i) { - if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i])) + if (which.isEnum() /// The correctness of strings are already checked by casting them to the Enum type. + || applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i])) { UInt64 key = 0; auto * dst = reinterpret_cast(&key); diff --git a/src/Functions/translate.cpp b/src/Functions/translate.cpp index 836cb4de2f3..ad5be7d9dfd 100644 --- a/src/Functions/translate.cpp +++ b/src/Functions/translate.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -31,7 +32,7 @@ struct TranslateImpl if (map_from.size() != map_to.size()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second and third arguments must be the same length"); - std::iota(map.begin(), map.end(), 0); + iota(map.data(), map.size(), UInt8(0)); for (size_t i = 0; i < map_from.size(); ++i) { @@ -129,7 +130,7 @@ struct TranslateUTF8Impl if (map_from_size != map_to_size) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second and third arguments must be the same length"); - std::iota(map_ascii.begin(), map_ascii.end(), 0); + iota(map_ascii.data(), map_ascii.size(), UInt32(0)); const UInt8 * map_from_ptr = reinterpret_cast(map_from.data()); const UInt8 * map_from_end = map_from_ptr + map_from.size(); diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index af6c87e8c88..785a5005f87 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include namespace DB diff --git a/src/IO/BitHelpers.h b/src/IO/BitHelpers.h index a9c7343f991..45c9b1ba572 100644 --- a/src/IO/BitHelpers.h +++ b/src/IO/BitHelpers.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include diff --git a/src/IO/MMapReadBufferFromFileWithCache.cpp b/src/IO/MMapReadBufferFromFileWithCache.cpp index d13cf5db2f7..d53f3bc325d 100644 --- a/src/IO/MMapReadBufferFromFileWithCache.cpp +++ b/src/IO/MMapReadBufferFromFileWithCache.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB diff --git a/src/IO/PeekableReadBuffer.cpp b/src/IO/PeekableReadBuffer.cpp index ce9c20e7a53..be650f2f3b4 100644 --- a/src/IO/PeekableReadBuffer.cpp +++ b/src/IO/PeekableReadBuffer.cpp @@ -20,33 +20,6 @@ PeekableReadBuffer::PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ checkStateCorrect(); } -void PeekableReadBuffer::reset() -{ - checkStateCorrect(); -} - -void PeekableReadBuffer::setSubBuffer(ReadBuffer & sub_buf_) -{ - sub_buf = &sub_buf_; - resetImpl(); -} - -void PeekableReadBuffer::resetImpl() -{ - peeked_size = 0; - checkpoint = std::nullopt; - checkpoint_in_own_memory = false; - use_stack_memory = true; - - if (!currentlyReadFromOwnMemory()) - sub_buf->position() = pos; - - Buffer & sub_working = sub_buf->buffer(); - BufferBase::set(sub_working.begin(), sub_working.size(), sub_buf->offset()); - - checkStateCorrect(); -} - bool PeekableReadBuffer::peekNext() { checkStateCorrect(); diff --git a/src/IO/PeekableReadBuffer.h b/src/IO/PeekableReadBuffer.h index 78cb319327d..2ee209ffd6c 100644 --- a/src/IO/PeekableReadBuffer.h +++ b/src/IO/PeekableReadBuffer.h @@ -74,12 +74,6 @@ public: /// This data will be lost after destruction of peekable buffer. bool hasUnreadData() const; - // for streaming reading (like in Kafka) we need to restore initial state of the buffer - // without recreating the buffer. - void reset(); - - void setSubBuffer(ReadBuffer & sub_buf_); - const ReadBuffer & getSubBuffer() const { return *sub_buf; } private: diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index ff5743a63af..05d35a57b12 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -89,7 +89,7 @@ void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) else out << " before: " << quote << String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())); - throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected {}", out.str()); + throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected {}", out.str()); } @@ -562,7 +562,7 @@ static ReturnType readAnyQuotedStringInto(Vector & s, ReadBuffer & buf) if (buf.eof() || *buf.position() != quote) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_QUOTED_STRING, + throw Exception(ErrorCodes::CANNOT_PARSE_QUOTED_STRING, "Cannot parse quoted string: expected opening quote '{}', got '{}'", std::string{quote}, buf.eof() ? "EOF" : std::string{*buf.position()}); else @@ -608,7 +608,7 @@ static ReturnType readAnyQuotedStringInto(Vector & s, ReadBuffer & buf) } if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_QUOTED_STRING, "Cannot parse quoted string: expected closing quote"); + throw Exception(ErrorCodes::CANNOT_PARSE_QUOTED_STRING, "Cannot parse quoted string: expected closing quote"); else return ReturnType(false); } @@ -958,7 +958,7 @@ ReturnType readJSONStringInto(Vector & s, ReadBuffer & buf) auto error = [](FormatStringHelper<> message [[maybe_unused]], int code [[maybe_unused]]) { if constexpr (throw_exception) - throw ParsingException(code, std::move(message)); + throw Exception(code, std::move(message)); return ReturnType(false); }; @@ -1009,7 +1009,7 @@ ReturnType readJSONObjectOrArrayPossiblyInvalid(Vector & s, ReadBuffer & buf) auto error = [](FormatStringHelper<> message [[maybe_unused]], int code [[maybe_unused]]) { if constexpr (throw_exception) - throw ParsingException(code, std::move(message)); + throw Exception(code, std::move(message)); return ReturnType(false); }; @@ -1185,7 +1185,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D else { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime"); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime"); else return false; } @@ -1212,7 +1212,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D s_pos[size] = 0; if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s); else return false; } @@ -1235,7 +1235,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D s_pos[size] = 0; if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", s); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", s); else return false; } @@ -1266,7 +1266,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D if (too_short && negative_multiplier != -1) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime"); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime"); else return false; } @@ -1382,8 +1382,12 @@ void skipJSONField(ReadBuffer & buf, StringRef name_of_field) } else { - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected symbol '{}' for key '{}'", - std::string(*buf.position(), 1), name_of_field.toString()); + throw Exception( + ErrorCodes::INCORRECT_DATA, + "Cannot read JSON field here: '{}'. Unexpected symbol '{}'{}", + String(buf.position(), std::min(buf.available(), size_t(10))), + std::string(1, *buf.position()), + name_of_field.empty() ? "" : " for key " + name_of_field.toString()); } } @@ -1753,7 +1757,7 @@ void readQuotedField(String & s, ReadBuffer & buf) void readJSONField(String & s, ReadBuffer & buf) { s.clear(); - auto parse_func = [](ReadBuffer & in) { skipJSONField(in, "json_field"); }; + auto parse_func = [](ReadBuffer & in) { skipJSONField(in, ""); }; readParsedValueInto(s, buf, parse_func); } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 17f3d3d4151..85584d63ee8 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -41,6 +41,7 @@ #include #include +#include #include static constexpr auto DEFAULT_MAX_STRING_SIZE = 1_GiB; @@ -295,7 +296,7 @@ inline void readBoolTextWord(bool & x, ReadBuffer & buf, bool support_upper_case [[fallthrough]]; } default: - throw ParsingException(ErrorCodes::CANNOT_PARSE_BOOL, "Unexpected Bool value"); + throw Exception(ErrorCodes::CANNOT_PARSE_BOOL, "Unexpected Bool value"); } } @@ -339,7 +340,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf) if (has_sign) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot parse number with multiple sign (+/-) characters"); else return ReturnType(false); @@ -356,7 +357,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf) if (has_sign) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot parse number with multiple sign (+/-) characters"); else return ReturnType(false); @@ -367,7 +368,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf) else { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Unsigned type must not contain '-' symbol"); + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Unsigned type must not contain '-' symbol"); else return ReturnType(false); } @@ -429,7 +430,7 @@ end: if (has_sign && !has_number) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot parse number with a sign character but without any numeric character"); else return ReturnType(false); @@ -836,7 +837,7 @@ inline ReturnType readUUIDTextImpl(UUID & uuid, ReadBuffer & buf) if constexpr (throw_exception) { - throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", s); + throw Exception(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", s); } else { @@ -854,7 +855,7 @@ inline ReturnType readUUIDTextImpl(UUID & uuid, ReadBuffer & buf) if constexpr (throw_exception) { - throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", s); + throw Exception(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", s); } else { @@ -880,7 +881,7 @@ inline ReturnType readIPv4TextImpl(IPv4 & ip, ReadBuffer & buf) return ReturnType(true); if constexpr (std::is_same_v) - throw ParsingException(ErrorCodes::CANNOT_PARSE_IPV4, "Cannot parse IPv4 {}", std::string_view(buf.position(), buf.available())); + throw Exception(ErrorCodes::CANNOT_PARSE_IPV4, "Cannot parse IPv4 {}", std::string_view(buf.position(), buf.available())); else return ReturnType(false); } @@ -902,7 +903,7 @@ inline ReturnType readIPv6TextImpl(IPv6 & ip, ReadBuffer & buf) return ReturnType(true); if constexpr (std::is_same_v) - throw ParsingException(ErrorCodes::CANNOT_PARSE_IPV6, "Cannot parse IPv6 {}", std::string_view(buf.position(), buf.available())); + throw Exception(ErrorCodes::CANNOT_PARSE_IPV6, "Cannot parse IPv6 {}", std::string_view(buf.position(), buf.available())); else return ReturnType(false); } @@ -943,7 +944,7 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons if (!buf.eof() && !isNumericASCII(*buf.position())) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse datetime"); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse datetime"); else return false; } @@ -1016,7 +1017,7 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re { readDateTimeTextImpl(whole, buf, date_lut); } - catch (const DB::ParsingException &) + catch (const DB::Exception &) { if (buf.eof() || *buf.position() != '.') throw; @@ -1124,7 +1125,7 @@ inline void readDateTimeText(LocalDateTime & datetime, ReadBuffer & buf) if (10 != size) { s[size] = 0; - throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s); } datetime.year((s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0')); @@ -1140,7 +1141,7 @@ inline void readDateTimeText(LocalDateTime & datetime, ReadBuffer & buf) if (8 != size) { s[size] = 0; - throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", s); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", s); } datetime.hour((s[0] - '0') * 10 + (s[1] - '0')); @@ -1173,7 +1174,7 @@ inline ReturnType readTimeTextImpl(time_t & time, ReadBuffer & buf) s[size] = 0; if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s); else return false; } @@ -1481,7 +1482,7 @@ void readQuoted(std::vector & x, ReadBuffer & buf) if (*buf.position() == ',') ++buf.position(); else - throw ParsingException(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Cannot read array from text"); + throw Exception(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Cannot read array from text"); } first = false; @@ -1504,7 +1505,7 @@ void readDoubleQuoted(std::vector & x, ReadBuffer & buf) if (*buf.position() == ',') ++buf.position(); else - throw ParsingException(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Cannot read array from text"); + throw Exception(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Cannot read array from text"); } first = false; diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 7658ea5941c..a65a82d9b40 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -125,12 +125,11 @@ std::unique_ptr Client::create( const std::shared_ptr & credentials_provider, const PocoHTTPClientConfiguration & client_configuration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, - bool use_virtual_addressing, - bool disable_checksum) + const ClientSettings & client_settings) { verifyClientConfiguration(client_configuration); return std::unique_ptr( - new Client(max_redirects_, std::move(sse_kms_config_), credentials_provider, client_configuration, sign_payloads, use_virtual_addressing, disable_checksum)); + new Client(max_redirects_, std::move(sse_kms_config_), credentials_provider, client_configuration, sign_payloads, client_settings)); } std::unique_ptr Client::clone() const @@ -160,14 +159,12 @@ Client::Client( const std::shared_ptr & credentials_provider_, const PocoHTTPClientConfiguration & client_configuration_, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads_, - bool use_virtual_addressing_, - bool disable_checksum_) - : Aws::S3::S3Client(credentials_provider_, client_configuration_, sign_payloads_, use_virtual_addressing_) + const ClientSettings & client_settings_) + : Aws::S3::S3Client(credentials_provider_, client_configuration_, sign_payloads_, client_settings_.use_virtual_addressing) , credentials_provider(credentials_provider_) , client_configuration(client_configuration_) , sign_payloads(sign_payloads_) - , use_virtual_addressing(use_virtual_addressing_) - , disable_checksum(disable_checksum_) + , client_settings(client_settings_) , max_redirects(max_redirects_) , sse_kms_config(std::move(sse_kms_config_)) , log(&Poco::Logger::get("S3Client")) @@ -207,13 +204,12 @@ Client::Client( Client::Client( const Client & other, const PocoHTTPClientConfiguration & client_configuration_) : Aws::S3::S3Client(other.credentials_provider, client_configuration_, other.sign_payloads, - other.use_virtual_addressing) + other.client_settings.use_virtual_addressing) , initial_endpoint(other.initial_endpoint) , credentials_provider(other.credentials_provider) , client_configuration(client_configuration_) , sign_payloads(other.sign_payloads) - , use_virtual_addressing(other.use_virtual_addressing) - , disable_checksum(other.disable_checksum) + , client_settings(other.client_settings) , explicit_region(other.explicit_region) , detect_region(other.detect_region) , provider_type(other.provider_type) @@ -417,7 +413,7 @@ Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(CompleteMu outcome = Aws::S3::Model::CompleteMultipartUploadOutcome(Aws::S3::Model::CompleteMultipartUploadResult()); } - if (outcome.IsSuccess() && provider_type == ProviderType::GCS) + if (outcome.IsSuccess() && provider_type == ProviderType::GCS && client_settings.gcs_issue_compose_request) { /// For GCS we will try to compose object at the end, otherwise we cannot do a native copy /// for the object (e.g. for backups) @@ -515,7 +511,7 @@ Client::doRequest(RequestType & request, RequestFn request_fn) const addAdditionalAMZHeadersToCanonicalHeadersList(request, client_configuration.extra_headers); const auto & bucket = request.GetBucket(); request.setApiMode(api_mode); - if (disable_checksum) + if (client_settings.disable_checksum) request.disableChecksum(); if (auto region = getRegionForBucket(bucket); !region.empty()) @@ -574,6 +570,9 @@ Client::doRequest(RequestType & request, RequestFn request_fn) const if (!new_uri) return result; + if (initial_endpoint.substr(11) == "amazonaws.com") // Check if user didn't mention any region + new_uri->addRegionToURI(request.getRegionOverride()); + const auto & current_uri_override = request.getURIOverride(); /// we already tried with this URI if (current_uri_override && current_uri_override->uri == new_uri->uri) @@ -849,8 +848,7 @@ ClientFactory & ClientFactory::instance() std::unique_ptr ClientFactory::create( // NOLINT const PocoHTTPClientConfiguration & cfg_, - bool is_virtual_hosted_style, - bool disable_checksum, + ClientSettings client_settings, const String & access_key_id, const String & secret_access_key, const String & server_side_encryption_customer_key_base64, @@ -889,14 +887,17 @@ std::unique_ptr ClientFactory::create( // NOLINT client_configuration.retryStrategy = std::make_shared(client_configuration.s3_retry_attempts); + /// Use virtual addressing if endpoint is not specified. + if (client_configuration.endpointOverride.empty()) + client_settings.use_virtual_addressing = true; + return Client::create( client_configuration.s3_max_redirects, std::move(sse_kms_config), credentials_provider, client_configuration, // Client configuration. Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, - is_virtual_hosted_style || client_configuration.endpointOverride.empty(), /// Use virtual addressing if endpoint is not specified. - disable_checksum + client_settings ); } diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index d0a21a2dafe..b137f0605dc 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -92,6 +92,23 @@ private: std::unordered_map> client_caches; }; +struct ClientSettings +{ + bool use_virtual_addressing; + /// Disable checksum to avoid extra read of the input stream + bool disable_checksum; + /// Should client send ComposeObject request after upload to GCS. + /// + /// Previously ComposeObject request was required to make Copy possible, + /// but not anymore (see [1]). + /// + /// [1]: https://cloud.google.com/storage/docs/release-notes#June_23_2023 + /// + /// Ability to enable it preserved since likely it is required for old + /// files. + bool gcs_issue_compose_request; +}; + /// Client that improves the client from the AWS SDK /// - inject region and URI into requests so they are rerouted to the correct destination if needed /// - automatically detect endpoint and regions for each bucket and cache them @@ -116,8 +133,7 @@ public: const std::shared_ptr & credentials_provider, const PocoHTTPClientConfiguration & client_configuration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, - bool use_virtual_addressing, - bool disable_checksum); + const ClientSettings & client_settings); std::unique_ptr clone() const; @@ -195,7 +211,6 @@ public: Model::DeleteObjectsOutcome DeleteObjects(DeleteObjectsRequest & request) const; using ComposeObjectOutcome = Aws::Utils::Outcome; - ComposeObjectOutcome ComposeObject(ComposeObjectRequest & request) const; using Aws::S3::S3Client::EnableRequestProcessing; using Aws::S3::S3Client::DisableRequestProcessing; @@ -212,8 +227,7 @@ private: const std::shared_ptr & credentials_provider_, const PocoHTTPClientConfiguration & client_configuration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, - bool use_virtual_addressing, - bool disable_checksum_); + const ClientSettings & client_settings_); Client( const Client & other, const PocoHTTPClientConfiguration & client_configuration); @@ -236,6 +250,8 @@ private: using Aws::S3::S3Client::DeleteObject; using Aws::S3::S3Client::DeleteObjects; + ComposeObjectOutcome ComposeObject(ComposeObjectRequest & request) const; + template std::invoke_result_t doRequest(RequestType & request, RequestFn request_fn) const; @@ -258,8 +274,7 @@ private: std::shared_ptr credentials_provider; PocoHTTPClientConfiguration client_configuration; Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads; - bool use_virtual_addressing; - bool disable_checksum; + ClientSettings client_settings; std::string explicit_region; mutable bool detect_region = true; @@ -289,8 +304,7 @@ public: std::unique_ptr create( const PocoHTTPClientConfiguration & cfg, - bool is_virtual_hosted_style, - bool disable_checksum, + ClientSettings client_settings, const String & access_key_id, const String & secret_access_key, const String & server_side_encryption_customer_key_base64, diff --git a/src/IO/S3/Requests.h b/src/IO/S3/Requests.h index eae45491fe6..bfb94a5a67e 100644 --- a/src/IO/S3/Requests.h +++ b/src/IO/S3/Requests.h @@ -58,6 +58,11 @@ public: return BaseRequest::GetChecksumAlgorithmName(); } + std::string getRegionOverride() const + { + return region_override; + } + void overrideRegion(std::string region) const { region_override = std::move(region); diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index e05e0882329..e990875dd2f 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -146,6 +146,12 @@ URI::URI(const std::string & uri_) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bucket or key name are invalid in S3 URI."); } +void URI::addRegionToURI(const std::string ®ion) +{ + if (auto pos = endpoint.find("amazonaws.com"); pos != std::string::npos) + endpoint = endpoint.substr(0, pos) + region + "." + endpoint.substr(pos); +} + void URI::validateBucket(const String & bucket, const Poco::URI & uri) { /// S3 specification requires at least 3 and at most 63 characters in bucket name. diff --git a/src/IO/S3/URI.h b/src/IO/S3/URI.h index f8f40cf9108..2873728bc78 100644 --- a/src/IO/S3/URI.h +++ b/src/IO/S3/URI.h @@ -32,6 +32,7 @@ struct URI URI() = default; explicit URI(const std::string & uri_); + void addRegionToURI(const std::string & region); static void validateBucket(const std::string & bucket, const Poco::URI & uri); }; diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 0b44698ac2c..33917314bca 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -94,7 +94,7 @@ void doWriteRequest(std::shared_ptr client, const DB::S3:: client, uri.bucket, uri.key, - DBMS_DEFAULT_BUFFER_SIZE, + DB::DBMS_DEFAULT_BUFFER_SIZE, request_settings, {} ); @@ -140,10 +140,15 @@ void testServerSideEncryption( bool use_environment_credentials = false; bool use_insecure_imds_request = false; + DB::S3::ClientSettings client_settings{ + .use_virtual_addressing = uri.is_virtual_hosted_style, + .disable_checksum = disable_checksum, + .gcs_issue_compose_request = false, + }; + std::shared_ptr client = DB::S3::ClientFactory::instance().create( client_configuration, - uri.is_virtual_hosted_style, - disable_checksum, + client_settings, access_key_id, secret_access_key, server_side_encryption_customer_key_base64, diff --git a/src/IO/examples/lzma_buffers.cpp b/src/IO/examples/lzma_buffers.cpp index 126a192737b..f9e4fc0c5db 100644 --- a/src/IO/examples/lzma_buffers.cpp +++ b/src/IO/examples/lzma_buffers.cpp @@ -19,7 +19,7 @@ try { auto buf - = std::make_unique("test_lzma_buffers.xz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); + = std::make_unique("test_lzma_buffers.xz", DB::DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); DB::LZMADeflatingWriteBuffer lzma_buf(std::move(buf), /*compression level*/ 3); stopwatch.restart(); diff --git a/src/IO/examples/zlib_buffers.cpp b/src/IO/examples/zlib_buffers.cpp index a36b7a7a41d..1497e2c3f8e 100644 --- a/src/IO/examples/zlib_buffers.cpp +++ b/src/IO/examples/zlib_buffers.cpp @@ -21,7 +21,7 @@ try Stopwatch stopwatch; { - auto buf = std::make_unique("test_zlib_buffers.gz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); + auto buf = std::make_unique("test_zlib_buffers.gz", DB::DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); DB::ZlibDeflatingWriteBuffer deflating_buf(std::move(buf), DB::CompressionMethod::Gzip, /* compression_level = */ 3); stopwatch.restart(); diff --git a/src/IO/examples/zstd_buffers.cpp b/src/IO/examples/zstd_buffers.cpp index 26c8899605a..dc9913b81a6 100644 --- a/src/IO/examples/zstd_buffers.cpp +++ b/src/IO/examples/zstd_buffers.cpp @@ -21,7 +21,7 @@ try { auto buf - = std::make_unique("test_zstd_buffers.zst", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); + = std::make_unique("test_zstd_buffers.zst", DB::DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); DB::ZstdDeflatingWriteBuffer zstd_buf(std::move(buf), /*compression level*/ 3); stopwatch.restart(); diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 83fde8e8830..9734ba1c84f 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -95,7 +95,7 @@ ReturnType parseDateTimeBestEffortImpl( FmtArgs && ...fmt_args [[maybe_unused]]) { if constexpr (std::is_same_v) - throw ParsingException(error_code, std::move(fmt_string), std::forward(fmt_args)...); + throw Exception(error_code, std::move(fmt_string), std::forward(fmt_args)...); else return false; }; diff --git a/src/IO/readDecimalText.h b/src/IO/readDecimalText.h index 9fd9c439b87..3417310a990 100644 --- a/src/IO/readDecimalText.h +++ b/src/IO/readDecimalText.h @@ -121,7 +121,7 @@ inline bool readDigits(ReadBuffer & buf, T & x, uint32_t & digits, int32_t & exp if (!tryReadIntText(addition_exp, buf)) { if constexpr (_throw_on_error) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot parse exponent while reading decimal"); + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot parse exponent while reading decimal"); else return false; } @@ -134,7 +134,7 @@ inline bool readDigits(ReadBuffer & buf, T & x, uint32_t & digits, int32_t & exp if (digits_only) { if constexpr (_throw_on_error) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Unexpected symbol while reading decimal"); + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Unexpected symbol while reading decimal"); return false; } stop = true; diff --git a/src/IO/readFloatText.h b/src/IO/readFloatText.h index b0682576183..23e904f305a 100644 --- a/src/IO/readFloatText.h +++ b/src/IO/readFloatText.h @@ -160,7 +160,7 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf) if (unlikely(res.ec != std::errc())) { if constexpr (throw_exception) - throw ParsingException( + throw Exception( ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value here: {}", String(initial_position, buf.buffer().end() - initial_position)); @@ -253,7 +253,7 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf) if (unlikely(res.ec != std::errc() || res.ptr - tmp_buf != num_copied_chars)) { if constexpr (throw_exception) - throw ParsingException( + throw Exception( ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value here: {}", String(tmp_buf, num_copied_chars)); else return ReturnType(false); @@ -342,7 +342,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) if (in.eof()) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value"); + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value"); else return false; } @@ -400,7 +400,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) if (in.eof()) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after exponent"); + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after exponent"); else return false; } @@ -438,7 +438,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) if (in.eof()) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: no digits read"); + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: no digits read"); else return false; } @@ -449,14 +449,14 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) if (in.eof()) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after plus sign"); + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after plus sign"); else return false; } else if (negative) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: plus after minus sign"); + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: plus after minus sign"); else return false; } diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index 5880b40c408..7210dc6fbbf 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -210,10 +210,13 @@ struct Client : DB::S3::Client std::make_shared("", ""), GetClientConfiguration(), Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, - /* use_virtual_addressing = */ true, - /* disable_checksum_= */ false) + DB::S3::ClientSettings{ + .use_virtual_addressing = true, + .disable_checksum= false, + .gcs_issue_compose_request = false, + }) , store(mock_s3_store) - { } + {} static std::shared_ptr CreateClient(String bucket = "mock-s3-bucket") { diff --git a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp index b62f3a8b0bd..56608644425 100644 --- a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -160,4 +161,13 @@ void InterpreterCreateQuotaQuery::updateQuotaFromQuery(Quota & quota, const ASTC updateQuotaFromQueryImpl(quota, query, {}, {}); } +void registerInterpreterCreateQuotaQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateQuotaQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp index fef1f285c8b..4936bd15262 100644 --- a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -122,4 +123,14 @@ void InterpreterCreateRoleQuery::updateRoleFromQuery(Role & role, const ASTCreat { updateRoleFromQueryImpl(role, query, {}, {}); } + +void registerInterpreterCreateRoleQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateRoleQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index e4593222f6d..b48c3880c59 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -148,4 +149,13 @@ AccessRightsElements InterpreterCreateRowPolicyQuery::getRequiredAccess() const return res; } +void registerInterpreterCreateRowPolicyQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateRowPolicyQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp index 3a96c0a96ff..029deff9b22 100644 --- a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -138,4 +139,14 @@ void InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(Setti { updateSettingsProfileFromQueryImpl(SettingsProfile, query, {}, {}, {}); } + +void registerInterpreterCreateSettingsProfileQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateSettingsProfileQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 00e21f27d2e..32c51b745c7 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -261,4 +262,13 @@ void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreat updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, {}, allow_no_password, allow_plaintext_password, true); } +void registerInterpreterCreateUserQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateUserQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp index 371ed248306..612000c3480 100644 --- a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -95,4 +96,13 @@ AccessRightsElements InterpreterDropAccessEntityQuery::getRequiredAccess() const throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{}: type is not supported by DROP query", toString(query.type)); } +void registerInterpreterDropAccessEntityQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropAccessEntityQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index 259c6b39524..0f2d65abb5e 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -480,4 +481,13 @@ void InterpreterGrantQuery::updateRoleFromQuery(Role & role, const ASTGrantQuery updateFromQuery(role, query); } +void registerInterpreterGrantQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterGrantQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp index 49e90783a59..a05fc86f866 100644 --- a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -90,4 +91,13 @@ AccessRightsElements InterpreterMoveAccessEntityQuery::getRequiredAccess() const throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{}: type is not supported by DROP query", toString(query.type)); } +void registerInterpreterMoveAccessEntityQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterMoveAccessEntityQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterSetRoleQuery.cpp b/src/Interpreters/Access/InterpreterSetRoleQuery.cpp index 69740f90d4c..24467923542 100644 --- a/src/Interpreters/Access/InterpreterSetRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterSetRoleQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -90,4 +91,13 @@ void InterpreterSetRoleQuery::updateUserSetDefaultRoles(User & user, const Roles user.default_roles = roles_from_query; } +void registerInterpreterSetRoleQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterSetRoleQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp index bffb47ac714..76979ed86c8 100644 --- a/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -125,4 +126,13 @@ String InterpreterShowAccessEntitiesQuery::getRewrittenQuery() const (order.empty() ? "" : " ORDER BY " + order); } +void registerInterpreterShowAccessEntitiesQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowAccessEntitiesQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterShowAccessQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp index e9862e99393..23ce66fe663 100644 --- a/src/Interpreters/Access/InterpreterShowAccessQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -80,4 +81,13 @@ ASTs InterpreterShowAccessQuery::getCreateAndGrantQueries() const return result; } +void registerInterpreterShowAccessQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowAccessQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index ec2e60b2ef7..a55588baeaa 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -420,4 +421,14 @@ AccessRightsElements InterpreterShowCreateAccessEntityQuery::getRequiredAccess() } throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{}: type is not supported by SHOW CREATE query", toString(show_query.type)); } + +void registerInterpreterShowCreateAccessEntityQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowCreateAccessEntityQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp b/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp index 56fbb34a577..ba96dafbf0d 100644 --- a/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -198,4 +199,13 @@ ASTs InterpreterShowGrantsQuery::getAttachGrantQueries(const IAccessEntity & use return getGrantQueriesImpl(user_or_role, nullptr, true); } +void registerInterpreterShowGrantsQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowGrantsQuery", create_fn); +} + } diff --git a/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp b/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp index 1a0b441a06d..42b7fc51022 100644 --- a/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -15,4 +16,14 @@ BlockIO InterpreterShowPrivilegesQuery::execute() return executeQuery("SELECT * FROM system.privileges", context, QueryFlags{ .internal = true }).second; } +void registerInterpreterShowPrivilegesQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowPrivilegesQuery", create_fn); +} + + } diff --git a/src/Interpreters/ActionLocksManager.cpp b/src/Interpreters/ActionLocksManager.cpp index fb5ef4b98ae..65f13ebd66c 100644 --- a/src/Interpreters/ActionLocksManager.cpp +++ b/src/Interpreters/ActionLocksManager.cpp @@ -18,6 +18,7 @@ namespace ActionLocks extern const StorageActionBlockType PartsMove = 7; extern const StorageActionBlockType PullReplicationLog = 8; extern const StorageActionBlockType Cleanup = 9; + extern const StorageActionBlockType ViewRefresh = 10; } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 94b6b1ac41d..f18ae5d5c75 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -115,6 +115,7 @@ public: explicit ActionsDAG(const ColumnsWithTypeAndName & inputs_); const Nodes & getNodes() const { return nodes; } + static Nodes detachNodes(ActionsDAG && dag) { return std::move(dag.nodes); } const NodeRawConstPtrs & getOutputs() const { return outputs; } /** Output nodes can contain any column returned from DAG. * You may manually change it if needed. diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 827914eaefe..1789cc6c4b1 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1419,7 +1419,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return set; } - FutureSetPtr external_table_set; + FutureSetFromSubqueryPtr external_table_set; /// A special case is if the name of the table is specified on the right side of the IN statement, /// and the table has the type Set (a previously prepared set). diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index 08d159b42ca..b977a73d461 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -87,6 +88,12 @@ public: visit(child); } + void visit(ASTRefreshStrategy & refresh) const + { + ASTPtr unused; + visit(refresh, unused); + } + private: ContextPtr context; @@ -148,8 +155,6 @@ private: { if (table_expression.database_and_table_name) tryVisit(table_expression.database_and_table_name); - else if (table_expression.subquery) - tryVisit(table_expression.subquery); } void visit(const ASTTableIdentifier & identifier, ASTPtr & ast) const @@ -167,11 +172,6 @@ private: ast = qualified_identifier; } - void visit(ASTSubquery & subquery, ASTPtr &) const - { - tryVisit(subquery.children[0]); - } - void visit(ASTFunction & function, ASTPtr &) const { bool is_operator_in = functionIsInOrGlobalInOperator(function.name); @@ -236,6 +236,13 @@ private: } } + void visit(ASTRefreshStrategy & refresh, ASTPtr &) const + { + if (refresh.dependencies) + for (auto & table : refresh.dependencies->children) + tryVisit(table); + } + void visitChildren(IAST & ast) const { for (auto & child : ast.children) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index b43edfb8d3e..cdc4292a79c 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -976,12 +976,12 @@ void Aggregator::executeOnBlockSmall( initDataVariantsWithSizeHint(result, method_chosen, params); else result.init(method_chosen); + result.keys_size = params.keys_size; result.key_sizes = key_sizes; } executeImpl(result, row_begin, row_end, key_columns, aggregate_instructions); - CurrentMemoryTracker::check(); } @@ -1014,7 +1014,9 @@ void Aggregator::mergeOnBlockSmall( #define M(NAME, IS_TWO_LEVEL) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ mergeStreamsImpl(result.aggregates_pool, *result.NAME, result.NAME->data, \ - result.without_key, /* no_more_keys= */ false, \ + result.without_key, \ + result.consecutive_keys_cache_stats, \ + /* no_more_keys= */ false, \ row_begin, row_end, \ aggregate_columns_data, key_columns, result.aggregates_pool); @@ -1038,17 +1040,14 @@ void Aggregator::executeImpl( { #define M(NAME, IS_TWO_LEVEL) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ - executeImpl(*result.NAME, result.aggregates_pool, row_begin, row_end, key_columns, aggregate_instructions, no_more_keys, all_keys_are_const, overflow_row); + executeImpl(*result.NAME, result.aggregates_pool, row_begin, row_end, key_columns, aggregate_instructions, \ + result.consecutive_keys_cache_stats, no_more_keys, all_keys_are_const, overflow_row); if (false) {} // NOLINT APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M } -/** It's interesting - if you remove `noinline`, then gcc for some reason will inline this function, and the performance decreases (~ 10%). - * (Probably because after the inline of this function, more internal functions no longer be inlined.) - * Inline does not make sense, since the inner loop is entirely inside this function. - */ template void NO_INLINE Aggregator::executeImpl( Method & method, @@ -1057,12 +1056,44 @@ void NO_INLINE Aggregator::executeImpl( size_t row_end, ColumnRawPtrs & key_columns, AggregateFunctionInstruction * aggregate_instructions, + LastElementCacheStats & consecutive_keys_cache_stats, bool no_more_keys, bool all_keys_are_const, AggregateDataPtr overflow_row) const { - typename Method::State state(key_columns, key_sizes, aggregation_state_cache); + UInt64 total_rows = consecutive_keys_cache_stats.hits + consecutive_keys_cache_stats.misses; + double cache_hit_rate = total_rows ? static_cast(consecutive_keys_cache_stats.hits) / total_rows : 1.0; + bool use_cache = cache_hit_rate >= params.min_hit_rate_to_use_consecutive_keys_optimization; + if (use_cache) + { + typename Method::State state(key_columns, key_sizes, aggregation_state_cache); + executeImpl(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, no_more_keys, all_keys_are_const, overflow_row); + consecutive_keys_cache_stats.update(row_end - row_begin, state.getCacheMissesSinceLastReset()); + } + else + { + typename Method::StateNoCache state(key_columns, key_sizes, aggregation_state_cache); + executeImpl(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, no_more_keys, all_keys_are_const, overflow_row); + } +} + +/** It's interesting - if you remove `noinline`, then gcc for some reason will inline this function, and the performance decreases (~ 10%). + * (Probably because after the inline of this function, more internal functions no longer be inlined.) + * Inline does not make sense, since the inner loop is entirely inside this function. + */ +template +void NO_INLINE Aggregator::executeImpl( + Method & method, + State & state, + Arena * aggregates_pool, + size_t row_begin, + size_t row_end, + AggregateFunctionInstruction * aggregate_instructions, + bool no_more_keys, + bool all_keys_are_const, + AggregateDataPtr overflow_row) const +{ if (!no_more_keys) { /// Prefetching doesn't make sense for small hash tables, because they fit in caches entirely. @@ -1096,10 +1127,10 @@ void NO_INLINE Aggregator::executeImpl( } } -template +template void NO_INLINE Aggregator::executeImplBatch( Method & method, - typename Method::State & state, + State & state, Arena * aggregates_pool, size_t row_begin, size_t row_end, @@ -1119,8 +1150,6 @@ void NO_INLINE Aggregator::executeImplBatch( if constexpr (no_more_keys) return; - /// For all rows. - /// This pointer is unused, but the logic will compare it for nullptr to check if the cell is set. AggregateDataPtr place = reinterpret_cast(0x1); if (all_keys_are_const) @@ -1129,6 +1158,7 @@ void NO_INLINE Aggregator::executeImplBatch( } else { + /// For all rows. for (size_t i = row_begin; i < row_end; ++i) { if constexpr (prefetch && HasPrefetchMemberFunc) @@ -1191,21 +1221,23 @@ void NO_INLINE Aggregator::executeImplBatch( /// - and plus this will require other changes in the interface. std::unique_ptr places(new AggregateDataPtr[all_keys_are_const ? 1 : row_end]); - /// For all rows. - size_t start, end; + size_t key_start, key_end; /// If all keys are const, key columns contain only 1 row. if (all_keys_are_const) { - start = 0; - end = 1; + key_start = 0; + key_end = 1; } else { - start = row_begin; - end = row_end; + key_start = row_begin; + key_end = row_end; } - for (size_t i = start; i < end; ++i) + state.resetCache(); + + /// For all rows. + for (size_t i = key_start; i < key_end; ++i) { AggregateDataPtr aggregate_data = nullptr; @@ -1213,7 +1245,7 @@ void NO_INLINE Aggregator::executeImplBatch( { if constexpr (prefetch && HasPrefetchMemberFunc) { - if (i == row_begin + prefetching.iterationsToMeasure()) + if (i == key_start + prefetching.iterationsToMeasure()) prefetch_look_ahead = prefetching.calcPrefetchLookAhead(); if (i + prefetch_look_ahead < row_end) @@ -1305,10 +1337,10 @@ void NO_INLINE Aggregator::executeImplBatch( columns_data.emplace_back(getColumnData(inst->batch_arguments[argument_index])); } - if (all_keys_are_const) + if (all_keys_are_const || (!no_more_keys && state.hasOnlyOneValueSinceLastReset())) { auto add_into_aggregate_states_function_single_place = compiled_aggregate_functions_holder->compiled_aggregate_functions.add_into_aggregate_states_function_single_place; - add_into_aggregate_states_function_single_place(row_begin, row_end, columns_data.data(), places[0]); + add_into_aggregate_states_function_single_place(row_begin, row_end, columns_data.data(), places[key_start]); } else { @@ -1329,24 +1361,10 @@ void NO_INLINE Aggregator::executeImplBatch( AggregateFunctionInstruction * inst = aggregate_instructions + i; - if (all_keys_are_const) - { - if (inst->offsets) - inst->batch_that->addBatchSinglePlace(inst->offsets[static_cast(row_begin) - 1], inst->offsets[row_end - 1], places[0] + inst->state_offset, inst->batch_arguments, aggregates_pool); - else if (inst->has_sparse_arguments) - inst->batch_that->addBatchSparseSinglePlace(row_begin, row_end, places[0] + inst->state_offset, inst->batch_arguments, aggregates_pool); - else - inst->batch_that->addBatchSinglePlace(row_begin, row_end, places[0] + inst->state_offset, inst->batch_arguments, aggregates_pool); - } + if (all_keys_are_const || (!no_more_keys && state.hasOnlyOneValueSinceLastReset())) + addBatchSinglePlace(row_begin, row_end, inst, places[key_start] + inst->state_offset, aggregates_pool); else - { - if (inst->offsets) - inst->batch_that->addBatchArray(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, inst->offsets, aggregates_pool); - else if (inst->has_sparse_arguments) - inst->batch_that->addBatchSparse(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool); - else - inst->batch_that->addBatch(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool); - } + addBatch(row_begin, row_end, inst, places.get(), aggregates_pool); } } @@ -1410,28 +1428,63 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl( continue; #endif - if (inst->offsets) - inst->batch_that->addBatchSinglePlace( - inst->offsets[static_cast(row_begin) - 1], - inst->offsets[row_end - 1], - res + inst->state_offset, - inst->batch_arguments, - arena); - else if (inst->has_sparse_arguments) - inst->batch_that->addBatchSparseSinglePlace( - row_begin, row_end, - res + inst->state_offset, - inst->batch_arguments, - arena); - else - inst->batch_that->addBatchSinglePlace( - row_begin, row_end, - res + inst->state_offset, - inst->batch_arguments, - arena); + addBatchSinglePlace(row_begin, row_end, inst, res + inst->state_offset, arena); } } +void Aggregator::addBatch( + size_t row_begin, size_t row_end, + AggregateFunctionInstruction * inst, + AggregateDataPtr * places, + Arena * arena) +{ + if (inst->offsets) + inst->batch_that->addBatchArray( + row_begin, row_end, places, + inst->state_offset, + inst->batch_arguments, + inst->offsets, + arena); + else if (inst->has_sparse_arguments) + inst->batch_that->addBatchSparse( + row_begin, row_end, places, + inst->state_offset, + inst->batch_arguments, + arena); + else + inst->batch_that->addBatch( + row_begin, row_end, places, + inst->state_offset, + inst->batch_arguments, + arena); +} + + +void Aggregator::addBatchSinglePlace( + size_t row_begin, size_t row_end, + AggregateFunctionInstruction * inst, + AggregateDataPtr place, + Arena * arena) +{ + if (inst->offsets) + inst->batch_that->addBatchSinglePlace( + inst->offsets[static_cast(row_begin) - 1], + inst->offsets[row_end - 1], + place, + inst->batch_arguments, + arena); + else if (inst->has_sparse_arguments) + inst->batch_that->addBatchSparseSinglePlace( + row_begin, row_end, place, + inst->batch_arguments, + arena); + else + inst->batch_that->addBatchSinglePlace( + row_begin, row_end, place, + inst->batch_arguments, + arena); +} + void NO_INLINE Aggregator::executeOnIntervalWithoutKey( AggregatedDataVariants & data_variants, size_t row_begin, size_t row_end, AggregateFunctionInstruction * aggregate_instructions) const @@ -2867,20 +2920,17 @@ ManyAggregatedDataVariants Aggregator::prepareVariantsToMerge(ManyAggregatedData return non_empty_data; } -template +template void NO_INLINE Aggregator::mergeStreamsImplCase( Arena * aggregates_pool, - Method & method [[maybe_unused]], + State & state, Table & data, AggregateDataPtr overflow_row, size_t row_begin, size_t row_end, const AggregateColumnsConstData & aggregate_columns_data, - const ColumnRawPtrs & key_columns, Arena * arena_for_keys) const { - typename Method::State state(key_columns, key_sizes, aggregation_state_cache); - std::unique_ptr places(new AggregateDataPtr[row_end]); if (!arena_for_keys) @@ -2890,7 +2940,7 @@ void NO_INLINE Aggregator::mergeStreamsImplCase( { AggregateDataPtr aggregate_data = nullptr; - if (!no_more_keys) + if constexpr (!no_more_keys) { auto emplace_result = state.emplaceKey(data, i, *arena_for_keys); // NOLINT if (emplace_result.isInserted()) @@ -2936,6 +2986,7 @@ void NO_INLINE Aggregator::mergeStreamsImpl( Method & method, Table & data, AggregateDataPtr overflow_row, + LastElementCacheStats & consecutive_keys_cache_stats, bool no_more_keys, Arena * arena_for_keys) const { @@ -2943,15 +2994,17 @@ void NO_INLINE Aggregator::mergeStreamsImpl( const ColumnRawPtrs & key_columns = params.makeRawKeyColumns(block); mergeStreamsImpl( - aggregates_pool, method, data, overflow_row, no_more_keys, 0, block.rows(), aggregate_columns_data, key_columns, arena_for_keys); + aggregates_pool, method, data, overflow_row, consecutive_keys_cache_stats, + no_more_keys, 0, block.rows(), aggregate_columns_data, key_columns, arena_for_keys); } template void NO_INLINE Aggregator::mergeStreamsImpl( Arena * aggregates_pool, - Method & method, + Method & method [[maybe_unused]], Table & data, AggregateDataPtr overflow_row, + LastElementCacheStats & consecutive_keys_cache_stats, bool no_more_keys, size_t row_begin, size_t row_end, @@ -2959,12 +3012,30 @@ void NO_INLINE Aggregator::mergeStreamsImpl( const ColumnRawPtrs & key_columns, Arena * arena_for_keys) const { - if (!no_more_keys) - mergeStreamsImplCase( - aggregates_pool, method, data, overflow_row, row_begin, row_end, aggregate_columns_data, key_columns, arena_for_keys); + UInt64 total_rows = consecutive_keys_cache_stats.hits + consecutive_keys_cache_stats.misses; + double cache_hit_rate = total_rows ? static_cast(consecutive_keys_cache_stats.hits) / total_rows : 1.0; + bool use_cache = cache_hit_rate >= params.min_hit_rate_to_use_consecutive_keys_optimization; + + if (use_cache) + { + typename Method::State state(key_columns, key_sizes, aggregation_state_cache); + + if (!no_more_keys) + mergeStreamsImplCase(aggregates_pool, state, data, overflow_row, row_begin, row_end, aggregate_columns_data, arena_for_keys); + else + mergeStreamsImplCase(aggregates_pool, state, data, overflow_row, row_begin, row_end, aggregate_columns_data, arena_for_keys); + + consecutive_keys_cache_stats.update(row_end - row_begin, state.getCacheMissesSinceLastReset()); + } else - mergeStreamsImplCase( - aggregates_pool, method, data, overflow_row, row_begin, row_end, aggregate_columns_data, key_columns, arena_for_keys); + { + typename Method::StateNoCache state(key_columns, key_sizes, aggregation_state_cache); + + if (!no_more_keys) + mergeStreamsImplCase(aggregates_pool, state, data, overflow_row, row_begin, row_end, aggregate_columns_data, arena_for_keys); + else + mergeStreamsImplCase(aggregates_pool, state, data, overflow_row, row_begin, row_end, aggregate_columns_data, arena_for_keys); + } } @@ -3024,7 +3095,7 @@ bool Aggregator::mergeOnBlock(Block block, AggregatedDataVariants & result, bool mergeBlockWithoutKeyStreamsImpl(std::move(block), result); #define M(NAME, IS_TWO_LEVEL) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(std::move(block), result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys); + mergeStreamsImpl(std::move(block), result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, result.consecutive_keys_cache_stats, no_more_keys); APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M @@ -3127,9 +3198,11 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari for (Block & block : bucket_to_blocks[bucket]) { + /// Copy to avoid race. + auto consecutive_keys_cache_stats_copy = result.consecutive_keys_cache_stats; #define M(NAME) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(std::move(block), aggregates_pool, *result.NAME, result.NAME->data.impls[bucket], nullptr, false); + mergeStreamsImpl(std::move(block), aggregates_pool, *result.NAME, result.NAME->data.impls[bucket], nullptr, consecutive_keys_cache_stats_copy, false); if (false) {} // NOLINT APPLY_FOR_VARIANTS_TWO_LEVEL(M) @@ -3184,7 +3257,7 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari #define M(NAME, IS_TWO_LEVEL) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(std::move(block), result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys); + mergeStreamsImpl(std::move(block), result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, result.consecutive_keys_cache_stats, no_more_keys); APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M @@ -3262,7 +3335,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final) #define M(NAME, IS_TWO_LEVEL) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(std::move(block), result.aggregates_pool, *result.NAME, result.NAME->data, nullptr, false, arena_for_keys.get()); + mergeStreamsImpl(std::move(block), result.aggregates_pool, *result.NAME, result.NAME->data, nullptr, result.consecutive_keys_cache_stats, false, arena_for_keys.get()); APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index ab53f76d2ce..6fc3ac2f6d6 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -205,8 +205,17 @@ struct AggregationMethodOneNumber } /// To use one `Method` in different threads, use different `State`. - using State = ColumnsHashing::HashMethodOneNumber; + template + using StateImpl = ColumnsHashing::HashMethodOneNumber< + typename Data::value_type, + Mapped, + FieldType, + use_cache && consecutive_keys_optimization, + /*need_offset=*/ false, + nullable>; + + using State = StateImpl; + using StateNoCache = StateImpl; /// Use optimization for low cardinality. static const bool low_cardinality_optimization = false; @@ -259,7 +268,11 @@ struct AggregationMethodString explicit AggregationMethodString(size_t size_hint) : data(size_hint) { } - using State = ColumnsHashing::HashMethodString; + template + using StateImpl = ColumnsHashing::HashMethodString; + + using State = StateImpl; + using StateNoCache = StateImpl; static const bool low_cardinality_optimization = false; static const bool one_key_nullable_optimization = false; @@ -292,7 +305,11 @@ struct AggregationMethodStringNoCache { } - using State = ColumnsHashing::HashMethodString; + template + using StateImpl = ColumnsHashing::HashMethodString; + + using State = StateImpl; + using StateNoCache = StateImpl; static const bool low_cardinality_optimization = false; static const bool one_key_nullable_optimization = nullable; @@ -334,7 +351,11 @@ struct AggregationMethodFixedString { } - using State = ColumnsHashing::HashMethodFixedString; + template + using StateImpl = ColumnsHashing::HashMethodFixedString; + + using State = StateImpl; + using StateNoCache = StateImpl; static const bool low_cardinality_optimization = false; static const bool one_key_nullable_optimization = false; @@ -366,7 +387,11 @@ struct AggregationMethodFixedStringNoCache { } - using State = ColumnsHashing::HashMethodFixedString; + template + using StateImpl = ColumnsHashing::HashMethodFixedString; + + using State = StateImpl; + using StateNoCache = StateImpl; static const bool low_cardinality_optimization = false; static const bool one_key_nullable_optimization = nullable; @@ -392,20 +417,24 @@ template struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod { using Base = SingleColumnMethod; - using BaseState = typename Base::State; - using Data = typename Base::Data; using Key = typename Base::Key; using Mapped = typename Base::Mapped; - using Base::data; + template + using BaseStateImpl = typename Base::template StateImpl; + AggregationMethodSingleLowCardinalityColumn() = default; template explicit AggregationMethodSingleLowCardinalityColumn(const Other & other) : Base(other) {} - using State = ColumnsHashing::HashMethodSingleLowCardinalityColumn; + template + using StateImpl = ColumnsHashing::HashMethodSingleLowCardinalityColumn, Mapped, use_cache>; + + using State = StateImpl; + using StateNoCache = StateImpl; static const bool low_cardinality_optimization = true; @@ -429,7 +458,7 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod /// For the case where all keys are of fixed length, and they fit in N (for example, 128) bits. -template +template struct AggregationMethodKeysFixed { using Data = TData; @@ -449,13 +478,17 @@ struct AggregationMethodKeysFixed { } - using State = ColumnsHashing::HashMethodKeysFixed< + template + using StateImpl = ColumnsHashing::HashMethodKeysFixed< typename Data::value_type, Key, Mapped, has_nullable_keys, has_low_cardinality, - use_cache>; + use_cache && consecutive_keys_optimization>; + + using State = StateImpl; + using StateNoCache = StateImpl; static const bool low_cardinality_optimization = false; static const bool one_key_nullable_optimization = false; @@ -546,7 +579,11 @@ struct AggregationMethodSerialized { } - using State = ColumnsHashing::HashMethodSerialized; + template + using StateImpl = ColumnsHashing::HashMethodSerialized; + + using State = StateImpl; + using StateNoCache = StateImpl; static const bool low_cardinality_optimization = false; static const bool one_key_nullable_optimization = false; @@ -566,6 +603,7 @@ class Aggregator; using ColumnsHashing::HashMethodContext; using ColumnsHashing::HashMethodContextPtr; +using ColumnsHashing::LastElementCacheStats; struct AggregatedDataVariants : private boost::noncopyable { @@ -599,6 +637,10 @@ struct AggregatedDataVariants : private boost::noncopyable */ AggregatedDataWithoutKey without_key = nullptr; + /// Stats of a cache for consecutive keys optimization. + /// Stats can be used to disable the cache in case of a lot of misses. + LastElementCacheStats consecutive_keys_cache_stats; + // Disable consecutive key optimization for Uint8/16, because they use a FixedHashMap // and the lookup there is almost free, so we don't need to cache the last lookup result std::unique_ptr> key8; @@ -1025,6 +1067,8 @@ public: bool optimize_group_by_constant_keys; + const double min_hit_rate_to_use_consecutive_keys_optimization; + struct StatsCollectingParams { StatsCollectingParams(); @@ -1042,6 +1086,7 @@ public: const size_t max_entries_for_hash_table_stats = 0; const size_t max_size_to_preallocate_for_aggregation = 0; }; + StatsCollectingParams stats_collecting_params; Params( @@ -1063,7 +1108,8 @@ public: bool enable_prefetch_, bool only_merge_, // true for projections bool optimize_group_by_constant_keys_, - const StatsCollectingParams & stats_collecting_params_ = {}) + double min_hit_rate_to_use_consecutive_keys_optimization_, + const StatsCollectingParams & stats_collecting_params_) : keys(keys_) , aggregates(aggregates_) , keys_size(keys.size()) @@ -1084,14 +1130,15 @@ public: , only_merge(only_merge_) , enable_prefetch(enable_prefetch_) , optimize_group_by_constant_keys(optimize_group_by_constant_keys_) + , min_hit_rate_to_use_consecutive_keys_optimization(min_hit_rate_to_use_consecutive_keys_optimization_) , stats_collecting_params(stats_collecting_params_) { } /// Only parameters that matter during merge. - Params(const Names & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_, size_t max_block_size_) + Params(const Names & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_, size_t max_block_size_, double min_hit_rate_to_use_consecutive_keys_optimization_) : Params( - keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, 0, 0, 0, false, nullptr, max_threads_, 0, false, 0, max_block_size_, false, true, {}) + keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, 0, 0, 0, false, nullptr, max_threads_, 0, false, 0, max_block_size_, false, true, false, min_hit_rate_to_use_consecutive_keys_optimization_, {}) { } @@ -1295,15 +1342,28 @@ private: size_t row_end, ColumnRawPtrs & key_columns, AggregateFunctionInstruction * aggregate_instructions, + LastElementCacheStats & consecutive_keys_cache_stats, + bool no_more_keys, + bool all_keys_are_const, + AggregateDataPtr overflow_row) const; + + template + void executeImpl( + Method & method, + State & state, + Arena * aggregates_pool, + size_t row_begin, + size_t row_end, + AggregateFunctionInstruction * aggregate_instructions, bool no_more_keys, bool all_keys_are_const, AggregateDataPtr overflow_row) const; /// Specialization for a particular value no_more_keys. - template + template void executeImplBatch( Method & method, - typename Method::State & state, + State & state, Arena * aggregates_pool, size_t row_begin, size_t row_end, @@ -1413,16 +1473,15 @@ private: bool final, ThreadPool * thread_pool) const; - template + template void mergeStreamsImplCase( Arena * aggregates_pool, - Method & method, + State & state, Table & data, AggregateDataPtr overflow_row, size_t row_begin, size_t row_end, const AggregateColumnsConstData & aggregate_columns_data, - const ColumnRawPtrs & key_columns, Arena * arena_for_keys) const; /// `arena_for_keys` used to store serialized aggregation keys (in methods like `serialized`) to save some space. @@ -1434,6 +1493,7 @@ private: Method & method, Table & data, AggregateDataPtr overflow_row, + LastElementCacheStats & consecutive_keys_cache_stats, bool no_more_keys, Arena * arena_for_keys = nullptr) const; @@ -1443,6 +1503,7 @@ private: Method & method, Table & data, AggregateDataPtr overflow_row, + LastElementCacheStats & consecutive_keys_cache_stats, bool no_more_keys, size_t row_begin, size_t row_end, @@ -1453,6 +1514,7 @@ private: void mergeBlockWithoutKeyStreamsImpl( Block block, AggregatedDataVariants & result) const; + void mergeWithoutKeyStreamsImpl( AggregatedDataVariants & result, size_t row_begin, @@ -1507,6 +1569,18 @@ private: MutableColumns & final_key_columns) const; static bool hasSparseArguments(AggregateFunctionInstruction * aggregate_instructions); + + static void addBatch( + size_t row_begin, size_t row_end, + AggregateFunctionInstruction * inst, + AggregateDataPtr * places, + Arena * arena); + + static void addBatchSinglePlace( + size_t row_begin, size_t row_end, + AggregateFunctionInstruction * inst, + AggregateDataPtr place, + Arena * arena); }; diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index a0750122a5c..63ee62cdef4 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -767,7 +767,6 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( }; StreamingFormatExecutor executor(header, format, std::move(on_error), std::move(adding_defaults_transform)); - std::unique_ptr last_buffer; auto chunk_info = std::make_shared(); auto query_for_logging = serializeQuery(*key.query, insert_context->getSettingsRef().log_queries_cut_to_length); @@ -783,11 +782,6 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( auto buffer = std::make_unique(*bytes); size_t num_bytes = bytes->size(); size_t num_rows = executor.execute(*buffer); - - /// Keep buffer, because it still can be used - /// in destructor, while resetting buffer at next iteration. - last_buffer = std::move(buffer); - total_rows += num_rows; chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); @@ -796,8 +790,6 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( current_exception.clear(); } - format->addBuffer(std::move(last_buffer)); - Chunk chunk(executor.getResultColumns(), total_rows); chunk.setChunkInfo(std::move(chunk_info)); return chunk; diff --git a/src/Interpreters/Cache/FileCacheFactory.cpp b/src/Interpreters/Cache/FileCacheFactory.cpp index 84eafde9afd..3e857d8a8e3 100644 --- a/src/Interpreters/Cache/FileCacheFactory.cpp +++ b/src/Interpreters/Cache/FileCacheFactory.cpp @@ -50,12 +50,35 @@ FileCachePtr FileCacheFactory::getOrCreate( { std::lock_guard lock(mutex); - auto it = caches_by_name.find(cache_name); + auto it = std::find_if(caches_by_name.begin(), caches_by_name.end(), [&](const auto & cache_by_name) + { + return cache_by_name.second->getSettings().base_path == file_cache_settings.base_path; + }); + if (it == caches_by_name.end()) { auto cache = std::make_shared(cache_name, file_cache_settings); - it = caches_by_name.emplace( - cache_name, std::make_unique(cache, file_cache_settings, config_path)).first; + + bool inserted; + std::tie(it, inserted) = caches_by_name.emplace( + cache_name, std::make_unique(cache, file_cache_settings, config_path)); + + if (!inserted) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cache with name {} exists, but it has a different path", cache_name); + } + } + else if (it->second->getSettings() != file_cache_settings) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Found more than one cache configuration with the same path, " + "but with different cache settings ({} and {})", + it->first, cache_name); + } + else if (it->first != cache_name) + { + caches_by_name.emplace(cache_name, it->second); } return it->second->cache; @@ -69,12 +92,33 @@ FileCachePtr FileCacheFactory::create( std::lock_guard lock(mutex); auto it = caches_by_name.find(cache_name); + if (it != caches_by_name.end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cache with name {} already exists", cache_name); - auto cache = std::make_shared(cache_name, file_cache_settings); - it = caches_by_name.emplace( - cache_name, std::make_unique(cache, file_cache_settings, config_path)).first; + it = std::find_if(caches_by_name.begin(), caches_by_name.end(), [&](const auto & cache_by_name) + { + return cache_by_name.second->getSettings().base_path == file_cache_settings.base_path; + }); + + if (it == caches_by_name.end()) + { + auto cache = std::make_shared(cache_name, file_cache_settings); + it = caches_by_name.emplace( + cache_name, std::make_unique(cache, file_cache_settings, config_path)).first; + } + else if (it->second->getSettings() != file_cache_settings) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Found more than one cache configuration with the same path, " + "but with different cache settings ({} and {})", + it->first, cache_name); + } + else + { + [[maybe_unused]] bool inserted = caches_by_name.emplace(cache_name, it->second).second; + chassert(inserted); + } return it->second->cache; } @@ -98,11 +142,14 @@ void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfig caches_by_name_copy = caches_by_name; } + std::unordered_set checked_paths; for (const auto & [_, cache_info] : caches_by_name_copy) { - if (cache_info->config_path.empty()) + if (cache_info->config_path.empty() || checked_paths.contains(cache_info->config_path)) continue; + checked_paths.emplace(cache_info->config_path); + FileCacheSettings new_settings; new_settings.loadFromConfig(config, cache_info->config_path); diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index e2f7f54e203..06261b19db7 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -8,7 +8,7 @@ static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 32 * 1024 * 1024; static constexpr int FILECACHE_DEFAULT_FILE_SEGMENT_ALIGNMENT = 4 * 1024 * 1024; /// 4Mi static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS = 5; static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_QUEUE_SIZE_LIMIT = 5000; -static constexpr int FILECACHE_DEFAULT_LOAD_METADATA_THREADS = 1; +static constexpr int FILECACHE_DEFAULT_LOAD_METADATA_THREADS = 16; static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024; diff --git a/src/Interpreters/Cache/IFileCachePriority.cpp b/src/Interpreters/Cache/IFileCachePriority.cpp index 9109e76562f..eb396a1e323 100644 --- a/src/Interpreters/Cache/IFileCachePriority.cpp +++ b/src/Interpreters/Cache/IFileCachePriority.cpp @@ -13,7 +13,7 @@ namespace DB IFileCachePriority::IFileCachePriority(size_t max_size_, size_t max_elements_) : max_size(max_size_), max_elements(max_elements_) { - CurrentMetrics::set(CurrentMetrics::FilesystemCacheSizeLimit, max_size_); + CurrentMetrics::add(CurrentMetrics::FilesystemCacheSizeLimit, max_size_); } IFileCachePriority::Entry::Entry( diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index 15a80667cc4..73d93514db5 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -7,6 +7,7 @@ #include #include +#include namespace DB { diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 18f7280dd19..c448206ed78 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -412,7 +412,8 @@ void executeQueryWithParallelReplicas( new_cluster = not_optimized_cluster->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas); } - auto coordinator = std::make_shared(new_cluster->getShardCount()); + auto coordinator + = std::make_shared(new_cluster->getShardCount(), settings.parallel_replicas_mark_segment_size); auto external_tables = new_context->getExternalTables(); auto read_from_remote = std::make_unique( query_ast, diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 1a8e0ad96fa..8e73bc8b484 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -44,7 +44,8 @@ ConcurrentHashJoin::ConcurrentHashJoin(ContextPtr context_, std::shared_ptr(); - inner_hash_join->data = std::make_unique(table_join_, right_sample_block, any_take_last_row_); + + inner_hash_join->data = std::make_unique(table_join_, right_sample_block, any_take_last_row_, 0, fmt::format("concurrent{}", i)); hash_joins.emplace_back(std::move(inner_hash_join)); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 25146ebc10d..38944b21c49 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -32,6 +33,7 @@ #include #include #include +#include #include #include #include @@ -43,6 +45,7 @@ #include #include #include +#include #include #include #include @@ -95,6 +98,7 @@ #include #include #include +#include #include #include #include @@ -289,6 +293,7 @@ struct ContextSharedPart : boost::noncopyable MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) ReplicatedFetchList replicated_fetch_list; + RefreshSet refresh_set; /// The list of active refreshes (for MaterializedView) ConfigurationPtr users_config TSA_GUARDED_BY(mutex); /// Config with the users, profiles and quotas sections. InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. @@ -325,6 +330,9 @@ struct ContextSharedPart : boost::noncopyable mutable ThrottlerPtr backups_server_throttler; /// A server-wide throttler for BACKUPs + mutable ThrottlerPtr mutations_throttler; /// A server-wide throttler for mutations + mutable ThrottlerPtr merges_throttler; /// A server-wide throttler for merges + MultiVersion macros; /// Substitutions extracted from config. std::unique_ptr ddl_worker TSA_GUARDED_BY(mutex); /// Process ddl commands from zk. LoadTaskPtr ddl_worker_startup_task; /// To postpone `ddl_worker->startup()` after all tables startup @@ -733,6 +741,12 @@ struct ContextSharedPart : boost::noncopyable if (auto bandwidth = server_settings.max_backup_bandwidth_for_server) backups_server_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_mutations_bandwidth_for_server) + mutations_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_merges_bandwidth_for_server) + merges_throttler = std::make_shared(bandwidth); } }; @@ -825,6 +839,8 @@ MovesList & Context::getMovesList() { return shared->moves_list; } const MovesList & Context::getMovesList() const { return shared->moves_list; } ReplicatedFetchList & Context::getReplicatedFetchList() { return shared->replicated_fetch_list; } const ReplicatedFetchList & Context::getReplicatedFetchList() const { return shared->replicated_fetch_list; } +RefreshSet & Context::getRefreshSet() { return shared->refresh_set; } +const RefreshSet & Context::getRefreshSet() const { return shared->refresh_set; } String Context::resolveDatabase(const String & database_name) const { @@ -2994,6 +3010,16 @@ ThrottlerPtr Context::getBackupsThrottler() const return throttler; } +ThrottlerPtr Context::getMutationsThrottler() const +{ + return shared->mutations_throttler; +} + +ThrottlerPtr Context::getMergesThrottler() const +{ + return shared->merges_throttler; +} + bool Context::hasDistributedDDL() const { return getConfigRef().has("distributed_ddl"); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 39d2212ce80..640aeb0539c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -74,6 +74,7 @@ class BackgroundSchedulePool; class MergeList; class MovesList; class ReplicatedFetchList; +class RefreshSet; class Cluster; class Compiler; class MarkCache; @@ -922,6 +923,9 @@ public: ReplicatedFetchList & getReplicatedFetchList(); const ReplicatedFetchList & getReplicatedFetchList() const; + RefreshSet & getRefreshSet(); + const RefreshSet & getRefreshSet() const; + /// If the current session is expired at the time of the call, synchronously creates and returns a new session with the startNewSession() call. /// If no ZooKeeper configured, throws an exception. std::shared_ptr getZooKeeper() const; @@ -1324,6 +1328,9 @@ public: ThrottlerPtr getBackupsThrottler() const; + ThrottlerPtr getMutationsThrottler() const; + ThrottlerPtr getMergesThrottler() const; + /// Kitchen sink using ContextData::KitchenSink; using ContextData::kitchen_sink; diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 6e9155ab2a2..d418be51cc5 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -215,20 +215,47 @@ ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const Z } -bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper) +bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper, const std::optional & config_host_name) { bool host_in_hostlist = false; std::exception_ptr first_exception = nullptr; + const auto maybe_secure_port = global_context->getTCPPortSecure(); + const auto port = global_context->getTCPPort(); + + if (config_host_name) + { + bool is_local_port = (maybe_secure_port && HostID(*config_host_name, *maybe_secure_port).isLocalAddress(*maybe_secure_port)) || + HostID(*config_host_name, port).isLocalAddress(port); + + if (!is_local_port) + throw Exception( + ErrorCodes::DNS_ERROR, + "{} is not a local address. Check parameter 'host_name' in the configuration", + *config_host_name); + } + for (const HostID & host : entry.hosts) { - auto maybe_secure_port = global_context->getTCPPortSecure(); + if (config_host_name) + { + if (config_host_name != host.host_name) + continue; + + if (maybe_secure_port != host.port && port != host.port) + continue; + + host_in_hostlist = true; + host_id = host; + host_id_str = host.toString(); + break; + } try { /// The port is considered local if it matches TCP or TCP secure port that the server is listening. bool is_local_port - = (maybe_secure_port && host.isLocalAddress(*maybe_secure_port)) || host.isLocalAddress(global_context->getTCPPort()); + = (maybe_secure_port && host.isLocalAddress(*maybe_secure_port)) || host.isLocalAddress(port); if (!is_local_port) continue; diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 1ceb74c7048..bc45b46bf0f 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -44,6 +44,9 @@ struct HostID explicit HostID(const Cluster::Address & address) : host_name(address.host_name), port(address.port) {} + HostID(const String & host_name_, UInt16 port_) + : host_name(host_name_), port(port_) {} + static HostID fromString(const String & host_port_str); String toString() const @@ -143,7 +146,7 @@ struct DDLTask : public DDLTaskBase { DDLTask(const String & name, const String & path) : DDLTaskBase(name, path) {} - bool findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper); + bool findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper, const std::optional & config_host_name); void setClusterInfo(ContextPtr context, Poco::Logger * log); diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index f08fd72ff7f..c0611dfaf7d 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -107,6 +107,9 @@ DDLWorker::DDLWorker( cleanup_delay_period = config->getUInt64(prefix + ".cleanup_delay_period", static_cast(cleanup_delay_period)); max_tasks_in_queue = std::max(1, config->getUInt64(prefix + ".max_tasks_in_queue", max_tasks_in_queue)); + if (config->has(prefix + ".host_name")) + config_host_name = config->getString(prefix + ".host_name"); + if (config->has(prefix + ".profile")) context->setSetting("profile", config->getString(prefix + ".profile")); } @@ -214,7 +217,7 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r /// Stage 2: resolve host_id and check if we should execute query or not /// Multiple clusters can use single DDL queue path in ZooKeeper, /// So we should skip task if we cannot find current host in cluster hosts list. - if (!task->findCurrentHostID(context, log, zookeeper)) + if (!task->findCurrentHostID(context, log, zookeeper, config_host_name)) { out_reason = "There is no a local address in host list"; return add_to_skip_set(); diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index d34a4135199..adc9a491d81 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -153,6 +153,8 @@ protected: ContextMutablePtr context; Poco::Logger * log; + std::optional config_host_name; /// host_name from config + std::string host_fqdn; /// current host domain name std::string host_fqdn_id; /// host_name:port std::string queue_dir; /// dir with queue of queries diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 6d8fd84557c..19882b0b828 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -82,8 +82,8 @@ private: using DDLGuardPtr = std::unique_ptr; -class FutureSet; -using FutureSetPtr = std::shared_ptr; +class FutureSetFromSubquery; +using FutureSetFromSubqueryPtr = std::shared_ptr; /// Creates temporary table in `_temporary_and_external_tables` with randomly generated unique StorageID. /// Such table can be accessed from everywhere by its ID. @@ -116,7 +116,7 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext IDatabase * temporary_tables = nullptr; UUID id = UUIDHelpers::Nil; - FutureSetPtr future_set; + FutureSetFromSubqueryPtr future_set; }; ///TODO maybe remove shared_ptr from here? diff --git a/src/Interpreters/EmbeddedDictionaries.h b/src/Interpreters/EmbeddedDictionaries.h index 674b3a7f01e..e71098636fe 100644 --- a/src/Interpreters/EmbeddedDictionaries.h +++ b/src/Interpreters/EmbeddedDictionaries.h @@ -12,14 +12,13 @@ namespace Poco { class Logger; namespace Util { class AbstractConfiguration; } } +namespace DB +{ + class RegionsHierarchies; class RegionsNames; class GeoDictionariesLoader; - -namespace DB -{ - /// Metrica's Dictionaries which can be used in functions. class EmbeddedDictionaries : WithContext diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 89ea3a326cc..26d666a8913 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -271,7 +272,7 @@ GraceHashJoin::GraceHashJoin( , left_key_names(table_join->getOnlyClause().key_names_left) , right_key_names(table_join->getOnlyClause().key_names_right) , tmp_data(std::make_unique(tmp_data_, CurrentMetrics::TemporaryFilesForJoin)) - , hash_join(makeInMemoryJoin()) + , hash_join(makeInMemoryJoin("grace0")) , hash_join_sample_block(hash_join->savedBlockSample()) { if (!GraceHashJoin::isSupported(table_join)) @@ -424,8 +425,10 @@ void GraceHashJoin::initialize(const Block & sample_block) { left_sample_block = sample_block.cloneEmpty(); output_sample_block = left_sample_block.cloneEmpty(); - ExtraBlockPtr not_processed; + ExtraBlockPtr not_processed = nullptr; hash_join->joinBlock(output_sample_block, not_processed); + if (not_processed) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unhandled not processed block in GraceHashJoin"); initBuckets(); } @@ -447,9 +450,6 @@ void GraceHashJoin::joinBlock(Block & block, std::shared_ptr & not_p block = std::move(blocks[current_bucket->idx]); hash_join->joinBlock(block, not_processed); - if (not_processed) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unhandled not processed block in GraceHashJoin"); - flushBlocksToBuckets(blocks, buckets); } @@ -528,6 +528,29 @@ public: Block nextImpl() override { + ExtraBlockPtr not_processed = nullptr; + + { + std::lock_guard lock(extra_block_mutex); + if (!not_processed_blocks.empty()) + { + not_processed = std::move(not_processed_blocks.front()); + not_processed_blocks.pop_front(); + } + } + + if (not_processed) + { + Block block = std::move(not_processed->block); + hash_join->joinBlock(block, not_processed); + if (not_processed) + { + std::lock_guard lock(extra_block_mutex); + not_processed_blocks.emplace_back(std::move(not_processed)); + } + return block; + } + Block block; size_t num_buckets = buckets.size(); size_t current_idx = buckets[current_bucket]->idx; @@ -565,12 +588,12 @@ public: } } while (block.rows() == 0); - ExtraBlockPtr not_processed; hash_join->joinBlock(block, not_processed); - if (not_processed) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unsupported hash join type"); - + { + std::lock_guard lock(extra_block_mutex); + not_processed_blocks.emplace_back(std::move(not_processed)); + } return block; } @@ -582,6 +605,9 @@ public: Names left_key_names; Names right_key_names; + + std::mutex extra_block_mutex; + std::list not_processed_blocks TSA_GUARDED_BY(extra_block_mutex); }; IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() @@ -611,7 +637,7 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() continue; } - hash_join = makeInMemoryJoin(prev_keys_num); + hash_join = makeInMemoryJoin(fmt::format("grace{}", bucket_idx), prev_keys_num); auto right_reader = current_bucket->startJoining(); size_t num_rows = 0; /// count rows that were written and rehashed while (Block block = right_reader.read()) @@ -632,10 +658,9 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() return nullptr; } -GraceHashJoin::InMemoryJoinPtr GraceHashJoin::makeInMemoryJoin(size_t reserve_num) +GraceHashJoin::InMemoryJoinPtr GraceHashJoin::makeInMemoryJoin(const String & bucket_id, size_t reserve_num) { - auto ret = std::make_unique(table_join, right_sample_block, any_take_last_row, reserve_num); - return std::move(ret); + return std::make_unique(table_join, right_sample_block, any_take_last_row, reserve_num, bucket_id); } Block GraceHashJoin::prepareRightBlock(const Block & block) @@ -661,7 +686,7 @@ void GraceHashJoin::addBlockToJoinImpl(Block block) { std::lock_guard lock(hash_join_mutex); if (!hash_join) - hash_join = makeInMemoryJoin(); + hash_join = makeInMemoryJoin(fmt::format("grace{}", bucket_index)); // buckets size has been changed in other threads. Need to scatter current_block again. // rehash could only happen under hash_join_mutex's scope. @@ -705,7 +730,7 @@ void GraceHashJoin::addBlockToJoinImpl(Block block) current_block = concatenateBlocks(current_blocks); } - hash_join = makeInMemoryJoin(prev_keys_num); + hash_join = makeInMemoryJoin(fmt::format("grace{}", bucket_index), prev_keys_num); if (current_block.rows() > 0) hash_join->addBlockToJoin(current_block, /* check_limits = */ false); diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index 44949440467..2cadeee10b9 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -44,9 +44,8 @@ class GraceHashJoin final : public IJoin { class FileBucket; class DelayedBlocks; - using InMemoryJoin = HashJoin; - using InMemoryJoinPtr = std::shared_ptr; + using InMemoryJoinPtr = std::shared_ptr; public: using BucketPtr = std::shared_ptr; @@ -91,7 +90,7 @@ public: private: void initBuckets(); /// Create empty join for in-memory processing. - InMemoryJoinPtr makeInMemoryJoin(size_t reserve_num = 0); + InMemoryJoinPtr makeInMemoryJoin(const String & bucket_id, size_t reserve_num = 0); /// Add right table block to the @join. Calls @rehash on overflow. void addBlockToJoinImpl(Block block); diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 0d7c40cc27d..a84e1ec2175 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include @@ -233,7 +234,8 @@ static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nulla JoinCommon::removeColumnNullability(column); } -HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_sample_block_, bool any_take_last_row_, size_t reserve_num) +HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_sample_block_, + bool any_take_last_row_, size_t reserve_num, const String & instance_id_) : table_join(table_join_) , kind(table_join->kind()) , strictness(table_join->strictness()) @@ -241,10 +243,11 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , asof_inequality(table_join->getAsofInequality()) , data(std::make_shared()) , right_sample_block(right_sample_block_) + , instance_log_id(!instance_id_.empty() ? "(" + instance_id_ + ") " : "") , log(&Poco::Logger::get("HashJoin")) { - LOG_DEBUG(log, "({}) Datatype: {}, kind: {}, strictness: {}, right header: {}", fmt::ptr(this), data->type, kind, strictness, right_sample_block.dumpStructure()); - LOG_DEBUG(log, "({}) Keys: {}", fmt::ptr(this), TableJoin::formatClauses(table_join->getClauses(), true)); + LOG_TRACE(log, "{}Keys: {}, datatype: {}, kind: {}, strictness: {}, right header: {}", + instance_log_id, TableJoin::formatClauses(table_join->getClauses(), true), data->type, kind, strictness, right_sample_block.dumpStructure()); if (isCrossOrComma(kind)) { @@ -1165,9 +1168,27 @@ public: std::vector join_on_keys; + size_t max_joined_block_rows = 0; size_t rows_to_add; std::unique_ptr offsets_to_replicate; bool need_filter = false; + IColumn::Filter filter; + + void reserve(bool need_replicate) + { + if (!max_joined_block_rows) + return; + + /// Do not allow big allocations when user set max_joined_block_rows to huge value + size_t reserve_size = std::min(max_joined_block_rows, DEFAULT_BLOCK_SIZE * 2); + + if (need_replicate) + /// Reserve 10% more space for columns, because some rows can be repeated + reserve_size = static_cast(1.1 * reserve_size); + + for (auto & column : columns) + column->reserve(reserve_size); + } private: std::vector type_name; @@ -1356,7 +1377,7 @@ void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unuse /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). template -NO_INLINE IColumn::Filter joinRightColumns( +NO_INLINE size_t joinRightColumns( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -1365,9 +1386,8 @@ NO_INLINE IColumn::Filter joinRightColumns( constexpr JoinFeatures join_features; size_t rows = added_columns.rows_to_add; - IColumn::Filter filter; if constexpr (need_filter) - filter = IColumn::Filter(rows, 0); + added_columns.filter = IColumn::Filter(rows, 0); Arena pool; @@ -1375,9 +1395,20 @@ NO_INLINE IColumn::Filter joinRightColumns( added_columns.offsets_to_replicate = std::make_unique(rows); IColumn::Offset current_offset = 0; - - for (size_t i = 0; i < rows; ++i) + size_t max_joined_block_rows = added_columns.max_joined_block_rows; + size_t i = 0; + for (; i < rows; ++i) { + if constexpr (join_features.need_replication) + { + if (unlikely(current_offset > max_joined_block_rows)) + { + added_columns.offsets_to_replicate->resize_assume_reserved(i); + added_columns.filter.resize_assume_reserved(i); + break; + } + } + bool right_row_found = false; KnownRowsHolder known_rows; @@ -1402,7 +1433,7 @@ NO_INLINE IColumn::Filter joinRightColumns( auto row_ref = mapped->findAsof(left_asof_key, i); if (row_ref.block) { - setUsed(filter, i); + setUsed(added_columns.filter, i); if constexpr (multiple_disjuncts) used_flags.template setUsed(row_ref.block, row_ref.row_num, 0); else @@ -1415,7 +1446,7 @@ NO_INLINE IColumn::Filter joinRightColumns( } else if constexpr (join_features.is_all_join) { - setUsed(filter, i); + setUsed(added_columns.filter, i); used_flags.template setUsed(find_result); auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); @@ -1427,7 +1458,7 @@ NO_INLINE IColumn::Filter joinRightColumns( if (used_once) { auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; - setUsed(filter, i); + setUsed(added_columns.filter, i); addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); } } @@ -1438,7 +1469,7 @@ NO_INLINE IColumn::Filter joinRightColumns( /// Use first appeared left key only if (used_once) { - setUsed(filter, i); + setUsed(added_columns.filter, i); added_columns.appendFromBlock(*mapped.block, mapped.row_num); } @@ -1455,7 +1486,7 @@ NO_INLINE IColumn::Filter joinRightColumns( } else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) { - setUsed(filter, i); + setUsed(added_columns.filter, i); used_flags.template setUsed(find_result); added_columns.appendFromBlock(*mapped.block, mapped.row_num); @@ -1470,7 +1501,7 @@ NO_INLINE IColumn::Filter joinRightColumns( if (!right_row_found) { if constexpr (join_features.is_anti_join && join_features.left) - setUsed(filter, i); + setUsed(added_columns.filter, i); addNotFoundRow(added_columns, current_offset); } @@ -1481,11 +1512,11 @@ NO_INLINE IColumn::Filter joinRightColumns( } added_columns.applyLazyDefaults(); - return filter; + return i; } template -IColumn::Filter joinRightColumnsSwitchMultipleDisjuncts( +size_t joinRightColumnsSwitchMultipleDisjuncts( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -1497,7 +1528,7 @@ IColumn::Filter joinRightColumnsSwitchMultipleDisjuncts( } template -IColumn::Filter joinRightColumnsSwitchNullability( +size_t joinRightColumnsSwitchNullability( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -1514,7 +1545,7 @@ IColumn::Filter joinRightColumnsSwitchNullability( } template -IColumn::Filter switchJoinRightColumns( +size_t switchJoinRightColumns( const std::vector & mapv, AddedColumns & added_columns, HashJoin::Type type, @@ -1597,10 +1628,27 @@ ColumnWithTypeAndName copyLeftKeyColumnToRight( return right_column; } +/// Cut first num_rows rows from block in place and returns block with remaining rows +Block sliceBlock(Block & block, size_t num_rows) +{ + size_t total_rows = block.rows(); + if (num_rows >= total_rows) + return {}; + size_t remaining_rows = total_rows - num_rows; + Block remaining_block = block.cloneEmpty(); + for (size_t i = 0; i < block.columns(); ++i) + { + auto & col = block.getByPosition(i); + remaining_block.getByPosition(i).column = col.column->cut(num_rows, remaining_rows); + col.column = col.column->cut(0, num_rows); + } + return remaining_block; +} + } /// nameless template -void HashJoin::joinBlockImpl( +Block HashJoin::joinBlockImpl( Block & block, const Block & block_with_columns_to_add, const std::vector & maps_, @@ -1642,8 +1690,16 @@ void HashJoin::joinBlockImpl( bool has_required_right_keys = (required_right_keys.columns() != 0); added_columns.need_filter = join_features.need_filter || has_required_right_keys; + added_columns.max_joined_block_rows = table_join->maxJoinedBlockRows(); + if (!added_columns.max_joined_block_rows) + added_columns.max_joined_block_rows = std::numeric_limits::max(); + else + added_columns.reserve(join_features.need_replication); - IColumn::Filter row_filter = switchJoinRightColumns(maps_, added_columns, data->type, used_flags); + size_t num_joined = switchJoinRightColumns(maps_, added_columns, data->type, used_flags); + /// Do not hold memory for join_on_keys anymore + added_columns.join_on_keys.clear(); + Block remaining_block = sliceBlock(block, num_joined); for (size_t i = 0; i < added_columns.size(); ++i) block.insert(added_columns.moveColumn(i)); @@ -1654,7 +1710,7 @@ void HashJoin::joinBlockImpl( { /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. for (size_t i = 0; i < existing_columns; ++i) - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(row_filter, -1); + block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns.filter, -1); /// Add join key columns from right block if needed using value from left table because of equality for (size_t i = 0; i < required_right_keys.columns(); ++i) @@ -1688,7 +1744,7 @@ void HashJoin::joinBlockImpl( continue; const auto & left_column = block.getByName(required_right_keys_sources[i]); - auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &row_filter); + auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &added_columns.filter); block.insert(std::move(right_col)); if constexpr (join_features.need_replication) @@ -1709,6 +1765,8 @@ void HashJoin::joinBlockImpl( for (size_t pos : right_keys_to_replicate) block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); } + + return remaining_block; } void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const @@ -1885,7 +1943,11 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) if (joinDispatch(kind, strictness, maps_vector, [&](auto kind_, auto strictness_, auto & maps_vector_) { - joinBlockImpl(block, sample_block_with_columns_to_add, maps_vector_); + Block remaining_block = joinBlockImpl(block, sample_block_with_columns_to_add, maps_vector_); + if (remaining_block.rows()) + not_processed = std::make_shared(ExtraBlock{std::move(remaining_block)}); + else + not_processed.reset(); })) { /// Joined @@ -1899,10 +1961,10 @@ HashJoin::~HashJoin() { if (!data) { - LOG_TRACE(log, "({}) Join data has been already released", fmt::ptr(this)); + LOG_TRACE(log, "{}Join data has been already released", instance_log_id); return; } - LOG_TRACE(log, "({}) Join data is being destroyed, {} bytes and {} rows in hash table", fmt::ptr(this), getTotalByteCount(), getTotalRowCount()); + LOG_TRACE(log, "{}Join data is being destroyed, {} bytes and {} rows in hash table", instance_log_id, getTotalByteCount(), getTotalRowCount()); } template @@ -2183,7 +2245,7 @@ void HashJoin::reuseJoinedData(const HashJoin & join) BlocksList HashJoin::releaseJoinedBlocks(bool restructure) { - LOG_TRACE(log, "({}) Join data is being released, {} bytes and {} rows in hash table", fmt::ptr(this), getTotalByteCount(), getTotalRowCount()); + LOG_TRACE(log, "{}Join data is being released, {} bytes and {} rows in hash table", instance_log_id, getTotalByteCount(), getTotalRowCount()); BlocksList right_blocks = std::move(data->blocks); if (!restructure) diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index d125e56057f..284cf5d0e7f 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -147,7 +147,8 @@ class HashJoin : public IJoin { public: HashJoin( - std::shared_ptr table_join_, const Block & right_sample_block, bool any_take_last_row_ = false, size_t reserve_num = 0); + std::shared_ptr table_join_, const Block & right_sample_block, + bool any_take_last_row_ = false, size_t reserve_num = 0, const String & instance_id_ = ""); ~HashJoin() override; @@ -436,6 +437,10 @@ private: bool shrink_blocks = false; Int64 memory_usage_before_adding_blocks = 0; + /// Identifier to distinguish different HashJoin instances in logs + /// Several instances can be created, for example, in GraceHashJoin to handle different buckets + String instance_log_id; + Poco::Logger * log; /// Should be set via setLock to protect hash table from modification from StorageJoin @@ -447,7 +452,7 @@ private: void initRightBlockStructure(Block & saved_block_sample); template - void joinBlockImpl( + Block joinBlockImpl( Block & block, const Block & block_with_columns_to_add, const std::vector & maps_, diff --git a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp index 478735c432b..a4e86879596 100644 --- a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -26,4 +27,13 @@ BlockIO InterpreterAlterNamedCollectionQuery::execute() return {}; } +void registerInterpreterAlterNamedCollectionQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterAlterNamedCollectionQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index db93467c0a4..bfcb0d6dd39 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -460,6 +461,11 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS required_access.emplace_back(AccessType::ALTER_VIEW_MODIFY_QUERY, database, table); break; } + case ASTAlterCommand::MODIFY_REFRESH: + { + required_access.emplace_back(AccessType::ALTER_VIEW_MODIFY_REFRESH, database, table); + break; + } case ASTAlterCommand::LIVE_VIEW_REFRESH: { required_access.emplace_back(AccessType::ALTER_VIEW_REFRESH, database, table); @@ -530,4 +536,13 @@ void InterpreterAlterQuery::extendQueryLogElemImpl(QueryLogElement & elem, const } } +void registerInterpreterAlterQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterAlterQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index be5fcedce27..6f76b21a7b8 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -47,4 +48,13 @@ BlockIO InterpreterBackupQuery::execute() return res_io; } +void registerInterpreterBackupQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterBackupQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 6f28aee5f13..0cc4afd62f2 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include @@ -472,4 +473,13 @@ BlockIO InterpreterCheckQuery::execute() return res; } +void registerInterpreterCheckQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCheckQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index ea59115b077..18e9ba4a64b 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -53,4 +54,13 @@ BlockIO InterpreterCreateFunctionQuery::execute() return {}; } +void registerInterpreterCreateFunctionQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateFunctionQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterCreateIndexQuery.cpp b/src/Interpreters/InterpreterCreateIndexQuery.cpp index ed29c82a0f0..cd2f996c74c 100644 --- a/src/Interpreters/InterpreterCreateIndexQuery.cpp +++ b/src/Interpreters/InterpreterCreateIndexQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -99,4 +100,13 @@ BlockIO InterpreterCreateIndexQuery::execute() return {}; } +void registerInterpreterCreateIndexQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateIndexQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp index 3b0fba5fd9f..41e87bb73dd 100644 --- a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -26,4 +27,13 @@ BlockIO InterpreterCreateNamedCollectionQuery::execute() return {}; } +void registerInterpreterCreateNamedCollectionQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateNamedCollectionQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 1eadb325e95..2bddb4935de 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -40,6 +40,7 @@ #include #include #include +#include #include #include #include @@ -282,7 +283,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) else if (create.uuid != UUIDHelpers::Nil && !DatabaseCatalog::instance().hasUUIDMapping(create.uuid)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find UUID mapping for {}, it's a bug", create.uuid); - DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", getContext()); + DatabasePtr database = DatabaseFactory::instance().get(create, metadata_path / "", getContext()); if (create.uuid != UUIDHelpers::Nil) create.setDatabase(TABLE_WITH_UUID_NAME_PLACEHOLDER); @@ -1089,6 +1090,13 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data "{} UUID specified, but engine of database {} is not Atomic", kind, create.getDatabase()); } + if (create.refresh_strategy && database->getEngineName() != "Atomic") + throw Exception(ErrorCodes::INCORRECT_QUERY, + "Refreshable materialized view requires Atomic database engine, but database {} has engine {}", create.getDatabase(), database->getEngineName()); + /// TODO: Support Replicated databases, only with Shared/ReplicatedMergeTree. + /// Figure out how to make the refreshed data appear all at once on other + /// replicas; maybe a replicated SYSTEM SYNC REPLICA query before the rename? + /// The database doesn't support UUID so we'll ignore it. The UUID could be set here because of either /// a) the initiator of `ON CLUSTER` query generated it to ensure the same UUIDs are used on different hosts; or /// b) `RESTORE from backup` query generated it to ensure the same UUIDs are used on different hosts. @@ -1210,6 +1218,16 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) visitor.visit(*create.select); } + if (create.refresh_strategy) + { + if (!getContext()->getSettingsRef().allow_experimental_refreshable_materialized_view) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Refreshable materialized views are experimental. Enable allow_experimental_refreshable_materialized_view to use."); + + AddDefaultDatabaseVisitor visitor(getContext(), current_database); + visitor.visit(*create.refresh_strategy); + } + if (create.columns_list) { AddDefaultDatabaseVisitor visitor(getContext(), current_database); @@ -1878,4 +1896,13 @@ void InterpreterCreateQuery::addColumnsDescriptionToCreateQueryIfNecessary(ASTCr } } +void registerInterpreterCreateQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 52723416400..5c13a1145d1 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -110,4 +111,13 @@ BlockIO InterpreterDeleteQuery::execute() } } +void registerInterpreterDeleteQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDeleteQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDescribeCacheQuery.cpp b/src/Interpreters/InterpreterDescribeCacheQuery.cpp index 54b43a8850b..c7e863bf260 100644 --- a/src/Interpreters/InterpreterDescribeCacheQuery.cpp +++ b/src/Interpreters/InterpreterDescribeCacheQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -68,4 +69,13 @@ BlockIO InterpreterDescribeCacheQuery::execute() return res; } +void registerInterpreterDescribeCacheQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDescribeCacheQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 755de7e4c5f..1aab72afcc1 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -271,4 +272,13 @@ void InterpreterDescribeQuery::addSubcolumns(const ColumnDescription & column, b }, ISerialization::SubstreamData(type->getDefaultSerialization()).withType(type)); } +void registerInterpreterDescribeQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDescribeQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDropFunctionQuery.cpp b/src/Interpreters/InterpreterDropFunctionQuery.cpp index c2cd24044da..2661fd9058c 100644 --- a/src/Interpreters/InterpreterDropFunctionQuery.cpp +++ b/src/Interpreters/InterpreterDropFunctionQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -49,4 +50,13 @@ BlockIO InterpreterDropFunctionQuery::execute() return {}; } +void registerInterpreterDropFunctionQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropFunctionQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDropIndexQuery.cpp b/src/Interpreters/InterpreterDropIndexQuery.cpp index 98d48942487..025677eeb91 100644 --- a/src/Interpreters/InterpreterDropIndexQuery.cpp +++ b/src/Interpreters/InterpreterDropIndexQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -68,4 +69,13 @@ BlockIO InterpreterDropIndexQuery::execute() return {}; } +void registerInterpreterDropIndexQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropIndexQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp index fe49b1cfd7c..baadc85f443 100644 --- a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -26,4 +27,13 @@ BlockIO InterpreterDropNamedCollectionQuery::execute() return {}; } +void registerInterpreterDropNamedCollectionQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropNamedCollectionQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index b8c9d5dabb5..711100b5de1 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -519,4 +520,12 @@ bool InterpreterDropQuery::supportsTransactions() const && drop.table; } +void registerInterpreterDropQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterExistsQuery.cpp b/src/Interpreters/InterpreterExistsQuery.cpp index 90fa15bf63f..e4176a44533 100644 --- a/src/Interpreters/InterpreterExistsQuery.cpp +++ b/src/Interpreters/InterpreterExistsQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -81,4 +82,12 @@ QueryPipeline InterpreterExistsQuery::executeImpl() "result" }})); } +void registerInterpreterExistsQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterExistsQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 4c70dcb3cc3..458be843b59 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -607,4 +608,13 @@ QueryPipeline InterpreterExplainQuery::executeImpl() return QueryPipeline(std::make_shared(sample_block.cloneWithColumns(std::move(res_columns)))); } +void registerInterpreterExplainQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterExplainQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterExternalDDLQuery.cpp b/src/Interpreters/InterpreterExternalDDLQuery.cpp index c0acb1e03eb..e68ff18dbb6 100644 --- a/src/Interpreters/InterpreterExternalDDLQuery.cpp +++ b/src/Interpreters/InterpreterExternalDDLQuery.cpp @@ -1,6 +1,7 @@ #include "config.h" #include +#include #include #include @@ -67,4 +68,13 @@ BlockIO InterpreterExternalDDLQuery::execute() return BlockIO(); } +void registerInterpreterExternalDDLQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterExternalDDLQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index e32cbe4ccad..84432415f5e 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -30,7 +30,6 @@ #include #include #include -#include #include #include @@ -50,68 +49,14 @@ #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include -#include -#include -#include -#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - #include - -#include #include #include #include @@ -131,10 +76,22 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_TYPE_OF_QUERY; + extern const int LOGICAL_ERROR; } +InterpreterFactory & InterpreterFactory::instance() +{ + static InterpreterFactory interpreter_fact; + return interpreter_fact; +} -std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMutablePtr context, const SelectQueryOptions & options) +void InterpreterFactory::registerInterpreter(const std::string & name, CreatorFn creator_fn) +{ + if (!interpreters.emplace(name, std::move(creator_fn)).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "InterpreterFactory: the interpreter name '{}' is not unique", name); +} + +InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, ContextMutablePtr context, const SelectQueryOptions & options) { ProfileEvents::increment(ProfileEvents::Query); @@ -147,258 +104,247 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut ProfileEvents::increment(ProfileEvents::QueriesWithSubqueries); } + Arguments arguments { + .query = query, + .context = context, + .options = options + }; + + String interpreter_name; + if (query->as()) { if (context->getSettingsRef().allow_experimental_analyzer) - return std::make_unique(query, context, options); - + interpreter_name = "InterpreterSelectQueryAnalyzer"; /// This is internal part of ASTSelectWithUnionQuery. /// Even if there is SELECT without union, it is represented by ASTSelectWithUnionQuery with single ASTSelectQuery as a child. - return std::make_unique(query, context, options); + else + interpreter_name = "InterpreterSelectQuery"; } else if (query->as()) { ProfileEvents::increment(ProfileEvents::SelectQuery); if (context->getSettingsRef().allow_experimental_analyzer) - return std::make_unique(query, context, options); - - return std::make_unique(query, context, options); + interpreter_name = "InterpreterSelectQueryAnalyzer"; + else + interpreter_name = "InterpreterSelectWithUnionQuery"; } else if (query->as()) { - return std::make_unique(query, context, options); + interpreter_name = "InterpreterSelectIntersectExceptQuery"; } else if (query->as()) { ProfileEvents::increment(ProfileEvents::InsertQuery); bool allow_materialized = static_cast(context->getSettingsRef().insert_allow_materialized_columns); - return std::make_unique(query, context, allow_materialized); + arguments.allow_materialized = allow_materialized; + interpreter_name = "InterpreterInsertQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDropQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterUndropQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterRenameQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowTablesQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowColumnsQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowIndexesQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowSettingQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowEnginesQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowFunctionsQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterUseQuery"; } else if (query->as()) { /// readonly is checked inside InterpreterSetQuery - return std::make_unique(query, context); + interpreter_name = "InterpreterSetQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterSetRoleQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterOptimizeQuery"; } - else if (query->as()) + else if (query->as() || query->as() || query->as() || query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterExistsQuery"; } - else if (query->as()) + else if (query->as() || query->as() || query->as() || query->as()) { - return std::make_unique(query, context); - } - else if (query->as()) - { - return std::make_unique(query, context); - } - else if (query->as()) - { - return std::make_unique(query, context); - } - else if (query->as()) - { - return std::make_unique(query, context); - } - else if (query->as()) - { - return std::make_unique(query, context); - } - else if (query->as()) - { - return std::make_unique(query, context); - } - else if (query->as()) - { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowCreateQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDescribeQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDescribeCacheQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterExplainQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowProcesslistQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterAlterQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterAlterNamedCollectionQuery"; } else if (query->as() || query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCheckQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterKillQueryQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterSystemQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterWatchQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateUserQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateRoleQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateQuotaQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateRowPolicyQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateSettingsProfileQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDropAccessEntityQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterMoveAccessEntityQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDropNamedCollectionQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterGrantQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowCreateAccessEntityQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowGrantsQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowAccessEntitiesQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name= "InterpreterShowAccessQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterShowPrivilegesQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterExternalDDLQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterTransactionControlQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateFunctionQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDropFunctionQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateIndexQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterCreateNamedCollectionQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDropIndexQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterBackupQuery"; } else if (query->as()) { - return std::make_unique(query, context); + interpreter_name = "InterpreterDeleteQuery"; } - else - { + + if (!interpreters.contains(interpreter_name)) throw Exception(ErrorCodes::UNKNOWN_TYPE_OF_QUERY, "Unknown type of query: {}", query->getID()); - } + + // creator_fn creates and returns a InterpreterPtr with the supplied arguments + auto creator_fn = interpreters.at(interpreter_name); + + return creator_fn(arguments); } } diff --git a/src/Interpreters/InterpreterFactory.h b/src/Interpreters/InterpreterFactory.h index 774cbd1cb0f..3cf3b02d826 100644 --- a/src/Interpreters/InterpreterFactory.h +++ b/src/Interpreters/InterpreterFactory.h @@ -11,13 +11,34 @@ namespace DB class Context; -class InterpreterFactory +class InterpreterFactory : private boost::noncopyable { public: - static std::unique_ptr get( + static InterpreterFactory & instance(); + + struct Arguments + { + ASTPtr & query; + ContextMutablePtr context; + const SelectQueryOptions & options; + bool allow_materialized; + }; + + using InterpreterPtr = std::unique_ptr; + + InterpreterPtr get( ASTPtr & query, ContextMutablePtr context, const SelectQueryOptions & options = {}); + + using CreatorFn = std::function; + + using Interpreters = std::unordered_map; + + void registerInterpreter(const std::string & name, CreatorFn creator_fn); + +private: + Interpreters interpreters; }; } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 283289f0dfc..c8e05fcd5e3 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -680,4 +681,12 @@ void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, cons extendQueryLogElemImpl(elem, context_); } +void registerInterpreterInsertQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context, args.allow_materialized); + }; + factory.registerInterpreter("InterpreterInsertQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 6e1422f2938..5efffdaa194 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -452,4 +453,13 @@ AccessRightsElements InterpreterKillQueryQuery::getRequiredAccessForDDLOnCluster return required_access; } +void registerInterpreterKillQueryQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterKillQueryQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index 6be78deb897..1e2eaa50ab1 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -79,7 +80,7 @@ BlockIO InterpreterOptimizeQuery::execute() if (auto * snapshot_data = dynamic_cast(storage_snapshot->data.get())) snapshot_data->parts = {}; - table->optimize(query_ptr, metadata_snapshot, ast.partition, ast.final, ast.deduplicate, column_names, getContext()); + table->optimize(query_ptr, metadata_snapshot, ast.partition, ast.final, ast.deduplicate, column_names, ast.cleanup, getContext()); return {}; } @@ -93,4 +94,12 @@ AccessRightsElements InterpreterOptimizeQuery::getRequiredAccess() const return required_access; } +void registerInterpreterOptimizeQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterOptimizeQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index ae79b3f932e..52001fdcaf4 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -228,4 +229,13 @@ void InterpreterRenameQuery::extendQueryLogElemImpl(QueryLogElement & elem, cons } } +void registerInterpreterRenameQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterRenameQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 2218ed4417b..6eac2db20c9 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -210,4 +211,13 @@ void InterpreterSelectIntersectExceptQuery::extendQueryLogElemImpl(QueryLogEleme } } +void registerInterpreterSelectIntersectExceptQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context, args.options); + }; + factory.registerInterpreter("InterpreterSelectIntersectExceptQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8543b5ca552..e057b4fd7e0 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -22,6 +22,7 @@ #include #include +#include #include #include #include @@ -2037,7 +2038,7 @@ static void executeMergeAggregatedImpl( * but it can work more slowly. */ - Aggregator::Params params(keys, aggregates, overflow_row, settings.max_threads, settings.max_block_size); + Aggregator::Params params(keys, aggregates, overflow_row, settings.max_threads, settings.max_block_size, settings.min_hit_rate_to_use_consecutive_keys_optimization); auto merging_aggregated = std::make_unique( query_plan.getCurrentDataStream(), @@ -2378,12 +2379,25 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle else { // It's possible to optimize count() given only partition predicates - SelectQueryInfo temp_query_info; - temp_query_info.query = query_ptr; - temp_query_info.syntax_analyzer_result = syntax_analyzer_result; - temp_query_info.prepared_sets = query_analyzer->getPreparedSets(); + ActionsDAG::NodeRawConstPtrs filter_nodes; + if (analysis_result.hasPrewhere()) + { + auto & prewhere_info = analysis_result.prewhere_info; + filter_nodes.push_back(&prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name)); - return storage->totalRowsByPartitionPredicate(temp_query_info, context); + if (prewhere_info->row_level_filter) + filter_nodes.push_back(&prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name)); + } + if (analysis_result.hasWhere()) + { + filter_nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name)); + } + + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes, {}, context); + if (!filter_actions_dag) + return {}; + + return storage->totalRowsByPartitionPredicate(filter_actions_dag, context); } } @@ -2501,7 +2515,12 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc max_block_size = std::max(1, max_block_limited); max_threads_execute_query = max_streams = 1; } - if (max_block_limited < local_limits.local_limits.size_limits.max_rows) + if (local_limits.local_limits.size_limits.max_rows != 0) + { + if (max_block_limited < local_limits.local_limits.size_limits.max_rows) + query_info.limit = max_block_limited; + } + else { query_info.limit = max_block_limited; } @@ -2703,6 +2722,7 @@ static Aggregator::Params getAggregatorParams( settings.enable_software_prefetch_in_aggregation, /* only_merge */ false, settings.optimize_group_by_constant_keys, + settings.min_hit_rate_to_use_consecutive_keys_optimization, stats_collecting_params }; } @@ -3314,5 +3334,13 @@ bool InterpreterSelectQuery::isQueryWithFinal(const SelectQueryInfo & info) return result; } +void registerInterpreterSelectQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context, args.options); + }; + factory.registerInterpreter("InterpreterSelectQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index eed9d03ab5a..f498fc7ec85 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -267,4 +268,13 @@ void InterpreterSelectQueryAnalyzer::extendQueryLogElemImpl(QueryLogElement & el elem.used_row_policies.emplace(used_row_policy); } +void registerInterpreterSelectQueryAnalyzer(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context, args.options); + }; + factory.registerInterpreter("InterpreterSelectQueryAnalyzer", create_fn); +} + } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 2ae74955e4f..16bc4b1fe2e 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -410,4 +411,13 @@ void InterpreterSelectWithUnionQuery::extendQueryLogElemImpl(QueryLogElement & e } } +void registerInterpreterSelectWithUnionQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context, args.options); + }; + factory.registerInterpreter("InterpreterSelectWithUnionQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index 2c0baa0d4b3..261c781e0ba 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -91,4 +92,12 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta } } +void registerInterpreterSetQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterSetQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index a5b22387448..149ba6d7575 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -164,5 +165,13 @@ BlockIO InterpreterShowColumnsQuery::execute() return executeQuery(getRewrittenQuery(), getContext(), QueryFlags{ .internal = true }).second; } +void registerInterpreterShowColumnsQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowColumnsQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 0d60f13af66..9edac1fd8e1 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -104,4 +105,13 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl() "statement"}})); } +void registerInterpreterShowCreateQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + + factory.registerInterpreter("InterpreterShowCreateQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterShowEngineQuery.cpp b/src/Interpreters/InterpreterShowEngineQuery.cpp index 2927fbd0f2d..f2d057a3fcf 100644 --- a/src/Interpreters/InterpreterShowEngineQuery.cpp +++ b/src/Interpreters/InterpreterShowEngineQuery.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -15,4 +16,13 @@ BlockIO InterpreterShowEnginesQuery::execute() return executeQuery("SELECT * FROM system.table_engines ORDER BY name", getContext(), QueryFlags{ .internal = true }).second; } +void registerInterpreterShowEnginesQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowEnginesQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterShowFunctionsQuery.cpp b/src/Interpreters/InterpreterShowFunctionsQuery.cpp index a9da01b0988..e83f61eac53 100644 --- a/src/Interpreters/InterpreterShowFunctionsQuery.cpp +++ b/src/Interpreters/InterpreterShowFunctionsQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -43,4 +44,13 @@ FROM {}.{})", return rewritten_query; } +void registerInterpreterShowFunctionsQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowFunctionsQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index 09b70e951db..e8005ead91e 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -104,5 +105,13 @@ BlockIO InterpreterShowIndexesQuery::execute() return executeQuery(getRewrittenQuery(), getContext(), QueryFlags{ .internal = true }).second; } +void registerInterpreterShowIndexesQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowIndexesQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterShowProcesslistQuery.cpp b/src/Interpreters/InterpreterShowProcesslistQuery.cpp index f711cc0dac9..7bdb94482da 100644 --- a/src/Interpreters/InterpreterShowProcesslistQuery.cpp +++ b/src/Interpreters/InterpreterShowProcesslistQuery.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -15,4 +16,13 @@ BlockIO InterpreterShowProcesslistQuery::execute() return executeQuery("SELECT * FROM system.processes ORDER BY elapsed DESC", getContext(), QueryFlags{ .internal = true }).second; } +void registerInterpreterShowProcesslistQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowProcesslistQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterShowSettingQuery.cpp b/src/Interpreters/InterpreterShowSettingQuery.cpp index 45e9b8a1f1c..90acaa7b083 100644 --- a/src/Interpreters/InterpreterShowSettingQuery.cpp +++ b/src/Interpreters/InterpreterShowSettingQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -29,5 +30,13 @@ BlockIO InterpreterShowSettingQuery::execute() return executeQuery(getRewrittenQuery(), getContext(), QueryFlags{ .internal = true }).second; } +void registerInterpreterShowSettingQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowSettingQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 0ca6578128d..51038aaca46 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -221,4 +222,14 @@ BlockIO InterpreterShowTablesQuery::execute() /// sort the output of SHOW otherwise (SELECT * FROM (SHOW ...) ORDER BY ...) is rejected) and 3. some /// SQL tests can take advantage of this. + +void registerInterpreterShowTablesQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterShowTablesQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index fc040e2af04..8d3ed095405 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -54,6 +55,7 @@ #include #include #include +#include #include #include #include @@ -108,6 +110,7 @@ namespace ActionLocks extern const StorageActionBlockType PartsMove; extern const StorageActionBlockType PullReplicationLog; extern const StorageActionBlockType Cleanup; + extern const StorageActionBlockType ViewRefresh; } @@ -165,6 +168,8 @@ AccessType getRequiredAccessType(StorageActionBlockType action_type) return AccessType::SYSTEM_PULLING_REPLICATION_LOG; else if (action_type == ActionLocks::Cleanup) return AccessType::SYSTEM_CLEANUP; + else if (action_type == ActionLocks::ViewRefresh) + return AccessType::SYSTEM_VIEWS; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown action type: {}", std::to_string(action_type)); } @@ -605,6 +610,23 @@ BlockIO InterpreterSystemQuery::execute() case Type::START_CLEANUP: startStopAction(ActionLocks::Cleanup, true); break; + case Type::START_VIEW: + case Type::START_VIEWS: + startStopAction(ActionLocks::ViewRefresh, true); + break; + case Type::STOP_VIEW: + case Type::STOP_VIEWS: + startStopAction(ActionLocks::ViewRefresh, false); + break; + case Type::REFRESH_VIEW: + getRefreshTask()->run(); + break; + case Type::CANCEL_VIEW: + getRefreshTask()->cancel(); + break; + case Type::TEST_VIEW: + getRefreshTask()->setFakeTime(query.fake_time_for_view); + break; case Type::DROP_REPLICA: dropReplica(query); break; @@ -1092,6 +1114,17 @@ void InterpreterSystemQuery::flushDistributed(ASTSystemQuery &) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "SYSTEM RESTART DISK is not supported"); } +RefreshTaskHolder InterpreterSystemQuery::getRefreshTask() +{ + auto ctx = getContext(); + ctx->checkAccess(AccessType::SYSTEM_VIEWS); + auto task = ctx->getRefreshSet().getTask(table_id); + if (!task) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Refreshable view {} doesn't exist", table_id.getNameForLogs()); + return task; +} + AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() const { @@ -1241,6 +1274,20 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_REPLICATION_QUEUES, query.getDatabase(), query.getTable()); break; } + case Type::REFRESH_VIEW: + case Type::START_VIEW: + case Type::START_VIEWS: + case Type::STOP_VIEW: + case Type::STOP_VIEWS: + case Type::CANCEL_VIEW: + case Type::TEST_VIEW: + { + if (!query.table) + required_access.emplace_back(AccessType::SYSTEM_VIEWS); + else + required_access.emplace_back(AccessType::SYSTEM_VIEWS, query.getDatabase(), query.getTable()); + break; + } case Type::DROP_REPLICA: case Type::DROP_DATABASE_REPLICA: { @@ -1332,4 +1379,13 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() return required_access; } +void registerInterpreterSystemQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterSystemQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 462449623d0..89de7402b4d 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -72,6 +73,8 @@ private: void flushDistributed(ASTSystemQuery & query); [[noreturn]] void restartDisk(String & name); + RefreshTaskHolder getRefreshTask(); + AccessRightsElements getRequiredAccessForDDLOnCluster() const; void startStopAction(StorageActionBlockType action_type, bool start); }; diff --git a/src/Interpreters/InterpreterTransactionControlQuery.cpp b/src/Interpreters/InterpreterTransactionControlQuery.cpp index a0a82121ba8..b1758013f18 100644 --- a/src/Interpreters/InterpreterTransactionControlQuery.cpp +++ b/src/Interpreters/InterpreterTransactionControlQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -135,4 +136,13 @@ BlockIO InterpreterTransactionControlQuery::executeSetSnapshot(ContextMutablePtr return {}; } +void registerInterpreterTransactionControlQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterTransactionControlQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterUndropQuery.cpp b/src/Interpreters/InterpreterUndropQuery.cpp index bdd72b6d3ea..8401c47df6b 100644 --- a/src/Interpreters/InterpreterUndropQuery.cpp +++ b/src/Interpreters/InterpreterUndropQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -72,4 +73,13 @@ AccessRightsElements InterpreterUndropQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::UNDROP_TABLE, undrop.getDatabase(), undrop.getTable()); return required_access; } + +void registerInterpreterUndropQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterUndropQuery", create_fn); +} } diff --git a/src/Interpreters/InterpreterUseQuery.cpp b/src/Interpreters/InterpreterUseQuery.cpp index b71f3a9cc1c..58be12927b9 100644 --- a/src/Interpreters/InterpreterUseQuery.cpp +++ b/src/Interpreters/InterpreterUseQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -16,4 +17,13 @@ BlockIO InterpreterUseQuery::execute() return {}; } +void registerInterpreterUseQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterUseQuery", create_fn); +} + } diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index e1af704a358..2b68c5d7a10 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -12,6 +12,7 @@ limitations under the License. */ #include #include #include +#include #include #include #include @@ -61,7 +62,7 @@ QueryPipelineBuilder InterpreterWatchQuery::buildQueryPipeline() storage = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); if (!storage) - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist.", table_id.getNameForLogs()); + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist.", table_id.getNameForLogs()); auto storage_name = storage->getName(); if (storage_name == "LiveView" @@ -103,4 +104,13 @@ QueryPipelineBuilder InterpreterWatchQuery::buildQueryPipeline() return pipeline; } +void registerInterpreterWatchQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterWatchQuery", create_fn); +} + } diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index bf50766c165..a6ea03f8a03 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1280,6 +1280,7 @@ void MutationsInterpreter::Source::read( VirtualColumns virtual_columns(std::move(required_columns), part); createReadFromPartStep( + MergeTreeSequentialSourceType::Mutation, plan, *data, storage_snapshot, part, std::move(virtual_columns.columns_to_read), apply_deleted_mask_, filter, context_, diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 18a25482b7f..cc3db726f01 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -97,7 +97,7 @@ FutureSetFromSubquery::FutureSetFromSubquery( String key, std::unique_ptr source_, StoragePtr external_table_, - FutureSetPtr external_table_set_, + std::shared_ptr external_table_set_, const Settings & settings, bool in_subquery_) : external_table(std::move(external_table_)) @@ -168,6 +168,24 @@ std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & conte return plan; } +void FutureSetFromSubquery::buildSetInplace(const ContextPtr & context) +{ + if (external_table_set) + external_table_set->buildSetInplace(context); + + auto plan = build(context); + + if (!plan) + return; + + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); +} + SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) { if (!context->getSettingsRef().use_index_for_in_with_subqueries) @@ -233,7 +251,7 @@ String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & return buf.str(); } -FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings) +FutureSetFromTuplePtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings) { auto from_tuple = std::make_shared(std::move(block), settings); const auto & set_types = from_tuple->getTypes(); @@ -247,7 +265,7 @@ FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Set return from_tuple; } -FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) +FutureSetFromStoragePtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) { auto from_storage = std::make_shared(std::move(set_)); auto [it, inserted] = sets_from_storage.emplace(key, from_storage); @@ -258,11 +276,11 @@ FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) return from_storage; } -FutureSetPtr PreparedSets::addFromSubquery( +FutureSetFromSubqueryPtr PreparedSets::addFromSubquery( const Hash & key, std::unique_ptr source, StoragePtr external_table, - FutureSetPtr external_table_set, + FutureSetFromSubqueryPtr external_table_set, const Settings & settings, bool in_subquery) { @@ -282,7 +300,7 @@ FutureSetPtr PreparedSets::addFromSubquery( return from_subquery; } -FutureSetPtr PreparedSets::addFromSubquery( +FutureSetFromSubqueryPtr PreparedSets::addFromSubquery( const Hash & key, QueryTreeNodePtr query_tree, const Settings & settings) @@ -300,7 +318,7 @@ FutureSetPtr PreparedSets::addFromSubquery( return from_subquery; } -FutureSetPtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) const +FutureSetFromTuplePtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) const { auto it = sets_from_tuple.find(key); if (it == sets_from_tuple.end()) diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 9f8bac9f71c..7178cff73b9 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -69,6 +69,8 @@ private: SetPtr set; }; +using FutureSetFromStoragePtr = std::shared_ptr; + /// Set from tuple is filled as well as set from storage. /// Additionally, it can be converted to set useful for PK. class FutureSetFromTuple final : public FutureSet @@ -86,6 +88,8 @@ private: SetKeyColumns set_key_columns; }; +using FutureSetFromTuplePtr = std::shared_ptr; + /// Set from subquery can be built inplace for PK or in CreatingSet step. /// If use_index_for_in_with_subqueries_max_values is reached, set for PK won't be created, /// but ordinary set would be created instead. @@ -96,7 +100,7 @@ public: String key, std::unique_ptr source_, StoragePtr external_table_, - FutureSetPtr external_table_set_, + std::shared_ptr external_table_set_, const Settings & settings, bool in_subquery_); @@ -110,6 +114,7 @@ public: SetPtr buildOrderedSetInplace(const ContextPtr & context) override; std::unique_ptr build(const ContextPtr & context); + void buildSetInplace(const ContextPtr & context); QueryTreeNodePtr detachQueryTree() { return std::move(query_tree); } void setQueryPlan(std::unique_ptr source_); @@ -119,7 +124,7 @@ public: private: SetAndKeyPtr set_and_key; StoragePtr external_table; - FutureSetPtr external_table_set; + std::shared_ptr external_table_set; std::unique_ptr source; QueryTreeNodePtr query_tree; @@ -130,6 +135,8 @@ private: // with new analyzer it's not a case }; +using FutureSetFromSubqueryPtr = std::shared_ptr; + /// Container for all the sets used in query. class PreparedSets { @@ -141,32 +148,32 @@ public: UInt64 operator()(const Hash & key) const { return key.low64 ^ key.high64; } }; - using SetsFromTuple = std::unordered_map>, Hashing>; - using SetsFromStorage = std::unordered_map, Hashing>; - using SetsFromSubqueries = std::unordered_map, Hashing>; + using SetsFromTuple = std::unordered_map, Hashing>; + using SetsFromStorage = std::unordered_map; + using SetsFromSubqueries = std::unordered_map; - FutureSetPtr addFromStorage(const Hash & key, SetPtr set_); - FutureSetPtr addFromTuple(const Hash & key, Block block, const Settings & settings); + FutureSetFromStoragePtr addFromStorage(const Hash & key, SetPtr set_); + FutureSetFromTuplePtr addFromTuple(const Hash & key, Block block, const Settings & settings); - FutureSetPtr addFromSubquery( + FutureSetFromSubqueryPtr addFromSubquery( const Hash & key, std::unique_ptr source, StoragePtr external_table, - FutureSetPtr external_table_set, + FutureSetFromSubqueryPtr external_table_set, const Settings & settings, bool in_subquery = false); - FutureSetPtr addFromSubquery( + FutureSetFromSubqueryPtr addFromSubquery( const Hash & key, QueryTreeNodePtr query_tree, const Settings & settings); - FutureSetPtr findTuple(const Hash & key, const DataTypes & types) const; - std::shared_ptr findStorage(const Hash & key) const; - std::shared_ptr findSubquery(const Hash & key) const; + FutureSetFromTuplePtr findTuple(const Hash & key, const DataTypes & types) const; + FutureSetFromStoragePtr findStorage(const Hash & key) const; + FutureSetFromSubqueryPtr findSubquery(const Hash & key) const; void markAsINSubquery(const Hash & key); - using Subqueries = std::vector>; + using Subqueries = std::vector; Subqueries getSubqueries() const; bool hasSubqueries() const { return !sets_from_subqueries.empty(); } diff --git a/src/Interpreters/RequiredSourceColumnsData.h b/src/Interpreters/RequiredSourceColumnsData.h index dd4e2dc3d68..501f6961efa 100644 --- a/src/Interpreters/RequiredSourceColumnsData.h +++ b/src/Interpreters/RequiredSourceColumnsData.h @@ -36,7 +36,6 @@ struct RequiredSourceColumnsData bool has_table_join = false; bool has_array_join = false; - bool visit_index_hint = false; bool addColumnAliasIfAny(const IAST & ast); void addColumnIdentifier(const ASTIdentifier & node); diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index c07d783788a..3971c8b58f4 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -72,11 +72,6 @@ void RequiredSourceColumnsMatcher::visit(const ASTPtr & ast, Data & data) } if (auto * t = ast->as()) { - /// "indexHint" is a special function for index analysis. - /// Everything that is inside it is not calculated. See KeyCondition - if (!data.visit_index_hint && t->name == "indexHint") - return; - data.addColumnAliasIfAny(*ast); visit(*t, ast, data); return; diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 8cf7dc39d97..31d4a4e51a4 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -255,6 +255,9 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values size_t total_number_of_rows_system = 0; size_t total_number_of_parts_system = 0; + size_t total_primary_key_bytes_memory = 0; + size_t total_primary_key_bytes_memory_allocated = 0; + for (const auto & db : databases) { /// Check if database can contain MergeTree tables @@ -293,6 +296,15 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values total_number_of_rows_system += rows; total_number_of_parts_system += parts; } + + // only fetch the parts which are in active state + auto all_parts = table_merge_tree->getDataPartsVectorForInternalUsage(); + + for (const auto & part : all_parts) + { + total_primary_key_bytes_memory += part->getIndexSizeInBytes(); + total_primary_key_bytes_memory_allocated += part->getIndexSizeInAllocatedBytes(); + } } if (StorageReplicatedMergeTree * table_replicated_merge_tree = typeid_cast(table.get())) @@ -347,11 +359,14 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values new_values["TotalPartsOfMergeTreeTables"] = { total_number_of_parts, "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." }; - new_values["NumberOfTablesSystem"] = { total_number_of_tables_system, "Total number of tables in the system database on the server stored in tables of MergeTree family."}; + new_values["NumberOfTablesSystem"] = { total_number_of_tables_system, "Total number of tables in the system database on the server stored in tables of MergeTree family." }; new_values["TotalBytesOfMergeTreeTablesSystem"] = { total_number_of_bytes_system, "Total amount of bytes (compressed, including data and indices) stored in tables of MergeTree family in the system database." }; new_values["TotalRowsOfMergeTreeTablesSystem"] = { total_number_of_rows_system, "Total amount of rows (records) stored in tables of MergeTree family in the system database." }; new_values["TotalPartsOfMergeTreeTablesSystem"] = { total_number_of_parts_system, "Total amount of data parts in tables of MergeTree family in the system database." }; + + new_values["TotalPrimaryKeyBytesInMemory"] = { total_primary_key_bytes_memory, "The total amount of memory (in bytes) used by primary key values (only takes active parts into account)." }; + new_values["TotalPrimaryKeyBytesInMemoryAllocated"] = { total_primary_key_bytes_memory_allocated, "The total amount of memory (in bytes) reserved for primary key values (only takes active parts into account)." }; } #if USE_NURAFT diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 9cbf24091e3..6ed3ff2f1e6 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -995,13 +995,12 @@ void TreeRewriterResult::collectSourceColumns(bool add_special) /// Calculate which columns are required to execute the expression. /// Then, delete all other columns from the list of available columns. /// After execution, columns will only contain the list of columns needed to read from the table. -bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint, bool no_throw) +bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool no_throw) { /// We calculate required_source_columns with source_columns modifications and swap them on exit required_source_columns = source_columns; RequiredSourceColumnsVisitor::Data columns_context; - columns_context.visit_index_hint = visit_index_hint; RequiredSourceColumnsVisitor(columns_context).visit(query); NameSet source_column_names; @@ -1385,7 +1384,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.window_function_asts = getWindowFunctions(query, *select_query); result.expressions_with_window_function = getExpressionsWithWindowFunctions(query); - result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key); + result.collectUsedColumns(query, true); if (!result.missed_subcolumns.empty()) { @@ -1422,7 +1421,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.aggregates = getAggregates(query, *select_query); result.window_function_asts = getWindowFunctions(query, *select_query); result.expressions_with_window_function = getExpressionsWithWindowFunctions(query); - result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key); + result.collectUsedColumns(query, true); } } @@ -1499,7 +1498,7 @@ TreeRewriterResultPtr TreeRewriter::analyze( else assertNoAggregates(query, "in wrong place"); - bool is_ok = result.collectUsedColumns(query, false, settings.query_plan_optimize_primary_key, no_throw); + bool is_ok = result.collectUsedColumns(query, false, no_throw); if (!is_ok) return {}; diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index 1858488afa3..205b4760423 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -88,7 +88,7 @@ struct TreeRewriterResult bool add_special = true); void collectSourceColumns(bool add_special); - bool collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint, bool no_throw = false); + bool collectUsedColumns(const ASTPtr & query, bool is_select, bool no_throw = false); Names requiredSourceColumns() const { return required_source_columns.getNames(); } const Names & requiredSourceColumnsForAccessCheck() const { return required_source_columns_before_expanding_alias_columns; } NameSet getArrayJoinSourceNameSet() const; diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 9486350a0f6..6b6054fdae3 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -200,8 +200,6 @@ public: Status prepare() override; private: - static Strings getChildrenAllowNoNode(const std::shared_ptr & zookeeper, const String & node_path); - static Block getSampleBlock(ContextPtr context_, bool hosts_to_wait); Strings getNewAndUpdate(const Strings & current_list_of_finished_hosts); @@ -228,7 +226,8 @@ private: NameSet waiting_hosts; /// hosts from task host list NameSet finished_hosts; /// finished hosts from host list NameSet ignoring_hosts; /// appeared hosts that are not in hosts list - Strings current_active_hosts; /// Hosts that were in active state at the last check + Strings current_active_hosts; /// Hosts that are currently executing the task + NameSet offline_hosts; /// Hosts that are not currently running size_t num_hosts_finished = 0; /// Save the first detected error and throw it at the end of execution @@ -237,7 +236,10 @@ private: Int64 timeout_seconds = 120; bool is_replicated_database = false; bool throw_on_timeout = true; + bool only_running_hosts = false; + bool timeout_exceeded = false; + bool stop_waiting_offline_hosts = false; }; @@ -310,12 +312,15 @@ DDLQueryStatusSource::DDLQueryStatusSource( , log(&Poco::Logger::get("DDLQueryStatusSource")) { auto output_mode = context->getSettingsRef().distributed_ddl_output_mode; - throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::NONE; + throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE + || output_mode == DistributedDDLOutputMode::NONE; if (hosts_to_wait) { waiting_hosts = NameSet(hosts_to_wait->begin(), hosts_to_wait->end()); is_replicated_database = true; + only_running_hosts = output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE || + output_mode == DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE; } else { @@ -377,6 +382,38 @@ Chunk DDLQueryStatusSource::generateChunkWithUnfinishedHosts() const return Chunk(std::move(columns), unfinished_hosts.size()); } +static NameSet getOfflineHosts(const String & node_path, const NameSet & hosts_to_wait, const ZooKeeperPtr & zookeeper, Poco::Logger * log) +{ + fs::path replicas_path; + if (node_path.ends_with('/')) + replicas_path = fs::path(node_path).parent_path().parent_path().parent_path() / "replicas"; + else + replicas_path = fs::path(node_path).parent_path().parent_path() / "replicas"; + + Strings paths; + Strings hosts_array; + for (const auto & host : hosts_to_wait) + { + hosts_array.push_back(host); + paths.push_back(replicas_path / host / "active"); + } + + NameSet offline; + auto res = zookeeper->tryGet(paths); + for (size_t i = 0; i < res.size(); ++i) + if (res[i].error == Coordination::Error::ZNONODE) + offline.insert(hosts_array[i]); + + if (offline.size() == hosts_to_wait.size()) + { + /// Avoid reporting that all hosts are offline + LOG_WARNING(log, "Did not find active hosts, will wait for all {} hosts. This should not happen often", offline.size()); + return {}; + } + + return offline; +} + Chunk DDLQueryStatusSource::generate() { bool all_hosts_finished = num_hosts_finished >= waiting_hosts.size(); @@ -398,7 +435,7 @@ Chunk DDLQueryStatusSource::generate() if (isCancelled()) return {}; - if (timeout_seconds >= 0 && watch.elapsedSeconds() > timeout_seconds) + if (stop_waiting_offline_hosts || (timeout_seconds >= 0 && watch.elapsedSeconds() > timeout_seconds)) { timeout_exceeded = true; @@ -406,7 +443,7 @@ Chunk DDLQueryStatusSource::generate() size_t num_active_hosts = current_active_hosts.size(); constexpr auto msg_format = "Watching task {} is executing longer than distributed_ddl_task_timeout (={}) seconds. " - "There are {} unfinished hosts ({} of them are currently active), " + "There are {} unfinished hosts ({} of them are currently executing the task), " "they are going to execute the query in background"; if (throw_on_timeout) { @@ -425,10 +462,7 @@ Chunk DDLQueryStatusSource::generate() return generateChunkWithUnfinishedHosts(); } - if (num_hosts_finished != 0 || try_number != 0) - { - sleepForMilliseconds(std::min(1000, 50 * (try_number + 1))); - } + sleepForMilliseconds(std::min(1000, 50 * try_number)); bool node_exists = false; Strings tmp_hosts; @@ -440,9 +474,21 @@ Chunk DDLQueryStatusSource::generate() retries_ctl.retryLoop([&]() { auto zookeeper = context->getZooKeeper(); - node_exists = zookeeper->exists(node_path); - tmp_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / node_to_wait); - tmp_active_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / "active"); + Strings paths = {String(fs::path(node_path) / node_to_wait), String(fs::path(node_path) / "active")}; + auto res = zookeeper->tryGetChildren(paths); + for (size_t i = 0; i < res.size(); ++i) + if (res[i].error != Coordination::Error::ZOK && res[i].error != Coordination::Error::ZNONODE) + throw Coordination::Exception::fromPath(res[i].error, paths[i]); + + if (res[0].error == Coordination::Error::ZNONODE) + node_exists = zookeeper->exists(node_path); + else + node_exists = true; + tmp_hosts = res[0].names; + tmp_active_hosts = res[1].names; + + if (only_running_hosts) + offline_hosts = getOfflineHosts(node_path, waiting_hosts, zookeeper, log); }); } @@ -460,6 +506,17 @@ Chunk DDLQueryStatusSource::generate() Strings new_hosts = getNewAndUpdate(tmp_hosts); ++try_number; + + if (only_running_hosts) + { + size_t num_finished_or_offline = 0; + for (const auto & host : waiting_hosts) + num_finished_or_offline += finished_hosts.contains(host) || offline_hosts.contains(host); + + if (num_finished_or_offline == waiting_hosts.size()) + stop_waiting_offline_hosts = true; + } + if (new_hosts.empty()) continue; @@ -470,7 +527,13 @@ Chunk DDLQueryStatusSource::generate() { ExecutionStatus status(-1, "Cannot obtain error message"); - if (node_to_wait == "finished") + /// Replicated database retries in case of error, it should not write error status. +#ifdef ABORT_ON_LOGICAL_ERROR + bool need_check_status = true; +#else + bool need_check_status = !is_replicated_database; +#endif + if (need_check_status) { String status_data; bool finished_exists = false; @@ -496,7 +559,6 @@ Chunk DDLQueryStatusSource::generate() if (status.code != 0 && !first_exception && context->getSettingsRef().distributed_ddl_output_mode != DistributedDDLOutputMode::NEVER_THROW) { - /// Replicated database retries in case of error, it should not write error status. if (is_replicated_database) throw Exception(ErrorCodes::LOGICAL_ERROR, "There was an error on {}: {} (probably it's a bug)", host_id, status.message); @@ -555,15 +617,6 @@ IProcessor::Status DDLQueryStatusSource::prepare() return ISource::prepare(); } -Strings DDLQueryStatusSource::getChildrenAllowNoNode(const std::shared_ptr & zookeeper, const String & node_path) -{ - Strings res; - Coordination::Error code = zookeeper->tryGetChildren(node_path, res); - if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) - throw Coordination::Exception::fromPath(code, node_path); - return res; -} - Strings DDLQueryStatusSource::getNewAndUpdate(const Strings & current_list_of_finished_hosts) { Strings diff; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 63804d2d86f..27986df1185 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1043,7 +1043,7 @@ static std::tuple executeQueryImpl( } } - interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); + interpreter = InterpreterFactory::instance().get(ast, context, SelectQueryOptions(stage).setInternal(internal)); const auto & query_settings = context->getSettingsRef(); if (context->getCurrentTransaction() && query_settings.throw_on_unsupported_query_inside_transaction) diff --git a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp index 40e2325e46e..6f84a60f2af 100644 --- a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp +++ b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp @@ -1,7 +1,9 @@ #include #include +#include #include "Processors/Executors/PullingPipelineExecutor.h" +#include #include #include #include @@ -28,9 +30,11 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); + registerInterpreters(); registerFunctions(); registerAggregateFunctions(); registerTableFunctions(); + registerDatabases(); registerStorages(); registerDictionaries(); registerDisks(/* global_skip_access_check= */ true); diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 541f9c6ee89..b2fd43c178c 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -1,3 +1,4 @@ +#include #include #include diff --git a/src/Interpreters/registerInterpreters.cpp b/src/Interpreters/registerInterpreters.cpp new file mode 100644 index 00000000000..481d0597a85 --- /dev/null +++ b/src/Interpreters/registerInterpreters.cpp @@ -0,0 +1,120 @@ +#include + +namespace DB +{ + +void registerInterpreterSelectQuery(InterpreterFactory & factory); +void registerInterpreterSelectQueryAnalyzer(InterpreterFactory & factory); +void registerInterpreterSelectWithUnionQuery(InterpreterFactory & factory); +void registerInterpreterSelectIntersectExceptQuery(InterpreterFactory & factory); +void registerInterpreterInsertQuery(InterpreterFactory & factory); +void registerInterpreterCreateQuery(InterpreterFactory & factory); +void registerInterpreterDropQuery(InterpreterFactory & factory); +void registerInterpreterUndropQuery(InterpreterFactory & factory); +void registerInterpreterRenameQuery(InterpreterFactory & factory); +void registerInterpreterShowTablesQuery(InterpreterFactory & factory); +void registerInterpreterShowColumnsQuery(InterpreterFactory & factory); +void registerInterpreterShowIndexesQuery(InterpreterFactory & factory); +void registerInterpreterShowSettingQuery(InterpreterFactory & factory); +void registerInterpreterShowEnginesQuery(InterpreterFactory & factory); +void registerInterpreterShowFunctionsQuery(InterpreterFactory & factory); +void registerInterpreterUseQuery(InterpreterFactory & factory); +void registerInterpreterSetQuery(InterpreterFactory & factory); +void registerInterpreterSetRoleQuery(InterpreterFactory & factory); +void registerInterpreterOptimizeQuery(InterpreterFactory & factory); +void registerInterpreterExistsQuery(InterpreterFactory & factory); +void registerInterpreterShowCreateQuery(InterpreterFactory & factory); +void registerInterpreterDescribeQuery(InterpreterFactory & factory); +void registerInterpreterDescribeCacheQuery(InterpreterFactory & factory); +void registerInterpreterExplainQuery(InterpreterFactory & factory); +void registerInterpreterShowProcesslistQuery(InterpreterFactory & factory); +void registerInterpreterAlterQuery(InterpreterFactory & factory); +void registerInterpreterAlterNamedCollectionQuery(InterpreterFactory & factory); +void registerInterpreterCheckQuery(InterpreterFactory & factory); +void registerInterpreterKillQueryQuery(InterpreterFactory & factory); +void registerInterpreterSystemQuery(InterpreterFactory & factory); +void registerInterpreterWatchQuery(InterpreterFactory & factory); +void registerInterpreterCreateUserQuery(InterpreterFactory & factory); +void registerInterpreterCreateRoleQuery(InterpreterFactory & factory); +void registerInterpreterCreateQuotaQuery(InterpreterFactory & factory); +void registerInterpreterCreateRowPolicyQuery(InterpreterFactory & factory); +void registerInterpreterCreateSettingsProfileQuery(InterpreterFactory & factory); +void registerInterpreterDropAccessEntityQuery(InterpreterFactory & factory); +void registerInterpreterMoveAccessEntityQuery(InterpreterFactory & factory); +void registerInterpreterDropNamedCollectionQuery(InterpreterFactory & factory); +void registerInterpreterGrantQuery(InterpreterFactory & factory); +void registerInterpreterShowCreateAccessEntityQuery(InterpreterFactory & factory); +void registerInterpreterShowGrantsQuery(InterpreterFactory & factory); +void registerInterpreterShowAccessEntitiesQuery(InterpreterFactory & factory); +void registerInterpreterShowAccessQuery(InterpreterFactory & factory); +void registerInterpreterShowPrivilegesQuery(InterpreterFactory & factory); +void registerInterpreterExternalDDLQuery(InterpreterFactory & factory); +void registerInterpreterTransactionControlQuery(InterpreterFactory & factory); +void registerInterpreterCreateFunctionQuery(InterpreterFactory & factory); +void registerInterpreterDropFunctionQuery(InterpreterFactory & factory); +void registerInterpreterCreateIndexQuery(InterpreterFactory & factory); +void registerInterpreterCreateNamedCollectionQuery(InterpreterFactory & factory); +void registerInterpreterDropIndexQuery(InterpreterFactory & factory); +void registerInterpreterBackupQuery(InterpreterFactory & factory); +void registerInterpreterDeleteQuery(InterpreterFactory & factory); + +void registerInterpreters() +{ + auto & factory = InterpreterFactory::instance(); + + registerInterpreterSelectQuery(factory); + registerInterpreterSelectQueryAnalyzer(factory); + registerInterpreterSelectWithUnionQuery(factory); + registerInterpreterSelectIntersectExceptQuery(factory); + registerInterpreterInsertQuery(factory); + registerInterpreterCreateQuery(factory); + registerInterpreterDropQuery(factory); + registerInterpreterUndropQuery(factory); + registerInterpreterRenameQuery(factory); + registerInterpreterShowTablesQuery(factory); + registerInterpreterShowColumnsQuery(factory); + registerInterpreterShowIndexesQuery(factory); + registerInterpreterShowSettingQuery(factory); + registerInterpreterShowEnginesQuery(factory); + registerInterpreterShowFunctionsQuery(factory); + registerInterpreterUseQuery(factory); + registerInterpreterSetQuery(factory); + registerInterpreterSetRoleQuery(factory); + registerInterpreterOptimizeQuery(factory); + registerInterpreterExistsQuery(factory); + registerInterpreterShowCreateQuery(factory); + registerInterpreterDescribeQuery(factory); + registerInterpreterDescribeCacheQuery(factory); + registerInterpreterExplainQuery(factory); + registerInterpreterShowProcesslistQuery(factory); + registerInterpreterAlterQuery(factory); + registerInterpreterAlterNamedCollectionQuery(factory); + registerInterpreterCheckQuery(factory); + registerInterpreterKillQueryQuery(factory); + registerInterpreterSystemQuery(factory); + registerInterpreterWatchQuery(factory); + registerInterpreterCreateUserQuery(factory); + registerInterpreterCreateRoleQuery(factory); + registerInterpreterCreateQuotaQuery(factory); + registerInterpreterCreateRowPolicyQuery(factory); + registerInterpreterCreateSettingsProfileQuery(factory); + registerInterpreterDropAccessEntityQuery(factory); + registerInterpreterMoveAccessEntityQuery(factory); + registerInterpreterDropNamedCollectionQuery(factory); + registerInterpreterGrantQuery(factory); + registerInterpreterShowCreateAccessEntityQuery(factory); + registerInterpreterShowGrantsQuery(factory); + registerInterpreterShowAccessEntitiesQuery(factory); + registerInterpreterShowAccessQuery(factory); + registerInterpreterShowPrivilegesQuery(factory); + registerInterpreterExternalDDLQuery(factory); + registerInterpreterTransactionControlQuery(factory); + registerInterpreterCreateFunctionQuery(factory); + registerInterpreterDropFunctionQuery(factory); + registerInterpreterCreateIndexQuery(factory); + registerInterpreterCreateNamedCollectionQuery(factory); + registerInterpreterDropIndexQuery(factory); + registerInterpreterBackupQuery(factory); + registerInterpreterDeleteQuery(factory); +} +} diff --git a/src/Interpreters/registerInterpreters.h b/src/Interpreters/registerInterpreters.h new file mode 100644 index 00000000000..9f0c3bbec22 --- /dev/null +++ b/src/Interpreters/registerInterpreters.h @@ -0,0 +1,6 @@ +#pragma once + +namespace DB +{ +void registerInterpreters(); +} diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index 89c4220ccdf..d75786f33b9 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #ifdef __SSE2__ #include @@ -155,8 +156,7 @@ void getBlockSortPermutationImpl(const Block & block, const SortDescription & de { size_t size = block.rows(); permutation.resize(size); - for (size_t i = 0; i < size; ++i) - permutation[i] = i; + iota(permutation.data(), size, IColumn::Permutation::value_type(0)); if (limit >= size) limit = 0; diff --git a/src/Interpreters/tests/gtest_filecache.cpp b/src/Interpreters/tests/gtest_filecache.cpp index 1005e6090b8..3e061db4f56 100644 --- a/src/Interpreters/tests/gtest_filecache.cpp +++ b/src/Interpreters/tests/gtest_filecache.cpp @@ -11,6 +11,7 @@ #include #include +#include #include #include #include @@ -788,7 +789,7 @@ TEST_F(FileCacheTest, writeBuffer) /// get random permutation of indexes std::vector indexes(data.size()); - std::iota(indexes.begin(), indexes.end(), 0); + iota(indexes.data(), indexes.size(), size_t(0)); std::shuffle(indexes.begin(), indexes.end(), rng); for (auto i : indexes) diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index ed9de6a46eb..84355817b2c 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -453,6 +453,12 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & << (settings.hilite ? hilite_none : ""); select->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::MODIFY_REFRESH) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY REFRESH " << settings.nl_or_ws + << (settings.hilite ? hilite_none : ""); + refresh->formatImpl(settings, state, frame); + } else if (type == ASTAlterCommand::LIVE_VIEW_REFRESH) { settings.ostr << (settings.hilite ? hilite_keyword : "") << "REFRESH " << (settings.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 77c540aed33..0b115537a6d 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -40,6 +40,7 @@ public: MODIFY_SETTING, RESET_SETTING, MODIFY_QUERY, + MODIFY_REFRESH, REMOVE_TTL, REMOVE_SAMPLE_BY, @@ -166,6 +167,9 @@ public: */ ASTPtr values; + /// For MODIFY REFRESH + ASTPtr refresh; + bool detach = false; /// true for DETACH PARTITION bool part = false; /// true for ATTACH PART, DROP DETACHED PART and MOVE diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 1562586bd93..9d5f0bcddbd 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include @@ -340,6 +339,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat formatOnCluster(settings); } + if (refresh_strategy) + { + settings.ostr << settings.nl_or_ws; + refresh_strategy->formatImpl(settings, state, frame); + } + if (to_table_id) { assert((is_materialized_view || is_window_view) && to_inner_uuid == UUIDHelpers::Nil); diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 28f5e05802b..49a0140625c 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -5,6 +5,7 @@ #include #include #include +#include #include namespace DB @@ -116,6 +117,7 @@ public: ASTExpressionList * dictionary_attributes_list = nullptr; /// attributes of ASTDictionary * dictionary = nullptr; /// dictionary definition (layout, primary key, etc.) + ASTRefreshStrategy * refresh_strategy = nullptr; // For CREATE MATERIALIZED VIEW ... REFRESH ... std::optional live_view_periodic_refresh; /// For CREATE LIVE VIEW ... WITH [PERIODIC] REFRESH ... bool is_watermark_strictly_ascending{false}; /// STRICTLY ASCENDING WATERMARK STRATEGY FOR WINDOW VIEW diff --git a/src/Parsers/ASTOptimizeQuery.cpp b/src/Parsers/ASTOptimizeQuery.cpp index 720c7699fb6..173310f7930 100644 --- a/src/Parsers/ASTOptimizeQuery.cpp +++ b/src/Parsers/ASTOptimizeQuery.cpp @@ -24,6 +24,9 @@ void ASTOptimizeQuery::formatQueryImpl(const FormatSettings & settings, FormatSt if (deduplicate) settings.ostr << (settings.hilite ? hilite_keyword : "") << " DEDUPLICATE" << (settings.hilite ? hilite_none : ""); + if (cleanup) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " CLEANUP" << (settings.hilite ? hilite_none : ""); + if (deduplicate_by_columns) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " BY " << (settings.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTOptimizeQuery.h b/src/Parsers/ASTOptimizeQuery.h index 584b2f38fe6..4c914c11912 100644 --- a/src/Parsers/ASTOptimizeQuery.h +++ b/src/Parsers/ASTOptimizeQuery.h @@ -21,10 +21,12 @@ public: bool deduplicate = false; /// Deduplicate by columns. ASTPtr deduplicate_by_columns; + /// Delete 'is_deleted' data + bool cleanup = false; /** Get the text that identifies this element. */ String getID(char delim) const override { - return "OptimizeQuery" + (delim + getDatabase()) + delim + getTable() + (final ? "_final" : "") + (deduplicate ? "_deduplicate" : ""); + return "OptimizeQuery" + (delim + getDatabase()) + delim + getTable() + (final ? "_final" : "") + (deduplicate ? "_deduplicate" : "")+ (cleanup ? "_cleanup" : ""); } ASTPtr clone() const override diff --git a/src/Parsers/ASTRefreshStrategy.cpp b/src/Parsers/ASTRefreshStrategy.cpp new file mode 100644 index 00000000000..2e0c6ee4638 --- /dev/null +++ b/src/Parsers/ASTRefreshStrategy.cpp @@ -0,0 +1,71 @@ +#include + +#include + +namespace DB +{ + +ASTPtr ASTRefreshStrategy::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (period) + res->set(res->period, period->clone()); + if (offset) + res->set(res->offset, offset->clone()); + if (spread) + res->set(res->spread, spread->clone()); + if (settings) + res->set(res->settings, settings->clone()); + if (dependencies) + res->set(res->dependencies, dependencies->clone()); + res->schedule_kind = schedule_kind; + return res; +} + +void ASTRefreshStrategy::formatImpl( + const IAST::FormatSettings & f_settings, IAST::FormatState & state, IAST::FormatStateStacked frame) const +{ + frame.need_parens = false; + + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << "REFRESH " << (f_settings.hilite ? hilite_none : ""); + using enum RefreshScheduleKind; + switch (schedule_kind) + { + case AFTER: + f_settings.ostr << "AFTER " << (f_settings.hilite ? hilite_none : ""); + period->formatImpl(f_settings, state, frame); + break; + case EVERY: + f_settings.ostr << "EVERY " << (f_settings.hilite ? hilite_none : ""); + period->formatImpl(f_settings, state, frame); + if (offset) + { + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " OFFSET " << (f_settings.hilite ? hilite_none : ""); + offset->formatImpl(f_settings, state, frame); + } + break; + default: + f_settings.ostr << (f_settings.hilite ? hilite_none : ""); + break; + } + + if (spread) + { + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " RANDOMIZE FOR " << (f_settings.hilite ? hilite_none : ""); + spread->formatImpl(f_settings, state, frame); + } + if (dependencies) + { + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " DEPENDS ON " << (f_settings.hilite ? hilite_none : ""); + dependencies->formatImpl(f_settings, state, frame); + } + if (settings) + { + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " SETTINGS " << (f_settings.hilite ? hilite_none : ""); + settings->formatImpl(f_settings, state, frame); + } +} + +} diff --git a/src/Parsers/ASTRefreshStrategy.h b/src/Parsers/ASTRefreshStrategy.h new file mode 100644 index 00000000000..ca248b76b40 --- /dev/null +++ b/src/Parsers/ASTRefreshStrategy.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +enum class RefreshScheduleKind : UInt8 +{ + UNKNOWN = 0, + AFTER, + EVERY +}; + +/// Strategy for MATERIALIZED VIEW ... REFRESH .. +class ASTRefreshStrategy : public IAST +{ +public: + ASTSetQuery * settings = nullptr; + ASTExpressionList * dependencies = nullptr; + ASTTimeInterval * period = nullptr; + ASTTimeInterval * offset = nullptr; + ASTTimeInterval * spread = nullptr; + RefreshScheduleKind schedule_kind{RefreshScheduleKind::UNKNOWN}; + + String getID(char) const override { return "Refresh strategy definition"; } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; +}; + +} diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 8e6100fe7b4..fc26f5dee1c 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -90,6 +90,13 @@ public: STOP_CLEANUP, START_CLEANUP, RESET_COVERAGE, + REFRESH_VIEW, + START_VIEW, + START_VIEWS, + STOP_VIEW, + STOP_VIEWS, + CANCEL_VIEW, + TEST_VIEW, END }; @@ -133,6 +140,10 @@ public: ServerType server_type; + /// For SYSTEM TEST VIEW (SET FAKE TIME
diff --git a/tests/config/config.d/reverse_dns_query_function.xml b/tests/config/config.d/reverse_dns_query_function.xml deleted file mode 100644 index 8cd7695b4d3..00000000000 --- a/tests/config/config.d/reverse_dns_query_function.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - \ No newline at end of file diff --git a/tests/integration/test_parallel_replicas_distributed_read_from_all/__init__.py b/tests/integration/test_asynchronous_metrics_pk_bytes_fields/__init__.py similarity index 100% rename from tests/integration/test_parallel_replicas_distributed_read_from_all/__init__.py rename to tests/integration/test_asynchronous_metrics_pk_bytes_fields/__init__.py diff --git a/tests/integration/test_asynchronous_metrics_pk_bytes_fields/configs/asynchronous_metrics_update_period_s.xml b/tests/integration/test_asynchronous_metrics_pk_bytes_fields/configs/asynchronous_metrics_update_period_s.xml new file mode 100644 index 00000000000..47e88730482 --- /dev/null +++ b/tests/integration/test_asynchronous_metrics_pk_bytes_fields/configs/asynchronous_metrics_update_period_s.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_asynchronous_metrics_pk_bytes_fields/test.py b/tests/integration/test_asynchronous_metrics_pk_bytes_fields/test.py new file mode 100644 index 00000000000..154048df35c --- /dev/null +++ b/tests/integration/test_asynchronous_metrics_pk_bytes_fields/test.py @@ -0,0 +1,132 @@ +import pytest +import time +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/asynchronous_metrics_update_period_s.xml"], +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def greater(a, b): + return b > a + + +def lesser(a, b): + return b < a + + +def query_until_condition(a, b, condition, retries=20, timeout=60, delay=0.5): + """ + + :param a: could be an input lambda that returns an int or just an int + :param b: could be an input lambda that returns an int or just an int + :param condition: lambda that returns a boolean after comparing a and b + :param retries: number of times to retry until the condition is met + :param timeout: time in seconds after which stop retrying + :param delay: time in seconds between each retry + :return: values of a and b (value post evaluation if lambda) + """ + retries_done = 0 + start_time = time.time() + while True: + res1 = a() if callable(a) else a + res2 = b() if callable(b) else b + if condition(res1, res2): + return res1, res2 + retries_done += 1 + if retries_done >= retries or (time.time() - start_time) > timeout: + return res1, res2 + time.sleep(delay) + + +def test_total_pk_bytes_in_memory_fields(started_cluster): + query_create = """CREATE TABLE test_pk_bytes + ( + a UInt64, + b UInt64 + ) + Engine=MergeTree() + ORDER BY a SETTINGS index_granularity=1""" + node.query(query_create) + + query_pk_bytes = "SELECT value FROM system.asynchronous_metrics WHERE metric = 'TotalPrimaryKeyBytesInMemory';" + query_pk_bytes_allocated = """SELECT value FROM system.asynchronous_metrics + WHERE metric = 'TotalPrimaryKeyBytesInMemoryAllocated';""" + + # query for metrics before inserting anything into the table + pk_bytes_before = int(node.query(query_pk_bytes).strip()) + pk_bytes_allocated_before = int(node.query(query_pk_bytes_allocated).strip()) + + # insert data into the table and select + node.query( + """INSERT INTO test_pk_bytes SELECT number + 20, number * 20 from numbers(1000000)""" + ) + + node.query("""SELECT * FROM test_pk_bytes where a > 1000000""") + + # functions to query primary key bytes used and allocated in memory + def res_pk_bytes(): + return int(node.query(query_pk_bytes).strip()) + + def res_pk_bytes_allocated(): + return int(node.query(query_pk_bytes_allocated).strip()) + + # query again after data insertion (make a reasonable amount of retries) + # metrics should be greater after inserting data + pk_bytes_before, pk_bytes_after = query_until_condition( + pk_bytes_before, res_pk_bytes, condition=greater + ) + assert pk_bytes_after > pk_bytes_before + + pk_bytes_allocated_before, pk_bytes_allocated_after = query_until_condition( + pk_bytes_allocated_before, res_pk_bytes_allocated, condition=greater + ) + assert pk_bytes_allocated_after > pk_bytes_allocated_before + + # insert some more data + node.query( + """INSERT INTO test_pk_bytes SELECT number + 100, number * 200 from numbers(1000000)""" + ) + node.query("""SELECT * FROM test_pk_bytes""") + + # query again and compare the metrics. + # metrics should be greater after inserting more data + pk_bytes_after, pk_bytes_after_2 = query_until_condition( + pk_bytes_after, res_pk_bytes, condition=greater + ) + assert pk_bytes_after_2 > pk_bytes_after + + pk_bytes_allocated_after, pk_bytes_allocated_after_2 = query_until_condition( + pk_bytes_allocated_after, res_pk_bytes_allocated, condition=greater + ) + assert pk_bytes_allocated_after_2 > pk_bytes_allocated_after + + # drop all the data + node.query("TRUNCATE table test_pk_bytes;") + + # query again and compare the metrics. + # metrics should be lesser after dropping some data + before_drop, after_drop = query_until_condition( + pk_bytes_after_2, res_pk_bytes, condition=lesser + ) + assert before_drop > after_drop + + before_drop, after_drop = query_until_condition( + pk_bytes_allocated_after_2, res_pk_bytes_allocated, condition=lesser + ) + assert before_drop > after_drop + + # finally drop the table + node.query("DROP table test_pk_bytes;") diff --git a/tests/integration/test_parallel_replicas_working_set/__init__.py b/tests/integration/test_attach_table_from_s3_plain_readonly/__init__.py similarity index 100% rename from tests/integration/test_parallel_replicas_working_set/__init__.py rename to tests/integration/test_attach_table_from_s3_plain_readonly/__init__.py diff --git a/tests/integration/test_attach_table_from_s3_plain_readonly/configs/config.xml b/tests/integration/test_attach_table_from_s3_plain_readonly/configs/config.xml new file mode 100644 index 00000000000..106edfc8351 --- /dev/null +++ b/tests/integration/test_attach_table_from_s3_plain_readonly/configs/config.xml @@ -0,0 +1,22 @@ + + + + + s3_plain + http://minio1:9001/root/data/disks/disk_s3_plain/ + minio + minio123 + true + + + + + +
+ disk_s3_plain_readonly +
+
+
+
+
+
diff --git a/tests/integration/test_attach_table_from_s3_plain_readonly/configs/settings.xml b/tests/integration/test_attach_table_from_s3_plain_readonly/configs/settings.xml new file mode 100644 index 00000000000..3e6d615557d --- /dev/null +++ b/tests/integration/test_attach_table_from_s3_plain_readonly/configs/settings.xml @@ -0,0 +1,12 @@ + + + + 1 + + + + + default + + + diff --git a/tests/integration/test_attach_table_from_s3_plain_readonly/test.py b/tests/integration/test_attach_table_from_s3_plain_readonly/test.py new file mode 100644 index 00000000000..15ba934e621 --- /dev/null +++ b/tests/integration/test_attach_table_from_s3_plain_readonly/test.py @@ -0,0 +1,112 @@ +import re +import os +import logging +import pytest + +from helpers.cluster import ClickHouseCluster +from minio.error import S3Error +from pathlib import Path + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", + main_configs=["configs/config.xml"], + user_configs=["configs/settings.xml"], + with_zookeeper=True, + with_minio=True, + stay_alive=True, + macros={"shard": 1, "replica": 1}, +) + +node2 = cluster.add_instance( + "node2", + main_configs=["configs/config.xml"], + user_configs=["configs/settings.xml"], + with_zookeeper=True, + with_minio=True, + stay_alive=True, + macros={"shard": 1, "replica": 2}, +) + +uuid_regex = re.compile("[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}") + + +def upload_to_minio(minio_client, bucket_name, local_path, minio_path=""): + local_path = Path(local_path) + for root, _, files in os.walk(local_path): + for file in files: + local_file_path = Path(root) / file + minio_object_name = minio_path + str( + local_file_path.relative_to(local_path) + ) + + try: + with open(local_file_path, "rb") as data: + file_stat = os.stat(local_file_path) + minio_client.put_object( + bucket_name, minio_object_name, data, file_stat.st_size + ) + logging.info(f"Uploaded {local_file_path} to {minio_object_name}") + except S3Error as e: + logging.error(f"Error uploading {local_file_path}: {e}") + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_attach_table_from_s3_plain_readonly(started_cluster): + # Create an atomic DB with mergetree sample data + node1.query( + """ + create database local_db; + + create table local_db.test_table (num UInt32) engine=MergeTree() order by num; + + insert into local_db.test_table (*) Values (5) + """ + ) + + assert int(node1.query("select num from local_db.test_table limit 1")) == 5 + + # Copy local MergeTree data into minio bucket + table_data_path = os.path.join(node1.path, f"database/store") + minio = cluster.minio_client + upload_to_minio( + minio, cluster.minio_bucket, table_data_path, "data/disks/disk_s3_plain/store/" + ) + + # Drop the non-replicated table, we don't need it anymore + table_uuid = node1.query( + "SELECT uuid FROM system.tables WHERE database='local_db' AND table='test_table'" + ).strip() + node1.query("drop table local_db.test_table SYNC;") + + # Create a replicated database + node1.query( + "create database s3_plain_test_db ENGINE = Replicated('/test/s3_plain_test_db', 'shard1', 'replica1');" + ) + node2.query( + "create database s3_plain_test_db ENGINE = Replicated('/test/s3_plain_test_db', 'shard1', 'replica2');" + ) + + # Create a MergeTree table at one node, by attaching the merge tree data + node1.query( + f""" + attach table s3_plain_test_db.test_table UUID '{table_uuid}' (num UInt32) + engine=MergeTree() + order by num + settings storage_policy = 's3_plain_readonly' + """ + ) + + # Check that both nodes can query and get result. + assert int(node1.query("select num from s3_plain_test_db.test_table limit 1")) == 5 + assert int(node2.query("select num from s3_plain_test_db.test_table limit 1")) == 5 diff --git a/tests/integration/test_reverse_dns_query/__init__.py b/tests/integration/test_ddl_config_hostname/__init__.py similarity index 100% rename from tests/integration/test_reverse_dns_query/__init__.py rename to tests/integration/test_ddl_config_hostname/__init__.py diff --git a/tests/integration/test_ddl_config_hostname/configs/remote_servers.xml b/tests/integration/test_ddl_config_hostname/configs/remote_servers.xml new file mode 100644 index 00000000000..8c6a507951d --- /dev/null +++ b/tests/integration/test_ddl_config_hostname/configs/remote_servers.xml @@ -0,0 +1,19 @@ + + + + + true + + node1 + 9000 + + + + + + 1 + + + node1 + + diff --git a/tests/integration/test_ddl_config_hostname/test.py b/tests/integration/test_ddl_config_hostname/test.py new file mode 100644 index 00000000000..724e766c9dc --- /dev/null +++ b/tests/integration/test_ddl_config_hostname/test.py @@ -0,0 +1,76 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", + main_configs=["configs/remote_servers.xml"], + with_zookeeper=True, + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_ddl_queue_delete_add_replica(started_cluster): + # Some query started on the cluster, then we deleted some unfinished node + # and added a new node to the cluster. Considering that there are less + # finished nodes than expected and we can't resolve deleted node's hostname + # the queue will be stuck on a new node. + # inside allows us to simply discard deleted + # node's hostname by simple comparison without trying to resolve it. + + node1.query( + "create table hostname_change on cluster test_cluster (n int) engine=Log" + ) + + # There's no easy way to change hostname of a container, so let's update values in zk + query_znode = node1.query( + "select max(name) from system.zookeeper where path='/clickhouse/task_queue/ddl'" + )[:-1] + + value = ( + node1.query( + f"select value from system.zookeeper where path='/clickhouse/task_queue/ddl' and name='{query_znode}' format TSVRaw" + )[:-1] + .replace( + "hosts: ['node1:9000']", "hosts: ['finished_node:9000','deleted_node:9000']" + ) + .replace("initiator: node1:9000", "initiator: finished_node:9000") + .replace("\\'", "#") + .replace("'", "\\'") + .replace("\n", "\\n") + .replace("#", "\\'") + ) + + finished_znode = node1.query( + f"select name from system.zookeeper where path='/clickhouse/task_queue/ddl/{query_znode}/finished' and name like '%node1%'" + )[:-1] + + node1.query( + f"insert into system.zookeeper (name, path, value) values ('{query_znode}', '/clickhouse/task_queue/ddl', '{value}')" + ) + started_cluster.get_kazoo_client("zoo1").delete( + f"/clickhouse/task_queue/ddl/{query_znode}/finished/{finished_znode}" + ) + + finished_znode = finished_znode.replace("node1", "finished_node") + + node1.query( + f"insert into system.zookeeper (name, path, value) values ('{finished_znode}', '/clickhouse/task_queue/ddl/{query_znode}/finished', '0\\n')" + ) + + node1.restart_clickhouse(kill=True) + + node1.query( + "create table hostname_change2 on cluster test_cluster (n int) engine=Log" + ) diff --git a/tests/integration/test_filesystem_cache/config.d/storage_conf_2.xml b/tests/integration/test_filesystem_cache/config.d/storage_conf_2.xml new file mode 100644 index 00000000000..a068d7b954c --- /dev/null +++ b/tests/integration/test_filesystem_cache/config.d/storage_conf_2.xml @@ -0,0 +1,24 @@ + + + + + local_blob_storage + / + + + cache + hdd_blob + /cache1/ + 1Mi + 1 + + + cache + hdd_blob + /cache1/ + 1Mi + 1 + + + + diff --git a/tests/integration/test_filesystem_cache/test.py b/tests/integration/test_filesystem_cache/test.py index 3a6a1ef76eb..ab1bc4e4344 100644 --- a/tests/integration/test_filesystem_cache/test.py +++ b/tests/integration/test_filesystem_cache/test.py @@ -21,6 +21,12 @@ def cluster(): ], stay_alive=True, ) + cluster.add_instance( + "node_caches_with_same_path", + main_configs=[ + "config.d/storage_conf_2.xml", + ], + ) logging.info("Starting cluster...") cluster.start() @@ -87,3 +93,104 @@ def test_parallel_cache_loading_on_startup(cluster, node_name): ) node.query("SELECT * FROM test FORMAT Null") assert count == int(node.query("SELECT count() FROM test")) + + +@pytest.mark.parametrize("node_name", ["node"]) +def test_caches_with_the_same_configuration(cluster, node_name): + node = cluster.instances[node_name] + cache_path = "cache1" + + node.query(f"SYSTEM DROP FILESYSTEM CACHE;") + for table in ["test", "test2"]: + node.query( + f""" + DROP TABLE IF EXISTS {table} SYNC; + + CREATE TABLE {table} (key UInt32, value String) + Engine=MergeTree() + ORDER BY value + SETTINGS disk = disk( + type = cache, + name = {table}, + path = '{cache_path}', + disk = 'hdd_blob', + max_file_segment_size = '1Ki', + boundary_alignment = '1Ki', + cache_on_write_operations=1, + max_size = '1Mi'); + + SET enable_filesystem_cache_on_write_operations=1; + INSERT INTO {table} SELECT * FROM generateRandom('a Int32, b String') + LIMIT 1000; + """ + ) + + size = int( + node.query( + "SELECT value FROM system.metrics WHERE name = 'FilesystemCacheSize'" + ) + ) + assert ( + node.query( + "SELECT cache_name, sum(size) FROM system.filesystem_cache GROUP BY cache_name ORDER BY cache_name" + ).strip() + == f"test\t{size}\ntest2\t{size}" + ) + + table = "test3" + assert ( + "Found more than one cache configuration with the same path, but with different cache settings" + in node.query_and_get_error( + f""" + DROP TABLE IF EXISTS {table} SYNC; + + CREATE TABLE {table} (key UInt32, value String) + Engine=MergeTree() + ORDER BY value + SETTINGS disk = disk( + type = cache, + name = {table}, + path = '{cache_path}', + disk = 'hdd_blob', + max_file_segment_size = '1Ki', + boundary_alignment = '1Ki', + cache_on_write_operations=0, + max_size = '2Mi'); + """ + ) + ) + + +@pytest.mark.parametrize("node_name", ["node_caches_with_same_path"]) +def test_caches_with_the_same_configuration_2(cluster, node_name): + node = cluster.instances[node_name] + cache_path = "cache1" + + node.query(f"SYSTEM DROP FILESYSTEM CACHE;") + for table in ["cache1", "cache2"]: + node.query( + f""" + DROP TABLE IF EXISTS {table} SYNC; + + CREATE TABLE {table} (key UInt32, value String) + Engine=MergeTree() + ORDER BY value + SETTINGS disk = '{table}'; + + SET enable_filesystem_cache_on_write_operations=1; + INSERT INTO {table} SELECT * FROM generateRandom('a Int32, b String') + LIMIT 1000; + """ + ) + + size = int( + node.query( + "SELECT value FROM system.metrics WHERE name = 'FilesystemCacheSize'" + ) + ) + assert ( + node.query( + "SELECT cache_name, sum(size) FROM system.filesystem_cache GROUP BY cache_name ORDER BY cache_name" + ).strip() + == f"cache1\t{size}\ncache2\t{size}" + ) diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index c8a0ee541e2..a86a1208f49 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -188,7 +188,7 @@ def test_grant_all_on_table(): instance.query("SHOW GRANTS FOR B") == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, " "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 DROP REPLICA, SYSTEM SYNC REPLICA, " + "SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, 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 FLUSH DISTRIBUTED, dictGet ON test.table TO B\n" ) instance.query("REVOKE ALL ON test.table FROM B", user="A") diff --git a/tests/integration/test_kafka_bad_messages/test.py b/tests/integration/test_kafka_bad_messages/test.py index 1633f230f83..954b6042305 100644 --- a/tests/integration/test_kafka_bad_messages/test.py +++ b/tests/integration/test_kafka_bad_messages/test.py @@ -294,7 +294,7 @@ def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20): ]: print(format_name) - kafka_create_topic(admin_client, f"{format_name}_err") + kafka_create_topic(admin_client, f"{format_name}_parsing_err") instance.query( f""" @@ -305,7 +305,7 @@ def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20): CREATE TABLE kafka_{format_name} (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{format_name}_err', + kafka_topic_list = '{format_name}_parsing_err', kafka_group_name = '{format_name}', kafka_format = '{format_name}', kafka_num_consumers = 1; @@ -316,16 +316,18 @@ def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20): ) kafka_produce( - kafka_cluster, f"{format_name}_err", ["qwertyuiop", "asdfghjkl", "zxcvbnm"] + kafka_cluster, + f"{format_name}_parsing_err", + ["qwertyuiop", "asdfghjkl", "zxcvbnm"], ) - expected_result = """avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_err, partition: 0, offset: 0)\\'|1|1|1|default|kafka_Avro -Cannot parse input: expected \\'{\\' before: \\'qwertyuiop\\': while parsing Kafka message (topic: JSONEachRow_err, partition: 0, offset: 0|1|1|1|default|kafka_JSONEachRow + expected_result = """avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_parsing_err, partition: 0, offset: 0)\\'|1|1|1|default|kafka_Avro +Cannot parse input: expected \\'{\\' before: \\'qwertyuiop\\': (at row 1)\\n: while parsing Kafka message (topic: JSONEachRow_parsing_err, partition:|1|1|1|default|kafka_JSONEachRow """ # filter out stacktrace in exceptions.text[1] because it is hardly stable enough result_system_kafka_consumers = instance.query_with_retry( """ - SELECT substr(exceptions.text[1], 1, 131), length(exceptions.text) > 1 AND length(exceptions.text) < 15, length(exceptions.time) > 1 AND length(exceptions.time) < 15, abs(dateDiff('second', exceptions.time[1], now())) < 40, database, table FROM system.kafka_consumers WHERE table in('kafka_Avro', 'kafka_JSONEachRow') ORDER BY table, assignments.partition_id[1] + SELECT substr(exceptions.text[1], 1, 139), length(exceptions.text) > 1 AND length(exceptions.text) < 15, length(exceptions.time) > 1 AND length(exceptions.time) < 15, abs(dateDiff('second', exceptions.time[1], now())) < 40, database, table FROM system.kafka_consumers WHERE table in('kafka_Avro', 'kafka_JSONEachRow') ORDER BY table, assignments.partition_id[1] """, retry_count=max_retries, sleep_time=1, @@ -338,7 +340,7 @@ Cannot parse input: expected \\'{\\' before: \\'qwertyuiop\\': while parsing Kaf "Avro", "JSONEachRow", ]: - kafka_delete_topic(admin_client, f"{format_name}_err") + kafka_delete_topic(admin_client, f"{format_name}_parsing_err") def test_bad_messages_to_mv(kafka_cluster, max_retries=20): diff --git a/tests/queries/0_stateless/02948_optimize_cleanup_as_noop.reference b/tests/integration/test_parallel_replicas_all_marks_read/__init__.py similarity index 100% rename from tests/queries/0_stateless/02948_optimize_cleanup_as_noop.reference rename to tests/integration/test_parallel_replicas_all_marks_read/__init__.py diff --git a/tests/integration/test_parallel_replicas_all_marks_read/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_all_marks_read/configs/remote_servers.xml new file mode 100644 index 00000000000..1ad562334f5 --- /dev/null +++ b/tests/integration/test_parallel_replicas_all_marks_read/configs/remote_servers.xml @@ -0,0 +1,32 @@ + + + + + + node0 + 9000 + + + node1 + 9000 + + + node2 + 9000 + + + node3 + 9000 + + + node4 + 9000 + + + node5 + 9000 + + + + + diff --git a/tests/integration/test_parallel_replicas_all_marks_read/test.py b/tests/integration/test_parallel_replicas_all_marks_read/test.py new file mode 100644 index 00000000000..7776ccb0c09 --- /dev/null +++ b/tests/integration/test_parallel_replicas_all_marks_read/test.py @@ -0,0 +1,156 @@ +import json +import pytest + +from helpers.cluster import ClickHouseCluster +from random import randint + +cluster = ClickHouseCluster(__file__) +cluster_name = "parallel_replicas_with_unavailable_nodes" + +nodes = [ + cluster.add_instance( + f"node{num}", main_configs=["configs/remote_servers.xml"], with_zookeeper=True + ) + for num in range(3) +] + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def _create_tables(table_name, table_size, index_granularity): + for num in range(len(nodes)): + nodes[num].query(f"DROP TABLE IF EXISTS {table_name}") + + nodes[num].query( + f""" + CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) + Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', '{num}') + ORDER BY (key) + SETTINGS index_granularity = {index_granularity} + """ + ) + + nodes[0].query( + f""" + INSERT INTO {table_name} + SELECT number, toString(number) FROM numbers_mt({table_size}) + """ + ) + + +def _create_query(query_tmpl, table_name): + rand_set = [randint(0, 500) for i in range(42)] + return query_tmpl.format(table_name=table_name, rand_set=rand_set) + + +def _get_result_without_parallel_replicas(query): + return nodes[0].query( + query, + settings={ + "allow_experimental_parallel_reading_from_replicas": 0, + }, + ) + + +def _get_result_with_parallel_replicas( + query, query_id, cluster_name, parallel_replicas_mark_segment_size +): + return nodes[0].query( + query, + settings={ + "allow_experimental_parallel_reading_from_replicas": 2, + "max_parallel_replicas": 6, + "cluster_for_parallel_replicas": f"{cluster_name}", + "parallel_replicas_mark_segment_size": parallel_replicas_mark_segment_size, + "query_id": query_id, + }, + ) + + +def _get_expected_amount_of_marks_to_read(query): + return json.loads( + nodes[0].query( + f""" + EXPLAIN ESTIMATE + {query} + FORMAT JSONEachRow + """ + ) + )["marks"] + + +def _get_number_of_marks_read_by_replicas(query_id): + nodes[0].query("SYSTEM FLUSH LOGS") + return ( + nodes[0] + .query( + f""" + SELECT sum( + ProfileEvents['ParallelReplicasReadAssignedMarks'] + + ProfileEvents['ParallelReplicasReadUnassignedMarks'] + + ProfileEvents['ParallelReplicasReadAssignedForStealingMarks'] + ) + FROM system.query_log + WHERE query_id = '{query_id}' + """ + ) + .strip() + ) + + +@pytest.mark.parametrize( + "query_tmpl", + [ + "SELECT sum(cityHash64(*)) FROM {table_name}", + "SELECT sum(cityHash64(*)) FROM {table_name} WHERE intDiv(key, 100) IN {rand_set}", + ], +) +@pytest.mark.parametrize( + "table_size", + [1000, 10000, 100000], +) +@pytest.mark.parametrize( + "index_granularity", + [10, 100], +) +@pytest.mark.parametrize( + "parallel_replicas_mark_segment_size", + [1, 10], +) +def test_number_of_marks_read( + start_cluster, + query_tmpl, + table_size, + index_granularity, + parallel_replicas_mark_segment_size, +): + if nodes[0].is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers (too slow)") + + table_name = f"tbl_{len(query_tmpl)}_{cluster_name}_{table_size}_{index_granularity}_{parallel_replicas_mark_segment_size}" + _create_tables(table_name, table_size, index_granularity) + + if "where" in query_tmpl.lower(): + # We need all the replicas to see the same state of parts to make sure that index analysis will pick the same amount of marks for reading + # regardless of which replica's state will be chosen as the working set. This should became redundant once we start to always use initiator's snapshot. + nodes[0].query(f"OPTIMIZE TABLE {table_name} FINAL", settings={"alter_sync": 2}) + for node in nodes: + node.query(f"SYSTEM SYNC REPLICA {table_name} STRICT") + + query = _create_query(query_tmpl, table_name) + query_id = f"{table_name}_{randint(0, 1e9)}" + + assert _get_result_with_parallel_replicas( + query, query_id, cluster_name, parallel_replicas_mark_segment_size + ) == _get_result_without_parallel_replicas(query) + + assert _get_number_of_marks_read_by_replicas( + query_id + ) == _get_expected_amount_of_marks_to_read(query) diff --git a/tests/integration/test_parallel_replicas_distributed_read_from_all/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_distributed_read_from_all/configs/remote_servers.xml deleted file mode 100644 index 02a315479f8..00000000000 --- a/tests/integration/test_parallel_replicas_distributed_read_from_all/configs/remote_servers.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - - true - - n1 - 9000 - - - n2 - 9000 - - - n3 - 9000 - - - - - - diff --git a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py deleted file mode 100644 index 8af7bb12595..00000000000 --- a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py +++ /dev/null @@ -1,156 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -nodes = [ - cluster.add_instance( - f"n{i}", main_configs=["configs/remote_servers.xml"], with_zookeeper=True - ) - for i in (1, 2, 3) -] - - -@pytest.fixture(scope="module", autouse=True) -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def create_tables(cluster, table_name): - """create replicated tables in special way - - each table is populated by equal number of rows - - fetches are disabled, so each replica will have different set of rows - which enforce parallel replicas read from each replica - """ - - # create replicated tables - for node in nodes: - node.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - - nodes[0].query( - f"""CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') - ORDER BY (key)""" - ) - nodes[1].query( - f"""CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') - ORDER BY (key)""" - ) - nodes[2].query( - f"""CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') - ORDER BY (key)""" - ) - # stop merges - nodes[0].query(f"system stop merges {table_name}") - nodes[1].query(f"system stop merges {table_name}") - nodes[2].query(f"system stop merges {table_name}") - # stop fetches - nodes[0].query(f"system stop fetches {table_name}") - nodes[1].query(f"system stop fetches {table_name}") - nodes[2].query(f"system stop fetches {table_name}") - - # create distributed table - nodes[0].query(f"DROP TABLE IF EXISTS {table_name}_d SYNC") - nodes[0].query( - f""" - CREATE TABLE {table_name}_d AS {table_name} - Engine=Distributed( - {cluster}, - currentDatabase(), - {table_name}, - rand() - ) - """ - ) - - # populate data, equal number of rows for each replica - nodes[0].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(10)", - settings={"distributed_foreground_insert": 1}, - ) - nodes[0].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(10, 10)", - settings={"distributed_foreground_insert": 1}, - ) - nodes[1].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(20, 10)", - settings={"distributed_foreground_insert": 1}, - ) - nodes[1].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(30, 10)", - settings={"distributed_foreground_insert": 1}, - ) - nodes[2].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(40, 10)", - settings={"distributed_foreground_insert": 1}, - ) - nodes[2].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(50, 10)", - settings={"distributed_foreground_insert": 1}, - ) - - return "60\t0\t59\t1770\n" - - -@pytest.mark.parametrize( - "prefer_localhost_replica", - [ - pytest.param(0), - pytest.param(1), - ], -) -def test_read_equally_from_each_replica(start_cluster, prefer_localhost_replica): - """create and populate table in special way (see create_table()), - so parallel replicas will read equal number of rows from each replica - """ - - cluster = "test_single_shard_multiple_replicas" - table_name = "test_table" - expected_result = create_tables(cluster, table_name) - - # parallel replicas - assert ( - nodes[0].query( - f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", - settings={ - "allow_experimental_parallel_reading_from_replicas": 2, - "prefer_localhost_replica": prefer_localhost_replica, - "max_parallel_replicas": 3, - }, - ) - == expected_result - ) - - # check logs for coordinator statistic - for n in nodes: - n.query("SYSTEM FLUSH LOGS") - - # each replica has 2 distinct parts (non-intersecting with another replicas), - # each part less then index granularity, therefore 2 marks for each replica to handle - coordinator_statistic = "replica 0 - {requests: 3 marks: 2}; replica 1 - {requests: 3 marks: 2}; replica 2 - {requests: 3 marks: 2}" - assert ( - nodes[0].contains_in_log(coordinator_statistic) - or nodes[1].contains_in_log(coordinator_statistic) - or nodes[2].contains_in_log(coordinator_statistic) - ) - - # w/o parallel replicas - # start fetches back, otherwise the result will be not as expected - nodes[0].query(f"system start fetches {table_name}") - nodes[1].query(f"system start fetches {table_name}") - nodes[2].query(f"system start fetches {table_name}") - # ensure that replica in sync before querying it to get stable result - nodes[0].query(f"system start merges {table_name}") - nodes[0].query(f"system sync replica {table_name}") - assert ( - nodes[0].query( - f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", - settings={ - "allow_experimental_parallel_reading_from_replicas": 0, - }, - ) - == expected_result - ) diff --git a/tests/integration/test_parallel_replicas_working_set/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_working_set/configs/remote_servers.xml deleted file mode 100644 index 02a315479f8..00000000000 --- a/tests/integration/test_parallel_replicas_working_set/configs/remote_servers.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - - true - - n1 - 9000 - - - n2 - 9000 - - - n3 - 9000 - - - - - - diff --git a/tests/integration/test_parallel_replicas_working_set/test.py b/tests/integration/test_parallel_replicas_working_set/test.py deleted file mode 100644 index 0ede9d9b1a5..00000000000 --- a/tests/integration/test_parallel_replicas_working_set/test.py +++ /dev/null @@ -1,140 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -nodes = [ - cluster.add_instance( - f"n{i}", main_configs=["configs/remote_servers.xml"], with_zookeeper=True - ) - for i in (1, 2, 3) -] - - -@pytest.fixture(scope="module", autouse=True) -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def create_tables(cluster, table_name, node_with_covering_part): - # create replicated tables - for node in nodes: - node.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - - nodes[0].query( - f"""CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') - ORDER BY (key)""" - ) - nodes[1].query( - f"""CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') - ORDER BY (key)""" - ) - nodes[2].query( - f"""CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') - ORDER BY (key)""" - ) - # stop merges to keep original parts - # stop fetches to keep only parts created on the nodes - for i in (0, 1, 2): - if i != node_with_covering_part: - nodes[i].query(f"system stop fetches {table_name}") - nodes[i].query(f"system stop merges {table_name}") - - # populate data, equal number of rows for each replica - nodes[0].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(10)", - ) - nodes[0].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(10, 10)" - ) - nodes[1].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(20, 10)" - ) - nodes[1].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(30, 10)" - ) - nodes[2].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(40, 10)" - ) - nodes[2].query( - f"INSERT INTO {table_name} SELECT number, number FROM numbers(50, 10)" - ) - nodes[node_with_covering_part].query(f"system sync replica {table_name}") - nodes[node_with_covering_part].query(f"optimize table {table_name}") - - # check we have expected set of parts - expected_active_parts = "" - if node_with_covering_part == 0: - expected_active_parts = ( - "all_0_5_1\nall_2_2_0\nall_3_3_0\nall_4_4_0\nall_5_5_0\n" - ) - - if node_with_covering_part == 1: - expected_active_parts = ( - "all_0_0_0\nall_0_5_1\nall_1_1_0\nall_4_4_0\nall_5_5_0\n" - ) - - if node_with_covering_part == 2: - expected_active_parts = ( - "all_0_0_0\nall_0_5_1\nall_1_1_0\nall_2_2_0\nall_3_3_0\n" - ) - - assert ( - nodes[0].query( - f"select distinct name from clusterAllReplicas({cluster}, system.parts) where table='{table_name}' and active order by name" - ) - == expected_active_parts - ) - - -@pytest.mark.parametrize("node_with_covering_part", [0, 1, 2]) -def test_covering_part_in_announcement(start_cluster, node_with_covering_part): - """create and populate table in special way (see create_table()), - node_with_covering_part contains all parts merged into one, - other nodes contain only parts which are result of insert via the node - """ - - cluster = "test_single_shard_multiple_replicas" - table_name = "test_table" - create_tables(cluster, table_name, node_with_covering_part) - - # query result can be one of the following outcomes - # (1) query result if parallel replicas working set contains all_0_5_1 - expected_full_result = "60\t0\t59\t1770\n" - expected_results = {expected_full_result} - - # (2) query result if parallel replicas working set DOESN'T contain all_0_5_1 - if node_with_covering_part == 0: - expected_results.add("40\t20\t59\t1580\n") - if node_with_covering_part == 1: - expected_results.add("40\t0\t59\t1180\n") - if node_with_covering_part == 2: - expected_results.add("40\t0\t39\t780\n") - - # parallel replicas - result = nodes[0].query( - f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}", - settings={ - "allow_experimental_parallel_reading_from_replicas": 2, - "prefer_localhost_replica": 0, - "max_parallel_replicas": 3, - "use_hedged_requests": 0, - "cluster_for_parallel_replicas": cluster, - }, - ) - assert result in expected_results - - # w/o parallel replicas - assert ( - nodes[node_with_covering_part].query( - f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}", - settings={ - "allow_experimental_parallel_reading_from_replicas": 0, - }, - ) - == expected_full_result - ) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 3ced82ebb57..1fc3fe37044 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -507,7 +507,7 @@ def test_alters_from_different_replicas(started_cluster): settings = {"distributed_ddl_task_timeout": 5} assert ( - "There are 1 unfinished hosts (0 of them are currently active)" + "There are 1 unfinished hosts (0 of them are currently executing the task" in competing_node.query_and_get_error( "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN Added0 UInt32;", settings=settings, diff --git a/tests/integration/test_replicated_database_cluster_groups/test.py b/tests/integration/test_replicated_database_cluster_groups/test.py index b14581c1fe6..647626d8014 100644 --- a/tests/integration/test_replicated_database_cluster_groups/test.py +++ b/tests/integration/test_replicated_database_cluster_groups/test.py @@ -96,7 +96,7 @@ def test_cluster_groups(started_cluster): main_node_2.stop_clickhouse() settings = {"distributed_ddl_task_timeout": 5} assert ( - "There are 1 unfinished hosts (0 of them are currently active)" + "There are 1 unfinished hosts (0 of them are currently executing the task)" in main_node_1.query_and_get_error( "CREATE TABLE cluster_groups.table_2 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);", settings=settings, diff --git a/tests/integration/test_reverse_dns_query/configs/config.xml b/tests/integration/test_reverse_dns_query/configs/config.xml deleted file mode 100644 index 5ce55afa2a7..00000000000 --- a/tests/integration/test_reverse_dns_query/configs/config.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_reverse_dns_query/configs/listen_host.xml b/tests/integration/test_reverse_dns_query/configs/listen_host.xml deleted file mode 100644 index 9c27c612f63..00000000000 --- a/tests/integration/test_reverse_dns_query/configs/listen_host.xml +++ /dev/null @@ -1,5 +0,0 @@ - - :: - 0.0.0.0 - 1 - diff --git a/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml b/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml deleted file mode 100644 index 35d0a07c6a6..00000000000 --- a/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_reverse_dns_query/coredns_config/Corefile b/tests/integration/test_reverse_dns_query/coredns_config/Corefile deleted file mode 100644 index 3edf37dafa5..00000000000 --- a/tests/integration/test_reverse_dns_query/coredns_config/Corefile +++ /dev/null @@ -1,8 +0,0 @@ -. { - hosts /example.com { - reload "20ms" - fallthrough - } - forward . 127.0.0.11 - log -} diff --git a/tests/integration/test_reverse_dns_query/coredns_config/example.com b/tests/integration/test_reverse_dns_query/coredns_config/example.com deleted file mode 100644 index 6c6e4cbee2e..00000000000 --- a/tests/integration/test_reverse_dns_query/coredns_config/example.com +++ /dev/null @@ -1 +0,0 @@ -filled in runtime, but needs to exist in order to be volume mapped in docker diff --git a/tests/integration/test_reverse_dns_query/test.py b/tests/integration/test_reverse_dns_query/test.py deleted file mode 100644 index 00c3956f74f..00000000000 --- a/tests/integration/test_reverse_dns_query/test.py +++ /dev/null @@ -1,74 +0,0 @@ -import pytest -import socket -from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check -from time import sleep -import os - -DOCKER_COMPOSE_PATH = get_docker_compose_path() -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - -cluster = ClickHouseCluster(__file__) - -ch_server = cluster.add_instance( - "clickhouse-server", - with_coredns=True, - main_configs=[ - "configs/config.xml", - "configs/reverse_dns_function.xml", - "configs/listen_host.xml", - ], -) - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def check_ptr_record(ip, hostname): - try: - host, aliaslist, ipaddrlist = socket.gethostbyaddr(ip) - if hostname.lower() == host.lower(): - return True - except socket.herror: - pass - return False - - -def setup_dns_server(ip): - domains_string = "test.example.com" - example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' - run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) - - # DNS server takes time to reload the configuration. - for try_num in range(10): - if all(check_ptr_record(ip, host) for host in domains_string.split()): - break - sleep(1) - - -def setup_ch_server(dns_server_ip): - ch_server.exec_in_container( - (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) - ) - ch_server.exec_in_container( - (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) - ) - ch_server.query("SYSTEM DROP DNS CACHE") - - -def test_reverse_dns_query(started_cluster): - dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) - random_ipv6 = "4ae8:fa0f:ee1d:68c5:0b76:1b79:7ae6:1549" # https://commentpicker.com/ip-address-generator.php - setup_dns_server(random_ipv6) - setup_ch_server(dns_server_ip) - - for _ in range(0, 200): - response = ch_server.query(f"select reverseDNSQuery('{random_ipv6}')") - assert response == "['test.example.com']\n" diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index d5f8d04e258..9a75dc50d61 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -463,7 +463,9 @@ def test_schema_inference(started_cluster, format_version): create_iceberg_table(instance, TABLE_NAME, format) - res = instance.query(f"DESC {TABLE_NAME} FORMAT TSVRaw") + res = instance.query( + f"DESC {TABLE_NAME} FORMAT TSVRaw", settings={"print_pretty_type_names": 0} + ) expected = TSV( [ ["intC", "Nullable(Int32)"], diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/key_value_message.capnp b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/key_value_message.capnp new file mode 100644 index 00000000000..19b7029dba3 --- /dev/null +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/key_value_message.capnp @@ -0,0 +1,7 @@ +@0x99f75f775fe63dae; + +struct Message +{ + key @0 : UInt64; + value @1 : UInt64; +} \ No newline at end of file diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/key_value_message.proto b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/key_value_message.proto new file mode 100644 index 00000000000..7c9d4ad0850 --- /dev/null +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/key_value_message.proto @@ -0,0 +1,6 @@ +syntax = "proto3"; + +message Message { + uint64 key = 1; + uint64 value = 1; +} diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index b1191af60b7..2176b0151ff 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4834,6 +4834,103 @@ JSONExtractString(rdkafka_stat, 'type'): consumer kafka_delete_topic(admin_client, topic) +def test_formats_errors(kafka_cluster): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + for format_name in [ + "Template", + "Regexp", + "TSV", + "TSVWithNamesAndTypes", + "TSKV", + "CSV", + "CSVWithNames", + "CSVWithNamesAndTypes", + "CustomSeparated", + "CustomSeparatedWithNames", + "CustomSeparatedWithNamesAndTypes", + "Values", + "JSON", + "JSONEachRow", + "JSONStringsEachRow", + "JSONCompactEachRow", + "JSONCompactEachRowWithNamesAndTypes", + "JSONObjectEachRow", + "Avro", + "RowBinary", + "RowBinaryWithNamesAndTypes", + "MsgPack", + "JSONColumns", + "JSONCompactColumns", + "JSONColumnsWithMetadata", + "BSONEachRow", + "Native", + "Arrow", + "Parquet", + "ORC", + "JSONCompactColumns", + "Npy", + "ParquetMetadata", + "CapnProto", + "Protobuf", + "ProtobufSingle", + "ProtobufList", + "DWARF", + "HiveText", + "MySQLDump", + ]: + kafka_create_topic(admin_client, format_name) + table_name = f"kafka_{format_name}" + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.{table_name}; + + CREATE TABLE test.{table_name} (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{format_name}', + kafka_group_name = '{format_name}', + kafka_format = '{format_name}', + kafka_max_rows_per_message = 5, + format_template_row='template_row.format', + format_regexp='id: (.+?)', + input_format_with_names_use_header=0, + format_schema='key_value_message:Message'; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.{table_name}; + """ + ) + + kafka_produce( + kafka_cluster, + format_name, + ["Broken message\nBroken message\nBroken message\n"], + ) + + attempt = 0 + num_errors = 0 + while attempt < 200: + num_errors = int( + instance.query( + f"SELECT length(exceptions.text) from system.kafka_consumers where database = 'test' and table = '{table_name}'" + ) + ) + if num_errors > 0: + break + attempt += 1 + + assert num_errors > 0 + + kafka_delete_topic(admin_client, format_name) + instance.query(f"DROP TABLE test.{table_name}") + instance.query("DROP TABLE test.view") + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index b1163a549b1..b83c095a7a6 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -919,4 +919,6 @@ def test_drop_table(started_cluster): node.query(f"DROP TABLE {table_name} SYNC") assert node.contains_in_log( f"StorageS3Queue ({table_name}): Table is being dropped" + ) or node.contains_in_log( + f"StorageS3Queue ({table_name}): Shutdown was called, stopping sync" ) diff --git a/tests/integration/test_throttling/configs/server_overrides.xml b/tests/integration/test_throttling/configs/dynamic_overrides.xml similarity index 100% rename from tests/integration/test_throttling/configs/server_overrides.xml rename to tests/integration/test_throttling/configs/dynamic_overrides.xml diff --git a/tests/integration/test_throttling/configs/server_backups.xml b/tests/integration/test_throttling/configs/static_overrides.xml similarity index 83% rename from tests/integration/test_throttling/configs/server_backups.xml rename to tests/integration/test_throttling/configs/static_overrides.xml index a8c43f8beaf..9f3bad2f882 100644 --- a/tests/integration/test_throttling/configs/server_backups.xml +++ b/tests/integration/test_throttling/configs/static_overrides.xml @@ -31,4 +31,7 @@ default /backups/ + + 1000000 + 1000000
diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 04d02cc859d..c53c2bb1ddf 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -34,8 +34,8 @@ node = cluster.add_instance( "node", stay_alive=True, main_configs=[ - "configs/server_backups.xml", - "configs/server_overrides.xml", + "configs/static_overrides.xml", + "configs/dynamic_overrides.xml", "configs/ssl.xml", ], user_configs=[ @@ -64,7 +64,7 @@ def revert_config(): [ "bash", "-c", - f"echo '' > /etc/clickhouse-server/config.d/server_overrides.xml", + f"echo '' > /etc/clickhouse-server/config.d/dynamic_overrides.xml", ] ) node.exec_in_container( @@ -96,7 +96,7 @@ def node_update_config(mode, setting, value=None): if mode is None: return if mode == "server": - config_path = "/etc/clickhouse-server/config.d/server_overrides.xml" + config_path = "/etc/clickhouse-server/config.d/dynamic_overrides.xml" config_content = f""" <{setting}>{value} """ @@ -430,3 +430,32 @@ def test_write_throttling(policy, mode, setting, value, should_took): ) _, took = elapsed(node.query, f"insert into data select * from numbers(1e6)") assert_took(took, should_took) + + +def test_max_mutations_bandwidth_for_server(): + node.query( + """ + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9; + """ + ) + node.query("insert into data select * from numbers(1e6)") + _, took = elapsed( + node.query, + "alter table data update key = -key where 1 settings mutations_sync = 1", + ) + # reading 1e6*8 bytes with 1M/s bandwith should take (8-1)/1=7 seconds + assert_took(took, 7) + + +def test_max_merges_bandwidth_for_server(): + node.query( + """ + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9; + """ + ) + node.query("insert into data select * from numbers(1e6)") + _, took = elapsed(node.query, "optimize table data final") + # reading 1e6*8 bytes with 1M/s bandwith should take (8-1)/1=7 seconds + assert_took(took, 7) diff --git a/tests/integration/test_user_valid_until/test.py b/tests/integration/test_user_valid_until/test.py index e34771e55a9..d6d5bf8b18e 100644 --- a/tests/integration/test_user_valid_until/test.py +++ b/tests/integration/test_user_valid_until/test.py @@ -78,7 +78,9 @@ def test_details(started_cluster): # 2. Time only is not supported node.query("CREATE USER user_details_time_only VALID UNTIL '22:03:40'") + until_year = datetime.today().strftime("%Y") + assert ( node.query("SHOW CREATE USER user_details_time_only") - == "CREATE USER user_details_time_only VALID UNTIL \\'2023-01-01 22:03:40\\'\n" + == f"CREATE USER user_details_time_only VALID UNTIL \\'{until_year}-01-01 22:03:40\\'\n" ) diff --git a/tests/integration/test_wrong_db_or_table_name/test.py b/tests/integration/test_wrong_db_or_table_name/test.py index 641501eac84..a5096d80ca9 100644 --- a/tests/integration/test_wrong_db_or_table_name/test.py +++ b/tests/integration/test_wrong_db_or_table_name/test.py @@ -57,6 +57,31 @@ def test_drop_wrong_database_name(start): node.query("DROP DATABASE test;") +def test_database_engine_name(start): + # test with a valid database engine + node.query( + """ + CREATE DATABASE test_atomic ENGINE = Atomic; + CREATE TABLE test_atomic.table_test_atomic (i Int64) ENGINE = MergeTree() ORDER BY i; + INSERT INTO test_atomic.table_test_atomic SELECT 1; + """ + ) + assert 1 == int(node.query("SELECT * FROM test_atomic.table_test_atomic".strip())) + # test with a invalid database engine + with pytest.raises( + QueryRuntimeException, + match="DB::Exception: Unknown database engine Atomic123. Maybe you meant: \\['Atomic'\\].", + ): + node.query("CREATE DATABASE test_atomic123 ENGINE = Atomic123;") + + node.query( + """ + DROP TABLE test_atomic.table_test_atomic; + DROP DATABASE test_atomic; + """ + ) + + def test_wrong_table_name(start): node.query( """ diff --git a/tests/performance/README.md b/tests/performance/README.md index f554e96203b..289ecaba034 100644 --- a/tests/performance/README.md +++ b/tests/performance/README.md @@ -18,5 +18,5 @@ TODO @akuzm ``` pip3 install clickhouse_driver scipy -../../docker/test/performance-comparison/perf.py --runs 1 insert_parallel.xml +../../tests/performance/scripts/perf.py --runs 1 insert_parallel.xml ``` diff --git a/tests/performance/agg_functions_min_max_any.xml b/tests/performance/agg_functions_min_max_any.xml index 2926a5ed3c8..f8469244643 100644 --- a/tests/performance/agg_functions_min_max_any.xml +++ b/tests/performance/agg_functions_min_max_any.xml @@ -87,4 +87,9 @@ select any(FromTag) from hits_100m_single where FromTag != '' group by intHash32(UserID) % {group_scale} FORMAT Null select anyHeavy(FromTag) from hits_100m_single where FromTag != '' group by intHash32(UserID) % {group_scale} FORMAT Null + +select min((WatchID, CounterID)) from hits_100m_single FORMAT Null +select max((WatchID, CounterID)) from hits_100m_single FORMAT Null +select any((WatchID, CounterID)) from hits_100m_single FORMAT Null +select anyHeavy((WatchID, CounterID)) from hits_100m_single FORMAT Null diff --git a/tests/performance/bounding_ratio.xml b/tests/performance/bounding_ratio.xml index e3a15f90013..ed0b25848df 100644 --- a/tests/performance/bounding_ratio.xml +++ b/tests/performance/bounding_ratio.xml @@ -1,4 +1,4 @@ - SELECT boundingRatio(number, number) FROM numbers(100000000) - SELECT (argMax(number, number) - argMin(number, number)) / (max(number) - min(number)) FROM numbers(100000000) + SELECT boundingRatio(number, number) FROM numbers(30000000) + SELECT (argMax(number, number) - argMin(number, number)) / (max(number) - min(number)) FROM numbers(30000000) diff --git a/tests/performance/decimal_parse.xml b/tests/performance/decimal_parse.xml index 19e940b13df..966363d6fec 100644 --- a/tests/performance/decimal_parse.xml +++ b/tests/performance/decimal_parse.xml @@ -1,3 +1,3 @@ - SELECT count() FROM zeros(10000000) WHERE NOT ignore(toDecimal32OrZero(toString(rand() % 10000), 5)) + SELECT count() FROM zeros(3000000) WHERE NOT ignore(toDecimal32OrZero(toString(rand() % 10000), 5)) diff --git a/tests/performance/group_by_consecutive_keys.xml b/tests/performance/group_by_consecutive_keys.xml new file mode 100644 index 00000000000..c5c885d2bb6 --- /dev/null +++ b/tests/performance/group_by_consecutive_keys.xml @@ -0,0 +1,8 @@ + + SELECT toUInt64(intDiv(number, 1000000)) AS n, count(), sum(number) FROM numbers(10000000) GROUP BY n FORMAT Null + SELECT toString(intDiv(number, 1000000)) AS n, count(), sum(number) FROM numbers(10000000) GROUP BY n FORMAT Null + SELECT toUInt64(intDiv(number, 1000000)) AS n, count(), uniq(number) FROM numbers(10000000) GROUP BY n FORMAT Null + SELECT toUInt64(intDiv(number, 100000)) AS n, count(), sum(number) FROM numbers(10000000) GROUP BY n FORMAT Null + SELECT toUInt64(intDiv(number, 100)) AS n, count(), sum(number) FROM numbers(10000000) GROUP BY n FORMAT Null + SELECT toUInt64(intDiv(number, 10)) AS n, count(), sum(number) FROM numbers(10000000) GROUP BY n FORMAT Null + diff --git a/tests/performance/group_by_fixed_keys.xml b/tests/performance/group_by_fixed_keys.xml index a64208eb3de..d74b65ad47a 100644 --- a/tests/performance/group_by_fixed_keys.xml +++ b/tests/performance/group_by_fixed_keys.xml @@ -11,7 +11,7 @@ create table group_by_fk(a UInt32, b UInt32, c LowCardinality(UInt32), d Nullable(UInt32), e UInt64, f UInt64, g UInt64, h LowCardinality(UInt64), i Nullable(UInt64)) engine=MergeTree order by tuple() - insert into group_by_fk select number, number, number % 10000, number % 2 == 0 ? number : Null, number, number, number, number % 10000, number % 2 == 0 ? number : Null from numbers_mt(3e7) + insert into group_by_fk select number, number, number % 10000, number % 2 == 0 ? number : Null, number, number, number, number % 10000, number % 2 == 0 ? number : Null from numbers_mt(1e7) settings max_insert_threads=8 select a, b from group_by_fk group by a, b format Null diff --git a/tests/performance/group_by_sundy_li.xml b/tests/performance/group_by_sundy_li.xml index 694fafcbbcd..46f659d9cc0 100644 --- a/tests/performance/group_by_sundy_li.xml +++ b/tests/performance/group_by_sundy_li.xml @@ -16,10 +16,10 @@ ORDER BY (d, n) - insert into a select '2000-01-01', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(100000000) - insert into a select '2000-01-02', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(100000000) - insert into a select '2000-01-03', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(100000000) - insert into a select '2000-01-04', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(100000000) + insert into a select '2000-01-01', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(10000000) + insert into a select '2000-01-02', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(10000000) + insert into a select '2000-01-03', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(10000000) + insert into a select '2000-01-04', ['aa','bb','cc','dd'][number % 4 + 1], number from numbers_mt(10000000) OPTIMIZE TABLE a FINAL diff --git a/tests/performance/hashed_dictionary.xml b/tests/performance/hashed_dictionary.xml index e9038e694c6..b9de02a70e0 100644 --- a/tests/performance/hashed_dictionary.xml +++ b/tests/performance/hashed_dictionary.xml @@ -82,7 +82,6 @@ elements_count 5000000 - 7500000 @@ -90,16 +89,14 @@ WITH rand64() % toUInt64({elements_count}) as key SELECT dictGet('default.simple_key_hashed_dictionary', {column_name}, key) - FROM system.numbers - LIMIT {elements_count} + FROM numbers_mt({elements_count}) FORMAT Null; WITH rand64() % toUInt64({elements_count}) as key SELECT dictHas('default.simple_key_hashed_dictionary', key) - FROM system.numbers - LIMIT {elements_count} + FROM numbers_mt({elements_count}) FORMAT Null; @@ -111,16 +108,14 @@ WITH (rand64() % toUInt64({elements_count}), toString(rand64() % toUInt64({elements_count}))) as key SELECT dictGet('default.complex_key_hashed_dictionary', {column_name}, key) - FROM system.numbers - LIMIT {elements_count} + FROM numbers_mt({elements_count}) FORMAT Null; WITH (rand64() % toUInt64({elements_count}), toString(rand64() % toUInt64({elements_count}))) as key SELECT dictHas('default.complex_key_hashed_dictionary', key) - FROM system.numbers - LIMIT {elements_count} + FROM numbers_mt({elements_count}) FORMAT Null; diff --git a/tests/performance/hashjoin_with_large_output.xml b/tests/performance/hashjoin_with_large_output.xml new file mode 100644 index 00000000000..f4b61c15f82 --- /dev/null +++ b/tests/performance/hashjoin_with_large_output.xml @@ -0,0 +1,64 @@ + + + 16 + 10G + + + + + settings + + join_algorithm='hash' + join_algorithm='grace_hash' + + + + + + create table test_left + ( + k1 String, + v1 String + ) + engine = Memory(); + + + create table test_right + ( + k1 String, + v1 String, + v2 String, + v3 String, + v4 String, + v5 String, + v6 String, + v7 String, + v8 String, + v9 String + ) + engine = Memory(); + + insert into test_left SELECT toString(number % 20), toString(number) from system.numbers limit 10000; + + insert into test_right + SELECT + toString(number % 20), + toString(number * 10000), + toString(number * 10000 + 1), + toString(number * 10000 + 2), + toString(number * 10000 + 3), + toString(number * 10000 + 4), + toString(number * 10000 + 5), + toString(number * 10000 + 6), + toString(number * 10000 + 7), + toString(number * 10000 + 8) + from system.numbers limit 10000; + + + + select * from test_left all inner join test_right on test_left.k1 = test_right.k1 SETTINGS {settings} format Null + + + DROP TABLE IF EXISTS test_left + DROP TABLE IF EXISTS test_right + diff --git a/tests/performance/join_used_flags.xml b/tests/performance/join_used_flags.xml index cd2073ee106..1bb994f7be2 100644 --- a/tests/performance/join_used_flags.xml +++ b/tests/performance/join_used_flags.xml @@ -1,6 +1,6 @@ CREATE TABLE test_join_used_flags (i64 Int64, i32 Int32) ENGINE = Memory - INSERT INTO test_join_used_flags SELECT number AS i64, rand32() AS i32 FROM numbers(20000000) + INSERT INTO test_join_used_flags SELECT number AS i64, rand32() AS i32 FROM numbers_mt(1500000) SELECT l.i64, r.i64, l.i32, r.i32 FROM test_join_used_flags l RIGHT JOIN test_join_used_flags r USING i64 format Null DROP TABLE IF EXISTS test_join_used_flags diff --git a/tests/performance/min_max_index.xml b/tests/performance/min_max_index.xml index b7b5d4fb991..518696144e2 100644 --- a/tests/performance/min_max_index.xml +++ b/tests/performance/min_max_index.xml @@ -1,7 +1,7 @@ CREATE TABLE index_test (z UInt32, INDEX i_x (mortonDecode(2, z).1) TYPE minmax, INDEX i_y (mortonDecode(2, z).2) TYPE minmax) ENGINE = MergeTree ORDER BY z - INSERT INTO index_test SELECT number FROM numbers(0x100000000) WHERE rand() % 3 = 1 + INSERT INTO index_test SELECT number * 10 FROM numbers_mt(toUInt64(0x100000000 / 10)) SETTINGS max_insert_threads=8 = 20000 AND mortonDecode(2, z).1 <= 20100 AND mortonDecode(2, z).2 >= 10000 AND mortonDecode(2, z).2 <= 10100 diff --git a/tests/performance/polymorphic_parts_l.xml b/tests/performance/polymorphic_parts_l.xml index d2ae9417bf7..66c5b73caa8 100644 --- a/tests/performance/polymorphic_parts_l.xml +++ b/tests/performance/polymorphic_parts_l.xml @@ -25,8 +25,8 @@ - INSERT INTO hits_wide(UserID) SELECT rand() FROM numbers(100000) - INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(100000) + INSERT INTO hits_wide(UserID) SELECT rand() FROM numbers(100000) + INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(100000) INSERT INTO hits_buffer(UserID) SELECT rand() FROM numbers(100000) DROP TABLE IF EXISTS hits_wide diff --git a/tests/performance/polymorphic_parts_m.xml b/tests/performance/polymorphic_parts_m.xml index 54a81def55e..0a44038ffbd 100644 --- a/tests/performance/polymorphic_parts_m.xml +++ b/tests/performance/polymorphic_parts_m.xml @@ -25,8 +25,8 @@ - INSERT INTO hits_wide(UserID) SELECT rand() FROM numbers(10000) - INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(100000) + INSERT INTO hits_wide(UserID) SELECT rand() FROM numbers(10000) + INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(10000) INSERT INTO hits_buffer(UserID) SELECT rand() FROM numbers(10000) DROP TABLE IF EXISTS hits_wide diff --git a/tests/performance/reinterpret_as.xml b/tests/performance/reinterpret_as.xml index dbf6df160ed..d05ef3bb038 100644 --- a/tests/performance/reinterpret_as.xml +++ b/tests/performance/reinterpret_as.xml @@ -19,7 +19,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -38,7 +38,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -57,7 +57,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -76,7 +76,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -95,7 +95,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(10000000) + FROM numbers_mt(5000000) SETTINGS max_threads = 8 FORMAT Null @@ -115,7 +115,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -134,7 +134,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -153,7 +153,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -172,7 +172,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(200000000) + FROM numbers_mt(100000000) SETTINGS max_threads = 8 FORMAT Null @@ -191,7 +191,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(100000000) + FROM numbers_mt(50000000) SETTINGS max_threads = 8 FORMAT Null @@ -210,7 +210,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(10000000) + FROM numbers_mt(5000000) SETTINGS max_threads = 8 FORMAT Null @@ -230,7 +230,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(20000000) + FROM numbers_mt(10000000) SETTINGS max_threads = 8 FORMAT Null @@ -249,7 +249,7 @@ toInt256(number) as d, toString(number) as f, toFixedString(f, 20) as g - FROM numbers_mt(100000000) + FROM numbers_mt(50000000) SETTINGS max_threads = 8 FORMAT Null diff --git a/tests/performance/scripts/compare.sh b/tests/performance/scripts/compare.sh index 454b8903e5a..6d1a271355e 100755 --- a/tests/performance/scripts/compare.sh +++ b/tests/performance/scripts/compare.sh @@ -1220,15 +1220,23 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') 0 test_duration_ms, 'https://s3.amazonaws.com/clickhouse-test-reports/$PR_TO_TEST/$SHA_TO_TEST/${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX}/report.html#fail1' report_url union all - select test || ' #' || toString(query_index), 'slower' test_status, 0 test_duration_ms, - 'https://s3.amazonaws.com/clickhouse-test-reports/$PR_TO_TEST/$SHA_TO_TEST/${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX}/report.html#changes-in-performance.' - || test || '.' || toString(query_index) report_url - from queries where changed_fail != 0 and diff > 0 + select + test || ' #' || toString(query_index) || '::' || test_desc_.1 test_name, + 'slower' test_status, + test_desc_.2 test_duration_ms, + 'https://s3.amazonaws.com/clickhouse-test-reports/$PR_TO_TEST/$SHA_TO_TEST/${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX}/report.html#changes-in-performance.' || test || '.' || toString(query_index) report_url + from queries + array join map('old', left, 'new', right) as test_desc_ + where changed_fail != 0 and diff > 0 union all - select test || ' #' || toString(query_index), 'unstable' test_status, 0 test_duration_ms, - 'https://s3.amazonaws.com/clickhouse-test-reports/$PR_TO_TEST/$SHA_TO_TEST/${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX}/report.html#unstable-queries.' - || test || '.' || toString(query_index) report_url - from queries where unstable_fail != 0 + select + test || ' #' || toString(query_index) || '::' || test_desc_.1 test_name, + 'unstable' test_status, + test_desc_.2 test_duration_ms, + 'https://s3.amazonaws.com/clickhouse-test-reports/$PR_TO_TEST/$SHA_TO_TEST/${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX}/report.html#unstable-queries.' || test || '.' || toString(query_index) report_url + from queries + array join map('old', left, 'new', right) as test_desc_ + where unstable_fail != 0 ) ; " diff --git a/tests/performance/sum_map.xml b/tests/performance/sum_map.xml index f55af077023..ffb9b9507ae 100644 --- a/tests/performance/sum_map.xml +++ b/tests/performance/sum_map.xml @@ -7,7 +7,7 @@ scale - 1000000 + 100000 diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 31b8d64e285..3a83126ea11 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -66,33 +66,101 @@ WHERE -- FIXME some of the following messages are not informative and it has to be fixed -create temporary table known_short_messages (s String) as select * from (select -['', '{} ({})', '({}) Keys: {}', '({}) {}', 'Aggregating', 'Became leader', 'Cleaning queue', -'Creating set.', 'Cyclic aliases', 'Detaching {}', 'Executing {}', 'Fire events: {}', -'Found part {}', 'Loaded queue', 'No sharding key', 'No tables', 'Query: {}', 'Removed', -'Removed part {}', 'Removing parts.', 'Request URI: {}', 'Sending part {}', -'Sent handshake', 'Starting {}', 'Will mimic {}', 'Writing to {}', 'dropIfEmpty', -'loadAll {}', '{} ({}:{})', '{} -> {}', '{} {}', '{}: {}', '{}%', 'Read object: {}', -'New segment: {}', 'Convert overflow', 'Division by zero', 'Files set to {}', -'Bytes set to {}', 'Numeric overflow', 'Invalid mode: {}', -'Write file: {}', 'Unable to parse JSONPath', 'Host is empty in S3 URI.', 'Expected end of line', -'inflate failed: {}{}', 'Center is not valid', 'Column ''{}'' is ambiguous', 'Cannot parse object', 'Invalid date: {}', -'There is no cache by name: {}', 'No part {} in table', '`{}` should be a String', 'There are duplicate id {}', -'Invalid replica name: {}', 'Unexpected value {} in enum', 'Unknown BSON type: {}', 'Point is not valid', -'Invalid qualified name: {}', 'INTO OUTFILE is not allowed', 'Arguments must not be NaN', 'Cell is not valid', -'brotli decode error{}', 'Invalid H3 index: {}', 'Too large node state size', 'No additional keys found.', -'Attempt to read after EOF.', 'Replication was stopped', '{} building file infos', 'Cannot parse uuid {}', -'Query was cancelled', 'Cancelled merging parts', 'Cancelled mutating parts', 'Log pulling is cancelled', -'Transaction was cancelled', 'Could not find table: {}', 'Table {} does not exist', -'Database {} does not exist', 'Dictionary ({}) not found', 'Unknown table function {}', -'Unknown format {}', 'Unknown explain kind ''{}''', 'Unknown setting {}', 'Unknown input format {}', -'Unknown identifier: ''{}''', 'User name is empty', 'Expected function, got: {}', -'Attempt to read after eof', 'String size is too big ({}), maximum: {}', -'Processed: {}%', 'Creating {}: {}', 'Table {}.{} doesn''t exist', 'Invalid cache key hex: {}', -'User has been dropped', 'Illegal type {} of argument of function {}. Should be DateTime or DateTime64', -'Unknown statistic column: {}', -'Bad SSH public key provided', 'Database {} does not exist', 'Substitution {} is not set', 'Invalid cache key hex: {}' -] as arr) array join arr; +create temporary table known_short_messages (s String) as select * from (select [ + '', + '({}) Keys: {}', + '({}) {}', + 'Aggregating', + 'Attempt to read after EOF.', + 'Attempt to read after eof', + 'Bad SSH public key provided', + 'Became leader', + 'Bytes set to {}', + 'Cancelled merging parts', + 'Cancelled mutating parts', + 'Cannot parse date here: {}', + 'Cannot parse object', + 'Cannot parse uuid {}', + 'Cleaning queue', + 'Column \'{}\' is ambiguous', + 'Convert overflow', + 'Could not find table: {}', + 'Creating {}: {}', + 'Cyclic aliases', + 'Database {} does not exist', + 'Detaching {}', + 'Dictionary ({}) not found', + 'Division by zero', + 'Executing {}', + 'Expected end of line', + 'Expected function, got: {}', + 'Files set to {}', + 'Fire events: {}', + 'Found part {}', + 'Host is empty in S3 URI.', + 'INTO OUTFILE is not allowed', + 'Illegal type {} of argument of function {}. Should be DateTime or DateTime64', + 'Illegal UTF-8 sequence, while processing \'{}\'', + 'Invalid cache key hex: {}', + 'Invalid date: {}', + 'Invalid mode: {}', + 'Invalid qualified name: {}', + 'Invalid replica name: {}', + 'Loaded queue', + 'Log pulling is cancelled', + 'New segment: {}', + 'No additional keys found.', + 'No part {} in table', + 'No sharding key', + 'No tables', + 'Numeric overflow', + 'Path to archive is empty', + 'Processed: {}%', + 'Query was cancelled', + 'Query: {}', + 'Read object: {}', + 'Removed part {}', + 'Removing parts.', + 'Replication was stopped', + 'Request URI: {}', + 'Sending part {}', + 'Sent handshake', + 'Starting {}', + 'String size is too big ({}), maximum: {}', + 'Substitution {} is not set', + 'Table {} does not exist', + 'Table {}.{} doesn\'t exist', + 'There are duplicate id {}', + 'There is no cache by name: {}', + 'Too large node state size', + 'Transaction was cancelled', + 'Unable to parse JSONPath', + 'Unexpected value {} in enum', + 'Unknown BSON type: {}', + 'Unknown explain kind \'{}\'', + 'Unknown format {}', + 'Unknown identifier: \'{}\'', + 'Unknown input format {}', + 'Unknown setting {}', + 'Unknown statistic column: {}', + 'Unknown table function {}', + 'User has been dropped', + 'User name is empty', + 'Will mimic {}', + 'Write file: {}', + 'Writing to {}', + '`{}` should be a String', + 'brotli decode error{}', + 'dropIfEmpty', + 'inflate failed: {}{}', + 'loadAll {}', + '{} ({})', + '{} ({}:{})', + '{} -> {}', + '{} {}', + '{}%', + '{}: {}' + ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. WITH 1 AS max_messages diff --git a/tests/queries/0_stateless/00502_sum_map.reference b/tests/queries/0_stateless/00502_sum_map.reference index b1cd0303004..0c9bebefd0b 100644 --- a/tests/queries/0_stateless/00502_sum_map.reference +++ b/tests/queries/0_stateless/00502_sum_map.reference @@ -63,7 +63,7 @@ SELECT sumMap(statusMap.goal_id, statusMap.revenue) FROM sum_map_decimal; SELECT sumMapWithOverflow(statusMap.goal_id, statusMap.revenue) FROM sum_map_decimal; ([1,2,3,4,5,6,7,8],[1,2,6,8,10,12,7,8]) DROP TABLE sum_map_decimal; -CREATE TABLE sum_map_decimal_nullable (`statusMap` Array(Tuple(goal_id UInt16, revenue Nullable(Decimal(9, 5))))) engine=Log; +CREATE TABLE sum_map_decimal_nullable (`statusMap` Nested(goal_id UInt16, revenue Nullable(Decimal(9, 5)))) engine=Log; INSERT INTO sum_map_decimal_nullable VALUES ([1, 2, 3], [1.0, 2.0, 3.0]), ([3, 4, 5], [3.0, 4.0, 5.0]), ([4, 5, 6], [4.0, 5.0, 6.0]), ([6, 7, 8], [6.0, 7.0, 8.0]); SELECT sumMap(statusMap.goal_id, statusMap.revenue) FROM sum_map_decimal_nullable; ([1,2,3,4,5,6,7,8],[1,2,6,8,10,12,7,8]) diff --git a/tests/queries/0_stateless/00502_sum_map.sql b/tests/queries/0_stateless/00502_sum_map.sql index 30037d49784..7d44bde6d50 100644 --- a/tests/queries/0_stateless/00502_sum_map.sql +++ b/tests/queries/0_stateless/00502_sum_map.sql @@ -56,7 +56,7 @@ SELECT sumMapWithOverflow(statusMap.goal_id, statusMap.revenue) FROM sum_map_dec DROP TABLE sum_map_decimal; -CREATE TABLE sum_map_decimal_nullable (`statusMap` Array(Tuple(goal_id UInt16, revenue Nullable(Decimal(9, 5))))) engine=Log; +CREATE TABLE sum_map_decimal_nullable (`statusMap` Nested(goal_id UInt16, revenue Nullable(Decimal(9, 5)))) engine=Log; INSERT INTO sum_map_decimal_nullable VALUES ([1, 2, 3], [1.0, 2.0, 3.0]), ([3, 4, 5], [3.0, 4.0, 5.0]), ([4, 5, 6], [4.0, 5.0, 6.0]), ([6, 7, 8], [6.0, 7.0, 8.0]); SELECT sumMap(statusMap.goal_id, statusMap.revenue) FROM sum_map_decimal_nullable; DROP TABLE sum_map_decimal_nullable; diff --git a/tests/queries/0_stateless/00547_named_tuples.reference b/tests/queries/0_stateless/00547_named_tuples.reference index 70cd0054bdd..041ead4ca79 100644 --- a/tests/queries/0_stateless/00547_named_tuples.reference +++ b/tests/queries/0_stateless/00547_named_tuples.reference @@ -1 +1 @@ -(1,'Hello') Tuple(x UInt64, s String) 1 Hello 1 Hello +(1,'Hello') Tuple(\n x UInt64,\n s String) 1 Hello 1 Hello diff --git a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference index a4d91178d73..6bac6173183 100644 --- a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference +++ b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference @@ -3,7 +3,5 @@ 2018-01-01 2 2 2018-01-01 2 2 == (Replicas) Test optimize == -d1 2 1 d2 1 0 -d3 2 1 d4 1 0 diff --git a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql index 9e293d0f7e2..871f96bb019 100644 --- a/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql +++ b/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql @@ -3,28 +3,28 @@ set optimize_on_insert = 0; drop table if exists tab_00577; create table tab_00577 (date Date, version UInt64, val UInt64) engine = ReplacingMergeTree(version) partition by date order by date settings enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 0, vertical_merge_algorithm_min_columns_to_activate = 0, min_rows_for_wide_part = 0, - min_bytes_for_wide_part = 0; + min_bytes_for_wide_part = 0, allow_experimental_replacing_merge_with_cleanup=1; insert into tab_00577 values ('2018-01-01', 2, 2), ('2018-01-01', 1, 1); insert into tab_00577 values ('2018-01-01', 0, 0); select * from tab_00577 order by version; -OPTIMIZE TABLE tab_00577 FINAL; +OPTIMIZE TABLE tab_00577 FINAL CLEANUP; select * from tab_00577; drop table tab_00577; DROP TABLE IF EXISTS testCleanupR1; CREATE TABLE testCleanupR1 (uid String, version UInt32, is_deleted UInt8) - ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{database}/tables/test_cleanup/', 'r1', version) + ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{database}/tables/test_cleanup/', 'r1', version, is_deleted) ORDER BY uid SETTINGS enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 0, vertical_merge_algorithm_min_columns_to_activate = 0, min_rows_for_wide_part = 0, - min_bytes_for_wide_part = 0; + min_bytes_for_wide_part = 0, allow_experimental_replacing_merge_with_cleanup=1; INSERT INTO testCleanupR1 (*) VALUES ('d1', 1, 0),('d2', 1, 0),('d3', 1, 0),('d4', 1, 0); INSERT INTO testCleanupR1 (*) VALUES ('d3', 2, 1); INSERT INTO testCleanupR1 (*) VALUES ('d1', 2, 1); SYSTEM SYNC REPLICA testCleanupR1; -- Avoid "Cannot select parts for optimization: Entry for part all_2_2_0 hasn't been read from the replication log yet" -OPTIMIZE TABLE testCleanupR1 FINAL; +OPTIMIZE TABLE testCleanupR1 FINAL CLEANUP; -- Only d3 to d5 remain SELECT '== (Replicas) Test optimize =='; SELECT * FROM testCleanupR1 order by uid; -DROP TABLE IF EXISTS testCleanupR1 +DROP TABLE IF EXISTS testCleanupR1 \ No newline at end of file diff --git a/tests/queries/0_stateless/00578_merge_table_and_table_virtual_column.sql b/tests/queries/0_stateless/00578_merge_table_and_table_virtual_column.sql index c2bc334ea38..f292eb30648 100644 --- a/tests/queries/0_stateless/00578_merge_table_and_table_virtual_column.sql +++ b/tests/queries/0_stateless/00578_merge_table_and_table_virtual_column.sql @@ -13,6 +13,8 @@ CREATE TABLE numbers5 ENGINE = MergeTree ORDER BY number AS SELECT number FROM n SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$'); SELECT DISTINCT count() FROM merge(currentDatabase(), '^numbers\\d+$') GROUP BY number; +SET optimize_aggregation_in_order = 0; -- FIXME : in order may happen before filter push down + SET max_rows_to_read = 1000; SET max_threads = 'auto'; diff --git a/tests/queries/0_stateless/00621_regression_for_in_operator.reference b/tests/queries/0_stateless/00621_regression_for_in_operator.reference index ab8bcf307eb..b68f550a742 100644 --- a/tests/queries/0_stateless/00621_regression_for_in_operator.reference +++ b/tests/queries/0_stateless/00621_regression_for_in_operator.reference @@ -10,7 +10,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.regression_for_in_operator_view + TABLE id: 3, alias: __table1, table_name: default.regression_for_in_operator_view WHERE FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -27,7 +27,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.regression_for_in_operator_view + TABLE id: 3, alias: __table1, table_name: default.regression_for_in_operator_view WHERE FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/00736_disjunction_optimisation.reference b/tests/queries/0_stateless/00736_disjunction_optimisation.reference index 84477a64057..f28dcacef0e 100644 --- a/tests/queries/0_stateless/00736_disjunction_optimisation.reference +++ b/tests/queries/0_stateless/00736_disjunction_optimisation.reference @@ -34,7 +34,7 @@ QUERY id: 0 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug WHERE FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -77,7 +77,7 @@ QUERY id: 0 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS k UInt64 s UInt64 @@ -86,7 +86,7 @@ QUERY id: 0 COLUMN id: 6, column_name: k, result_type: UInt64, source_id: 7 COLUMN id: 8, column_name: s, result_type: UInt64, source_id: 7 JOIN TREE - TABLE id: 7, table_name: default.bug + TABLE id: 7, alias: __table2, table_name: default.bug WHERE FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -151,7 +151,7 @@ QUERY id: 0 COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 16, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug SETTINGS allow_experimental_analyzer=1 21 1 22 1 @@ -184,7 +184,7 @@ QUERY id: 0 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug SETTINGS allow_experimental_analyzer=1 1 21 1 22 @@ -222,7 +222,7 @@ QUERY id: 0 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug WHERE FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -265,7 +265,7 @@ QUERY id: 0 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS k UInt64 s UInt64 @@ -274,7 +274,7 @@ QUERY id: 0 COLUMN id: 6, column_name: k, result_type: UInt64, source_id: 7 COLUMN id: 8, column_name: s, result_type: UInt64, source_id: 7 JOIN TREE - TABLE id: 7, table_name: default.bug + TABLE id: 7, alias: __table2, table_name: default.bug WHERE FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -347,7 +347,7 @@ QUERY id: 0 COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 21, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug SETTINGS allow_experimental_analyzer=1 21 1 22 1 @@ -380,7 +380,7 @@ QUERY id: 0 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug SETTINGS allow_experimental_analyzer=1 21 1 22 1 @@ -413,5 +413,5 @@ QUERY id: 0 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE - TABLE id: 3, table_name: default.bug + TABLE id: 3, alias: __table1, table_name: default.bug SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/00918_json_functions.reference b/tests/queries/0_stateless/00918_json_functions.reference index be8e603f8dc..5264d51fa73 100644 --- a/tests/queries/0_stateless/00918_json_functions.reference +++ b/tests/queries/0_stateless/00918_json_functions.reference @@ -69,8 +69,8 @@ hello 123456.1234 Decimal(20, 4) 123456.1234 Decimal(20, 4) 123456789012345.12 Decimal(30, 4) -(1234567890.1234567890123456789,'test') Tuple(a Decimal(35, 20), b LowCardinality(String)) -(1234567890.12345678901234567890123456789,'test') Tuple(a Decimal(45, 30), b LowCardinality(String)) +(1234567890.1234567890123456789,'test') Tuple(\n a Decimal(35, 20),\n b LowCardinality(String)) +(1234567890.12345678901234567890123456789,'test') Tuple(\n a Decimal(45, 30),\n b LowCardinality(String)) 123456789012345.1136 123456789012345.1136 1234567890.12345677879616925706 (1234567890.12345677879616925706,'test') 1234567890.123456695758468374595199311875 (1234567890.123456695758468374595199311875,'test') diff --git a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference index 39979a98bde..b9a66a1e1a9 100644 --- a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference +++ b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference @@ -3,7 +3,7 @@ Received exception from server: Code: 57. Error: Received from localhost:9000. Error: There was an error on [localhost:9000]: Code: 57. Error: Table default.none already exists. (TABLE_ALREADY_EXISTS) (query: create table none on cluster test_shard_localhost (n int) engine=Memory;) Received exception from server: -Code: 159. Error: Received from localhost:9000. Error: Watching task is executing longer than distributed_ddl_task_timeout (=1) seconds. There are 1 unfinished hosts (0 of them are currently active), they are going to execute the query in background. (TIMEOUT_EXCEEDED) +Code: 159. Error: Received from localhost:9000. Error: Watching task is executing longer than distributed_ddl_task_timeout (=1) seconds. There are 1 unfinished hosts (0 of them are currently executing the task), they are going to execute the query in background. (TIMEOUT_EXCEEDED) (query: drop table if exists none on cluster test_unavailable_shard;) throw localhost 9000 0 0 0 @@ -12,7 +12,7 @@ Code: 57. Error: Received from localhost:9000. Error: There was an error on [loc (query: create table throw on cluster test_shard_localhost (n int) engine=Memory format Null;) localhost 9000 0 1 0 Received exception from server: -Code: 159. Error: Received from localhost:9000. Error: Watching task is executing longer than distributed_ddl_task_timeout (=1) seconds. There are 1 unfinished hosts (0 of them are currently active), they are going to execute the query in background. (TIMEOUT_EXCEEDED) +Code: 159. Error: Received from localhost:9000. Error: Watching task is executing longer than distributed_ddl_task_timeout (=1) seconds. There are 1 unfinished hosts (0 of them are currently executing the task), they are going to execute the query in background. (TIMEOUT_EXCEEDED) (query: drop table if exists throw on cluster test_unavailable_shard;) null_status_on_timeout localhost 9000 0 0 0 diff --git a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh index d2695e602c5..12e142adda9 100755 --- a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh +++ b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh @@ -33,7 +33,7 @@ function run_until_out_contains() done } -RAND_COMMENT="01175_DDL_$RANDOM" +RAND_COMMENT="01175_DDL_$CLICKHOUSE_DATABASE" LOG_COMMENT="${CLICKHOUSE_LOG_COMMENT}_$RAND_COMMENT" CLICKHOUSE_CLIENT_WITH_SETTINGS=${CLICKHOUSE_CLIENT/--log_comment ${CLICKHOUSE_LOG_COMMENT}/--log_comment ${LOG_COMMENT}} diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index e2c0655b2bc..1a3a271528c 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -48,6 +48,7 @@ ALTER TABLE [] \N ALTER ALTER DATABASE [] \N ALTER ALTER VIEW REFRESH ['ALTER LIVE VIEW REFRESH','REFRESH VIEW'] VIEW ALTER VIEW ALTER VIEW MODIFY QUERY ['ALTER TABLE MODIFY QUERY'] VIEW ALTER VIEW +ALTER VIEW MODIFY REFRESH ['ALTER TABLE MODIFY QUERY'] VIEW ALTER VIEW ALTER VIEW [] \N ALTER ALTER [] \N ALL CREATE DATABASE [] DATABASE CREATE @@ -127,6 +128,7 @@ SYSTEM FETCHES ['SYSTEM STOP FETCHES','SYSTEM START FETCHES','STOP FETCHES','STA SYSTEM MOVES ['SYSTEM STOP MOVES','SYSTEM START MOVES','STOP MOVES','START MOVES'] TABLE SYSTEM SYSTEM PULLING REPLICATION LOG ['SYSTEM STOP PULLING REPLICATION LOG','SYSTEM START PULLING REPLICATION LOG'] TABLE SYSTEM SYSTEM CLEANUP ['SYSTEM STOP CLEANUP','SYSTEM START CLEANUP'] TABLE SYSTEM +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 SYSTEM DISTRIBUTED SENDS ['SYSTEM STOP DISTRIBUTED SENDS','SYSTEM START DISTRIBUTED SENDS','STOP DISTRIBUTED SENDS','START DISTRIBUTED SENDS'] TABLE SYSTEM SENDS SYSTEM REPLICATED SENDS ['SYSTEM STOP REPLICATED SENDS','SYSTEM START REPLICATED SENDS','STOP REPLICATED SENDS','START REPLICATED SENDS'] TABLE SYSTEM SENDS SYSTEM SENDS ['SYSTEM STOP SENDS','SYSTEM START SENDS','STOP SENDS','START SENDS'] \N SYSTEM diff --git a/tests/queries/0_stateless/01300_group_by_other_keys_having.reference b/tests/queries/0_stateless/01300_group_by_other_keys_having.reference index a9be79800c1..f861da3da2b 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys_having.reference +++ b/tests/queries/0_stateless/01300_group_by_other_keys_having.reference @@ -49,7 +49,7 @@ QUERY id: 0 LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 @@ -124,7 +124,7 @@ QUERY id: 0 LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 @@ -194,7 +194,7 @@ QUERY id: 0 COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7 CONSTANT id: 11, constant_value: UInt64_5, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 7, table_function_name: numbers + TABLE_FUNCTION id: 7, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 @@ -276,7 +276,7 @@ QUERY id: 0 LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 diff --git a/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference b/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference index bf184d142ec..d47f12ff4d1 100644 --- a/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference @@ -49,14 +49,14 @@ QUERY id: 0 LIST id: 3, nodes: 1 COLUMN id: 4, column_name: x, result_type: UInt64, source_id: 5 JOIN TREE - QUERY id: 5, is_subquery: 1 + QUERY id: 5, alias: __table1, is_subquery: 1 PROJECTION COLUMNS x UInt64 PROJECTION LIST id: 6, nodes: 1 COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8 JOIN TREE - TABLE_FUNCTION id: 8, table_function_name: numbers + TABLE_FUNCTION id: 8, alias: __table2, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 @@ -83,14 +83,14 @@ QUERY id: 0 LIST id: 3, nodes: 1 COLUMN id: 4, column_name: x, result_type: UInt64, source_id: 5 JOIN TREE - QUERY id: 5, is_subquery: 1 + QUERY id: 5, alias: __table1, is_subquery: 1 PROJECTION COLUMNS x UInt64 PROJECTION LIST id: 6, nodes: 1 COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8 JOIN TREE - TABLE_FUNCTION id: 8, table_function_name: numbers + TABLE_FUNCTION id: 8, alias: __table2, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 @@ -119,14 +119,14 @@ QUERY id: 0 LIST id: 3, nodes: 1 COLUMN id: 4, column_name: x, result_type: UInt64, source_id: 5 JOIN TREE - QUERY id: 5, is_subquery: 1 + QUERY id: 5, alias: __table1, is_subquery: 1 PROJECTION COLUMNS x UInt64 PROJECTION LIST id: 6, nodes: 1 COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8 JOIN TREE - TABLE_FUNCTION id: 8, table_function_name: numbers + TABLE_FUNCTION id: 8, alias: __table2, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 @@ -171,7 +171,7 @@ QUERY id: 0 JOIN TREE JOIN id: 8, strictness: ALL, kind: FULL LEFT TABLE EXPRESSION - QUERY id: 3, alias: s, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS key UInt64 PROJECTION @@ -182,12 +182,12 @@ QUERY id: 0 COLUMN id: 12, column_name: number, result_type: UInt64, source_id: 13 CONSTANT id: 14, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 13, table_function_name: numbers + TABLE_FUNCTION id: 13, alias: __table2, table_function_name: numbers ARGUMENTS LIST id: 15, nodes: 1 CONSTANT id: 16, constant_value: UInt64_4, constant_value_type: UInt8 RIGHT TABLE EXPRESSION - TABLE id: 5, alias: t, table_name: default.test + TABLE id: 5, alias: __table3, table_name: default.test JOIN EXPRESSION LIST id: 17, nodes: 1 COLUMN id: 18, column_name: key, result_type: UInt64, source_id: 8 @@ -220,7 +220,7 @@ QUERY id: 0 COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: a, result_type: UInt8, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.test + TABLE id: 3, alias: __table1, table_name: default.test ORDER BY LIST id: 5, nodes: 2 SORT id: 6, sort_direction: ASCENDING, with_fill: 0 @@ -246,7 +246,7 @@ QUERY id: 0 COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: a, result_type: UInt8, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.test + TABLE id: 3, alias: __table1, table_name: default.test ORDER BY LIST id: 5, nodes: 2 SORT id: 6, sort_direction: ASCENDING, with_fill: 0 @@ -270,7 +270,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.test + TABLE id: 3, alias: __table1, table_name: default.test GROUP BY LIST id: 4, nodes: 1 COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3 @@ -297,9 +297,9 @@ QUERY id: 0 JOIN TREE JOIN id: 6, strictness: ALL, kind: INNER LEFT TABLE EXPRESSION - TABLE id: 3, table_name: default.t1 + TABLE id: 3, alias: __table1, table_name: default.t1 RIGHT TABLE EXPRESSION - TABLE id: 5, table_name: default.t2 + TABLE id: 5, alias: __table2, table_name: default.t2 JOIN EXPRESSION FUNCTION id: 7, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/01455_opentelemetry_distributed.reference b/tests/queries/0_stateless/01455_opentelemetry_distributed.reference index a6d43856aec..2920b387aa2 100644 --- a/tests/queries/0_stateless/01455_opentelemetry_distributed.reference +++ b/tests/queries/0_stateless/01455_opentelemetry_distributed.reference @@ -1,9 +1,9 @@ ===http=== {"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} {"query":"DESC TABLE system.one","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} -{"query":"SELECT 1 AS `1` FROM `system`.`one`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} +{"query":"SELECT 1 AS `1` FROM `system`.`one` AS `__table1`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} {"query":"DESC TABLE system.one","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} -{"query":"SELECT 1 AS `1` FROM `system`.`one`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} +{"query":"SELECT 1 AS `1` FROM `system`.`one` AS `__table1`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} {"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} {"total spans":"3","unique spans":"3","unique non-zero parent spans":"3"} {"initial query spans with proper parent":"2"} diff --git a/tests/queries/0_stateless/01458_named_tuple_millin.reference b/tests/queries/0_stateless/01458_named_tuple_millin.reference index d6d6d7ae8d4..954dfe36563 100644 --- a/tests/queries/0_stateless/01458_named_tuple_millin.reference +++ b/tests/queries/0_stateless/01458_named_tuple_millin.reference @@ -3,10 +3,10 @@ CREATE TABLE default.tuple `j` Tuple(a Int8, b String) ) ENGINE = Memory -j Tuple(a Int8, b String) +j Tuple(\n a Int8,\n b String) CREATE TABLE default.tuple ( `j` Tuple(a Int8, b String) ) ENGINE = Memory -j Tuple(a Int8, b String) +j Tuple(\n a Int8,\n b String) diff --git a/tests/queries/0_stateless/01532_tuple_with_name_type.reference b/tests/queries/0_stateless/01532_tuple_with_name_type.reference index f9f6b5995ce..66b85f05fa6 100644 --- a/tests/queries/0_stateless/01532_tuple_with_name_type.reference +++ b/tests/queries/0_stateless/01532_tuple_with_name_type.reference @@ -1,5 +1,4 @@ -a Tuple(key String, value String) -a Tuple(Tuple(key String, value String)) -a.key Array(String) -a.value Array(String) -a Tuple(UInt8, Tuple(key String, value String)) +a Tuple(\n key String,\n value String) +a Tuple(Tuple(\n key String,\n value String)) +a Array(Tuple(\n key String,\n value String)) +a Tuple(UInt8, Tuple(\n key String,\n value String)) diff --git a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference index 8a34751b071..2631199cbab 100644 --- a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference +++ b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference @@ -2,7 +2,7 @@ execute: --allow_experimental_analyzer=1 "foo" 1 execute: --allow_experimental_analyzer=1 --stage fetch_columns -"dummy_0" +"__table1.dummy" 0 execute: --allow_experimental_analyzer=1 --stage with_mergeable_state "1_UInt8" diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index 5d12a09a846..156f36f7dba 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -917,9 +917,9 @@ from ; Expression ((Project names + Projection)) Window (Window step for window \'\') - Window (Window step for window \'PARTITION BY p_0\') - Window (Window step for window \'PARTITION BY p_0 ORDER BY o_1 ASC\') - Sorting (Sorting for window \'PARTITION BY p_0 ORDER BY o_1 ASC\') + Window (Window step for window \'PARTITION BY __table1.p\') + Window (Window step for window \'PARTITION BY __table1.p ORDER BY __table1.o ASC\') + Sorting (Sorting for window \'PARTITION BY __table1.p ORDER BY __table1.o ASC\') Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) ReadFromSystemNumbers explain select @@ -930,11 +930,11 @@ from from numbers(16)) t ; Expression ((Project names + Projection)) - Window (Window step for window \'ORDER BY o_0 ASC, number_1 ASC\') - Sorting (Sorting for window \'ORDER BY o_0 ASC, number_1 ASC\') - Window (Window step for window \'ORDER BY number_1 ASC\') + Window (Window step for window \'ORDER BY __table1.o ASC, __table1.number ASC\') + Sorting (Sorting for window \'ORDER BY __table1.o ASC, __table1.number ASC\') + Window (Window step for window \'ORDER BY __table1.number ASC\') Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))) [lifted up part]) - Sorting (Sorting for window \'ORDER BY number_1 ASC\') + Sorting (Sorting for window \'ORDER BY __table1.number ASC\') Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) ReadFromSystemNumbers -- A test case for the sort comparator found by fuzzer. diff --git a/tests/queries/0_stateless/01603_decimal_mult_float.reference b/tests/queries/0_stateless/01603_decimal_mult_float.reference index 4c9d45423ee..72b10d768f1 100644 --- a/tests/queries/0_stateless/01603_decimal_mult_float.reference +++ b/tests/queries/0_stateless/01603_decimal_mult_float.reference @@ -1,14 +1,14 @@ 2.4 10.165 -0.00012000000000000002 -150.16500000000002 -7.775900000000001 -56.622689999999984 -598.8376688440277 -299.41883695311844 -0.7485470860550345 -2.2456412771483882 -1.641386318314034 -1.641386318314034 -1.6413863258732018 -1.6413863258732018 +0.00012 +150.165 +7.7759 +56.62269 +598.837669 +299.418837 +0.748547 +2.245641 +1.641386 +1.641386 +1.641386 +1.641386 diff --git a/tests/queries/0_stateless/01603_decimal_mult_float.sql b/tests/queries/0_stateless/01603_decimal_mult_float.sql index 799ab91d332..1a4652df23a 100644 --- a/tests/queries/0_stateless/01603_decimal_mult_float.sql +++ b/tests/queries/0_stateless/01603_decimal_mult_float.sql @@ -1,9 +1,9 @@ SET optimize_arithmetic_operations_in_aggregate_functions = 0; -SELECT toDecimal32(2, 2) * 1.2; -SELECT toDecimal64(0.5, 2) * 20.33; -SELECT 0.00001 * toDecimal32(12, 2); -SELECT 30.033 * toDecimal32(5, 1); +SELECT round(toDecimal32(2, 2) * 1.2, 6); +SELECT round(toDecimal64(0.5, 2) * 20.33, 6); +SELECT round(0.00001 * toDecimal32(12, 2), 6); +SELECT round(30.033 * toDecimal32(5, 1), 6); CREATE TABLE IF NOT EXISTS test01603 ( f64 Float64, @@ -13,17 +13,17 @@ CREATE TABLE IF NOT EXISTS test01603 ( INSERT INTO test01603(f64) SELECT 1 / (number + 1) FROM system.numbers LIMIT 1000; -SELECT sum(d * 1.1) FROM test01603; -SELECT sum(8.01 * d) FROM test01603; +SELECT round(sum(d * 1.1), 6) FROM test01603; +SELECT round(sum(8.01 * d), 6) FROM test01603; -SELECT sum(f64 * toDecimal64(80, 2)) FROM test01603; -SELECT sum(toDecimal64(40, 2) * f32) FROM test01603; -SELECT sum(f64 * toDecimal64(0.1, 2)) FROM test01603; -SELECT sum(toDecimal64(0.3, 2) * f32) FROM test01603; +SELECT round(sum(f64 * toDecimal64(80, 2)), 6) FROM test01603; +SELECT round(sum(toDecimal64(40, 2) * f32), 6) FROM test01603; +SELECT round(sum(f64 * toDecimal64(0.1, 2)), 6) FROM test01603; +SELECT round(sum(toDecimal64(0.3, 2) * f32), 6) FROM test01603; -SELECT sum(f64 * d) FROM test01603; -SELECT sum(d * f64) FROM test01603; -SELECT sum(f32 * d) FROM test01603; -SELECT sum(d * f32) FROM test01603; +SELECT round(sum(f64 * d), 6) FROM test01603; +SELECT round(sum(d * f64), 6) FROM test01603; +SELECT round(sum(f32 * d), 6) FROM test01603; +SELECT round(sum(d * f32), 6) FROM test01603; DROP TABLE IF EXISTS test01603; diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index ef6425b485b..d267df2237f 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -45,7 +45,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.constraint_test_constants + TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants WHERE FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -63,7 +63,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.constraint_test_constants + TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants WHERE FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -80,5 +80,5 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.constraint_test_constants + TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.reference b/tests/queries/0_stateless/01622_constraints_where_optimization.reference index b5520d75b0e..3f6e8211f1a 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.reference @@ -8,7 +8,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where + TABLE id: 3, alias: __table1, table_name: default.t_constraints_where WHERE CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 @@ -22,7 +22,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where + TABLE id: 3, alias: __table1, table_name: default.t_constraints_where WHERE CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 @@ -36,7 +36,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where + TABLE id: 3, alias: __table1, table_name: default.t_constraints_where WHERE CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 @@ -50,7 +50,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where + TABLE id: 3, alias: __table1, table_name: default.t_constraints_where WHERE FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -68,7 +68,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where + TABLE id: 3, alias: __table1, table_name: default.t_constraints_where PREWHERE FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -85,5 +85,5 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE id: 3, table_name: default.t_constraints_where + TABLE id: 3, alias: __table1, table_name: default.t_constraints_where SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index 3639ad47228..555a4c93f70 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -20,7 +20,7 @@ QUERY id: 0 COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.column_swap_test_test + TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -50,7 +50,7 @@ QUERY id: 0 COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.column_swap_test_test + TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test PREWHERE FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -80,7 +80,7 @@ QUERY id: 0 COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.column_swap_test_test + TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -110,7 +110,7 @@ QUERY id: 0 COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.column_swap_test_test + TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -140,7 +140,7 @@ QUERY id: 0 COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.column_swap_test_test + TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -162,7 +162,7 @@ QUERY id: 0 COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5 CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.column_swap_test_test + TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 7, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -191,7 +191,7 @@ QUERY id: 0 CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 COLUMN id: 9, column_name: a, result_type: String, source_id: 7 JOIN TREE - TABLE id: 7, table_name: default.column_swap_test_test + TABLE id: 7, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -223,7 +223,7 @@ QUERY id: 0 CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8 COLUMN id: 9, column_name: a, result_type: String, source_id: 7 JOIN TREE - TABLE id: 7, table_name: default.column_swap_test_test + TABLE id: 7, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -248,7 +248,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: String, source_id: 3 COLUMN id: 4, column_name: a, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.column_swap_test_test + TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -270,7 +270,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: String, source_id: 3 COLUMN id: 4, column_name: a, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.column_swap_test_test + TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -292,7 +292,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: String, source_id: 3 COLUMN id: 4, column_name: a, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.column_swap_test_test + TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -310,7 +310,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.column_swap_test_test + TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test WHERE FUNCTION id: 4, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -327,5 +327,5 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: UInt32, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.t_bad_constraint + TABLE id: 3, alias: __table1, table_name: default.t_bad_constraint SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/01646_rewrite_sum_if.reference b/tests/queries/0_stateless/01646_rewrite_sum_if.reference index 871c75737c6..af582908f03 100644 --- a/tests/queries/0_stateless/01646_rewrite_sum_if.reference +++ b/tests/queries/0_stateless/01646_rewrite_sum_if.reference @@ -56,7 +56,7 @@ QUERY id: 0 CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8 CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 12, table_function_name: numbers + TABLE_FUNCTION id: 12, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 15, nodes: 1 CONSTANT id: 16, constant_value: UInt64_100, constant_value_type: UInt8 @@ -82,7 +82,7 @@ QUERY id: 0 CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8 CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 12, table_function_name: numbers + TABLE_FUNCTION id: 12, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 15, nodes: 1 CONSTANT id: 16, constant_value: UInt64_100, constant_value_type: UInt8 @@ -111,7 +111,7 @@ QUERY id: 0 CONSTANT id: 15, constant_value: UInt64_2, constant_value_type: UInt8 CONSTANT id: 16, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 14, table_function_name: numbers + TABLE_FUNCTION id: 14, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 17, nodes: 1 CONSTANT id: 18, constant_value: UInt64_100, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 54ca55d2068..436d06c5076 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -28,7 +28,7 @@ Aggregating Filter Filter > (analyzer) filter should be pushed down after aggregating, column after aggregation is const -COLUMN Const(UInt8) -> notEquals(y_1, 0_UInt8) +COLUMN Const(UInt8) -> notEquals(__table1.y, 0_UInt8) Aggregating Filter Filter @@ -49,9 +49,9 @@ Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased Filter column -ALIAS notEquals(s_0, 4_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 4_UInt8)) +ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8)) Aggregating -Filter column: notEquals(y_1, 0_UInt8) +Filter column: notEquals(__table1.y, 0_UInt8) 0 1 1 2 2 3 @@ -68,9 +68,9 @@ Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted Filter column -FUNCTION and(minus(s_0, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 4_UInt8)) UInt8 : 2 +FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2 Aggregating -Filter column: notEquals(y_1, 0_UInt8) +Filter column: notEquals(__table1.y, 0_UInt8) 0 1 1 2 2 3 @@ -87,9 +87,9 @@ Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column -FUNCTION and(minus(s_0, 8_UInt8) :: 0, minus(s_0, 4_UInt8) :: 2) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 8_UInt8), minus(s_0, 4_UInt8)) +FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8)) Aggregating -Filter column: notEquals(y_1, 0_UInt8) +Filter column: notEquals(__table1.y, 0_UInt8) 0 1 1 2 2 3 @@ -105,9 +105,9 @@ Aggregating Filter column: and(notEquals(y, 0), minus(y, 4)) > (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column -ALIAS notEquals(s_0, 8_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 8_UInt8), minus(y_1, 4_UInt8)) +ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8)) Aggregating -Filter column: and(notEquals(y_1, 0_UInt8), minus(y_1, 4_UInt8)) +Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8)) 0 1 1 2 2 3 @@ -121,9 +121,9 @@ Filter column: and(notEquals(y, 2), notEquals(x, 0)) ARRAY JOIN x Filter column: notEquals(y, 2) > (analyzer) filter is split, one part is filtered before ARRAY JOIN -Filter column: and(notEquals(y_1, 2_UInt8), notEquals(x_0, 0_UInt8)) -ARRAY JOIN x_0 -Filter column: notEquals(y_1, 2_UInt8) +Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__table1.x, 0_UInt8)) +ARRAY JOIN __table1.x +Filter column: notEquals(__table2.y, 2_UInt8) 1 3 > filter is pushed down before Distinct Distinct @@ -132,7 +132,7 @@ Filter column: notEquals(y, 2) > (analyzer) filter is pushed down before Distinct Distinct Distinct -Filter column: notEquals(y_1, 2_UInt8) +Filter column: notEquals(__table1.y, 2_UInt8) 0 0 0 1 1 0 @@ -144,7 +144,7 @@ Filter column: and(notEquals(x, 0), notEquals(y, 0)) > (analyzer) filter is pushed down before sorting steps Sorting Sorting -Filter column: and(notEquals(x_0, 0_UInt8), notEquals(y_1, 0_UInt8)) +Filter column: and(notEquals(__table1.x, 0_UInt8), notEquals(__table1.y, 0_UInt8)) 1 2 1 1 > filter is pushed down before TOTALS HAVING and aggregating @@ -154,7 +154,7 @@ Filter column: notEquals(y, 2) > (analyzer) filter is pushed down before TOTALS HAVING and aggregating TotalsHaving Aggregating -Filter column: notEquals(y_0, 2_UInt8) +Filter column: notEquals(__table1.y, 2_UInt8) 0 12 1 15 3 10 @@ -174,7 +174,7 @@ Join > (analyzer) one condition of filter is pushed down before LEFT JOIN Join Join -Filter column: notEquals(number_0, 1_UInt8) +Filter column: notEquals(__table1.number, 1_UInt8) 0 0 3 3 > one condition of filter is pushed down before INNER JOIN @@ -185,7 +185,7 @@ Join > (analyzer) one condition of filter is pushed down before INNER JOIN Join Join -Filter column: notEquals(number_0, 1_UInt8) +Filter column: notEquals(__table1.number, 1_UInt8) 3 3 > filter is pushed down before UNION Union diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index a765a6ea4fa..5a517264243 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -36,7 +36,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " explain actions = 1 select s, y, y != 0 from (select sum(x) as s, y from ( select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 - settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(y_1, 0_UInt8)" + settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(__table1.y, 0_UInt8)" $CLICKHOUSE_CLIENT -q " select s, y, y != 0 from (select sum(x) as s, y from ( select number as x, number + 1 as y from numbers(10)) group by y @@ -56,7 +56,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|ALIAS notEquals(s_0, 4_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -76,7 +76,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|FUNCTION and(minus(s_0, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 4_UInt8)) UInt8 : 2" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -96,7 +96,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|FUNCTION and(minus(s_0, 8_UInt8) :: 0, minus(s_0, 4_UInt8) :: 2) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 8_UInt8), minus(s_0, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -116,7 +116,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y_1, 0_UInt8), minus(y_1, 4_UInt8))\|ALIAS notEquals(s_0, 8_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 8_UInt8), minus(y_1, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))\|ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -134,7 +134,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " explain actions = 1 select x, y from ( select range(number) as x, number + 1 as y from numbers(3) ) array join x where y != 2 and x != 0" | - grep -o "Filter column: and(notEquals(y_1, 2_UInt8), notEquals(x_0, 0_UInt8))\|ARRAY JOIN x_0\|Filter column: notEquals(y_1, 2_UInt8)" + grep -o "Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__table1.x, 0_UInt8))\|ARRAY JOIN __table1.x\|Filter column: notEquals(__table2.y, 2_UInt8)" $CLICKHOUSE_CLIENT -q " select x, y from ( select range(number) as x, number + 1 as y from numbers(3) @@ -166,7 +166,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10)) ) where y != 2 settings enable_optimize_predicate_expression=0" | - grep -o "Distinct\|Filter column: notEquals(y_1, 2_UInt8)" + grep -o "Distinct\|Filter column: notEquals(__table1.y, 2_UInt8)" $CLICKHOUSE_CLIENT -q " select x, y from ( select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10)) @@ -186,7 +186,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " select number % 2 as x, number % 3 as y from numbers(6) order by y desc ) where x != 0 and y != 0 settings enable_optimize_predicate_expression = 0" | - grep -o "Sorting\|Filter column: and(notEquals(x_0, 0_UInt8), notEquals(y_1, 0_UInt8))" + grep -o "Sorting\|Filter column: and(notEquals(__table1.x, 0_UInt8), notEquals(__table1.y, 0_UInt8))" $CLICKHOUSE_CLIENT -q " select x, y from ( select number % 2 as x, number % 3 as y from numbers(6) order by y desc @@ -206,7 +206,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals ) where y != 2 settings enable_optimize_predicate_expression=0" | - grep -o "TotalsHaving\|Aggregating\|Filter column: notEquals(y_0, 2_UInt8)" + grep -o "TotalsHaving\|Aggregating\|Filter column: notEquals(__table1.y, 2_UInt8)" $CLICKHOUSE_CLIENT -q " select * from ( select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals @@ -236,7 +236,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select number as a, r.b from numbers(4) as l any left join ( select number + 2 as b from numbers(3) ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | - grep -o "Join\|Filter column: notEquals(number_0, 1_UInt8)" + grep -o "Join\|Filter column: notEquals(__table1.number, 1_UInt8)" $CLICKHOUSE_CLIENT -q " select number as a, r.b from numbers(4) as l any left join ( select number + 2 as b from numbers(3) @@ -255,7 +255,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select number as a, r.b from numbers(4) as l any inner join ( select number + 2 as b from numbers(3) ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | - grep -o "Join\|Filter column: notEquals(number_0, 1_UInt8)" + grep -o "Join\|Filter column: notEquals(__table1.number, 1_UInt8)" $CLICKHOUSE_CLIENT -q " select number as a, r.b from numbers(4) as l any inner join ( select number + 2 as b from numbers(3) diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference index 8a33df9fad2..7c2753124b3 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference @@ -7,19 +7,19 @@ Partial sorting plan Prefix sort description: n ASC Result sort description: n ASC, x ASC optimize_read_in_window_order=1, allow_experimental_analyzer=1 - Prefix sort description: n_0 ASC - Result sort description: n_0 ASC, x_1 ASC + Prefix sort description: __table1.n ASC + Result sort description: __table1.n ASC, __table1.x ASC No sorting plan optimize_read_in_window_order=0 Sort description: n ASC, x ASC optimize_read_in_window_order=0, allow_experimental_analyzer=1 - Sort description: n_0 ASC, x_1 ASC + Sort description: __table1.n ASC, __table1.x ASC optimize_read_in_window_order=1 Prefix sort description: n ASC, x ASC Result sort description: n ASC, x ASC optimize_read_in_window_order=1, allow_experimental_analyzer=1 - Prefix sort description: n_0 ASC, x_1 ASC - Result sort description: n_0 ASC, x_1 ASC + Prefix sort description: __table1.n ASC, __table1.x ASC + Result sort description: __table1.n ASC, __table1.x ASC Complex ORDER BY optimize_read_in_window_order=0 3 3 1 diff --git a/tests/queries/0_stateless/01732_race_condition_storage_join_long.sh b/tests/queries/0_stateless/01732_race_condition_storage_join_long.sh index 48e726aca9d..5fc41890a18 100755 --- a/tests/queries/0_stateless/01732_race_condition_storage_join_long.sh +++ b/tests/queries/0_stateless/01732_race_condition_storage_join_long.sh @@ -15,7 +15,7 @@ echo " function read_thread_big() { - while true; do + while true; do echo " SELECT * FROM ( SELECT number AS x FROM numbers(100000) ) AS t1 ALL FULL JOIN storage_join_race USING (x) FORMAT Null; " | $CLICKHOUSE_CLIENT -n @@ -24,7 +24,7 @@ function read_thread_big() function read_thread_small() { - while true; do + while true; do echo " SELECT * FROM ( SELECT number AS x FROM numbers(10) ) AS t1 ALL FULL JOIN storage_join_race USING (x) FORMAT Null; " | $CLICKHOUSE_CLIENT -n @@ -51,8 +51,11 @@ timeout $TIMEOUT bash -c read_thread_big 2> /dev/null & timeout $TIMEOUT bash -c read_thread_small 2> /dev/null & timeout $TIMEOUT bash -c read_thread_select 2> /dev/null & +# Run insert query with a sleep to make sure that it is executed all the time during the read queries. echo " - INSERT INTO storage_join_race SELECT number AS x, number AS y FROM numbers (10000000); + INSERT INTO storage_join_race + SELECT number AS x, sleepEachRow(0.1) + number AS y FROM numbers ($TIMEOUT * 10) + SETTINGS function_sleep_max_microseconds_per_block = 100000000, max_block_size = 10; " | $CLICKHOUSE_CLIENT -n wait diff --git a/tests/queries/0_stateless/01823_explain_json.reference b/tests/queries/0_stateless/01823_explain_json.reference index befbf82f4fb..23fb34c2192 100644 --- a/tests/queries/0_stateless/01823_explain_json.reference +++ b/tests/queries/0_stateless/01823_explain_json.reference @@ -37,59 +37,59 @@ "Node Type": "Aggregating", "Header": [ { - "Name": "number_0", + "Name": "__table1.number", "Type": "UInt64" }, { - "Name": "quantile(0.2_Float64)(number_0)", + "Name": "quantile(0.2_Float64)(__table1.number)", "Type": "Float64" }, { - "Name": "sumIf(number_0, greater(number_0, 0_UInt8))", + "Name": "sumIf(__table1.number, greater(__table1.number, 0_UInt8))", "Type": "UInt64" } ], - "Keys": ["number_0"], + "Keys": ["__table1.number"], "Aggregates": [ { - "Name": "quantile(0.2_Float64)(number_0)", + "Name": "quantile(0.2_Float64)(__table1.number)", "Function": { "Name": "quantile", "Parameters": ["0.2"], "Argument Types": ["UInt64"], "Result Type": "Float64" }, - "Arguments": ["number_0"] + "Arguments": ["__table1.number"] }, { - "Name": "sumIf(number_0, greater(number_0, 0_UInt8))", + "Name": "sumIf(__table1.number, greater(__table1.number, 0_UInt8))", "Function": { "Name": "sumIf", "Argument Types": ["UInt64", "UInt8"], "Result Type": "UInt64" }, - "Arguments": ["number_0", "greater(number_0, 0_UInt8)"] + "Arguments": ["__table1.number", "greater(__table1.number, 0_UInt8)"] } ], -------- "Node Type": "ArrayJoin", "Left": false, - "Columns": ["x_0", "y_1"], + "Columns": ["__table1.x", "__table1.y"], -------- "Node Type": "Distinct", - "Columns": ["intDiv(number_0, 2_UInt8)", "intDiv(number_0, 3_UInt8)"], + "Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"], -- "Node Type": "Distinct", - "Columns": ["intDiv(number_0, 2_UInt8)", "intDiv(number_0, 3_UInt8)"], + "Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"], -------- "Sort Description": [ { - "Column": "number_0", + "Column": "__table1.number", "Ascending": false, "With Fill": false }, { - "Column": "plus(number_0, 1_UInt8)", + "Column": "plus(__table1.number, 1_UInt8)", "Ascending": true, "With Fill": false } diff --git a/tests/queries/0_stateless/01825_type_json_10.reference b/tests/queries/0_stateless/01825_type_json_10.reference index 53fe604fa51..4161fb59c93 100644 --- a/tests/queries/0_stateless/01825_type_json_10.reference +++ b/tests/queries/0_stateless/01825_type_json_10.reference @@ -1,4 +1,4 @@ -Tuple(a Tuple(b Int8, c Nested(d Int8, e Array(Int16), f Int8))) +Tuple(\n a Tuple(\n b Int8,\n c Nested(d Int8, e Array(Int16), f Int8))) {"o":{"a":{"b":1,"c":[{"d":10,"e":[31],"f":0},{"d":20,"e":[63,127],"f":0}]}}} {"o":{"a":{"b":2,"c":[]}}} {"o":{"a":{"b":3,"c":[{"d":0,"e":[32],"f":20},{"d":0,"e":[64,128],"f":30}]}}} diff --git a/tests/queries/0_stateless/01825_type_json_11.reference b/tests/queries/0_stateless/01825_type_json_11.reference index 27569620cd7..0575743e019 100644 --- a/tests/queries/0_stateless/01825_type_json_11.reference +++ b/tests/queries/0_stateless/01825_type_json_11.reference @@ -1,4 +1,4 @@ -Tuple(id Int8, key_1 Nested(key_2 Int32, key_3 Nested(key_4 Nested(key_5 Int8), key_7 Int16))) +Tuple(\n id Int8,\n key_1 Nested(key_2 Int32, key_3 Nested(key_4 Nested(key_5 Int8), key_7 Int16))) {"obj":{"id":1,"key_1":[{"key_2":100,"key_3":[{"key_4":[{"key_5":-2}],"key_7":257}]},{"key_2":65536,"key_3":[]}]}} {"obj":{"id":2,"key_1":[{"key_2":101,"key_3":[{"key_4":[{"key_5":-2}],"key_7":0}]},{"key_2":102,"key_3":[{"key_4":[],"key_7":257}]},{"key_2":65536,"key_3":[]}]}} {"obj.key_1.key_3":[[{"key_4":[{"key_5":-2}],"key_7":257}],[]]} diff --git a/tests/queries/0_stateless/01825_type_json_12.reference b/tests/queries/0_stateless/01825_type_json_12.reference index 7f4f5bf190e..ff60ba33f94 100644 --- a/tests/queries/0_stateless/01825_type_json_12.reference +++ b/tests/queries/0_stateless/01825_type_json_12.reference @@ -1,3 +1,3 @@ -Tuple(id Int8, key_0 Nested(key_1 Nested(key_3 Nested(key_4 String, key_5 Float64, key_6 String, key_7 Float64)))) +Tuple(\n id Int8,\n key_0 Nested(key_1 Nested(key_3 Nested(key_4 String, key_5 Float64, key_6 String, key_7 Float64)))) {"obj":{"id":1,"key_0":[{"key_1":[{"key_3":[{"key_4":"1048576","key_5":0.0001048576,"key_6":"25.5","key_7":1025},{"key_4":"","key_5":0,"key_6":"","key_7":2}]}]},{"key_1":[]},{"key_1":[{"key_3":[{"key_4":"","key_5":-1,"key_6":"aqbjfiruu","key_7":-922337203685477600},{"key_4":"","key_5":0,"key_6":"","key_7":65537}]},{"key_3":[{"key_4":"ghdqyeiom","key_5":1048575,"key_6":"","key_7":21474836.48}]}]}]}} [[['1048576','']],[],[['',''],['ghdqyeiom']]] [[[0.0001048576,0]],[],[[-1,0],[1048575]]] [[['25.5','']],[],[['aqbjfiruu',''],['']]] [[[1025,2]],[],[[-922337203685477600,65537],[21474836.48]]] diff --git a/tests/queries/0_stateless/01825_type_json_13.reference b/tests/queries/0_stateless/01825_type_json_13.reference index e420021f406..fa105f1a4c6 100644 --- a/tests/queries/0_stateless/01825_type_json_13.reference +++ b/tests/queries/0_stateless/01825_type_json_13.reference @@ -1,3 +1,3 @@ -Tuple(id Int8, key_1 Nested(key_2 Nested(key_3 Nested(key_4 Nested(key_5 Float64, key_6 Int64, key_7 Int32), key_8 Int32)))) +Tuple(\n id Int8,\n key_1 Nested(key_2 Nested(key_3 Nested(key_4 Nested(key_5 Float64, key_6 Int64, key_7 Int32), key_8 Int32)))) {"obj":{"id":1,"key_1":[{"key_2":[{"key_3":[{"key_4":[],"key_8":65537},{"key_4":[{"key_5":-0.02,"key_6":"0","key_7":0},{"key_5":0,"key_6":"0","key_7":1023},{"key_5":0,"key_6":"9223372036854775807","key_7":1}],"key_8":0},{"key_4":[{"key_5":0,"key_6":"0","key_7":65537}],"key_8":0}]}]}]}} [[[65537,0,0]]] [[[[],[-0.02,0,0],[0]]]] [[[[],[0,0,9223372036854775807],[0]]]] [[[[],[0,1023,1],[65537]]]] diff --git a/tests/queries/0_stateless/01825_type_json_15.reference b/tests/queries/0_stateless/01825_type_json_15.reference index ab4b1b82877..4f13731d35a 100644 --- a/tests/queries/0_stateless/01825_type_json_15.reference +++ b/tests/queries/0_stateless/01825_type_json_15.reference @@ -1,3 +1,3 @@ -Tuple(id Int8, key_0 Nested(key_0 Float64, key_1 Tuple(key_2 Array(Int8), key_8 String), key_10 Float64)) +Tuple(\n id Int8,\n key_0 Nested(key_0 Float64, key_1 Tuple(key_2 Array(Int8), key_8 String), key_10 Float64)) {"obj":{"id":1,"key_0":[{"key_0":-1,"key_1":{"key_2":[1,2,3],"key_8":"sffjx"},"key_10":65535},{"key_0":922337203.685,"key_1":{"key_2":[],"key_8":""},"key_10":10.23}]}} [[1,2,3],[]] ['sffjx',''] [65535,10.23] [-1,922337203.685] diff --git a/tests/queries/0_stateless/01825_type_json_16.reference b/tests/queries/0_stateless/01825_type_json_16.reference index f40f0d747d5..a8cc682f8e1 100644 --- a/tests/queries/0_stateless/01825_type_json_16.reference +++ b/tests/queries/0_stateless/01825_type_json_16.reference @@ -1,3 +1,3 @@ -Tuple(id Int8, key_0 Nested(key_1 Nested(key_2 Tuple(key_3 Nested(key_4 Int32, key_6 Int8, key_7 Int16), key_5 Nested(key_6 Int8, key_7 String))))) +Tuple(\n id Int8,\n key_0 Nested(key_1 Nested(key_2 Tuple(key_3 Nested(key_4 Int32, key_6 Int8, key_7 Int16), key_5 Nested(key_6 Int8, key_7 String))))) {"obj":{"id":1,"key_0":[{"key_1":[{"key_2":{"key_3":[{"key_4":255,"key_6":0,"key_7":0},{"key_4":65535,"key_6":0,"key_7":0},{"key_4":0,"key_6":3,"key_7":255}],"key_5":[{"key_6":1,"key_7":"nnpqx"},{"key_6":3,"key_7":"255"}]}}]}]}} [[[255,65535,0]]] [[[0,0,3]]] [[[0,0,255]]] [[[1,3]]] [[['nnpqx','255']]] diff --git a/tests/queries/0_stateless/01825_type_json_17.reference b/tests/queries/0_stateless/01825_type_json_17.reference index 0f97bfed5bc..c830cf41cf1 100644 --- a/tests/queries/0_stateless/01825_type_json_17.reference +++ b/tests/queries/0_stateless/01825_type_json_17.reference @@ -1,4 +1,4 @@ -Tuple(arr Nested(k1 Nested(k2 String, k3 String, k4 Int8), k5 Tuple(k6 String)), id Int8) +Tuple(\n arr Nested(k1 Nested(k2 String, k3 String, k4 Int8), k5 Tuple(k6 String)),\n id Int8) {"obj":{"arr":[{"k1":[{"k2":"aaa","k3":"bbb","k4":0},{"k2":"ccc","k3":"","k4":0}],"k5":{"k6":""}}],"id":1}} {"obj":{"arr":[{"k1":[{"k2":"","k3":"ddd","k4":10},{"k2":"","k3":"","k4":20}],"k5":{"k6":"foo"}}],"id":2}} [['bbb','']] [['aaa','ccc']] @@ -6,7 +6,7 @@ Tuple(arr Nested(k1 Nested(k2 String, k3 String, k4 Int8), k5 Tuple(k6 String)), 1 [[0,0]] [[10,20]] -Tuple(arr Nested(k1 Nested(k2 String, k3 Nested(k4 Int8))), id Int8) +Tuple(\n arr Nested(k1 Nested(k2 String, k3 Nested(k4 Int8))),\n id Int8) {"obj":{"arr":[{"k1":[{"k2":"aaa","k3":[]}]}],"id":1}} {"obj":{"arr":[{"k1":[{"k2":"bbb","k3":[{"k4":10}]},{"k2":"ccc","k3":[{"k4":20}]}]}],"id":2}} [['aaa']] [[[]]] @@ -14,7 +14,7 @@ Tuple(arr Nested(k1 Nested(k2 String, k3 Nested(k4 Int8))), id Int8) 1 [[[]]] [[[10],[20]]] -Tuple(arr Nested(k1 Nested(k2 String, k4 Nested(k5 Int8)), k3 String), id Int8) +Tuple(\n arr Nested(k1 Nested(k2 String, k4 Nested(k5 Int8)), k3 String),\n id Int8) {"obj":{"arr":[{"k1":[],"k3":"qqq"},{"k1":[],"k3":"www"}],"id":1}} {"obj":{"arr":[{"k1":[{"k2":"aaa","k4":[]}],"k3":"eee"}],"id":2}} {"obj":{"arr":[{"k1":[{"k2":"bbb","k4":[{"k5":10}]},{"k2":"ccc","k4":[{"k5":20}]}],"k3":"rrr"}],"id":3}} diff --git a/tests/queries/0_stateless/01825_type_json_18.reference b/tests/queries/0_stateless/01825_type_json_18.reference index d93f9bda63c..d61baf5eb6f 100644 --- a/tests/queries/0_stateless/01825_type_json_18.reference +++ b/tests/queries/0_stateless/01825_type_json_18.reference @@ -1,2 +1,2 @@ -1 (1) Tuple(k1 Int8) -1 ([1,2]) Tuple(k1 Array(Int8)) +1 (1) Tuple(\n k1 Int8) +1 ([1,2]) Tuple(\n k1 Array(Int8)) diff --git a/tests/queries/0_stateless/01825_type_json_2.reference b/tests/queries/0_stateless/01825_type_json_2.reference index 8524035a3a4..790d825a894 100644 --- a/tests/queries/0_stateless/01825_type_json_2.reference +++ b/tests/queries/0_stateless/01825_type_json_2.reference @@ -1,24 +1,24 @@ -1 (1,2,0) Tuple(k1 Int8, k2 Int8, k3 Int8) -2 (0,3,4) Tuple(k1 Int8, k2 Int8, k3 Int8) +1 (1,2,0) Tuple(\n k1 Int8,\n k2 Int8,\n k3 Int8) +2 (0,3,4) Tuple(\n k1 Int8,\n k2 Int8,\n k3 Int8) 1 1 2 0 2 0 3 4 -1 (1,2,'0') Tuple(k1 Int8, k2 Int8, k3 String) -2 (0,3,'4') Tuple(k1 Int8, k2 Int8, k3 String) -3 (0,0,'10') Tuple(k1 Int8, k2 Int8, k3 String) -4 (0,5,'str') Tuple(k1 Int8, k2 Int8, k3 String) +1 (1,2,'0') Tuple(\n k1 Int8,\n k2 Int8,\n k3 String) +2 (0,3,'4') Tuple(\n k1 Int8,\n k2 Int8,\n k3 String) +3 (0,0,'10') Tuple(\n k1 Int8,\n k2 Int8,\n k3 String) +4 (0,5,'str') Tuple(\n k1 Int8,\n k2 Int8,\n k3 String) 1 1 2 0 2 0 3 4 3 0 0 10 4 0 5 str ============ -1 ([1,2,3.3]) Tuple(k1 Array(Float64)) +1 ([1,2,3.3]) Tuple(\n k1 Array(Float64)) 1 [1,2,3.3] -1 (['1','2','3.3']) Tuple(k1 Array(String)) -2 (['a','4','b']) Tuple(k1 Array(String)) +1 (['1','2','3.3']) Tuple(\n k1 Array(String)) +2 (['a','4','b']) Tuple(\n k1 Array(String)) 1 ['1','2','3.3'] 2 ['a','4','b'] ============ -1 ([(11,0,0),(0,22,0)]) Tuple(k1 Nested(k2 Int8, k3 Int8, k4 Int8)) -2 ([(0,33,0),(0,0,44),(0,55,66)]) Tuple(k1 Nested(k2 Int8, k3 Int8, k4 Int8)) +1 ([(11,0,0),(0,22,0)]) Tuple(\n k1 Nested(k2 Int8, k3 Int8, k4 Int8)) +2 ([(0,33,0),(0,0,44),(0,55,66)]) Tuple(\n k1 Nested(k2 Int8, k3 Int8, k4 Int8)) 1 [11,0] [0,22] [0,0] 2 [0,0,0] [33,0,55] [0,44,66] diff --git a/tests/queries/0_stateless/01825_type_json_3.reference.j2 b/tests/queries/0_stateless/01825_type_json_3.reference.j2 index 23f38b74fd1..8646cf48872 100644 --- a/tests/queries/0_stateless/01825_type_json_3.reference.j2 +++ b/tests/queries/0_stateless/01825_type_json_3.reference.j2 @@ -1,17 +1,17 @@ {% for engine in ["ReplicatedMergeTree('/clickhouse/tables/{database}/test_01825_3/t_json_3', 'r1') ORDER BY tuple()", "Memory"] -%} -1 ('',0) Tuple(k1 String, k2 Int8) -2 ('v1',2) Tuple(k1 String, k2 Int8) +1 ('',0) Tuple(\n k1 String,\n k2 Int8) +2 ('v1',2) Tuple(\n k1 String,\n k2 Int8) 1 0 2 v1 2 ======== -1 ([]) Tuple(k1 Nested(k2 String, k3 String)) -2 ([('v1','v3'),('v4','')]) Tuple(k1 Nested(k2 String, k3 String)) +1 ([]) Tuple(\n k1 Nested(k2 String, k3 String)) +2 ([('v1','v3'),('v4','')]) Tuple(\n k1 Nested(k2 String, k3 String)) 1 [] [] 2 ['v1','v4'] ['v3',''] -1 ([]) Tuple(k1 Nested(k2 String, k3 String)) -2 ([('v1','v3'),('v4','')]) Tuple(k1 Nested(k2 String, k3 String)) -3 ([]) Tuple(k1 Nested(k2 String, k3 String)) -4 ([]) Tuple(k1 Nested(k2 String, k3 String)) +1 ([]) Tuple(\n k1 Nested(k2 String, k3 String)) +2 ([('v1','v3'),('v4','')]) Tuple(\n k1 Nested(k2 String, k3 String)) +3 ([]) Tuple(\n k1 Nested(k2 String, k3 String)) +4 ([]) Tuple(\n k1 Nested(k2 String, k3 String)) 1 [] [] 2 ['v1','v4'] ['v3',''] 3 [] [] @@ -26,9 +26,9 @@ data Tuple(k1 Nested(k2 String, k3 String)) 3 [] [] 4 [] [] ======== -1 ((1,'foo'),[]) Tuple(k1 Tuple(k2 Int8, k3 String), k4 Array(Int8)) -2 ((0,''),[1,2,3]) Tuple(k1 Tuple(k2 Int8, k3 String), k4 Array(Int8)) -3 ((10,''),[]) Tuple(k1 Tuple(k2 Int8, k3 String), k4 Array(Int8)) +1 ((1,'foo'),[]) Tuple(\n k1 Tuple(\n k2 Int8,\n k3 String),\n k4 Array(Int8)) +2 ((0,''),[1,2,3]) Tuple(\n k1 Tuple(\n k2 Int8,\n k3 String),\n k4 Array(Int8)) +3 ((10,''),[]) Tuple(\n k1 Tuple(\n k2 Int8,\n k3 String),\n k4 Array(Int8)) 1 1 foo [] 2 0 [1,2,3] 3 10 [] diff --git a/tests/queries/0_stateless/01825_type_json_4.reference b/tests/queries/0_stateless/01825_type_json_4.reference index 1b23bf2213e..58b1d067a2b 100644 --- a/tests/queries/0_stateless/01825_type_json_4.reference +++ b/tests/queries/0_stateless/01825_type_json_4.reference @@ -1,5 +1,5 @@ Code: 645 Code: 15 Code: 53 -1 ('v1') Tuple(k1 String) +1 ('v1') Tuple(\n k1 String) 1 v1 diff --git a/tests/queries/0_stateless/01825_type_json_5.reference b/tests/queries/0_stateless/01825_type_json_5.reference index 4ac0aa26ffd..3c21f2840a2 100644 --- a/tests/queries/0_stateless/01825_type_json_5.reference +++ b/tests/queries/0_stateless/01825_type_json_5.reference @@ -2,4 +2,4 @@ {"s":{"a.b":1,"a.c":2}} 1 [22,33] 2 qqq [44] -Tuple(k1 Int8, k2 Tuple(k3 String, k4 Array(Int8))) +Tuple(\n k1 Int8,\n k2 Tuple(\n k3 String,\n k4 Array(Int8))) diff --git a/tests/queries/0_stateless/01825_type_json_6.reference b/tests/queries/0_stateless/01825_type_json_6.reference index 7fcd2a40826..15e1ab3ac80 100644 --- a/tests/queries/0_stateless/01825_type_json_6.reference +++ b/tests/queries/0_stateless/01825_type_json_6.reference @@ -1,3 +1,3 @@ -Tuple(key String, out Nested(outputs Nested(index Int32, n Int8), type Int8, value Int8)) +Tuple(\n key String,\n out Nested(outputs Nested(index Int32, n Int8), type Int8, value Int8)) v1 [0,0] [1,2] [[],[1960131]] [[],[0]] v2 [1,1] [4,3] [[1881212],[]] [[1],[]] diff --git a/tests/queries/0_stateless/01825_type_json_7.reference b/tests/queries/0_stateless/01825_type_json_7.reference index 263f1688a91..cf6b32d73e8 100644 --- a/tests/queries/0_stateless/01825_type_json_7.reference +++ b/tests/queries/0_stateless/01825_type_json_7.reference @@ -1,4 +1,4 @@ -Tuple(categories Array(String), key String) +Tuple(\n categories Array(String),\n key String) v1 [] v2 ['foo','bar'] v3 [] diff --git a/tests/queries/0_stateless/01825_type_json_8.reference b/tests/queries/0_stateless/01825_type_json_8.reference index b64e6d0c9b9..27770317862 100644 --- a/tests/queries/0_stateless/01825_type_json_8.reference +++ b/tests/queries/0_stateless/01825_type_json_8.reference @@ -1,2 +1,2 @@ -([[(1,2),(3,4)],[(5,6)]]) Tuple(k1 Array(Nested(k2 Int8, k3 Int8))) -([([1,3,4,5],[6,7]),([8],[9,10,11])]) Tuple(k1 Nested(k2 Array(Int8), k3 Array(Int8))) +([[(1,2),(3,4)],[(5,6)]]) Tuple(\n k1 Array(Nested(k2 Int8, k3 Int8))) +([([1,3,4,5],[6,7]),([8],[9,10,11])]) Tuple(\n k1 Nested(k2 Array(Int8), k3 Array(Int8))) diff --git a/tests/queries/0_stateless/01825_type_json_9.reference b/tests/queries/0_stateless/01825_type_json_9.reference index a426b09a100..f58a64eda5a 100644 --- a/tests/queries/0_stateless/01825_type_json_9.reference +++ b/tests/queries/0_stateless/01825_type_json_9.reference @@ -1 +1 @@ -Tuple(foo Int8, k1 Int8, k2 Int8) +Tuple(\n foo Int8,\n k1 Int8,\n k2 Int8) diff --git a/tests/queries/0_stateless/01825_type_json_bools.reference b/tests/queries/0_stateless/01825_type_json_bools.reference index bed8c2ad2c3..6b4d2382dc2 100644 --- a/tests/queries/0_stateless/01825_type_json_bools.reference +++ b/tests/queries/0_stateless/01825_type_json_bools.reference @@ -1 +1 @@ -(1,0) Tuple(k1 UInt8, k2 UInt8) +(1,0) Tuple(\n k1 UInt8,\n k2 UInt8) diff --git a/tests/queries/0_stateless/01825_type_json_btc.reference b/tests/queries/0_stateless/01825_type_json_btc.reference index cee3b31a798..e85c0ef45bd 100644 --- a/tests/queries/0_stateless/01825_type_json_btc.reference +++ b/tests/queries/0_stateless/01825_type_json_btc.reference @@ -1,5 +1,5 @@ 100 -data Tuple(double_spend UInt8, fee Int32, hash String, inputs Nested(index Int8, prev_out Tuple(addr String, n Int16, script String, spending_outpoints Nested(n Int8, tx_index Int64), spent UInt8, tx_index Int64, type Int8, value Int64), script String, sequence Int64, witness String), lock_time Int32, out Nested(addr String, n Int8, script String, spending_outpoints Nested(n Int8, tx_index Int64), spent UInt8, tx_index Int64, type Int8, value Int64), rbf UInt8, relayed_by String, size Int16, time Int32, tx_index Int64, ver Int8, vin_sz Int8, vout_sz Int8, weight Int16) +data Tuple(\n double_spend UInt8,\n fee Int32,\n hash String,\n inputs Nested(index Int8, prev_out Tuple(addr String, n Int16, script String, spending_outpoints Nested(n Int8, tx_index Int64), spent UInt8, tx_index Int64, type Int8, value Int64), script String, sequence Int64, witness String),\n lock_time Int32,\n out Nested(addr String, n Int8, script String, spending_outpoints Nested(n Int8, tx_index Int64), spent UInt8, tx_index Int64, type Int8, value Int64),\n rbf UInt8,\n relayed_by String,\n size Int16,\n time Int32,\n tx_index Int64,\n ver Int8,\n vin_sz Int8,\n vout_sz Int8,\n weight Int16) 8174.56 2680 2.32 1 [[],[(0,359661801933760)]] diff --git a/tests/queries/0_stateless/01825_type_json_describe.reference b/tests/queries/0_stateless/01825_type_json_describe.reference index 629b60cb629..98b2bf8be83 100644 --- a/tests/queries/0_stateless/01825_type_json_describe.reference +++ b/tests/queries/0_stateless/01825_type_json_describe.reference @@ -1,3 +1,3 @@ data Object(\'json\') -data Tuple(k1 Int8) -data Tuple(k1 String, k2 Array(Int8)) +data Tuple(\n k1 Int8) +data Tuple(\n k1 String,\n k2 Array(Int8)) diff --git a/tests/queries/0_stateless/01825_type_json_distributed.reference b/tests/queries/0_stateless/01825_type_json_distributed.reference index 9ae85ac888c..9735fec2fe5 100644 --- a/tests/queries/0_stateless/01825_type_json_distributed.reference +++ b/tests/queries/0_stateless/01825_type_json_distributed.reference @@ -1,4 +1,4 @@ -(2,('qqq',[44,55])) Tuple(k1 Int8, k2 Tuple(k3 String, k4 Array(Int8))) -(2,('qqq',[44,55])) Tuple(k1 Int8, k2 Tuple(k3 String, k4 Array(Int8))) +(2,('qqq',[44,55])) Tuple(\n k1 Int8,\n k2 Tuple(\n k3 String,\n k4 Array(Int8))) +(2,('qqq',[44,55])) Tuple(\n k1 Int8,\n k2 Tuple(\n k3 String,\n k4 Array(Int8))) 2 qqq [44,55] 2 qqq [44,55] diff --git a/tests/queries/0_stateless/01825_type_json_field.reference b/tests/queries/0_stateless/01825_type_json_field.reference index b5637b1fbb7..8afd0110b63 100644 --- a/tests/queries/0_stateless/01825_type_json_field.reference +++ b/tests/queries/0_stateless/01825_type_json_field.reference @@ -1,12 +1,12 @@ 1 10 a -Tuple(a UInt8, s String) +Tuple(\n a UInt8,\n s String) 1 10 a 0 2 sss b 300 3 20 c 0 -Tuple(a String, b UInt16, s String) +Tuple(\n a String,\n b UInt16,\n s String) 1 10 a 0 2 sss b 300 3 20 c 0 4 30 400 5 0 qqq 0 foo -Tuple(a String, b UInt16, s String, t String) +Tuple(\n a String,\n b UInt16,\n s String,\n t String) diff --git a/tests/queries/0_stateless/01825_type_json_from_map.reference b/tests/queries/0_stateless/01825_type_json_from_map.reference index dbcf67faef3..90680ee383b 100644 --- a/tests/queries/0_stateless/01825_type_json_from_map.reference +++ b/tests/queries/0_stateless/01825_type_json_from_map.reference @@ -1,4 +1,4 @@ 800000 2000000 1400000 900000 800000 2000000 1400000 900000 -Tuple(col0 UInt64, col1 UInt64, col2 UInt64, col3 UInt64, col4 UInt64, col5 UInt64, col6 UInt64, col7 UInt64, col8 UInt64) +Tuple(\n col0 UInt64,\n col1 UInt64,\n col2 UInt64,\n col3 UInt64,\n col4 UInt64,\n col5 UInt64,\n col6 UInt64,\n col7 UInt64,\n col8 UInt64) 1600000 4000000 2800000 1800000 diff --git a/tests/queries/0_stateless/01825_type_json_in_array.reference b/tests/queries/0_stateless/01825_type_json_in_array.reference index c36a22e6951..82207f53a21 100644 --- a/tests/queries/0_stateless/01825_type_json_in_array.reference +++ b/tests/queries/0_stateless/01825_type_json_in_array.reference @@ -5,7 +5,7 @@ {"arr":{"k1":1,"k2":{"k3":2,"k4":3,"k5":""}}} {"arr":{"k1":2,"k2":{"k3":0,"k4":0,"k5":"foo"}}} {"arr":{"k1":3,"k2":{"k3":4,"k4":5,"k5":""}}} -Array(Tuple(k1 Int8, k2 Tuple(k3 Int8, k4 Int8, k5 String))) +Array(Tuple(\n k1 Int8,\n k2 Tuple(\n k3 Int8,\n k4 Int8,\n k5 String))) {"id":1,"arr":[{"k1":[{"k2":"aaa","k3":"bbb","k4":0},{"k2":"ccc","k3":"","k4":0}],"k5":{"k6":""}}]} {"id":2,"arr":[{"k1":[{"k2":"","k3":"ddd","k4":10},{"k2":"","k3":"","k4":20}],"k5":{"k6":"foo"}}]} 1 [['aaa','ccc']] [['bbb','']] [[0,0]] [''] @@ -14,7 +14,7 @@ Array(Tuple(k1 Int8, k2 Tuple(k3 Int8, k4 Int8, k5 String))) {"k1":{"k2":"","k3":"ddd","k4":10}} {"k1":{"k2":"aaa","k3":"bbb","k4":0}} {"k1":{"k2":"ccc","k3":"","k4":0}} -Tuple(k2 String, k3 String, k4 Int8) +Tuple(\n k2 String,\n k3 String,\n k4 Int8) {"arr":[{"x":1}]} {"arr":{"x":{"y":1},"t":{"y":2}}} {"arr":[1,{"y":1}]} diff --git a/tests/queries/0_stateless/01825_type_json_in_other_types.reference b/tests/queries/0_stateless/01825_type_json_in_other_types.reference index b94885a65ab..fa8af729cc7 100644 --- a/tests/queries/0_stateless/01825_type_json_in_other_types.reference +++ b/tests/queries/0_stateless/01825_type_json_in_other_types.reference @@ -1,4 +1,4 @@ -Tuple(String, Map(String, Array(Tuple(k1 Nested(k2 Int8, k3 Int8, k5 String), k4 String))), Tuple(k1 String, k2 Tuple(k3 String, k4 String))) +Tuple(String, Map(String, Array(Tuple(\n k1 Nested(k2 Int8, k3 Int8, k5 String),\n k4 String))), Tuple(\n k1 String,\n k2 Tuple(\n k3 String,\n k4 String))) ============= {"id":1,"data":["foo",{"aa":[{"k1":[{"k2":1,"k3":2,"k5":""},{"k2":0,"k3":3,"k5":""}],"k4":""},{"k1":[{"k2":4,"k3":0,"k5":""},{"k2":0,"k3":5,"k5":""},{"k2":6,"k3":0,"k5":""}],"k4":"qqq"}],"bb":[{"k1":[],"k4":"www"},{"k1":[{"k2":7,"k3":8,"k5":""},{"k2":9,"k3":10,"k5":""},{"k2":11,"k3":12,"k5":""}],"k4":""}]},{"k1":"aa","k2":{"k3":"bb","k4":"c"}}]} {"id":2,"data":["bar",{"aa":[{"k1":[{"k2":13,"k3":14,"k5":""},{"k2":15,"k3":16,"k5":""}],"k4":"www"}]},{"k1":"","k2":{"k3":"","k4":""}}]} diff --git a/tests/queries/0_stateless/01825_type_json_insert_select.reference b/tests/queries/0_stateless/01825_type_json_insert_select.reference index 6778da508f2..cb46a9c607e 100644 --- a/tests/queries/0_stateless/01825_type_json_insert_select.reference +++ b/tests/queries/0_stateless/01825_type_json_insert_select.reference @@ -1,10 +1,10 @@ -Tuple(k1 Int8, k2 String) +Tuple(\n k1 Int8,\n k2 String) 1 (1,'foo') -Tuple(k1 Int8, k2 String, k3 String) +Tuple(\n k1 Int8,\n k2 String,\n k3 String) 1 (1,'foo','') 2 (2,'bar','') 3 (3,'','aaa') -Tuple(arr Nested(k11 Int8, k22 String, k33 Int8), k1 Int8, k2 String, k3 String) +Tuple(\n arr Nested(k11 Int8, k22 String, k33 Int8),\n k1 Int8,\n k2 String,\n k3 String) 1 ([],1,'foo','') 2 ([],2,'bar','') 3 ([],3,'','aaa') @@ -12,7 +12,7 @@ Tuple(arr Nested(k11 Int8, k22 String, k33 Int8), k1 Int8, k2 String, k3 String) 5 ([(0,'str1',0)],0,'','') {"data":{"k1":1,"k10":[{"a":"1","b":"2","c":{"k11":""}},{"a":"2","b":"3","c":{"k11":""}}]}} {"data":{"k1":2,"k10":[{"a":"1","b":"2","c":{"k11":"haha"}}]}} -Tuple(k1 Int8, k10 Nested(a String, b String, c Tuple(k11 String))) +Tuple(\n k1 Int8,\n k10 Nested(a String, b String, c Tuple(k11 String))) {"data":{"k1":1,"k10":[{"a":"1","b":"2","c":{"k11":""}},{"a":"2","b":"3","c":{"k11":""}}]}} {"data":{"k1":2,"k10":[{"a":"1","b":"2","c":{"k11":"haha"}}]}} -Tuple(k1 Int8, k10 Nested(a String, b String, c Tuple(k11 String))) +Tuple(\n k1 Int8,\n k10 Nested(a String, b String, c Tuple(k11 String))) diff --git a/tests/queries/0_stateless/01825_type_json_missed_values.reference b/tests/queries/0_stateless/01825_type_json_missed_values.reference index b480493995b..2a4b3a6f671 100644 --- a/tests/queries/0_stateless/01825_type_json_missed_values.reference +++ b/tests/queries/0_stateless/01825_type_json_missed_values.reference @@ -1,2 +1,2 @@ -Tuple(foo Int8, k1 Int8, k2 Int8) +Tuple(\n foo Int8,\n k1 Int8,\n k2 Int8) 1 diff --git a/tests/queries/0_stateless/01825_type_json_multiple_files.reference b/tests/queries/0_stateless/01825_type_json_multiple_files.reference index b887abc8590..6dcdb00e139 100644 --- a/tests/queries/0_stateless/01825_type_json_multiple_files.reference +++ b/tests/queries/0_stateless/01825_type_json_multiple_files.reference @@ -4,11 +4,11 @@ {"data":{"k0":0,"k1":0,"k2":0,"k3":100,"k4":0,"k5":0}} {"data":{"k0":0,"k1":0,"k2":0,"k3":0,"k4":100,"k5":0}} {"data":{"k0":0,"k1":0,"k2":0,"k3":0,"k4":0,"k5":100}} -Tuple(k0 Int8, k1 Int8, k2 Int8, k3 Int8, k4 Int8, k5 Int8) +Tuple(\n k0 Int8,\n k1 Int8,\n k2 Int8,\n k3 Int8,\n k4 Int8,\n k5 Int8) {"data":{"k0":100,"k1":0,"k2":0}} {"data":{"k0":0,"k1":100,"k2":0}} {"data":{"k0":0,"k1":0,"k2":100}} -Tuple(k0 Int8, k1 Int8, k2 Int8) +Tuple(\n k0 Int8,\n k1 Int8,\n k2 Int8) {"data":{"k1":100,"k3":0}} {"data":{"k1":0,"k3":100}} -Tuple(k1 Int8, k3 Int8) +Tuple(\n k1 Int8,\n k3 Int8) diff --git a/tests/queries/0_stateless/01825_type_json_nbagames.reference b/tests/queries/0_stateless/01825_type_json_nbagames.reference index 5aa63dceb86..70df8f967f3 100644 --- a/tests/queries/0_stateless/01825_type_json_nbagames.reference +++ b/tests/queries/0_stateless/01825_type_json_nbagames.reference @@ -1,5 +1,5 @@ 1000 -data Tuple(_id Tuple(`$oid` String), date Tuple(`$date` String), teams Nested(abbreviation String, city String, home UInt8, name String, players Nested(ast Int8, blk Int8, drb Int8, fg Int8, fg3 Int8, fg3_pct String, fg3a Int8, fg_pct String, fga Int8, ft Int8, ft_pct String, fta Int8, mp String, orb Int8, pf Int8, player String, pts Int8, stl Int8, tov Int8, trb Int8), results Tuple(ast Int8, blk Int8, drb Int8, fg Int8, fg3 Int8, fg3_pct String, fg3a Int8, fg_pct String, fga Int8, ft Int8, ft_pct String, fta Int8, mp Int16, orb Int8, pf Int8, pts Int16, stl Int8, tov Int8, trb Int8), score Int16, won Int8)) +data Tuple(\n _id Tuple(\n `$oid` String),\n date Tuple(\n `$date` String),\n teams Nested(abbreviation String, city String, home UInt8, name String, players Nested(ast Int8, blk Int8, drb Int8, fg Int8, fg3 Int8, fg3_pct String, fg3a Int8, fg_pct String, fga Int8, ft Int8, ft_pct String, fta Int8, mp String, orb Int8, pf Int8, player String, pts Int8, stl Int8, tov Int8, trb Int8), results Tuple(ast Int8, blk Int8, drb Int8, fg Int8, fg3 Int8, fg3_pct String, fg3a Int8, fg_pct String, fga Int8, ft Int8, ft_pct String, fta Int8, mp Int16, orb Int8, pf Int8, pts Int16, stl Int8, tov Int8, trb Int8), score Int16, won Int8)) Boston Celtics 70 Los Angeles Lakers 64 Milwaukee Bucks 61 diff --git a/tests/queries/0_stateless/01825_type_json_nullable.reference b/tests/queries/0_stateless/01825_type_json_nullable.reference index 587fb1b1bc9..597ede47615 100644 --- a/tests/queries/0_stateless/01825_type_json_nullable.reference +++ b/tests/queries/0_stateless/01825_type_json_nullable.reference @@ -1,17 +1,17 @@ -1 (1,2,NULL) Tuple(k1 Nullable(Int8), k2 Nullable(Int8), k3 Nullable(Int8)) -2 (NULL,3,4) Tuple(k1 Nullable(Int8), k2 Nullable(Int8), k3 Nullable(Int8)) +1 (1,2,NULL) Tuple(\n k1 Nullable(Int8),\n k2 Nullable(Int8),\n k3 Nullable(Int8)) +2 (NULL,3,4) Tuple(\n k1 Nullable(Int8),\n k2 Nullable(Int8),\n k3 Nullable(Int8)) 1 1 2 \N 2 \N 3 4 -1 (1,2,NULL) Tuple(k1 Nullable(Int8), k2 Nullable(Int8), k3 Nullable(String)) -2 (NULL,3,'4') Tuple(k1 Nullable(Int8), k2 Nullable(Int8), k3 Nullable(String)) -3 (NULL,NULL,'10') Tuple(k1 Nullable(Int8), k2 Nullable(Int8), k3 Nullable(String)) -4 (NULL,5,'str') Tuple(k1 Nullable(Int8), k2 Nullable(Int8), k3 Nullable(String)) +1 (1,2,NULL) Tuple(\n k1 Nullable(Int8),\n k2 Nullable(Int8),\n k3 Nullable(String)) +2 (NULL,3,'4') Tuple(\n k1 Nullable(Int8),\n k2 Nullable(Int8),\n k3 Nullable(String)) +3 (NULL,NULL,'10') Tuple(\n k1 Nullable(Int8),\n k2 Nullable(Int8),\n k3 Nullable(String)) +4 (NULL,5,'str') Tuple(\n k1 Nullable(Int8),\n k2 Nullable(Int8),\n k3 Nullable(String)) 1 1 2 \N 2 \N 3 4 3 \N \N 10 4 \N 5 str ============ -1 ([(11,NULL,NULL),(NULL,22,NULL)]) Tuple(k1 Nested(k2 Nullable(Int8), k3 Nullable(Int8), k4 Nullable(Int8))) -2 ([(NULL,33,NULL),(NULL,NULL,44),(NULL,55,66)]) Tuple(k1 Nested(k2 Nullable(Int8), k3 Nullable(Int8), k4 Nullable(Int8))) +1 ([(11,NULL,NULL),(NULL,22,NULL)]) Tuple(\n k1 Nested(k2 Nullable(Int8), k3 Nullable(Int8), k4 Nullable(Int8))) +2 ([(NULL,33,NULL),(NULL,NULL,44),(NULL,55,66)]) Tuple(\n k1 Nested(k2 Nullable(Int8), k3 Nullable(Int8), k4 Nullable(Int8))) 1 [11,NULL] [NULL,22] [NULL,NULL] 2 [NULL,NULL,NULL] [33,NULL,55] [NULL,44,66] diff --git a/tests/queries/0_stateless/01825_type_json_parallel_insert.reference b/tests/queries/0_stateless/01825_type_json_parallel_insert.reference index 158d61d46f7..e93e0aeb956 100644 --- a/tests/queries/0_stateless/01825_type_json_parallel_insert.reference +++ b/tests/queries/0_stateless/01825_type_json_parallel_insert.reference @@ -1 +1 @@ -Tuple(k1 Int8, k2 String) 500000 +Tuple(\n k1 Int8,\n k2 String) 500000 diff --git a/tests/queries/0_stateless/01825_type_json_schema_inference.reference b/tests/queries/0_stateless/01825_type_json_schema_inference.reference index a1dd269f9b4..72e3b58b8a8 100644 --- a/tests/queries/0_stateless/01825_type_json_schema_inference.reference +++ b/tests/queries/0_stateless/01825_type_json_schema_inference.reference @@ -1,5 +1,5 @@ {"id":"1","obj":{"k1":1,"k2":{"k3":"2","k4":[{"k5":3,"k6":null},{"k5":4,"k6":null}]},"some":null},"s":"foo"} {"id":"2","obj":{"k1":null,"k2":{"k3":"str","k4":[{"k5":null,"k6":55}]},"some":42},"s":"bar"} -Tuple(k1 Nullable(Int8), k2 Tuple(k3 Nullable(String), k4 Nested(k5 Nullable(Int8), k6 Nullable(Int8))), some Nullable(Int8)) +Tuple(\n k1 Nullable(Int8),\n k2 Tuple(\n k3 Nullable(String),\n k4 Nested(k5 Nullable(Int8), k6 Nullable(Int8))),\n some Nullable(Int8)) {"id":"1","obj":"aaa","s":"foo"} {"id":"2","obj":"bbb","s":"bar"} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index fafefd72cb8..bd20d34b684 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -30,7 +30,7 @@ SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == SELECT '--'; SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2; -SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND 0; -- { serverError INVALID_JOIN_ON_EXPRESSION,NOT_FOUND_COLUMN_IN_BLOCK } SELECT '--'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id > 2; diff --git a/tests/queries/0_stateless/02026_describe_include_subcolumns.reference b/tests/queries/0_stateless/02026_describe_include_subcolumns.reference index ba792ea9f74..ac114a03837 100644 --- a/tests/queries/0_stateless/02026_describe_include_subcolumns.reference +++ b/tests/queries/0_stateless/02026_describe_include_subcolumns.reference @@ -1,23 +1,33 @@ -┌─name─┬─type────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┐ -│ d │ Date │ │ │ │ │ │ -│ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ -│ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ -│ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ -│ t │ Tuple(s String, a Array(Tuple(a UInt32, b UInt32))) │ │ │ │ ZSTD(1) │ │ -└──────┴─────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┘ -┌─name───────┬─type────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┬─is_subcolumn─┐ -│ d │ Date │ │ │ │ │ │ 0 │ -│ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ 0 │ -│ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 0 │ -│ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ 0 │ -│ t │ Tuple(s String, a Array(Tuple(a UInt32, b UInt32))) │ │ │ │ ZSTD(1) │ │ 0 │ -│ n.null │ UInt8 │ │ │ It is a nullable column │ │ │ 1 │ -│ arr1.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ arr2.size0 │ UInt64 │ │ │ │ │ d + toIntervalDay(1) │ 1 │ -│ arr2.size1 │ Array(UInt64) │ │ │ │ │ d + toIntervalDay(1) │ 1 │ -│ t.s │ String │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.a │ Array(Tuple(a UInt32, b UInt32)) │ │ │ │ │ │ 1 │ -│ t.a.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ t.a.a │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.a.b │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ -└────────────┴─────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┴──────────────┘ +┌─name─┬─type──────────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┐ +│ d │ Date │ │ │ │ │ │ +│ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ +│ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ +│ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ +│ t │ Tuple( + s String, + a Array(Tuple( + a UInt32, + b UInt32))) │ │ │ │ ZSTD(1) │ │ +└──────┴───────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┘ +┌─name───────┬─type──────────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┬─is_subcolumn─┐ +│ d │ Date │ │ │ │ │ │ 0 │ +│ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ 0 │ +│ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 0 │ +│ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ 0 │ +│ t │ Tuple( + s String, + a Array(Tuple( + a UInt32, + b UInt32))) │ │ │ │ ZSTD(1) │ │ 0 │ +│ n.null │ UInt8 │ │ │ It is a nullable column │ │ │ 1 │ +│ arr1.size0 │ UInt64 │ │ │ │ │ │ 1 │ +│ arr2.size0 │ UInt64 │ │ │ │ │ d + toIntervalDay(1) │ 1 │ +│ arr2.size1 │ Array(UInt64) │ │ │ │ │ d + toIntervalDay(1) │ 1 │ +│ t.s │ String │ │ │ │ ZSTD(1) │ │ 1 │ +│ t.a │ Array(Tuple( + a UInt32, + b UInt32)) │ │ │ │ │ │ 1 │ +│ t.a.size0 │ UInt64 │ │ │ │ │ │ 1 │ +│ t.a.a │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ +│ t.a.b │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ +└────────────┴───────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┴──────────────┘ diff --git a/tests/queries/0_stateless/02030_tuple_filter.sql b/tests/queries/0_stateless/02030_tuple_filter.sql index f2fc3a30aa6..1b79ad6c83c 100644 --- a/tests/queries/0_stateless/02030_tuple_filter.sql +++ b/tests/queries/0_stateless/02030_tuple_filter.sql @@ -33,6 +33,7 @@ SET force_primary_key = 0; SELECT * FROM test_tuple_filter WHERE (1, value) = (id, 'A'); SELECT * FROM test_tuple_filter WHERE tuple(id) = tuple(1); +SELECT * FROM test_tuple_filter WHERE (id, (id, id) = (1, NULL)) == (NULL, NULL); SELECT * FROM test_tuple_filter WHERE (log_date, value) = tuple('2021-01-01'); -- { serverError 43 } SELECT * FROM test_tuple_filter WHERE (id, value) = tuple(1); -- { serverError 43 } diff --git a/tests/queries/0_stateless/02047_log_family_complex_structs_data_file_dumps.sh b/tests/queries/0_stateless/02047_log_family_complex_structs_data_file_dumps.sh index 015a162221d..55c01e63294 100755 --- a/tests/queries/0_stateless/02047_log_family_complex_structs_data_file_dumps.sh +++ b/tests/queries/0_stateless/02047_log_family_complex_structs_data_file_dumps.sh @@ -11,7 +11,7 @@ do echo "$engine:" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS tbl" - $CLICKHOUSE_CLIENT --query="CREATE TABLE tbl(x Array(Array(Int32)), y Array(Tuple(z String, w Float32))) ENGINE=$engine" + $CLICKHOUSE_CLIENT --query="CREATE TABLE tbl(x Array(Array(Int32)), y Nested(z String, w Float32)) ENGINE=$engine" data_dir=$($CLICKHOUSE_CLIENT --query="SELECT data_paths[1] FROM system.tables WHERE name='tbl' AND database=currentDatabase()") echo "empty:" diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference index 8a34751b071..2631199cbab 100644 --- a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference @@ -2,7 +2,7 @@ execute: --allow_experimental_analyzer=1 "foo" 1 execute: --allow_experimental_analyzer=1 --stage fetch_columns -"dummy_0" +"__table1.dummy" 0 execute: --allow_experimental_analyzer=1 --stage with_mergeable_state "1_UInt8" diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.reference b/tests/queries/0_stateless/02051_symlinks_to_user_files.reference index d86bac9de59..2c94e483710 100644 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.reference +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.reference @@ -1 +1,2 @@ OK +OK diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh index 0af71e4deee..eab44e74d88 100755 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh @@ -8,13 +8,14 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # See 01658_read_file_to_string_column.sh user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -FILE_PATH="${user_files_path}/file/" +FILE_PATH="${user_files_path}/file" mkdir -p ${FILE_PATH} chmod 777 ${FILE_PATH} FILE="test_symlink_${CLICKHOUSE_DATABASE}" symlink_path=${FILE_PATH}/${FILE} +symlink_path_with_regex="${FILE_PATH}*/${FILE}" file_path=$CUR_DIR/${FILE} touch ${file_path} @@ -29,3 +30,4 @@ trap cleanup EXIT ${CLICKHOUSE_CLIENT} --query="insert into table function file('${symlink_path}', 'Values', 'a String') select 'OK'"; ${CLICKHOUSE_CLIENT} --query="select * from file('${symlink_path}', 'Values', 'a String') order by a"; +${CLICKHOUSE_CLIENT} --query="select * from file('${symlink_path_with_regex}', 'Values', 'a String') order by a"; diff --git a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference index 4c1d5dc829f..beda9e36223 100644 --- a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference +++ b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference @@ -1,2 +1,2 @@ -CREATE TABLE _local.table\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\') +CREATE TABLE default.table\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\') CREATE TABLE foo.table\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\') diff --git a/tests/queries/0_stateless/02149_external_schema_inference.reference b/tests/queries/0_stateless/02149_external_schema_inference.reference index ebc30e874da..194c8ca62cb 100644 --- a/tests/queries/0_stateless/02149_external_schema_inference.reference +++ b/tests/queries/0_stateless/02149_external_schema_inference.reference @@ -31,8 +31,8 @@ lotteryWin Float64 someRatio Float32 temperature Float32 randomBigNumber Int64 -measureUnits Array(Tuple(unit String, coef Float32)) -nestiness_a_b_c Tuple(d UInt32, e Array(UInt32)) +measureUnits Array(Tuple(\n unit String,\n coef Float32)) +nestiness_a_b_c Tuple(\n d UInt32,\n e Array(UInt32)) location Array(Int32) pi Float32 @@ -78,8 +78,8 @@ lotteryWin String someRatio String temperature String randomBigNumber String -measureUnits Tuple(unit Array(String), coef Array(String)) -nestiness_a_b_c Tuple(d String, e Array(String)) +measureUnits Tuple(\n unit Array(String),\n coef Array(String)) +nestiness_a_b_c Tuple(\n d String,\n e Array(String)) uuid String name String @@ -101,14 +101,14 @@ lotteryWin Float64 someRatio Float32 temperature Float32 randomBigNumber Int64 -measureunits Tuple(coef Array(Float32), unit Array(String)) -nestiness_a_b_c Tuple(d UInt32, e Array(UInt32)) +measureunits Tuple(\n coef Array(Float32),\n unit Array(String)) +nestiness_a_b_c Tuple(\n d UInt32,\n e Array(UInt32)) newFieldStr String newFieldInt Int32 newBool UInt8 identifier String -modules Array(Tuple(module_id UInt32, supply UInt32, temp UInt32, nodes Array(Tuple(node_id UInt32, opening_time UInt32, closing_time UInt32, current UInt32, coords_y Float32)))) +modules Array(Tuple(\n module_id UInt32,\n supply UInt32,\n temp UInt32,\n nodes Array(Tuple(\n node_id UInt32,\n opening_time UInt32,\n closing_time UInt32,\n current UInt32,\n coords_y Float32)))) Capnproto @@ -123,15 +123,15 @@ lc2 Nullable(String) lc3 Array(Nullable(String)) value UInt64 -nested Tuple(a Tuple(b UInt64, c Array(Array(UInt64))), d Array(Tuple(e Array(Array(Tuple(f UInt64, g UInt64))), h Array(Tuple(k Array(UInt64)))))) +nested Tuple(\n a Tuple(\n b UInt64,\n c Array(Array(UInt64))),\n d Array(Tuple(\n e Array(Array(Tuple(\n f UInt64,\n g UInt64))),\n h Array(Tuple(\n k Array(UInt64)))))) -nested Tuple(value Array(UInt64), array Array(Array(UInt64)), tuple Array(Tuple(one UInt64, two UInt64))) +nested Tuple(\n value Array(UInt64),\n array Array(Array(UInt64)),\n tuple Array(Tuple(\n one UInt64,\n two UInt64))) -a Tuple(b UInt64, c Tuple(d UInt64, e Tuple(f UInt64))) +a Tuple(\n b UInt64,\n c Tuple(\n d UInt64,\n e Tuple(\n f UInt64))) nullable Nullable(UInt64) array Array(Nullable(UInt64)) -tuple Tuple(nullable Nullable(UInt64)) +tuple Tuple(\n nullable Nullable(UInt64)) int8 Int8 uint8 UInt8 @@ -151,8 +151,8 @@ datetime UInt32 datetime64 Int64 value UInt64 -tuple1 Tuple(one UInt64, two Tuple(three UInt64, four UInt64)) -tuple2 Tuple(nested1 Tuple(nested2 Tuple(x UInt64))) +tuple1 Tuple(\n one UInt64,\n two Tuple(\n three UInt64,\n four UInt64)) +tuple2 Tuple(\n nested1 Tuple(\n nested2 Tuple(\n x UInt64))) RawBLOB diff --git a/tests/queries/0_stateless/02149_schema_inference.reference b/tests/queries/0_stateless/02149_schema_inference.reference index 6d70c4682f5..ca634ac1701 100644 --- a/tests/queries/0_stateless/02149_schema_inference.reference +++ b/tests/queries/0_stateless/02149_schema_inference.reference @@ -37,30 +37,30 @@ d Array(Nullable(Int64)) JSONCompactEachRow c1 Nullable(Float64) c2 Array(Tuple(Nullable(Int64), Nullable(String))) -c3 Tuple(key Nullable(Int64), key2 Nullable(Int64)) +c3 Tuple(\n key Nullable(Int64),\n key2 Nullable(Int64)) c4 Nullable(Bool) 42.42 [(1,'String'),(2,'abcd')] (42,24) true c1 Nullable(Int64) c2 Array(Tuple(Nullable(Int64), Nullable(String))) -c3 Tuple(key1 Nullable(Int64), key2 Nullable(Int64)) +c3 Tuple(\n key1 Nullable(Int64),\n key2 Nullable(Int64)) c4 Nullable(Bool) \N [(1,'String'),(2,NULL)] (NULL,24) \N 32 [(2,'String 2'),(3,'hello')] (4242,2424) true JSONCompactEachRowWithNames a Nullable(Float64) b Array(Tuple(Nullable(Int64), Nullable(String))) -c Tuple(key Nullable(Int64), key2 Nullable(Int64)) +c Tuple(\n key Nullable(Int64),\n key2 Nullable(Int64)) d Nullable(Bool) 42.42 [(1,'String'),(2,'abcd')] (42,24) true JSONEachRow a Nullable(Float64) b Array(Tuple(Nullable(Int64), Nullable(String))) -c Tuple(key Nullable(Int64), key2 Nullable(Int64)) +c Tuple(\n key Nullable(Int64),\n key2 Nullable(Int64)) d Nullable(Bool) 42.42 [(1,'String'),(2,'abcd')] (42,24) true a Nullable(Int64) b Array(Tuple(Nullable(Int64), Nullable(String))) -c Tuple(key1 Nullable(Int64), key2 Nullable(Int64)) +c Tuple(\n key1 Nullable(Int64),\n key2 Nullable(Int64)) d Nullable(Bool) \N [(1,'String'),(2,NULL)] (NULL,24) \N 32 [(2,'String 2'),(3,'hello')] (4242,2424) true diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.reference index 4e020427ad0..ee83ed63dc1 100644 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.reference +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.reference @@ -24,12 +24,12 @@ fixed_string Nullable(FixedString(3)) Str: 0 100 Str: 1 200 array Array(Nullable(UInt64)) -tuple Tuple(`1` Nullable(UInt64), `2` Nullable(String)) +tuple Tuple(\n `1` Nullable(UInt64),\n `2` Nullable(String)) map Map(String, Nullable(UInt64)) [0,1] (0,'0') {'0':0} [1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`1` Array(Nullable(UInt64)), `2` Map(String, Nullable(UInt64)))) -nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(UInt64))), `2` Map(UInt64, Array(Tuple(`1` Nullable(UInt64), `2` Nullable(String))))), `2` Nullable(UInt8)) +nested1 Array(Tuple(\n `1` Array(Nullable(UInt64)),\n `2` Map(String, Nullable(UInt64)))) +nested2 Tuple(\n `1` Tuple(\n `1` Array(Array(Nullable(UInt64))),\n `2` Map(UInt64, Array(Tuple(\n `1` Nullable(UInt64),\n `2` Nullable(String))))),\n `2` Nullable(UInt8)) [([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) [([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) ArrowStream @@ -58,12 +58,12 @@ fixed_string Nullable(FixedString(3)) Str: 0 100 Str: 1 200 array Array(Nullable(UInt64)) -tuple Tuple(`1` Nullable(UInt64), `2` Nullable(String)) +tuple Tuple(\n `1` Nullable(UInt64),\n `2` Nullable(String)) map Map(String, Nullable(UInt64)) [0,1] (0,'0') {'0':0} [1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`1` Array(Nullable(UInt64)), `2` Map(String, Nullable(UInt64)))) -nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(UInt64))), `2` Map(UInt64, Array(Tuple(`1` Nullable(UInt64), `2` Nullable(String))))), `2` Nullable(UInt8)) +nested1 Array(Tuple(\n `1` Array(Nullable(UInt64)),\n `2` Map(String, Nullable(UInt64)))) +nested2 Tuple(\n `1` Tuple(\n `1` Array(Array(Nullable(UInt64))),\n `2` Map(UInt64, Array(Tuple(\n `1` Nullable(UInt64),\n `2` Nullable(String))))),\n `2` Nullable(UInt8)) [([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) [([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) Parquet @@ -92,12 +92,12 @@ fixed_string Nullable(FixedString(3)) Str: 0 100 Str: 1 200 array Array(Nullable(UInt64)) -tuple Tuple(`1` Nullable(UInt64), `2` Nullable(String)) +tuple Tuple(\n `1` Nullable(UInt64),\n `2` Nullable(String)) map Map(String, Nullable(UInt64)) [0,1] (0,'0') {'0':0} [1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`1` Array(Nullable(UInt64)), `2` Map(String, Nullable(UInt64)))) -nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(UInt64))), `2` Map(UInt64, Array(Tuple(`1` Nullable(UInt64), `2` Nullable(String))))), `2` Nullable(UInt8)) +nested1 Array(Tuple(\n `1` Array(Nullable(UInt64)),\n `2` Map(String, Nullable(UInt64)))) +nested2 Tuple(\n `1` Tuple(\n `1` Array(Array(Nullable(UInt64))),\n `2` Map(UInt64, Array(Tuple(\n `1` Nullable(UInt64),\n `2` Nullable(String))))),\n `2` Nullable(UInt8)) [([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) [([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) ORC @@ -126,12 +126,12 @@ fixed_string Nullable(String) Str: 0 100 Str: 1 200 array Array(Nullable(Int64)) -tuple Tuple(`1` Nullable(Int64), `2` Nullable(String)) +tuple Tuple(\n `1` Nullable(Int64),\n `2` Nullable(String)) map Map(String, Nullable(Int64)) [0,1] (0,'0') {'0':0} [1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`1` Array(Nullable(Int64)), `2` Map(String, Nullable(Int64)))) -nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(Int64))), `2` Map(Int64, Array(Tuple(`1` Nullable(Int64), `2` Nullable(String))))), `2` Nullable(Int8)) +nested1 Array(Tuple(\n `1` Array(Nullable(Int64)),\n `2` Map(String, Nullable(Int64)))) +nested2 Tuple(\n `1` Tuple(\n `1` Array(Array(Nullable(Int64))),\n `2` Map(Int64, Array(Tuple(\n `1` Nullable(Int64),\n `2` Nullable(String))))),\n `2` Nullable(Int8)) [([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) [([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) Native diff --git a/tests/queries/0_stateless/02179_map_cast_to_array.reference b/tests/queries/0_stateless/02179_map_cast_to_array.reference index 81bb9fba537..e87d1c69c1b 100644 --- a/tests/queries/0_stateless/02179_map_cast_to_array.reference +++ b/tests/queries/0_stateless/02179_map_cast_to_array.reference @@ -6,4 +6,4 @@ {1:{1:'1234'}} [(1,{1:1234})] [(1,{1:1234})] {1:{1:'1234'}} [(1,[(1,'1234')])] [(1,[(1,'1234')])] {1:{1:'1234'}} [(1,[(1,1234)])] [(1,[(1,1234)])] -[(1,'val1'),(2,'val2')] Array(Tuple(k UInt32, v String)) +[(1,'val1'),(2,'val2')] Array(Tuple(\n k UInt32,\n v String)) diff --git a/tests/queries/0_stateless/02226_analyzer_or_like_combine.reference b/tests/queries/0_stateless/02226_analyzer_or_like_combine.reference index d741391067c..0ff24b39709 100644 --- a/tests/queries/0_stateless/02226_analyzer_or_like_combine.reference +++ b/tests/queries/0_stateless/02226_analyzer_or_like_combine.reference @@ -11,7 +11,7 @@ QUERY id: 0 LIST id: 3, nodes: 1 CONSTANT id: 4, constant_value: \'Привет, World\', constant_value_type: String JOIN TREE - TABLE id: 5, table_name: system.one + TABLE id: 5, alias: __table1, table_name: system.one WHERE FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -54,7 +54,7 @@ QUERY id: 0 LIST id: 3, nodes: 1 CONSTANT id: 4, constant_value: \'Привет, World\', constant_value_type: String JOIN TREE - TABLE id: 5, table_name: system.one + TABLE id: 5, alias: __table1, table_name: system.one WHERE FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/02227_union_match_by_name.reference b/tests/queries/0_stateless/02227_union_match_by_name.reference index 42b9b01a529..c28035fab49 100644 --- a/tests/queries/0_stateless/02227_union_match_by_name.reference +++ b/tests/queries/0_stateless/02227_union_match_by_name.reference @@ -4,15 +4,15 @@ EXPLAIN header = 1, optimize = 0 SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 Expression (Project names) Header: avgWeighted(x, y) Nullable(Float64) Expression (Projection) - Header: avgWeighted(x_0, y_1) Nullable(Float64) + Header: avgWeighted(__table1.x, __table1.y) Nullable(Float64) Aggregating - Header: avgWeighted(x_0, y_1) Nullable(Float64) + Header: avgWeighted(__table1.x, __table1.y) Nullable(Float64) Expression (Before GROUP BY) - Header: x_0 Nullable(UInt8) - y_1 UInt8 + Header: __table1.x Nullable(UInt8) + __table1.y UInt8 Expression (Change column names to column identifiers) - Header: x_0 Nullable(UInt8) - y_1 UInt8 + Header: __table1.x Nullable(UInt8) + __table1.y UInt8 Union Header: x Nullable(UInt8) y UInt8 @@ -26,7 +26,7 @@ Header: avgWeighted(x, y) Nullable(Float64) Header: 255_UInt8 UInt8 1_UInt8 UInt8 Expression (Change column names to column identifiers) - Header: dummy_0 UInt8 + Header: __table3.dummy UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 Expression (Conversion before UNION) @@ -39,7 +39,7 @@ Header: avgWeighted(x, y) Nullable(Float64) Header: NULL_Nullable(Nothing) Nullable(Nothing) 1_UInt8 UInt8 Expression (Change column names to column identifiers) - Header: dummy_0 UInt8 + Header: __table5.dummy UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 AS x, 1 AS y UNION ALL SELECT y, NULL AS x, 1 AS y); diff --git a/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.reference b/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.reference index 2ecce985eb4..cd39bf8879b 100644 --- a/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.reference +++ b/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.reference @@ -2,7 +2,7 @@ Arrow x Nullable(UInt64) arr1 Array(Nullable(UInt64)) arr2 Array(Array(Nullable(String))) -arr3 Array(Tuple(`1` Nullable(String), `2` Nullable(UInt64))) +arr3 Array(Tuple(\n `1` Nullable(String),\n `2` Nullable(UInt64))) 0 [0,1] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,0)] \N [NULL,2] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,1)] 2 [2,3] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,2)] @@ -12,7 +12,7 @@ ArrowStream x Nullable(UInt64) arr1 Array(Nullable(UInt64)) arr2 Array(Array(Nullable(String))) -arr3 Array(Tuple(`1` Nullable(String), `2` Nullable(UInt64))) +arr3 Array(Tuple(\n `1` Nullable(String),\n `2` Nullable(UInt64))) 0 [0,1] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,0)] \N [NULL,2] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,1)] 2 [2,3] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,2)] @@ -22,7 +22,7 @@ Parquet x Nullable(UInt64) arr1 Array(Nullable(UInt64)) arr2 Array(Array(Nullable(String))) -arr3 Array(Tuple(`1` Nullable(String), `2` Nullable(UInt64))) +arr3 Array(Tuple(\n `1` Nullable(String),\n `2` Nullable(UInt64))) 0 [0,1] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,0)] \N [NULL,2] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,1)] 2 [2,3] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,2)] @@ -32,7 +32,7 @@ ORC x Nullable(Int64) arr1 Array(Nullable(Int64)) arr2 Array(Array(Nullable(String))) -arr3 Array(Tuple(`1` Nullable(String), `2` Nullable(Int64))) +arr3 Array(Tuple(\n `1` Nullable(String),\n `2` Nullable(Int64))) 0 [0,1] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,0)] \N [NULL,2] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,1)] 2 [2,3] [[NULL,'String'],[NULL],[]] [(NULL,NULL),('String',NULL),(NULL,2)] diff --git a/tests/queries/0_stateless/02246_flatten_tuple.reference b/tests/queries/0_stateless/02246_flatten_tuple.reference index 0320150025d..ad0ca1fa03a 100644 --- a/tests/queries/0_stateless/02246_flatten_tuple.reference +++ b/tests/queries/0_stateless/02246_flatten_tuple.reference @@ -1,4 +1,4 @@ -([1,2],['a','b'],3,'c',4) Tuple(`t1.a` Array(UInt32), `t1.s` Array(String), b UInt32, `t2.k` String, `t2.v` UInt32) -Tuple(id Int8, obj Tuple(k1 Int8, k2 Tuple(k3 String, k4 Nested(k5 Int8, k6 Int8)), some Int8), s String) Tuple(id Int8, `obj.k1` Int8, `obj.k2.k3` String, `obj.k2.k4.k5` Array(Int8), `obj.k2.k4.k6` Array(Int8), `obj.some` Int8, s String) +([1,2],['a','b'],3,'c',4) Tuple(\n `t1.a` Array(UInt32),\n `t1.s` Array(String),\n b UInt32,\n `t2.k` String,\n `t2.v` UInt32) +Tuple(\n id Int8,\n obj Tuple(\n k1 Int8,\n k2 Tuple(\n k3 String,\n k4 Nested(k5 Int8, k6 Int8)),\n some Int8),\n s String) Tuple(\n id Int8,\n `obj.k1` Int8,\n `obj.k2.k3` String,\n `obj.k2.k4.k5` Array(Int8),\n `obj.k2.k4.k6` Array(Int8),\n `obj.some` Int8,\n s String) 1 1 2 [3,4] [0,0] 0 foo 2 0 str [0] [55] 42 bar diff --git a/tests/queries/0_stateless/02286_parallel_final.reference b/tests/queries/0_stateless/02286_parallel_final.reference index f6573cb9042..5801fb46908 100644 --- a/tests/queries/0_stateless/02286_parallel_final.reference +++ b/tests/queries/0_stateless/02286_parallel_final.reference @@ -1,9 +1,13 @@ +Test intersecting ranges 2 2 3 5 -8 -8 -8 -8 -8 +Test intersecting ranges finished +Test non intersecting ranges +0 +0 +0 +0 +0 +Test non intersecting ranges finished diff --git a/tests/queries/0_stateless/02286_parallel_final.sh b/tests/queries/0_stateless/02286_parallel_final.sh index de0cca0e966..0ac510208f3 100755 --- a/tests/queries/0_stateless/02286_parallel_final.sh +++ b/tests/queries/0_stateless/02286_parallel_final.sh @@ -5,13 +5,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +echo "Test intersecting ranges" + test_random_values() { layers=$1 $CLICKHOUSE_CLIENT -n -q " + drop table if exists tbl_8parts_${layers}granules_rnd; create table tbl_8parts_${layers}granules_rnd (key1 UInt32, sign Int8) engine = CollapsingMergeTree(sign) order by (key1) partition by (key1 % 8); insert into tbl_8parts_${layers}granules_rnd select number, 1 from numbers_mt($((layers * 8 * 8192))); optimize table tbl_8parts_${layers}granules_rnd final; - explain pipeline select * from tbl_8parts_${layers}granules_rnd final settings max_threads = 16;" 2>&1 | + explain pipeline select * from tbl_8parts_${layers}granules_rnd final settings max_threads = 16, do_not_merge_across_partitions_select_final = 0; + drop table tbl_8parts_${layers}granules_rnd;" 2>&1 | grep -c "CollapsingSortedTransform" } @@ -19,16 +23,24 @@ for layers in 2 3 5 8; do test_random_values $layers done; +echo "Test intersecting ranges finished" + +echo "Test non intersecting ranges" + test_sequential_values() { layers=$1 $CLICKHOUSE_CLIENT -n -q " + drop table if exists tbl_8parts_${layers}granules_seq; create table tbl_8parts_${layers}granules_seq (key1 UInt32, sign Int8) engine = CollapsingMergeTree(sign) order by (key1) partition by (key1 / $((layers * 8192)))::UInt64; insert into tbl_8parts_${layers}granules_seq select number, 1 from numbers_mt($((layers * 8 * 8192))); optimize table tbl_8parts_${layers}granules_seq final; - explain pipeline select * from tbl_8parts_${layers}granules_seq final settings max_threads = 8;" 2>&1 | + explain pipeline select * from tbl_8parts_${layers}granules_seq final settings max_threads = 8, do_not_merge_across_partitions_select_final = 0; + drop table tbl_8parts_${layers}granules_seq;" 2>&1 | grep -c "CollapsingSortedTransform" } for layers in 2 3 5 8 16; do test_sequential_values $layers done; + +echo "Test non intersecting ranges finished" diff --git a/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference b/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference index 5f330409b2a..21348493d1d 100644 --- a/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference +++ b/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference @@ -4,7 +4,7 @@ CREATE TABLE default.t_tuple_numeric\n(\n `t` Tuple(`1` Tuple(`2` Int32, `3` 2 3 4 2 3 4 2 3 4 -Tuple(`1` Tuple(`2` Int8, `3` Int8), `4` Int8) +Tuple(\n `1` Tuple(\n `2` Int8,\n `3` Int8),\n `4` Int8) {"t":{"1":{"2":2,"3":3},"4":4}} 2 3 4 (('value')) diff --git a/tests/queries/0_stateless/02287_type_object_convert.reference b/tests/queries/0_stateless/02287_type_object_convert.reference index 2df54dcbcbc..501536f1f3e 100644 --- a/tests/queries/0_stateless/02287_type_object_convert.reference +++ b/tests/queries/0_stateless/02287_type_object_convert.reference @@ -1,15 +1,15 @@ -1 (1) Tuple(x Nullable(Int8)) -1 (1,NULL) Tuple(x Nullable(Int8), y Nullable(Int8)) -2 (NULL,2) Tuple(x Nullable(Int8), y Nullable(Int8)) -1 (1,NULL) Tuple(x Nullable(Int8), y Nullable(Int8)) -2 (NULL,2) Tuple(x Nullable(Int8), y Nullable(Int8)) -3 (1,2) Tuple(x Nullable(Int8), y Nullable(Int8)) +1 (1) Tuple(\n x Nullable(Int8)) +1 (1,NULL) Tuple(\n x Nullable(Int8),\n y Nullable(Int8)) +2 (NULL,2) Tuple(\n x Nullable(Int8),\n y Nullable(Int8)) +1 (1,NULL) Tuple(\n x Nullable(Int8),\n y Nullable(Int8)) +2 (NULL,2) Tuple(\n x Nullable(Int8),\n y Nullable(Int8)) +3 (1,2) Tuple(\n x Nullable(Int8),\n y Nullable(Int8)) 1 1 \N 2 \N 2 3 1 2 -1 (1) Tuple(x Int8) -1 (1,0) Tuple(x Int8, y Int8) -2 (0,2) Tuple(x Int8, y Int8) +1 (1) Tuple(\n x Int8) +1 (1,0) Tuple(\n x Int8,\n y Int8) +2 (0,2) Tuple(\n x Int8,\n y Int8) {"x":1} {"x":1} {"x":[[1],[1,2]]} diff --git a/tests/queries/0_stateless/02303_query_kind.reference b/tests/queries/0_stateless/02303_query_kind.reference index 8d119fb22b2..53a0df682b2 100644 --- a/tests/queries/0_stateless/02303_query_kind.reference +++ b/tests/queries/0_stateless/02303_query_kind.reference @@ -2,35 +2,35 @@ clickhouse-client --allow_experimental_analyzer=1 --query_kind secondary_query - Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-local --allow_experimental_analyzer=1 --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-client --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-local --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(dummy_0) String + Header: toString(__table1.dummy) String ReadFromStorage (SystemOne) Header: dummy UInt8 diff --git a/tests/queries/0_stateless/02313_avro_records_and_maps.reference b/tests/queries/0_stateless/02313_avro_records_and_maps.reference index 24fc635cdce..329462a4dda 100644 --- a/tests/queries/0_stateless/02313_avro_records_and_maps.reference +++ b/tests/queries/0_stateless/02313_avro_records_and_maps.reference @@ -1,8 +1,8 @@ -t Tuple(a Int32, b String) +t Tuple(\n a Int32,\n b String) (0,'String') (1,'String') (2,'String') -t Tuple(a Int32, b Tuple(c Int32, d Int32), e Array(Int32)) +t Tuple(\n a Int32,\n b Tuple(\n c Int32,\n d Int32),\n e Array(Int32)) (0,(1,2),[]) (1,(2,3),[0]) (2,(3,4),[0,1]) @@ -11,7 +11,7 @@ a.c Array(Int32) [0,1] [2,3] [1,2] [3,4] [2,3] [4,5] -a.b Array(Array(Tuple(c Int32, d Int32))) +a.b Array(Array(Tuple(\n c Int32,\n d Int32))) [[(0,1),(2,3)]] [[(1,2),(3,4)]] [[(2,3),(4,5)]] @@ -19,7 +19,7 @@ m Map(String, Int64) {'key_0':0} {'key_1':1} {'key_2':2} -m Map(String, Tuple(`1` Int64, `2` Array(Int64))) +m Map(String, Tuple(\n `1` Int64,\n `2` Array(Int64))) {'key_0':(0,[])} {'key_1':(1,[0])} {'key_2':(2,[0,1])} diff --git a/tests/queries/0_stateless/02314_avro_null_as_default.reference b/tests/queries/0_stateless/02314_avro_null_as_default.reference index ba38a15f924..e5d1b1c3752 100644 --- a/tests/queries/0_stateless/02314_avro_null_as_default.reference +++ b/tests/queries/0_stateless/02314_avro_null_as_default.reference @@ -1,5 +1,5 @@ a Nullable(Int64) -b Array(Tuple(c Nullable(Int64), d Nullable(String))) +b Array(Tuple(\n c Nullable(Int64),\n d Nullable(String))) 1 [(100,'Q'),(200,'W')] 0 0 diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index da07e94cead..69571551c2b 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -83,36 +83,36 @@ Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC === enable new analyzer === -- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct -Sorting (Stream): a_1 ASC, b_0 ASC -Sorting (Stream): a_1 ASC, b_0 ASC -Sorting (Stream): a_1 ASC, b_0 ASC -Sorting (Stream): a_1 ASC, b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, b ASC -- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns -Sorting (Stream): a_1 ASC -Sorting (Stream): a_1 ASC -Sorting (Stream): a_1 ASC +Sorting (Stream): __table1.a ASC +Sorting (Stream): __table1.a ASC +Sorting (Stream): __table1.a ASC Sorting (Stream): a ASC -- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause -Sorting (Stream): a_1 ASC, b_0 ASC -Sorting (Stream): a_1 ASC, b_0 ASC -Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC Sorting (Stream): a ASC, b ASC -- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause -Sorting (Stream): a_1 DESC, b_0 DESC -Sorting (Stream): a_1 DESC, b_0 DESC -Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): __table1.a DESC, __table1.b DESC +Sorting (Stream): __table1.a DESC, __table1.b DESC +Sorting (Stream): __table1.a DESC, __table1.b DESC Sorting (Stream): a DESC, b DESC -- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause -Sorting (Stream): a_0 ASC, b_1 ASC -Sorting (Stream): a_0 ASC, b_1 ASC -Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC Sorting (Stream): a ASC, b ASC -- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause -Sorting (Stream): a_1 DESC, b_0 DESC -Sorting (Stream): a_1 DESC, b_0 DESC -Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): __table1.a DESC, __table1.b DESC +Sorting (Stream): __table1.a DESC, __table1.b DESC +Sorting (Stream): __table1.a DESC, __table1.b DESC Sorting (Stream): a DESC, b DESC -- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization -Sorting (Stream): a_0 ASC, b_1 ASC -Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC +Sorting (Stream): __table1.a ASC, __table1.b ASC Sorting (Stream): a ASC, b ASC diff --git a/tests/queries/0_stateless/02325_dates_schema_inference.reference b/tests/queries/0_stateless/02325_dates_schema_inference.reference index a37360dae62..c8eebd3262e 100644 --- a/tests/queries/0_stateless/02325_dates_schema_inference.reference +++ b/tests/queries/0_stateless/02325_dates_schema_inference.reference @@ -5,14 +5,14 @@ x Nullable(DateTime64(9)) x Array(Nullable(Date)) x Array(Nullable(DateTime64(9))) x Array(Nullable(DateTime64(9))) -x Tuple(date1 Nullable(DateTime64(9)), date2 Nullable(Date)) +x Tuple(\n date1 Nullable(DateTime64(9)),\n date2 Nullable(Date)) x Array(Nullable(DateTime64(9))) x Array(Nullable(DateTime64(9))) x Nullable(DateTime64(9)) x Array(Nullable(String)) x Nullable(String) x Array(Nullable(String)) -x Tuple(key1 Array(Array(Nullable(DateTime64(9)))), key2 Array(Array(Nullable(String)))) +x Tuple(\n key1 Array(Array(Nullable(DateTime64(9)))),\n key2 Array(Array(Nullable(String)))) CSV c1 Nullable(Date) c1 Nullable(DateTime64(9)) diff --git a/tests/queries/0_stateless/02326_settings_changes_system_table.reference b/tests/queries/0_stateless/02326_settings_changes_system_table.reference index c4a3c71edfd..1c8c4fa1880 100644 --- a/tests/queries/0_stateless/02326_settings_changes_system_table.reference +++ b/tests/queries/0_stateless/02326_settings_changes_system_table.reference @@ -1,3 +1,3 @@ version String -changes Array(Tuple(name String, previous_value String, new_value String, reason String)) +changes Array(Tuple(\n name String,\n previous_value String,\n new_value String,\n reason String)) 22.5 [('memory_overcommit_ratio_denominator','0','1073741824','Enable memory overcommit feature by default'),('memory_overcommit_ratio_denominator_for_user','0','1073741824','Enable memory overcommit feature by default')] diff --git a/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.reference b/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.reference index a0e0f8f6b5e..d190476a7da 100644 --- a/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.reference +++ b/tests/queries/0_stateless/02327_try_infer_integers_schema_inference.reference @@ -1,12 +1,12 @@ JSONEachRow x Nullable(Int64) x Array(Nullable(Int64)) -x Tuple(a Array(Nullable(Int64))) -x Tuple(a Array(Nullable(Int64)), b Array(Nullable(Int64))) +x Tuple(\n a Array(Nullable(Int64))) +x Tuple(\n a Array(Nullable(Int64)),\n b Array(Nullable(Int64))) x Nullable(Float64) x Nullable(Float64) x Array(Nullable(Float64)) -x Tuple(a Array(Nullable(Int64)), b Array(Nullable(Float64))) +x Tuple(\n a Array(Nullable(Int64)),\n b Array(Nullable(Float64))) CSV c1 Nullable(Int64) c1 Array(Nullable(Int64)) diff --git a/tests/queries/0_stateless/02342_analyzer_compound_types.reference b/tests/queries/0_stateless/02342_analyzer_compound_types.reference index 51e0bbe6e92..c384b548473 100644 --- a/tests/queries/0_stateless/02342_analyzer_compound_types.reference +++ b/tests/queries/0_stateless/02342_analyzer_compound_types.reference @@ -8,33 +8,33 @@ Constant tuple Tuple -- id UInt64 -value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +value Tuple(\n value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String),\n value_1_level_0 String) 0 (('value_0_level_1','value_1_level_1'),'value_1_level_0') -- id UInt64 -value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +value Tuple(\n value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String),\n value_1_level_0 String) 0 (('value_0_level_1','value_1_level_1'),'value_1_level_0') -- -value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +value.value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String) value.value_1_level_0 String ('value_0_level_1','value_1_level_1') value_1_level_0 -- -alias_value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) -alias_value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(\n value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String),\n value_1_level_0 String) +alias_value.value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String) alias_value.value_1_level_0 String (('value_0_level_1','value_1_level_1'),'value_1_level_0') ('value_0_level_1','value_1_level_1') value_1_level_0 -- -alias_value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) -alias_value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(\n value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String),\n value_1_level_0 String) +alias_value.value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String) alias_value.value_1_level_0 String (('value_0_level_1','value_1_level_1'),'value_1_level_0') ('value_0_level_1','value_1_level_1') value_1_level_0 -- -alias_value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +alias_value Tuple(\n value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String),\n value_1_level_0 String) toString(alias_value.value_0_level_0) String toString(alias_value.value_1_level_0) String (('value_0_level_1','value_1_level_1'),'value_1_level_0') (\'value_0_level_1\',\'value_1_level_1\') value_1_level_0 -- -value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +value.value_0_level_0 Tuple(\n value_0_level_1 String,\n value_1_level_1 String) value.value_1_level_0 String ('value_0_level_1','value_1_level_1') value_1_level_0 -- @@ -46,17 +46,17 @@ value.value_0_level_0.value_0_level_1 String value.value_0_level_0.value_1_level_1 String value_0_level_1 value_1_level_1 -- -alias_value Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(\n value_0_level_1 String,\n value_1_level_1 String) alias_value.value_0_level_1 String alias_value.value_1_level_1 String ('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 -- -alias_value Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(\n value_0_level_1 String,\n value_1_level_1 String) alias_value.value_0_level_1 String alias_value.value_1_level_1 String ('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 -- -alias_value Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(\n value_0_level_1 String,\n value_1_level_1 String) toString(alias_value.value_0_level_1) String toString(alias_value.value_1_level_1) String ('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 9a7e579c95c..db8182e30bb 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1,2 @@ 1 -102400 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/02344_describe_cache_test 5 5000 0 1 +102400 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/02344_describe_cache_test 5 5000 0 16 diff --git a/tests/queries/0_stateless/02360_clickhouse_local_config-option.sh b/tests/queries/0_stateless/02360_clickhouse_local_config-option.sh index df0bdf38b4d..b58cfd7ec21 100755 --- a/tests/queries/0_stateless/02360_clickhouse_local_config-option.sh +++ b/tests/queries/0_stateless/02360_clickhouse_local_config-option.sh @@ -6,6 +6,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +SAFE_DIR="${CUR_DIR}/${CLICKHOUSE_DATABASE}_02360_local" +mkdir -p "${SAFE_DIR}" + echo " trace @@ -14,7 +17,7 @@ echo " 9000 - ./ + ${SAFE_DIR} 0 @@ -23,7 +26,7 @@ echo " users.xml -" > $CUR_DIR/config.xml +" > $SAFE_DIR/config.xml echo " @@ -42,13 +45,12 @@ echo " - " > $CUR_DIR/users.xml + " > $SAFE_DIR/users.xml local_opts=( - "--config-file=$CUR_DIR/config.xml" + "--config-file=$SAFE_DIR/config.xml" "--send_logs_level=none") ${CLICKHOUSE_LOCAL} "${local_opts[@]}" --query 'Select 1' |& grep -v -e 'Processing configuration file' -rm -rf $CUR_DIR/users.xml -rm -rf $CUR_DIR/config.xml +rm -rf "${SAFE_DIR}" diff --git a/tests/queries/0_stateless/02366_explain_query_tree.reference b/tests/queries/0_stateless/02366_explain_query_tree.reference index 769d7661e68..acbedbd0622 100644 --- a/tests/queries/0_stateless/02366_explain_query_tree.reference +++ b/tests/queries/0_stateless/02366_explain_query_tree.reference @@ -22,7 +22,7 @@ QUERY id: 0 COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 COLUMN id: 4, column_name: value, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.test_table + TABLE id: 3, alias: __table1, table_name: default.test_table -- QUERY id: 0 PROJECTION @@ -64,7 +64,7 @@ QUERY id: 0 CONSTANT id: 9, constant_value: UInt64_1, constant_value_type: UInt8 CONSTANT id: 10, constant_value: Array_[UInt64_1, UInt64_2, UInt64_3], constant_value_type: Array(UInt8) JOIN TREE - TABLE id: 11, table_name: default.test_table + TABLE id: 11, alias: __table1, table_name: default.test_table -- QUERY id: 0 WITH @@ -99,4 +99,4 @@ QUERY id: 0 COLUMN id: 4, column_name: id, result_type: UInt64, source_id: 5 CONSTANT id: 6, constant_value: UInt64_1, constant_value_type: UInt8 JOIN TREE - TABLE id: 5, table_name: default.test_table + TABLE id: 5, alias: __table1, table_name: default.test_table diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference index 5c9e39805b7..2c50d1028fe 100644 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference @@ -8,7 +8,7 @@ Sorting (None) -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): a_0 ASC +Sorting (Global): __table1.a ASC Sorting (None) Sorting (None) -- disable optimization -> sorting order is NOT propagated from subquery -> full sort @@ -36,8 +36,8 @@ Sorting (Stream): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): a_0 ASC -Sorting (Stream): a_0 ASC +Sorting (Global): __table1.a ASC +Sorting (Stream): __table1.a ASC Sorting (Stream): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) @@ -48,8 +48,8 @@ Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Global): plus(a_0, 1_UInt8) ASC -Sorting (Chunk): a_0 ASC +Sorting (Global): plus(__table1.a, 1_UInt8) ASC +Sorting (Chunk): __table1.a ASC Sorting (Chunk): a ASC -- ExpressionStep breaks sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 @@ -61,7 +61,7 @@ Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 Sorting (Global): plus(a, 1) ASC Sorting (Sorting for ORDER BY) -Sorting (Global): plus(a_0, 1_UInt8) ASC +Sorting (Global): plus(__table1.a, 1_UInt8) ASC Sorting (None) Sorting (Chunk): a ASC -- FilterStep preserves sort mode @@ -71,7 +71,7 @@ Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0 Sorting (Chunk): a ASC -Sorting (Chunk): a_0 ASC +Sorting (Chunk): __table1.a ASC Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC @@ -79,7 +79,7 @@ Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC -Sorting (Chunk): a_0 ASC +Sorting (Chunk): __table1.a ASC Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC @@ -87,7 +87,7 @@ Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC -Sorting (Chunk): a_0 ASC +Sorting (Chunk): __table1.a ASC Sorting (Chunk): a ASC -- FilterStep breaks sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0 @@ -119,11 +119,11 @@ Sorting (Stream): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): a_0 ASC +Sorting (Global): __table1.a ASC Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Global): a_2 ASC -Sorting (Stream): a_2 ASC +Sorting (Global): __table3.a ASC +Sorting (Stream): __table3.a ASC Sorting (Stream): a ASC -- aliases DONT break sorting order -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) @@ -135,8 +135,8 @@ Sorting (Stream): a ASC, b ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) Sorting (Global): a ASC, b ASC Sorting (Sorting for ORDER BY) -Sorting (Global): x_2 ASC, y_3 ASC -Sorting (Stream): x_2 ASC, y_3 ASC +Sorting (Global): __table2.x ASC, __table2.y ASC +Sorting (Stream): __table2.x ASC, __table2.y ASC Sorting (Stream): a ASC, b ASC -- actions chain breaks sorting order: input(column a)->sipHash64(column a)->alias(sipHash64(column a), a)->plus(alias a, 1) -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 @@ -151,11 +151,11 @@ Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Global): plus(a_0, 1_UInt8) ASC -Sorting (Global): plus(a_3, 1_UInt8) ASC +Sorting (Global): plus(__table1.a, 1_UInt8) ASC +Sorting (Global): plus(__table3.a, 1_UInt8) ASC Sorting (Sorting for ORDER BY) -Sorting (Global): plus(a_3, 1_UInt8) ASC -Sorting (Chunk): a_3 ASC +Sorting (Global): plus(__table3.a, 1_UInt8) ASC +Sorting (Chunk): __table3.a ASC Sorting (Chunk): a ASC -- check that correct sorting info is provided in case of only prefix of sorting key is in ORDER BY clause but all sorting key columns returned by query -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a @@ -167,6 +167,6 @@ Sorting (Stream): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): a_0 ASC -Sorting (Stream): a_0 ASC +Sorting (Global): __table1.a ASC +Sorting (Stream): __table1.a ASC Sorting (Stream): a ASC diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.reference b/tests/queries/0_stateless/02378_analyzer_projection_names.reference index f8b18e6df15..fd5bc7d4ae8 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.reference +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.reference @@ -13,7 +13,7 @@ concat(\'Value_1\', \'Value_2\') String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)')); -CAST((1, \'Value\'), \'Tuple (id UInt64, value String)\') Tuple(id UInt64, value String) +CAST((1, \'Value\'), \'Tuple (id UInt64, value String)\') Tuple(\n id UInt64,\n value String) SELECT 'Columns'; Columns DESCRIBE (SELECT test_table.id, test_table.id, id FROM test_table); @@ -77,45 +77,45 @@ e String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.id, a.value); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) a.id UInt64 a.value String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.*); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) a.id UInt64 a.value String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT id); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) a.value String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT value); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) a.id UInt64 SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT value APPLY toString); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) toString(a.id) String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT value APPLY x -> toString(x)); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) toString(a.id) String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, untuple(a)); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) tupleElement(a, \'id\') UInt64 tupleElement(a, \'value\') String SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, untuple(a) AS b); -a Tuple(id UInt64, value String) +a Tuple(\n id UInt64,\n value String) b.id UInt64 b.value String SELECT 'Columns with aliases'; @@ -199,63 +199,63 @@ arrayMap(lambda(tuple(x), toString(id)), [1, 2, 3]) Array(String) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.*, [1,2,3])); -compound_value Tuple(id UInt64) +compound_value Tuple(\n id UInt64) arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.* APPLY x -> x, [1,2,3])); -compound_value Tuple(id UInt64) +compound_value Tuple(\n id UInt64) arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.* APPLY toString, [1,2,3])); -compound_value Tuple(id UInt64) +compound_value Tuple(\n id UInt64) arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.* APPLY x -> toString(x), [1,2,3])); -compound_value Tuple(id UInt64) +compound_value Tuple(\n id UInt64) arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value, [1,2,3])); -compound_value Tuple(id UInt64, value String) +compound_value Tuple(\n id UInt64,\n value String) arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value APPLY x -> x, [1,2,3])); -compound_value Tuple(id UInt64, value String) +compound_value Tuple(\n id UInt64,\n value String) arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value APPLY toString, [1,2,3])); -compound_value Tuple(id UInt64, value String) +compound_value Tuple(\n id UInt64,\n value String) arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value APPLY x -> toString(x), [1,2,3])); -compound_value Tuple(id UInt64, value String) +compound_value Tuple(\n id UInt64,\n value String) arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, arrayMap(x -> untuple(a), [1,2,3]) FROM test_table); -a Tuple(id UInt64) +a Tuple(\n id UInt64) arrayMap(lambda(tuple(x), tupleElement(a, \'id\')), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, arrayMap(x -> untuple(a) AS untupled_value, [1,2,3]) FROM test_table); -a Tuple(id UInt64) +a Tuple(\n id UInt64) arrayMap(untupled_value, [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, untuple(a) AS untupled_value, arrayMap(x -> untupled_value, [1,2,3]) FROM test_table); -a Tuple(id UInt64) +a Tuple(\n id UInt64) untupled_value.id UInt64 arrayMap(lambda(tuple(x), untupled_value.id), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, untuple(a) AS untupled_value, arrayMap(x -> untupled_value AS untupled_value_in_lambda, [1,2,3]) FROM test_table); -a Tuple(id UInt64) +a Tuple(\n id UInt64) untupled_value.id UInt64 arrayMap(untupled_value_in_lambda, [1, 2, 3]) Array(UInt64) SELECT 'Standalone lambda'; @@ -285,13 +285,13 @@ arrayMap(lambda(tuple(x), _subquery_3), [1, 2, 3]) Array(Nullable(UInt8)) SELECT '--'; -- DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b) AS c, c.a, c.b); -c Tuple(a UInt8, b UInt8) +c Tuple(\n a UInt8,\n b UInt8) c.a UInt8 c.b UInt8 SELECT '--'; -- DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b) AS c, c.*); -c Tuple(a UInt8, b UInt8) +c Tuple(\n a UInt8,\n b UInt8) c.a UInt8 c.b UInt8 SELECT '--'; @@ -311,13 +311,13 @@ arrayMap(lambda(tuple(x), _subquery_3), [1, 2, 3]) Array(Nullable(UInt8)) SELECT '--'; -- DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b UNION DISTINCT SELECT 1, 2) AS c, c.a, c.b); -c Tuple(a UInt8, b UInt8) +c Tuple(\n a UInt8,\n b UInt8) c.a UInt8 c.b UInt8 SELECT '--'; -- DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b UNION DISTINCT SELECT 1, 2) AS c, c.*); -c Tuple(a UInt8, b UInt8) +c Tuple(\n a UInt8,\n b UInt8) c.a UInt8 c.b UInt8 SELECT '--'; diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference index dd5c9d4616e..5dd39c39852 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference @@ -2,51 +2,51 @@ Expression Header: key String value String Join - Header: key_0 String - value_1 String + Header: __table1.key String + __table3.value String Expression - Header: key_0 String + Header: __table1.key String ReadFromStorage Header: dummy UInt8 Union - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String Expression - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String ReadFromStorage Header: dummy UInt8 Expression - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String ReadFromStorage Header: dummy UInt8 Expression Header: key String value String Join - Header: key_0 String - key_2 String - value_1 String + Header: __table1.key String + __table3.key String + __table3.value String Sorting - Header: key_0 String + Header: __table1.key String Expression - Header: key_0 String + Header: __table1.key String ReadFromStorage Header: dummy UInt8 Sorting - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String Union - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String Expression - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String ReadFromStorage Header: dummy UInt8 Expression - Header: key_2 String - value_1 String + Header: __table3.key String + __table3.value String ReadFromStorage Header: dummy UInt8 diff --git a/tests/queries/0_stateless/02406_minmax_behaviour.reference b/tests/queries/0_stateless/02406_minmax_behaviour.reference new file mode 100644 index 00000000000..d52ba640a0e --- /dev/null +++ b/tests/queries/0_stateless/02406_minmax_behaviour.reference @@ -0,0 +1,192 @@ +-- { echoOn } +SET compile_aggregate_expressions=0; +WITH + arrayJoin([1, 2, 3, nan, 4, 5]) AS data, + arrayJoin([nan, 1, 2, 3, 4]) AS data2, + arrayJoin([1, 2, 3, 4, nan]) AS data3, + arrayJoin([nan, nan, nan]) AS data4, + arrayJoin([nan, 1, 2, 3, nan]) AS data5 +SELECT + min(data), + min(data2), + min(data3), + min(data4), + min(data5); +1 nan 1 nan nan +WITH + arrayJoin([1, 2, 3, nan, 4, 5]) AS data, + arrayJoin([nan, 1, 2, 3, 4]) AS data2, + arrayJoin([1, 2, 3, 4, nan]) AS data3, + arrayJoin([nan, nan, nan]) AS data4, + arrayJoin([nan, 1, 2, 3, nan]) AS data5 +SELECT + max(data), + max(data2), + max(data3), + max(data4), + max(data5); +5 nan 4 nan nan +Select max(number) from numbers(100) settings max_threads=1, max_block_size=10; +99 +Select max(-number) from numbers(100); +0 +Select min(number) from numbers(100) settings max_threads=1, max_block_size=10; +0 +Select min(-number) from numbers(100); +-99 +SELECT minIf(number, rand() % 2 == 3) from numbers(10) settings max_threads=1, max_block_size=5; +0 +SELECT maxIf(number, rand() % 2 == 3) from numbers(10) settings max_threads=1, max_block_size=5; +0 +SELECT minIf(number::Float64, rand() % 2 == 3) from numbers(10) settings max_threads=1, max_block_size=5; +0 +SELECT maxIf(number::Float64, rand() % 2 == 3) from numbers(10) settings max_threads=1, max_block_size=5; +0 +SELECT minIf(number::String, number < 10) as number from numbers(10, 1000); + +SELECT maxIf(number::String, number < 10) as number from numbers(10, 1000); + +SELECT maxIf(number::String, number % 3), maxIf(number::String, number % 5), minIf(number::String, number % 3), minIf(number::String, number > 10) from numbers(400); +98 99 1 100 +SELECT minIf(number::Nullable(String), number < 10) as number from numbers(10, 1000); +\N +SELECT maxIf(number::Nullable(String), number < 10) as number from numbers(10, 1000); +\N +SELECT min(n::Nullable(String)) from (Select if(number < 15 and number % 2 == 1, number * 2, NULL) as n from numbers(10, 20)); +22 +SELECT max(n::Nullable(String)) from (Select if(number < 15 and number % 2 == 1, number * 2, NULL) as n from numbers(10, 20)); +26 +SELECT argMax(number, now()) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +10 +SELECT argMax(number, now()) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +10 +SELECT argMax(number, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +10 +SELECT argMax(number, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +10 +SELECT argMax(number::String, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +10 +SELECT argMax(number::String, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +10 +SELECT argMax(number, now() + number) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +10009 +SELECT argMax(number, now() + number) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +10009 +SELECT argMaxIf(number, now() + number, number % 10 < 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +10009 +SELECT argMaxIf(number, now() + number, number % 10 < 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +10009 +SELECT argMaxIf(number, now() + number, number % 10 > 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +0 +SELECT argMaxIf(number, now() + number, number % 10 > 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +0 +SELECT argMax(number, number::Float64) from numbers(2029); +2028 +SELECT argMaxIf(number, number::Float64, number > 2030) from numbers(2029); +0 +SELECT argMaxIf(number, number::Float64, number > 2030) from numbers(2032); +2031 +SELECT argMax(number, -number::Float64) from numbers(2029); +0 +SELECT argMaxIf(number, -number::Float64, number > 2030) from numbers(2029); +0 +SELECT argMaxIf(number, -number::Float64, number > 2030) from numbers(2032); +2031 +SELECT argMin(number, now()) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +10 +SELECT argMin(number, now()) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +10 +SELECT argMin(number, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +10 +SELECT argMin(number, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +10 +SELECT argMin(number::String, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +10 +SELECT argMin(number::String, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +10 +SELECT argMin(number, now() + number) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +10 +SELECT argMin(number, now() + number) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +10 +SELECT argMinIf(number, now() + number, number % 10 < 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +10 +SELECT argMinIf(number, now() + number, number % 10 < 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +10 +SELECT argMinIf(number, now() + number, number % 10 > 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +0 +SELECT argMinIf(number, now() + number, number % 10 > 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +0 +SELECT argMin(number, number::Float64) from numbers(2029); +0 +SELECT argMinIf(number, number::Float64, number > 2030) from numbers(2029); +0 +SELECT argMinIf(number, number::Float64, number > 2030) from numbers(2032); +2031 +SELECT argMin(number, -number::Float64) from numbers(2029); +2028 +SELECT argMinIf(number, -number::Float64, number > 2030) from numbers(2029); +0 +SELECT argMinIf(number, -number::Float64, number > 2030) from numbers(2032); +2031 +Select argMax((n, n), n) t, toTypeName(t) FROM (Select if(number % 3 == 0, NULL, number) as n from numbers(10)); +(8,8) Tuple(Nullable(UInt64), Nullable(UInt64)) +Select argMaxIf((n, n), n, n < 5) t, toTypeName(t) FROM (Select if(number % 3 == 0, NULL, number) as n from numbers(10)); +(4,4) Tuple(Nullable(UInt64), Nullable(UInt64)) +Select argMaxIf((n, n), n, n > 5) t, toTypeName(t) FROM (Select if(number % 3 == 0, NULL, number) as n from numbers(10)); +(8,8) Tuple(Nullable(UInt64), Nullable(UInt64)) +Select argMin((n, n), n) t, toTypeName(t) FROM (Select if(number % 3 == 0, NULL, number) as n from numbers(10)); +(1,1) Tuple(Nullable(UInt64), Nullable(UInt64)) +Select argMinIf((n, n), n, n < 5) t, toTypeName(t) FROM (Select if(number % 3 == 0, NULL, number) as n from numbers(10)); +(1,1) Tuple(Nullable(UInt64), Nullable(UInt64)) +Select argMinIf((n, n), n, n > 5) t, toTypeName(t) FROM (Select if(number % 3 == 0, NULL, number) as n from numbers(10)); +(7,7) Tuple(Nullable(UInt64), Nullable(UInt64)) +SET compile_aggregate_expressions=1; +SET min_count_to_compile_aggregate_expression=0; +WITH + arrayJoin([1, 2, 3, nan, 4, 5]) AS data, + arrayJoin([nan, 1, 2, 3, 4]) AS data2, + arrayJoin([1, 2, 3, 4, nan]) AS data3, + arrayJoin([nan, nan, nan]) AS data4, + arrayJoin([nan, 1, 2, 3, nan]) AS data5 +SELECT + min(data), + min(data2), + min(data3), + min(data4), + min(data5); +1 nan 1 nan nan +WITH + arrayJoin([1, 2, 3, nan, 4, 5]) AS data, + arrayJoin([nan, 1, 2, 3, 4]) AS data2, + arrayJoin([1, 2, 3, 4, nan]) AS data3, + arrayJoin([nan, nan, nan]) AS data4, + arrayJoin([nan, 1, 2, 3, nan]) AS data5 +SELECT + max(data), + max(data2), + max(data3), + max(data4), + max(data5); +5 nan 4 nan nan +SELECT minIf(number, rand() % 2 == 3) from numbers(10); +0 +SELECT maxIf(number, rand() % 2 == 3) from numbers(10); +0 +SELECT minIf(number::Float64, rand() % 2 == 3) from numbers(10); +0 +SELECT maxIf(number::Float64, rand() % 2 == 3) from numbers(10); +0 +SELECT minIf(number::String, number < 10) as number from numbers(10, 1000); + +SELECT maxIf(number::String, number < 10) as number from numbers(10, 1000); + +SELECT maxIf(number::String, number % 3), maxIf(number::String, number % 5), minIf(number::String, number % 3), minIf(number::String, number > 10) from numbers(400); +98 99 1 100 +SELECT minIf(number::Nullable(String), number < 10) as number from numbers(10, 1000); +\N +SELECT maxIf(number::Nullable(String), number < 10) as number from numbers(10, 1000); +\N +SELECT min(n::Nullable(String)) from (Select if(number < 15 and number % 2 == 1, number * 2, NULL) as n from numbers(10, 20)); +22 +SELECT max(n::Nullable(String)) from (Select if(number < 15 and number % 2 == 1, number * 2, NULL) as n from numbers(10, 20)); +26 diff --git a/tests/queries/0_stateless/02406_minmax_behaviour.sql b/tests/queries/0_stateless/02406_minmax_behaviour.sql new file mode 100644 index 00000000000..a3afe7d40b0 --- /dev/null +++ b/tests/queries/0_stateless/02406_minmax_behaviour.sql @@ -0,0 +1,140 @@ +-- { echoOn } +SET compile_aggregate_expressions=0; + +WITH + arrayJoin([1, 2, 3, nan, 4, 5]) AS data, + arrayJoin([nan, 1, 2, 3, 4]) AS data2, + arrayJoin([1, 2, 3, 4, nan]) AS data3, + arrayJoin([nan, nan, nan]) AS data4, + arrayJoin([nan, 1, 2, 3, nan]) AS data5 +SELECT + min(data), + min(data2), + min(data3), + min(data4), + min(data5); + +WITH + arrayJoin([1, 2, 3, nan, 4, 5]) AS data, + arrayJoin([nan, 1, 2, 3, 4]) AS data2, + arrayJoin([1, 2, 3, 4, nan]) AS data3, + arrayJoin([nan, nan, nan]) AS data4, + arrayJoin([nan, 1, 2, 3, nan]) AS data5 +SELECT + max(data), + max(data2), + max(data3), + max(data4), + max(data5); + +Select max(number) from numbers(100) settings max_threads=1, max_block_size=10; +Select max(-number) from numbers(100); +Select min(number) from numbers(100) settings max_threads=1, max_block_size=10; +Select min(-number) from numbers(100); + +SELECT minIf(number, rand() % 2 == 3) from numbers(10) settings max_threads=1, max_block_size=5; +SELECT maxIf(number, rand() % 2 == 3) from numbers(10) settings max_threads=1, max_block_size=5; + +SELECT minIf(number::Float64, rand() % 2 == 3) from numbers(10) settings max_threads=1, max_block_size=5; +SELECT maxIf(number::Float64, rand() % 2 == 3) from numbers(10) settings max_threads=1, max_block_size=5; + +SELECT minIf(number::String, number < 10) as number from numbers(10, 1000); +SELECT maxIf(number::String, number < 10) as number from numbers(10, 1000); +SELECT maxIf(number::String, number % 3), maxIf(number::String, number % 5), minIf(number::String, number % 3), minIf(number::String, number > 10) from numbers(400); + +SELECT minIf(number::Nullable(String), number < 10) as number from numbers(10, 1000); +SELECT maxIf(number::Nullable(String), number < 10) as number from numbers(10, 1000); + +SELECT min(n::Nullable(String)) from (Select if(number < 15 and number % 2 == 1, number * 2, NULL) as n from numbers(10, 20)); +SELECT max(n::Nullable(String)) from (Select if(number < 15 and number % 2 == 1, number * 2, NULL) as n from numbers(10, 20)); + +SELECT argMax(number, now()) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +SELECT argMax(number, now()) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +SELECT argMax(number, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +SELECT argMax(number, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +SELECT argMax(number::String, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +SELECT argMax(number::String, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +SELECT argMax(number, now() + number) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +SELECT argMax(number, now() + number) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +SELECT argMaxIf(number, now() + number, number % 10 < 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +SELECT argMaxIf(number, now() + number, number % 10 < 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +SELECT argMaxIf(number, now() + number, number % 10 > 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +SELECT argMaxIf(number, now() + number, number % 10 > 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +SELECT argMax(number, number::Float64) from numbers(2029); +SELECT argMaxIf(number, number::Float64, number > 2030) from numbers(2029); +SELECT argMaxIf(number, number::Float64, number > 2030) from numbers(2032); +SELECT argMax(number, -number::Float64) from numbers(2029); +SELECT argMaxIf(number, -number::Float64, number > 2030) from numbers(2029); +SELECT argMaxIf(number, -number::Float64, number > 2030) from numbers(2032); + +SELECT argMin(number, now()) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +SELECT argMin(number, now()) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +SELECT argMin(number, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +SELECT argMin(number, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +SELECT argMin(number::String, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +SELECT argMin(number::String, 1) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +SELECT argMin(number, now() + number) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +SELECT argMin(number, now() + number) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +SELECT argMinIf(number, now() + number, number % 10 < 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +SELECT argMinIf(number, now() + number, number % 10 < 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +SELECT argMinIf(number, now() + number, number % 10 > 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=100; +SELECT argMinIf(number, now() + number, number % 10 > 20) FROM (Select number as number from numbers(10, 10000)) settings max_threads=1, max_block_size=20000; +SELECT argMin(number, number::Float64) from numbers(2029); +SELECT argMinIf(number, number::Float64, number > 2030) from numbers(2029); +SELECT argMinIf(number, number::Float64, number > 2030) from numbers(2032); +SELECT argMin(number, -number::Float64) from numbers(2029); +SELECT argMinIf(number, -number::Float64, number > 2030) from numbers(2029); +SELECT argMinIf(number, -number::Float64, number > 2030) from numbers(2032); + +Select argMax((n, n), n) t, toTypeName(t) FROM (Select if(number % 3 == 0, NULL, number) as n from numbers(10)); +Select argMaxIf((n, n), n, n < 5) t, toTypeName(t) FROM (Select if(number % 3 == 0, NULL, number) as n from numbers(10)); +Select argMaxIf((n, n), n, n > 5) t, toTypeName(t) FROM (Select if(number % 3 == 0, NULL, number) as n from numbers(10)); + +Select argMin((n, n), n) t, toTypeName(t) FROM (Select if(number % 3 == 0, NULL, number) as n from numbers(10)); +Select argMinIf((n, n), n, n < 5) t, toTypeName(t) FROM (Select if(number % 3 == 0, NULL, number) as n from numbers(10)); +Select argMinIf((n, n), n, n > 5) t, toTypeName(t) FROM (Select if(number % 3 == 0, NULL, number) as n from numbers(10)); + +SET compile_aggregate_expressions=1; +SET min_count_to_compile_aggregate_expression=0; + +WITH + arrayJoin([1, 2, 3, nan, 4, 5]) AS data, + arrayJoin([nan, 1, 2, 3, 4]) AS data2, + arrayJoin([1, 2, 3, 4, nan]) AS data3, + arrayJoin([nan, nan, nan]) AS data4, + arrayJoin([nan, 1, 2, 3, nan]) AS data5 +SELECT + min(data), + min(data2), + min(data3), + min(data4), + min(data5); + +WITH + arrayJoin([1, 2, 3, nan, 4, 5]) AS data, + arrayJoin([nan, 1, 2, 3, 4]) AS data2, + arrayJoin([1, 2, 3, 4, nan]) AS data3, + arrayJoin([nan, nan, nan]) AS data4, + arrayJoin([nan, 1, 2, 3, nan]) AS data5 +SELECT + max(data), + max(data2), + max(data3), + max(data4), + max(data5); + +SELECT minIf(number, rand() % 2 == 3) from numbers(10); +SELECT maxIf(number, rand() % 2 == 3) from numbers(10); + +SELECT minIf(number::Float64, rand() % 2 == 3) from numbers(10); +SELECT maxIf(number::Float64, rand() % 2 == 3) from numbers(10); + +SELECT minIf(number::String, number < 10) as number from numbers(10, 1000); +SELECT maxIf(number::String, number < 10) as number from numbers(10, 1000); +SELECT maxIf(number::String, number % 3), maxIf(number::String, number % 5), minIf(number::String, number % 3), minIf(number::String, number > 10) from numbers(400); + +SELECT minIf(number::Nullable(String), number < 10) as number from numbers(10, 1000); +SELECT maxIf(number::Nullable(String), number < 10) as number from numbers(10, 1000); + +SELECT min(n::Nullable(String)) from (Select if(number < 15 and number % 2 == 1, number * 2, NULL) as n from numbers(10, 20)); +SELECT max(n::Nullable(String)) from (Select if(number < 15 and number % 2 == 1, number * 2, NULL) as n from numbers(10, 20)); diff --git a/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.reference b/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.reference index 57cafb6c8e0..3f4eeac37b3 100644 --- a/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.reference +++ b/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.reference @@ -1,3 +1,3 @@ x Array(Array(Nullable(Int64))) x Tuple(Array(Array(Nullable(Int64))), Nullable(Int64)) -x Tuple(key Array(Nullable(Int64))) +x Tuple(\n key Array(Nullable(Int64))) diff --git a/tests/queries/0_stateless/02421_explain_subquery.sql b/tests/queries/0_stateless/02421_explain_subquery.sql index 4b970f81219..2970003cb1c 100644 --- a/tests/queries/0_stateless/02421_explain_subquery.sql +++ b/tests/queries/0_stateless/02421_explain_subquery.sql @@ -34,7 +34,7 @@ DROP TABLE t1; SET allow_experimental_analyzer = 1; -SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: number__ UInt64%'; +SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: \_\_table1.number UInt64%'; SELECT count() > 0 FROM (EXPLAIN PLAN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%'; SELECT count() > 0 FROM (EXPLAIN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%'; SELECT count() > 0 FROM (EXPLAIN CURRENT TRANSACTION); diff --git a/tests/queries/0_stateless/02421_type_json_empty_parts.reference b/tests/queries/0_stateless/02421_type_json_empty_parts.reference index f360b4b92cd..3c1d2aafec1 100644 --- a/tests/queries/0_stateless/02421_type_json_empty_parts.reference +++ b/tests/queries/0_stateless/02421_type_json_empty_parts.reference @@ -3,24 +3,24 @@ Collapsing 0 id UInt64 s Int8 -data Tuple(_dummy UInt8) +data Tuple(\n _dummy UInt8) DELETE all 2 1 id UInt64 -data Tuple(k1 String, k2 String) +data Tuple(\n k1 String,\n k2 String) 0 0 id UInt64 -data Tuple(_dummy UInt8) +data Tuple(\n _dummy UInt8) TTL 1 1 id UInt64 d Date -data Tuple(k1 String, k2 String) +data Tuple(\n k1 String,\n k2 String) 0 0 id UInt64 d Date -data Tuple(_dummy UInt8) +data Tuple(\n _dummy UInt8) diff --git a/tests/queries/0_stateless/02447_drop_database_replica.reference b/tests/queries/0_stateless/02447_drop_database_replica.reference index f2b41569540..1af3ee244f1 100644 --- a/tests/queries/0_stateless/02447_drop_database_replica.reference +++ b/tests/queries/0_stateless/02447_drop_database_replica.reference @@ -13,9 +13,15 @@ t rdb_default 1 1 s1 r1 1 2 2 +s1 r1 OK 2 0 +s1 r2 QUEUED 2 0 +s2 r1 QUEUED 2 0 +2 rdb_default 1 1 s1 r1 1 rdb_default 1 2 s1 r2 0 2 2 t +t2 +t3 rdb_default_4 1 1 s1 r1 1 diff --git a/tests/queries/0_stateless/02447_drop_database_replica.sh b/tests/queries/0_stateless/02447_drop_database_replica.sh index d5b3ceef46a..fb89db5045b 100755 --- a/tests/queries/0_stateless/02447_drop_database_replica.sh +++ b/tests/queries/0_stateless/02447_drop_database_replica.sh @@ -32,6 +32,10 @@ $CLICKHOUSE_CLIENT -q "system sync database replica $db" $CLICKHOUSE_CLIENT -q "select cluster, shard_num, replica_num, database_shard_name, database_replica_name, is_active from system.clusters where cluster='$db' and shard_num=1 and replica_num=1" $CLICKHOUSE_CLIENT -q "system drop database replica 's1|r1' from database $db2" 2>&1| grep -Fac "is active, cannot drop it" +# Also check that it doesn't exceed distributed_ddl_task_timeout waiting for inactive replicas +timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=throw_only_active -q "create table $db.t2 (n int) engine=Log" 2>&1| grep -Fac "TIMEOUT_EXCEEDED" +timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=null_status_on_timeout_only_active -q "create table $db.t3 (n int) engine=Log" | sort + $CLICKHOUSE_CLIENT -q "detach database $db3" $CLICKHOUSE_CLIENT -q "system drop database replica 'r1' from shard 's2' from database $db" $CLICKHOUSE_CLIENT -q "attach database $db3" 2>/dev/null diff --git a/tests/queries/0_stateless/02451_order_by_monotonic.reference b/tests/queries/0_stateless/02451_order_by_monotonic.reference index 05f20a9bad8..4b2f9f7e227 100644 --- a/tests/queries/0_stateless/02451_order_by_monotonic.reference +++ b/tests/queries/0_stateless/02451_order_by_monotonic.reference @@ -4,19 +4,19 @@ 2022-09-09 12:00:00 0x 2022-09-09 12:00:00 1 2022-09-09 12:00:00 1x - Prefix sort description: toStartOfMinute(t_0) ASC - Result sort description: toStartOfMinute(t_0) ASC, c1_1 ASC - Prefix sort description: toStartOfMinute(t_0) ASC - Result sort description: toStartOfMinute(t_0) ASC - Prefix sort description: negate(a_0) ASC - Result sort description: negate(a_0) ASC - Prefix sort description: negate(a_0) ASC, negate(b_1) ASC - Result sort description: negate(a_0) ASC, negate(b_1) ASC - Prefix sort description: a_0 DESC, negate(b_1) ASC - Result sort description: a_0 DESC, negate(b_1) ASC - Prefix sort description: negate(a_0) ASC, b_1 DESC - Result sort description: negate(a_0) ASC, b_1 DESC - Prefix sort description: negate(a_0) ASC - Result sort description: negate(a_0) ASC, b_1 ASC - Prefix sort description: a_0 ASC - Result sort description: a_0 ASC, negate(b_1) ASC + Prefix sort description: toStartOfMinute(__table1.t) ASC + Result sort description: toStartOfMinute(__table1.t) ASC, __table1.c1 ASC + Prefix sort description: toStartOfMinute(__table1.t) ASC + Result sort description: toStartOfMinute(__table1.t) ASC + Prefix sort description: negate(__table1.a) ASC + Result sort description: negate(__table1.a) ASC + Prefix sort description: negate(__table1.a) ASC, negate(__table1.b) ASC + Result sort description: negate(__table1.a) ASC, negate(__table1.b) ASC + Prefix sort description: __table1.a DESC, negate(__table1.b) ASC + Result sort description: __table1.a DESC, negate(__table1.b) ASC + Prefix sort description: negate(__table1.a) ASC, __table1.b DESC + Result sort description: negate(__table1.a) ASC, __table1.b DESC + Prefix sort description: negate(__table1.a) ASC + Result sort description: negate(__table1.a) ASC, __table1.b ASC + Prefix sort description: __table1.a ASC + Result sort description: __table1.a ASC, negate(__table1.b) ASC diff --git a/tests/queries/0_stateless/02475_bson_each_row_format.reference b/tests/queries/0_stateless/02475_bson_each_row_format.reference index f90867d92b1..5659e5201b1 100644 --- a/tests/queries/0_stateless/02475_bson_each_row_format.reference +++ b/tests/queries/0_stateless/02475_bson_each_row_format.reference @@ -166,7 +166,7 @@ Tuple ('Hello',4) OK OK -tuple Tuple(x Nullable(Int64), s Nullable(String)) +tuple Tuple(\n x Nullable(Int64),\n s Nullable(String)) (0,'Hello') (1,'Hello') (2,'Hello') @@ -214,7 +214,7 @@ Nested types [[0,1,2],[0,1,2,3]] ((3,'Hello'),'Hello') {'a':{'a.a':3,'a.b':4},'b':{'b.a':3,'b.b':4}} [[0,1,2,3],[0,1,2,3,4]] ((4,'Hello'),'Hello') {'a':{'a.a':4,'a.b':5},'b':{'b.a':4,'b.b':5}} nested1 Array(Array(Nullable(Int64))) -nested2 Tuple(Tuple(x Nullable(Int64), s Nullable(String)), Nullable(String)) +nested2 Tuple(Tuple(\n x Nullable(Int64),\n s Nullable(String)), Nullable(String)) nested3 Map(String, Map(String, Nullable(Int64))) [[],[0]] ((0,'Hello'),'Hello') {'a':{'a.a':0,'a.b':1},'b':{'b.a':0,'b.b':1}} [[0],[0,1]] ((1,'Hello'),'Hello') {'a':{'a.a':1,'a.b':2},'b':{'b.a':1,'b.b':2}} diff --git a/tests/queries/0_stateless/02476_fuse_sum_count.reference b/tests/queries/0_stateless/02476_fuse_sum_count.reference index 43a39e8b7e5..1eb156743b0 100644 --- a/tests/queries/0_stateless/02476_fuse_sum_count.reference +++ b/tests/queries/0_stateless/02476_fuse_sum_count.reference @@ -21,7 +21,7 @@ QUERY id: 0 LIST id: 7, nodes: 1 COLUMN id: 4, column_name: a, result_type: Nullable(Int8), source_id: 5 JOIN TREE - TABLE id: 5, table_name: default.fuse_tbl + TABLE id: 5, alias: __table1, table_name: default.fuse_tbl QUERY id: 0 PROJECTION COLUMNS sum(b) Int64 @@ -59,7 +59,7 @@ QUERY id: 0 COLUMN id: 6, column_name: b, result_type: Int8, source_id: 7 CONSTANT id: 18, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE id: 7, table_name: default.fuse_tbl + TABLE id: 7, alias: __table1, table_name: default.fuse_tbl QUERY id: 0 PROJECTION COLUMNS sum(plus(a, 1)) Nullable(Int64) @@ -138,7 +138,7 @@ QUERY id: 0 LIST id: 39, nodes: 1 COLUMN id: 6, column_name: a, result_type: Nullable(Int8), source_id: 7 JOIN TREE - TABLE id: 7, table_name: default.fuse_tbl + TABLE id: 7, alias: __table1, table_name: default.fuse_tbl QUERY id: 0 PROJECTION COLUMNS multiply(avg(b), 3) Float64 @@ -215,14 +215,14 @@ QUERY id: 0 COLUMN id: 10, column_name: b, result_type: Int8, source_id: 11 CONSTANT id: 37, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - QUERY id: 11, is_subquery: 1 + QUERY id: 11, alias: __table1, is_subquery: 1 PROJECTION COLUMNS b Int8 PROJECTION LIST id: 38, nodes: 1 COLUMN id: 39, column_name: b, result_type: Int8, source_id: 40 JOIN TREE - TABLE id: 40, table_name: default.fuse_tbl + TABLE id: 40, alias: __table2, table_name: default.fuse_tbl QUERY id: 0 PROJECTION COLUMNS sum(b) Int64 @@ -246,14 +246,14 @@ QUERY id: 0 COLUMN id: 6, column_name: b, result_type: Int64, source_id: 7 CONSTANT id: 11, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - QUERY id: 7, is_subquery: 1 + QUERY id: 7, alias: __table1, is_subquery: 1 PROJECTION COLUMNS b Int64 PROJECTION LIST id: 12, nodes: 1 COLUMN id: 13, column_name: x, result_type: Int64, source_id: 14 JOIN TREE - QUERY id: 14, is_subquery: 1 + QUERY id: 14, alias: __table2, is_subquery: 1 PROJECTION COLUMNS x Int64 count(b) UInt64 @@ -276,7 +276,7 @@ QUERY id: 0 COLUMN id: 20, column_name: b, result_type: Int8, source_id: 21 CONSTANT id: 25, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE id: 21, table_name: default.fuse_tbl + TABLE id: 21, alias: __table3, table_name: default.fuse_tbl 0 0 nan 0 0 nan 45 10 4.5 Decimal(38, 0) UInt64 Float64 diff --git a/tests/queries/0_stateless/02477_fuse_quantiles.reference b/tests/queries/0_stateless/02477_fuse_quantiles.reference index 7c7d581f7fb..7603381416c 100644 --- a/tests/queries/0_stateless/02477_fuse_quantiles.reference +++ b/tests/queries/0_stateless/02477_fuse_quantiles.reference @@ -34,7 +34,7 @@ QUERY id: 0 COLUMN id: 9, column_name: b, result_type: Float64, source_id: 10 CONSTANT id: 14, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - QUERY id: 10, is_subquery: 1 + QUERY id: 10, alias: __table1, is_subquery: 1 PROJECTION COLUMNS b Float64 PROJECTION @@ -45,7 +45,7 @@ QUERY id: 0 COLUMN id: 18, column_name: x, result_type: Float64, source_id: 19 CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8 JOIN TREE - QUERY id: 19, is_subquery: 1 + QUERY id: 19, alias: __table2, is_subquery: 1 PROJECTION COLUMNS x Float64 quantile(0.9)(b) Float64 @@ -76,7 +76,7 @@ QUERY id: 0 COLUMN id: 29, column_name: b, result_type: Int32, source_id: 30 CONSTANT id: 34, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE id: 30, table_name: default.fuse_tbl + TABLE id: 30, alias: __table3, table_name: default.fuse_tbl GROUP BY LIST id: 35, nodes: 1 COLUMN id: 18, column_name: x, result_type: Float64, source_id: 19 diff --git a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference index 84589668d64..649b037fafa 100644 --- a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference +++ b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference @@ -8,7 +8,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality + TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -26,7 +26,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality + TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -36,6 +36,42 @@ QUERY id: 0 SETTINGS allow_experimental_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality +WHERE (a != \'x\') AND (a != \'y\') +QUERY id: 0 + PROJECTION COLUMNS + a LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality + WHERE + FUNCTION id: 4, function_name: notIn, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + CONSTANT id: 6, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) + SETTINGS allow_experimental_analyzer=1 +SELECT a +FROM t_logical_expressions_optimizer_low_cardinality +WHERE (a != \'x\') AND (\'y\' != a) +QUERY id: 0 + PROJECTION COLUMNS + a LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality + WHERE + FUNCTION id: 4, function_name: notIn, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + CONSTANT id: 6, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) + SETTINGS allow_experimental_analyzer=1 +SELECT a +FROM t_logical_expressions_optimizer_low_cardinality WHERE (b = 0) OR (b = 1) QUERY id: 0 PROJECTION COLUMNS @@ -44,7 +80,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality + TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -60,3 +96,29 @@ QUERY id: 0 COLUMN id: 8, column_name: b, result_type: UInt32, source_id: 3 CONSTANT id: 12, constant_value: UInt64_1, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 +SELECT a +FROM t_logical_expressions_optimizer_low_cardinality +WHERE (b != 0) AND (b != 1) +QUERY id: 0 + PROJECTION COLUMNS + a LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: notEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: b, result_type: UInt32, source_id: 3 + CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + FUNCTION id: 10, function_name: notEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + COLUMN id: 8, column_name: b, result_type: UInt32, source_id: 3 + CONSTANT id: 12, constant_value: UInt64_1, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql index 14f8ad830e7..976b21a7e29 100644 --- a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql +++ b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql @@ -2,13 +2,24 @@ DROP TABLE IF EXISTS t_logical_expressions_optimizer_low_cardinality; set optimize_min_equality_disjunction_chain_length=3; CREATE TABLE t_logical_expressions_optimizer_low_cardinality (a LowCardinality(String), b UInt32) ENGINE = Memory; --- LowCardinality case, ignore optimize_min_equality_disjunction_chain_length limit, optimzer applied +-- LowCardinality case, ignore optimize_min_equality_disjunction_chain_length limit, optimizer applied +-- Chain of OR equals EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR a = 'y'; EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR a = 'y' SETTINGS allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR 'y' = a; EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR 'y' = a SETTINGS allow_experimental_analyzer = 1; +-- Chain of AND notEquals +EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a <> 'x' AND a <> 'y'; +EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a <> 'x' AND a <> 'y' SETTINGS allow_experimental_analyzer = 1; +EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a <> 'x' AND 'y' <> a; +EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a <> 'x' AND 'y' <> a SETTINGS allow_experimental_analyzer = 1; + -- Non-LowCardinality case, optimizer not applied for short chains +-- Chain of OR equals EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b = 0 OR b = 1; EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b = 0 OR b = 1 SETTINGS allow_experimental_analyzer = 1; +-- Chain of AND notEquals +EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b <> 0 AND b <> 1; +EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b <> 0 AND b <> 1 SETTINGS allow_experimental_analyzer = 1; DROP TABLE t_logical_expressions_optimizer_low_cardinality; diff --git a/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql b/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql index 99f20290ff0..50248665bc9 100644 --- a/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql +++ b/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql @@ -24,4 +24,4 @@ SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id, SELECT '--'; -SELECT b.dt FROM (SELECT NULL > NULL AS pk, 1 AS dt FROM numbers(5)) AS a ASOF LEFT JOIN (SELECT NULL AS pk, 1 AS dt) AS b ON (a.pk = b.pk) AND 1 != 1 AND (a.dt >= b.dt); -- { serverError 403 } +SELECT b.dt FROM (SELECT NULL > NULL AS pk, 1 AS dt FROM numbers(5)) AS a ASOF LEFT JOIN (SELECT NULL AS pk, 1 AS dt) AS b ON (a.pk = b.pk) AND 1 != 1 AND (a.dt >= b.dt); -- { serverError 403, NOT_FOUND_COLUMN_IN_BLOCK } diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference index f4dd01bc184..6838dacc3b3 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference @@ -50,7 +50,7 @@ QUERY id: 0 COLUMN id: 5, column_name: b, result_type: String, source_id: 3 COLUMN id: 6, column_name: c, result_type: String, source_id: 3 JOIN TREE - TABLE_FUNCTION id: 3, table_function_name: mysql + TABLE_FUNCTION id: 3, alias: __table1, table_function_name: mysql ARGUMENTS LIST id: 7, nodes: 5 CONSTANT id: 8, constant_value: \'127.0.0.1:9004\', constant_value_type: String @@ -63,10 +63,10 @@ QUERY id: 0 SETTINGS connection_wait_timeout=123 connect_timeout=40123002 read_write_timeout=40123001 connection_pool_size=3 SELECT - key AS key, - a AS a, - b AS b, - c AS c -FROM mysql(\'127.0.0.1:9004\', \'default\', foo, \'default\', \'\', SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3) + __table1.key AS key, + __table1.a AS a, + __table1.b AS b, + __table1.c AS c +FROM mysql(\'127.0.0.1:9004\', \'default\', foo, \'default\', \'\', SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3) AS __table1 --- 5 diff --git a/tests/queries/0_stateless/02479_race_condition_between_insert_and_droppin_mv.sh b/tests/queries/0_stateless/02479_race_condition_between_insert_and_droppin_mv.sh index 9ce4b459fce..6899b31d1d9 100755 --- a/tests/queries/0_stateless/02479_race_condition_between_insert_and_droppin_mv.sh +++ b/tests/queries/0_stateless/02479_race_condition_between_insert_and_droppin_mv.sh @@ -14,7 +14,7 @@ function insert { offset=500 while true; do - ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_race_condition_landing SELECT number, toString(number), toString(number) from system.numbers limit $i, $offset" + ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_race_condition_landing SELECT number, toString(number), toString(number) from system.numbers limit $i, $offset settings ignore_materialized_views_with_dropped_target_table=1" i=$(( $i + $RANDOM % 100 + 400 )) done } diff --git a/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference b/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference index b11f3e3a1d3..969ec320790 100644 --- a/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference +++ b/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference @@ -6,5 +6,5 @@ Order: a ASC, c ASC ReadFromMergeTree (default.tab) Aggregating - Order: a_0 ASC, c_2 ASC + Order: __table1.a ASC, __table1.c ASC ReadFromMergeTree (default.tab) diff --git a/tests/queries/0_stateless/02481_analyzer_optimize_aggregation_arithmetics.reference b/tests/queries/0_stateless/02481_analyzer_optimize_aggregation_arithmetics.reference index 22dda253066..a26773baae2 100644 --- a/tests/queries/0_stateless/02481_analyzer_optimize_aggregation_arithmetics.reference +++ b/tests/queries/0_stateless/02481_analyzer_optimize_aggregation_arithmetics.reference @@ -20,7 +20,7 @@ QUERY id: 0 LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8 @@ -44,7 +44,7 @@ QUERY id: 0 LIST id: 10, nodes: 1 CONSTANT id: 11, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 7, table_function_name: numbers + TABLE_FUNCTION id: 7, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.reference b/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.reference index 03722034708..9f9c1da5e88 100644 --- a/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.reference +++ b/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.reference @@ -17,7 +17,7 @@ QUERY id: 0, group_by_type: grouping_sets LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 @@ -103,7 +103,7 @@ QUERY id: 0, group_by_type: grouping_sets LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 @@ -180,7 +180,7 @@ QUERY id: 0, group_by_type: grouping_sets LIST id: 9, nodes: 1 COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32 @@ -253,7 +253,7 @@ QUERY id: 0, group_by_type: grouping_sets LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - TABLE_FUNCTION id: 3, table_function_name: numbers + TABLE_FUNCTION id: 3, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 4, nodes: 1 CONSTANT id: 5, constant_value: UInt64_1000, constant_value_type: UInt16 diff --git a/tests/queries/0_stateless/02483_test_reverse_dns_resolution.reference b/tests/queries/0_stateless/02483_test_reverse_dns_resolution.reference deleted file mode 100644 index 2bae467069f..00000000000 --- a/tests/queries/0_stateless/02483_test_reverse_dns_resolution.reference +++ /dev/null @@ -1,14 +0,0 @@ --- { echoOn } --- Expect dns.google on both queries -select reverseDNSQuery('8.8.8.8'); -['dns.google'] -select reverseDNSQuery('2001:4860:4860::8888'); -['dns.google'] --- Expect empty response -select reverseDNSQuery(''); -[] --- Expect error, invalid column type -select reverseDNSQuery(1); -- {serverError 36} --- Expect error, wrong number of arguments -select reverseDNSQuery(); -- {serverError 42} -select reverseDNSQuery(1, 2); -- {serverError 42} diff --git a/tests/queries/0_stateless/02483_test_reverse_dns_resolution.sql b/tests/queries/0_stateless/02483_test_reverse_dns_resolution.sql deleted file mode 100644 index d9576c0641a..00000000000 --- a/tests/queries/0_stateless/02483_test_reverse_dns_resolution.sql +++ /dev/null @@ -1,14 +0,0 @@ --- { echoOn } --- Expect dns.google on both queries -select reverseDNSQuery('8.8.8.8'); -select reverseDNSQuery('2001:4860:4860::8888'); - --- Expect empty response -select reverseDNSQuery(''); - --- Expect error, invalid column type -select reverseDNSQuery(1); -- {serverError 36} - --- Expect error, wrong number of arguments -select reverseDNSQuery(); -- {serverError 42} -select reverseDNSQuery(1, 2); -- {serverError 42} diff --git a/tests/queries/0_stateless/02486_truncate_and_unexpected_parts.reference b/tests/queries/0_stateless/02486_truncate_and_unexpected_parts.reference index 2ece1147d78..824d4bbec98 100644 --- a/tests/queries/0_stateless/02486_truncate_and_unexpected_parts.reference +++ b/tests/queries/0_stateless/02486_truncate_and_unexpected_parts.reference @@ -13,3 +13,5 @@ 5 rmt2 7 rmt2 9 rmt2 +1 +3 diff --git a/tests/queries/0_stateless/02486_truncate_and_unexpected_parts.sql b/tests/queries/0_stateless/02486_truncate_and_unexpected_parts.sql index 52e8be236c8..5c90313b6b8 100644 --- a/tests/queries/0_stateless/02486_truncate_and_unexpected_parts.sql +++ b/tests/queries/0_stateless/02486_truncate_and_unexpected_parts.sql @@ -50,3 +50,20 @@ system sync replica rmt1; system sync replica rmt2; select *, _table from merge(currentDatabase(), '') order by _table, (*,); + + +create table rmt3 (n int) engine=ReplicatedMergeTree('/test/02468/{database}3', '1') order by tuple() settings replicated_max_ratio_of_wrong_parts=0, max_suspicious_broken_parts=0, max_suspicious_broken_parts_bytes=0; +set insert_keeper_fault_injection_probability=0; +insert into rmt3 values (1); +insert into rmt3 values (2); +insert into rmt3 values (3); + +system stop cleanup rmt3; +system sync replica rmt3 pull; +alter table rmt3 drop part 'all_1_1_0'; +optimize table rmt3 final; + +detach table rmt3 sync; +attach table rmt3; + +select * from rmt3 order by n; diff --git a/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.reference b/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.reference new file mode 100644 index 00000000000..00c825f598a --- /dev/null +++ b/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.reference @@ -0,0 +1,3 @@ +0 +broken-on-start broken-on-start_all_0_0_0 +42 diff --git a/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh b/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh new file mode 100755 index 00000000000..b01f16e1cad --- /dev/null +++ b/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, zookeeper + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02488/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') order by n + settings min_bytes_for_wide_part=0, allow_remote_fs_zero_copy_replication=1, storage_policy='s3_cache'" +$CLICKHOUSE_CLIENT -q "create table rmt2 (n int) engine=ReplicatedMergeTree('/test/02488/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '2') order by n + settings min_bytes_for_wide_part=0, allow_remote_fs_zero_copy_replication=1, storage_policy='s3_cache'" + +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt2 values (42)" +$CLICKHOUSE_CLIENT -q "system sync replica rmt1" + +path=$($CLICKHOUSE_CLIENT -q "select path from system.parts where database='$CLICKHOUSE_DATABASE' and table='rmt2' and name='all_0_0_0'") +# ensure that path is absolute before removing +$CLICKHOUSE_CLIENT -q "select throwIf(substring('$path', 1, 1) != '/', 'Path is relative: $path')" || exit +rm -f $path/count.txt + +$CLICKHOUSE_CLIENT -q "detach table rmt2 sync" +$CLICKHOUSE_CLIENT --send_logs_level='fatal' -q "attach table rmt2" + +$CLICKHOUSE_CLIENT -q "select reason, name from system.detached_parts where database='$CLICKHOUSE_DATABASE' and table='rmt2'" + +$CLICKHOUSE_CLIENT -q "drop table rmt2 sync" + +$CLICKHOUSE_CLIENT -q "select * from rmt1" + +$CLICKHOUSE_CLIENT -q "drop table rmt1" diff --git a/tests/queries/0_stateless/02490_replacing_merge_tree_is_deleted_column.reference b/tests/queries/0_stateless/02490_replacing_merge_tree_is_deleted_column.reference new file mode 100644 index 00000000000..c897004b4e3 --- /dev/null +++ b/tests/queries/0_stateless/02490_replacing_merge_tree_is_deleted_column.reference @@ -0,0 +1,121 @@ +== Test SELECT ... FINAL - no is_deleted == +d1 5 0 +d2 1 0 +d3 1 0 +d4 3 0 +d5 1 0 +d6 2 1 +d1 5 0 +d2 1 0 +d3 1 0 +d4 3 0 +d5 1 0 +d6 2 1 +== Test SELECT ... FINAL - no is_deleted SETTINGS clean_deleted_rows=Always == +d1 5 0 +d2 1 0 +d3 1 0 +d4 3 0 +d5 1 0 +d6 2 1 +d1 5 0 +d2 1 0 +d3 1 0 +d4 3 0 +d5 1 0 +d6 2 1 +== Test SELECT ... FINAL == +d1 5 0 +d2 1 0 +d3 1 0 +d4 3 0 +d5 1 0 +d1 5 0 +d2 1 0 +d3 1 0 +d4 3 0 +d5 1 0 +d6 2 1 +== Insert backups == +d1 5 0 +d2 1 0 +d3 1 0 +d4 3 0 +d5 1 0 +== Insert a second batch with overlaping data == +d1 5 0 +d2 3 0 +d3 3 0 +d4 3 0 +d5 1 0 +== Only last version remains after OPTIMIZE W/ CLEANUP == +d1 5 0 +d2 1 0 +d3 1 0 +d4 1 0 +d5 1 0 +d6 3 0 +== OPTIMIZE W/ CLEANUP (remove d6) == +d1 5 0 +d2 1 0 +d3 1 0 +d4 1 0 +d5 1 0 +== Test of the SETTINGS clean_deleted_rows as Always == +d1 5 0 +d2 1 0 +d3 1 0 +d4 3 0 +d5 1 0 +d1 5 0 +d2 1 0 +d3 1 0 +d4 3 0 +d5 1 0 +d6 2 1 +d1 5 0 +d2 1 0 +d3 1 0 +d4 3 0 +d5 1 0 +== Test of the SETTINGS clean_deleted_rows as Never == +d1 5 0 +d2 1 0 +d3 1 0 +d4 3 0 +d5 1 0 +d6 2 1 +== (Replicas) Test optimize == +d2 1 0 +d4 1 0 +== (Replicas) Test settings == +c2 1 0 +c4 1 0 +no cleanup 1 d1 5 0 +no cleanup 1 d2 1 0 +no cleanup 1 d3 1 0 +no cleanup 1 d4 3 0 +no cleanup 1 d5 1 0 +no cleanup 2 d1 5 0 +no cleanup 2 d2 1 0 +no cleanup 2 d3 1 0 +no cleanup 2 d4 3 0 +no cleanup 2 d5 1 0 +no cleanup 2 d6 2 1 +no cleanup 3 d1 5 0 +no cleanup 3 d2 1 0 +no cleanup 3 d3 1 0 +no cleanup 3 d4 3 0 +no cleanup 3 d5 1 0 +no cleanup 4 d1 5 0 +no cleanup 4 d2 1 0 +no cleanup 4 d3 1 0 +no cleanup 4 d4 3 0 +no cleanup 4 d5 1 0 +no cleanup 4 d6 2 1 +== Check cleanup & settings for other merge trees == +d1 1 1 +d1 1 1 +d1 1 1 +d1 1 1 1 +d1 1 1 1 diff --git a/tests/queries/0_stateless/02490_replacing_merge_tree_is_deleted_column.sql b/tests/queries/0_stateless/02490_replacing_merge_tree_is_deleted_column.sql new file mode 100644 index 00000000000..80c18ae308b --- /dev/null +++ b/tests/queries/0_stateless/02490_replacing_merge_tree_is_deleted_column.sql @@ -0,0 +1,174 @@ +-- Tags: zookeeper + +-- Settings allow_deprecated_syntax_for_merge_tree prevent to enable the is_deleted column +set allow_deprecated_syntax_for_merge_tree=0; + +-- Test the bahaviour without the is_deleted column +DROP TABLE IF EXISTS test; +CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version) Order by (uid) settings allow_experimental_replacing_merge_with_cleanup=1; +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); +SELECT '== Test SELECT ... FINAL - no is_deleted =='; +select * from test FINAL order by uid; +OPTIMIZE TABLE test FINAL CLEANUP; +select * from test order by uid; + +DROP TABLE IF EXISTS test; +CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version) Order by (uid) SETTINGS clean_deleted_rows='Always', allow_experimental_replacing_merge_with_cleanup=1; +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); +SELECT '== Test SELECT ... FINAL - no is_deleted SETTINGS clean_deleted_rows=Always =='; +select * from test FINAL order by uid; +OPTIMIZE TABLE test FINAL CLEANUP; +select * from test order by uid; + +-- Test the new behaviour +DROP TABLE IF EXISTS test; +CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version, is_deleted) Order by (uid) settings allow_experimental_replacing_merge_with_cleanup=1; +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); +SELECT '== Test SELECT ... FINAL =='; +select * from test FINAL order by uid; +select * from test order by uid; + +SELECT '== Insert backups =='; +INSERT INTO test (*) VALUES ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1); +select * from test FINAL order by uid; + +SELECT '== Insert a second batch with overlaping data =='; +INSERT INTO test (*) VALUES ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 1), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0), ('d2', 2, 1), ('d2', 3, 0), ('d3', 2, 1), ('d3', 3, 0); +select * from test FINAL order by uid; + +DROP TABLE IF EXISTS test; +CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version, is_deleted) Order by (uid) settings allow_experimental_replacing_merge_with_cleanup=1; + +-- Expect d6 to be version=3 is_deleted=false +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 3, 0); +-- Insert previous version of 'd6' but only v=3 is_deleted=false will remain +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 2, 1); +SELECT '== Only last version remains after OPTIMIZE W/ CLEANUP =='; +OPTIMIZE TABLE test FINAL CLEANUP; +select * from test order by uid; + +-- insert d6 v=3 is_deleted=true (timestamp more recent so this version should be the one take into acount) +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 3, 1); + +SELECT '== OPTIMIZE W/ CLEANUP (remove d6) =='; +OPTIMIZE TABLE test FINAL CLEANUP; +-- No d6 anymore +select * from test order by uid; + +DROP TABLE IF EXISTS test; +CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version, is_deleted) Order by (uid) SETTINGS clean_deleted_rows='Always', allow_experimental_replacing_merge_with_cleanup=1; + +SELECT '== Test of the SETTINGS clean_deleted_rows as Always =='; +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); +-- Even if the setting is set to Always, the SELECT FINAL doesn't delete rows +select * from test FINAL order by uid; +select * from test order by uid; + +OPTIMIZE TABLE test FINAL; +-- d6 has to be removed since we set clean_deleted_rows as 'Always' +select * from test where is_deleted=0 order by uid; + +SELECT '== Test of the SETTINGS clean_deleted_rows as Never =='; +ALTER TABLE test MODIFY SETTING clean_deleted_rows='Never'; +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); +OPTIMIZE TABLE test FINAL; +-- d6 has NOT to be removed since we set clean_deleted_rows as 'Never' +select * from test order by uid; + +DROP TABLE IF EXISTS testCleanupR1; + +CREATE TABLE testCleanupR1 (uid String, version UInt32, is_deleted UInt8) + ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{database}/tables/test_cleanup/', 'r1', version, is_deleted) + ORDER BY uid settings allow_experimental_replacing_merge_with_cleanup=1; + + +INSERT INTO testCleanupR1 (*) VALUES ('d1', 1, 0),('d2', 1, 0),('d3', 1, 0),('d4', 1, 0); +INSERT INTO testCleanupR1 (*) VALUES ('d3', 2, 1); +INSERT INTO testCleanupR1 (*) VALUES ('d1', 2, 1); +SYSTEM SYNC REPLICA testCleanupR1; -- Avoid "Cannot select parts for optimization: Entry for part all_2_2_0 hasn't been read from the replication log yet" + +OPTIMIZE TABLE testCleanupR1 FINAL CLEANUP; + +-- Only d3 to d5 remain +SELECT '== (Replicas) Test optimize =='; +SELECT * FROM testCleanupR1 order by uid; + +------------------------------ + +DROP TABLE IF EXISTS testSettingsR1; + +CREATE TABLE testSettingsR1 (col1 String, version UInt32, is_deleted UInt8) + ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{database}/tables/test_setting/', 'r1', version, is_deleted) + ORDER BY col1 + SETTINGS clean_deleted_rows = 'Always', allow_experimental_replacing_merge_with_cleanup=1; + +INSERT INTO testSettingsR1 (*) VALUES ('c1', 1, 1),('c2', 1, 0),('c3', 1, 1),('c4', 1, 0); +SYSTEM SYNC REPLICA testSettingsR1; -- Avoid "Cannot select parts for optimization: Entry for part all_2_2_0 hasn't been read from the replication log yet" + +OPTIMIZE TABLE testSettingsR1 FINAL; + +-- Only d3 to d5 remain +SELECT '== (Replicas) Test settings =='; +SELECT * FROM testSettingsR1 where is_deleted=0 order by col1; + + +------------------------------ +-- Check errors +DROP TABLE IF EXISTS test; +CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version, is_deleted) Order by (uid) settings allow_experimental_replacing_merge_with_cleanup=1; + +-- is_deleted == 0/1 +INSERT INTO test (*) VALUES ('d1', 1, 2); -- { serverError INCORRECT_DATA } + +DROP TABLE IF EXISTS test; +-- checkis_deleted type +CREATE TABLE test (uid String, version UInt32, is_deleted String) ENGINE = ReplacingMergeTree(version, is_deleted) Order by (uid); -- { serverError BAD_TYPE_OF_FIELD } + +CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version, is_deleted) Order by (uid); +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); +select 'no cleanup 1', * from test FINAL order by uid; +OPTIMIZE TABLE test FINAL CLEANUP; -- { serverError SUPPORT_IS_DISABLED } +select 'no cleanup 2', * from test order by uid; +DROP TABLE test; + +CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/{database}/tables/no_cleanup/', 'r1', version, is_deleted) Order by (uid); +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d2', 1, 0), ('d6', 1, 0), ('d4', 1, 0), ('d6', 2, 1), ('d3', 1, 0), ('d1', 2, 1), ('d5', 1, 0), ('d4', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d4', 3, 0), ('d1', 5, 0); +select 'no cleanup 3', * from test FINAL order by uid; +OPTIMIZE TABLE test FINAL CLEANUP; -- { serverError SUPPORT_IS_DISABLED } +select 'no cleanup 4', * from test order by uid; +DROP TABLE test; + +-- is_deleted column for other mergeTrees - ErrorCodes::LOGICAL_ERROR) + +-- Check clean_deleted_rows='Always' for other MergeTrees +SELECT '== Check cleanup & settings for other merge trees =='; +CREATE TABLE testMT (uid String, version UInt32, is_deleted UInt8) ENGINE = MergeTree() Order by (uid) SETTINGS clean_deleted_rows='Always', allow_experimental_replacing_merge_with_cleanup=1; +INSERT INTO testMT (*) VALUES ('d1', 1, 1); +OPTIMIZE TABLE testMT FINAL CLEANUP; -- { serverError CANNOT_ASSIGN_OPTIMIZE } +OPTIMIZE TABLE testMT FINAL; +SELECT * FROM testMT order by uid; + +CREATE TABLE testSummingMT (uid String, version UInt32, is_deleted UInt8) ENGINE = SummingMergeTree() Order by (uid) SETTINGS clean_deleted_rows='Always', allow_experimental_replacing_merge_with_cleanup=1; +INSERT INTO testSummingMT (*) VALUES ('d1', 1, 1); +OPTIMIZE TABLE testSummingMT FINAL CLEANUP; -- { serverError CANNOT_ASSIGN_OPTIMIZE } +OPTIMIZE TABLE testSummingMT FINAL; +SELECT * FROM testSummingMT order by uid; + +CREATE TABLE testAggregatingMT (uid String, version UInt32, is_deleted UInt8) ENGINE = AggregatingMergeTree() Order by (uid) SETTINGS clean_deleted_rows='Always', allow_experimental_replacing_merge_with_cleanup=1; +INSERT INTO testAggregatingMT (*) VALUES ('d1', 1, 1); +OPTIMIZE TABLE testAggregatingMT FINAL CLEANUP; -- { serverError CANNOT_ASSIGN_OPTIMIZE } +OPTIMIZE TABLE testAggregatingMT FINAL; +SELECT * FROM testAggregatingMT order by uid; + +CREATE TABLE testCollapsingMT (uid String, version UInt32, is_deleted UInt8, sign Int8) ENGINE = CollapsingMergeTree(sign) Order by (uid) SETTINGS clean_deleted_rows='Always', allow_experimental_replacing_merge_with_cleanup=1; +INSERT INTO testCollapsingMT (*) VALUES ('d1', 1, 1, 1); +OPTIMIZE TABLE testCollapsingMT FINAL CLEANUP; -- { serverError CANNOT_ASSIGN_OPTIMIZE } +OPTIMIZE TABLE testCollapsingMT FINAL; +SELECT * FROM testCollapsingMT order by uid; + +CREATE TABLE testVersionedCMT (uid String, version UInt32, is_deleted UInt8, sign Int8) ENGINE = VersionedCollapsingMergeTree(sign, version) Order by (uid) SETTINGS clean_deleted_rows='Always', allow_experimental_replacing_merge_with_cleanup=1; +INSERT INTO testVersionedCMT (*) VALUES ('d1', 1, 1, 1); +OPTIMIZE TABLE testVersionedCMT FINAL CLEANUP; -- { serverError CANNOT_ASSIGN_OPTIMIZE } +OPTIMIZE TABLE testVersionedCMT FINAL; +SELECT * FROM testVersionedCMT order by uid; diff --git a/tests/queries/0_stateless/02493_analyzer_sum_if_to_count_if.reference b/tests/queries/0_stateless/02493_analyzer_sum_if_to_count_if.reference index eccf51501ed..23e91dc2703 100644 --- a/tests/queries/0_stateless/02493_analyzer_sum_if_to_count_if.reference +++ b/tests/queries/0_stateless/02493_analyzer_sum_if_to_count_if.reference @@ -16,7 +16,7 @@ QUERY id: 0 CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8 CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8 @@ -41,7 +41,7 @@ QUERY id: 0 CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8 CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8 @@ -69,7 +69,7 @@ QUERY id: 0 CONSTANT id: 12, constant_value: UInt64_2, constant_value_type: UInt8 CONSTANT id: 13, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 14, nodes: 1 CONSTANT id: 15, constant_value: UInt64_10, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.reference b/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.reference index 5b808310f0e..01d7fa2a2cb 100644 --- a/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.reference +++ b/tests/queries/0_stateless/02493_analyzer_uniq_injective_functions_elimination.reference @@ -13,7 +13,7 @@ QUERY id: 0 LIST id: 6, nodes: 1 CONSTANT id: 7, constant_value: \'\', constant_value_type: String JOIN TREE - TABLE_FUNCTION id: 8, table_function_name: numbers + TABLE_FUNCTION id: 8, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/02494_analyzer_compound_expression_crash_fix.sql b/tests/queries/0_stateless/02494_analyzer_compound_expression_crash_fix.sql index 4eef7792180..b8d43acbef2 100644 --- a/tests/queries/0_stateless/02494_analyzer_compound_expression_crash_fix.sql +++ b/tests/queries/0_stateless/02494_analyzer_compound_expression_crash_fix.sql @@ -3,7 +3,7 @@ SET allow_experimental_analyzer = 1; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table ( fingerprint UInt16, - fields Array(Tuple(name Array(UInt32), value String)) + fields Nested(name Array(UInt32), value String) ) ENGINE = MergeTree ORDER BY fingerprint; diff --git a/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference b/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference index 88f23334d31..d77fd1028f2 100644 --- a/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference +++ b/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference @@ -35,7 +35,7 @@ QUERY id: 0 CONSTANT id: 15, constant_value: \'other\', constant_value_type: String CONSTANT id: 16, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String JOIN TREE - TABLE id: 7, table_name: system.numbers + TABLE id: 7, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt64 google @@ -78,7 +78,7 @@ QUERY id: 0 CONSTANT id: 17, constant_value: \'google\', constant_value_type: String CONSTANT id: 18, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2)\', constant_value_type: String JOIN TREE - TABLE id: 9, table_name: system.numbers + TABLE id: 9, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt64 other1 @@ -122,7 +122,7 @@ QUERY id: 0 CONSTANT id: 18, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String CONSTANT id: 19, constant_value: \'1\', constant_value_type: String JOIN TREE - TABLE id: 9, table_name: system.numbers + TABLE id: 9, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64 google1 @@ -169,7 +169,7 @@ QUERY id: 0 CONSTANT id: 20, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2)\', constant_value_type: String CONSTANT id: 21, constant_value: \'1\', constant_value_type: String JOIN TREE - TABLE id: 11, table_name: system.numbers + TABLE id: 11, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt64 google @@ -196,7 +196,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: String, source_id: 3 JOIN TREE - QUERY id: 3, alias: t1, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value String PROJECTION @@ -223,7 +223,7 @@ QUERY id: 0 CONSTANT id: 20, constant_value: \'google\', constant_value_type: String CONSTANT id: 21, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2)\', constant_value_type: String JOIN TREE - TABLE id: 12, table_name: system.numbers + TABLE id: 12, alias: __table2, table_name: system.numbers LIMIT CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt64 other @@ -250,7 +250,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: String, source_id: 3 JOIN TREE - QUERY id: 3, alias: t1, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value String PROJECTION @@ -274,7 +274,7 @@ QUERY id: 0 CONSTANT id: 18, constant_value: \'other\', constant_value_type: String CONSTANT id: 19, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String JOIN TREE - TABLE id: 10, table_name: system.numbers + TABLE id: 10, alias: __table2, table_name: system.numbers LIMIT CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64 google google @@ -341,7 +341,7 @@ QUERY id: 0 CONSTANT id: 17, constant_value: \'google\', constant_value_type: String CONSTANT id: 18, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2)\', constant_value_type: String JOIN TREE - TABLE id: 9, table_name: system.numbers + TABLE id: 9, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt64 other other @@ -402,7 +402,7 @@ QUERY id: 0 CONSTANT id: 15, constant_value: \'other\', constant_value_type: String CONSTANT id: 16, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String JOIN TREE - TABLE id: 7, table_name: system.numbers + TABLE id: 7, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt64 other @@ -446,14 +446,14 @@ QUERY id: 0 CONSTANT id: 15, constant_value: \'other\', constant_value_type: String CONSTANT id: 16, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String JOIN TREE - QUERY id: 7, is_subquery: 1 + QUERY id: 7, alias: __table1, is_subquery: 1 PROJECTION COLUMNS number Nullable(Nothing) PROJECTION LIST id: 17, nodes: 1 CONSTANT id: 18, constant_value: NULL, constant_value_type: Nullable(Nothing) JOIN TREE - TABLE id: 19, table_name: system.numbers + TABLE id: 19, alias: __table2, table_name: system.numbers LIMIT CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64 other @@ -482,7 +482,7 @@ QUERY id: 0 CONSTANT id: 7, constant_value: Array_[\'google\', \'censor.net\', \'yahoo\'], constant_value_type: Array(String) CONSTANT id: 8, constant_value: \'other\', constant_value_type: String JOIN TREE - TABLE id: 5, table_name: system.numbers + TABLE id: 5, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt64 google @@ -514,6 +514,6 @@ QUERY id: 0 CONSTANT id: 9, constant_value: \'censor.net\', constant_value_type: String CONSTANT id: 10, constant_value: \'google\', constant_value_type: String JOIN TREE - TABLE id: 7, table_name: system.numbers + TABLE id: 7, alias: __table1, table_name: system.numbers LIMIT CONSTANT id: 11, constant_value: UInt64_10, constant_value_type: UInt64 diff --git a/tests/queries/0_stateless/02498_analyzer_settings_push_down.reference b/tests/queries/0_stateless/02498_analyzer_settings_push_down.reference index 583da07380e..f24edd96996 100644 --- a/tests/queries/0_stateless/02498_analyzer_settings_push_down.reference +++ b/tests/queries/0_stateless/02498_analyzer_settings_push_down.reference @@ -12,7 +12,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value UInt64 PROJECTION @@ -23,7 +23,7 @@ QUERY id: 0 COLUMN id: 7, column_name: value, result_type: Tuple(a UInt64), source_id: 8 CONSTANT id: 9, constant_value: \'a\', constant_value_type: String JOIN TREE - TABLE id: 8, table_name: default.test_table + TABLE id: 8, alias: __table2, table_name: default.test_table SELECT '--'; -- EXPLAIN QUERY TREE SELECT value FROM ( @@ -36,14 +36,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value UInt64 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: value.a, result_type: UInt64, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_table + TABLE id: 6, alias: __table2, table_name: default.test_table SETTINGS optimize_functions_to_subcolumns=1 SELECT '--'; -- @@ -57,7 +57,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value UInt64 PROJECTION @@ -68,7 +68,7 @@ QUERY id: 0 COLUMN id: 7, column_name: value, result_type: Tuple(a UInt64), source_id: 8 CONSTANT id: 9, constant_value: \'a\', constant_value_type: String JOIN TREE - TABLE id: 8, table_name: default.test_table + TABLE id: 8, alias: __table2, table_name: default.test_table SETTINGS optimize_functions_to_subcolumns=0 SETTINGS optimize_functions_to_subcolumns=1 SELECT '--'; @@ -83,7 +83,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value UInt64 PROJECTION @@ -94,7 +94,7 @@ QUERY id: 0 COLUMN id: 7, column_name: value, result_type: Tuple(a UInt64), source_id: 8 CONSTANT id: 9, constant_value: \'a\', constant_value_type: String JOIN TREE - TABLE id: 8, table_name: default.test_table + TABLE id: 8, alias: __table2, table_name: default.test_table SETTINGS optimize_functions_to_subcolumns=0 SELECT '--'; -- @@ -108,13 +108,13 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS value UInt64 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: value.a, result_type: UInt64, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_table + TABLE id: 6, alias: __table2, table_name: default.test_table SETTINGS optimize_functions_to_subcolumns=1 SETTINGS optimize_functions_to_subcolumns=0 diff --git a/tests/queries/0_stateless/02500_prevent_drop_nested_if_empty_part.sql b/tests/queries/0_stateless/02500_prevent_drop_nested_if_empty_part.sql index 529f574d32d..d8564546b0e 100644 --- a/tests/queries/0_stateless/02500_prevent_drop_nested_if_empty_part.sql +++ b/tests/queries/0_stateless/02500_prevent_drop_nested_if_empty_part.sql @@ -2,41 +2,19 @@ DROP TABLE IF EXISTS 02500_nested; SET flatten_nested = 1; -CREATE TABLE 02500_nested(arr Array(Tuple(a Int32, b Int32))) Engine=MergeTree ORDER BY tuple(); -INSERT INTO 02500_nested(arr.a, arr.b) VALUES ([1], [2]); -ALTER TABLE 02500_nested ADD COLUMN z Int32; -ALTER TABLE 02500_nested DROP COLUMN arr; -- { serverError BAD_ARGUMENTS } -DROP TABLE 02500_nested; - -CREATE TABLE 02500_nested(arr Array(Tuple(a Int32, b Int32)), z Int32) Engine=MergeTree ORDER BY tuple(); -INSERT INTO 02500_nested(arr.a, arr.b, z) VALUES ([1], [2], 2); -ALTER TABLE 02500_nested DROP COLUMN arr; -DROP TABLE 02500_nested; - CREATE TABLE 02500_nested(nes Nested(a Int32, b Int32)) Engine=MergeTree ORDER BY tuple(); INSERT INTO 02500_nested(nes.a, nes.b) VALUES ([1], [2]); ALTER TABLE 02500_nested ADD COLUMN z Int32; ALTER TABLE 02500_nested DROP COLUMN nes; -- { serverError BAD_ARGUMENTS } DROP TABLE 02500_nested; -CREATE TABLE 02500_nested(nes Array(Tuple(a Int32, b Int32)), z Int32) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE 02500_nested(nes Nested(a Int32, b Int32), z Int32) Engine=MergeTree ORDER BY tuple(); INSERT INTO 02500_nested(nes.a, nes.b, z) VALUES ([1], [2], 2); ALTER TABLE 02500_nested DROP COLUMN nes; DROP TABLE 02500_nested; SET flatten_nested = 0; -CREATE TABLE 02500_nested(arr Array(Tuple(a Int32, b Int32))) Engine=MergeTree ORDER BY tuple(); -INSERT INTO 02500_nested(arr) VALUES ([(1, 2)]); -ALTER TABLE 02500_nested ADD COLUMN z Int32; -ALTER TABLE 02500_nested DROP COLUMN arr; -- { serverError BAD_ARGUMENTS } -DROP TABLE 02500_nested; - -CREATE TABLE 02500_nested(arr Array(Tuple(a Int32, b Int32)), z Int32) Engine=MergeTree ORDER BY tuple(); -INSERT INTO 02500_nested(arr, z) VALUES ([(1, 2)], 2); -ALTER TABLE 02500_nested DROP COLUMN arr; -DROP TABLE 02500_nested; - CREATE TABLE 02500_nested(nes Nested(a Int32, b Int32)) Engine=MergeTree ORDER BY tuple(); INSERT INTO 02500_nested(nes) VALUES ([(1, 2)]); ALTER TABLE 02500_nested ADD COLUMN z Int32; diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index 51e009dcd91..a5a71560d00 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -6,43 +6,43 @@ SELECT count() FROM a JOIN b ON b.b1 = a.a1 JOIN c ON c.c1 = b.b1 JOIN d ON d.d1 Expression ((Project names + Projection)) Header: count() UInt64 Aggregating - Header: a2_4 String + Header: __table1.a2 String count() UInt64 Expression ((Before GROUP BY + DROP unused columns after JOIN)) - Header: a2_4 String + Header: __table1.a2 String Join (JOIN FillRightFirst) - Header: a2_4 String - c1_2 UInt64 + Header: __table1.a2 String + __table3.c1 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) - Header: a2_4 String - c1_2 UInt64 + Header: __table1.a2 String + __table3.c1 UInt64 Join (JOIN FillRightFirst) - Header: a2_4 String - b1_0 UInt64 - c1_2 UInt64 + Header: __table1.a2 String + __table2.b1 UInt64 + __table3.c1 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) - Header: a2_4 String - b1_0 UInt64 + Header: __table1.a2 String + __table2.b1 UInt64 Join (JOIN FillRightFirst) - Header: a1_1 UInt64 - a2_4 String - b1_0 UInt64 + Header: __table1.a1 UInt64 + __table1.a2 String + __table2.b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: a1_1 UInt64 - a2_4 String + Header: __table1.a1 UInt64 + __table1.a2 String ReadFromMemoryStorage Header: a1 UInt64 a2 String Expression ((JOIN actions + Change column names to column identifiers)) - Header: b1_0 UInt64 + Header: __table2.b1 UInt64 ReadFromMemoryStorage Header: b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: c1_2 UInt64 + Header: __table3.c1 UInt64 ReadFromMemoryStorage Header: c1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: d1_3 UInt64 + Header: __table4.d1 UInt64 ReadFromMemoryStorage Header: d1 UInt64 EXPLAIN PLAN header = 1 @@ -52,38 +52,38 @@ Expression ((Project names + (Projection + DROP unused columns after JOIN))) Header: a2 String d2 String Join (JOIN FillRightFirst) - Header: a2_0 String - k_2 UInt64 - d2_1 String - Expression (DROP unused columns after JOIN) - Header: a2_0 String - k_2 UInt64 + Header: __table1.a2 String + __table1.k UInt64 + __table4.d2 String + Expression ((Actions for left table alias column keys + DROP unused columns after JOIN)) + Header: __table1.a2 String + __table1.k UInt64 Join (JOIN FillRightFirst) - Header: a2_0 String - k_2 UInt64 - Expression (DROP unused columns after JOIN) - Header: a2_0 String - k_2 UInt64 + Header: __table1.a2 String + __table1.k UInt64 + Expression ((Actions for left table alias column keys + DROP unused columns after JOIN)) + Header: __table1.a2 String + __table1.k UInt64 Join (JOIN FillRightFirst) - Header: a2_0 String - k_2 UInt64 - Expression (Change column names to column identifiers) - Header: a2_0 String - k_2 UInt64 + Header: __table1.a2 String + __table1.k UInt64 + Expression ((Actions for left table alias column keys + Change column names to column identifiers)) + Header: __table1.a2 String + __table1.k UInt64 ReadFromMemoryStorage Header: a2 String k UInt64 - Expression (Change column names to column identifiers) - Header: k_3 UInt64 + Expression ((Actions for right table alias column keys + Change column names to column identifiers)) + Header: __table2.k UInt64 ReadFromMemoryStorage Header: k UInt64 - Expression (Change column names to column identifiers) - Header: k_4 UInt64 + Expression ((Actions for right table alias column keys + Change column names to column identifiers)) + Header: __table3.k UInt64 ReadFromMemoryStorage Header: k UInt64 - Expression (Change column names to column identifiers) - Header: d2_1 String - k_5 UInt64 + Expression ((Actions for right table alias column keys + Change column names to column identifiers)) + Header: __table4.d2 String + __table4.k UInt64 ReadFromMemoryStorage Header: d2 String k UInt64 @@ -97,55 +97,55 @@ WHERE c.c2 != '' ORDER BY a.a2 Expression (Project names) Header: bx String Sorting (Sorting for ORDER BY) - Header: a2_6 String - bx_0 String + Header: __table1.a2 String + __table2.bx String Expression ((Before ORDER BY + (Projection + ))) - Header: a2_6 String - bx_0 String + Header: __table1.a2 String + __table2.bx String Join (JOIN FillRightFirst) - Header: a2_6 String - bx_0 String - c2_5 String - c1_3 UInt64 + Header: __table1.a2 String + __table2.bx String + __table4.c2 String + __table4.c1 UInt64 Expression - Header: a2_6 String - bx_0 String - c2_5 String - c1_3 UInt64 + Header: __table1.a2 String + __table2.bx String + __table4.c2 String + __table4.c1 UInt64 Join (JOIN FillRightFirst) - Header: a2_6 String - bx_0 String - b1_1 UInt64 - c2_5 String - c1_3 UInt64 + Header: __table1.a2 String + __table2.bx String + __table2.b1 UInt64 + __table4.c2 String + __table4.c1 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) - Header: a2_6 String - bx_0 String - b1_1 UInt64 + Header: __table1.a2 String + __table2.bx String + __table2.b1 UInt64 Join (JOIN FillRightFirst) - Header: a1_2 UInt64 - a2_6 String - bx_0 String - b1_1 UInt64 + Header: __table1.a1 UInt64 + __table1.a2 String + __table2.bx String + __table2.b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: a1_2 UInt64 - a2_6 String + Header: __table1.a1 UInt64 + __table1.a2 String ReadFromMemoryStorage Header: a1 UInt64 a2 String Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) - Header: b1_1 UInt64 - bx_0 String + Header: __table2.b1 UInt64 + __table2.bx String ReadFromMemoryStorage Header: b1 UInt64 b2 String Filter (( + (JOIN actions + Change column names to column identifiers))) - Header: c1_3 UInt64 - c2_5 String + Header: __table4.c1 UInt64 + __table4.c2 String ReadFromMemoryStorage Header: c1 UInt64 c2 String Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) - Header: d1_4 UInt64 + Header: __table5.d1 UInt64 ReadFromSystemNumbers Header: number UInt64 diff --git a/tests/queries/0_stateless/02518_rewrite_aggregate_function_with_if.reference b/tests/queries/0_stateless/02518_rewrite_aggregate_function_with_if.reference index 37680adf8e0..15543789c1d 100644 --- a/tests/queries/0_stateless/02518_rewrite_aggregate_function_with_if.reference +++ b/tests/queries/0_stateless/02518_rewrite_aggregate_function_with_if.reference @@ -17,7 +17,7 @@ QUERY id: 0 COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 @@ -40,7 +40,7 @@ QUERY id: 0 CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8 COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 @@ -63,7 +63,7 @@ QUERY id: 0 COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 CONSTANT id: 11, constant_value: NULL, constant_value_type: Nullable(Nothing) JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 @@ -86,7 +86,7 @@ QUERY id: 0 CONSTANT id: 11, constant_value: NULL, constant_value_type: Nullable(Nothing) COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 @@ -109,7 +109,7 @@ QUERY id: 0 COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 CONSTANT id: 11, constant_value: NULL, constant_value_type: Nullable(Nothing) JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 @@ -132,7 +132,7 @@ QUERY id: 0 CONSTANT id: 11, constant_value: NULL, constant_value_type: Nullable(Nothing) COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 @@ -160,7 +160,7 @@ QUERY id: 0 COLUMN id: 12, column_name: number, result_type: UInt64, source_id: 13 CONSTANT id: 15, constant_value: NULL, constant_value_type: Nullable(Nothing) JOIN TREE - TABLE_FUNCTION id: 13, table_function_name: numbers + TABLE_FUNCTION id: 13, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 16, nodes: 1 CONSTANT id: 17, constant_value: UInt64_100, constant_value_type: UInt8 @@ -188,7 +188,7 @@ QUERY id: 0 CONSTANT id: 15, constant_value: NULL, constant_value_type: Nullable(Nothing) COLUMN id: 12, column_name: number, result_type: UInt64, source_id: 13 JOIN TREE - TABLE_FUNCTION id: 13, table_function_name: numbers + TABLE_FUNCTION id: 13, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 16, nodes: 1 CONSTANT id: 17, constant_value: UInt64_100, constant_value_type: UInt8 @@ -207,7 +207,7 @@ QUERY id: 0 COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5 CONSTANT id: 8, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 5, table_function_name: numbers + TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_100, constant_value_type: UInt8 @@ -229,7 +229,7 @@ QUERY id: 0 COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 5, table_function_name: numbers + TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 11, nodes: 1 CONSTANT id: 12, constant_value: UInt64_100, constant_value_type: UInt8 @@ -248,7 +248,7 @@ QUERY id: 0 COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5 CONSTANT id: 8, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 5, table_function_name: numbers + TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_100, constant_value_type: UInt8 @@ -270,7 +270,7 @@ QUERY id: 0 COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 5, table_function_name: numbers + TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 11, nodes: 1 CONSTANT id: 12, constant_value: UInt64_100, constant_value_type: UInt8 @@ -289,7 +289,7 @@ QUERY id: 0 COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5 CONSTANT id: 8, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 5, table_function_name: numbers + TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 9, nodes: 1 CONSTANT id: 10, constant_value: UInt64_100, constant_value_type: UInt8 @@ -311,7 +311,7 @@ QUERY id: 0 COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5 CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 5, table_function_name: numbers + TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 11, nodes: 1 CONSTANT id: 12, constant_value: UInt64_100, constant_value_type: UInt8 @@ -335,7 +335,7 @@ QUERY id: 0 COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 CONSTANT id: 12, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 13, nodes: 1 CONSTANT id: 14, constant_value: UInt64_100, constant_value_type: UInt8 @@ -362,7 +362,7 @@ QUERY id: 0 COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9 CONSTANT id: 14, constant_value: UInt64_2, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 9, table_function_name: numbers + TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 15, nodes: 1 CONSTANT id: 16, constant_value: UInt64_100, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/02521_avro_union_null_nested.reference b/tests/queries/0_stateless/02521_avro_union_null_nested.reference index e4818b4bcac..a3cb5ba4858 100644 --- a/tests/queries/0_stateless/02521_avro_union_null_nested.reference +++ b/tests/queries/0_stateless/02521_avro_union_null_nested.reference @@ -5,7 +5,7 @@ added_snapshot_id Nullable(Int64) added_data_files_count Nullable(Int32) existing_data_files_count Nullable(Int32) deleted_data_files_count Nullable(Int32) -partitions Array(Tuple(contains_null Bool, contains_nan Nullable(Bool), lower_bound Nullable(String), upper_bound Nullable(String))) +partitions Array(Tuple(\n contains_null Bool,\n contains_nan Nullable(Bool),\n lower_bound Nullable(String),\n upper_bound Nullable(String))) added_rows_count Nullable(Int64) existing_rows_count Nullable(Int64) deleted_rows_count Nullable(Int64) diff --git a/tests/queries/0_stateless/02522_avro_complicate_schema.reference b/tests/queries/0_stateless/02522_avro_complicate_schema.reference index 55c0369020f..a885163d609 100644 --- a/tests/queries/0_stateless/02522_avro_complicate_schema.reference +++ b/tests/queries/0_stateless/02522_avro_complicate_schema.reference @@ -1,5 +1,5 @@ status Int32 snapshot_id Nullable(Int64) -data_file Tuple(file_path String, file_format String, partition Tuple(vendor_id Nullable(Int64)), record_count Int64, file_size_in_bytes Int64, block_size_in_bytes Int64, column_sizes Array(Tuple(key Int32, value Int64)), value_counts Array(Tuple(key Int32, value Int64)), null_value_counts Array(Tuple(key Int32, value Int64)), nan_value_counts Array(Tuple(key Int32, value Int64)), lower_bounds Array(Tuple(key Int32, value String)), upper_bounds Array(Tuple(key Int32, value String)), key_metadata Nullable(String), split_offsets Array(Int64), sort_order_id Nullable(Int32)) +data_file Tuple(\n file_path String,\n file_format String,\n partition Tuple(\n vendor_id Nullable(Int64)),\n record_count Int64,\n file_size_in_bytes Int64,\n block_size_in_bytes Int64,\n column_sizes Array(Tuple(\n key Int32,\n value Int64)),\n value_counts Array(Tuple(\n key Int32,\n value Int64)),\n null_value_counts Array(Tuple(\n key Int32,\n value Int64)),\n nan_value_counts Array(Tuple(\n key Int32,\n value Int64)),\n lower_bounds Array(Tuple(\n key Int32,\n value String)),\n upper_bounds Array(Tuple(\n key Int32,\n value String)),\n key_metadata Nullable(String),\n split_offsets Array(Int64),\n sort_order_id Nullable(Int32)) 1 6850377589038341628 ('file:/warehouse/nyc.db/taxis/data/vendor_id=1/00000-0-c070e655-dc44-43d2-a01a-484f107210cb-00001.parquet','PARQUET',(1),2,1565,67108864,[(1,87),(2,51),(3,51),(4,57),(5,51)],[(1,2),(2,2),(3,2),(4,2),(5,2)],[(1,0),(2,0),(3,0),(4,0),(5,0)],[(3,0),(4,0)],[(1,'\0\0\0\0\0\0\0'),(2,'C\0\0\0\0\0'),(3,'ff?'),(4,'p=\nף.@'),(5,'N')],[(1,'\0\0\0\0\0\0\0'),(2,'C\0\0\0\0\0'),(3,'ffA'),(4,'q=\nףE@'),(5,'Y')],NULL,[4],0) 1 6850377589038341628 ('file:/warehouse/nyc.db/taxis/data/vendor_id=2/00000-0-c070e655-dc44-43d2-a01a-484f107210cb-00002.parquet','PARQUET',(2),2,1620,67108864,[(1,87),(2,51),(3,51),(4,57),(5,89)],[(1,2),(2,2),(3,2),(4,2),(5,2)],[(1,0),(2,0),(3,0),(4,0),(5,0)],[(3,0),(4,0)],[(1,'\0\0\0\0\0\0\0'),(2,'C\0\0\0\0\0'),(3,'fff?'),(4,'Q"@'),(5,'N')],[(1,'\0\0\0\0\0\0\0'),(2,'C\0\0\0\0\0'),(3,'\0\0 @'),(4,'fffff&6@'),(5,'N')],NULL,[4],0) diff --git a/tests/queries/0_stateless/02534_analyzer_grouping_function.reference b/tests/queries/0_stateless/02534_analyzer_grouping_function.reference index fcbf625ef22..1b496644547 100644 --- a/tests/queries/0_stateless/02534_analyzer_grouping_function.reference +++ b/tests/queries/0_stateless/02534_analyzer_grouping_function.reference @@ -16,7 +16,7 @@ QUERY id: 0 LIST id: 7, nodes: 1 COLUMN id: 8, column_name: value, result_type: String, source_id: 5 JOIN TREE - TABLE id: 5, table_name: default.test_table + TABLE id: 5, alias: __table1, table_name: default.test_table GROUP BY LIST id: 9, nodes: 2 COLUMN id: 4, column_name: id, result_type: UInt64, source_id: 5 @@ -42,7 +42,7 @@ QUERY id: 0, group_by_type: rollup COLUMN id: 9, column_name: __grouping_set, result_type: UInt64 COLUMN id: 10, column_name: value, result_type: String, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_table + TABLE id: 6, alias: __table1, table_name: default.test_table GROUP BY LIST id: 11, nodes: 2 COLUMN id: 5, column_name: id, result_type: UInt64, source_id: 6 @@ -70,7 +70,7 @@ QUERY id: 0, group_by_type: cube COLUMN id: 9, column_name: __grouping_set, result_type: UInt64 COLUMN id: 10, column_name: value, result_type: String, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_table + TABLE id: 6, alias: __table1, table_name: default.test_table GROUP BY LIST id: 11, nodes: 2 COLUMN id: 5, column_name: id, result_type: UInt64, source_id: 6 @@ -99,7 +99,7 @@ QUERY id: 0, group_by_type: grouping_sets COLUMN id: 9, column_name: __grouping_set, result_type: UInt64 COLUMN id: 10, column_name: value, result_type: String, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_table + TABLE id: 6, alias: __table1, table_name: default.test_table GROUP BY LIST id: 11, nodes: 2 LIST id: 12, nodes: 1 @@ -128,7 +128,7 @@ QUERY id: 0, group_by_type: grouping_sets COLUMN id: 9, column_name: __grouping_set, result_type: UInt64 COLUMN id: 10, column_name: value, result_type: String, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_table + TABLE id: 6, alias: __table1, table_name: default.test_table GROUP BY LIST id: 11, nodes: 2 LIST id: 12, nodes: 1 diff --git a/tests/queries/0_stateless/02564_analyzer_cross_to_inner.reference b/tests/queries/0_stateless/02564_analyzer_cross_to_inner.reference index e4d7ff55b86..5b9bc206695 100644 --- a/tests/queries/0_stateless/02564_analyzer_cross_to_inner.reference +++ b/tests/queries/0_stateless/02564_analyzer_cross_to_inner.reference @@ -29,9 +29,9 @@ QUERY id: 0 LEFT TABLE EXPRESSION JOIN id: 11, strictness: ALL, kind: INNER LEFT TABLE EXPRESSION - TABLE id: 3, table_name: default.t1 + TABLE id: 3, alias: __table1, table_name: default.t1 RIGHT TABLE EXPRESSION - TABLE id: 6, table_name: default.t2 + TABLE id: 6, alias: __table2, table_name: default.t2 JOIN EXPRESSION FUNCTION id: 12, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -48,14 +48,14 @@ QUERY id: 0 COLUMN id: 21, column_name: a, result_type: UInt64, source_id: 6 CONSTANT id: 22, constant_value: UInt64_0, constant_value_type: UInt8 RIGHT TABLE EXPRESSION - QUERY id: 9, alias: t3, is_subquery: 1 + QUERY id: 9, alias: __table3, is_subquery: 1 PROJECTION COLUMNS x UInt64 PROJECTION LIST id: 23, nodes: 1 COLUMN id: 24, column_name: a, result_type: UInt64, source_id: 25 JOIN TREE - TABLE id: 25, table_name: default.t3 + TABLE id: 25, alias: __table4, table_name: default.t3 WHERE FUNCTION id: 26, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -97,18 +97,18 @@ QUERY id: 0 LEFT TABLE EXPRESSION JOIN id: 11, kind: COMMA LEFT TABLE EXPRESSION - TABLE id: 3, table_name: default.t1 + TABLE id: 3, alias: __table1, table_name: default.t1 RIGHT TABLE EXPRESSION - TABLE id: 6, table_name: default.t2 + TABLE id: 6, alias: __table2, table_name: default.t2 RIGHT TABLE EXPRESSION - QUERY id: 9, alias: t3, is_subquery: 1 + QUERY id: 9, alias: __table3, is_subquery: 1 PROJECTION COLUMNS x UInt64 PROJECTION LIST id: 12, nodes: 1 COLUMN id: 13, column_name: a, result_type: UInt64, source_id: 14 JOIN TREE - TABLE id: 14, table_name: default.t3 + TABLE id: 14, alias: __table4, table_name: default.t3 WHERE FUNCTION id: 15, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -166,9 +166,9 @@ QUERY id: 0 LEFT TABLE EXPRESSION JOIN id: 11, strictness: ALL, kind: INNER LEFT TABLE EXPRESSION - TABLE id: 3, table_name: default.t1 + TABLE id: 3, alias: __table1, table_name: default.t1 RIGHT TABLE EXPRESSION - TABLE id: 6, table_name: default.t2 + TABLE id: 6, alias: __table2, table_name: default.t2 JOIN EXPRESSION FUNCTION id: 12, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -185,14 +185,14 @@ QUERY id: 0 COLUMN id: 21, column_name: a, result_type: UInt64, source_id: 6 CONSTANT id: 22, constant_value: UInt64_0, constant_value_type: UInt8 RIGHT TABLE EXPRESSION - QUERY id: 9, alias: t3, is_subquery: 1 + QUERY id: 9, alias: __table3, is_subquery: 1 PROJECTION COLUMNS x UInt64 PROJECTION LIST id: 23, nodes: 1 COLUMN id: 24, column_name: a, result_type: UInt64, source_id: 25 JOIN TREE - TABLE id: 25, table_name: default.t3 + TABLE id: 25, alias: __table4, table_name: default.t3 WHERE FUNCTION id: 26, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.reference b/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.reference index b8c68f90135..dd107065380 100644 --- a/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.reference +++ b/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.reference @@ -1,21 +1,21 @@ Expression ((Project names + (Projection + ))) Header: number UInt64 -Actions: INPUT : 0 -> number_1 UInt64 : 0 - ALIAS number_1 :: 0 -> number UInt64 : 1 - ALIAS number :: 1 -> number_0 UInt64 : 0 - ALIAS number_0 :: 0 -> number UInt64 : 1 +Actions: INPUT : 0 -> __table2.number UInt64 : 0 + ALIAS __table2.number :: 0 -> number UInt64 : 1 + ALIAS number :: 1 -> __table1.number UInt64 : 0 + ALIAS __table1.number :: 0 -> number UInt64 : 1 Positions: 1 Sorting (Sorting for ORDER BY) Header: ignore(2_UInt8) UInt8 - number_1 UInt64 + __table2.number UInt64 Sort description: ignore(2_UInt8) ASC Filter (( + (Before ORDER BY + (Projection + Change column names to column identifiers)))) Header: ignore(2_UInt8) UInt8 - number_1 UInt64 + __table2.number UInt64 Filter column: ignore(2_UInt8) Actions: INPUT : 0 -> number UInt64 : 0 COLUMN Const(UInt8) -> 2_UInt8 UInt8 : 1 - ALIAS number :: 0 -> number_1 UInt64 : 2 + ALIAS number :: 0 -> __table2.number UInt64 : 2 FUNCTION ignore(2_UInt8 :: 1) -> ignore(2_UInt8) UInt8 : 0 Positions: 0 2 ReadFromSystemNumbers diff --git a/tests/queries/0_stateless/02576_rewrite_array_exists_to_has.reference b/tests/queries/0_stateless/02576_rewrite_array_exists_to_has.reference index b6964976c20..f4e09c4b4de 100644 --- a/tests/queries/0_stateless/02576_rewrite_array_exists_to_has.reference +++ b/tests/queries/0_stateless/02576_rewrite_array_exists_to_has.reference @@ -26,7 +26,7 @@ QUERY id: 0 LIST id: 14, nodes: 1 CONSTANT id: 15, constant_value: UInt64_10, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 16, table_function_name: numbers + TABLE_FUNCTION id: 16, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 17, nodes: 1 CONSTANT id: 18, constant_value: UInt64_10, constant_value_type: UInt8 @@ -58,7 +58,7 @@ QUERY id: 0 LIST id: 14, nodes: 1 CONSTANT id: 15, constant_value: UInt64_10, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 16, table_function_name: numbers + TABLE_FUNCTION id: 16, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 17, nodes: 1 CONSTANT id: 18, constant_value: UInt64_10, constant_value_type: UInt8 @@ -81,7 +81,7 @@ QUERY id: 0 CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt8 CONSTANT id: 10, constant_value: UInt64_5, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8 @@ -104,7 +104,7 @@ QUERY id: 0 CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt8 CONSTANT id: 10, constant_value: UInt64_5, constant_value_type: UInt8 JOIN TREE - TABLE_FUNCTION id: 11, table_function_name: numbers + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers ARGUMENTS LIST id: 12, nodes: 1 CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference index cc74b0237fe..cf60d63b1cf 100644 --- a/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference +++ b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference @@ -9,7 +9,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02668_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer WHERE FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -26,7 +26,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02668_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer WHERE FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -42,7 +42,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02668_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer WHERE CONSTANT id: 5, constant_value: UInt64_0, constant_value_type: UInt8 3 another @@ -55,7 +55,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02668_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer WHERE FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -80,12 +80,47 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02668_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer WHERE FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 6, nodes: 2 COLUMN id: 7, column_name: a, result_type: Int32, source_id: 3 CONSTANT id: 8, constant_value: UInt64_2, constant_value_type: UInt8 +2 test2 +QUERY id: 0 + PROJECTION COLUMNS + a Int32 + b LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 + COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer + WHERE + FUNCTION id: 5, function_name: notIn, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: a, result_type: Int32, source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(UInt64_1, UInt64_3), constant_value_type: Tuple(UInt8, UInt8) +2 test2 +3 another +QUERY id: 0 + PROJECTION COLUMNS + a Int32 + b LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 + COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer + WHERE + FUNCTION id: 5, function_name: notEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: a, result_type: Int32, source_id: 3 + CONSTANT id: 8, constant_value: UInt64_1, constant_value_type: UInt8 1 1 diff --git a/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.sql b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.sql index 5dee450086c..7d624195df9 100644 --- a/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.sql +++ b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.sql @@ -8,6 +8,7 @@ ENGINE=Memory; INSERT INTO 02668_logical_optimizer VALUES (1, 'test'), (2, 'test2'), (3, 'another'); +-- Chain of OR equals SET optimize_min_equality_disjunction_chain_length = 2; SELECT * FROM 02668_logical_optimizer WHERE a = 1 OR 3 = a OR 1 = a; @@ -16,6 +17,7 @@ EXPLAIN QUERY TREE SELECT * FROM 02668_logical_optimizer WHERE a = 1 OR 3 = a OR SELECT * FROM 02668_logical_optimizer WHERE a = 1 OR 1 = a; EXPLAIN QUERY TREE SELECT * FROM 02668_logical_optimizer WHERE a = 1 OR 1 = a; +-- Chain of AND equals SELECT * FROM 02668_logical_optimizer WHERE a = 1 AND 2 = a; EXPLAIN QUERY TREE SELECT * FROM 02668_logical_optimizer WHERE a = 1 AND 2 = a; @@ -25,6 +27,15 @@ EXPLAIN QUERY TREE SELECT * FROM 02668_logical_optimizer WHERE a = 3 AND b = 'an SELECT * FROM 02668_logical_optimizer WHERE a = 2 AND 2 = a; EXPLAIN QUERY TREE SELECT * FROM 02668_logical_optimizer WHERE a = 2 AND 2 = a; +-- Chain of AND notEquals +SET optimize_min_inequality_conjunction_chain_length = 2; + +SELECT * FROM 02668_logical_optimizer WHERE a <> 1 AND 3 <> a AND 1 <> a; +EXPLAIN QUERY TREE SELECT * FROM 02668_logical_optimizer WHERE a <> 1 AND 3 <> a AND 1 <> a; + +SELECT * FROM 02668_logical_optimizer WHERE a <> 1 AND 1 <> a; +EXPLAIN QUERY TREE SELECT * FROM 02668_logical_optimizer WHERE a <> 1 AND 1 <> a; + SELECT a FROM 02668_logical_optimizer WHERE (b = 'test') AND ('test' = b); SELECT (k = 3) OR ( (k = 1) OR (k = 2) OR ( (NULL OR 1) = k ) ) FROM ( SELECT materialize(1) AS k ); diff --git a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference index 2630c5b95b6..e6c4d5768af 100644 --- a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference +++ b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference @@ -2,31 +2,31 @@ Expression ((Project names + (Projection + ))) Header: t1.id UInt64 t1.value String t2.value String -Actions: INPUT : 0 -> id_0 UInt64 : 0 - INPUT : 1 -> value_1 String : 1 - INPUT : 2 -> value_2 String : 2 - ALIAS id_0 :: 0 -> t1.id UInt64 : 3 - ALIAS value_1 :: 1 -> t1.value String : 0 - ALIAS value_2 :: 2 -> t2.value String : 1 +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value String : 1 + INPUT : 2 -> __table2.value String : 2 + ALIAS __table1.id :: 0 -> t1.id UInt64 : 3 + ALIAS __table1.value :: 1 -> t1.value String : 0 + ALIAS __table2.value :: 2 -> t2.value String : 1 Positions: 3 0 1 FilledJoin (Filled JOIN) - Header: id_0 UInt64 - value_1 String - value_2 String + Header: __table1.id UInt64 + __table1.value String + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin - Clauses: [(id_0) = (id)] + Clauses: [(__table1.id) = (id)] Filter (( + (JOIN actions + Change column names to column identifiers))) - Header: id_0 UInt64 - value_1 String - Filter column: equals(id_0, 0_UInt8) (removed) + Header: __table1.id UInt64 + __table1.value String + Filter column: equals(__table1.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 2 - ALIAS id :: 0 -> id_0 UInt64 : 3 - ALIAS value :: 1 -> value_1 String : 0 - FUNCTION equals(id_0 : 3, 0_UInt8 :: 2) -> equals(id_0, 0_UInt8) UInt8 : 1 + ALIAS id :: 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 0 + FUNCTION equals(__table1.id : 3, 0_UInt8 :: 2) -> equals(__table1.id, 0_UInt8) UInt8 : 1 Positions: 1 3 0 ReadFromMergeTree (default.test_table) Header: id UInt64 diff --git a/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference b/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference index cc16a1fce02..4a4e338438b 100644 --- a/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference +++ b/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference @@ -29,10 +29,10 @@ Header: id UInt64 value String Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 - ALIAS id :: 0 -> id_0 UInt64 : 2 - ALIAS value :: 1 -> value_1 String : 0 - ALIAS id_0 :: 2 -> id UInt64 : 1 - ALIAS value_1 :: 0 -> value String : 2 + ALIAS id :: 0 -> __table1.id UInt64 : 2 + ALIAS value :: 1 -> __table1.value String : 0 + ALIAS __table1.id :: 2 -> id UInt64 : 1 + ALIAS __table1.value :: 0 -> value String : 2 Positions: 1 2 ReadFromMergeTree (default.test_table) Header: id UInt64 diff --git a/tests/queries/0_stateless/02698_marked_dropped_tables.reference b/tests/queries/0_stateless/02698_marked_dropped_tables.reference index 44906da9527..84e478cc383 100644 --- a/tests/queries/0_stateless/02698_marked_dropped_tables.reference +++ b/tests/queries/0_stateless/02698_marked_dropped_tables.reference @@ -6,3 +6,71 @@ uuid UUID engine String metadata_dropped_path String table_dropped_time DateTime +default 25400_dropped_tables all_1_1_0 +default 25400_dropped_tables all_2_2_0 +partition String +name String +uuid UUID +part_type String +active UInt8 +marks UInt64 +rows UInt64 +bytes_on_disk UInt64 +data_compressed_bytes UInt64 +data_uncompressed_bytes UInt64 +primary_key_size UInt64 +marks_bytes UInt64 +secondary_indices_compressed_bytes UInt64 +secondary_indices_uncompressed_bytes UInt64 +secondary_indices_marks_bytes UInt64 +modification_time DateTime +remove_time DateTime +refcount UInt32 +min_date Date +max_date Date +min_time DateTime +max_time DateTime +partition_id String +min_block_number Int64 +max_block_number Int64 +level UInt32 +data_version UInt64 +primary_key_bytes_in_memory UInt64 +primary_key_bytes_in_memory_allocated UInt64 +is_frozen UInt8 +database String +table String +engine String +disk_name String +path String +hash_of_all_files String +hash_of_uncompressed_files String +uncompressed_hash_of_compressed_files String +delete_ttl_info_min DateTime +delete_ttl_info_max DateTime +move_ttl_info.expression Array(String) +move_ttl_info.min Array(DateTime) +move_ttl_info.max Array(DateTime) +default_compression_codec String +recompression_ttl_info.expression Array(String) +recompression_ttl_info.min Array(DateTime) +recompression_ttl_info.max Array(DateTime) +group_by_ttl_info.expression Array(String) +group_by_ttl_info.min Array(DateTime) +group_by_ttl_info.max Array(DateTime) +rows_where_ttl_info.expression Array(String) +rows_where_ttl_info.min Array(DateTime) +rows_where_ttl_info.max Array(DateTime) +projections Array(String) +visible UInt8 +creation_tid Tuple(UInt64, UInt64, UUID) +removal_tid_lock UInt64 +removal_tid Tuple(UInt64, UInt64, UUID) +creation_csn UInt64 +removal_csn UInt64 +has_lightweight_delete UInt8 +last_removal_attempt_time DateTime +removal_state String +bytes UInt64 ALIAS bytes_on_disk +marks_size UInt64 ALIAS marks_bytes +part_name String ALIAS name diff --git a/tests/queries/0_stateless/02698_marked_dropped_tables.sql b/tests/queries/0_stateless/02698_marked_dropped_tables.sql index 9bf6579b583..a3686159a78 100644 --- a/tests/queries/0_stateless/02698_marked_dropped_tables.sql +++ b/tests/queries/0_stateless/02698_marked_dropped_tables.sql @@ -4,8 +4,11 @@ SET database_atomic_wait_for_drop_and_detach_synchronously = 0; DROP TABLE IF EXISTS 25400_dropped_tables; CREATE TABLE 25400_dropped_tables (id Int32) Engine=MergeTree() ORDER BY id; +INSERT INTO 25400_dropped_tables VALUES (1),(2); +INSERT INTO 25400_dropped_tables VALUES (3),(4); DROP TABLE 25400_dropped_tables; SELECT table, engine FROM system.dropped_tables WHERE database = currentDatabase() LIMIT 1; DESCRIBE TABLE system.dropped_tables; - +SELECT database, table, name FROM system.dropped_tables_parts WHERE database = currentDatabase() and table = '25400_dropped_tables'; +DESCRIBE TABLE system.dropped_tables_parts; diff --git a/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.reference b/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.reference index e7f46a974e6..c25b446dcdc 100644 --- a/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.reference +++ b/tests/queries/0_stateless/02702_logical_optimizer_with_nulls.reference @@ -9,7 +9,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02702_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02702_logical_optimizer WHERE FUNCTION id: 5, function_name: or, function_type: ordinary, result_type: Nullable(UInt8) ARGUMENTS @@ -41,7 +41,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02702_logical_optimizer + TABLE id: 3, alias: __table1, table_name: default.02702_logical_optimizer WHERE FUNCTION id: 5, function_name: or, function_type: ordinary, result_type: Nullable(UInt8) ARGUMENTS @@ -68,7 +68,7 @@ QUERY id: 0 COLUMN id: 2, column_name: a, result_type: Nullable(Int32), source_id: 3 COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.02702_logical_optimizer_with_null_column + TABLE id: 3, alias: __table1, table_name: default.02702_logical_optimizer_with_null_column WHERE FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: Nullable(UInt8) ARGUMENTS diff --git a/tests/queries/0_stateless/02725_parquet_preserve_order.reference b/tests/queries/0_stateless/02725_parquet_preserve_order.reference index e9c8f99bb33..3f410c13ec4 100644 --- a/tests/queries/0_stateless/02725_parquet_preserve_order.reference +++ b/tests/queries/0_stateless/02725_parquet_preserve_order.reference @@ -3,10 +3,10 @@ 2 (Expression) ExpressionTransform - (ReadFromStorage) + (ReadFromFile) File 0 → 1 (Expression) ExpressionTransform × 2 - (ReadFromStorage) + (ReadFromFile) Resize 1 → 2 File 0 → 1 diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference index 35573110550..3b8a394a522 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference @@ -9,4 +9,4 @@ 7885388429666205427 8124171311239967992 1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n allow_experimental_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1; -0 2 SELECT `join_inner_table__fuzz_146_replicated`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` SETTINGS allow_experimental_analyzer = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\', allow_experimental_parallel_reading_from_replicas = 1 +0 2 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1` SETTINGS allow_experimental_analyzer = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\', allow_experimental_parallel_reading_from_replicas = 1 diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference index 0fd2f694aeb..63658890119 100644 --- a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference +++ b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference @@ -8,7 +8,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -50,7 +50,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -92,7 +92,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -126,7 +126,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -160,7 +160,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -194,7 +194,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -228,7 +228,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -270,7 +270,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -335,7 +335,7 @@ QUERY id: 0 LIST id: 5, nodes: 1 COLUMN id: 6, column_name: date1, result_type: Date, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 7, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -377,7 +377,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -412,7 +412,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t PREWHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -452,7 +452,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -492,7 +492,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -529,7 +529,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -566,7 +566,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -608,7 +608,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -650,7 +650,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -692,7 +692,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -726,7 +726,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -760,7 +760,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -794,7 +794,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -828,7 +828,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date_t + TABLE id: 3, alias: __table1, table_name: default.date_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -878,7 +878,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.datetime_t + TABLE id: 3, alias: __table1, table_name: default.datetime_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -920,7 +920,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.datetime_t + TABLE id: 3, alias: __table1, table_name: default.datetime_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -962,7 +962,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date32_t + TABLE id: 3, alias: __table1, table_name: default.date32_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -1004,7 +1004,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.date32_t + TABLE id: 3, alias: __table1, table_name: default.date32_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -1046,7 +1046,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.datetime64_t + TABLE id: 3, alias: __table1, table_name: default.datetime64_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -1088,7 +1088,7 @@ QUERY id: 0 LIST id: 1, nodes: 1 COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 JOIN TREE - TABLE id: 3, table_name: default.datetime64_t + TABLE id: 3, alias: __table1, table_name: default.datetime64_t WHERE FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS diff --git a/tests/queries/0_stateless/02814_ReplacingMergeTree_fix_select_final_on_single_partition.reference b/tests/queries/0_stateless/02814_ReplacingMergeTree_fix_select_final_on_single_partition.reference new file mode 100644 index 00000000000..d19222b55ec --- /dev/null +++ b/tests/queries/0_stateless/02814_ReplacingMergeTree_fix_select_final_on_single_partition.reference @@ -0,0 +1,31 @@ +--- Based on https://github.com/ClickHouse/ClickHouse/issues/49685 +--- Verify that ReplacingMergeTree properly handles _is_deleted: +--- SELECT FINAL should take `_is_deleted` into consideration when there is only one partition. +-- { echoOn } + +DROP TABLE IF EXISTS t; +CREATE TABLE t +( + `account_id` UInt64, + `_is_deleted` UInt8, + `_version` UInt64 +) +ENGINE = ReplacingMergeTree(_version, _is_deleted) +ORDER BY (account_id); +INSERT INTO t SELECT number, 0, 1 FROM numbers(1e3); +-- Mark the first 100 rows as deleted. +INSERT INTO t SELECT number, 1, 1 FROM numbers(1e2); +-- Put everything in one partition +OPTIMIZE TABLE t FINAL; +SELECT count() FROM t; +1000 +SELECT count() FROM t FINAL; +900 +-- Both should produce the same number of rows. +-- Previously, `do_not_merge_across_partitions_select_final = 1` showed more rows, +-- as if no rows were deleted. +SELECT count() FROM t FINAL SETTINGS do_not_merge_across_partitions_select_final = 1; +900 +SELECT count() FROM t FINAL SETTINGS do_not_merge_across_partitions_select_final = 0; +900 +DROP TABLE t; diff --git a/tests/queries/0_stateless/02814_ReplacingMergeTree_fix_select_final_on_single_partition.sql b/tests/queries/0_stateless/02814_ReplacingMergeTree_fix_select_final_on_single_partition.sql new file mode 100644 index 00000000000..a89a1ff590a --- /dev/null +++ b/tests/queries/0_stateless/02814_ReplacingMergeTree_fix_select_final_on_single_partition.sql @@ -0,0 +1,32 @@ +--- Based on https://github.com/ClickHouse/ClickHouse/issues/49685 +--- Verify that ReplacingMergeTree properly handles _is_deleted: +--- SELECT FINAL should take `_is_deleted` into consideration when there is only one partition. +-- { echoOn } + +DROP TABLE IF EXISTS t; +CREATE TABLE t +( + `account_id` UInt64, + `_is_deleted` UInt8, + `_version` UInt64 +) +ENGINE = ReplacingMergeTree(_version, _is_deleted) +ORDER BY (account_id); + +INSERT INTO t SELECT number, 0, 1 FROM numbers(1e3); +-- Mark the first 100 rows as deleted. +INSERT INTO t SELECT number, 1, 1 FROM numbers(1e2); + +-- Put everything in one partition +OPTIMIZE TABLE t FINAL; + +SELECT count() FROM t; +SELECT count() FROM t FINAL; + +-- Both should produce the same number of rows. +-- Previously, `do_not_merge_across_partitions_select_final = 1` showed more rows, +-- as if no rows were deleted. +SELECT count() FROM t FINAL SETTINGS do_not_merge_across_partitions_select_final = 1; +SELECT count() FROM t FINAL SETTINGS do_not_merge_across_partitions_select_final = 0; + +DROP TABLE t; diff --git a/tests/queries/0_stateless/02817_structure_to_schema.reference b/tests/queries/0_stateless/02817_structure_to_schema.reference index 1f39a8ed50e..9fe41d838e7 100644 --- a/tests/queries/0_stateless/02817_structure_to_schema.reference +++ b/tests/queries/0_stateless/02817_structure_to_schema.reference @@ -189,7 +189,7 @@ struct Message } entries @0 : List(Entry); } - e1 @0 : List(E1); + e1 @0 : E1; struct E2 { struct Entry @@ -230,9 +230,9 @@ struct Message } entries @0 : List(Entry); } - e2 @1 : List(E2); + e2 @1 : E2; } - c1 @0 : C1; + c1 @0 : List(C1); } Read/write with no schema 0 @@ -400,49 +400,41 @@ message Message { message C1 { - message E1 + message E1Value { message E1Value { - message E1Value - { - repeated uint32 e1Value = 1; - } - repeated E1Value e1Value = 1; + repeated uint32 e1Value = 1; } - map e1 = 1; + repeated E1Value e1Value = 1; } - repeated E1 e1 = 1; - message E2 + map e1 = 1; + message E2Value { - message E2Value + message E1 { - message E1 - { - repeated bytes e1 = 1; - } - repeated E1 e1 = 1; + repeated bytes e1 = 1; + } + repeated E1 e1 = 1; + message E2 + { + uint32 e1 = 1; message E2 { - uint32 e1 = 1; - message E2 + message E1 { - message E1 - { - repeated bytes e1 = 1; - } - repeated E1 e1 = 1; - uint32 e2 = 2; + repeated bytes e1 = 1; } - E2 e2 = 2; + repeated E1 e1 = 1; + uint32 e2 = 2; } - repeated E2 e2 = 2; + E2 e2 = 2; } - map e2 = 1; + repeated E2 e2 = 2; } - repeated E2 e2 = 2; + map e2 = 2; } - C1 c1 = 1; + repeated C1 c1 = 1; } Read/write with no schema 0 diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference index 0cc2e802682..06f4a9cfc99 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.reference +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -3,31 +3,31 @@ Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String -Actions: INPUT : 0 -> id_0 UInt64 : 0 - INPUT : 1 -> value_1_1 String : 1 - INPUT : 2 -> value_1_3 String : 2 - INPUT : 3 -> id_2 UInt64 : 3 - ALIAS id_0 :: 0 -> id UInt64 : 4 - ALIAS value_1_1 :: 1 -> value_1 String : 0 - ALIAS value_1_3 :: 2 -> rhs.value_1 String : 1 - ALIAS id_2 :: 3 -> rhs.id UInt64 : 2 +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value_1 String : 1 + INPUT : 2 -> __table2.value_1 String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 + ALIAS __table1.value_1 :: 1 -> value_1 String : 0 + ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 Positions: 4 0 2 1 Join (JOIN FillRightFirst) - Header: id_0 UInt64 - value_1_1 String - value_1_3 String - id_2 UInt64 + Header: __table1.id UInt64 + __table1.value_1 String + __table2.value_1 String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin - Clauses: [(id_0) = (id_2)] + Clauses: [(__table1.id) = (__table2.id)] Expression ((JOIN actions + Change column names to column identifiers)) - Header: id_0 UInt64 - value_1_1 String + Header: __table1.id UInt64 + __table1.value_1 String Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value_1 String : 1 - ALIAS id :: 0 -> id_0 UInt64 : 2 - ALIAS value_1 :: 1 -> value_1_1 String : 0 + ALIAS id :: 0 -> __table1.id UInt64 : 2 + ALIAS value_1 :: 1 -> __table1.value_1 String : 0 Positions: 2 0 ReadFromMergeTree (default.test_table_1) Header: id UInt64 @@ -36,12 +36,12 @@ Positions: 4 0 2 1 Parts: 1 Granules: 1 Expression ((JOIN actions + Change column names to column identifiers)) - Header: id_2 UInt64 - value_1_3 String + Header: __table2.id UInt64 + __table2.value_1 String Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value_1 String : 1 - ALIAS id :: 0 -> id_2 UInt64 : 2 - ALIAS value_1 :: 1 -> value_1_3 String : 0 + ALIAS id :: 0 -> __table2.id UInt64 : 2 + ALIAS value_1 :: 1 -> __table2.value_1 String : 0 Positions: 2 0 ReadFromMergeTree (default.test_table_2) Header: id UInt64 @@ -55,39 +55,39 @@ Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String -Actions: INPUT : 0 -> id_0 UInt64 : 0 - INPUT : 1 -> value_1_1 String : 1 - INPUT :: 2 -> value_2_4 UInt64 : 2 - INPUT : 3 -> value_1_3 String : 3 - INPUT :: 4 -> value_2_5 UInt64 : 4 - INPUT : 5 -> id_2 UInt64 : 5 - ALIAS id_0 :: 0 -> id UInt64 : 6 - ALIAS value_1_1 :: 1 -> value_1 String : 0 - ALIAS value_1_3 :: 3 -> rhs.value_1 String : 1 - ALIAS id_2 :: 5 -> rhs.id UInt64 : 3 +Actions: INPUT : 0 -> __table1.id UInt64 : 0 + INPUT : 1 -> __table1.value_1 String : 1 + INPUT :: 2 -> __table1.value_2 UInt64 : 2 + INPUT : 3 -> __table2.value_1 String : 3 + INPUT :: 4 -> __table2.value_2 UInt64 : 4 + INPUT : 5 -> __table2.id UInt64 : 5 + ALIAS __table1.id :: 0 -> id UInt64 : 6 + ALIAS __table1.value_1 :: 1 -> value_1 String : 0 + ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 1 + ALIAS __table2.id :: 5 -> rhs.id UInt64 : 3 Positions: 6 0 3 1 Join (JOIN FillRightFirst) - Header: id_0 UInt64 - value_1_1 String - value_2_4 UInt64 - value_1_3 String - value_2_5 UInt64 - id_2 UInt64 + Header: __table1.id UInt64 + __table1.value_1 String + __table1.value_2 UInt64 + __table2.value_1 String + __table2.value_2 UInt64 + __table2.id UInt64 Type: INNER Strictness: ASOF Algorithm: HashJoin ASOF inequality: LESS - Clauses: [(id_0, value_2_4) = (id_2, value_2_5)] + Clauses: [(__table1.id, __table1.value_2) = (__table2.id, __table2.value_2)] Expression ((JOIN actions + Change column names to column identifiers)) - Header: id_0 UInt64 - value_1_1 String - value_2_4 UInt64 + Header: __table1.id UInt64 + __table1.value_1 String + __table1.value_2 UInt64 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value_1 String : 1 INPUT : 2 -> value_2 UInt64 : 2 - ALIAS id :: 0 -> id_0 UInt64 : 3 - ALIAS value_1 :: 1 -> value_1_1 String : 0 - ALIAS value_2 :: 2 -> value_2_4 UInt64 : 1 + ALIAS id :: 0 -> __table1.id UInt64 : 3 + ALIAS value_1 :: 1 -> __table1.value_1 String : 0 + ALIAS value_2 :: 2 -> __table1.value_2 UInt64 : 1 Positions: 3 0 1 ReadFromMergeTree (default.test_table_1) Header: id UInt64 @@ -97,15 +97,15 @@ Positions: 6 0 3 1 Parts: 1 Granules: 1 Expression ((JOIN actions + Change column names to column identifiers)) - Header: id_2 UInt64 - value_1_3 String - value_2_5 UInt64 + Header: __table2.id UInt64 + __table2.value_1 String + __table2.value_2 UInt64 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value_1 String : 1 INPUT : 2 -> value_2 UInt64 : 2 - ALIAS id :: 0 -> id_2 UInt64 : 3 - ALIAS value_1 :: 1 -> value_1_3 String : 0 - ALIAS value_2 :: 2 -> value_2_5 UInt64 : 1 + ALIAS id :: 0 -> __table2.id UInt64 : 3 + ALIAS value_1 :: 1 -> __table2.value_1 String : 0 + ALIAS value_2 :: 2 -> __table2.value_2 UInt64 : 1 Positions: 3 0 1 ReadFromMergeTree (default.test_table_2) Header: id UInt64 diff --git a/tests/queries/0_stateless/02861_replacing_merge_tree_with_cleanup.reference b/tests/queries/0_stateless/02861_replacing_merge_tree_with_cleanup.reference new file mode 100644 index 00000000000..9c9caa22139 --- /dev/null +++ b/tests/queries/0_stateless/02861_replacing_merge_tree_with_cleanup.reference @@ -0,0 +1,13 @@ +== Only last version remains after OPTIMIZE W/ CLEANUP == +d1 5 0 +d2 1 0 +d3 1 0 +d4 1 0 +d5 1 0 +d6 3 0 +== OPTIMIZE W/ CLEANUP (remove d6) == +d1 5 0 +d2 1 0 +d3 1 0 +d4 1 0 +d5 1 0 diff --git a/tests/queries/0_stateless/02861_replacing_merge_tree_with_cleanup.sql b/tests/queries/0_stateless/02861_replacing_merge_tree_with_cleanup.sql new file mode 100644 index 00000000000..4cd44a131e3 --- /dev/null +++ b/tests/queries/0_stateless/02861_replacing_merge_tree_with_cleanup.sql @@ -0,0 +1,24 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version, is_deleted) Order by (uid) SETTINGS vertical_merge_algorithm_min_rows_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 0, + min_rows_for_wide_part = 1, + min_bytes_for_wide_part = 1, + allow_experimental_replacing_merge_with_cleanup=1; + +-- Expect d6 to be version=3 is_deleted=false +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 3, 0); +-- Insert previous version of 'd6' but only v=3 is_deleted=false will remain +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 2, 1); +SELECT '== Only last version remains after OPTIMIZE W/ CLEANUP =='; +OPTIMIZE TABLE test FINAL CLEANUP; +select * from test order by uid; + +-- insert d6 v=3 is_deleted=true (timestamp more recent so this version should be the one take into acount) +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 3, 1); + +SELECT '== OPTIMIZE W/ CLEANUP (remove d6) =='; +OPTIMIZE TABLE test FINAL CLEANUP; +-- No d6 anymore +select * from test order by uid; + +DROP TABLE IF EXISTS test; diff --git a/tests/queries/0_stateless/02862_index_inverted_incorrect_args.sql b/tests/queries/0_stateless/02862_index_inverted_incorrect_args.sql index 0678023f2f4..7ba122a7155 100644 --- a/tests/queries/0_stateless/02862_index_inverted_incorrect_args.sql +++ b/tests/queries/0_stateless/02862_index_inverted_incorrect_args.sql @@ -3,7 +3,7 @@ DROP TABLE IF EXISTS tab; SET allow_experimental_inverted_index=1; CREATE TABLE tab (`k` UInt64, `s` Map(String, String), INDEX af mapKeys(s) TYPE inverted(2) GRANULARITY 1) ENGINE = MergeTree ORDER BY k SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; INSERT INTO tab (k) VALUES (0); -SELECT * FROM tab PREWHERE (s[NULL]) = 'Click a03' SETTINGS allow_experimental_analyzer=1; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } +SELECT * FROM tab PREWHERE (s[NULL]) = 'Click a03' SETTINGS allow_experimental_analyzer=1; SELECT * FROM tab PREWHERE (s[1]) = 'Click a03' SETTINGS allow_experimental_analyzer=1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT * FROM tab PREWHERE (s['foo']) = 'Click a03' SETTINGS allow_experimental_analyzer=1; DROP TABLE tab; diff --git a/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference b/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference index a2c441fa460..c2075f72f33 100644 --- a/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference +++ b/tests/queries/0_stateless/02868_distinct_to_count_optimization.reference @@ -15,14 +15,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, is_subquery: 1, is_distinct: 1 + QUERY id: 3, alias: __table1, is_subquery: 1, is_distinct: 1 PROJECTION COLUMNS a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count SETTINGS allow_experimental_analyzer=1 2. test distinct with subquery alias 3 @@ -41,14 +41,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, alias: t, is_subquery: 1, is_distinct: 1 + QUERY id: 3, alias: __table1, is_subquery: 1, is_distinct: 1 PROJECTION COLUMNS a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count SETTINGS allow_experimental_analyzer=1 3. test distinct with compound column name 3 @@ -67,14 +67,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, alias: t, is_subquery: 1, is_distinct: 1 + QUERY id: 3, alias: __table1, is_subquery: 1, is_distinct: 1 PROJECTION COLUMNS a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count SETTINGS allow_experimental_analyzer=1 4. test distinct with select expression alias 3 @@ -93,14 +93,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, alias: t, is_subquery: 1, is_distinct: 1 + QUERY id: 3, alias: __table1, is_subquery: 1, is_distinct: 1 PROJECTION COLUMNS alias_of_a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count SETTINGS allow_experimental_analyzer=1 5. test simple group by 3 @@ -122,14 +122,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count GROUP BY LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 @@ -154,14 +154,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, alias: t, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count GROUP BY LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 @@ -186,14 +186,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, alias: t, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS alias_of_a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count GROUP BY LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 @@ -218,14 +218,14 @@ QUERY id: 0 LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 JOIN TREE - QUERY id: 3, alias: t, is_subquery: 1 + QUERY id: 3, alias: __table1, is_subquery: 1 PROJECTION COLUMNS alias_of_a UInt8 PROJECTION LIST id: 4, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 JOIN TREE - TABLE id: 6, table_name: default.test_rewrite_uniq_to_count + TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count GROUP BY LIST id: 7, nodes: 1 COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6 diff --git a/tests/queries/0_stateless/02874_infer_objects_as_named_tuples.reference b/tests/queries/0_stateless/02874_infer_objects_as_named_tuples.reference index 01ef288d81a..06c152a0a3c 100644 --- a/tests/queries/0_stateless/02874_infer_objects_as_named_tuples.reference +++ b/tests/queries/0_stateless/02874_infer_objects_as_named_tuples.reference @@ -1,34 +1,34 @@ -obj Tuple(a Nullable(Int64), b Nullable(String), c Array(Nullable(Int64))) +obj Tuple(\n a Nullable(Int64),\n b Nullable(String),\n c Array(Nullable(Int64))) (42,'Hello',[1,2,3]) -obj Tuple(a Nullable(Int64), b Nullable(String), c Array(Nullable(Int64)), d Nullable(Date)) +obj Tuple(\n a Nullable(Int64),\n b Nullable(String),\n c Array(Nullable(Int64)),\n d Nullable(Date)) (42,'Hello',[1,2,3],NULL) (43,'World',[],'2020-01-01') -obj Tuple(a Nullable(Int64), b Nullable(String), c Array(Nullable(Int64)), d Nullable(Date)) +obj Tuple(\n a Nullable(Int64),\n b Nullable(String),\n c Array(Nullable(Int64)),\n d Nullable(Date)) (42,'Hello',[1,2,3],NULL) (43,'World',[],'2020-01-01') (NULL,NULL,[],NULL) -obj Tuple(a Nullable(Int64), b Nullable(String), c Array(Nullable(Int64)), d Nullable(String)) +obj Tuple(\n a Nullable(Int64),\n b Nullable(String),\n c Array(Nullable(Int64)),\n d Nullable(String)) (42,'Hello',[1,2,3],NULL) (43,'World',[],'2020-01-01') (NULL,NULL,[],NULL) (NULL,'2020-01-01',[],'Hello') -obj Array(Tuple(a Nullable(Int64), b Nullable(String), c Array(Nullable(Int64)), d Nullable(Date))) +obj Array(Tuple(\n a Nullable(Int64),\n b Nullable(String),\n c Array(Nullable(Int64)),\n d Nullable(Date))) [(42,'Hello',[1,2,3],NULL),(43,'World',[],'2020-01-01')] [(NULL,NULL,[],NULL)] -obj Tuple(nested_obj Tuple(a Nullable(Int64), b Nullable(String), c Array(Nullable(Int64)), d Nullable(Date))) +obj Tuple(\n nested_obj Tuple(\n a Nullable(Int64),\n b Nullable(String),\n c Array(Nullable(Int64)),\n d Nullable(Date))) ((42,'Hello',[1,2,3],NULL)) ((43,'World',[],'2020-01-01')) ((NULL,NULL,[],NULL)) -obj Tuple(a Tuple(b Nullable(Int64)), `a.b` Nullable(Int64), `a.b.c` Nullable(String)) +obj Tuple(\n a Tuple(\n b Nullable(Int64)),\n `a.b` Nullable(Int64),\n `a.b.c` Nullable(String)) ((1),NULL,NULL) ((NULL),2,'Hello') -obj Tuple(a Tuple(b Tuple(c Nullable(Int64)))) +obj Tuple(\n a Tuple(\n b Tuple(\n c Nullable(Int64)))) (((NULL))) (((10))) -obj Tuple(a Nullable(String)) +obj Tuple(\n a Nullable(String)) ('{}') obj Nullable(String) {} -obj Tuple(a Array(Tuple(b Array(Nullable(Int64)), c Tuple(d Nullable(Int64)), e Nullable(String)))) +obj Tuple(\n a Array(Tuple(\n b Array(Nullable(Int64)),\n c Tuple(\n d Nullable(Int64)),\n e Nullable(String)))) ([([],(NULL),NULL),([],(NULL),NULL),([],(10),NULL)]) ([([1,2,3],(NULL),'Hello')]) diff --git a/tests/queries/0_stateless/02875_final_invalid_read_ranges_bug.reference b/tests/queries/0_stateless/02875_final_invalid_read_ranges_bug.reference index 573541ac970..10fcc44daed 100644 --- a/tests/queries/0_stateless/02875_final_invalid_read_ranges_bug.reference +++ b/tests/queries/0_stateless/02875_final_invalid_read_ranges_bug.reference @@ -1 +1,28 @@ +5879429 2023-07-01 03:50:35 2023-07-01 03:50:35 -278 +5881397 2023-07-01 06:22:26 2023-07-01 06:22:27 2807 +5925060 2023-07-04 00:24:03 2023-07-04 00:24:02 -12 +5936591 2023-07-04 07:37:19 2023-07-04 07:37:18 -12 +5940709 2023-07-04 09:13:35 2023-07-04 09:13:35 2820 +5942342 2023-07-04 09:58:00 2023-07-04 09:57:59 -12 +5952231 2023-07-04 22:33:24 2023-07-04 22:33:24 1692 +5959449 2023-07-05 04:32:55 2023-07-05 04:32:54 -12 +5963240 2023-07-05 06:37:08 2023-07-05 06:37:09 1709 +5965742 2023-07-05 07:27:01 2023-07-05 07:27:02 1709 +5969948 2023-07-05 08:44:36 2023-07-05 08:44:37 2278 +5971673 2023-07-05 09:14:09 2023-07-05 09:14:09 5695 +6012987 2023-07-06 20:52:28 2023-07-06 20:52:27 -536 +0 +5879429 2023-07-01 03:50:35 2023-07-01 03:50:35 -278 +5881397 2023-07-01 06:22:26 2023-07-01 06:22:27 2807 +5925060 2023-07-04 00:24:03 2023-07-04 00:24:02 -12 +5936591 2023-07-04 07:37:19 2023-07-04 07:37:18 -12 +5940709 2023-07-04 09:13:35 2023-07-04 09:13:35 2820 +5942342 2023-07-04 09:58:00 2023-07-04 09:57:59 -12 +5952231 2023-07-04 22:33:24 2023-07-04 22:33:24 1692 +5959449 2023-07-05 04:32:55 2023-07-05 04:32:54 -12 +5963240 2023-07-05 06:37:08 2023-07-05 06:37:09 1709 +5965742 2023-07-05 07:27:01 2023-07-05 07:27:02 1709 +5969948 2023-07-05 08:44:36 2023-07-05 08:44:37 2278 +5971673 2023-07-05 09:14:09 2023-07-05 09:14:09 5695 +6012987 2023-07-06 20:52:28 2023-07-06 20:52:27 -536 0 diff --git a/tests/queries/0_stateless/02875_final_invalid_read_ranges_bug.sql b/tests/queries/0_stateless/02875_final_invalid_read_ranges_bug.sql index 4e91c2e3167..5557c572696 100644 --- a/tests/queries/0_stateless/02875_final_invalid_read_ranges_bug.sql +++ b/tests/queries/0_stateless/02875_final_invalid_read_ranges_bug.sql @@ -1,3 +1,4 @@ +DROP TABLE IF EXISTS t; CREATE TABLE t ( tid UInt64, @@ -13,8 +14,14 @@ SETTINGS index_granularity = 1; INSERT INTO t VALUES (5879429,'2023-07-01 03:50:35','2023-07-01 03:50:35',-278) (5881397,'2023-07-01 06:22:26','2023-07-01 06:22:27',2807) (5925060,'2023-07-04 00:24:03','2023-07-04 00:24:02',-12) (5936591,'2023-07-04 07:37:19','2023-07-04 07:37:18',-12) (5940709,'2023-07-04 09:13:35','2023-07-04 09:13:35',2820) (5942342,'2023-07-04 09:58:00','2023-07-04 09:57:59',-12) (5952231,'2023-07-04 22:33:24','2023-07-04 22:33:24',1692) (5959449,'2023-07-05 04:32:55','2023-07-05 04:32:54',-12) (5963240,'2023-07-05 06:37:08','2023-07-05 06:37:09',1709) (5965742,'2023-07-05 07:27:01','2023-07-05 07:27:02',1709) (5969948,'2023-07-05 08:44:36','2023-07-05 08:44:37',2278) (5971673,'2023-07-05 09:14:09','2023-07-05 09:14:09',5695) (6012987,'2023-07-06 20:52:28','2023-07-06 20:52:27',-536); -SELECT sum(amount) -FROM t FINAL -WHERE (processed_at >= '2023-09-19 00:00:00') AND (processed_at <= '2023-09-20 01:00:00'); +SELECT tid, processed_at, created_at, amount FROM t FINAL ORDER BY tid; + +SELECT sum(amount) FROM t FINAL WHERE (processed_at >= '2023-09-19 00:00:00') AND (processed_at <= '2023-09-20 01:00:00'); + +INSERT INTO t VALUES (5879429,'2023-07-01 03:50:35','2023-07-01 03:50:35',-278) (5881397,'2023-07-01 06:22:26','2023-07-01 06:22:27',2807) (5925060,'2023-07-04 00:24:03','2023-07-04 00:24:02',-12) (5936591,'2023-07-04 07:37:19','2023-07-04 07:37:18',-12) (5940709,'2023-07-04 09:13:35','2023-07-04 09:13:35',2820) (5942342,'2023-07-04 09:58:00','2023-07-04 09:57:59',-12) (5952231,'2023-07-04 22:33:24','2023-07-04 22:33:24',1692) (5959449,'2023-07-05 04:32:55','2023-07-05 04:32:54',-12) (5963240,'2023-07-05 06:37:08','2023-07-05 06:37:09',1709) (5965742,'2023-07-05 07:27:01','2023-07-05 07:27:02',1709) (5969948,'2023-07-05 08:44:36','2023-07-05 08:44:37',2278) (5971673,'2023-07-05 09:14:09','2023-07-05 09:14:09',5695) (6012987,'2023-07-06 20:52:28','2023-07-06 20:52:27',-536); + +SELECT tid, processed_at, created_at, amount FROM t FINAL ORDER BY tid; + +SELECT sum(amount) FROM t FINAL WHERE (processed_at >= '2023-09-19 00:00:00') AND (processed_at <= '2023-09-20 01:00:00'); DROP TABLE t; diff --git a/tests/queries/0_stateless/02876_json_incomplete_types_as_strings_inference.reference b/tests/queries/0_stateless/02876_json_incomplete_types_as_strings_inference.reference index db94ffc9466..b904568391b 100644 --- a/tests/queries/0_stateless/02876_json_incomplete_types_as_strings_inference.reference +++ b/tests/queries/0_stateless/02876_json_incomplete_types_as_strings_inference.reference @@ -2,6 +2,6 @@ a Nullable(String) b Nullable(String) c Array(Nullable(String)) \N {} [] -a Tuple(b Nullable(String), c Array(Array(Nullable(String)))) -d Tuple(e Array(Nullable(String)), f Nullable(String)) +a Tuple(\n b Nullable(String),\n c Array(Array(Nullable(String)))) +d Tuple(\n e Array(Nullable(String)),\n f Nullable(String)) (NULL,[[],[]]) (['{}','{}'],NULL) diff --git a/tests/queries/0_stateless/02889_file_log_save_errors.reference b/tests/queries/0_stateless/02889_file_log_save_errors.reference index c4a7c1f0bda..849da6ad6fa 100644 --- a/tests/queries/0_stateless/02889_file_log_save_errors.reference +++ b/tests/queries/0_stateless/02889_file_log_save_errors.reference @@ -1,20 +1,20 @@ -Cannot parse input: expected \'{\' before: \'Error 0\' Error 0 a.jsonl -Cannot parse input: expected \'{\' before: \'Error 1\' Error 1 a.jsonl -Cannot parse input: expected \'{\' before: \'Error 2\' Error 2 a.jsonl -Cannot parse input: expected \'{\' before: \'Error 3\' Error 3 a.jsonl -Cannot parse input: expected \'{\' before: \'Error 4\' Error 4 a.jsonl -Cannot parse input: expected \'{\' before: \'Error 5\' Error 5 a.jsonl -Cannot parse input: expected \'{\' before: \'Error 6\' Error 6 a.jsonl -Cannot parse input: expected \'{\' before: \'Error 7\' Error 7 a.jsonl -Cannot parse input: expected \'{\' before: \'Error 8\' Error 8 a.jsonl -Cannot parse input: expected \'{\' before: \'Error 9\' Error 9 a.jsonl -Cannot parse input: expected \'{\' before: \'Error 10\' Error 10 b.jsonl -Cannot parse input: expected \'{\' before: \'Error 11\' Error 11 b.jsonl -Cannot parse input: expected \'{\' before: \'Error 12\' Error 12 b.jsonl -Cannot parse input: expected \'{\' before: \'Error 13\' Error 13 b.jsonl -Cannot parse input: expected \'{\' before: \'Error 14\' Error 14 b.jsonl -Cannot parse input: expected \'{\' before: \'Error 15\' Error 15 b.jsonl -Cannot parse input: expected \'{\' before: \'Error 16\' Error 16 b.jsonl -Cannot parse input: expected \'{\' before: \'Error 17\' Error 17 b.jsonl -Cannot parse input: expected \'{\' before: \'Error 18\' Error 18 b.jsonl -Cannot parse input: expected \'{\' before: \'Error 19\' Error 19 b.jsonl +Cannot parse input: expected \'{\' before: \'Error 0\': (at row 1)\n Error 0 a.jsonl +Cannot parse input: expected \'{\' before: \'Error 1\': (at row 1)\n Error 1 a.jsonl +Cannot parse input: expected \'{\' before: \'Error 2\': (at row 1)\n Error 2 a.jsonl +Cannot parse input: expected \'{\' before: \'Error 3\': (at row 1)\n Error 3 a.jsonl +Cannot parse input: expected \'{\' before: \'Error 4\': (at row 1)\n Error 4 a.jsonl +Cannot parse input: expected \'{\' before: \'Error 5\': (at row 1)\n Error 5 a.jsonl +Cannot parse input: expected \'{\' before: \'Error 6\': (at row 1)\n Error 6 a.jsonl +Cannot parse input: expected \'{\' before: \'Error 7\': (at row 1)\n Error 7 a.jsonl +Cannot parse input: expected \'{\' before: \'Error 8\': (at row 1)\n Error 8 a.jsonl +Cannot parse input: expected \'{\' before: \'Error 9\': (at row 1)\n Error 9 a.jsonl +Cannot parse input: expected \'{\' before: \'Error 10\': (at row 1)\n Error 10 b.jsonl +Cannot parse input: expected \'{\' before: \'Error 11\': (at row 1)\n Error 11 b.jsonl +Cannot parse input: expected \'{\' before: \'Error 12\': (at row 1)\n Error 12 b.jsonl +Cannot parse input: expected \'{\' before: \'Error 13\': (at row 1)\n Error 13 b.jsonl +Cannot parse input: expected \'{\' before: \'Error 14\': (at row 1)\n Error 14 b.jsonl +Cannot parse input: expected \'{\' before: \'Error 15\': (at row 1)\n Error 15 b.jsonl +Cannot parse input: expected \'{\' before: \'Error 16\': (at row 1)\n Error 16 b.jsonl +Cannot parse input: expected \'{\' before: \'Error 17\': (at row 1)\n Error 17 b.jsonl +Cannot parse input: expected \'{\' before: \'Error 18\': (at row 1)\n Error 18 b.jsonl +Cannot parse input: expected \'{\' before: \'Error 19\': (at row 1)\n Error 19 b.jsonl diff --git a/tests/queries/0_stateless/02889_print_pretty_type_names.reference b/tests/queries/0_stateless/02889_print_pretty_type_names.reference index ea25df165bb..9af8e0142f8 100644 --- a/tests/queries/0_stateless/02889_print_pretty_type_names.reference +++ b/tests/queries/0_stateless/02889_print_pretty_type_names.reference @@ -5,18 +5,11 @@ a Tuple( e Array(UInt32), f Array(Tuple( g String, - h Map( - String, - Array(Tuple( - i String, - j UInt64 - )) - ) - )), - k Date - ), - l Nullable(String) -) + h Map(String, Array(Tuple( + i String, + j UInt64))))), + k Date), + l Nullable(String)) Tuple( b String, c Tuple( @@ -24,15 +17,8 @@ Tuple( e Array(UInt32), f Array(Tuple( g String, - h Map( - String, - Array(Tuple( - i String, - j UInt64 - )) - ) - )), - k Date - ), - l Nullable(String) -) + h Map(String, Array(Tuple( + i String, + j UInt64))))), + k Date), + l Nullable(String)) diff --git a/tests/queries/0_stateless/02890_describe_table_options.reference b/tests/queries/0_stateless/02890_describe_table_options.reference index 2974fd92f3c..5d99df36bb4 100644 --- a/tests/queries/0_stateless/02890_describe_table_options.reference +++ b/tests/queries/0_stateless/02890_describe_table_options.reference @@ -2,205 +2,237 @@ SET describe_compact_output = 0, describe_include_virtual_columns = 0, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐ -│ id │ UInt64 │ │ │ index column │ │ │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ -└──────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ +┌─name─┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐ +│ id │ UInt64 │ │ │ index column │ │ │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ +└──────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name─┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐ -│ id │ UInt64 │ │ │ index column │ │ │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ -└──────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ +┌─name─┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐ +│ id │ UInt64 │ │ │ index column │ │ │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ +└──────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ SET describe_compact_output = 0, describe_include_virtual_columns = 0, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ -└───────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ +┌─name──────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ +│ id │ UInt64 │ │ │ index column │ │ │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ +│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ +│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ +└───────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ -└───────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ +┌─name──────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ +│ id │ UInt64 │ │ │ index column │ │ │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ +│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ +│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ +└───────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─────────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _part_index │ UInt64 │ │ │ │ │ │ 1 │ -│ _part_uuid │ UUID │ │ │ │ │ │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _partition_value │ UInt8 │ │ │ │ │ │ 1 │ -│ _sample_factor │ Float64 │ │ │ │ │ │ 1 │ -│ _part_offset │ UInt64 │ │ │ │ │ │ 1 │ -│ _row_exists │ UInt8 │ │ │ │ │ │ 1 │ -│ _block_number │ UInt64 │ │ │ │ │ │ 1 │ -└──────────────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴────────────┘ +┌─name─────────────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐ +│ id │ UInt64 │ │ │ index column │ │ │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +│ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │ +│ _part_index │ UInt64 │ │ │ │ │ │ 1 │ +│ _part_uuid │ UUID │ │ │ │ │ │ 1 │ +│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 1 │ +│ _partition_value │ UInt8 │ │ │ │ │ │ 1 │ +│ _sample_factor │ Float64 │ │ │ │ │ │ 1 │ +│ _part_offset │ UInt64 │ │ │ │ │ │ 1 │ +│ _row_exists │ UInt8 │ │ │ │ │ │ 1 │ +│ _block_number │ UInt64 │ │ │ │ │ │ 1 │ +└──────────────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ _table │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _part_index │ UInt64 │ │ │ │ │ │ 1 │ -│ _part_uuid │ UUID │ │ │ │ │ │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 1 │ -│ _sample_factor │ Float64 │ │ │ │ │ │ 1 │ -│ _part_offset │ UInt64 │ │ │ │ │ │ 1 │ -│ _row_exists │ UInt8 │ │ │ │ │ │ 1 │ -│ _block_number │ UInt64 │ │ │ │ │ │ 1 │ -│ _shard_num │ UInt32 │ │ │ │ │ │ 1 │ -└────────────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴────────────┘ +┌─name───────────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐ +│ id │ UInt64 │ │ │ index column │ │ │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +│ _table │ LowCardinality(String) │ │ │ │ │ │ 1 │ +│ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │ +│ _part_index │ UInt64 │ │ │ │ │ │ 1 │ +│ _part_uuid │ UUID │ │ │ │ │ │ 1 │ +│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 1 │ +│ _sample_factor │ Float64 │ │ │ │ │ │ 1 │ +│ _part_offset │ UInt64 │ │ │ │ │ │ 1 │ +│ _row_exists │ UInt8 │ │ │ │ │ │ 1 │ +│ _block_number │ UInt64 │ │ │ │ │ │ 1 │ +│ _shard_num │ UInt32 │ │ │ │ │ │ 1 │ +└────────────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴────────────┘ SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─────────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ -│ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _partition_value │ UInt8 │ │ │ │ │ │ 0 │ 1 │ -│ _sample_factor │ Float64 │ │ │ │ │ │ 0 │ 1 │ -│ _part_offset │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _row_exists │ UInt8 │ │ │ │ │ │ 0 │ 1 │ -│ _block_number │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -└──────────────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ +┌─name─────────────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐ +│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ +│ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ +│ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │ +│ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │ +│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ +│ _partition_value │ UInt8 │ │ │ │ │ │ 0 │ 1 │ +│ _sample_factor │ Float64 │ │ │ │ │ │ 0 │ 1 │ +│ _part_offset │ UInt64 │ │ │ │ │ │ 0 │ 1 │ +│ _row_exists │ UInt8 │ │ │ │ │ │ 0 │ 1 │ +│ _block_number │ UInt64 │ │ │ │ │ │ 0 │ 1 │ +│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ +│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +└──────────────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ -│ _table │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ -│ _sample_factor │ Float64 │ │ │ │ │ │ 0 │ 1 │ -│ _part_offset │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _row_exists │ UInt8 │ │ │ │ │ │ 0 │ 1 │ -│ _block_number │ UInt64 │ │ │ │ │ │ 0 │ 1 │ -│ _shard_num │ UInt32 │ │ │ │ │ │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -└────────────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ +┌─name───────────┬─type─────────────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐ +│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ +│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ +│ _table │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ +│ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ +│ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │ +│ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │ +│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │ +│ _sample_factor │ Float64 │ │ │ │ │ │ 0 │ 1 │ +│ _part_offset │ UInt64 │ │ │ │ │ │ 0 │ 1 │ +│ _row_exists │ UInt8 │ │ │ │ │ │ 0 │ 1 │ +│ _block_number │ UInt64 │ │ │ │ │ │ 0 │ 1 │ +│ _shard_num │ UInt32 │ │ │ │ │ │ 0 │ 1 │ +│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ +│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +└────────────────┴──────────────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 0, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─┬─type──────────────────────┐ -│ id │ UInt64 │ -│ arr │ Array(UInt64) │ -│ t │ Tuple(a String, b UInt64) │ -└──────┴───────────────────────────┘ +┌─name─┬─type─────────────────────────────┐ +│ id │ UInt64 │ +│ arr │ Array(UInt64) │ +│ t │ Tuple( + a String, + b UInt64) │ +└──────┴──────────────────────────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name─┬─type──────────────────────┐ -│ id │ UInt64 │ -│ arr │ Array(UInt64) │ -│ t │ Tuple(a String, b UInt64) │ -└──────┴───────────────────────────┘ +┌─name─┬─type─────────────────────────────┐ +│ id │ UInt64 │ +│ arr │ Array(UInt64) │ +│ t │ Tuple( + a String, + b UInt64) │ +└──────┴──────────────────────────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 0, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type──────────────────────┬─is_subcolumn─┐ -│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ -│ arr.size0 │ UInt64 │ 1 │ -│ t.a │ String │ 1 │ -│ t.b │ UInt64 │ 1 │ -└───────────┴───────────────────────────┴──────────────┘ +┌─name──────┬─type─────────────────────────────┬─is_subcolumn─┐ +│ id │ UInt64 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ 0 │ +│ arr.size0 │ UInt64 │ 1 │ +│ t.a │ String │ 1 │ +│ t.b │ UInt64 │ 1 │ +└───────────┴──────────────────────────────────┴──────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type──────────────────────┬─is_subcolumn─┐ -│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ -│ arr.size0 │ UInt64 │ 1 │ -│ t.a │ String │ 1 │ -│ t.b │ UInt64 │ 1 │ -└───────────┴───────────────────────────┴──────────────┘ +┌─name──────┬─type─────────────────────────────┬─is_subcolumn─┐ +│ id │ UInt64 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ 0 │ +│ arr.size0 │ UInt64 │ 1 │ +│ t.a │ String │ 1 │ +│ t.b │ UInt64 │ 1 │ +└───────────┴──────────────────────────────────┴──────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─────────────┬─type──────────────────────┬─is_virtual─┐ -│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ -│ _part │ LowCardinality(String) │ 1 │ -│ _part_index │ UInt64 │ 1 │ -│ _part_uuid │ UUID │ 1 │ -│ _partition_id │ LowCardinality(String) │ 1 │ -│ _partition_value │ UInt8 │ 1 │ -│ _sample_factor │ Float64 │ 1 │ -│ _part_offset │ UInt64 │ 1 │ -│ _row_exists │ UInt8 │ 1 │ -│ _block_number │ UInt64 │ 1 │ -└──────────────────┴───────────────────────────┴────────────┘ +┌─name─────────────┬─type─────────────────────────────┬─is_virtual─┐ +│ id │ UInt64 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ 0 │ +│ _part │ LowCardinality(String) │ 1 │ +│ _part_index │ UInt64 │ 1 │ +│ _part_uuid │ UUID │ 1 │ +│ _partition_id │ LowCardinality(String) │ 1 │ +│ _partition_value │ UInt8 │ 1 │ +│ _sample_factor │ Float64 │ 1 │ +│ _part_offset │ UInt64 │ 1 │ +│ _row_exists │ UInt8 │ 1 │ +│ _block_number │ UInt64 │ 1 │ +└──────────────────┴──────────────────────────────────┴────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─is_virtual─┐ -│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ -│ _table │ LowCardinality(String) │ 1 │ -│ _part │ LowCardinality(String) │ 1 │ -│ _part_index │ UInt64 │ 1 │ -│ _part_uuid │ UUID │ 1 │ -│ _partition_id │ LowCardinality(String) │ 1 │ -│ _sample_factor │ Float64 │ 1 │ -│ _part_offset │ UInt64 │ 1 │ -│ _row_exists │ UInt8 │ 1 │ -│ _block_number │ UInt64 │ 1 │ -│ _shard_num │ UInt32 │ 1 │ -└────────────────┴───────────────────────────┴────────────┘ +┌─name───────────┬─type─────────────────────────────┬─is_virtual─┐ +│ id │ UInt64 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ 0 │ +│ _table │ LowCardinality(String) │ 1 │ +│ _part │ LowCardinality(String) │ 1 │ +│ _part_index │ UInt64 │ 1 │ +│ _part_uuid │ UUID │ 1 │ +│ _partition_id │ LowCardinality(String) │ 1 │ +│ _sample_factor │ Float64 │ 1 │ +│ _part_offset │ UInt64 │ 1 │ +│ _row_exists │ UInt8 │ 1 │ +│ _block_number │ UInt64 │ 1 │ +│ _shard_num │ UInt32 │ 1 │ +└────────────────┴──────────────────────────────────┴────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─────────────┬─type──────────────────────┬─is_subcolumn─┬─is_virtual─┐ -│ id │ UInt64 │ 0 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ 0 │ -│ _part │ LowCardinality(String) │ 0 │ 1 │ -│ _part_index │ UInt64 │ 0 │ 1 │ -│ _part_uuid │ UUID │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ -│ _partition_value │ UInt8 │ 0 │ 1 │ -│ _sample_factor │ Float64 │ 0 │ 1 │ -│ _part_offset │ UInt64 │ 0 │ 1 │ -│ _row_exists │ UInt8 │ 0 │ 1 │ -│ _block_number │ UInt64 │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ 1 │ 0 │ -│ t.a │ String │ 1 │ 0 │ -│ t.b │ UInt64 │ 1 │ 0 │ -└──────────────────┴───────────────────────────┴──────────────┴────────────┘ +┌─name─────────────┬─type─────────────────────────────┬─is_subcolumn─┬─is_virtual─┐ +│ id │ UInt64 │ 0 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ 0 │ 0 │ +│ _part │ LowCardinality(String) │ 0 │ 1 │ +│ _part_index │ UInt64 │ 0 │ 1 │ +│ _part_uuid │ UUID │ 0 │ 1 │ +│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ +│ _partition_value │ UInt8 │ 0 │ 1 │ +│ _sample_factor │ Float64 │ 0 │ 1 │ +│ _part_offset │ UInt64 │ 0 │ 1 │ +│ _row_exists │ UInt8 │ 0 │ 1 │ +│ _block_number │ UInt64 │ 0 │ 1 │ +│ arr.size0 │ UInt64 │ 1 │ 0 │ +│ t.a │ String │ 1 │ 0 │ +│ t.b │ UInt64 │ 1 │ 0 │ +└──────────────────┴──────────────────────────────────┴──────────────┴────────────┘ DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─is_subcolumn─┬─is_virtual─┐ -│ id │ UInt64 │ 0 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ 0 │ -│ _table │ LowCardinality(String) │ 0 │ 1 │ -│ _part │ LowCardinality(String) │ 0 │ 1 │ -│ _part_index │ UInt64 │ 0 │ 1 │ -│ _part_uuid │ UUID │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ -│ _sample_factor │ Float64 │ 0 │ 1 │ -│ _part_offset │ UInt64 │ 0 │ 1 │ -│ _row_exists │ UInt8 │ 0 │ 1 │ -│ _block_number │ UInt64 │ 0 │ 1 │ -│ _shard_num │ UInt32 │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ 1 │ 0 │ -│ t.a │ String │ 1 │ 0 │ -│ t.b │ UInt64 │ 1 │ 0 │ -└────────────────┴───────────────────────────┴──────────────┴────────────┘ +┌─name───────────┬─type─────────────────────────────┬─is_subcolumn─┬─is_virtual─┐ +│ id │ UInt64 │ 0 │ 0 │ +│ arr │ Array(UInt64) │ 0 │ 0 │ +│ t │ Tuple( + a String, + b UInt64) │ 0 │ 0 │ +│ _table │ LowCardinality(String) │ 0 │ 1 │ +│ _part │ LowCardinality(String) │ 0 │ 1 │ +│ _part_index │ UInt64 │ 0 │ 1 │ +│ _part_uuid │ UUID │ 0 │ 1 │ +│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ +│ _sample_factor │ Float64 │ 0 │ 1 │ +│ _part_offset │ UInt64 │ 0 │ 1 │ +│ _row_exists │ UInt8 │ 0 │ 1 │ +│ _block_number │ UInt64 │ 0 │ 1 │ +│ _shard_num │ UInt32 │ 0 │ 1 │ +│ arr.size0 │ UInt64 │ 1 │ 0 │ +│ t.a │ String │ 1 │ 0 │ +│ t.b │ UInt64 │ 1 │ 0 │ +└────────────────┴──────────────────────────────────┴──────────────┴────────────┘ diff --git a/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.reference b/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.reference index 4a321380536..838bf18b937 100644 --- a/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.reference +++ b/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.reference @@ -5,6 +5,6 @@ SELECT 1; DROP DATABASE foo; SELECT 2; 2 -USE _local; +USE default; SELECT 3; 3 diff --git a/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.sh b/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.sh index 1af40f8778d..3250c70a268 100755 --- a/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.sh +++ b/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.sh @@ -10,6 +10,6 @@ ${CLICKHOUSE_LOCAL} --echo --multiquery " SELECT 1; DROP DATABASE foo; SELECT 2; - USE _local; + USE default; SELECT 3; " diff --git a/tests/queries/0_stateless/02900_union_schema_inference_mode.reference b/tests/queries/0_stateless/02900_union_schema_inference_mode.reference index 864cd780ddb..31172c41262 100644 --- a/tests/queries/0_stateless/02900_union_schema_inference_mode.reference +++ b/tests/queries/0_stateless/02900_union_schema_inference_mode.reference @@ -1,5 +1,5 @@ a Nullable(Int64) -obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) +obj Tuple(\n f1 Nullable(Int64),\n f2 Nullable(String),\n f3 Nullable(Int64)) b Nullable(Int64) c Nullable(String) {"a":"1","obj":{"f1":"1","f2":"2020-01-01","f3":null},"b":null,"c":null} @@ -10,11 +10,11 @@ UNION data2.jsonl b Nullable(Int64), obj Tuple(f2 Nullable(String), f3 Nullable( UNION data3.jsonl c Nullable(String) c Nullable(String) a Nullable(Int64) -obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) +obj Tuple(\n f1 Nullable(Int64),\n f2 Nullable(String),\n f3 Nullable(Int64)) b Nullable(Int64) c Nullable(String) a Nullable(Int64) -obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) +obj Tuple(\n f1 Nullable(Int64),\n f2 Nullable(String),\n f3 Nullable(Int64)) b Nullable(Int64) c Nullable(String) {"a":"1","obj":{"f1":"1","f2":"2020-01-01","f3":null},"b":null,"c":null} @@ -25,7 +25,7 @@ UNION archive.tar::data2.jsonl b Nullable(Int64), obj Tuple(f2 Nullable(String), UNION archive.tar::data3.jsonl c Nullable(String) c Nullable(String) a Nullable(Int64) -obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) +obj Tuple(\n f1 Nullable(Int64),\n f2 Nullable(String),\n f3 Nullable(Int64)) b Nullable(Int64) c Nullable(String) 1 diff --git a/tests/queries/0_stateless/02906_flatten_only_true_nested.reference b/tests/queries/0_stateless/02906_flatten_only_true_nested.reference new file mode 100644 index 00000000000..b259b1e4563 --- /dev/null +++ b/tests/queries/0_stateless/02906_flatten_only_true_nested.reference @@ -0,0 +1,3 @@ +data.x Array(UInt32) +data.y Array(UInt32) +data Array(Tuple(\n x UInt64,\n y UInt64)) diff --git a/tests/queries/0_stateless/02906_flatten_only_true_nested.sql b/tests/queries/0_stateless/02906_flatten_only_true_nested.sql new file mode 100644 index 00000000000..e930b46bd70 --- /dev/null +++ b/tests/queries/0_stateless/02906_flatten_only_true_nested.sql @@ -0,0 +1,9 @@ +set flatten_nested = 1; +drop table if exists test_nested; +create table test_nested (data Nested(x UInt32, y UInt32)) engine=Memory; +desc test_nested; +drop table test_nested; +drop table if exists test_array_tuple; +create table test_array_tuple (data Array(Tuple(x UInt64, y UInt64))) engine=Memory; +desc test_array_tuple; +drop table test_array_tuple; diff --git a/tests/queries/0_stateless/02906_orc_tuple_field_prune.reference b/tests/queries/0_stateless/02906_orc_tuple_field_prune.reference index dfdd38f5e8e..46738c95cd5 100644 --- a/tests/queries/0_stateless/02906_orc_tuple_field_prune.reference +++ b/tests/queries/0_stateless/02906_orc_tuple_field_prune.reference @@ -1,9 +1,9 @@ int64_column Nullable(Int64) string_column Nullable(String) float64_column Nullable(Float64) -tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)) -array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))) -map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))) +tuple_column Tuple(\n a Nullable(String),\n b Nullable(Float64),\n c Nullable(Int64)) +array_tuple_column Array(Tuple(\n a Nullable(String),\n b Nullable(Float64),\n c Nullable(Int64))) +map_tuple_column Map(String, Tuple(\n a Nullable(String),\n b Nullable(Float64),\n c Nullable(Int64))) -- { echoOn } -- Test primitive types select int64_column, string_column, float64_column from file('02906.orc') where int64_column % 15 = 0; diff --git a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.reference b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.reference index 64c8d3a0b68..90c5e0e99a5 100644 --- a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.reference +++ b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.reference @@ -1,2 +1,2 @@ -1048576 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/collection_sql 5 5000 0 1 -1048576 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/collection 5 5000 0 1 +1048576 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/collection_sql 5 5000 0 16 +1048576 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/collection 5 5000 0 16 diff --git a/tests/queries/0_stateless/02910_replicated_merge_parameters_must_consistent.sql b/tests/queries/0_stateless/02910_replicated_merge_parameters_must_consistent.sql index c832e16e81e..0f452105e6d 100644 --- a/tests/queries/0_stateless/02910_replicated_merge_parameters_must_consistent.sql +++ b/tests/queries/0_stateless/02910_replicated_merge_parameters_must_consistent.sql @@ -8,13 +8,52 @@ CREATE TABLE t ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t/', 'r1', legacy_ver) ORDER BY id; -CREATE TABLE t_r +CREATE TABLE t_r_ok +( + `id` UInt64, + `val` String, + `legacy_ver` UInt64, +) +ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t/', 'r2', legacy_ver) +ORDER BY id; + +CREATE TABLE t_r_error ( `id` UInt64, `val` String, `legacy_ver` UInt64 ) -ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t/', 'r2') +ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t/', 'r3') +ORDER BY id; -- { serverError METADATA_MISMATCH } + +CREATE TABLE t2 +( + `id` UInt64, + `val` String, + `legacy_ver` UInt64, + `deleted` UInt8 +) +ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t2/', 'r1', legacy_ver) +ORDER BY id; + +CREATE TABLE t2_r_ok +( + `id` UInt64, + `val` String, + `legacy_ver` UInt64, + `deleted` UInt8 +) +ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t2/', 'r2', legacy_ver) +ORDER BY id; + +CREATE TABLE t2_r_error +( + `id` UInt64, + `val` String, + `legacy_ver` UInt64, + `deleted` UInt8 +) +ENGINE = ReplicatedReplacingMergeTree('/tables/{database}/t2/', 'r3', legacy_ver, deleted) ORDER BY id; -- { serverError METADATA_MISMATCH } CREATE TABLE t3 @@ -26,13 +65,23 @@ CREATE TABLE t3 ENGINE = ReplicatedSummingMergeTree('/tables/{database}/t3/', 'r1', metrics1) ORDER BY key; -CREATE TABLE t3_r +CREATE TABLE t3_r_ok ( `key` UInt64, `metrics1` UInt64, `metrics2` UInt64 ) -ENGINE = ReplicatedSummingMergeTree('/tables/{database}/t3/', 'r2', metrics2) +ENGINE = ReplicatedSummingMergeTree('/tables/{database}/t3/', 'r2', metrics1) +ORDER BY key; + + +CREATE TABLE t3_r_error +( + `key` UInt64, + `metrics1` UInt64, + `metrics2` UInt64 +) +ENGINE = ReplicatedSummingMergeTree('/tables/{database}/t3/', 'r3', metrics2) ORDER BY key; -- { serverError METADATA_MISMATCH } CREATE TABLE t4 @@ -47,7 +96,7 @@ CREATE TABLE t4 ENGINE = ReplicatedGraphiteMergeTree('/tables/{database}/t4/', 'r1', 'graphite_rollup') ORDER BY key; -CREATE TABLE t4_r +CREATE TABLE t4_r_ok ( `key` UInt32, `Path` String, @@ -56,5 +105,30 @@ CREATE TABLE t4_r `Version` UInt32, `col` UInt64 ) -ENGINE = ReplicatedGraphiteMergeTree('/tables/{database}/t4/', 'r2', 'graphite_rollup_alternative') +ENGINE = ReplicatedGraphiteMergeTree('/tables/{database}/t4/', 'r2', 'graphite_rollup') +ORDER BY key; + +CREATE TABLE t4_r_error +( + `key` UInt32, + `Path` String, + `Time` DateTime('UTC'), + `Value` Float64, + `Version` UInt32, + `col` UInt64 +) +ENGINE = ReplicatedGraphiteMergeTree('/tables/{database}/t4/', 'r3', 'graphite_rollup_alternative') ORDER BY key; -- { serverError METADATA_MISMATCH } + +-- https://github.com/ClickHouse/ClickHouse/issues/58451 +CREATE TABLE t4_r_error_2 +( + `key` UInt32, + `Path` String, + `Time` DateTime('UTC'), + `Value` Float64, + `Version` UInt32, + `col` UInt64 +) +ENGINE = ReplicatedGraphiteMergeTree('/tables/{database}/t4/', 'r4', 'graphite_rollup_alternative_no_function') +ORDER BY key; -- { serverError METADATA_MISMATCH } \ No newline at end of file diff --git a/tests/queries/0_stateless/02911_analyzer_order_by_read_in_order_query_plan.reference b/tests/queries/0_stateless/02911_analyzer_order_by_read_in_order_query_plan.reference index 5dd0d0d1820..d8f2decba37 100644 --- a/tests/queries/0_stateless/02911_analyzer_order_by_read_in_order_query_plan.reference +++ b/tests/queries/0_stateless/02911_analyzer_order_by_read_in_order_query_plan.reference @@ -13,8 +13,8 @@ select * from tab order by (a + b) * c; 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from tab order by (a + b) * c) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC select * from tab order by (a + b) * c desc; 4 4 4 4 4 4 4 4 @@ -27,8 +27,8 @@ select * from tab order by (a + b) * c desc; 0 0 0 0 0 0 0 0 select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC -- Exact match, full key select * from tab order by (a + b) * c, sin(a / b); 0 0 0 0 @@ -42,8 +42,8 @@ select * from tab order by (a + b) * c, sin(a / b); 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from tab order by (a + b) * c, sin(a / b)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC select * from tab order by (a + b) * c desc, sin(a / b) desc; 4 4 4 4 4 4 4 4 @@ -56,8 +56,8 @@ select * from tab order by (a + b) * c desc, sin(a / b) desc; 0 0 0 0 0 0 0 0 select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, sin(a / b) desc) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC, sin(divide(a_0, b_1)) DESC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC, sin(divide(a_0, b_1)) DESC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, sin(divide(__table1.a, __table1.b)) DESC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, sin(divide(__table1.a, __table1.b)) DESC -- Exact match, mixed direction select * from tab order by (a + b) * c desc, sin(a / b); 4 4 4 4 @@ -71,8 +71,8 @@ select * from tab order by (a + b) * c desc, sin(a / b); 0 0 0 0 0 0 0 0 select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, sin(a / b)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, sin(divide(__table1.a, __table1.b)) ASC select * from tab order by (a + b) * c, sin(a / b) desc; 0 0 0 0 0 0 0 0 @@ -85,8 +85,8 @@ select * from tab order by (a + b) * c, sin(a / b) desc; 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from tab order by (a + b) * c, sin(a / b) desc) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) DESC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) DESC -- Wrong order, full sort select * from tab order by sin(a / b), (a + b) * c; 1 1 1 1 @@ -100,32 +100,32 @@ select * from tab order by sin(a / b), (a + b) * c; 0 0 0 0 0 0 0 0 select * from (explain plan actions = 1 select * from tab order by sin(a / b), (a + b) * c) where explain ilike '%sort description%'; - Sort description: sin(divide(a_0, b_1)) ASC, multiply(plus(a_0, b_1), c_2) ASC + Sort description: sin(divide(__table1.a, __table1.b)) ASC, multiply(plus(__table1.a, __table1.b), __table1.c) ASC -- Fixed point select * from tab where (a + b) * c = 8 order by sin(a / b); 2 2 2 2 2 2 2 2 select * from (explain plan actions = 1 select * from tab where (a + b) * c = 8 order by sin(a / b)) where explain ilike '%sort description%'; - Prefix sort description: sin(divide(a_0, b_1)) ASC - Result sort description: sin(divide(a_0, b_1)) ASC + Prefix sort description: sin(divide(__table1.a, __table1.b)) ASC + Result sort description: sin(divide(__table1.a, __table1.b)) ASC select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c; 1 1 1 1 1 1 1 1 select * from (explain plan actions = 1 select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c) where explain ilike '%sort description%'; - Prefix sort description: multiply(plus(d_3, 1_UInt8), 4_UInt8) ASC, multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(d_3, 1_UInt8), 4_UInt8) ASC, multiply(plus(a_0, b_1), c_2) ASC + Prefix sort description: multiply(plus(__table1.d, 1_UInt8), 4_UInt8) ASC, multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.d, 1_UInt8), 4_UInt8) ASC, multiply(plus(__table1.a, __table1.b), __table1.c) ASC select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b); 2 2 2 2 2 2 2 2 select * from (explain plan actions = 1 select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b)) where explain ilike '%sort description%'; - Prefix sort description: multiply(plus(d_3, 1_UInt8), 4_UInt8) ASC, sin(divide(a_0, b_1)) ASC - Result sort description: multiply(plus(d_3, 1_UInt8), 4_UInt8) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.d, 1_UInt8), 4_UInt8) ASC, sin(divide(__table1.a, __table1.b)) ASC + Result sort description: multiply(plus(__table1.d, 1_UInt8), 4_UInt8) ASC, sin(divide(__table1.a, __table1.b)) ASC -- Wrong order with fixed point select * from tab where (a + b) * c = 8 order by sin(b / a); 2 2 2 2 2 2 2 2 select * from (explain plan actions = 1 select * from tab where (a + b) * c = 8 order by sin(b / a)) where explain ilike '%sort description%'; - Sort description: sin(divide(b_1, a_0)) ASC + Sort description: sin(divide(__table1.b, __table1.a)) ASC -- Monotonicity select * from tab order by intDiv((a + b) * c, 2); 0 0 0 0 @@ -139,8 +139,8 @@ select * from tab order by intDiv((a + b) * c, 2); 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from tab order by intDiv((a + b) * c, 2)) where explain like '%sort description%'; - Prefix sort description: intDiv(multiply(plus(a_0, b_1), c_2), 2_UInt8) ASC - Result sort description: intDiv(multiply(plus(a_0, b_1), c_2), 2_UInt8) ASC + Prefix sort description: intDiv(multiply(plus(__table1.a, __table1.b), __table1.c), 2_UInt8) ASC + Result sort description: intDiv(multiply(plus(__table1.a, __table1.b), __table1.c), 2_UInt8) ASC select * from tab order by intDiv((a + b) * c, 2), sin(a / b); 0 0 0 0 0 0 0 0 @@ -153,36 +153,36 @@ select * from tab order by intDiv((a + b) * c, 2), sin(a / b); 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from tab order by intDiv((a + b) * c, 2), sin(a / b)) where explain like '%sort description%'; - Prefix sort description: intDiv(multiply(plus(a_0, b_1), c_2), 2_UInt8) ASC - Result sort description: intDiv(multiply(plus(a_0, b_1), c_2), 2_UInt8) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: intDiv(multiply(plus(__table1.a, __table1.b), __table1.c), 2_UInt8) ASC + Result sort description: intDiv(multiply(plus(__table1.a, __table1.b), __table1.c), 2_UInt8) ASC, sin(divide(__table1.a, __table1.b)) ASC -- select * from tab order by (a + b) * c, intDiv(sin(a / b), 2); select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(sin(a / b), 2)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) ASC -- select * from tab order by (a + b) * c desc , intDiv(sin(a / b), 2); select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc , intDiv(sin(a / b), 2)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) ASC -- select * from tab order by (a + b) * c, intDiv(sin(a / b), 2) desc; select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(sin(a / b), 2) desc) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) DESC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) DESC -- select * from tab order by (a + b) * c desc, intDiv(sin(a / b), 2) desc; select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(sin(a / b), 2) desc) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) DESC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) DESC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) DESC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) DESC -- select * from tab order by (a + b) * c desc, intDiv(sin(a / b), -2); select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(sin(a / b), -2)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), -2_Int8) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), -2_Int8) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8) ASC -- select * from tab order by (a + b) * c desc, intDiv(intDiv(sin(a / b), -2), -3); select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(intDiv(sin(a / b), -2), -3)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC - Result sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(intDiv(sin(divide(a_0, b_1)), -2_Int8), -3_Int8) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8), -3_Int8) ASC -- select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3); select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3)) where explain like '%sort description%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(intDiv(sin(divide(a_0, b_1)), -2_Int8), -3_Int8) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(intDiv(sin(divide(a_0, b_1)), -2_Int8), -3_Int8) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8), -3_Int8) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8), -3_Int8) ASC -- Aliases select * from (select *, a + b as x from tab) order by x * c; 0 0 0 0 0 @@ -196,8 +196,8 @@ select * from (select *, a + b as x from tab) order by x * c; 4 4 4 4 8 4 4 4 4 8 select * from (explain plan actions = 1 select * from (select *, a + b as x from tab) order by x * c) where explain like '%sort description%'; - Prefix sort description: multiply(x_4, c_2) ASC - Result sort description: multiply(x_4, c_2) ASC + Prefix sort description: multiply(__table1.x, __table1.c) ASC + Result sort description: multiply(__table1.x, __table1.c) ASC select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y); 0 0 0 0 0 nan 0 0 0 0 0 nan @@ -210,8 +210,8 @@ select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y) 4 4 4 4 8 1 4 4 4 4 8 1 select * from (explain plan actions = 1 select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y)) where explain like '%sort description%'; - Prefix sort description: multiply(x_4, c_2) ASC, sin(y_5) ASC - Result sort description: multiply(x_4, c_2) ASC, sin(y_5) ASC + Prefix sort description: multiply(__table1.x, __table1.c) ASC, sin(__table1.y) ASC + Result sort description: multiply(__table1.x, __table1.c) ASC, sin(__table1.y) ASC select * from (select *, a / b as y from (select *, a + b as x from tab)) order by x * c, sin(y); 0 0 0 0 0 nan 0 0 0 0 0 nan @@ -224,8 +224,8 @@ select * from (select *, a / b as y from (select *, a + b as x from tab)) order 4 4 4 4 8 1 4 4 4 4 8 1 select * from (explain plan actions = 1 select * from (select *, a / b as y from (select *, a + b as x from tab)) order by x * c, sin(y)) where explain like '%sort description%'; - Prefix sort description: multiply(x_4, c_2) ASC, sin(y_5) ASC - Result sort description: multiply(x_4, c_2) ASC, sin(y_5) ASC + Prefix sort description: multiply(__table1.x, __table1.c) ASC, sin(__table1.y) ASC + Result sort description: multiply(__table1.x, __table1.c) ASC, sin(__table1.y) ASC -- { echoOn } select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intDiv(y, -2), -3); @@ -238,8 +238,8 @@ select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intD 2020-02-05 00:00:00 3 3 2020-02-05 00:00:00 3 3 select * from (explain plan actions = 1 select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; - Prefix sort description: toTimezone(toTimezone(x_0, \'UTC\'_String), \'CET\'_String) ASC, intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC - Result sort description: toTimezone(toTimezone(x_0, \'UTC\'_String), \'CET\'_String) ASC, intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC + Prefix sort description: toTimezone(toTimezone(__table1.x, \'UTC\'_String), \'CET\'_String) ASC, intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC + Result sort description: toTimezone(toTimezone(__table1.x, \'UTC\'_String), \'CET\'_String) ASC, intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3); 2020-02-02 00:00:00 0 0 2020-02-02 00:00:00 0 0 @@ -250,12 +250,12 @@ select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3); 2020-02-05 00:00:00 3 3 2020-02-05 00:00:00 3 3 select * from (explain plan actions = 1 select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; - Prefix sort description: toStartOfDay(x_0) ASC - Result sort description: toStartOfDay(x_0) ASC, intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC + Prefix sort description: toStartOfDay(__table1.x) ASC + Result sort description: toStartOfDay(__table1.x) ASC, intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC -- select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3); select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; - Prefix sort description: intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC - Result sort description: intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC + Prefix sort description: intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC + Result sort description: intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC -- { echoOn } -- Union (not fully supported) @@ -281,8 +281,8 @@ select * from (select * from tab union all select * from tab3) order by (a + b) 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab3) order by (a + b) * c, sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder ReadType: InOrder select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b); @@ -291,8 +291,8 @@ select * from (select * from tab where (a + b) * c = 8 union all select * from t 3 3 3 3 3 3 3 3 select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; - Prefix sort description: sin(divide(a_0, b_1)) ASC - Result sort description: sin(divide(a_0, b_1)) ASC + Prefix sort description: sin(divide(__table1.a, __table1.b)) ASC + Result sort description: sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder ReadType: InOrder select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b); @@ -309,8 +309,8 @@ select * from (select * from tab where (a + b) * c = 8 union all select * from t 0 0 0 0 0 0 0 0 select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; - Prefix sort description: sin(divide(a_0, b_1)) ASC - Result sort description: sin(divide(a_0, b_1)) ASC + Prefix sort description: sin(divide(__table1.a, __table1.b)) ASC + Result sort description: sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder ReadType: InOrder select * from (select * from tab union all select * from tab5) order by (a + b) * c; @@ -335,8 +335,8 @@ select * from (select * from tab union all select * from tab5) order by (a + b) 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c) where explain like '%sort description%' or explain like '%ReadType%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC ReadType: InOrder ReadType: InOrder select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b); @@ -361,11 +361,11 @@ select * from (select * from tab union all select * from tab5) order by (a + b) 4 4 4 4 4 4 4 4 select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder -- Union with limit select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b) limit 3; @@ -375,12 +375,12 @@ select * from (select * from tab union all select * from tab5) order by (a + b) select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b) limit 3) where explain ilike '%sort description%' or explain like '%ReadType%' or explain like '%Limit%'; Limit (preliminary LIMIT (without OFFSET)) Limit 3 - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC Limit 3 ReadType: InOrder - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder -- In this example, we read-in-order from tab up to ((a + b) * c, sin(a / b)) and from tab5 up to ((a + b) * c). -- In case of tab5, there would be two finish sorting transforms: ((a + b) * c) -> ((a + b) * c, sin(a / b)) -> ((a + b) * c, sin(a / b), d). @@ -393,14 +393,14 @@ select * from (select * from tab union all select * from tab5 union all select * select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5 union all select * from tab4) order by (a + b) * c, sin(a / b), d limit 3) where explain ilike '%sort description%' or explain like '%ReadType%' or explain like '%Limit%'; Limit (preliminary LIMIT (without OFFSET)) Limit 3 - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC, d_3 ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC, __table1.d ASC Limit 3 ReadType: InOrder - Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC - Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC + Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC + Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC ReadType: InOrder - Sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC, d_3 ASC + Sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC, __table1.d ASC Limit 3 ReadType: Default drop table if exists tab; diff --git a/tests/queries/0_stateless/02932_idna.reference b/tests/queries/0_stateless/02932_idna.reference new file mode 100644 index 00000000000..0947194c07f --- /dev/null +++ b/tests/queries/0_stateless/02932_idna.reference @@ -0,0 +1,88 @@ +-- Negative tests +-- Regular cases +straße.de xn--strae-oqa.de xn--strae-oqa.de straße.de straße.de +2001:4860:4860::8888 2001:4860:4860::8888 2001:4860:4860::8888 2001:4860:4860::8888 2001:4860:4860::8888 +AMAZON amazon amazon amazon amazon +aa-- aa-- aa-- aa-- aa-- +a†-- xn--a---kp0a xn--a---kp0a a†-- a†-- +ab--c ab--c ab--c ab--c ab--c +-† xn----xhn xn----xhn -† -† +-x.xn--zca -x.xn--zca -x.xn--zca -x.ß -x.ß +x-.xn--zca x-.xn--zca x-.xn--zca x-.ß x-.ß +x-.ß x-.xn--zca x-.xn--zca x-.ß x-.ß +x..ß x..xn--zca x..xn--zca x..ß x..ß +128.0,0.1 128.0,0.1 128.0,0.1 128.0,0.1 128.0,0.1 +xn--zca.xn--zca xn--zca.xn--zca xn--zca.xn--zca ß.ß ß.ß +xn--zca.ß xn--zca.xn--zca xn--zca.xn--zca ß.ß ß.ß +x01234567890123456789012345678901234567890123456789012345678901x x01234567890123456789012345678901234567890123456789012345678901x x01234567890123456789012345678901234567890123456789012345678901x x01234567890123456789012345678901234567890123456789012345678901x x01234567890123456789012345678901234567890123456789012345678901x +x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.ß x01234567890123456789012345678901234567890123456789012345678901x.ß +x01234567890123456789012345678901234567890123456789012345678901x.ß x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.xn--zca x01234567890123456789012345678901234567890123456789012345678901x.ß x01234567890123456789012345678901234567890123456789012345678901x.ß +01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x 01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x +≠ xn--1ch xn--1ch ≠ ≠ +aa-- aa-- aa-- aa-- +ab--c ab--c ab--c ab--c +-x -x -x -x + +xn--1ch ≠ xn--1ch xn--1ch +xn--dqd20apc ᄎᆞᆷ xn--dqd20apc xn--dqd20apc +xn--gdh ≮ xn--gdh xn--gdh +xn--80aaa0ahbbeh4c йайзаакпий xn--80aaa0ahbbeh4c xn--80aaa0ahbbeh4c +xn--3bs854c 团淄 xn--3bs854c xn--3bs854c +xn--mgb9awbf عمان xn--mgb9awbf xn--mgb9awbf +xn--mgbaam7a8h امارات xn--mgbaam7a8h xn--mgbaam7a8h +xn--mgbbh1a71e بھارت xn--mgbbh1a71e xn--mgbbh1a71e +xn--s7y.com 短.com xn--s7y.com xn--s7y.com +xn--55qx5d.xn--tckwe 公司.コム xn--55qx5d.xn--tckwe xn--55qx5d.xn--tckwe +xn--4dbrk0ce ישראל xn--4dbrk0ce xn--4dbrk0ce +xn--zckzah テスト xn--zckzah xn--zckzah +xn--p1ai.com рф.com xn--p1ai.com xn--p1ai.com +xn--mxahbxey0c.gr εχαμπλε.gr xn--mxahbxey0c.gr xn--mxahbxey0c.gr +xn--h2brj9c भारत xn--h2brj9c xn--h2brj9c +xn--d1acpjx3f.xn--p1ai яндекс.рф xn--d1acpjx3f.xn--p1ai xn--d1acpjx3f.xn--p1ai +xn--q9jyb4c みんな xn--q9jyb4c xn--q9jyb4c +xn--sterreich-z7a.at österreich.at xn--sterreich-z7a.at xn--sterreich-z7a.at +xn--h2breg3eve.xn--h2brj9c भारतम्.भारत xn--h2breg3eve.xn--h2brj9c xn--h2breg3eve.xn--h2brj9c +ejemplo.xn--q9jyb4c ejemplo.みんな ejemplo.xn--q9jyb4c ejemplo.xn--q9jyb4c +xn--9t4b11yi5a.com 테스트.com xn--9t4b11yi5a.com xn--9t4b11yi5a.com +xn--gk3at1e.com 通販.com xn--gk3at1e.com xn--gk3at1e.com +xn--42c2d9a คอม xn--42c2d9a xn--42c2d9a +1xn-- 1xn-- 1xn-- 1xn-- +xn--bih.com ⌘.com xn--bih.com xn--bih.com +xn--4gbrim.xn----rmckbbajlc6dj7bxne2c.xn--wgbh1c موقع.وزارة-الأتصالات.مصر xn--4gbrim.xn----rmckbbajlc6dj7bxne2c.xn--wgbh1c xn--4gbrim.xn----rmckbbajlc6dj7bxne2c.xn--wgbh1c +xn--mgbb9fbpob موبايلي xn--mgbb9fbpob xn--mgbb9fbpob +xn--55qw42g.xn--55qw42g 公益.公益 xn--55qw42g.xn--55qw42g xn--55qw42g.xn--55qw42g +≠ ≠ xn--1ch xn--1ch +ファッション.biz ファッション.biz xn--bck1b9a5dre4c.biz xn--bck1b9a5dre4c.biz +-- Special cases +---- Empty input + + + +---- NULL input +\N +\N +\N +---- Garbage inputs for idnaEncode + + + + +---- Long input +Row 1: +────── +idna: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +ascii: wenn sie ...xn-- vom hauptbahnhof in mnchen -n7c...xn-- mit zehn minuten, ohne, dass sie am flughafen noch einchecken mssen, dann starten sie im grunde genommen am flughafen -8gm... am ...xn-- am hauptbahnhof in mnchen starten sie ihren flug-0cf. zehn minuten.xn-- schauen sie sich mal die groen flughfen an, wenn sie in heathrow in london oder sonst wo, meine se -83h23c...xn-- charles de gaulle h in frankreich oder in -jvd...xn--h-zfa... in ... in...xn--h-zfa...in rom. wenn sie sich mal die entfernungen ansehen, wenn sie frankfurt sich ansehen, dann werden sie feststellen, dass zehn minuten... sie jederzeit locker in frankfurt brauchen, um ihr gate zu finden. wenn sie vom flug ... vom ... vom hauptbahnhof starten - sie steigen in den hauptbahnhof ein, sie fahren mit dem transrapid in zehn minuten an den flughafen in ...xn-- an den flughafen franz josef strau-z2c.xn-- dann starten sie praktisch hier am hauptbahnhof in mnchen-t9f.xn-- das bedeutet natrlich, dass der hauptbahnhof im grunde genommen nher an bayern -lxg23q...xn-- an die bayerischen stdte heranwchst, weil das ja klar ist, weil auf dem hauptbahnhof viele linien aus bayern zusammenlaufen-1hkk. +ascii_try: wenn sie ...xn-- vom hauptbahnhof in mnchen -n7c...xn-- mit zehn minuten, ohne, dass sie am flughafen noch einchecken mssen, dann starten sie im grunde genommen am flughafen -8gm... am ...xn-- am hauptbahnhof in mnchen starten sie ihren flug-0cf. zehn minuten.xn-- schauen sie sich mal die groen flughfen an, wenn sie in heathrow in london oder sonst wo, meine se -83h23c...xn-- charles de gaulle h in frankreich oder in -jvd...xn--h-zfa... in ... in...xn--h-zfa...in rom. wenn sie sich mal die entfernungen ansehen, wenn sie frankfurt sich ansehen, dann werden sie feststellen, dass zehn minuten... sie jederzeit locker in frankfurt brauchen, um ihr gate zu finden. wenn sie vom flug ... vom ... vom hauptbahnhof starten - sie steigen in den hauptbahnhof ein, sie fahren mit dem transrapid in zehn minuten an den flughafen in ...xn-- an den flughafen franz josef strau-z2c.xn-- dann starten sie praktisch hier am hauptbahnhof in mnchen-t9f.xn-- das bedeutet natrlich, dass der hauptbahnhof im grunde genommen nher an bayern -lxg23q...xn-- an die bayerischen stdte heranwchst, weil das ja klar ist, weil auf dem hauptbahnhof viele linien aus bayern zusammenlaufen-1hkk. +original: wenn sie ... vom hauptbahnhof in münchen ... mit zehn minuten, ohne, dass sie am flughafen noch einchecken müssen, dann starten sie im grunde genommen am flughafen ... am ... am hauptbahnhof in münchen starten sie ihren flug. zehn minuten. schauen sie sich mal die großen flughäfen an, wenn sie in heathrow in london oder sonst wo, meine se ... charles de gaulle äh in frankreich oder in ...äh... in ... in...äh...in rom. wenn sie sich mal die entfernungen ansehen, wenn sie frankfurt sich ansehen, dann werden sie feststellen, dass zehn minuten... sie jederzeit locker in frankfurt brauchen, um ihr gate zu finden. wenn sie vom flug ... vom ... vom hauptbahnhof starten - sie steigen in den hauptbahnhof ein, sie fahren mit dem transrapid in zehn minuten an den flughafen in ... an den flughafen franz josef strauß. dann starten sie praktisch hier am hauptbahnhof in münchen. das bedeutet natürlich, dass der hauptbahnhof im grunde genommen näher an bayern ... an die bayerischen städte heranwächst, weil das ja klar ist, weil auf dem hauptbahnhof viele linien aus bayern zusammenlaufen. +original_try: wenn sie ... vom hauptbahnhof in münchen ... mit zehn minuten, ohne, dass sie am flughafen noch einchecken müssen, dann starten sie im grunde genommen am flughafen ... am ... am hauptbahnhof in münchen starten sie ihren flug. zehn minuten. schauen sie sich mal die großen flughäfen an, wenn sie in heathrow in london oder sonst wo, meine se ... charles de gaulle äh in frankreich oder in ...äh... in ... in...äh...in rom. wenn sie sich mal die entfernungen ansehen, wenn sie frankfurt sich ansehen, dann werden sie feststellen, dass zehn minuten... sie jederzeit locker in frankfurt brauchen, um ihr gate zu finden. wenn sie vom flug ... vom ... vom hauptbahnhof starten - sie steigen in den hauptbahnhof ein, sie fahren mit dem transrapid in zehn minuten an den flughafen in ... an den flughafen franz josef strauß. dann starten sie praktisch hier am hauptbahnhof in münchen. das bedeutet natürlich, dass der hauptbahnhof im grunde genommen näher an bayern ... an die bayerischen städte heranwächst, weil das ja klar ist, weil auf dem hauptbahnhof viele linien aus bayern zusammenlaufen. +---- Non-const input + +münchen xn--mnchen-3ya xn--mnchen-3ya münchen münchen +straße.münchen.de xn--strae-oqa.xn--mnchen-3ya.de xn--strae-oqa.xn--mnchen-3ya.de straße.münchen.de straße.münchen.de +---- Non-const input with invalid values sprinkled in +london.co.uk london.co.uk london.co.uk +microsoft.com microsoft.com microsoft.com +xn-- +xn-- +xn--tešla +ytraße.münchen.de xn--ytrae-oqa.xn--mnchen-3ya.de ytraße.münchen.de diff --git a/tests/queries/0_stateless/02932_idna.sql b/tests/queries/0_stateless/02932_idna.sql new file mode 100644 index 00000000000..db7688064f2 --- /dev/null +++ b/tests/queries/0_stateless/02932_idna.sql @@ -0,0 +1,124 @@ +-- Tags: no-fasttest +-- no-fasttest: requires idna library + +-- See also 02932_punycode.sql + +SELECT '-- Negative tests'; + +SELECT idnaEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryIdnaEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT idnaDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT idnaEncode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT tryIdnaEncode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT idnaDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT idnaEncode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryIdnaEncode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT idnaDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT idnaEncode(toFixedString('two', 3)); -- { serverError NOT_IMPLEMENTED } +SELECT tryIdnaEncode(toFixedString('two', 3)); -- { serverError NOT_IMPLEMENTED } +SELECT idnaDecode(toFixedString('two', 3)); -- { serverError NOT_IMPLEMENTED } + +SELECT '-- Regular cases'; + +-- The test cases originate from the ada idna unit tests: +-- - https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/to_ascii_alternating.txt +-- - https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/to_unicode_alternating.txt +-- +SELECT 'straße.de' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT '2001:4860:4860::8888' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'AMAZON' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'aa--' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'a†--' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'ab--c' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT '-†' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT '-x.xn--zca' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'x-.xn--zca' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'x-.ß' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'x..ß' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT '128.0,0.1' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'xn--zca.xn--zca' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'xn--zca.ß' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'x01234567890123456789012345678901234567890123456789012345678901x' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'x01234567890123456789012345678901234567890123456789012345678901x.xn--zca' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT 'x01234567890123456789012345678901234567890123456789012345678901x.ß' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT '01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.01234567890123456789012345678901234567890123456789.0123456789012345678901234567890123456789012345678.x' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; +SELECT '≠' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(idna) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try; + +SELECT 'aa--' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'ab--c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT '-x' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT '' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--1ch' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--dqd20apc' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--gdh' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--80aaa0ahbbeh4c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--3bs854c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--mgb9awbf' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--mgbaam7a8h' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--mgbbh1a71e' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--s7y.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--55qx5d.xn--tckwe' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--4dbrk0ce' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--zckzah' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--p1ai.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--mxahbxey0c.gr' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--h2brj9c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--d1acpjx3f.xn--p1ai' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--q9jyb4c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--sterreich-z7a.at' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--h2breg3eve.xn--h2brj9c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'ejemplo.xn--q9jyb4c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--9t4b11yi5a.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--gk3at1e.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--42c2d9a' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT '1xn--' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--bih.com' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--4gbrim.xn----rmckbbajlc6dj7bxne2c.xn--wgbh1c' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--mgbb9fbpob' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'xn--55qw42g.xn--55qw42g' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT '≠' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +SELECT 'ファッション.biz' AS ascii, idnaDecode(ascii) AS unicode, idnaEncode(unicode) AS original, tryIdnaEncode(unicode) AS original_try; +-- +SELECT '-- Special cases'; + +SELECT '---- Empty input'; +SELECT idnaEncode(''); +SELECT tryIdnaEncode(''); +SELECT idnaDecode(''); + +SELECT '---- NULL input'; +SELECT idnaEncode(NULL); +SELECT tryIdnaEncode(NULL); +SELECT idnaDecode(NULL); + +SELECT '---- Garbage inputs for idnaEncode'; +-- - https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/to_ascii_invalid.txt +SELECT idnaEncode('xn--'); -- { serverError BAD_ARGUMENTS } +SELECT tryIdnaEncode('xn--'); +SELECT idnaEncode('ﻱa'); -- { serverError BAD_ARGUMENTS } +SELECT tryIdnaEncode('ﻱa'); +SELECT idnaEncode('xn--a-yoc'); -- { serverError BAD_ARGUMENTS } +SELECT tryIdnaEncode('xn--a-yoc'); +SELECT idnaEncode('xn--tešla'); -- { serverError BAD_ARGUMENTS } +SELECT tryIdnaEncode('xn--tešla'); + +SELECT '---- Long input'; +SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS idna, idnaEncode(idna) AS ascii, tryIdnaEncode(ascii) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try FORMAT Vertical; + +SELECT '---- Non-const input'; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (idna String) ENGINE=MergeTree ORDER BY idna; +INSERT INTO tab VALUES ('straße.münchen.de') ('') ('münchen'); +SELECT idna, idnaEncode(idna) AS ascii, tryIdnaEncode(ascii) AS ascii_try, idnaDecode(ascii) AS original, idnaDecode(ascii_try) AS original_try FROM tab; +DROP TABLE tab; + +SELECT '---- Non-const input with invalid values sprinkled in'; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (idna String) ENGINE=MergeTree ORDER BY idna; +INSERT INTO tab VALUES ('xn--') ('london.co.uk') ('ytraße.münchen.de') ('xn--tešla') ('microsoft.com') ('xn--'); +SELECT idna, idnaEncode(idna) AS ascii FROM tab; -- { serverError BAD_ARGUMENTS } +SELECT idna, tryIdnaEncode(idna) AS ascii, idnaDecode(ascii) AS original FROM tab; +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02932_kill_query_sleep.sh b/tests/queries/0_stateless/02932_kill_query_sleep.sh index 08c375b875d..84e84204aa1 100755 --- a/tests/queries/0_stateless/02932_kill_query_sleep.sh +++ b/tests/queries/0_stateless/02932_kill_query_sleep.sh @@ -8,18 +8,31 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function wait_query_started() { local query_id="$1" - $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" - while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.query_log WHERE query_id='$query_id' AND current_database = currentDatabase()") == 0 ]]; do - sleep 0.1; - $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS;" + timeout=60 + start=$EPOCHSECONDS + while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id'") == 0 ]]; do + if ((EPOCHSECONDS-start > timeout )); then + echo "Timeout while waiting for query $query_id to start" + exit 1 + fi + sleep 0.1 done } + function kill_query() { local query_id="$1" $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id='$query_id'" >/dev/null - while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id'") != 0 ]]; do sleep 0.1; done + timeout=60 + start=$EPOCHSECONDS + while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id'") != 0 ]]; do + if ((EPOCHSECONDS-start > timeout )); then + echo "Timeout while waiting for query $query_id to cancel" + exit 1 + fi + sleep 0.1 + done } diff --git a/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.reference b/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.reference new file mode 100644 index 00000000000..8fb8a08e3f9 --- /dev/null +++ b/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.reference @@ -0,0 +1,4 @@ +42 +42 +42 +42 diff --git a/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.sql b/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.sql new file mode 100644 index 00000000000..af6dbf24473 --- /dev/null +++ b/tests/queries/0_stateless/02932_materialized_view_with_dropped_target_table_no_exception.sql @@ -0,0 +1,21 @@ +set ignore_materialized_views_with_dropped_target_table = 1; +drop table if exists from_table; +drop table if exists to_table; +drop table if exists mv; + +create table from_table (x UInt32) engine=MergeTree order by x; +create table to_table (x UInt32) engine=MergeTree order by x; +create materialized view mv to to_table as select * from from_table; + +insert into from_table select 42; +select * from from_table; +select * from to_table; + +drop table to_table; + +insert into from_table select 42; +select * from from_table; + +drop table from_table; +drop view mv; + diff --git a/tests/queries/0_stateless/02932_punycode.reference b/tests/queries/0_stateless/02932_punycode.reference index 7a39a221e08..ff05eaa72a3 100644 --- a/tests/queries/0_stateless/02932_punycode.reference +++ b/tests/queries/0_stateless/02932_punycode.reference @@ -1,35 +1,55 @@ -- Negative tests -- Regular cases -a a- a -A A- A --- --- -- -London London- London -Lloyd-Atkinson Lloyd-Atkinson- Lloyd-Atkinson -This has spaces This has spaces- This has spaces --> $1.00 <- -> $1.00 <-- -> $1.00 <- -а 80a а -ü tda ü -α mxa α -例 fsq 例 -😉 n28h 😉 -αβγ mxacd αβγ -München Mnchen-3ya München -Mnchen-3ya Mnchen-3ya- Mnchen-3ya -München-Ost Mnchen-Ost-9db München-Ost -Bahnhof München-Ost Bahnhof Mnchen-Ost-u6b Bahnhof München-Ost -abæcdöef abcdef-qua4k abæcdöef -правда 80aafi6cg правда -ยจฆฟคฏข 22cdfh1b8fsa ยจฆฟคฏข -ドメイン名例 eckwd4c7cu47r2wf ドメイン名例 -MajiでKoiする5秒前 MajiKoi5-783gue6qz075azm5e MajiでKoiする5秒前 -「bücher」 bcher-kva8445foa 「bücher」 -团淄 3bs854c 团淄 +a a- a a +A A- A A +-- --- -- -- +London London- London London +Lloyd-Atkinson Lloyd-Atkinson- Lloyd-Atkinson Lloyd-Atkinson +This has spaces This has spaces- This has spaces This has spaces +-> $1.00 <- -> $1.00 <-- -> $1.00 <- -> $1.00 <- +а 80a а а +ü tda ü ü +α mxa α α +例 fsq 例 例 +😉 n28h 😉 😉 +αβγ mxacd αβγ αβγ +München Mnchen-3ya München München +Mnchen-3ya Mnchen-3ya- Mnchen-3ya Mnchen-3ya +München-Ost Mnchen-Ost-9db München-Ost München-Ost +Bahnhof München-Ost Bahnhof Mnchen-Ost-u6b Bahnhof München-Ost Bahnhof München-Ost +abæcdöef abcdef-qua4k abæcdöef abæcdöef +правда 80aafi6cg правда правда +ยจฆฟคฏข 22cdfh1b8fsa ยจฆฟคฏข ยจฆฟคฏข +ドメイン名例 eckwd4c7cu47r2wf ドメイン名例 ドメイン名例 +MajiでKoiする5秒前 MajiKoi5-783gue6qz075azm5e MajiでKoiする5秒前 MajiでKoiする5秒前 +「bücher」 bcher-kva8445foa 「bücher」 「bücher」 +团淄 3bs854c 团淄 团淄 -- Special cases +---- Empty input + +---- NULL input \N \N -Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. -München Mnchen-3ya München -abc abc- abc -aäoöuü aou-qla5gqb aäoöuü +\N +---- Garbage Punycode-encoded input + +---- Long input +Row 1: +────── +str: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +puny: Wenn Sie ... vom Hauptbahnhof in Mnchen ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken mssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in Mnchen starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die groen Flughfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle h in Frankreich oder in ...h... in ... in...h...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strau. Dann starten Sie praktisch hier am Hauptbahnhof in Mnchen. Das bedeutet natrlich, dass der Hauptbahnhof im Grunde genommen nher an Bayern ... an die bayerischen Stdte heranwchst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.-pu7fjtp0npc1ar54cibk471wdc9d18axa +original: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +original_try: Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen. +---- Non-const values +München Mnchen-3ya München München +abc abc- abc abc +aäoöuü aou-qla5gqb aäoöuü aäoöuü +---- Non-const values with invalid values sprinkled in +Also no punycode +London- London +Mnchen-3ya München +No punycode +Rtting-3ya Rütting +XYZ no punycode diff --git a/tests/queries/0_stateless/02932_punycode.sql b/tests/queries/0_stateless/02932_punycode.sql index dd18a43ecc9..b9bcf933641 100644 --- a/tests/queries/0_stateless/02932_punycode.sql +++ b/tests/queries/0_stateless/02932_punycode.sql @@ -1,63 +1,86 @@ -- Tags: no-fasttest -- no-fasttest: requires idna library +-- See also 02932_idna.sql + SELECT '-- Negative tests'; -SELECT punycodeDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT punycodeEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT punycodeDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT punycodeDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryPunycodeDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + SELECT punycodeEncode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT punycodeDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT punycodeDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT tryPunycodeDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + SELECT punycodeEncode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT punycodeDecode(toFixedString('two', 3)); -- { serverError ILLEGAL_COLUMN } -SELECT punycodeEncode(toFixedString('two', 3)); -- { serverError ILLEGAL_COLUMN } +SELECT punycodeDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryPunycodeDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT punycodeEncode(toFixedString('two', 3)); -- { serverError NOT_IMPLEMENTED } +SELECT punycodeDecode(toFixedString('two', 3)); -- { serverError NOT_IMPLEMENTED } +SELECT tryPunycodeDecode(toFixedString('two', 3)); -- { serverError NOT_IMPLEMENTED } SELECT '-- Regular cases'; -- The test cases originate from the ada idna unit tests: ---- https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/utf8_punycode_alternating.txt - -SELECT 'a' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'A' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT '--' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'London' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'Lloyd-Atkinson' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'This has spaces' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT '-> $1.00 <-' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'а' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'ü' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'α' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT '例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT '😉' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'αβγ' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'München' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'Mnchen-3ya' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'Bahnhof München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'abæcdöef' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'правда' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'ยจฆฟคฏข' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'ドメイン名例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT 'MajiでKoiする5秒前' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT '「bücher」' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; -SELECT '团淄' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +-- - https://github.com/ada-url/idna/blob/8cd03ef867dbd06be87bd61df9cf69aa1182ea21/tests/fixtures/utf8_punycode_alternating.txt +SELECT 'a' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'A' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT '--' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'London' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'Lloyd-Atkinson' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'This has spaces' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT '-> $1.00 <-' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'а' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'ü' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'α' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT '例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT '😉' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'αβγ' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'München' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'Mnchen-3ya' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'Bahnhof München-Ost' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'abæcdöef' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'правда' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'ยจฆฟคฏข' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'ドメイン名例' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT 'MajiでKoiする5秒前' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT '「bücher」' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +SELECT '团淄' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try; +-- SELECT '-- Special cases'; -SELECT punycodeDecode(''); +SELECT '---- Empty input'; SELECT punycodeEncode(''); -SELECT punycodeDecode(NULL); +SELECT punycodeDecode(''); +SELECT tryPunycodeDecode(''); + +SELECT '---- NULL input'; SELECT punycodeEncode(NULL); +SELECT punycodeDecode(NULL); +SELECT tryPunycodeDecode(NULL); --- garbage Punycode-encoded values +SELECT '---- Garbage Punycode-encoded input'; SELECT punycodeDecode('no punycode'); -- { serverError BAD_ARGUMENTS } +SELECT tryPunycodeDecode('no punycode'); --- long input -SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original; +SELECT '---- Long input'; +SELECT 'Wenn Sie ... vom Hauptbahnhof in München ... mit zehn Minuten, ohne, dass Sie am Flughafen noch einchecken müssen, dann starten Sie im Grunde genommen am Flughafen ... am ... am Hauptbahnhof in München starten Sie Ihren Flug. Zehn Minuten. Schauen Sie sich mal die großen Flughäfen an, wenn Sie in Heathrow in London oder sonst wo, meine se ... Charles de Gaulle äh in Frankreich oder in ...äh... in ... in...äh...in Rom. Wenn Sie sich mal die Entfernungen ansehen, wenn Sie Frankfurt sich ansehen, dann werden Sie feststellen, dass zehn Minuten... Sie jederzeit locker in Frankfurt brauchen, um ihr Gate zu finden. Wenn Sie vom Flug ... vom ... vom Hauptbahnhof starten - Sie steigen in den Hauptbahnhof ein, Sie fahren mit dem Transrapid in zehn Minuten an den Flughafen in ... an den Flughafen Franz Josef Strauß. Dann starten Sie praktisch hier am Hauptbahnhof in München. Das bedeutet natürlich, dass der Hauptbahnhof im Grunde genommen näher an Bayern ... an die bayerischen Städte heranwächst, weil das ja klar ist, weil auf dem Hauptbahnhof viele Linien aus Bayern zusammenlaufen.' AS str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try FORMAT Vertical; --- non-const values +SELECT '---- Non-const values'; DROP TABLE IF EXISTS tab; CREATE TABLE tab (str String) ENGINE=MergeTree ORDER BY str; INSERT INTO tab VALUES ('abc') ('aäoöuü') ('München'); -SELECT str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original FROM tab; +SELECT str, punycodeEncode(str) AS puny, punycodeDecode(puny) AS original, tryPunycodeDecode(puny) AS original_try FROM tab; +DROP TABLE tab; + +SELECT '---- Non-const values with invalid values sprinkled in'; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (puny String) ENGINE=MergeTree ORDER BY puny; +INSERT INTO tab VALUES ('Also no punycode') ('London-') ('Mnchen-3ya') ('No punycode') ('Rtting-3ya') ('XYZ no punycode'); +SELECT puny, punycodeDecode(puny) AS original FROM tab; -- { serverError BAD_ARGUMENTS } +SELECT puny, tryPunycodeDecode(puny) AS original FROM tab; DROP TABLE tab; diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference new file mode 100644 index 00000000000..4c5b678cfa5 --- /dev/null +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference @@ -0,0 +1,44 @@ +<1: created view> a [] 1 +CREATE MATERIALIZED VIEW default.a\nREFRESH AFTER 1 SECOND\n(\n `x` UInt64\n)\nENGINE = Memory AS\nSELECT number AS x\nFROM numbers(2)\nUNION ALL\nSELECT rand64() AS x +<2: refreshed> 3 1 1 +<3: time difference at least> 500 +<4: next refresh in> 1 +<4.5: altered> Scheduled Finished 2052-01-01 00:00:00 +CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` Int16\n)\nENGINE = Memory AS\nSELECT x * 2 AS x\nFROM default.src +<5: no refresh> 3 +<6: refreshed> 2 +<7: refreshed> Scheduled Finished 2054-01-01 00:00:00 +CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192 AS\nSELECT x * 10 AS y\nFROM default.a +<8: refreshed> 20 +<9: refreshed> a Scheduled Finished 2054-01-01 00:00:00 +<9: refreshed> b Scheduled Finished 2054-01-01 00:00:00 +<10: waiting> a Scheduled [] 2054-01-01 00:00:00 +<10: waiting> b WaitingForDependencies ['default.a'] 2054-01-01 00:00:00 +<11: chain-refreshed a> 4 +<12: chain-refreshed b> 40 +<13: chain-refreshed> a Scheduled [] Finished 2054-01-01 00:00:01 2056-01-01 00:00:00 +<13: chain-refreshed> b Scheduled ['default.a'] Finished 2054-01-24 23:22:21 2056-01-01 00:00:00 +<14: waiting for next cycle> a Scheduled [] 2058-01-01 00:00:00 +<14: waiting for next cycle> b WaitingForDependencies ['default.a'] 2060-01-01 00:00:00 +<15: chain-refreshed a> 6 +<16: chain-refreshed b> 60 +<17: chain-refreshed> a Scheduled 2062-01-01 00:00:00 +<17: chain-refreshed> b Scheduled 2062-01-01 00:00:00 +<18: removed dependency> b Scheduled [] 2062-03-03 03:03:03 2064-01-01 00:00:00 5 +CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192 AS\nSELECT x * 10 AS y\nFROM default.a +<19: exception> 1 +<20: unexception> 1 +<21: rename> 1 +<22: rename> d Finished +<23: simple refresh> 1 +<24: rename during refresh> 1 +<25: rename during refresh> f Running +<27: cancelled> f Scheduled +CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory AS\nSELECT 42 +<29: randomize> 1 1 +CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n) AS\nSELECT x * 10 AS x\nFROM default.src +<30: to existing table> 10 +<31: to existing table> 10 +<31: to existing table> 20 +<32: empty> i Scheduled Unknown +<32: empty> j Scheduled Finished diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh new file mode 100755 index 00000000000..8daea063fc5 --- /dev/null +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -0,0 +1,303 @@ +#!/usr/bin/env bash +# Tags: atomic-database + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# Set session timezone to UTC to make all DateTime formatting and parsing use UTC, because refresh +# scheduling is done in UTC. +CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT" | sed 's/--session_timezone[= ][^ ]*//g'`" +CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT --allow_experimental_refreshable_materialized_view=1 --session_timezone Etc/UTC"`" + +$CLICKHOUSE_CLIENT -nq "create view refreshes as select * from system.view_refreshes where database = '$CLICKHOUSE_DATABASE' order by view" + + +# Basic refreshing. +$CLICKHOUSE_CLIENT -nq " + create materialized view a + refresh after 1 second + engine Memory + empty + as select number as x from numbers(2) union all select rand64() as x" +$CLICKHOUSE_CLIENT -nq "select '<1: created view>', view, remaining_dependencies, exception, last_refresh_result in ('Unknown', 'Finished') from refreshes"; +$CLICKHOUSE_CLIENT -nq "show create a" +# Wait for any refresh. (xargs trims the string and turns \t and \n into spaces) +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" == 'Unknown' ] +do + sleep 0.1 +done +# Check table contents. +$CLICKHOUSE_CLIENT -nq "select '<2: refreshed>', count(), sum(x=0), sum(x=1) from a" +# Wait for table contents to change. +res1="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values'`" +while : +do + res2="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" + [ "$res2" == "$res1" ] || break + sleep 0.1 +done +time2="`$CLICKHOUSE_CLIENT -nq "select reinterpret(now64(), 'Int64')"`" +# Wait for another change. +while : +do + res3="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" + [ "$res3" == "$res2" ] || break + sleep 0.1 +done +# Check that the two changes were at least 500ms apart, in particular that we're not refreshing +# like crazy. This is potentially flaky, but we need at least one test that uses non-mocked timer +# to make sure the clock+timer code works at all. If it turns out flaky, increase refresh period above. +$CLICKHOUSE_CLIENT -nq " + select '<3: time difference at least>', min2(reinterpret(now64(), 'Int64') - $time2, 500); + select '<4: next refresh in>', next_refresh_time-last_refresh_time from refreshes;" + +# Create a source table from which views will read. +$CLICKHOUSE_CLIENT -nq " + create table src (x Int8) engine Memory as select 1" + +# Switch to fake clock, change refresh schedule, change query. +$CLICKHOUSE_CLIENT -nq " + system test view a set fake time '2050-01-01 00:00:01';" +while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:02' ] +do + sleep 0.1 +done +$CLICKHOUSE_CLIENT -nq " + alter table a modify refresh every 2 year; + alter table a modify query select x*2 as x from src; + select '<4.5: altered>', status, last_refresh_result, next_refresh_time from refreshes; + show create a;" +# Advance time to trigger the refresh. +$CLICKHOUSE_CLIENT -nq " + select '<5: no refresh>', count() from a; + system test view a set fake time '2052-02-03 04:05:06';" +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_time from refreshes -- $LINENO" | xargs`" != '2052-02-03 04:05:06' ] +do + sleep 0.1 +done +$CLICKHOUSE_CLIENT -nq " + select '<6: refreshed>', * from a; + select '<7: refreshed>', status, last_refresh_result, next_refresh_time from refreshes;" + +# Create a dependent view, refresh it once. +$CLICKHOUSE_CLIENT -nq " + create materialized view b refresh every 2 year depends on a (y Int32) engine MergeTree order by y empty as select x*10 as y from a; + show create b; + system test view b set fake time '2052-11-11 11:11:11'; + system refresh view b;" +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2052-11-11 11:11:11' ] +do + sleep 0.1 +done +# Next refresh shouldn't start until the dependency refreshes. +$CLICKHOUSE_CLIENT -nq " + select '<8: refreshed>', * from b; + select '<9: refreshed>', view, status, last_refresh_result, next_refresh_time from refreshes; + system test view b set fake time '2054-01-24 23:22:21';" +while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies 2054-01-01 00:00:00' ] +do + sleep 0.1 +done +# Update source table (by dropping and re-creating it - to test that tables are looked up by name +# rather than uuid), kick off refresh of the dependency. +$CLICKHOUSE_CLIENT -nq " + select '<10: waiting>', view, status, remaining_dependencies, next_refresh_time from refreshes; + drop table src; + create table src (x Int16) engine Memory as select 2; + system test view a set fake time '2054-01-01 00:00:01';" +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled' ] +do + sleep 0.1 +done +# Both tables should've refreshed. +$CLICKHOUSE_CLIENT -nq " + select '<11: chain-refreshed a>', * from a; + select '<12: chain-refreshed b>', * from b; + select '<13: chain-refreshed>', view, status, remaining_dependencies, last_refresh_result, last_refresh_time, next_refresh_time, exception from refreshes;" + +# Make the dependent table run ahead by one refresh cycle, make sure it waits for the dependency to +# catch up to the same cycle. +$CLICKHOUSE_CLIENT -nq " + system test view b set fake time '2059-01-01 00:00:00'; + system refresh view b;" +while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2060-01-01 00:00:00' ] +do + sleep 0.1 +done +$CLICKHOUSE_CLIENT -nq " + system test view b set fake time '2061-01-01 00:00:00'; + system test view a set fake time '2057-01-01 00:00:00';" +while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2058-01-01 00:00:00 WaitingForDependencies 2060-01-01 00:00:00' ] +do + sleep 0.1 +done +sleep 1 +$CLICKHOUSE_CLIENT -nq " + select '<14: waiting for next cycle>', view, status, remaining_dependencies, next_refresh_time from refreshes; + truncate src; + insert into src values (3); + system test view a set fake time '2060-02-02 02:02:02';" +while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2062-01-01 00:00:00' ] +do + sleep 0.1 +done +$CLICKHOUSE_CLIENT -nq " + select '<15: chain-refreshed a>', * from a; + select '<16: chain-refreshed b>', * from b; + select '<17: chain-refreshed>', view, status, next_refresh_time from refreshes;" + +# Get to WaitingForDependencies state and remove the depencency. +$CLICKHOUSE_CLIENT -nq " + system test view b set fake time '2062-03-03 03:03:03'" +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies' ] +do + sleep 0.1 +done +$CLICKHOUSE_CLIENT -nq " + alter table b modify refresh every 2 year" +while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled 2062-03-03 03:03:03' ] +do + sleep 0.1 +done +$CLICKHOUSE_CLIENT -nq " + select '<18: removed dependency>', view, status, remaining_dependencies, last_refresh_time,next_refresh_time, refresh_count from refreshes where view = 'b'; + show create b;" + +# Select from a table that doesn't exist, get an exception. +$CLICKHOUSE_CLIENT -nq " + drop table a; + drop table b; + create materialized view c refresh every 1 second (x Int64) engine Memory empty as select * from src; + drop table src;" +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Exception' ] +do + sleep 0.1 +done +# Check exception, create src, expect successful refresh. +$CLICKHOUSE_CLIENT -nq " + select '<19: exception>', exception ilike '%UNKNOWN_TABLE%' from refreshes; + create table src (x Int64) engine Memory as select 1; + system refresh view c;" +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] +do + sleep 0.1 +done +# Rename table. +$CLICKHOUSE_CLIENT -nq " + select '<20: unexception>', * from c; + rename table c to d; + select '<21: rename>', * from d; + select '<22: rename>', view, last_refresh_result from refreshes;" + +# Do various things during a refresh. +# First make a nonempty view. +$CLICKHOUSE_CLIENT -nq " + drop table d; + truncate src; + insert into src values (1) + create materialized view e refresh every 1 second (x Int64) engine MergeTree order by x empty as select x + sleepEachRow(1) as x from src settings max_block_size = 1;" +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] +do + sleep 0.1 +done +# Stop refreshes. +$CLICKHOUSE_CLIENT -nq " + select '<23: simple refresh>', * from e; + system stop view e;" +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Disabled' ] +do + sleep 0.1 +done +# Make refreshes slow, wait for a slow refresh to start. (We stopped refreshes first to make sure +# we wait for a slow refresh, not a previous fast one.) +$CLICKHOUSE_CLIENT -nq " + insert into src select * from numbers(1000) settings max_block_size=1; + system start view e;" +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Running' ] +do + sleep 0.1 +done +# Rename. +$CLICKHOUSE_CLIENT -nq " + rename table e to f; + select '<24: rename during refresh>', * from f; + select '<25: rename during refresh>', view, status from refreshes; + alter table f modify refresh after 10 year;" +sleep 2 # make it likely that at least one row was processed +# Cancel. +$CLICKHOUSE_CLIENT -nq " + system cancel view f;" +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Cancelled' ] +do + sleep 0.1 +done +# Check that another refresh doesn't immediately start after the cancelled one. +sleep 1 +$CLICKHOUSE_CLIENT -nq " + select '<27: cancelled>', view, status from refreshes; + system refresh view f;" +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Running' ] +do + sleep 0.1 +done +# Drop. +$CLICKHOUSE_CLIENT -nq " + drop table f; + select '<28: drop during refresh>', view, status from refreshes;" + +# Try OFFSET and RANDOMIZE FOR. +$CLICKHOUSE_CLIENT -nq " + create materialized view g refresh every 1 week offset 3 day 4 hour randomize for 4 day 1 hour (x Int64) engine Memory empty as select 42; + show create g; + system test view g set fake time '2050-02-03 15:30:13';" +while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time > '2049-01-01' from refreshes -- $LINENO" | xargs`" != '1' ] +do + sleep 0.1 +done +$CLICKHOUSE_CLIENT -nq " + with '2050-02-10 04:00:00'::DateTime as expected + select '<29: randomize>', abs(next_refresh_time::Int64 - expected::Int64) <= 3600*(24*4+1), next_refresh_time != expected from refreshes;" + +# Send data 'TO' an existing table. +$CLICKHOUSE_CLIENT -nq " + drop table g; + create table dest (x Int64) engine MergeTree order by x; + truncate src; + insert into src values (1); + create materialized view h refresh every 1 second to dest empty as select x*10 as x from src; + show create h;" +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] +do + sleep 0.1 +done +$CLICKHOUSE_CLIENT -nq " + select '<30: to existing table>', * from dest; + insert into src values (2);" +while [ "`$CLICKHOUSE_CLIENT -nq "select count() from dest -- $LINENO" | xargs`" != '2' ] +do + sleep 0.1 +done +$CLICKHOUSE_CLIENT -nq " + select '<31: to existing table>', * from dest; + drop table dest; + drop table src; + drop table h;" + +# EMPTY +$CLICKHOUSE_CLIENT -nq " + create materialized view i refresh after 1 year engine Memory empty as select number as x from numbers(2); + create materialized view j refresh after 1 year engine Memory as select number as x from numbers(2)" +while [ "`$CLICKHOUSE_CLIENT -nq "select sum(last_success_time is null) from refreshes -- $LINENO" | xargs`" == '2' ] +do + sleep 0.1 +done +$CLICKHOUSE_CLIENT -nq " + select '<32: empty>', view, status, last_refresh_result from refreshes order by view; + drop table i; + drop table j" + +$CLICKHOUSE_CLIENT -nq " + drop table refreshes;" diff --git a/tests/queries/0_stateless/02933_change_cache_setting_without_restart.reference b/tests/queries/0_stateless/02933_change_cache_setting_without_restart.reference index d4dd4da0c5d..17a25d82824 100644 --- a/tests/queries/0_stateless/02933_change_cache_setting_without_restart.reference +++ b/tests/queries/0_stateless/02933_change_cache_setting_without_restart.reference @@ -1,7 +1,7 @@ -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 0 0 0 1 -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 10 1000 0 1 -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 5 1000 0 1 -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 15 1000 0 1 -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 2 1000 0 1 -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 0 1000 0 1 -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 0 0 0 1 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 0 0 0 16 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 10 1000 0 16 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 5 1000 0 16 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 15 1000 0 16 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 2 1000 0 16 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 0 1000 0 16 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 0 0 0 16 diff --git a/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.reference b/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.reference index aac3e471264..aac8c4f777e 100644 --- a/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.reference +++ b/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.reference @@ -1 +1 @@ -data Array(Tuple(Nullable(Int64), Tuple(a Nullable(Int64), b Nullable(Int64)), Nullable(Int64), Nullable(String))) +data Array(Tuple(Nullable(Int64), Tuple(\n a Nullable(Int64),\n b Nullable(Int64)), Nullable(Int64), Nullable(String))) diff --git a/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.reference b/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh b/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh new file mode 100755 index 00000000000..431f59d7918 --- /dev/null +++ b/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh @@ -0,0 +1,98 @@ +#!/usr/bin/env bash +# Tags: no-parallel +# Tag no-parallel: failpoint is in use + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +set -e + +function wait_part() +{ + local table=$1 && shift + local part=$1 && shift + + for ((i = 0; i < 100; ++i)); do + if [[ $($CLICKHOUSE_CLIENT -q "select count() from system.parts where database = '$CLICKHOUSE_DATABASE' and table = '$table' and active and name = '$part'") -eq 1 ]]; then + return + fi + sleep 0.1 + done + + echo "Part $table::$part does not appeared" >&2 +} + +function restore_failpoints() +{ + # restore entry error with failpoints (to avoid endless errors in logs) + $CLICKHOUSE_CLIENT -nm -q " + system enable failpoint replicated_queue_unfail_entries; + system sync replica $failed_replica; + system disable failpoint replicated_queue_unfail_entries; + " +} +trap restore_failpoints EXIT + +$CLICKHOUSE_CLIENT -nm --insert_keeper_fault_injection_probability=0 -q " + drop table if exists data_r1; + drop table if exists data_r2; + + create table data_r1 (key Int, value Int, index value_idx value type minmax) engine=ReplicatedMergeTree('/clickhouse/tables/{database}/data', '{table}') order by key; + create table data_r2 (key Int, value Int, index value_idx value type minmax) engine=ReplicatedMergeTree('/clickhouse/tables/{database}/data', '{table}') order by key; + + insert into data_r1 (key) values (1); -- part all_0_0_0 +" + +# will fail ALTER_METADATA on one of replicas +$CLICKHOUSE_CLIENT -nm -q " + system enable failpoint replicated_queue_fail_next_entry; + alter table data_r1 drop index value_idx settings alter_sync=0; -- part all_0_0_0_1 + + system sync replica data_r1 pull; + system sync replica data_r2 pull; +" + +# replica on which ALTER_METADATA had been succeed +success_replica= +for ((i = 0; i < 100; ++i)); do + for table in data_r1 data_r2; do + mutations="$($CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = '$CLICKHOUSE_DATABASE' and table = '$table' and is_done = 0")" + if [[ $mutations -eq 0 ]]; then + success_replica=$table + fi + done + if [[ -n $success_replica ]]; then + break + fi + sleep 0.1 +done +case "$success_replica" in + data_r1) failed_replica=data_r2;; + data_r2) failed_replica=data_r1;; + *) echo "ALTER_METADATA does not succeed on any replica" >&2 && exit 1;; +esac +mutations_on_failed_replica="$($CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = '$CLICKHOUSE_DATABASE' and table = '$failed_replica' and is_done = 0")" +if [[ $mutations_on_failed_replica != 1 ]]; then + echo "Wrong number of mutations on failed replica $failed_replica, mutations $mutations_on_failed_replica" >&2 +fi + +# This will create MERGE_PARTS, on failed replica it will be fetched from source replica (since it does not have all parts to execute merge) +$CLICKHOUSE_CLIENT -q "optimize table $success_replica final settings optimize_throw_if_noop=1, alter_sync=1" # part all_0_0_1_1 + +$CLICKHOUSE_CLIENT -nm --insert_keeper_fault_injection_probability=0 -q " + insert into $success_replica (key) values (2); -- part all_2_2_0 + optimize table $success_replica final settings optimize_throw_if_noop=1, alter_sync=1; -- part all_0_2_2_1 + system sync replica $failed_replica pull; +" + +# Wait for part to be merged on failed replica, that will trigger CHECKSUM_DOESNT_MATCH +wait_part "$failed_replica" all_0_2_2_1 + +# Already after part fetched there will CHECKSUM_DOESNT_MATCH in case of ALTER_METADATA re-order, but let's restore fail points and sync failed replica first. +restore_failpoints +trap '' EXIT + +$CLICKHOUSE_CLIENT -q "system flush logs" +# check for error "Different number of files: 5 compressed (expected 3) and 2 uncompressed ones (expected 2). (CHECKSUM_DOESNT_MATCH)" +$CLICKHOUSE_CLIENT -q "select part_name, merge_reason, event_type, errorCodeToName(error) from system.part_log where database = '$CLICKHOUSE_DATABASE' and error != 0 order by event_time_microseconds" diff --git a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference new file mode 100644 index 00000000000..1cf1644fe0a --- /dev/null +++ b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference @@ -0,0 +1,38 @@ +1 Hello ClickHouse +2 Hello World +1 Hello ClickHouse +2 Hello World + Granules: 6/6 + Granules: 2/6 + Granules: 6/6 + Granules: 2/6 + Granules: 6/6 + Granules: 2/6 + Granules: 6/6 + Granules: 2/6 +--- +1 Hello ClickHouse +2 Hello World +6 World Champion +1 Hello ClickHouse +2 Hello World +6 World Champion + Granules: 6/6 + Granules: 3/6 + Granules: 6/6 + Granules: 3/6 + Granules: 6/6 + Granules: 3/6 + Granules: 6/6 + Granules: 3/6 +--- +5 OLAP Database +5 OLAP Database + Granules: 6/6 + Granules: 1/6 + Granules: 6/6 + Granules: 1/6 + Granules: 6/6 + Granules: 1/6 + Granules: 6/6 + Granules: 1/6 diff --git a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql new file mode 100644 index 00000000000..49d39c601ef --- /dev/null +++ b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql @@ -0,0 +1,185 @@ +DROP TABLE IF EXISTS tokenbf_tab; +DROP TABLE IF EXISTS ngrambf_tab; + +CREATE TABLE tokenbf_tab +( + id UInt32, + str String, + INDEX idx str TYPE tokenbf_v1(256, 2, 0) +) +ENGINE = MergeTree +ORDER BY id +SETTINGS index_granularity = 1; + +CREATE TABLE ngrambf_tab +( + id UInt32, + str String, + INDEX idx str TYPE ngrambf_v1(3, 256, 2, 0) +) +ENGINE = MergeTree +ORDER BY id +SETTINGS index_granularity = 1; + +INSERT INTO tokenbf_tab VALUES (1, 'Hello ClickHouse'), (2, 'Hello World'), (3, 'Good Weather'), (4, 'Say Hello'), (5, 'OLAP Database'), (6, 'World Champion'); +INSERT INTO ngrambf_tab VALUES (1, 'Hello ClickHouse'), (2, 'Hello World'), (3, 'Good Weather'), (4, 'Say Hello'), (5, 'OLAP Database'), (6, 'World Champion'); + +SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id; +SELECT * FROM ngrambf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id; + +-- Read 2/6 granules +-- Required string: 'Hello ' +-- Alternatives: 'Hello ClickHouse', 'Hello World' + +SELECT * +FROM +( + EXPLAIN PLAN indexes=1 + SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 0; + +SELECT * +FROM +( + EXPLAIN PLAN indexes=1 + SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 1; + +SELECT * +FROM +( + EXPLAIN PLAN indexes=1 + SELECT * FROM ngrambf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 0; + +SELECT * +FROM +( + EXPLAIN PLAN indexes=1 + SELECT * FROM ngrambf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 1; + + +SELECT '---'; + +SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id; +SELECT * FROM ngrambf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id; + +-- Read 3/6 granules +-- Required string: - +-- Alternatives: 'ClickHouse', 'World' + +SELECT * +FROM +( + EXPLAIN PLAN indexes = 1 + SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 0; + +SELECT * +FROM +( + EXPLAIN PLAN indexes = 1 + SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 1; + +SELECT * +FROM +( + EXPLAIN PLAN indexes = 1 + SELECT * FROM ngrambf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 0; + +SELECT * +FROM +( + EXPLAIN PLAN indexes = 1 + SELECT * FROM ngrambf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 1; + +SELECT '---'; + +SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP.*') ORDER BY id; +SELECT * FROM ngrambf_tab WHERE match(str, 'OLAP.*') ORDER BY id; + +-- Read 1/6 granules +-- Required string: 'OLAP' +-- Alternatives: - + +SELECT * +FROM +( + EXPLAIN PLAN indexes = 1 + SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 0; +SELECT * +FROM +( + EXPLAIN PLAN indexes = 1 + SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 1; + +SELECT * +FROM +( + EXPLAIN PLAN indexes = 1 + SELECT * FROM ngrambf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 0; + +SELECT * +FROM +( + EXPLAIN PLAN indexes = 1 + SELECT * FROM ngrambf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 1; + +DROP TABLE tokenbf_tab; +DROP TABLE ngrambf_tab; diff --git a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference index 8620171cb99..4a6bc8498e1 100644 --- a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference +++ b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference @@ -1,20 +1,20 @@ -100 10 10 10 0 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 1 +100 10 10 10 0 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 16 0 10 98 set max_size from 100 to 10 -10 10 10 10 0 0 8 1 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 1 +10 10 10 10 0 0 8 1 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 16 1 8 set max_size from 10 to 100 -100 10 10 10 0 0 8 1 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 1 +100 10 10 10 0 0 8 1 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 16 10 98 set max_elements from 10 to 2 -100 2 10 10 0 0 18 2 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 1 +100 2 10 10 0 0 18 2 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 16 2 18 set max_elements from 2 to 10 -100 10 10 10 0 0 18 2 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 1 +100 10 10 10 0 0 18 2 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 5 5000 0 16 10 98 diff --git a/tests/queries/0_stateless/02946_merge_tree_final_split_ranges_by_primary_key.reference b/tests/queries/0_stateless/02946_merge_tree_final_split_ranges_by_primary_key.reference new file mode 100644 index 00000000000..59acae1c7ef --- /dev/null +++ b/tests/queries/0_stateless/02946_merge_tree_final_split_ranges_by_primary_key.reference @@ -0,0 +1,85 @@ +1 +-- +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +-- +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +-- +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +-- +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 +21 21 +22 22 +23 23 +24 24 +25 25 +26 26 +27 27 +28 28 +29 29 +30 30 +31 31 diff --git a/tests/queries/0_stateless/02946_merge_tree_final_split_ranges_by_primary_key.sql b/tests/queries/0_stateless/02946_merge_tree_final_split_ranges_by_primary_key.sql new file mode 100644 index 00000000000..780ed5b7984 --- /dev/null +++ b/tests/queries/0_stateless/02946_merge_tree_final_split_ranges_by_primary_key.sql @@ -0,0 +1,34 @@ +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=ReplacingMergeTree ORDER BY id SETTINGS index_granularity = 2; + +INSERT INTO test_table SELECT 0, '0'; +INSERT INTO test_table SELECT number + 1, number + 1 FROM numbers(15); +OPTIMIZE TABLE test_table; + +SELECT COUNT() FROM system.parts WHERE database = currentDatabase() AND table = 'test_table' AND active = 1; +SYSTEM STOP MERGES test_table; + +SELECT '--'; + +SELECT id, value FROM test_table FINAL ORDER BY id; + +SELECT '--'; + +INSERT INTO test_table SELECT 5, '5'; +SELECT id, value FROM test_table FINAL ORDER BY id; + +SELECT '--'; + +INSERT INTO test_table SELECT number + 8, number + 8 FROM numbers(8); +SELECT id, value FROM test_table FINAL ORDER BY id; + +SELECT '--'; + +INSERT INTO test_table SELECT number, number FROM numbers(32); +SELECT id, value FROM test_table FINAL ORDER BY id; + +DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02947_dropped_tables_parts.reference b/tests/queries/0_stateless/02947_dropped_tables_parts.reference new file mode 100644 index 00000000000..086d55c3d93 --- /dev/null +++ b/tests/queries/0_stateless/02947_dropped_tables_parts.reference @@ -0,0 +1,2 @@ +default 02947_table_1 all_1_1_0 +default 02947_table_2 all_1_1_0 diff --git a/tests/queries/0_stateless/02947_dropped_tables_parts.sql b/tests/queries/0_stateless/02947_dropped_tables_parts.sql new file mode 100644 index 00000000000..554a19ca6b1 --- /dev/null +++ b/tests/queries/0_stateless/02947_dropped_tables_parts.sql @@ -0,0 +1,14 @@ + +DROP TABLE IF EXISTS 02947_table_1; +DROP TABLE IF EXISTS 02947_table_2; + +CREATE TABLE 02947_table_1 (id Int32) Engine=MergeTree() ORDER BY id; +CREATE TABLE 02947_table_2 (id Int32) Engine=MergeTree() ORDER BY id; +INSERT INTO 02947_table_1 VALUES (1),(2); +INSERT INTO 02947_table_2 VALUES (3),(4); + +SELECT database, table, name FROM system.parts WHERE database = currentDatabase() AND startsWith(table, '02947_table_'); +select * from system.dropped_tables_parts format Null; + +DROP TABLE 02947_table_1; +DROP TABLE 02947_table_2; diff --git a/tests/queries/0_stateless/02948_optimize_cleanup_as_noop.sql b/tests/queries/0_stateless/02948_optimize_cleanup_as_noop.sql deleted file mode 100644 index 002d696e62f..00000000000 --- a/tests/queries/0_stateless/02948_optimize_cleanup_as_noop.sql +++ /dev/null @@ -1,7 +0,0 @@ -# There was a wrong, harmful feature, leading to bugs and data corruption. -# This feature is removed, but we take care to maintain compatibility on the syntax level, so now it works as a no-op. - -DROP TABLE IF EXISTS t; -CREATE TABLE t (x UInt8, PRIMARY KEY x) ENGINE = ReplacingMergeTree; -OPTIMIZE TABLE t CLEANUP; -DROP TABLE t; diff --git a/tests/queries/0_stateless/02949_ttl_group_by_bug.reference b/tests/queries/0_stateless/02949_ttl_group_by_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02949_ttl_group_by_bug.sql b/tests/queries/0_stateless/02949_ttl_group_by_bug.sql new file mode 100644 index 00000000000..2888f6e7d66 --- /dev/null +++ b/tests/queries/0_stateless/02949_ttl_group_by_bug.sql @@ -0,0 +1,29 @@ +DROP TABLE IF EXISTS ttl_group_by_bug; + +CREATE TABLE ttl_group_by_bug +(key UInt32, ts DateTime, value UInt32, min_value UInt32 default value, max_value UInt32 default value) +ENGINE = MergeTree() PARTITION BY toYYYYMM(ts) +ORDER BY (key, toStartOfInterval(ts, toIntervalMinute(3)), ts) +TTL ts + INTERVAL 5 MINUTE GROUP BY key, toStartOfInterval(ts, toIntervalMinute(3)) +SET value = sum(value), min_value = min(min_value), max_value = max(max_value), ts=min(toStartOfInterval(ts, toIntervalMinute(3))); + +INSERT INTO ttl_group_by_bug(key, ts, value) SELECT number%5 as key, now() - interval 10 minute + number, 0 FROM numbers(1000); + +OPTIMIZE TABLE ttl_group_by_bug FINAL; + +SELECT * +FROM +( + SELECT + _part, + rowNumberInAllBlocks(), + (key, toStartOfInterval(ts, toIntervalMinute(3)), ts) AS cur, + lagInFrame((key, toStartOfInterval(ts, toIntervalMinute(3)), ts), 1) OVER () AS prev, + 1 + FROM ttl_group_by_bug +) +WHERE cur < prev +LIMIT 2 +SETTINGS max_threads = 1; + +DROP TABLE IF EXISTS ttl_group_by_bug; diff --git a/tests/queries/0_stateless/02950_part_offset_as_primary_key.reference b/tests/queries/0_stateless/02950_part_offset_as_primary_key.reference new file mode 100644 index 00000000000..368f8dd9871 --- /dev/null +++ b/tests/queries/0_stateless/02950_part_offset_as_primary_key.reference @@ -0,0 +1,14 @@ +-4 +-3 +-2 +-1 +0 +-3 +0 +-4 +-2 +-1 +0 +10 +40 +400 diff --git a/tests/queries/0_stateless/02950_part_offset_as_primary_key.sql b/tests/queries/0_stateless/02950_part_offset_as_primary_key.sql new file mode 100644 index 00000000000..736d54023ce --- /dev/null +++ b/tests/queries/0_stateless/02950_part_offset_as_primary_key.sql @@ -0,0 +1,40 @@ +drop table if exists a; + +create table a (i int) engine MergeTree order by i settings index_granularity = 2; +insert into a select -number from numbers(5); + +-- nothing to read +select i from a where _part_offset >= 5 order by i settings max_bytes_to_read = 1; + +-- one granule +select i from a where _part_offset = 0 order by i settings max_rows_to_read = 2; +select i from a where _part_offset = 1 order by i settings max_rows_to_read = 2; +select i from a where _part_offset = 2 order by i settings max_rows_to_read = 2; +select i from a where _part_offset = 3 order by i settings max_rows_to_read = 2; +select i from a where _part_offset = 4 order by i settings max_rows_to_read = 1; + +-- other predicates +select i from a where _part_offset in (1, 4) order by i settings max_rows_to_read = 3; +select i from a where _part_offset not in (1, 4) order by i settings max_rows_to_read = 4; + +-- the force_primary_key check still works +select i from a where _part_offset = 4 order by i settings force_primary_key = 1; -- { serverError INDEX_NOT_USED } + +-- combining with other primary keys doesn't work (makes no sense) +select i from a where i = -3 or _part_offset = 4 order by i settings force_primary_key = 1; -- { serverError INDEX_NOT_USED } + +drop table a; + +drop table if exists b; + +create table b (i int) engine MergeTree order by tuple() settings index_granularity = 2; + +-- all_1_1_0 +insert into b select number * 10 from numbers(5); +-- all_2_2_0 +insert into b select number * 100 from numbers(5); + +-- multiple parts with _part predicate +select i from b where (_part = 'all_1_1_0' and _part_offset in (1, 4)) or (_part = 'all_2_2_0' and _part_offset in (0, 4)) order by i settings max_rows_to_read = 6; + +drop table b; diff --git a/tests/queries/0_stateless/02950_reading_array_tuple_subcolumns.reference b/tests/queries/0_stateless/02950_reading_array_tuple_subcolumns.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02950_reading_array_tuple_subcolumns.sql b/tests/queries/0_stateless/02950_reading_array_tuple_subcolumns.sql new file mode 100644 index 00000000000..85bf16a885b --- /dev/null +++ b/tests/queries/0_stateless/02950_reading_array_tuple_subcolumns.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test +( + `id` UInt64, + `t` Tuple(a UInt64, b Array(Tuple(c UInt64, d UInt64))) +) +ENGINE = MergeTree +ORDER BY id +SETTINGS min_rows_for_wide_part = 1, min_bytes_for_wide_part = 1, index_granularity = 8192; +INSERT INTO test SELECT number, tuple(number, arrayMap(x -> tuple(number + 1, number + 2), range(number % 10))) FROM numbers(100000); +INSERT INTO test SELECT number, tuple(number, arrayMap(x -> tuple(number + 1, number + 2), range(number % 10))) FROM numbers(100000); +INSERT INTO test SELECT number, tuple(number, arrayMap(x -> tuple(number + 1, number + 2), range(number % 10))) FROM numbers(100000); +SELECT t.b, t.b.c FROM test ORDER BY id FORMAT Null; +DROP TABLE test; + diff --git a/tests/queries/0_stateless/02952_conjunction_optimization.reference b/tests/queries/0_stateless/02952_conjunction_optimization.reference new file mode 100644 index 00000000000..eeadfaae21d --- /dev/null +++ b/tests/queries/0_stateless/02952_conjunction_optimization.reference @@ -0,0 +1,117 @@ +3 another +3 +QUERY id: 0 + PROJECTION COLUMNS + a Int32 + b String + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 + COLUMN id: 4, column_name: b, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization + WHERE + FUNCTION id: 5, function_name: notIn, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: a, result_type: Int32, source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_4), constant_value_type: Tuple(UInt8, UInt8, UInt8) +3 another +3 +QUERY id: 0 + PROJECTION COLUMNS + a Int32 + b String + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 + COLUMN id: 4, column_name: b, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization + WHERE + FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: Bool + ARGUMENTS + LIST id: 6, nodes: 2 + CONSTANT id: 7, constant_value: UInt64_1, constant_value_type: Bool + FUNCTION id: 8, function_name: notIn, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: a, result_type: Int32, source_id: 3 + CONSTANT id: 11, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_4), constant_value_type: Tuple(UInt8, UInt8, UInt8) +3 another +QUERY id: 0 + PROJECTION COLUMNS + a Int32 + b String + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 + COLUMN id: 4, column_name: b, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization + WHERE + FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + FUNCTION id: 7, function_name: notEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 8, nodes: 2 + COLUMN id: 9, column_name: b, result_type: String, source_id: 3 + CONSTANT id: 10, constant_value: \'\', constant_value_type: String + FUNCTION id: 11, function_name: notIn, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: a, result_type: Int32, source_id: 3 + CONSTANT id: 14, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_4), constant_value_type: Tuple(UInt8, UInt8, UInt8) +3 +QUERY id: 0 + PROJECTION COLUMNS + a Int32 + b String + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 + COLUMN id: 4, column_name: b, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization + WHERE + FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + FUNCTION id: 7, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 8, nodes: 2 + COLUMN id: 9, column_name: b, result_type: String, source_id: 3 + CONSTANT id: 10, constant_value: \'\', constant_value_type: String + FUNCTION id: 11, function_name: notIn, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: a, result_type: Int32, source_id: 3 + CONSTANT id: 14, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_4), constant_value_type: Tuple(UInt8, UInt8, UInt8) +3 another +3 +4 +QUERY id: 0 + PROJECTION COLUMNS + a Int32 + b String + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 + COLUMN id: 4, column_name: b, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization + WHERE + FUNCTION id: 5, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + FUNCTION id: 7, function_name: notIn, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 8, nodes: 2 + COLUMN id: 9, column_name: a, result_type: Int32, source_id: 3 + CONSTANT id: 10, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_4), constant_value_type: Tuple(UInt8, UInt8, UInt8) + FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: b, result_type: String, source_id: 3 + CONSTANT id: 14, constant_value: \'\', constant_value_type: String diff --git a/tests/queries/0_stateless/02952_conjunction_optimization.sql b/tests/queries/0_stateless/02952_conjunction_optimization.sql new file mode 100644 index 00000000000..94bc352e4c5 --- /dev/null +++ b/tests/queries/0_stateless/02952_conjunction_optimization.sql @@ -0,0 +1,26 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS 02952_disjunction_optimization; + +CREATE TABLE 02952_disjunction_optimization +(a Int32, b String) +ENGINE=Memory; + +INSERT INTO 02952_disjunction_optimization VALUES (1, 'test'), (2, 'test2'), (3, 'another'), (3, ''), (4, ''); + +SELECT * FROM 02952_disjunction_optimization WHERE a <> 1 AND a <> 2 AND a <> 4; +EXPLAIN QUERY TREE SELECT * FROM 02952_disjunction_optimization WHERE a <> 1 AND a <> 2 AND a <> 4; + +SELECT * FROM 02952_disjunction_optimization WHERE a <> 1 AND a <> 2 AND a <> 4 AND true; +EXPLAIN QUERY TREE SELECT * FROM 02952_disjunction_optimization WHERE a <> 1 AND a <> 2 AND a <> 4 AND true; + +SELECT * FROM 02952_disjunction_optimization WHERE a <> 1 AND a <> 2 AND a <> 4 AND b <> ''; +EXPLAIN QUERY TREE SELECT * FROM 02952_disjunction_optimization WHERE a <> 1 AND a <> 2 AND a <> 4 AND b <> ''; + +SELECT * FROM 02952_disjunction_optimization WHERE a <> 1 AND a <> 2 AND b = '' AND a <> 4; +EXPLAIN QUERY TREE SELECT * FROM 02952_disjunction_optimization WHERE a <> 1 AND a <> 2 AND b = '' AND a <> 4; + +SELECT * FROM 02952_disjunction_optimization WHERE (a <> 1 AND a <> 2 AND a <> 4) OR b = ''; +EXPLAIN QUERY TREE SELECT * FROM 02952_disjunction_optimization WHERE (a <> 1 AND a <> 2 AND a <> 4) OR b = ''; + +DROP TABLE 02952_disjunction_optimization; diff --git a/tests/queries/0_stateless/02953_slow_create_view.reference b/tests/queries/0_stateless/02953_slow_create_view.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02953_slow_create_view.sql b/tests/queries/0_stateless/02953_slow_create_view.sql new file mode 100644 index 00000000000..7824bd97b92 --- /dev/null +++ b/tests/queries/0_stateless/02953_slow_create_view.sql @@ -0,0 +1,44 @@ +drop view if exists slow_view1; + +create view slow_view1 as +with c1 as (select 1 as a), + c2 as (select a from c1), + c3 as (select a from c2), + c4 as (select a from c3), + c5 as (select a from c4), + c6 as (select a from c5), + c7 as (select a from c6), + c8 as (select a from c7), + c9 as (select a from c8), + c10 as (select a from c9), + c11 as (select a from c10), + c12 as (select a from c11), + c13 as (select a from c12), + c14 as (select a from c13), + c15 as (select a from c14), + c16 as (select a from c15), + c17 as (select a from c16), + c18 as (select a from c17), + c19 as (select a from c18), + c20 as (select a from c19), + c21 as (select a from c20), + c22 as (select a from c21), + c23 as (select a from c22), + c24 as (select a from c23), + c25 as (select a from c24), + c26 as (select a from c25), + c27 as (select a from c26), + c28 as (select a from c27), + c29 as (select a from c28), + c30 as (select a from c29), + c31 as (select a from c30), + c32 as (select a from c31), + c33 as (select a from c32), + c34 as (select a from c33), + c35 as (select a from c34), + c36 as (select a from c35), + c37 as (select a from c36), + c38 as (select a from c37), + c39 as (select a from c38), + c40 as (select a from c39) +select a from c21; diff --git a/tests/queries/0_stateless/02954_analyzer_fuzz_i57086.reference b/tests/queries/0_stateless/02954_analyzer_fuzz_i57086.reference new file mode 100644 index 00000000000..f2386499865 --- /dev/null +++ b/tests/queries/0_stateless/02954_analyzer_fuzz_i57086.reference @@ -0,0 +1,2 @@ +limit w/ GROUP BY 0 0 +limit w/ GROUP BY 0 0 diff --git a/tests/queries/0_stateless/02954_analyzer_fuzz_i57086.sql b/tests/queries/0_stateless/02954_analyzer_fuzz_i57086.sql new file mode 100644 index 00000000000..a8029fdd3d6 --- /dev/null +++ b/tests/queries/0_stateless/02954_analyzer_fuzz_i57086.sql @@ -0,0 +1,15 @@ +--https://github.com/ClickHouse/ClickHouse/issues/57086 +SELECT + 'limit w/ GROUP BY', + count(NULL), + number +FROM remote('127.{1,2}', view( + SELECT intDiv(number, 2147483647) AS number + FROM numbers(10) + )) +GROUP BY number +WITH ROLLUP +ORDER BY + count() ASC, + number DESC NULLS LAST + SETTINGS limit = 2, allow_experimental_analyzer = 1; diff --git a/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference b/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql b/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql new file mode 100644 index 00000000000..7983b43d7e5 --- /dev/null +++ b/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql @@ -0,0 +1,12 @@ +CREATE TABLE t1 (x Int16, y ALIAS x + x * 2) ENGINE=MergeTree() ORDER BY x; +CREATE TABLE t2 (y Int16, z Int16) ENGINE=MergeTree() ORDER BY y; + +INSERT INTO t1 VALUES (1231), (123); +INSERT INTO t2 VALUES (6666, 48); +INSERT INTO t2 VALUES (369, 50); + +SELECT count() FROM t1 INNER JOIN t2 USING (y); +SELECT count() FROM t2 INNER JOIN t1 USING (y); + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/02955_sparkBar_alias_sparkbar.reference b/tests/queries/0_stateless/02955_sparkBar_alias_sparkbar.reference new file mode 100644 index 00000000000..4600566772a --- /dev/null +++ b/tests/queries/0_stateless/02955_sparkBar_alias_sparkbar.reference @@ -0,0 +1,2 @@ +▂▅▂▃▆█ ▂ +▂▅▂▃▆█ ▂ diff --git a/tests/queries/0_stateless/02955_sparkBar_alias_sparkbar.sql b/tests/queries/0_stateless/02955_sparkBar_alias_sparkbar.sql new file mode 100644 index 00000000000..98259fc8029 --- /dev/null +++ b/tests/queries/0_stateless/02955_sparkBar_alias_sparkbar.sql @@ -0,0 +1,12 @@ +SET allow_experimental_analyzer = 1; +DROP TABLE IF EXISTS spark_bar_test; + +CREATE TABLE spark_bar_test (`value` Int64, `event_date` Date) ENGINE = MergeTree ORDER BY event_date; + +INSERT INTO spark_bar_test 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_test GROUP BY event_date); +SELECT sparkBar(9)(event_date,cnt) FROM (SELECT sum(value) as cnt, event_date FROM spark_bar_test GROUP BY event_date); + +DROP TABLE IF EXISTS spark_bar_test; + diff --git a/tests/queries/0_stateless/02956_clickhouse_local_system_parts.reference b/tests/queries/0_stateless/02956_clickhouse_local_system_parts.reference new file mode 100644 index 00000000000..30365d83930 --- /dev/null +++ b/tests/queries/0_stateless/02956_clickhouse_local_system_parts.reference @@ -0,0 +1 @@ +test all_1_1_0 1 diff --git a/tests/queries/0_stateless/02956_clickhouse_local_system_parts.sh b/tests/queries/0_stateless/02956_clickhouse_local_system_parts.sh new file mode 100755 index 00000000000..e9d8eb081fb --- /dev/null +++ b/tests/queries/0_stateless/02956_clickhouse_local_system_parts.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL --multiquery "CREATE TABLE test (x UInt8) ENGINE = MergeTree ORDER BY (); INSERT INTO test SELECT 1; SELECT table, name, rows FROM system.parts WHERE database = currentDatabase();" diff --git a/tests/queries/0_stateless/02956_format_constexpr.reference b/tests/queries/0_stateless/02956_format_constexpr.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02956_format_constexpr.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02956_format_constexpr.sql b/tests/queries/0_stateless/02956_format_constexpr.sql new file mode 100644 index 00000000000..32c61436306 --- /dev/null +++ b/tests/queries/0_stateless/02956_format_constexpr.sql @@ -0,0 +1 @@ +SELECT isConstant(format('{}, world', 'Hello')); diff --git a/tests/queries/0_stateless/02958_transform_enum.reference b/tests/queries/0_stateless/02958_transform_enum.reference new file mode 100644 index 00000000000..4c1476a8922 --- /dev/null +++ b/tests/queries/0_stateless/02958_transform_enum.reference @@ -0,0 +1,4 @@ +Hello 123 +world 456 +Hello test +world best diff --git a/tests/queries/0_stateless/02958_transform_enum.sql b/tests/queries/0_stateless/02958_transform_enum.sql new file mode 100644 index 00000000000..3b0fd40a282 --- /dev/null +++ b/tests/queries/0_stateless/02958_transform_enum.sql @@ -0,0 +1,3 @@ +WITH arrayJoin(['Hello', 'world'])::Enum('Hello', 'world') AS x SELECT x, transform(x, ['Hello', 'world'], [123, 456], 0); +WITH arrayJoin(['Hello', 'world'])::Enum('Hello', 'world') AS x SELECT x, transform(x, ['Hello', 'world', 'goodbye'], [123, 456], 0); -- { serverError UNKNOWN_ELEMENT_OF_ENUM } +WITH arrayJoin(['Hello', 'world'])::Enum('Hello', 'world') AS x SELECT x, transform(x, ['Hello', 'world'], ['test', 'best']::Array(Enum('test' = 123, 'best' = 456, '' = 0)), ''::Enum('test' = 123, 'best' = 456, '' = 0)) AS y; diff --git a/tests/queries/0_stateless/02959_system_database_engines.reference b/tests/queries/0_stateless/02959_system_database_engines.reference new file mode 100644 index 00000000000..c3cc6fe7c9d --- /dev/null +++ b/tests/queries/0_stateless/02959_system_database_engines.reference @@ -0,0 +1,3 @@ +Atomic +Lazy +Ordinary diff --git a/tests/queries/0_stateless/02959_system_database_engines.sql b/tests/queries/0_stateless/02959_system_database_engines.sql new file mode 100644 index 00000000000..67cb20f0400 --- /dev/null +++ b/tests/queries/0_stateless/02959_system_database_engines.sql @@ -0,0 +1 @@ +SELECT * FROM system.database_engines WHERE name IN ('Atomic', 'Lazy', 'Ordinary') ORDER BY name; diff --git a/tests/queries/0_stateless/02960_partition_by_udf.reference b/tests/queries/0_stateless/02960_partition_by_udf.reference new file mode 100644 index 00000000000..f599e28b8ab --- /dev/null +++ b/tests/queries/0_stateless/02960_partition_by_udf.reference @@ -0,0 +1 @@ +10 diff --git a/tests/queries/0_stateless/02960_partition_by_udf.sql b/tests/queries/0_stateless/02960_partition_by_udf.sql new file mode 100644 index 00000000000..3a5b7491694 --- /dev/null +++ b/tests/queries/0_stateless/02960_partition_by_udf.sql @@ -0,0 +1,19 @@ +-- Tags: no-parallel + +DROP FUNCTION IF EXISTS f1; +CREATE FUNCTION f1 AS (x) -> x; + +CREATE TABLE hit +( + `UserID` UInt32, + `URL` String, + `EventTime` DateTime +) +ENGINE = MergeTree +partition by f1(URL) +ORDER BY (EventTime); + +INSERT INTO hit SELECT * FROM generateRandom() LIMIT 10; +SELECT count() FROM hit; + +DROP TABLE hit; diff --git a/tests/queries/0_stateless/02960_validate_database_engines.reference b/tests/queries/0_stateless/02960_validate_database_engines.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02960_validate_database_engines.sql b/tests/queries/0_stateless/02960_validate_database_engines.sql new file mode 100644 index 00000000000..5d39a76867c --- /dev/null +++ b/tests/queries/0_stateless/02960_validate_database_engines.sql @@ -0,0 +1,14 @@ +-- Tags: no-parallel + +DROP DATABASE IF EXISTS test2960_valid_database_engine; + +-- create database with valid engine. Should succeed. +CREATE DATABASE test2960_valid_database_engine ENGINE = Atomic; + +-- create database with valid engine but arguments are not allowed. Should fail. +CREATE DATABASE test2960_database_engine_args_not_allowed ENGINE = Atomic('foo', 'bar'); -- { serverError BAD_ARGUMENTS } + +-- create database with an invalid engine. Should fail. +CREATE DATABASE test2960_invalid_database_engine ENGINE = Foo; -- { serverError UNKNOWN_DATABASE_ENGINE } + +DROP DATABASE IF EXISTS test2960_valid_database_engine; diff --git a/tests/queries/0_stateless/02961_read_bool_as_string_json.reference b/tests/queries/0_stateless/02961_read_bool_as_string_json.reference new file mode 100644 index 00000000000..56f15989a45 --- /dev/null +++ b/tests/queries/0_stateless/02961_read_bool_as_string_json.reference @@ -0,0 +1,12 @@ +true +false +str +true +false +str +['true','false'] +['false','true'] +['str1','str2'] +['true','false'] +['false','true'] +['str1','str2'] diff --git a/tests/queries/0_stateless/02961_read_bool_as_string_json.sql b/tests/queries/0_stateless/02961_read_bool_as_string_json.sql new file mode 100644 index 00000000000..b9f4a7926f9 --- /dev/null +++ b/tests/queries/0_stateless/02961_read_bool_as_string_json.sql @@ -0,0 +1,9 @@ +set input_format_json_read_bools_as_strings=1; +select * from format(JSONEachRow, 'x String', '{"x" : true}, {"x" : false}, {"x" : "str"}'); +select * from format(JSONEachRow, '{"x" : true}, {"x" : false}, {"x" : "str"}'); +select * from format(JSONEachRow, 'x String', '{"x" : tru}'); -- {serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED} +select * from format(JSONEachRow, 'x String', '{"x" : fals}'); -- {serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED} +select * from format(JSONEachRow, 'x String', '{"x" : atru}'); -- {serverError INCORRECT_DATA} +select * from format(JSONEachRow, 'x Array(String)', '{"x" : [true, false]}, {"x" : [false, true]}, {"x" : ["str1", "str2"]}'); +select * from format(JSONEachRow, '{"x" : [true, false]}, {"x" : [false, true]}, {"x" : ["str1", "str2"]}'); + diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index b1885654d1a..143332d9974 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -60,7 +60,7 @@ function check_replication_consistency() echo "==================== STACK TRACES ====================" $CLICKHOUSE_CLIENT -q "SELECT query_id, thread_name, thread_id, arrayStringConcat(arrayMap(x -> demangle(addressToSymbol(x)), trace), '\n') FROM system.stack_trace where query_id IN (SELECT query_id FROM system.processes WHERE current_database=currentDatabase() AND query LIKE '%$table_name_prefix%') SETTINGS allow_introspection_functions=1 FORMAT Vertical" echo "==================== MUTATIONS ====================" - $CLICKHOUSE_CLIENT -q "SELECT * FROM system.mutations WHERE current_database=currentDatabase() FORMAT Vertical" + $CLICKHOUSE_CLIENT -q "SELECT * FROM system.mutations WHERE database=currentDatabase() FORMAT Vertical" break fi done diff --git a/tests/queries/1_stateful/00172_hits_joins.sql.j2 b/tests/queries/1_stateful/00172_hits_joins.sql.j2 index 4617fe5aef8..e891f1ba3c3 100644 --- a/tests/queries/1_stateful/00172_hits_joins.sql.j2 +++ b/tests/queries/1_stateful/00172_hits_joins.sql.j2 @@ -4,6 +4,9 @@ SET max_rows_in_join = '{% if join_algorithm == 'grace_hash' %}10K{% else %}0{% endif %}'; SET grace_hash_join_initial_buckets = 4; +-- Test is slow with external sort / group by +SET max_bytes_before_external_sort = 0, max_bytes_before_external_group_by = 0; + SELECT '--- {{ join_algorithm }} ---'; SET join_algorithm = '{{ join_algorithm }}'; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 35afb8185fb..dcff1c82444 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 2646 +personal_ws-1.1 en 2657 AArch ACLs ALTERs @@ -344,6 +344,7 @@ Hypot IANA IDE IDEs +IDNA IMDS INFILE INSERTed @@ -752,6 +753,7 @@ Redash Reddit Refactorings ReferenceKeyed +Refreshable RegexpTree RemoteRead ReplacingMergeTree @@ -910,6 +912,7 @@ ThreadsInOvercommitTracker Timeunit TinyLog Tkachenko +ToASCII ToCenterChild ToChildren ToGeo @@ -918,10 +921,13 @@ ToIPv ToParent ToSnowflake ToString +ToUnicode Toolset TopK TotalBytesOfMergeTreeTables TotalPartsOfMergeTreeTables +TotalPrimaryKeyBytesInMemory +TotalPrimaryKeyBytesInMemoryAllocated TotalRowsOfMergeTreeTables TotalTemporaryFiles Tradeoff @@ -1649,6 +1655,8 @@ hyvor icosahedron icudata idempotency +idnaDecode +idnaEncode ifNotFinite ifNull iframe @@ -1846,14 +1854,14 @@ metrica metroHash mfedotov minMap +minSampleSizeContinuous +minSampleSizeConversion mindsdb minimalistic mininum miniselect minmap minmax -minSampleSizeContinuous -minSampleSizeConversion mins misconfiguration mispredictions @@ -2153,6 +2161,7 @@ reddit redis redisstreams refcounter +refreshable regexpExtract regexpQuoteMeta regionHierarchy @@ -2197,8 +2206,6 @@ retentions rethrow retransmit retriable -retuned -reverseDNSQuery reverseUTF rightPad rightPadUTF @@ -2522,6 +2529,8 @@ trimRight trunc tryBase tryDecrypt +tryIdnaEncode +tryPunycodeDecode tskv tsv tui diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 39d371e25d5..88b43afff26 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -429,3 +429,6 @@ join -v1 <(find $ROOT_PATH/{src,programs,utils} -name '*.h' -printf '%f\n' | sor # Don't allow dynamic compiler check with CMake, because we are using hermetic, reproducible, cross-compiled, static (TLDR, good) builds. ls -1d $ROOT_PATH/contrib/*-cmake | xargs -I@ find @ -name 'CMakeLists.txt' -or -name '*.cmake' | xargs grep --with-filename -i -P 'check_c_compiler_flag|check_cxx_compiler_flag|check_c_source_compiles|check_cxx_source_compiles|check_include_file|check_symbol_exists|cmake_push_check_state|cmake_pop_check_state|find_package|CMAKE_REQUIRED_FLAGS|CheckIncludeFile|CheckCCompilerFlag|CheckCXXCompilerFlag|CheckCSourceCompiles|CheckCXXSourceCompiles|CheckCSymbolExists|CheckCXXSymbolExists' | grep -v Rust && echo "^ It's not allowed to have dynamic compiler checks with CMake." + +# DOS/Windows newlines +find $ROOT_PATH/{base,src,programs,utils,docs} -name '*.md' -or -name '*.h' -or -name '*.cpp' -or -name '*.js' -or -name '*.py' -or -name '*.html' | xargs grep -l -P '\r$' && echo "^ Files contain DOS/Windows newlines (\r\n instead of \n)." diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index de3accea617..b2983033e44 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,6 +1,10 @@ +v23.12.2.59-stable 2024-01-05 +v23.12.1.1368-stable 2023-12-28 +v23.11.4.24-stable 2024-01-05 v23.11.3.23-stable 2023-12-21 v23.11.2.11-stable 2023-12-13 v23.11.1.2711-stable 2023-12-06 +v23.10.6.60-stable 2024-01-05 v23.10.5.20-stable 2023-11-25 v23.10.4.25-stable 2023-11-17 v23.10.3.5-stable 2023-11-10 @@ -12,6 +16,7 @@ v23.9.4.11-stable 2023-11-08 v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.9.54-lts 2024-01-05 v23.8.8.20-lts 2023-11-25 v23.8.7.24-lts 2023-11-17 v23.8.6.16-lts 2023-11-08 @@ -40,6 +45,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.19.32-lts 2024-01-05 v23.3.18.15-lts 2023-11-25 v23.3.17.13-lts 2023-11-17 v23.3.16.7-lts 2023-11-08