Merge branch 'master' of github.com:ClickHouse/ClickHouse into named-collections

This commit is contained in:
kssenii 2021-09-09 15:26:09 +03:00
commit 00be148fce
105 changed files with 2428 additions and 1084 deletions

View File

@ -1,3 +1,214 @@
### ClickHouse release v21.9, 2021-09-09
#### Backward Incompatible Change
* Do not output trailing zeros in text representation of `Decimal` types. Example: `1.23` will be printed instead of `1.230000` for decimal with scale 6. This closes [#15794](https://github.com/ClickHouse/ClickHouse/issues/15794). It may introduce slight incompatibility if your applications somehow relied on the trailing zeros. Serialization in output formats can be controlled with the setting `output_format_decimal_trailing_zeros`. Implementation of `toString` and casting to String is changed unconditionally. [#27680](https://github.com/ClickHouse/ClickHouse/pull/27680) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Do not allow to apply parametric aggregate function with `-Merge` combinator to aggregate function state if state was produced by aggregate function with different parameters. For example, state of `fooState(42)(x)` cannot be finalized with `fooMerge(s)` or `fooMerge(123)(s)`, parameters must be specified explicitly like `fooMerge(42)(s)` and must be equal. It does not affect some special aggregate functions like `quantile` and `sequence*` that use parameters for finalization only. [#26847](https://github.com/ClickHouse/ClickHouse/pull/26847) ([tavplubix](https://github.com/tavplubix)).
* Under clickhouse-local, always treat local addresses with a port as remote. [#26736](https://github.com/ClickHouse/ClickHouse/pull/26736) ([Raúl Marín](https://github.com/Algunenano)).
* Fix the issue that in case of some sophisticated query with column aliases identical to the names of expressions, bad cast may happen. This fixes [#25447](https://github.com/ClickHouse/ClickHouse/issues/25447). This fixes [#26914](https://github.com/ClickHouse/ClickHouse/issues/26914). This fix may introduce backward incompatibility: if there are different expressions with identical names, exception will be thrown. It may break some rare cases when `enable_optimize_predicate_expression` is set. [#26639](https://github.com/ClickHouse/ClickHouse/pull/26639) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Now, scalar subquery always returns `Nullable` result if it's type can be `Nullable`. It is needed because in case of empty subquery it's result should be `Null`. Previously, it was possible to get error about incompatible types (type deduction does not execute scalar subquery, and it could use not-nullable type). Scalar subquery with empty result which can't be converted to `Nullable` (like `Array` or `Tuple`) now throws error. Fixes [#25411](https://github.com/ClickHouse/ClickHouse/issues/25411). [#26423](https://github.com/ClickHouse/ClickHouse/pull/26423) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
#### New Feature
* Implementation of short circuit function evaluation, closes [#12587](https://github.com/ClickHouse/ClickHouse/issues/12587). Add settings `short_circuit_function_evaluation` to configure short circuit function evaluation. [#23367](https://github.com/ClickHouse/ClickHouse/pull/23367) ([Kruglov Pavel](https://github.com/Avogar)).
* Add support for INTERSECT, EXCEPT, ANY, ALL operators. [#24757](https://github.com/ClickHouse/ClickHouse/pull/24757) ([Kirill Ershov](https://github.com/zdikov)). ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add support for encryption at the virtual file system level (data encryption at rest) using AES-CTR algorithm. [#24206](https://github.com/ClickHouse/ClickHouse/pull/24206) ([Latysheva Alexandra](https://github.com/alexelex)). ([Vitaly Baranov](https://github.com/vitlibar)) [#26733](https://github.com/ClickHouse/ClickHouse/pull/26733) [#26377](https://github.com/ClickHouse/ClickHouse/pull/26377) [#26465](https://github.com/ClickHouse/ClickHouse/pull/26465).
* Added natural language processing (NLP) functions for tokenization, stemming, lemmatizing and search in synonyms extensions. [#24997](https://github.com/ClickHouse/ClickHouse/pull/24997) ([Nikolay Degterinsky](https://github.com/evillique)).
* Added integration with S2 geometry library. [#24980](https://github.com/ClickHouse/ClickHouse/pull/24980) ([Andr0901](https://github.com/Andr0901)). ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Add SQLite table engine, table function, database engine. [#24194](https://github.com/ClickHouse/ClickHouse/pull/24194) ([Arslan Gumerov](https://github.com/g-arslan)). ([Kseniia Sumarokova](https://github.com/kssenii)).
* Added support for custom query for `MySQL`, `PostgreSQL`, `ClickHouse`, `JDBC`, `Cassandra` dictionary source. Closes [#1270](https://github.com/ClickHouse/ClickHouse/issues/1270). [#26995](https://github.com/ClickHouse/ClickHouse/pull/26995) ([Maksim Kita](https://github.com/kitaisreal)).
* Introduce syntax for here documents. Example `SELECT $doc$ VALUE $doc$`. [#26671](https://github.com/ClickHouse/ClickHouse/pull/26671) ([Maksim Kita](https://github.com/kitaisreal)).
* Add shared (replicated) storage of user, roles, row policies, quotas and settings profiles through ZooKeeper. [#27426](https://github.com/ClickHouse/ClickHouse/pull/27426) ([Kevin Michel](https://github.com/kmichel-aiven)).
* Add compression for `INTO OUTFILE` that automatically choose compression algorithm. Closes [#3473](https://github.com/ClickHouse/ClickHouse/issues/3473). [#27134](https://github.com/ClickHouse/ClickHouse/pull/27134) ([Filatenkov Artur](https://github.com/FArthur-cmd)).
* Add `INSERT ... FROM INFILE` similarly to `SELECT ... INTO OUTFILE`. [#27655](https://github.com/ClickHouse/ClickHouse/pull/27655) ([Filatenkov Artur](https://github.com/FArthur-cmd)).
* Added `complex_key_range_hashed` dictionary. Closes [#22029](https://github.com/ClickHouse/ClickHouse/issues/22029). [#27629](https://github.com/ClickHouse/ClickHouse/pull/27629) ([Maksim Kita](https://github.com/kitaisreal)).
* Support expressions in JOIN ON section. Close [#21868](https://github.com/ClickHouse/ClickHouse/issues/21868). [#24420](https://github.com/ClickHouse/ClickHouse/pull/24420) ([Vladimir C](https://github.com/vdimir)).
* When client connects to server, it receives information about all warnings that are already were collected by server. (It can be disabled by using option `--no-warnings`). Add `system.warnings` table to collect warnings about server configuration. [#26246](https://github.com/ClickHouse/ClickHouse/pull/26246) ([Filatenkov Artur](https://github.com/FArthur-cmd)). [#26282](https://github.com/ClickHouse/ClickHouse/pull/26282) ([Filatenkov Artur](https://github.com/FArthur-cmd)).
* Allow using constant expressions from with and select in aggregate function parameters. Close [#10945](https://github.com/ClickHouse/ClickHouse/issues/10945). [#27531](https://github.com/ClickHouse/ClickHouse/pull/27531) ([abel-cheng](https://github.com/abel-cheng)).
* Add `tupleToNameValuePairs`, a function that turns a named tuple into an array of pairs. [#27505](https://github.com/ClickHouse/ClickHouse/pull/27505) ([Braulio Valdivielso Martínez](https://github.com/BraulioVM)).
* Add support for `bzip2` compression method for import/export. Closes [#22428](https://github.com/ClickHouse/ClickHouse/issues/22428). [#27377](https://github.com/ClickHouse/ClickHouse/pull/27377) ([Nikolay Degterinsky](https://github.com/evillique)).
* Added `bitmapSubsetOffsetLimit(bitmap, offset, cardinality_limit)` function. It creates a subset of bitmap limit the results to `cardinality_limit` with offset of `offset`. [#27234](https://github.com/ClickHouse/ClickHouse/pull/27234) ([DHBin](https://github.com/DHBin)).
* Add column `default_database` to `system.users`. [#27054](https://github.com/ClickHouse/ClickHouse/pull/27054) ([kevin wan](https://github.com/MaxWk)).
* Supported `cluster` macros inside table functions 'cluster' and 'clusterAllReplicas'. [#26913](https://github.com/ClickHouse/ClickHouse/pull/26913) ([polyprogrammist](https://github.com/PolyProgrammist)).
* Add new functions `currentRoles()`, `enabledRoles()`, `defaultRoles()`. [#26780](https://github.com/ClickHouse/ClickHouse/pull/26780) ([Vitaly Baranov](https://github.com/vitlibar)).
* New functions `currentProfiles()`, `enabledProfiles()`, `defaultProfiles()`. [#26714](https://github.com/ClickHouse/ClickHouse/pull/26714) ([Vitaly Baranov](https://github.com/vitlibar)).
* Add functions that return (initial_)query_id of the current query. This closes [#23682](https://github.com/ClickHouse/ClickHouse/issues/23682). [#26410](https://github.com/ClickHouse/ClickHouse/pull/26410) ([Alexey Boykov](https://github.com/mathalex)).
* Add `REPLACE GRANT` feature. [#26384](https://github.com/ClickHouse/ClickHouse/pull/26384) ([Caspian](https://github.com/Cas-pian)).
* Implement window function `nth_value(expr, N)` that returns the value of the Nth row of the window frame. [#26334](https://github.com/ClickHouse/ClickHouse/pull/26334) ([Zuo, RuoYu](https://github.com/ryzuo)).
* `EXPLAIN` query now has `EXPLAIN ESTIMATE ...` mode that will show information about read rows, marks and parts from MergeTree tables. Closes [#23941](https://github.com/ClickHouse/ClickHouse/issues/23941). [#26131](https://github.com/ClickHouse/ClickHouse/pull/26131) ([fastio](https://github.com/fastio)).
* Added `system.zookeeper_log` table. All actions of ZooKeeper client are logged into this table. Implements [#25449](https://github.com/ClickHouse/ClickHouse/issues/25449). [#26129](https://github.com/ClickHouse/ClickHouse/pull/26129) ([tavplubix](https://github.com/tavplubix)).
* Zero-copy replication for `ReplicatedMergeTree` over `HDFS` storage. [#25918](https://github.com/ClickHouse/ClickHouse/pull/25918) ([Zhichang Yu](https://github.com/yuzhichang)).
* Allow to insert Nested type as array of structs in `Arrow`, `ORC` and `Parquet` input format. [#25902](https://github.com/ClickHouse/ClickHouse/pull/25902) ([Kruglov Pavel](https://github.com/Avogar)).
* Add a new datatype `Date32` (store data as Int32), support date range same with `DateTime64` support load parquet date32 to ClickHouse `Date32` Add new function `toDate32` like `toDate`. [#25774](https://github.com/ClickHouse/ClickHouse/pull/25774) ([LiuNeng](https://github.com/liuneng1994)).
* Allow setting default database for users. [#25268](https://github.com/ClickHouse/ClickHouse/issues/25268). [#25687](https://github.com/ClickHouse/ClickHouse/pull/25687) ([kevin wan](https://github.com/MaxWk)).
* Add an optional parameter to `MongoDB` engine to accept connection string options and support SSL connection. Closes [#21189](https://github.com/ClickHouse/ClickHouse/issues/21189). Closes [#21041](https://github.com/ClickHouse/ClickHouse/issues/21041). [#22045](https://github.com/ClickHouse/ClickHouse/pull/22045) ([Omar Bazaraa](https://github.com/OmarBazaraa)).
#### Experimental Feature
* Added a compression codec `AES_128_GCM_SIV` which encrypts columns instead of compressing them. [#19896](https://github.com/ClickHouse/ClickHouse/pull/19896) ([PHO](https://github.com/depressed-pho)). Will be rewritten, do not use.
* Rename `MaterializeMySQL` to `MaterializedMySQL`. [#26822](https://github.com/ClickHouse/ClickHouse/pull/26822) ([tavplubix](https://github.com/tavplubix)).
#### Performance Improvement
* Improve the performance of fast queries when `max_execution_time = 0` by reducing the number of `clock_gettime` system calls. [#27325](https://github.com/ClickHouse/ClickHouse/pull/27325) ([filimonov](https://github.com/filimonov)).
* Specialize date time related comparison to achieve better performance. This fixes [#27083](https://github.com/ClickHouse/ClickHouse/issues/27083) . [#27122](https://github.com/ClickHouse/ClickHouse/pull/27122) ([Amos Bird](https://github.com/amosbird)).
* Share file descriptors in concurrent reads of the same files. There is no noticeable performance difference on Linux. But the number of opened files will be significantly (10..100 times) lower on typical servers and it makes operations easier. See [#26214](https://github.com/ClickHouse/ClickHouse/issues/26214). [#26768](https://github.com/ClickHouse/ClickHouse/pull/26768) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Improve latency of short queries, that require reading from tables with large number of columns. [#26371](https://github.com/ClickHouse/ClickHouse/pull/26371) ([Anton Popov](https://github.com/CurtizJ)).
* Don't build sets for indices when analyzing a query. [#26365](https://github.com/ClickHouse/ClickHouse/pull/26365) ([Raúl Marín](https://github.com/Algunenano)).
* Vectorize the SUM of Nullable integer types with native representation ([David Manzanares](https://github.com/davidmanzanares), [Raúl Marín](https://github.com/Algunenano)). [#26248](https://github.com/ClickHouse/ClickHouse/pull/26248) ([Raúl Marín](https://github.com/Algunenano)).
* Compile expressions involving columns with `Enum` types. [#26237](https://github.com/ClickHouse/ClickHouse/pull/26237) ([Maksim Kita](https://github.com/kitaisreal)).
* Compile aggregate functions `groupBitOr`, `groupBitAnd`, `groupBitXor`. [#26161](https://github.com/ClickHouse/ClickHouse/pull/26161) ([Maksim Kita](https://github.com/kitaisreal)).
* Improved memory usage with better block size prediction when reading empty DEFAULT columns. Closes [#17317](https://github.com/ClickHouse/ClickHouse/issues/17317). [#25917](https://github.com/ClickHouse/ClickHouse/pull/25917) ([Vladimir Chebotarev](https://github.com/excitoon)).
* Reduce memory usage and number of read rows in queries with `ORDER BY primary_key`. [#25721](https://github.com/ClickHouse/ClickHouse/pull/25721) ([Anton Popov](https://github.com/CurtizJ)).
* Enable `distributed_push_down_limit` by default. [#27104](https://github.com/ClickHouse/ClickHouse/pull/27104) ([Azat Khuzhin](https://github.com/azat)).
* Make `toTimeZone` monotonicity when timeZone is a constant value to support partition puring when use sql like:. [#26261](https://github.com/ClickHouse/ClickHouse/pull/26261) ([huangzhaowei](https://github.com/SaintBacchus)).
#### Improvement
* Mark window functions as ready for general use. Remove the `allow_experimental_window_functions` setting. [#27184](https://github.com/ClickHouse/ClickHouse/pull/27184) ([Alexander Kuzmenkov](https://github.com/akuzm)).
* Improve compatibility with non-whole-minute timezone offsets. [#27080](https://github.com/ClickHouse/ClickHouse/pull/27080) ([Raúl Marín](https://github.com/Algunenano)).
* If file descriptor in `File` table is regular file - allow to read multiple times from it. It allows `clickhouse-local` to read multiple times from stdin (with multiple SELECT queries or subqueries) if stdin is a regular file like `clickhouse-local --query "SELECT * FROM table UNION ALL SELECT * FROM table" ... < file`. This closes [#11124](https://github.com/ClickHouse/ClickHouse/issues/11124). Co-authored with ([alexey-milovidov](https://github.com/alexey-milovidov)). [#25960](https://github.com/ClickHouse/ClickHouse/pull/25960) ([BoloniniD](https://github.com/BoloniniD)).
* Remove duplicate index analysis and avoid possible invalid limit checks during projection analysis. [#27742](https://github.com/ClickHouse/ClickHouse/pull/27742) ([Amos Bird](https://github.com/amosbird)).
* Enable query parameters to be passed in the body of HTTP requests. [#27706](https://github.com/ClickHouse/ClickHouse/pull/27706) ([Hermano Lustosa](https://github.com/hllustosa)).
* Disallow `arrayJoin` on partition expressions. [#27648](https://github.com/ClickHouse/ClickHouse/pull/27648) ([Raúl Marín](https://github.com/Algunenano)).
* Log client IP address if authentication fails. [#27514](https://github.com/ClickHouse/ClickHouse/pull/27514) ([Misko Lee](https://github.com/imiskolee)).
* Use bytes instead of strings for binary data in the GRPC protocol. [#27431](https://github.com/ClickHouse/ClickHouse/pull/27431) ([Vitaly Baranov](https://github.com/vitlibar)).
* Send response with error message if HTTP port is not set and user tries to send HTTP request to TCP port. [#27385](https://github.com/ClickHouse/ClickHouse/pull/27385) ([Braulio Valdivielso Martínez](https://github.com/BraulioVM)).
* Add `_CAST` function for internal usage, which will not preserve type nullability, but non-internal cast will preserve according to setting `cast_keep_nullable`. Closes [#12636](https://github.com/ClickHouse/ClickHouse/issues/12636). [#27382](https://github.com/ClickHouse/ClickHouse/pull/27382) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add setting `log_formatted_queries` to log additional formatted query into `system.query_log`. It's useful for normalized query analysis because functions like `normalizeQuery` and `normalizeQueryKeepNames` don't parse/format queries in order to achieve better performance. [#27380](https://github.com/ClickHouse/ClickHouse/pull/27380) ([Amos Bird](https://github.com/amosbird)).
* Add two settings `max_hyperscan_regexp_length` and `max_hyperscan_regexp_total_length` to prevent huge regexp being used in hyperscan related functions, such as `multiMatchAny`. [#27378](https://github.com/ClickHouse/ClickHouse/pull/27378) ([Amos Bird](https://github.com/amosbird)).
* Memory consumed by bitmap aggregate functions now is taken into account for memory limits. This closes [#26555](https://github.com/ClickHouse/ClickHouse/issues/26555). [#27252](https://github.com/ClickHouse/ClickHouse/pull/27252) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Add new index data skipping minmax index format for proper Nullable support. [#27250](https://github.com/ClickHouse/ClickHouse/pull/27250) ([Azat Khuzhin](https://github.com/azat)).
* Add 10 seconds cache for S3 proxy resolver. [#27216](https://github.com/ClickHouse/ClickHouse/pull/27216) ([ianton-ru](https://github.com/ianton-ru)).
* Split global mutex into individual regexp construction. This helps avoid huge regexp construction blocking other related threads. [#27211](https://github.com/ClickHouse/ClickHouse/pull/27211) ([Amos Bird](https://github.com/amosbird)).
* Support schema for PostgreSQL database engine. Closes [#27166](https://github.com/ClickHouse/ClickHouse/issues/27166). [#27198](https://github.com/ClickHouse/ClickHouse/pull/27198) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Track memory usage in clickhouse-client. [#27191](https://github.com/ClickHouse/ClickHouse/pull/27191) ([Filatenkov Artur](https://github.com/FArthur-cmd)).
* Try recording `query_kind` in `system.query_log` even when query fails to start. [#27182](https://github.com/ClickHouse/ClickHouse/pull/27182) ([Amos Bird](https://github.com/amosbird)).
* Added columns `replica_is_active` that maps replica name to is replica active status to table `system.replicas`. Closes [#27138](https://github.com/ClickHouse/ClickHouse/issues/27138). [#27180](https://github.com/ClickHouse/ClickHouse/pull/27180) ([Maksim Kita](https://github.com/kitaisreal)).
* Allow to pass query settings via server URI in Web UI. [#27177](https://github.com/ClickHouse/ClickHouse/pull/27177) ([kolsys](https://github.com/kolsys)).
* Add a new metric called `MaxPushedDDLEntryID` which is the maximum ddl entry id that current node push to zookeeper. [#27174](https://github.com/ClickHouse/ClickHouse/pull/27174) ([Fuwang Hu](https://github.com/fuwhu)).
* Improved the existence condition judgment and empty string node judgment when `clickhouse-keeper` creates znode. [#27125](https://github.com/ClickHouse/ClickHouse/pull/27125) ([小路](https://github.com/nicelulu)).
* Merge JOIN correctly handles empty set in the right. [#27078](https://github.com/ClickHouse/ClickHouse/pull/27078) ([Vladimir C](https://github.com/vdimir)).
* Now functions can be shard-level constants, which means if it's executed in the context of some distributed table, it generates a normal column, otherwise it produces a constant value. Notable functions are: `hostName()`, `tcpPort()`, `version()`, `buildId()`, `uptime()`, etc. [#27020](https://github.com/ClickHouse/ClickHouse/pull/27020) ([Amos Bird](https://github.com/amosbird)).
* Updated `extractAllGroupsHorizontal` - upper limit on the number of matches per row can be set via optional third argument. [#26961](https://github.com/ClickHouse/ClickHouse/pull/26961) ([Vasily Nemkov](https://github.com/Enmk)).
* Expose `RocksDB` statistics via system.rocksdb table. Read rocksdb options from ClickHouse config (`rocksdb...` keys). NOTE: ClickHouse does not rely on RocksDB, it is just one of the additional integration storage engines. [#26821](https://github.com/ClickHouse/ClickHouse/pull/26821) ([Azat Khuzhin](https://github.com/azat)).
* Less verbose internal RocksDB logs. NOTE: ClickHouse does not rely on RocksDB, it is just one of the additional integration storage engines. This closes [#26252](https://github.com/ClickHouse/ClickHouse/issues/26252). [#26789](https://github.com/ClickHouse/ClickHouse/pull/26789) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Changing default roles affects new sessions only. [#26759](https://github.com/ClickHouse/ClickHouse/pull/26759) ([Vitaly Baranov](https://github.com/vitlibar)).
* Watchdog is disabled in docker by default. Fix for not handling ctrl+c. [#26757](https://github.com/ClickHouse/ClickHouse/pull/26757) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* `SET PROFILE` now applies constraints too if they're set for a passed profile. [#26730](https://github.com/ClickHouse/ClickHouse/pull/26730) ([Vitaly Baranov](https://github.com/vitlibar)).
* Improve handling of `KILL QUERY` requests. [#26675](https://github.com/ClickHouse/ClickHouse/pull/26675) ([Raúl Marín](https://github.com/Algunenano)).
* `mapPopulatesSeries` function supports `Map` type. [#26663](https://github.com/ClickHouse/ClickHouse/pull/26663) ([Ildus Kurbangaliev](https://github.com/ildus)).
* Fix excessive (x2) connect attempts with `skip_unavailable_shards`. [#26658](https://github.com/ClickHouse/ClickHouse/pull/26658) ([Azat Khuzhin](https://github.com/azat)).
* Avoid hanging `clickhouse-benchmark` if connection fails (i.e. on EMFILE). [#26656](https://github.com/ClickHouse/ClickHouse/pull/26656) ([Azat Khuzhin](https://github.com/azat)).
* Allow more threads to be used by the Kafka engine. [#26642](https://github.com/ClickHouse/ClickHouse/pull/26642) ([feihengye](https://github.com/feihengye)).
* Add round-robin support for `clickhouse-benchmark` (it does not differ from the regular multi host/port run except for statistics report). [#26607](https://github.com/ClickHouse/ClickHouse/pull/26607) ([Azat Khuzhin](https://github.com/azat)).
* Executable dictionaries (`executable`, `executable_pool`) enable creation with DDL query using `clickhouse-local`. Closes [#22355](https://github.com/ClickHouse/ClickHouse/issues/22355). [#26510](https://github.com/ClickHouse/ClickHouse/pull/26510) ([Maksim Kita](https://github.com/kitaisreal)).
* Set client query kind for `mysql` and `postgresql` compatibility protocol handlers. [#26498](https://github.com/ClickHouse/ClickHouse/pull/26498) ([anneji-dev](https://github.com/anneji-dev)).
* Apply `LIMIT` on the shards for queries like `SELECT * FROM dist ORDER BY key LIMIT 10` w/ `distributed_push_down_limit=1`. Avoid running `Distinct`/`LIMIT BY` steps for queries like `SELECT DISTINCT shading_key FROM dist ORDER BY key`. Now `distributed_push_down_limit` is respected by `optimize_distributed_group_by_sharding_key` optimization. [#26466](https://github.com/ClickHouse/ClickHouse/pull/26466) ([Azat Khuzhin](https://github.com/azat)).
* Updated protobuf to 3.17.3. Changelogs are available on https://github.com/protocolbuffers/protobuf/releases. [#26424](https://github.com/ClickHouse/ClickHouse/pull/26424) ([Ilya Yatsishin](https://github.com/qoega)).
* Enable `use_hedged_requests` setting that allows to mitigate tail latencies on large clusters. [#26380](https://github.com/ClickHouse/ClickHouse/pull/26380) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Improve behaviour with non-existing host in user allowed host list. [#26368](https://github.com/ClickHouse/ClickHouse/pull/26368) ([ianton-ru](https://github.com/ianton-ru)).
* Add ability to set `Distributed` directory monitor settings via CREATE TABLE (i.e. `CREATE TABLE dist (key Int) Engine=Distributed(cluster, db, table) SETTINGS monitor_batch_inserts=1` and similar). [#26336](https://github.com/ClickHouse/ClickHouse/pull/26336) ([Azat Khuzhin](https://github.com/azat)).
* Save server address in history URLs in web UI if it differs from the origin of web UI. This closes [#26044](https://github.com/ClickHouse/ClickHouse/issues/26044). [#26322](https://github.com/ClickHouse/ClickHouse/pull/26322) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Add events to profile calls to `sleep` / `sleepEachRow`. [#26320](https://github.com/ClickHouse/ClickHouse/pull/26320) ([Raúl Marín](https://github.com/Algunenano)).
* Allow to reuse connections of shards among different clusters. It also avoids creating new connections when using `cluster` table function. [#26318](https://github.com/ClickHouse/ClickHouse/pull/26318) ([Amos Bird](https://github.com/amosbird)).
* Control the execution period of clear old temporary directories by parameter with default value. [#26212](https://github.com/ClickHouse/ClickHouse/issues/26212). [#26313](https://github.com/ClickHouse/ClickHouse/pull/26313) ([fastio](https://github.com/fastio)).
* Add a setting `function_range_max_elements_in_block` to tune the safety threshold for data volume generated by function `range`. This closes [#26303](https://github.com/ClickHouse/ClickHouse/issues/26303). [#26305](https://github.com/ClickHouse/ClickHouse/pull/26305) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Check hash function at table creation, not at sampling. Add settings for MergeTree, if someone create a table with incorrect sampling column but sampling never be used, disable this settings for starting the server without exception. [#26256](https://github.com/ClickHouse/ClickHouse/pull/26256) ([zhaoyu](https://github.com/zxc111)).
* Added `output_format_avro_string_column_pattern` setting to put specified String columns to Avro as string instead of default bytes. Implements [#22414](https://github.com/ClickHouse/ClickHouse/issues/22414). [#26245](https://github.com/ClickHouse/ClickHouse/pull/26245) ([Ilya Golshtein](https://github.com/ilejn)).
* Add information about column sizes in `system.columns` table for `Log` and `TinyLog` tables. This closes [#9001](https://github.com/ClickHouse/ClickHouse/issues/9001). [#26241](https://github.com/ClickHouse/ClickHouse/pull/26241) ([Nikolay Degterinsky](https://github.com/evillique)).
* Don't throw exception when querying `system.detached_parts` table if there is custom disk configuration and `detached` directory does not exist on some disks. This closes [#26078](https://github.com/ClickHouse/ClickHouse/issues/26078). [#26236](https://github.com/ClickHouse/ClickHouse/pull/26236) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Check for non-deterministic functions in keys, including constant expressions like `now()`, `today()`. This closes [#25875](https://github.com/ClickHouse/ClickHouse/issues/25875). This closes [#11333](https://github.com/ClickHouse/ClickHouse/issues/11333). [#26235](https://github.com/ClickHouse/ClickHouse/pull/26235) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* convert timestamp and timestamptz data types to `DateTime64` in PostgreSQL table engine. [#26234](https://github.com/ClickHouse/ClickHouse/pull/26234) ([jasine](https://github.com/jasine)).
* Apply aggressive IN index analysis for projections so that better projection candidate can be selected. [#26218](https://github.com/ClickHouse/ClickHouse/pull/26218) ([Amos Bird](https://github.com/amosbird)).
* Remove GLOBAL keyword for IN when scalar function is passed. In previous versions, if user specified `GLOBAL IN f(x)` exception was thrown. [#26217](https://github.com/ClickHouse/ClickHouse/pull/26217) ([Amos Bird](https://github.com/amosbird)).
* Add error id (like `BAD_ARGUMENTS`) to exception messages. This closes [#25862](https://github.com/ClickHouse/ClickHouse/issues/25862). [#26172](https://github.com/ClickHouse/ClickHouse/pull/26172) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Fix incorrect output with --progress option for clickhouse-local. Progress bar will be cleared once it gets to 100% - same as it is done for clickhouse-client. Closes [#17484](https://github.com/ClickHouse/ClickHouse/issues/17484). [#26128](https://github.com/ClickHouse/ClickHouse/pull/26128) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add `merge_selecting_sleep_ms` setting. [#26120](https://github.com/ClickHouse/ClickHouse/pull/26120) ([lthaooo](https://github.com/lthaooo)).
* Remove complicated usage of Linux AIO with one block readahead and replace it with plain simple synchronous IO with O_DIRECT. In previous versions, the setting `min_bytes_to_use_direct_io` may not work correctly if `max_threads` is greater than one. Reading with direct IO (that is disabled by default for queries and enabled by default for large merges) will work in less efficient way. This closes [#25997](https://github.com/ClickHouse/ClickHouse/issues/25997). [#26003](https://github.com/ClickHouse/ClickHouse/pull/26003) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Flush `Distributed` table on `REPLACE TABLE` query. Resolves [#24566](https://github.com/ClickHouse/ClickHouse/issues/24566) - Do not replace (or create) table on `[CREATE OR] REPLACE TABLE ... AS SELECT` query if insertion into new table fails. Resolves [#23175](https://github.com/ClickHouse/ClickHouse/issues/23175). [#25895](https://github.com/ClickHouse/ClickHouse/pull/25895) ([tavplubix](https://github.com/tavplubix)).
* Add `views` column to system.query_log containing the names of the (materialized or live) views executed by the query. Adds a new log table (`system.query_views_log`) that contains information about each view executed during a query. Modifies view execution: When an exception is thrown while executing a view, any view that has already startedwill continue running until it finishes. This used to be the behaviour under parallel_view_processing=true and now it's always the same behaviour. - Dependent views now report reading progress to the context. [#25714](https://github.com/ClickHouse/ClickHouse/pull/25714) ([Raúl Marín](https://github.com/Algunenano)).
* Do connection draining asynchonously upon finishing executing distributed queries. A new server setting is added `max_threads_for_connection_collector` which specifies the number of workers to recycle connections in background. If the pool is full, connection will be drained synchronously but a bit different than before: It's drained after we send EOS to client, query will succeed immediately after receiving enough data, and any exception will be logged instead of throwing to the client. Added setting `drain_timeout` (3 seconds by default). Connection draining will disconnect upon timeout. [#25674](https://github.com/ClickHouse/ClickHouse/pull/25674) ([Amos Bird](https://github.com/amosbird)).
* Support for multiple includes in configuration. It is possible to include users configuration, remote servers configuration from multiple sources. Simply place `<include />` element with `from_zk`, `from_env` or `incl` attribute and it will be replaced with the substitution. [#24404](https://github.com/ClickHouse/ClickHouse/pull/24404) ([nvartolomei](https://github.com/nvartolomei)).
* Fix multiple block insertion into distributed table with `insert_distributed_one_random_shard = 1`. This is a marginal feature. Mark as improvement. [#23140](https://github.com/ClickHouse/ClickHouse/pull/23140) ([Amos Bird](https://github.com/amosbird)).
* Support `LowCardinality` and `FixedString` keys/values for `Map` type. [#21543](https://github.com/ClickHouse/ClickHouse/pull/21543) ([hexiaoting](https://github.com/hexiaoting)).
* Enable reloading of local disk config. [#19526](https://github.com/ClickHouse/ClickHouse/pull/19526) ([taiyang-li](https://github.com/taiyang-li)).
* Now KeyConditions can correctly skip nullable keys, including `isNull` and `isNotNull`. https://github.com/ClickHouse/ClickHouse/pull/12433. [#12455](https://github.com/ClickHouse/ClickHouse/pull/12455) ([Amos Bird](https://github.com/amosbird)).
#### Bug Fix
* Fix a couple of bugs that may cause replicas to diverge. [#27808](https://github.com/ClickHouse/ClickHouse/pull/27808) ([tavplubix](https://github.com/tavplubix)).
* Fix a rare bug in `DROP PART` which can lead to the error `Unexpected merged part intersects drop range`. [#27807](https://github.com/ClickHouse/ClickHouse/pull/27807) ([alesapin](https://github.com/alesapin)).
* Prevent crashes for some formats when NULL (tombstone) message was coming from Kafka. Closes [#19255](https://github.com/ClickHouse/ClickHouse/issues/19255). [#27794](https://github.com/ClickHouse/ClickHouse/pull/27794) ([filimonov](https://github.com/filimonov)).
* Fix column filtering with union distinct in subquery. Closes [#27578](https://github.com/ClickHouse/ClickHouse/issues/27578). [#27689](https://github.com/ClickHouse/ClickHouse/pull/27689) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix bad type cast when functions like `arrayHas` are applied to arrays of LowCardinality of Nullable of different non-numeric types like `DateTime` and `DateTime64`. In previous versions bad cast occurs. In new version it will lead to exception. This closes [#26330](https://github.com/ClickHouse/ClickHouse/issues/26330). [#27682](https://github.com/ClickHouse/ClickHouse/pull/27682) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Fix postgresql table function resulting in non-closing connections. Closes [#26088](https://github.com/ClickHouse/ClickHouse/issues/26088). [#27662](https://github.com/ClickHouse/ClickHouse/pull/27662) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fixed another case of `Unexpected merged part ... intersecting drop range ...` error. [#27656](https://github.com/ClickHouse/ClickHouse/pull/27656) ([tavplubix](https://github.com/tavplubix)).
* Fix an error with aliased column in `Distributed` table. [#27652](https://github.com/ClickHouse/ClickHouse/pull/27652) ([Vladimir C](https://github.com/vdimir)).
* After setting `max_memory_usage*` to non-zero value it was not possible to reset it back to 0 (unlimited). It's fixed. [#27638](https://github.com/ClickHouse/ClickHouse/pull/27638) ([tavplubix](https://github.com/tavplubix)).
* Fixed underflow of the time value when constructing it from components. Closes [#27193](https://github.com/ClickHouse/ClickHouse/issues/27193). [#27605](https://github.com/ClickHouse/ClickHouse/pull/27605) ([Vasily Nemkov](https://github.com/Enmk)).
* Fix crash during projection materialization when some parts contain missing columns. This fixes [#27512](https://github.com/ClickHouse/ClickHouse/issues/27512). [#27528](https://github.com/ClickHouse/ClickHouse/pull/27528) ([Amos Bird](https://github.com/amosbird)).
* fix metric `BackgroundMessageBrokerSchedulePoolTask`, maybe mistyped. [#27452](https://github.com/ClickHouse/ClickHouse/pull/27452) ([Ben](https://github.com/benbiti)).
* Fix distributed queries with zero shards and aggregation. [#27427](https://github.com/ClickHouse/ClickHouse/pull/27427) ([Azat Khuzhin](https://github.com/azat)).
* Compatibility when `/proc/meminfo` does not contain KB suffix. [#27361](https://github.com/ClickHouse/ClickHouse/pull/27361) ([Mike Kot](https://github.com/myrrc)).
* Fix incorrect result for query with row-level security, PREWHERE and LowCardinality filter. Fixes [#27179](https://github.com/ClickHouse/ClickHouse/issues/27179). [#27329](https://github.com/ClickHouse/ClickHouse/pull/27329) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fixed incorrect validation of partition id for MergeTree tables that created with old syntax. [#27328](https://github.com/ClickHouse/ClickHouse/pull/27328) ([tavplubix](https://github.com/tavplubix)).
* Fix MySQL protocol when using parallel formats (CSV / TSV). [#27326](https://github.com/ClickHouse/ClickHouse/pull/27326) ([Raúl Marín](https://github.com/Algunenano)).
* Fix `Cannot find column` error for queries with sampling. Was introduced in [#24574](https://github.com/ClickHouse/ClickHouse/issues/24574). Fixes [#26522](https://github.com/ClickHouse/ClickHouse/issues/26522). [#27301](https://github.com/ClickHouse/ClickHouse/pull/27301) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix errors like `Expected ColumnLowCardinality, gotUInt8` or `Bad cast from type DB::ColumnVector<char8_t> to DB::ColumnLowCardinality` for some queries with `LowCardinality` in `PREWHERE`. And more importantly, fix the lack of whitespace in the error message. Fixes [#23515](https://github.com/ClickHouse/ClickHouse/issues/23515). [#27298](https://github.com/ClickHouse/ClickHouse/pull/27298) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix `distributed_group_by_no_merge = 2` with `distributed_push_down_limit = 1` or `optimize_distributed_group_by_sharding_key = 1` with `LIMIT BY` and `LIMIT OFFSET`. [#27249](https://github.com/ClickHouse/ClickHouse/pull/27249) ([Azat Khuzhin](https://github.com/azat)). These are obscure combination of settings that no one is using.
* Fix mutation stuck on invalid partitions in non-replicated MergeTree. [#27248](https://github.com/ClickHouse/ClickHouse/pull/27248) ([Azat Khuzhin](https://github.com/azat)).
* In case of ambiguity, lambda functions prefer its arguments to other aliases or identifiers. [#27235](https://github.com/ClickHouse/ClickHouse/pull/27235) ([Raúl Marín](https://github.com/Algunenano)).
* Fix column structure in merge join, close [#27091](https://github.com/ClickHouse/ClickHouse/issues/27091). [#27217](https://github.com/ClickHouse/ClickHouse/pull/27217) ([Vladimir C](https://github.com/vdimir)).
* In rare cases `system.detached_parts` table might contain incorrect information for some parts, it's fixed. Fixes [#27114](https://github.com/ClickHouse/ClickHouse/issues/27114). [#27183](https://github.com/ClickHouse/ClickHouse/pull/27183) ([tavplubix](https://github.com/tavplubix)).
* Fix uninitialized memory in functions `multiSearch*` with empty array, close [#27169](https://github.com/ClickHouse/ClickHouse/issues/27169). [#27181](https://github.com/ClickHouse/ClickHouse/pull/27181) ([Vladimir C](https://github.com/vdimir)).
* Fix synchronization in GRPCServer. This PR fixes [#27024](https://github.com/ClickHouse/ClickHouse/issues/27024). [#27064](https://github.com/ClickHouse/ClickHouse/pull/27064) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fixed `cache`, `complex_key_cache`, `ssd_cache`, `complex_key_ssd_cache` configuration parsing. Options `allow_read_expired_keys`, `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds` were not parsed for dictionaries with non `cache` type. [#27032](https://github.com/ClickHouse/ClickHouse/pull/27032) ([Maksim Kita](https://github.com/kitaisreal)).
* Fix possible mutation stack due to race with DROP_RANGE. [#27002](https://github.com/ClickHouse/ClickHouse/pull/27002) ([Azat Khuzhin](https://github.com/azat)).
* Now partition ID in queries like `ALTER TABLE ... PARTITION ID xxx` validates for correctness. Fixes [#25718](https://github.com/ClickHouse/ClickHouse/issues/25718). [#26963](https://github.com/ClickHouse/ClickHouse/pull/26963) ([alesapin](https://github.com/alesapin)).
* Fix "Unknown column name" error with multiple JOINs in some cases, close [#26899](https://github.com/ClickHouse/ClickHouse/issues/26899). [#26957](https://github.com/ClickHouse/ClickHouse/pull/26957) ([Vladimir C](https://github.com/vdimir)).
* Fix reading of custom TLDs (stops processing with lower buffer or bigger file). [#26948](https://github.com/ClickHouse/ClickHouse/pull/26948) ([Azat Khuzhin](https://github.com/azat)).
* Fix error `Missing columns: 'xxx'` when `DEFAULT` column references other non materialized column without `DEFAULT` expression. Fixes [#26591](https://github.com/ClickHouse/ClickHouse/issues/26591). [#26900](https://github.com/ClickHouse/ClickHouse/pull/26900) ([alesapin](https://github.com/alesapin)).
* Fix loading of dictionary keys in `library-bridge` for `library` dictionary source. [#26834](https://github.com/ClickHouse/ClickHouse/pull/26834) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Aggregate function parameters might be lost when applying some combinators causing exceptions like `Conversion from AggregateFunction(topKArray, Array(String)) to AggregateFunction(topKArray(10), Array(String)) is not supported`. It's fixed. Fixes [#26196](https://github.com/ClickHouse/ClickHouse/issues/26196) and [#26433](https://github.com/ClickHouse/ClickHouse/issues/26433). [#26814](https://github.com/ClickHouse/ClickHouse/pull/26814) ([tavplubix](https://github.com/tavplubix)).
* Add `event_time_microseconds` value for `REMOVE_PART` in `system.part_log`. In previous versions is was not set. [#26720](https://github.com/ClickHouse/ClickHouse/pull/26720) ([Azat Khuzhin](https://github.com/azat)).
* Do not remove data on ReplicatedMergeTree table shutdown to avoid creating data to metadata inconsistency. [#26716](https://github.com/ClickHouse/ClickHouse/pull/26716) ([nvartolomei](https://github.com/nvartolomei)).
* Sometimes `SET ROLE` could work incorrectly, this PR fixes that. [#26707](https://github.com/ClickHouse/ClickHouse/pull/26707) ([Vitaly Baranov](https://github.com/vitlibar)).
* Some fixes for parallel formatting (https://github.com/ClickHouse/ClickHouse/issues/26694). [#26703](https://github.com/ClickHouse/ClickHouse/pull/26703) ([Raúl Marín](https://github.com/Algunenano)).
* Fix potential nullptr dereference in window functions. This fixes [#25276](https://github.com/ClickHouse/ClickHouse/issues/25276). [#26668](https://github.com/ClickHouse/ClickHouse/pull/26668) ([Alexander Kuzmenkov](https://github.com/akuzm)).
* Fix clickhouse-client history file conversion (when upgrading from the format of 3 years old version of clickhouse-client) if file is empty. [#26589](https://github.com/ClickHouse/ClickHouse/pull/26589) ([Azat Khuzhin](https://github.com/azat)).
* Fix incorrect function names of groupBitmapAnd/Or/Xor (can be displayed in some occasions). This fixes. [#26557](https://github.com/ClickHouse/ClickHouse/pull/26557) ([Amos Bird](https://github.com/amosbird)).
* Update `chown` cmd check in clickhouse-server docker entrypoint. It fixes the bug that cluster pod restart failed (or timeout) on kubernetes. [#26545](https://github.com/ClickHouse/ClickHouse/pull/26545) ([Ky Li](https://github.com/Kylinrix)).
* Fix crash in `RabbitMQ` shutdown in case `RabbitMQ` setup was not started. Closes [#26504](https://github.com/ClickHouse/ClickHouse/issues/26504). [#26529](https://github.com/ClickHouse/ClickHouse/pull/26529) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix issues with `CREATE DICTIONARY` query if dictionary name or database name was quoted. Closes [#26491](https://github.com/ClickHouse/ClickHouse/issues/26491). [#26508](https://github.com/ClickHouse/ClickHouse/pull/26508) ([Maksim Kita](https://github.com/kitaisreal)).
* Fix broken column name resolution after rewriting column aliases. This fixes [#26432](https://github.com/ClickHouse/ClickHouse/issues/26432). [#26475](https://github.com/ClickHouse/ClickHouse/pull/26475) ([Amos Bird](https://github.com/amosbird)).
* Fix some fuzzed msan crash. Fixes [#22517](https://github.com/ClickHouse/ClickHouse/issues/22517). [#26428](https://github.com/ClickHouse/ClickHouse/pull/26428) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix infinite non joined block stream in `partial_merge_join` close [#26325](https://github.com/ClickHouse/ClickHouse/issues/26325). [#26374](https://github.com/ClickHouse/ClickHouse/pull/26374) ([Vladimir C](https://github.com/vdimir)).
* Fix possible crash when login as dropped user. This PR fixes [#26073](https://github.com/ClickHouse/ClickHouse/issues/26073). [#26363](https://github.com/ClickHouse/ClickHouse/pull/26363) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fix `optimize_distributed_group_by_sharding_key` for multiple columns (leads to incorrect result w/ `optimize_skip_unused_shards=1`/`allow_nondeterministic_optimize_skip_unused_shards=1` and multiple columns in sharding key expression). [#26353](https://github.com/ClickHouse/ClickHouse/pull/26353) ([Azat Khuzhin](https://github.com/azat)).
* Fixed rare bug in lost replica recovery that may cause replicas to diverge. [#26321](https://github.com/ClickHouse/ClickHouse/pull/26321) ([tavplubix](https://github.com/tavplubix)).
* Fix zstd decompression (for import/export in zstd framing format that is unrelated to tables data) in case there are escape sequences at the end of internal buffer. Closes [#26013](https://github.com/ClickHouse/ClickHouse/issues/26013). [#26314](https://github.com/ClickHouse/ClickHouse/pull/26314) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix logical error on join with totals, close [#26017](https://github.com/ClickHouse/ClickHouse/issues/26017). [#26250](https://github.com/ClickHouse/ClickHouse/pull/26250) ([Vladimir C](https://github.com/vdimir)).
* Remove excessive newline in `thread_name` column in `system.stack_trace` table. This fixes [#24124](https://github.com/ClickHouse/ClickHouse/issues/24124). [#26210](https://github.com/ClickHouse/ClickHouse/pull/26210) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Fix potential crash if more than one `untuple` expression is used. [#26179](https://github.com/ClickHouse/ClickHouse/pull/26179) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Don't throw exception in `toString` for Nullable Enum if Enum does not have a value for zero, close [#25806](https://github.com/ClickHouse/ClickHouse/issues/25806). [#26123](https://github.com/ClickHouse/ClickHouse/pull/26123) ([Vladimir C](https://github.com/vdimir)).
* Fixed incorrect `sequence_id` in MySQL protocol packets that ClickHouse sends on exception during query execution. It might cause MySQL client to reset connection to ClickHouse server. Fixes [#21184](https://github.com/ClickHouse/ClickHouse/issues/21184). [#26051](https://github.com/ClickHouse/ClickHouse/pull/26051) ([tavplubix](https://github.com/tavplubix)).
* Fix for the case that `cutToFirstSignificantSubdomainCustom()`/`cutToFirstSignificantSubdomainCustomWithWWW()`/`firstSignificantSubdomainCustom()` returns incorrect type for consts, and hence `optimize_skip_unused_shards` does not work:. [#26041](https://github.com/ClickHouse/ClickHouse/pull/26041) ([Azat Khuzhin](https://github.com/azat)).
* Fix possible mismatched header when using normal projection with prewhere. This fixes [#26020](https://github.com/ClickHouse/ClickHouse/issues/26020). [#26038](https://github.com/ClickHouse/ClickHouse/pull/26038) ([Amos Bird](https://github.com/amosbird)).
* Fix sharding_key from column w/o function for remote() (before `select * from remote('127.1', system.one, dummy)` leads to `Unknown column: dummy, there are only columns .` error). [#25824](https://github.com/ClickHouse/ClickHouse/pull/25824) ([Azat Khuzhin](https://github.com/azat)).
* Fixed `Not found column ...` and `Missing column ...` errors when selecting from `MaterializeMySQL`. Fixes [#23708](https://github.com/ClickHouse/ClickHouse/issues/23708), [#24830](https://github.com/ClickHouse/ClickHouse/issues/24830), [#25794](https://github.com/ClickHouse/ClickHouse/issues/25794). [#25822](https://github.com/ClickHouse/ClickHouse/pull/25822) ([tavplubix](https://github.com/tavplubix)).
* Fix `optimize_skip_unused_shards_rewrite_in` for non-UInt64 types (may select incorrect shards eventually or throw `Cannot infer type of an empty tuple` or `Function tuple requires at least one argument`). [#25798](https://github.com/ClickHouse/ClickHouse/pull/25798) ([Azat Khuzhin](https://github.com/azat)).
#### Build/Testing/Packaging Improvement
* Now we ran stateful and stateless tests in random timezones. Fixes [#12439](https://github.com/ClickHouse/ClickHouse/issues/12439). Reading String as DateTime and writing DateTime as String in Protobuf format now respect timezone. Reading UInt16 as DateTime in Arrow and Parquet formats now treat it as Date and then converts to DateTime with respect to DateTime's timezone, because Date is serialized in Arrow and Parquet as UInt16. GraphiteMergeTree now respect time zone for rounding of times. Fixes [#5098](https://github.com/ClickHouse/ClickHouse/issues/5098). Author: @alexey-milovidov. [#15408](https://github.com/ClickHouse/ClickHouse/pull/15408) ([alesapin](https://github.com/alesapin)).
* `clickhouse-test` supports SQL tests with [Jinja2](https://jinja.palletsprojects.com/en/3.0.x/templates/#synopsis) templates. [#26579](https://github.com/ClickHouse/ClickHouse/pull/26579) ([Vladimir C](https://github.com/vdimir)).
* Add support for build with `clang-13`. This closes [#27705](https://github.com/ClickHouse/ClickHouse/issues/27705). [#27714](https://github.com/ClickHouse/ClickHouse/pull/27714) ([alexey-milovidov](https://github.com/alexey-milovidov)). [#27777](https://github.com/ClickHouse/ClickHouse/pull/27777) ([Sergei Semin](https://github.com/syominsergey))
* Add CMake options to build with or without specific CPU instruction set. This is for [#17469](https://github.com/ClickHouse/ClickHouse/issues/17469) and [#27509](https://github.com/ClickHouse/ClickHouse/issues/27509). [#27508](https://github.com/ClickHouse/ClickHouse/pull/27508) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Fix linking of auxiliar programs when using dynamic libraries. [#26958](https://github.com/ClickHouse/ClickHouse/pull/26958) ([Raúl Marín](https://github.com/Algunenano)).
* Update RocksDB to `2021-07-16` master. [#26411](https://github.com/ClickHouse/ClickHouse/pull/26411) ([alexey-milovidov](https://github.com/alexey-milovidov)).
### ClickHouse release v21.8, 2021-08-12
#### Upgrade Notes

View File

@ -165,6 +165,13 @@ if (COMPILER_CLANG)
if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE")
set(COMPILER_FLAGS "${COMPILER_FLAGS} -gdwarf-aranges")
endif ()
if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL 12.0.0)
if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Xclang -fuse-ctor-homing")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Xclang -fuse-ctor-homing")
endif()
endif()
endif ()
# If turned `ON`, assumes the user has either the system GTest library or the bundled one.

View File

@ -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 54455)
SET(VERSION_REVISION 54456)
SET(VERSION_MAJOR 21)
SET(VERSION_MINOR 10)
SET(VERSION_MINOR 11)
SET(VERSION_PATCH 1)
SET(VERSION_GITHASH 09df5018f95edcd0f759d4689ac5d029dd400c2a)
SET(VERSION_DESCRIBE v21.10.1.1-testing)
SET(VERSION_STRING 21.10.1.1)
SET(VERSION_GITHASH 7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7)
SET(VERSION_DESCRIBE v21.11.1.1-prestable)
SET(VERSION_STRING 21.11.1.1)
# end of autochange

2
contrib/boost vendored

@ -1 +1 @@
Subproject commit 9cf09dbfd55a5c6202dedbdf40781a51b02c2675
Subproject commit 66d17f060c4867aeea99fa2a20cfdae89ae2a2ec

View File

@ -16,7 +16,7 @@ if (NOT USE_INTERNAL_BOOST_LIBRARY)
graph
)
if(Boost_INCLUDE_DIR AND Boost_FILESYSTEM_LIBRARY AND Boost_FILESYSTEM_LIBRARY AND
if(Boost_INCLUDE_DIR AND Boost_FILESYSTEM_LIBRARY AND
Boost_PROGRAM_OPTIONS_LIBRARY AND Boost_REGEX_LIBRARY AND Boost_SYSTEM_LIBRARY AND Boost_CONTEXT_LIBRARY AND
Boost_COROUTINE_LIBRARY AND Boost_GRAPH_LIBRARY)
@ -238,4 +238,14 @@ if (NOT EXTERNAL_BOOST_FOUND)
target_include_directories (_boost_graph PRIVATE ${LIBRARY_DIR})
target_link_libraries(_boost_graph PRIVATE _boost_regex)
# circular buffer
add_library(_boost_circular_buffer INTERFACE)
add_library(boost::circular_buffer ALIAS _boost_circular_buffer)
target_include_directories(_boost_circular_buffer SYSTEM BEFORE INTERFACE ${LIBRARY_DIR})
# heap
add_library(_boost_heap INTERFACE)
add_library(boost::heap ALIAS _boost_heap)
target_include_directories(_boost_heap SYSTEM BEFORE INTERFACE ${LIBRARY_DIR})
endif ()

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (21.10.1.1) unstable; urgency=low
clickhouse (21.11.1.1) unstable; urgency=low
* Modified source code
-- clickhouse-release <clickhouse-release@yandex-team.ru> Sat, 17 Jul 2021 08:45:03 +0300
-- clickhouse-release <clickhouse-release@yandex-team.ru> Thu, 09 Sep 2021 12:03:26 +0300

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.10.1.*
ARG version=21.11.1.*
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list

View File

@ -17,6 +17,7 @@ RUN apt-get update \
devscripts \
libc++-dev \
libc++abi-dev \
libboost-all-dev \
libboost-program-options-dev \
libboost-system-dev \
libboost-filesystem-dev \

View File

@ -1,7 +1,7 @@
FROM ubuntu:20.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.10.1.*
ARG version=21.11.1.*
ARG gosu_ver=1.10
# set non-empty deb_location_url url to create a docker image

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.10.1.*
ARG version=21.11.1.*
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -38,6 +38,10 @@ You can also download and install packages manually from [here](https://repo.cli
- `clickhouse-client` — Creates a symbolic link for `clickhouse-client` and other client-related tools. and installs client configuration files.
- `clickhouse-common-static-dbg` — Installs ClickHouse compiled binary files with debug info.
!!! attention "Attention"
If you need to install specific version of ClickHouse you have to install all packages with the same version:
`sudo apt-get install clickhouse-server=21.8.5.7 clickhouse-client=21.8.5.7 clickhouse-common-static=21.8.5.7`
### From RPM Packages {#from-rpm-packages}
It is recommended to use official pre-compiled `rpm` packages for CentOS, RedHat, and all other rpm-based Linux distributions.

View File

@ -74,7 +74,7 @@ The `TabSeparated` format is convenient for processing data using custom program
The `TabSeparated` format supports outputting total values (when using WITH TOTALS) and extreme values (when extremes is set to 1). In these cases, the total values and extremes are output after the main data. The main result, total values, and extremes are separated from each other by an empty line. Example:
``` sql
SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated``
SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated
```
``` text

View File

@ -12,7 +12,7 @@ The following operations are available:
- `ALTER TABLE [db].name DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk.
- `ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name` - The query rebuilds the secondary index `name` in the partition `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations).
- `ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name` - The query rebuilds the secondary index `name` in the partition `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). To rebuild index over the whole data in the table you need to remove `IN PARTITION` from query.
The first two commands are lightweight in a sense that they only change metadata or remove files.

View File

@ -31,6 +31,19 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su
Если вы хотите использовать наиболее свежую версию, замените `stable` на `testing` (рекомендуется для тестовых окружений).
Также вы можете вручную скачать и установить пакеты из [репозитория](https://repo.clickhouse.tech/deb/stable/main/).
#### Пакеты {#packages}
- `clickhouse-common-static` — Устанавливает исполняемые файлы ClickHouse.
- `clickhouse-server` — Создает символические ссылки для `clickhouse-server` и устанавливает конфигурационные файлы.
- `clickhouse-client` — Создает символические ссылки для `clickhouse-client` и других клиентских инструментов и устанавливает конфигурационные файлы `clickhouse-client`.
- `clickhouse-common-static-dbg` — Устанавливает исполняемые файлы ClickHouse собранные с отладочной информацией.
!!! attention "Внимание"
Если вам нужно установить ClickHouse определенной версии, вы должны установить все пакеты одной версии:
`sudo apt-get install clickhouse-server=21.8.5.7 clickhouse-client=21.8.5.7 clickhouse-common-static=21.8.5.7`
### Из RPM пакетов {#from-rpm-packages}
Команда ClickHouse в Яндексе рекомендует использовать официальные предкомпилированные `rpm` пакеты для CentOS, RedHat и всех остальных дистрибутивов Linux, основанных на rpm.

View File

@ -19,7 +19,7 @@ ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name
Команда `ADD INDEX` добавляет описание индексов в метаданные, а `DROP INDEX` удаляет индекс из метаданных и стирает файлы индекса с диска, поэтому они легковесные и работают мгновенно.
Если индекс появился в метаданных, то он начнет считаться в последующих слияниях и записях в таблицу, а не сразу после выполнения операции `ALTER`.
`MATERIALIZE INDEX` - перестраивает индекс в указанной партиции. Реализовано как мутация.
`MATERIALIZE INDEX` - перестраивает индекс в указанной партиции. Реализовано как мутация. В случае если нужно перестроить индекс над всеми данными то писать `IN PARTITION` не нужно.
Запрос на изменение индексов реплицируется, сохраняя новые метаданные в ZooKeeper и применяя изменения на всех репликах.

View File

@ -8,6 +8,7 @@
#include <Poco/NullChannel.h>
#include <Databases/DatabaseMemory.h>
#include <Storages/System/attachSystemTables.h>
#include <Storages/System/attachInformationSchemaTables.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/executeQuery.h>
#include <Interpreters/loadMetadata.h>
@ -179,20 +180,18 @@ void LocalServer::tryInitPath()
}
static void attachSystemTables(ContextPtr context)
static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const String & database_name)
{
DatabasePtr system_database = DatabaseCatalog::instance().tryGetDatabase(DatabaseCatalog::SYSTEM_DATABASE);
DatabasePtr system_database = DatabaseCatalog::instance().tryGetDatabase(database_name);
if (!system_database)
{
/// TODO: add attachTableDelayed into DatabaseMemory to speedup loading
system_database = std::make_shared<DatabaseMemory>(DatabaseCatalog::SYSTEM_DATABASE, context);
DatabaseCatalog::instance().attachDatabase(DatabaseCatalog::SYSTEM_DATABASE, system_database);
system_database = std::make_shared<DatabaseMemory>(database_name, context);
DatabaseCatalog::instance().attachDatabase(database_name, system_database);
}
attachSystemTablesLocal(*system_database);
return system_database;
}
int LocalServer::main(const std::vector<std::string> & /*args*/)
try
{
@ -246,6 +245,8 @@ try
/// Sets external authenticators config (LDAP, Kerberos).
global_context->setExternalAuthenticatorsConfig(config());
global_context->initializeBackgroundExecutors();
setupUsers();
/// Limit on total number of concurrently executing queries.
@ -301,14 +302,18 @@ try
fs::create_directories(fs::path(path) / "data/");
fs::create_directories(fs::path(path) / "metadata/");
loadMetadataSystem(global_context);
attachSystemTables(global_context);
attachSystemTablesLocal(*createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE));
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA));
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE));
loadMetadata(global_context);
DatabaseCatalog::instance().loadDatabases();
LOG_DEBUG(log, "Loaded metadata.");
}
else if (!config().has("no-system-tables"))
{
attachSystemTables(global_context);
attachSystemTablesLocal(*createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE));
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA));
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE));
}
processQueries();

View File

@ -14,7 +14,7 @@
#include <Poco/Net/NetException.h>
#include <Poco/Util/HelpFormatter.h>
#include <Poco/Environment.h>
#include <common/scope_guard.h>
#include <common/scope_guard_safe.h>
#include <common/defines.h>
#include <common/logger_useful.h>
#include <common/phdr_cache.h>
@ -56,6 +56,7 @@
#include <Access/AccessControlManager.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/System/attachSystemTables.h>
#include <Storages/System/attachInformationSchemaTables.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Functions/registerFunctions.h>
#include <TableFunctions/registerTableFunctions.h>
@ -547,6 +548,8 @@ if (ThreadFuzzer::instance().isEffective())
// ignore `max_thread_pool_size` in configs we fetch from ZK, but oh well.
GlobalThreadPool::initialize(config().getUInt("max_thread_pool_size", 10000));
global_context->initializeBackgroundExecutors();
ConnectionCollector::init(global_context, config().getUInt("max_threads_for_connection_collector", 10));
bool has_zookeeper = config().has("zookeeper");
@ -1129,6 +1132,8 @@ if (ThreadFuzzer::instance().isEffective())
global_context->setSystemZooKeeperLogAfterInitializationIfNeeded();
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
attachSystemTablesServer(*database_catalog.getSystemDatabase(), has_zookeeper);
attachInformationSchema(global_context, *database_catalog.getDatabase(DatabaseCatalog::INFORMATION_SCHEMA));
attachInformationSchema(global_context, *database_catalog.getDatabase(DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE));
/// Firstly remove partially dropped databases, to avoid race with MaterializedMySQLSyncThread,
/// that may execute DROP before loadMarkedAsDroppedTables() in background,
/// and so loadMarkedAsDroppedTables() will find it and try to add, and UUID will overlap.
@ -1508,7 +1513,7 @@ if (ThreadFuzzer::instance().isEffective())
server.start();
LOG_INFO(log, "Ready for connections.");
SCOPE_EXIT({
SCOPE_EXIT_SAFE({
LOG_DEBUG(log, "Received termination signal.");
LOG_DEBUG(log, "Waiting for current connections to close.");

View File

@ -119,8 +119,10 @@ namespace
AccessRights res = access;
res.modifyFlags(modifier);
/// Anyone has access to the "system" database.
/// Anyone has access to the "system" and "information_schema" database.
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE);
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA);
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE);
return res;
}

View File

@ -349,6 +349,14 @@ dbms_target_link_libraries (
clickhouse_common_io
)
if (NOT_UNBUNDLED)
dbms_target_link_libraries (
PUBLIC
boost::circular_buffer
boost::heap
)
endif()
target_include_directories(clickhouse_common_io PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/Core/include") # uses some includes from core
dbms_target_include_directories(PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/Core/include")

View File

@ -31,8 +31,8 @@ public:
/// probably it worth to try to increase stack size for coroutines.
///
/// Current value is just enough for all tests in our CI. It's not selected in some special
/// way. We will have 36 pages with 4KB page size.
static constexpr size_t default_stack_size = 144 * 1024; /// 64KB was not enough for tests
/// way. We will have 40 pages with 4KB page size.
static constexpr size_t default_stack_size = 192 * 1024; /// 64KB was not enough for tests
explicit FiberStack(size_t stack_size_ = default_stack_size) : stack_size(stack_size_)
{

View File

@ -62,8 +62,8 @@ private:
void logMemoryUsage(Int64 current) const;
public:
MemoryTracker(VariableContext level_ = VariableContext::Thread);
MemoryTracker(MemoryTracker * parent_, VariableContext level_ = VariableContext::Thread);
explicit MemoryTracker(VariableContext level_ = VariableContext::Thread);
explicit MemoryTracker(MemoryTracker * parent_, VariableContext level_ = VariableContext::Thread);
~MemoryTracker();

View File

@ -74,6 +74,8 @@ void ThreadPoolImpl<Thread>::setQueueSize(size_t value)
{
std::lock_guard lock(mutex);
queue_size = value;
/// Reserve memory to get rid of allocations
jobs.reserve(queue_size);
}
@ -247,7 +249,7 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
if (!jobs.empty())
{
/// std::priority_queue does not provide interface for getting non-const reference to an element
/// boost::priority_queue does not provide interface for getting non-const reference to an element
/// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job.
job = std::move(const_cast<Job &>(jobs.top().job));
jobs.pop();
@ -257,6 +259,7 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
/// shutdown is true, simply finish the thread.
return;
}
}
if (!need_shutdown)

View File

@ -9,6 +9,8 @@
#include <list>
#include <optional>
#include <boost/heap/priority_queue.hpp>
#include <Poco/Event.h>
#include <Common/ThreadStatus.h>
#include <common/scope_guard.h>
@ -103,11 +105,10 @@ private:
}
};
std::priority_queue<JobWithPriority> jobs;
boost::heap::priority_queue<JobWithPriority> jobs;
std::list<Thread> threads;
std::exception_ptr first_exception;
template <typename ReturnType>
ReturnType scheduleImpl(Job job, int priority, std::optional<uint64_t> wait_microseconds);

View File

@ -22,6 +22,10 @@ namespace ErrorCodes
}
/// Cache thread_name to avoid prctl(PR_GET_NAME) for query_log/text_log
static thread_local std::string thread_name;
void setThreadName(const char * name)
{
#ifndef NDEBUG
@ -40,24 +44,29 @@ void setThreadName(const char * name)
if (0 != prctl(PR_SET_NAME, name, 0, 0, 0))
#endif
DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME, ...)", DB::ErrorCodes::PTHREAD_ERROR);
thread_name = name;
}
std::string getThreadName()
const std::string & getThreadName()
{
std::string name(16, '\0');
if (!thread_name.empty())
return thread_name;
thread_name.resize(16);
#if defined(__APPLE__) || defined(OS_SUNOS)
if (pthread_getname_np(pthread_self(), name.data(), name.size()))
if (pthread_getname_np(pthread_self(), thread_name.data(), thread_name.size()))
throw DB::Exception("Cannot get thread name with pthread_getname_np()", DB::ErrorCodes::PTHREAD_ERROR);
#elif defined(__FreeBSD__)
// TODO: make test. freebsd will have this function soon https://freshbsd.org/commit/freebsd/r337983
// if (pthread_get_name_np(pthread_self(), name.data(), name.size()))
// if (pthread_get_name_np(pthread_self(), thread_name.data(), thread_name.size()))
// throw DB::Exception("Cannot get thread name with pthread_get_name_np()", DB::ErrorCodes::PTHREAD_ERROR);
#else
if (0 != prctl(PR_GET_NAME, name.data(), 0, 0, 0))
if (0 != prctl(PR_GET_NAME, thread_name.data(), 0, 0, 0))
DB::throwFromErrno("Cannot get thread name with prctl(PR_GET_NAME)", DB::ErrorCodes::PTHREAD_ERROR);
#endif
name.resize(std::strlen(name.data()));
return name;
thread_name.resize(std::strlen(thread_name.data()));
return thread_name;
}

View File

@ -7,4 +7,4 @@
*/
void setThreadName(const char * name);
std::string getThreadName();
const std::string & getThreadName();

View File

@ -128,5 +128,8 @@
/// Default limit on recursion depth of recursive descend parser.
#define DBMS_DEFAULT_MAX_PARSER_DEPTH 1000
/// Default limit on query size.
#define DBMS_DEFAULT_MAX_QUERY_SIZE 262144
/// Max depth of hierarchical dictionary
#define DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH 1000

View File

@ -48,7 +48,7 @@ class IColumn;
M(MaxThreads, max_alter_threads, 0, "The maximum number of threads to execute the ALTER requests. By default, it is determined automatically.", 0) \
M(UInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.", 0) \
M(UInt64, max_distributed_connections, 1024, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).", 0) \
M(UInt64, max_query_size, 262144, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)", 0) \
M(UInt64, max_query_size, DBMS_DEFAULT_MAX_QUERY_SIZE, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)", 0) \
M(UInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \
M(Seconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \
M(Milliseconds, connect_timeout_with_failover_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS, "Connection timeout for selecting first healthy replica.", 0) \

View File

@ -786,7 +786,7 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl(
AggregatedDataWithoutKey & res,
size_t rows,
AggregateFunctionInstruction * aggregate_instructions,
Arena * arena)
Arena * arena) const
{
#if USE_EMBEDDED_COMPILER
if constexpr (use_compiled_functions)
@ -865,7 +865,7 @@ void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl(
void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns & aggregate_columns, Columns & materialized_columns,
AggregateFunctionInstructions & aggregate_functions_instructions, NestedColumnsHolder & nested_columns_holder)
AggregateFunctionInstructions & aggregate_functions_instructions, NestedColumnsHolder & nested_columns_holder) const
{
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_columns[i].resize(params.aggregates[i].arguments.size());
@ -917,7 +917,7 @@ void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns
bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & result,
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, bool & no_more_keys)
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, bool & no_more_keys) const
{
UInt64 num_rows = block.rows();
return executeOnBlock(block.getColumns(), num_rows, result, key_columns, aggregate_columns, no_more_keys);
@ -925,7 +925,7 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedDataVariants & result,
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, bool & no_more_keys)
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, bool & no_more_keys) const
{
/// `result` will destroy the states of aggregate functions in the destructor
result.aggregator = this;
@ -1058,7 +1058,7 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData
}
void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, const String & tmp_path)
void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, const String & tmp_path) const
{
Stopwatch watch;
size_t rows = data_variants.size();
@ -1130,7 +1130,7 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, co
}
void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants)
void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants) const
{
String tmp_path = params.tmp_volume->getDisk()->getPath();
return writeToTemporaryFile(data_variants, tmp_path);
@ -1192,7 +1192,7 @@ template <typename Method>
void Aggregator::writeToTemporaryFileImpl(
AggregatedDataVariants & data_variants,
Method & method,
IBlockOutputStream & out)
IBlockOutputStream & out) const
{
size_t max_temporary_block_size_rows = 0;
size_t max_temporary_block_size_bytes = 0;
@ -2311,7 +2311,7 @@ void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
block.clear();
}
bool Aggregator::mergeBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys)
bool Aggregator::mergeOnBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys) const
{
/// `result` will destroy the states of aggregate functions in the destructor
result.aggregator = this;
@ -2661,7 +2661,7 @@ void NO_INLINE Aggregator::convertBlockToTwoLevelImpl(
}
std::vector<Block> Aggregator::convertBlockToTwoLevel(const Block & block)
std::vector<Block> Aggregator::convertBlockToTwoLevel(const Block & block) const
{
if (!block)
return {};
@ -2753,7 +2753,7 @@ void Aggregator::destroyWithoutKey(AggregatedDataVariants & result) const
}
void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result)
void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result) const
{
if (result.empty())
return;

View File

@ -506,7 +506,7 @@ struct AggregatedDataVariants : private boost::noncopyable
* But this can hardly be done simply because it is planned to put variable-length strings into the same pool.
* In this case, the pool will not be able to know with what offsets objects are stored.
*/
Aggregator * aggregator = nullptr;
const Aggregator * aggregator = nullptr;
size_t keys_size{}; /// Number of keys. NOTE do we need this field?
Sizes key_sizes; /// Dimensions of keys, if keys of fixed length
@ -975,11 +975,14 @@ public:
/// Process one block. Return false if the processing should be aborted (with group_by_overflow_mode = 'break').
bool executeOnBlock(const Block & block, AggregatedDataVariants & result,
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block
bool & no_more_keys);
bool & no_more_keys) const;
bool executeOnBlock(Columns columns, UInt64 num_rows, AggregatedDataVariants & result,
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block
bool & no_more_keys);
bool & no_more_keys) const;
/// Used for aggregate projection.
bool mergeOnBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys) const;
/** Convert the aggregation data structure into a block.
* If overflow_row = true, then aggregates for rows that are not included in max_rows_to_group_by are put in the first block.
@ -996,8 +999,6 @@ public:
/// Merge partially aggregated blocks separated to buckets into one data structure.
void mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVariants & result, size_t max_threads);
bool mergeBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys);
/// Merge several partially aggregated blocks into one.
/// Precondition: for all blocks block.info.is_overflows flag must be the same.
/// (either all blocks are from overflow data or none blocks are).
@ -1007,11 +1008,11 @@ public:
/** Split block with partially-aggregated data to many blocks, as if two-level method of aggregation was used.
* This is needed to simplify merging of that data with other results, that are already two-level.
*/
std::vector<Block> convertBlockToTwoLevel(const Block & block);
std::vector<Block> convertBlockToTwoLevel(const Block & block) const;
/// For external aggregation.
void writeToTemporaryFile(AggregatedDataVariants & data_variants, const String & tmp_path);
void writeToTemporaryFile(AggregatedDataVariants & data_variants);
void writeToTemporaryFile(AggregatedDataVariants & data_variants, const String & tmp_path) const;
void writeToTemporaryFile(AggregatedDataVariants & data_variants) const;
bool hasTemporaryFiles() const { return !temporary_files.empty(); }
@ -1083,7 +1084,7 @@ private:
Poco::Logger * log = &Poco::Logger::get("Aggregator");
/// For external aggregation.
TemporaryFiles temporary_files;
mutable TemporaryFiles temporary_files;
#if USE_EMBEDDED_COMPILER
std::shared_ptr<CompiledAggregateFunctionsHolder> compiled_aggregate_functions_holder;
@ -1106,7 +1107,7 @@ private:
/** Call `destroy` methods for states of aggregate functions.
* Used in the exception handler for aggregation, since RAII in this case is not applicable.
*/
void destroyAllAggregateStates(AggregatedDataVariants & result);
void destroyAllAggregateStates(AggregatedDataVariants & result) const;
/// Process one data block, aggregate the data into a hash table.
@ -1136,7 +1137,7 @@ private:
AggregatedDataWithoutKey & res,
size_t rows,
AggregateFunctionInstruction * aggregate_instructions,
Arena * arena);
Arena * arena) const;
static void executeOnIntervalWithoutKeyImpl(
AggregatedDataWithoutKey & res,
@ -1149,7 +1150,7 @@ private:
void writeToTemporaryFileImpl(
AggregatedDataVariants & data_variants,
Method & method,
IBlockOutputStream & out);
IBlockOutputStream & out) const;
/// Merge NULL key data from hash table `src` into `dst`.
template <typename Method, typename Table>
@ -1304,7 +1305,7 @@ private:
AggregateColumns & aggregate_columns,
Columns & materialized_columns,
AggregateFunctionInstructions & instructions,
NestedColumnsHolder & nested_columns_holder);
NestedColumnsHolder & nested_columns_holder) const;
void addSingleKeyToAggregateColumns(
const AggregatedDataVariants & data_variants,

View File

@ -60,7 +60,7 @@ public:
private:
static void visit(const ASTSelectQuery & node, ASTPtr &, Data & data)
{
ASTPtr array_join_expression_list = node.arrayJoinExpressionList();
auto [array_join_expression_list, _] = node.arrayJoinExpressionList();
if (!array_join_expression_list)
throw Exception("Logical error: no ARRAY JOIN", ErrorCodes::LOGICAL_ERROR);

View File

@ -78,7 +78,7 @@
#include <Common/RemoteHostFilter.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
#include <Storages/MergeTree/BackgroundJobsExecutor.h>
#include <Storages/MergeTree/BackgroundJobsAssignee.h>
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
#include <Interpreters/SynonymsExtensions.h>
#include <Interpreters/Lemmatizers.h>
@ -101,6 +101,13 @@ namespace CurrentMetrics
extern const Metric BackgroundBufferFlushSchedulePoolTask;
extern const Metric BackgroundDistributedSchedulePoolTask;
extern const Metric BackgroundMessageBrokerSchedulePoolTask;
extern const Metric DelayedInserts;
extern const Metric BackgroundPoolTask;
extern const Metric BackgroundMovePoolTask;
extern const Metric BackgroundFetchesPoolTask;
}
namespace DB
@ -223,6 +230,11 @@ struct ContextSharedPart
std::optional<StorageS3Settings> storage_s3_settings; /// Settings of S3 storage
std::vector<String> warnings; /// Store warning messages about server configuration.
/// Background executors for *MergeTree tables
MergeTreeBackgroundExecutorPtr merge_mutate_executor;
MergeTreeBackgroundExecutorPtr moves_executor;
MergeTreeBackgroundExecutorPtr fetch_executor;
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
std::optional<TraceCollector> trace_collector; /// Thread collecting traces from threads executing queries
@ -292,6 +304,13 @@ struct ContextSharedPart
DatabaseCatalog::shutdown();
if (merge_mutate_executor)
merge_mutate_executor->wait();
if (fetch_executor)
fetch_executor->wait();
if (moves_executor)
moves_executor->wait();
std::unique_ptr<SystemLogs> delete_system_logs;
{
auto lock = std::lock_guard(mutex);
@ -2718,6 +2737,53 @@ PartUUIDsPtr Context::getIgnoredPartUUIDs() const
}
void Context::initializeBackgroundExecutors()
{
// Initialize background executors with callbacks to be able to change pool size and tasks count at runtime.
shared->merge_mutate_executor = MergeTreeBackgroundExecutor::create
(
MergeTreeBackgroundExecutor::Type::MERGE_MUTATE,
getSettingsRef().background_pool_size,
getSettingsRef().background_pool_size,
CurrentMetrics::BackgroundPoolTask
);
shared->moves_executor = MergeTreeBackgroundExecutor::create
(
MergeTreeBackgroundExecutor::Type::MOVE,
getSettingsRef().background_move_pool_size,
getSettingsRef().background_move_pool_size,
CurrentMetrics::BackgroundMovePoolTask
);
shared->fetch_executor = MergeTreeBackgroundExecutor::create
(
MergeTreeBackgroundExecutor::Type::FETCH,
getSettingsRef().background_fetches_pool_size,
getSettingsRef().background_fetches_pool_size,
CurrentMetrics::BackgroundFetchesPoolTask
);
}
MergeTreeBackgroundExecutorPtr Context::getMergeMutateExecutor() const
{
return shared->merge_mutate_executor;
}
MergeTreeBackgroundExecutorPtr Context::getMovesExecutor() const
{
return shared->moves_executor;
}
MergeTreeBackgroundExecutorPtr Context::getFetchesExecutor() const
{
return shared->fetch_executor;
}
ReadSettings Context::getReadSettings() const
{
ReadSettings res;

View File

@ -101,6 +101,8 @@ using StoragePolicyPtr = std::shared_ptr<const IStoragePolicy>;
using StoragePoliciesMap = std::map<String, StoragePolicyPtr>;
class StoragePolicySelector;
using StoragePolicySelectorPtr = std::shared_ptr<const StoragePolicySelector>;
class MergeTreeBackgroundExecutor;
using MergeTreeBackgroundExecutorPtr = std::shared_ptr<MergeTreeBackgroundExecutor>;
struct PartUUIDs;
using PartUUIDsPtr = std::shared_ptr<PartUUIDs>;
class KeeperDispatcher;
@ -830,6 +832,13 @@ public:
ReadTaskCallback getReadTaskCallback() const;
void setReadTaskCallback(ReadTaskCallback && callback);
/// Background executors related methods
void initializeBackgroundExecutors();
MergeTreeBackgroundExecutorPtr getMergeMutateExecutor() const;
MergeTreeBackgroundExecutorPtr getMovesExecutor() const;
MergeTreeBackgroundExecutorPtr getFetchesExecutor() const;
/** Get settings for reading from filesystem. */
ReadSettings getReadSettings() const;

View File

@ -123,6 +123,8 @@ class DatabaseCatalog : boost::noncopyable, WithMutableContext
public:
static constexpr const char * TEMPORARY_DATABASE = "_temporary_and_external_tables";
static constexpr const char * SYSTEM_DATABASE = "system";
static constexpr const char * INFORMATION_SCHEMA = "information_schema";
static constexpr const char * INFORMATION_SCHEMA_UPPERCASE = "INFORMATION_SCHEMA";
static DatabaseCatalog & init(ContextMutablePtr global_context_);
static DatabaseCatalog & instance();

View File

@ -1042,7 +1042,7 @@ ExpressionActionsChain::JoinStep::JoinStep(
required_columns.emplace_back(column.name, column.type);
NamesAndTypesList result_names_and_types = required_columns;
analyzed_join->addJoinedColumnsAndCorrectTypes(result_names_and_types);
analyzed_join->addJoinedColumnsAndCorrectTypes(result_names_and_types, true);
for (const auto & [name, type] : result_names_and_types)
/// `column` is `nullptr` because we don't care on constness here, it may be changed in join
result_columns.emplace_back(nullptr, type, name);

View File

@ -43,7 +43,6 @@
#include <Common/StringUtils/StringUtils.h>
#include <DataTypes/DataTypeFactory.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/GetAggregatesVisitor.h>
@ -152,6 +151,9 @@ ExpressionAnalyzer::ExpressionAnalyzer(
/// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers.
initGlobalSubqueriesAndExternalTables(do_global);
auto temp_actions = std::make_shared<ActionsDAG>(sourceColumns());
columns_after_array_join = getColumnsAfterArrayJoin(temp_actions, sourceColumns());
columns_after_join = analyzeJoin(temp_actions, columns_after_array_join);
/// has_aggregation, aggregation_keys, aggregate_descriptions, aggregated_columns.
/// This analysis should be performed after processing global subqueries, because otherwise,
/// if the aggregate function contains a global subquery, then `analyzeAggregation` method will save
@ -159,7 +161,7 @@ ExpressionAnalyzer::ExpressionAnalyzer(
/// global subquery. Then, when you call `initGlobalSubqueriesAndExternalTables` method, this
/// the global subquery will be replaced with a temporary table, resulting in aggregate_descriptions
/// will contain out-of-date information, which will lead to an error when the query is executed.
analyzeAggregation();
analyzeAggregation(temp_actions);
}
static ASTPtr checkPositionalArgument(ASTPtr argument, const ASTSelectQuery * select_query, ASTSelectQuery::Expression expression)
@ -193,7 +195,64 @@ static ASTPtr checkPositionalArgument(ASTPtr argument, const ASTSelectQuery * se
return nullptr;
}
void ExpressionAnalyzer::analyzeAggregation()
NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns)
{
const auto * select_query = query->as<ASTSelectQuery>();
if (!select_query)
return {};
auto [array_join_expression_list, is_array_join_left] = select_query->arrayJoinExpressionList();
if (!array_join_expression_list)
return src_columns;
getRootActionsNoMakeSet(array_join_expression_list, true, actions, false);
auto array_join = addMultipleArrayJoinAction(actions, is_array_join_left);
auto sample_columns = actions->getResultColumns();
array_join->prepare(sample_columns);
actions = std::make_shared<ActionsDAG>(sample_columns);
NamesAndTypesList new_columns_after_array_join;
NameSet added_columns;
for (auto & column : actions->getResultColumns())
{
if (syntax->array_join_result_to_source.count(column.name))
{
new_columns_after_array_join.emplace_back(column.name, column.type);
added_columns.emplace(column.name);
}
}
for (const auto & column : src_columns)
if (added_columns.count(column.name) == 0)
new_columns_after_array_join.emplace_back(column.name, column.type);
return new_columns_after_array_join;
}
NamesAndTypesList ExpressionAnalyzer::analyzeJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns)
{
const auto * select_query = query->as<ASTSelectQuery>();
if (!select_query)
return {};
const ASTTablesInSelectQueryElement * join = select_query->join();
if (join)
{
getRootActionsNoMakeSet(analyzedJoin().leftKeysList(), true, actions, false);
auto sample_columns = actions->getNamesAndTypesList();
syntax->analyzed_join->addJoinedColumnsAndCorrectTypes(sample_columns, true);
actions = std::make_shared<ActionsDAG>(sample_columns);
}
NamesAndTypesList result_columns = src_columns;
syntax->analyzed_join->addJoinedColumnsAndCorrectTypes(result_columns,false);
return result_columns;
}
void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
{
/** Find aggregation keys (aggregation_keys), information about aggregate functions (aggregate_descriptions),
* as well as a set of columns obtained after the aggregation, if any,
@ -204,146 +263,90 @@ void ExpressionAnalyzer::analyzeAggregation()
auto * select_query = query->as<ASTSelectQuery>();
auto temp_actions = std::make_shared<ActionsDAG>(sourceColumns());
makeAggregateDescriptions(temp_actions, aggregate_descriptions);
has_aggregation = !aggregate_descriptions.empty() || (select_query && (select_query->groupBy() || select_query->having()));
if (select_query)
if (!has_aggregation)
{
NamesAndTypesList array_join_columns;
columns_after_array_join = sourceColumns();
bool is_array_join_left;
if (ASTPtr array_join_expression_list = select_query->arrayJoinExpressionList(is_array_join_left))
{
getRootActionsNoMakeSet(array_join_expression_list, true, temp_actions, false);
auto array_join = addMultipleArrayJoinAction(temp_actions, is_array_join_left);
auto sample_columns = temp_actions->getResultColumns();
array_join->prepare(sample_columns);
temp_actions = std::make_shared<ActionsDAG>(sample_columns);
NamesAndTypesList new_columns_after_array_join;
NameSet added_columns;
for (auto & column : temp_actions->getResultColumns())
{
if (syntax->array_join_result_to_source.count(column.name))
{
new_columns_after_array_join.emplace_back(column.name, column.type);
added_columns.emplace(column.name);
}
}
for (auto & column : columns_after_array_join)
if (added_columns.count(column.name) == 0)
new_columns_after_array_join.emplace_back(column.name, column.type);
columns_after_array_join.swap(new_columns_after_array_join);
}
columns_after_array_join.insert(columns_after_array_join.end(), array_join_columns.begin(), array_join_columns.end());
const ASTTablesInSelectQueryElement * join = select_query->join();
if (join)
{
getRootActionsNoMakeSet(analyzedJoin().leftKeysList(), true, temp_actions, false);
auto sample_columns = temp_actions->getNamesAndTypesList();
analyzedJoin().addJoinedColumnsAndCorrectTypes(sample_columns);
temp_actions = std::make_shared<ActionsDAG>(sample_columns);
}
columns_after_join = columns_after_array_join;
analyzedJoin().addJoinedColumnsAndCorrectTypes(columns_after_join, false);
aggregated_columns = temp_actions->getNamesAndTypesList();
return;
}
has_aggregation = makeAggregateDescriptions(temp_actions);
if (select_query && (select_query->groupBy() || select_query->having()))
has_aggregation = true;
if (has_aggregation)
/// Find out aggregation keys.
if (select_query)
{
/// Find out aggregation keys.
if (select_query)
if (ASTPtr group_by_ast = select_query->groupBy())
{
if (select_query->groupBy())
NameSet unique_keys;
ASTs & group_asts = group_by_ast->children;
for (ssize_t i = 0; i < ssize_t(group_asts.size()); ++i)
{
NameSet unique_keys;
ASTs & group_asts = select_query->groupBy()->children;
ssize_t size = group_asts.size();
getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false);
for (ssize_t i = 0; i < ssize_t(group_asts.size()); ++i)
if (getContext()->getSettingsRef().enable_positional_arguments)
{
ssize_t size = group_asts.size();
getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false);
auto new_argument = checkPositionalArgument(group_asts[i], select_query, ASTSelectQuery::Expression::GROUP_BY);
if (new_argument)
group_asts[i] = new_argument;
}
if (getContext()->getSettingsRef().enable_positional_arguments)
const auto & column_name = group_asts[i]->getColumnName();
const auto * node = temp_actions->tryFindInIndex(column_name);
if (!node)
throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER);
/// Only removes constant keys if it's an initiator or distributed_group_by_no_merge is enabled.
if (getContext()->getClientInfo().distributed_depth == 0 || settings.distributed_group_by_no_merge > 0)
{
/// Constant expressions have non-null column pointer at this stage.
if (node->column && isColumnConst(*node->column))
{
auto new_argument = checkPositionalArgument(group_asts[i], select_query, ASTSelectQuery::Expression::GROUP_BY);
if (new_argument)
group_asts[i] = new_argument;
}
select_query->group_by_with_constant_keys = true;
const auto & column_name = group_asts[i]->getColumnName();
const auto * node = temp_actions->tryFindInIndex(column_name);
if (!node)
throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER);
/// Only removes constant keys if it's an initiator or distributed_group_by_no_merge is enabled.
if (getContext()->getClientInfo().distributed_depth == 0 || settings.distributed_group_by_no_merge > 0)
{
/// Constant expressions have non-null column pointer at this stage.
if (node->column && isColumnConst(*node->column))
/// But don't remove last key column if no aggregate functions, otherwise aggregation will not work.
if (!aggregate_descriptions.empty() || size > 1)
{
select_query->group_by_with_constant_keys = true;
if (i + 1 < static_cast<ssize_t>(size))
group_asts[i] = std::move(group_asts.back());
/// But don't remove last key column if no aggregate functions, otherwise aggregation will not work.
if (!aggregate_descriptions.empty() || size > 1)
{
if (i + 1 < static_cast<ssize_t>(size))
group_asts[i] = std::move(group_asts.back());
group_asts.pop_back();
group_asts.pop_back();
--i;
continue;
}
--i;
continue;
}
}
NameAndTypePair key{column_name, node->result_type};
/// Aggregation keys are uniqued.
if (!unique_keys.count(key.name))
{
unique_keys.insert(key.name);
aggregation_keys.push_back(key);
/// Key is no longer needed, therefore we can save a little by moving it.
aggregated_columns.push_back(std::move(key));
}
}
if (group_asts.empty())
NameAndTypePair key{column_name, node->result_type};
/// Aggregation keys are uniqued.
if (!unique_keys.count(key.name))
{
select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, {});
has_aggregation = select_query->having() || !aggregate_descriptions.empty();
unique_keys.insert(key.name);
aggregation_keys.push_back(key);
/// Key is no longer needed, therefore we can save a little by moving it.
aggregated_columns.push_back(std::move(key));
}
}
if (group_asts.empty())
{
select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, {});
has_aggregation = select_query->having() || !aggregate_descriptions.empty();
}
}
else
aggregated_columns = temp_actions->getNamesAndTypesList();
/// Constant expressions are already removed during first 'analyze' run.
/// So for second `analyze` information is taken from select_query.
if (select_query)
has_const_aggregation_keys = select_query->group_by_with_constant_keys;
for (const auto & desc : aggregate_descriptions)
aggregated_columns.emplace_back(desc.column_name, desc.function->getReturnType());
has_const_aggregation_keys = select_query->group_by_with_constant_keys;
}
else
{
aggregated_columns = temp_actions->getNamesAndTypesList();
}
for (const auto & desc : aggregate_descriptions)
aggregated_columns.emplace_back(desc.column_name, desc.function->getReturnType());
}
@ -487,7 +490,7 @@ void ExpressionAnalyzer::getRootActionsForHaving(const ASTPtr & ast, bool no_sub
}
bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, AggregateDescriptions & descriptions)
{
for (const ASTFunction * node : aggregates())
{
@ -520,10 +523,8 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
aggregate.parameters = (node->parameters) ? getAggregateFunctionParametersArray(node->parameters, "", getContext()) : Array();
aggregate.function = AggregateFunctionFactory::instance().get(node->name, types, aggregate.parameters, properties);
aggregate_descriptions.push_back(aggregate);
descriptions.push_back(aggregate);
}
return !aggregates().empty();
}
void makeWindowDescriptionFromAST(const Context & context,
@ -804,8 +805,7 @@ ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActi
{
const auto * select_query = getSelectQuery();
bool is_array_join_left;
ASTPtr array_join_expression_list = select_query->arrayJoinExpressionList(is_array_join_left);
auto [array_join_expression_list, is_array_join_left] = select_query->arrayJoinExpressionList();
if (!array_join_expression_list)
return nullptr;
@ -832,14 +832,14 @@ bool SelectQueryExpressionAnalyzer::appendJoinLeftKeys(ExpressionActionsChain &
return true;
}
JoinPtr SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain)
JoinPtr SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, ActionsDAGPtr & converting_join_columns)
{
const ColumnsWithTypeAndName & left_sample_columns = chain.getLastStep().getResultColumns();
JoinPtr table_join = makeTableJoin(*syntax->ast_join, left_sample_columns);
JoinPtr table_join = makeTableJoin(*syntax->ast_join, left_sample_columns, converting_join_columns);
if (syntax->analyzed_join->needConvert())
if (converting_join_columns)
{
chain.steps.push_back(std::make_unique<ExpressionActionsChain::ExpressionActionsStep>(syntax->analyzed_join->leftConvertingActions()));
chain.steps.push_back(std::make_unique<ExpressionActionsChain::ExpressionActionsStep>(converting_join_columns));
chain.addStep();
}
@ -850,14 +850,6 @@ JoinPtr SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain
return table_join;
}
static JoinPtr tryGetStorageJoin(std::shared_ptr<TableJoin> analyzed_join)
{
if (auto * table = analyzed_join->joined_storage.get())
if (auto * storage_join = dynamic_cast<StorageJoin *>(table))
return storage_join->getJoinLocked(analyzed_join);
return {};
}
static ActionsDAGPtr createJoinedBlockActions(ContextPtr context, const TableJoin & analyzed_join)
{
ASTPtr expression_list = analyzed_join.rightKeysList();
@ -865,44 +857,13 @@ static ActionsDAGPtr createJoinedBlockActions(ContextPtr context, const TableJoi
return ExpressionAnalyzer(expression_list, syntax_result, context).getActionsDAG(true, false);
}
static bool allowDictJoin(StoragePtr joined_storage, ContextPtr context, String & dict_name, String & key_name)
static std::shared_ptr<IJoin> chooseJoinAlgorithm(std::shared_ptr<TableJoin> analyzed_join, const Block & sample_block, ContextPtr context)
{
if (!joined_storage->isDictionary())
return false;
StorageDictionary & storage_dictionary = static_cast<StorageDictionary &>(*joined_storage);
dict_name = storage_dictionary.getDictionaryName();
auto dictionary = context->getExternalDictionariesLoader().getDictionary(dict_name, context);
if (!dictionary)
return false;
const DictionaryStructure & structure = dictionary->getStructure();
if (structure.id)
{
key_name = structure.id->name;
return true;
}
return false;
}
static std::shared_ptr<IJoin> makeJoin(std::shared_ptr<TableJoin> analyzed_join, const Block & sample_block, ContextPtr context)
{
bool allow_merge_join = analyzed_join->allowMergeJoin();
/// HashJoin with Dictionary optimisation
String dict_name;
String key_name;
if (analyzed_join->joined_storage && allowDictJoin(analyzed_join->joined_storage, context, dict_name, key_name))
{
Names original_names;
NamesAndTypesList result_columns;
if (analyzed_join->allowDictJoin(key_name, sample_block, original_names, result_columns))
{
analyzed_join->dictionary_reader = std::make_shared<DictionaryReader>(dict_name, original_names, result_columns, context);
return std::make_shared<HashJoin>(analyzed_join, sample_block);
}
}
if (analyzed_join->tryInitDictJoin(sample_block, context))
return std::make_shared<HashJoin>(analyzed_join, sample_block);
bool allow_merge_join = analyzed_join->allowMergeJoin();
if (analyzed_join->forceHashJoin() || (analyzed_join->preferMergeJoin() && !allow_merge_join))
return std::make_shared<HashJoin>(analyzed_join, sample_block);
else if (analyzed_join->forceMergeJoin() || (analyzed_join->preferMergeJoin() && allow_merge_join))
@ -910,79 +871,91 @@ static std::shared_ptr<IJoin> makeJoin(std::shared_ptr<TableJoin> analyzed_join,
return std::make_shared<JoinSwitcher>(analyzed_join, sample_block);
}
static std::unique_ptr<QueryPlan> buildJoinedPlan(
ContextPtr context,
const ASTTablesInSelectQueryElement & join_element,
TableJoin & analyzed_join,
SelectQueryOptions query_options)
{
/// Actions which need to be calculated on joined block.
auto joined_block_actions = createJoinedBlockActions(context, analyzed_join);
Names original_right_columns;
NamesWithAliases required_columns_with_aliases = analyzed_join.getRequiredColumns(
Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames());
for (auto & pr : required_columns_with_aliases)
original_right_columns.push_back(pr.first);
/** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs
* - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1,
* in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`.
* - this function shows the expression JOIN _data1.
*/
auto interpreter = interpretSubquery(
join_element.table_expression, context, original_right_columns, query_options.copy().setWithAllColumns());
auto joined_plan = std::make_unique<QueryPlan>();
interpreter->buildQueryPlan(*joined_plan);
{
auto sample_block = interpreter->getSampleBlock();
auto rename_dag = std::make_unique<ActionsDAG>(sample_block.getColumnsWithTypeAndName());
for (const auto & name_with_alias : required_columns_with_aliases)
{
if (sample_block.has(name_with_alias.first))
{
auto pos = sample_block.getPositionByName(name_with_alias.first);
const auto & alias = rename_dag->addAlias(*rename_dag->getInputs()[pos], name_with_alias.second);
rename_dag->getIndex()[pos] = &alias;
}
}
auto rename_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), std::move(rename_dag));
rename_step->setStepDescription("Rename joined columns");
joined_plan->addStep(std::move(rename_step));
}
auto joined_actions_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), std::move(joined_block_actions));
joined_actions_step->setStepDescription("Joined actions");
joined_plan->addStep(std::move(joined_actions_step));
return joined_plan;
}
JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin(
const ASTTablesInSelectQueryElement & join_element, const ColumnsWithTypeAndName & left_sample_columns)
const ASTTablesInSelectQueryElement & join_element,
const ColumnsWithTypeAndName & left_columns,
ActionsDAGPtr & left_convert_actions)
{
/// Two JOINs are not supported with the same subquery, but different USINGs.
if (joined_plan)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table join was already created for query");
/// Use StorageJoin if any.
JoinPtr join = tryGetStorageJoin(syntax->analyzed_join);
ActionsDAGPtr right_convert_actions = nullptr;
if (!join)
const auto & analyzed_join = syntax->analyzed_join;
if (auto storage = analyzed_join->getStorageJoin())
{
/// Actions which need to be calculated on joined block.
auto joined_block_actions = createJoinedBlockActions(getContext(), analyzedJoin());
Names original_right_columns;
NamesWithAliases required_columns_with_aliases = analyzedJoin().getRequiredColumns(
Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames());
for (auto & pr : required_columns_with_aliases)
original_right_columns.push_back(pr.first);
/** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs
* - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1,
* in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`.
* - this function shows the expression JOIN _data1.
*/
auto interpreter = interpretSubquery(
join_element.table_expression, getContext(), original_right_columns, query_options.copy().setWithAllColumns());
{
joined_plan = std::make_unique<QueryPlan>();
interpreter->buildQueryPlan(*joined_plan);
auto sample_block = interpreter->getSampleBlock();
auto rename_dag = std::make_unique<ActionsDAG>(sample_block.getColumnsWithTypeAndName());
for (const auto & name_with_alias : required_columns_with_aliases)
{
if (sample_block.has(name_with_alias.first))
{
auto pos = sample_block.getPositionByName(name_with_alias.first);
const auto & alias = rename_dag->addAlias(*rename_dag->getInputs()[pos], name_with_alias.second);
rename_dag->getIndex()[pos] = &alias;
}
}
auto rename_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), std::move(rename_dag));
rename_step->setStepDescription("Rename joined columns");
joined_plan->addStep(std::move(rename_step));
}
auto joined_actions_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), std::move(joined_block_actions));
joined_actions_step->setStepDescription("Joined actions");
joined_plan->addStep(std::move(joined_actions_step));
const ColumnsWithTypeAndName & right_sample_columns = joined_plan->getCurrentDataStream().header.getColumnsWithTypeAndName();
bool need_convert = syntax->analyzed_join->applyJoinKeyConvert(left_sample_columns, right_sample_columns);
if (need_convert)
{
auto converting_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), syntax->analyzed_join->rightConvertingActions());
converting_step->setStepDescription("Convert joined columns");
joined_plan->addStep(std::move(converting_step));
}
join = makeJoin(syntax->analyzed_join, joined_plan->getCurrentDataStream().header, getContext());
/// Do not make subquery for join over dictionary.
if (syntax->analyzed_join->dictionary_reader)
joined_plan.reset();
std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, {});
return storage->getJoinLocked(analyzed_join);
}
else
syntax->analyzed_join->applyJoinKeyConvert(left_sample_columns, {});
joined_plan = buildJoinedPlan(getContext(), join_element, *analyzed_join, query_options);
const ColumnsWithTypeAndName & right_columns = joined_plan->getCurrentDataStream().header.getColumnsWithTypeAndName();
std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, right_columns);
if (right_convert_actions)
{
auto converting_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), right_convert_actions);
converting_step->setStepDescription("Convert joined columns");
joined_plan->addStep(std::move(converting_step));
}
JoinPtr join = chooseJoinAlgorithm(analyzed_join, joined_plan->getCurrentDataStream().header, getContext());
/// Do not make subquery for join over dictionary.
if (analyzed_join->getDictionaryReader())
joined_plan.reset();
return join;
}
@ -1574,8 +1547,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
{
query_analyzer.appendJoinLeftKeys(chain, only_types || !first_stage);
before_join = chain.getLastActions();
join = query_analyzer.appendJoin(chain);
converting_join_columns = query_analyzer.analyzedJoin().leftConvertingActions();
join = query_analyzer.appendJoin(chain, converting_join_columns);
chain.addStep();
}

View File

@ -92,7 +92,7 @@ private:
const SizeLimits size_limits_for_set;
const UInt64 distributed_group_by_no_merge;
ExtractedSettings(const Settings & settings_);
explicit ExtractedSettings(const Settings & settings_);
};
public:
@ -188,12 +188,15 @@ protected:
* or after all the actions that are normally performed before aggregation.
* Set has_aggregation = true if there is GROUP BY or at least one aggregate function.
*/
void analyzeAggregation();
bool makeAggregateDescriptions(ActionsDAGPtr & actions);
void analyzeAggregation(ActionsDAGPtr & temp_actions);
void makeAggregateDescriptions(ActionsDAGPtr & actions, AggregateDescriptions & descriptions);
const ASTSelectQuery * getSelectQuery() const;
bool isRemoteStorage() const { return syntax->is_remote_storage; }
NamesAndTypesList getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns);
NamesAndTypesList analyzeJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns);
};
class SelectQueryExpressionAnalyzer;
@ -338,7 +341,8 @@ private:
JoinPtr makeTableJoin(
const ASTTablesInSelectQueryElement & join_element,
const ColumnsWithTypeAndName & left_sample_columns);
const ColumnsWithTypeAndName & left_columns,
ActionsDAGPtr & left_convert_actions);
const ASTSelectQuery * getAggregatingQuery() const;
@ -359,7 +363,8 @@ private:
/// Before aggregation:
ArrayJoinActionPtr appendArrayJoin(ExpressionActionsChain & chain, ActionsDAGPtr & before_array_join, bool only_types);
bool appendJoinLeftKeys(ExpressionActionsChain & chain, bool only_types);
JoinPtr appendJoin(ExpressionActionsChain & chain);
JoinPtr appendJoin(ExpressionActionsChain & chain, ActionsDAGPtr & converting_join_columns);
/// remove_filter is set in ExpressionActionsChain::finalize();
/// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier).
ActionsDAGPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns);

View File

@ -211,7 +211,7 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
if (nullable_right_side)
JoinCommon::convertColumnsToNullable(sample_block_with_columns_to_add);
if (table_join->dictionary_reader)
if (table_join->getDictionaryReader())
{
LOG_DEBUG(log, "Performing join over dict");
data->type = Type::DICT;
@ -331,7 +331,8 @@ public:
KeyGetterForDict(const TableJoin & table_join, const ColumnRawPtrs & key_columns)
{
table_join.dictionary_reader->readKeys(*key_columns[0], read_result, found, positions);
assert(table_join.getDictionaryReader());
table_join.getDictionaryReader()->readKeys(*key_columns[0], read_result, found, positions);
for (ColumnWithTypeAndName & column : read_result)
if (table_join.rightBecomeNullable(column.type))

View File

@ -854,7 +854,7 @@ static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery &
static UInt64 getLimitForSorting(const ASTSelectQuery & query, ContextPtr context)
{
/// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY, neither ARRAY JOIN.
if (!query.distinct && !query.limitBy() && !query.limit_with_ties && !query.arrayJoinExpressionList() && query.limitLength())
if (!query.distinct && !query.limitBy() && !query.limit_with_ties && !query.arrayJoinExpressionList().first && query.limitLength())
{
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context);
if (limit_length > std::numeric_limits<UInt64>::max() - limit_offset)
@ -1132,8 +1132,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
}
/// Optional step to convert key columns to common supertype.
/// Columns with changed types will be returned to user,
/// so its only suitable for `USING` join.
if (expressions.converting_join_columns)
{
QueryPlanStepPtr convert_join_step = std::make_unique<ExpressionStep>(
@ -1354,17 +1352,15 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
bool apply_prelimit = apply_limit &&
query.limitLength() && !query.limit_with_ties &&
!hasWithTotalsInAnySubqueryInFromClause(query) &&
!query.arrayJoinExpressionList() &&
!query.arrayJoinExpressionList().first &&
!query.distinct &&
!expressions.hasLimitBy() &&
!settings.extremes &&
!has_withfill;
bool apply_offset = options.to_stage != QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit;
bool limit_applied = false;
if (apply_prelimit)
{
executePreLimit(query_plan, /* do_not_skip_offset= */!apply_offset);
limit_applied = true;
}
/** If there was more than one stream,
@ -1386,7 +1382,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
if (query.limit_with_ties && apply_offset)
{
executeLimit(query_plan);
limit_applied = true;
}
/// Projection not be done on the shards, since then initiator will not find column in blocks.
@ -1400,6 +1395,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
/// Extremes are calculated before LIMIT, but after LIMIT BY. This is Ok.
executeExtremes(query_plan);
bool limit_applied = apply_prelimit || (query.limit_with_ties && apply_offset);
/// Limit is no longer needed if there is prelimit.
///
/// NOTE: that LIMIT cannot be applied if OFFSET should not be applied,

View File

@ -299,16 +299,17 @@ std::shared_ptr<TableJoin> JoinedTables::makeTableJoin(const ASTSelectQuery & se
if (table_to_join.database_and_table_name)
{
auto joined_table_id = context->resolveStorageID(table_to_join.database_and_table_name);
StoragePtr table = DatabaseCatalog::instance().tryGetTable(joined_table_id, context);
if (table)
StoragePtr storage = DatabaseCatalog::instance().tryGetTable(joined_table_id, context);
if (storage)
{
if (dynamic_cast<StorageJoin *>(table.get()) ||
dynamic_cast<StorageDictionary *>(table.get()))
table_join->joined_storage = table;
if (auto storage_join = std::dynamic_pointer_cast<StorageJoin>(storage); storage_join)
table_join->setStorageJoin(storage_join);
else if (auto storage_dict = std::dynamic_pointer_cast<StorageDictionary>(storage); storage_dict)
table_join->setStorageJoin(storage_dict);
}
}
if (!table_join->joined_storage &&
if (!table_join->isSpecialStorage() &&
settings.enable_optimize_predicate_expression)
replaceJoinedTable(select_query);

View File

@ -39,7 +39,7 @@ bool PredicateExpressionsOptimizer::optimize(ASTSelectQuery & select_query)
if (!select_query.tables() || select_query.tables()->children.empty())
return false;
if ((!select_query.where() && !select_query.prewhere()) || select_query.arrayJoinExpressionList())
if ((!select_query.where() && !select_query.prewhere()) || select_query.arrayJoinExpressionList().first)
return false;
const auto & tables_predicates = extractTablesPredicates(select_query.where(), select_query.prewhere());

View File

@ -1,5 +1,6 @@
#include <Interpreters/TableJoin.h>
#include <Common/StringUtils/StringUtils.h>
#include <Core/Block.h>
@ -7,10 +8,20 @@
#include <Core/Settings.h>
#include <DataTypes/DataTypeNullable.h>
#include <Dictionaries/DictionaryStructure.h>
#include <Interpreters/DictionaryReader.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/queryToString.h>
#include <Storages/IStorage.h>
#include <Storages/StorageDictionary.h>
#include <Storages/StorageJoin.h>
#include <common/logger_useful.h>
@ -20,6 +31,24 @@ namespace DB
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int LOGICAL_ERROR;
}
namespace
{
std::string formatTypeMap(const TableJoin::NameToTypeMap & target, const TableJoin::NameToTypeMap & source)
{
std::vector<std::string> text;
for (const auto & [k, v] : target)
{
auto src_type_it = source.find(k);
std::string src_type_name = src_type_it != source.end() ? src_type_it->second->getName() : "";
text.push_back(fmt::format("{} : {} -> {}", k, src_type_name, v->getName()));
}
return fmt::format("{}", fmt::join(text, ", "));
}
}
TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_)
@ -49,8 +78,6 @@ void TableJoin::resetCollected()
renames.clear();
left_type_map.clear();
right_type_map.clear();
left_converting_actions = nullptr;
right_converting_actions = nullptr;
}
void TableJoin::addUsingKey(const ASTPtr & ast)
@ -184,7 +211,7 @@ Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vecto
{
const Names & left_keys = keyNamesLeft();
const Names & right_keys = keyNamesRight();
NameSet required_keys(requiredRightKeys().begin(), requiredRightKeys().end());
NameSet required_keys = requiredRightKeys();
Block required_right_keys;
for (size_t i = 0; i < right_keys.size(); ++i)
@ -202,7 +229,6 @@ Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vecto
return required_right_keys;
}
bool TableJoin::leftBecomeNullable(const DataTypePtr & column_type) const
{
return forceNullableLeft() && JoinCommon::canBecomeNullable(column_type);
@ -215,36 +241,54 @@ bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const
void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column)
{
DataTypePtr type = joined_column.type;
if (hasUsing())
{
if (auto it = right_type_map.find(joined_column.name); it != right_type_map.end())
type = it->second;
}
if (rightBecomeNullable(type))
type = JoinCommon::convertTypeToNullable(type);
columns_added_by_join.emplace_back(joined_column.name, type);
columns_added_by_join.emplace_back(joined_column);
}
void TableJoin::addJoinedColumnsAndCorrectTypes(NamesAndTypesList & names_and_types, bool correct_nullability) const
NamesAndTypesList TableJoin::correctedColumnsAddedByJoin() const
{
for (auto & col : names_and_types)
NamesAndTypesList result;
for (const auto & col : columns_added_by_join)
{
DataTypePtr type = col.type;
if (hasUsing())
{
if (auto it = right_type_map.find(col.name); it != right_type_map.end())
type = it->second;
}
if (rightBecomeNullable(type))
type = JoinCommon::convertTypeToNullable(type);
result.emplace_back(col.name, type);
}
return result;
}
void TableJoin::addJoinedColumnsAndCorrectTypes(NamesAndTypesList & left_columns, bool correct_nullability)
{
for (auto & col : left_columns)
{
if (hasUsing())
{
/*
* Join with `USING` semantic allows to have columns with changed types in result table.
* But `JOIN ON` should preserve types from original table.
* So we need to know changed types in result tables before further analysis (e.g. analyzeAggregation)
* For `JOIN ON expr1 == expr2` we will infer common type later in makeTableJoin,
* when part of plan built and types of expression will be known.
*/
inferJoinKeyCommonType(left_columns, columns_from_joined_table, !isSpecialStorage());
if (auto it = left_type_map.find(col.name); it != left_type_map.end())
col.type = it->second;
}
if (correct_nullability && leftBecomeNullable(col.type))
col.type = JoinCommon::convertTypeToNullable(col.type);
}
/// Types in columns_added_by_join already converted and set nullable if needed
for (const auto & col : columns_added_by_join)
names_and_types.emplace_back(col.name, col.type);
for (const auto & col : correctedColumnsAddedByJoin())
left_columns.emplace_back(col.name, col.type);
}
bool TableJoin::sameStrictnessAndKind(ASTTableJoin::Strictness strictness_, ASTTableJoin::Kind kind_) const
@ -282,7 +326,18 @@ bool TableJoin::needStreamWithNonJoinedRows() const
return isRightOrFull(kind());
}
bool TableJoin::allowDictJoin(const String & dict_key, const Block & sample_block, Names & src_names, NamesAndTypesList & dst_columns) const
static std::optional<String> getDictKeyName(const String & dict_name , ContextPtr context)
{
auto dictionary = context->getExternalDictionariesLoader().getDictionary(dict_name, context);
if (!dictionary)
return {};
if (const auto & structure = dictionary->getStructure(); structure.id)
return structure.id->name;
return {};
}
bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context)
{
/// Support ALL INNER, [ANY | ALL | SEMI | ANTI] LEFT
if (!isLeft(kind()) && !(isInner(kind()) && strictness() == ASTTableJoin::Strictness::All))
@ -297,9 +352,17 @@ bool TableJoin::allowDictJoin(const String & dict_key, const Block & sample_bloc
if (it_key == original_names.end())
return false;
if (dict_key != it_key->second)
if (!right_storage_dictionary)
return false;
auto dict_name = right_storage_dictionary->getDictionaryName();
auto dict_key = getDictKeyName(dict_name, context);
if (!dict_key.has_value() || *dict_key != it_key->second)
return false; /// JOIN key != Dictionary key
Names src_names;
NamesAndTypesList dst_columns;
for (const auto & col : sample_block)
{
if (col.name == right_keys[0])
@ -313,51 +376,35 @@ bool TableJoin::allowDictJoin(const String & dict_key, const Block & sample_bloc
dst_columns.push_back({col.name, col.type});
}
}
dictionary_reader = std::make_shared<DictionaryReader>(dict_name, src_names, dst_columns, context);
return true;
}
bool TableJoin::applyJoinKeyConvert(const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns)
std::pair<ActionsDAGPtr, ActionsDAGPtr>
TableJoin::createConvertingActions(const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns)
{
bool need_convert = needConvert();
if (!need_convert && !hasUsing())
{
/// For `USING` we already inferred common type an syntax analyzer stage
NamesAndTypesList left_list;
NamesAndTypesList right_list;
for (const auto & col : left_sample_columns)
left_list.emplace_back(col.name, col.type);
for (const auto & col : right_sample_columns)
right_list.emplace_back(col.name, col.type);
inferJoinKeyCommonType(left_sample_columns, right_sample_columns, !isSpecialStorage());
need_convert = inferJoinKeyCommonType(left_list, right_list);
}
auto left_converting_actions = applyKeyConvertToTable(left_sample_columns, left_type_map, key_names_left);
auto right_converting_actions = applyKeyConvertToTable(right_sample_columns, right_type_map, key_names_right);
if (need_convert)
{
left_converting_actions = applyKeyConvertToTable(left_sample_columns, left_type_map, key_names_left);
right_converting_actions = applyKeyConvertToTable(right_sample_columns, right_type_map, key_names_right);
}
return need_convert;
return {left_converting_actions, right_converting_actions};
}
bool TableJoin::inferJoinKeyCommonType(const NamesAndTypesList & left, const NamesAndTypesList & right)
template <typename LeftNamesAndTypes, typename RightNamesAndTypes>
bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool allow_right)
{
std::unordered_map<String, DataTypePtr> left_types;
for (const auto & col : left)
{
left_types[col.name] = col.type;
}
if (!left_type_map.empty() || !right_type_map.empty())
return true;
std::unordered_map<String, DataTypePtr> right_types;
NameToTypeMap left_types;
for (const auto & col : left)
left_types[col.name] = col.type;
NameToTypeMap right_types;
for (const auto & col : right)
{
if (auto it = renames.find(col.name); it != renames.end())
right_types[it->second] = col.type;
else
right_types[col.name] = col.type;
}
right_types[renamedRightColumnName(col.name)] = col.type;
for (size_t i = 0; i < key_names_left.size(); ++i)
{
@ -374,37 +421,37 @@ bool TableJoin::inferJoinKeyCommonType(const NamesAndTypesList & left, const Nam
if (JoinCommon::typesEqualUpToNullability(ltype->second, rtype->second))
continue;
DataTypePtr supertype;
DataTypePtr common_type;
try
{
supertype = DB::getLeastSupertype({ltype->second, rtype->second});
/// TODO(vdimir): use getMostSubtype if possible
common_type = DB::getLeastSupertype({ltype->second, rtype->second});
}
catch (DB::Exception & ex)
{
throw Exception(
"Type mismatch of columns to JOIN by: " +
key_names_left[i] + ": " + ltype->second->getName() + " at left, " +
key_names_right[i] + ": " + rtype->second->getName() + " at right. " +
"Can't get supertype: " + ex.message(),
ErrorCodes::TYPE_MISMATCH);
throw DB::Exception(ErrorCodes::TYPE_MISMATCH,
"Can't infer common type for joined columns: {}: {} at left, {}: {} at right. {}",
key_names_left[i], ltype->second->getName(),
key_names_right[i], rtype->second->getName(),
ex.message());
}
left_type_map[key_names_left[i]] = right_type_map[key_names_right[i]] = supertype;
if (!allow_right && !common_type->equals(*rtype->second))
{
throw DB::Exception(ErrorCodes::TYPE_MISMATCH,
"Can't change type for right table: {}: {} -> {}.",
key_names_right[i], rtype->second->getName(), common_type->getName());
}
left_type_map[key_names_left[i]] = right_type_map[key_names_right[i]] = common_type;
}
if (!left_type_map.empty() || !right_type_map.empty())
{
auto format_type_map = [](NameToTypeMap mapping) -> std::string
{
std::vector<std::string> text;
for (const auto & [k, v] : mapping)
text.push_back(k + ": " + v->getName());
return fmt::format("{}", fmt::join(text, ", "));
};
LOG_TRACE(
&Poco::Logger::get("TableJoin"),
"Infer supertype for joined columns. Left: [{}], Right: [{}]",
format_type_map(left_type_map),
format_type_map(right_type_map));
formatTypeMap(left_type_map, left_types),
formatTypeMap(right_type_map, right_types));
}
return !left_type_map.empty();
@ -413,15 +460,20 @@ bool TableJoin::inferJoinKeyCommonType(const NamesAndTypesList & left, const Nam
ActionsDAGPtr TableJoin::applyKeyConvertToTable(
const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, Names & names_to_rename) const
{
bool has_some_to_do = false;
ColumnsWithTypeAndName cols_dst = cols_src;
for (auto & col : cols_dst)
{
if (auto it = type_mapping.find(col.name); it != type_mapping.end())
{
has_some_to_do = true;
col.type = it->second;
col.column = nullptr;
}
}
if (!has_some_to_do)
return nullptr;
NameToNameMap key_column_rename;
/// Returns converting actions for tables that need to be performed before join
@ -437,6 +489,20 @@ ActionsDAGPtr TableJoin::applyKeyConvertToTable(
return dag;
}
void TableJoin::setStorageJoin(std::shared_ptr<StorageJoin> storage)
{
if (right_storage_dictionary)
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "StorageJoin and Dictionary join are mutually exclusive");
right_storage_join = storage;
}
void TableJoin::setStorageJoin(std::shared_ptr<StorageDictionary> storage)
{
if (right_storage_join)
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "StorageJoin and Dictionary join are mutually exclusive");
right_storage_dictionary = storage;
}
String TableJoin::renamedRightColumnName(const String & name) const
{
if (const auto it = renames.find(name); it != renames.end())

View File

@ -24,6 +24,8 @@ class ASTSelectQuery;
struct DatabaseAndTableWithAlias;
class Block;
class DictionaryReader;
class StorageJoin;
class StorageDictionary;
struct ColumnWithTypeAndName;
using ColumnsWithTypeAndName = std::vector<ColumnWithTypeAndName>;
@ -86,16 +88,14 @@ private:
/// All columns which can be read from joined table. Duplicating names are qualified.
NamesAndTypesList columns_from_joined_table;
/// Columns will be added to block by JOIN.
/// It's a subset of columns_from_joined_table with corrected Nullability and type (if inplace type conversion is required)
/// It's a subset of columns_from_joined_table
/// Note: without corrected Nullability or type, see correctedColumnsAddedByJoin
NamesAndTypesList columns_added_by_join;
/// Target type to convert key columns before join
NameToTypeMap left_type_map;
NameToTypeMap right_type_map;
ActionsDAGPtr left_converting_actions;
ActionsDAGPtr right_converting_actions;
/// Name -> original name. Names are the same as in columns_from_joined_table list.
std::unordered_map<String, String> original_names;
/// Original name -> name. Only renamed columns.
@ -103,12 +103,23 @@ private:
VolumePtr tmp_volume;
std::shared_ptr<StorageJoin> right_storage_join;
std::shared_ptr<StorageDictionary> right_storage_dictionary;
std::shared_ptr<DictionaryReader> dictionary_reader;
Names requiredJoinedNames() const;
/// Create converting actions and change key column names if required
ActionsDAGPtr applyKeyConvertToTable(
const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, Names & names_to_rename) const;
/// Calculates common supertypes for corresponding join key columns.
template <typename LeftNamesAndTypes, typename RightNamesAndTypes>
bool inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool allow_right);
NamesAndTypesList correctedColumnsAddedByJoin() const;
public:
TableJoin() = default;
TableJoin(const Settings &, VolumePtr tmp_volume);
@ -126,16 +137,12 @@ public:
table_join.strictness = strictness;
}
StoragePtr joined_storage;
std::shared_ptr<DictionaryReader> dictionary_reader;
ASTTableJoin::Kind kind() const { return table_join.kind; }
ASTTableJoin::Strictness strictness() const { return table_join.strictness; }
bool sameStrictnessAndKind(ASTTableJoin::Strictness, ASTTableJoin::Kind) const;
const SizeLimits & sizeLimits() const { return size_limits; }
VolumePtr getTemporaryVolume() { return tmp_volume; }
bool allowMergeJoin() const;
bool allowDictJoin(const String & dict_key, const Block & sample_block, Names &, NamesAndTypesList &) const;
bool preferMergeJoin() const { return join_algorithm == JoinAlgorithm::PREFER_PARTIAL_MERGE; }
bool forceMergeJoin() const { return join_algorithm == JoinAlgorithm::PARTIAL_MERGE; }
bool forceHashJoin() const
@ -190,21 +197,13 @@ public:
bool rightBecomeNullable(const DataTypePtr & column_type) const;
void addJoinedColumn(const NameAndTypePair & joined_column);
void addJoinedColumnsAndCorrectTypes(NamesAndTypesList & names_and_types, bool correct_nullability = true) const;
/// Calculates common supertypes for corresponding join key columns.
bool inferJoinKeyCommonType(const NamesAndTypesList & left, const NamesAndTypesList & right);
void addJoinedColumnsAndCorrectTypes(NamesAndTypesList & left_columns, bool correct_nullability);
/// Calculate converting actions, rename key columns in required
/// For `USING` join we will convert key columns inplace and affect into types in the result table
/// For `JOIN ON` we will create new columns with converted keys to join by.
bool applyJoinKeyConvert(const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns);
bool needConvert() const { return !left_type_map.empty(); }
/// Key columns should be converted before join.
ActionsDAGPtr leftConvertingActions() const { return left_converting_actions; }
ActionsDAGPtr rightConvertingActions() const { return right_converting_actions; }
std::pair<ActionsDAGPtr, ActionsDAGPtr>
createConvertingActions(const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns);
void setAsofInequality(ASOF::Inequality inequality) { asof_inequality = inequality; }
ASOF::Inequality getAsofInequality() { return asof_inequality; }
@ -215,6 +214,7 @@ public:
const Names & keyNamesLeft() const { return key_names_left; }
const Names & keyNamesRight() const { return key_names_right; }
const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; }
Names columnsAddedByJoin() const
{
Names res;
@ -230,6 +230,16 @@ public:
String renamedRightColumnName(const String & name) const;
std::unordered_map<String, String> leftToRightKeyRemap() const;
void setStorageJoin(std::shared_ptr<StorageJoin> storage);
void setStorageJoin(std::shared_ptr<StorageDictionary> storage);
std::shared_ptr<StorageJoin> getStorageJoin() { return right_storage_join; }
bool tryInitDictJoin(const Block & sample_block, ContextPtr context);
bool isSpecialStorage() const { return right_storage_dictionary || right_storage_join; }
const DictionaryReader * getDictionaryReader() const { return dictionary_reader.get(); }
};
}

View File

@ -422,46 +422,44 @@ void executeScalarSubqueries(ASTPtr & query, ContextPtr context, size_t subquery
void getArrayJoinedColumns(ASTPtr & query, TreeRewriterResult & result, const ASTSelectQuery * select_query,
const NamesAndTypesList & source_columns, const NameSet & source_columns_set)
{
if (ASTPtr array_join_expression_list = select_query->arrayJoinExpressionList())
if (!select_query->arrayJoinExpressionList().first)
return;
ArrayJoinedColumnsVisitor::Data visitor_data{
result.aliases, result.array_join_name_to_alias, result.array_join_alias_to_name, result.array_join_result_to_source};
ArrayJoinedColumnsVisitor(visitor_data).visit(query);
/// If the result of ARRAY JOIN is not used, it is necessary to ARRAY-JOIN any column,
/// to get the correct number of rows.
if (result.array_join_result_to_source.empty())
{
ArrayJoinedColumnsVisitor::Data visitor_data{result.aliases,
result.array_join_name_to_alias,
result.array_join_alias_to_name,
result.array_join_result_to_source};
ArrayJoinedColumnsVisitor(visitor_data).visit(query);
if (select_query->arrayJoinExpressionList().first->children.empty())
throw DB::Exception("ARRAY JOIN requires an argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
/// If the result of ARRAY JOIN is not used, it is necessary to ARRAY-JOIN any column,
/// to get the correct number of rows.
if (result.array_join_result_to_source.empty())
ASTPtr expr = select_query->arrayJoinExpressionList().first->children.at(0);
String source_name = expr->getColumnName();
String result_name = expr->getAliasOrColumnName();
/// This is an array.
if (!expr->as<ASTIdentifier>() || source_columns_set.count(source_name))
{
if (select_query->arrayJoinExpressionList()->children.empty())
throw DB::Exception("ARRAY JOIN requires an argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTPtr expr = select_query->arrayJoinExpressionList()->children.at(0);
String source_name = expr->getColumnName();
String result_name = expr->getAliasOrColumnName();
/// This is an array.
if (!expr->as<ASTIdentifier>() || source_columns_set.count(source_name))
result.array_join_result_to_source[result_name] = source_name;
}
else /// This is a nested table.
{
bool found = false;
for (const auto & column : source_columns)
{
result.array_join_result_to_source[result_name] = source_name;
}
else /// This is a nested table.
{
bool found = false;
for (const auto & column : source_columns)
auto split = Nested::splitName(column.name);
if (split.first == source_name && !split.second.empty())
{
auto split = Nested::splitName(column.name);
if (split.first == source_name && !split.second.empty())
{
result.array_join_result_to_source[Nested::concatenateName(result_name, split.second)] = column.name;
found = true;
break;
}
result.array_join_result_to_source[Nested::concatenateName(result_name, split.second)] = column.name;
found = true;
break;
}
if (!found)
throw Exception("No columns in nested table " + source_name, ErrorCodes::EMPTY_NESTED_TABLE);
}
if (!found)
throw Exception("No columns in nested table " + source_name, ErrorCodes::EMPTY_NESTED_TABLE);
}
}
}
@ -519,13 +517,6 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_
const auto & keys = table_join.using_expression_list->as<ASTExpressionList &>();
for (const auto & key : keys.children)
analyzed_join.addUsingKey(key);
/// `USING` semantic allows to have columns with changed types in result table.
/// `JOIN ON` should preserve types from original table
/// We can infer common type on syntax stage for `USING` because join is performed only by columns (not expressions)
/// We need to know changed types in result tables because some analysis (e.g. analyzeAggregation) performed before join
/// For `JOIN ON expr1 == expr2` we will infer common type later in ExpressionAnalyzer, when types of expression will be known
analyzed_join.inferJoinKeyCommonType(tables[0].columns, tables[1].columns);
}
else if (table_join.on_expression)
{

View File

@ -73,7 +73,7 @@ struct TreeRewriterResult
/// Results of scalar sub queries
Scalars scalars;
TreeRewriterResult(
explicit TreeRewriterResult(
const NamesAndTypesList & source_columns_,
ConstStoragePtr storage_ = {},
const StorageMetadataPtr & metadata_snapshot_ = {},
@ -84,7 +84,6 @@ struct TreeRewriterResult
Names requiredSourceColumns() const { return required_source_columns.getNames(); }
const Names & requiredSourceColumnsForAccessCheck() const { return required_source_columns_before_expanding_alias_columns; }
NameSet getArrayJoinSourceNameSet() const;
Names getExpandedAliases() const { return {expanded_aliases.begin(), expanded_aliases.end()}; }
const Scalars & getScalars() const { return scalars; }
};

View File

@ -47,6 +47,12 @@ static void executeCreateQuery(
interpreter.execute();
}
static bool isSystemOrInformationSchema(const String & database_name)
{
return database_name == DatabaseCatalog::SYSTEM_DATABASE ||
database_name == DatabaseCatalog::INFORMATION_SCHEMA ||
database_name == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE;
}
static void loadDatabase(
ContextMutablePtr context,
@ -116,7 +122,7 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam
if (fs::path(current_file).extension() == ".sql")
{
String db_name = fs::path(current_file).stem();
if (db_name != DatabaseCatalog::SYSTEM_DATABASE)
if (!isSystemOrInformationSchema(db_name))
databases.emplace(unescapeForFileName(db_name), fs::path(path) / db_name);
}
@ -142,7 +148,7 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam
if (current_file.at(0) == '.')
continue;
if (current_file == DatabaseCatalog::SYSTEM_DATABASE)
if (isSystemOrInformationSchema(current_file))
continue;
databases.emplace(unescapeForFileName(current_file), it->path().string());
@ -171,25 +177,31 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam
}
}
void loadMetadataSystem(ContextMutablePtr context)
static void loadSystemDatabaseImpl(ContextMutablePtr context, const String & database_name, const String & default_engine)
{
String path = context->getPath() + "metadata/" + DatabaseCatalog::SYSTEM_DATABASE;
String path = context->getPath() + "metadata/" + database_name;
String metadata_file = path + ".sql";
if (fs::exists(fs::path(path)) || fs::exists(fs::path(metadata_file)))
{
/// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted.
loadDatabase(context, DatabaseCatalog::SYSTEM_DATABASE, path, true);
loadDatabase(context, database_name, path, true);
}
else
{
/// Initialize system database manually
String database_create_query = "CREATE DATABASE ";
database_create_query += DatabaseCatalog::SYSTEM_DATABASE;
database_create_query += " ENGINE=Atomic";
executeCreateQuery(database_create_query, context, DatabaseCatalog::SYSTEM_DATABASE, "<no file>", true);
database_create_query += database_name;
database_create_query += " ENGINE=";
database_create_query += default_engine;
executeCreateQuery(database_create_query, context, database_name, "<no file>", true);
}
}
void loadMetadataSystem(ContextMutablePtr context)
{
loadSystemDatabaseImpl(context, DatabaseCatalog::SYSTEM_DATABASE, "Atomic");
loadSystemDatabaseImpl(context, DatabaseCatalog::INFORMATION_SCHEMA, "Memory");
loadSystemDatabaseImpl(context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE, "Memory");
}
}

View File

@ -10,7 +10,8 @@ namespace DB
/// You should first load system database, then attach system tables that you need into it, then load other databases.
void loadMetadataSystem(ContextMutablePtr context);
/// Load tables from databases and add them to context. Database 'system' is ignored. Use separate function to load system tables.
/// Load tables from databases and add them to context. Database 'system' and 'information_schema' is ignored.
/// Use separate function to load system tables.
void loadMetadata(ContextMutablePtr context, const String & default_database_name = {});
}

View File

@ -319,24 +319,16 @@ bool ASTSelectQuery::withFill() const
}
ASTPtr ASTSelectQuery::arrayJoinExpressionList(bool & is_left) const
std::pair<ASTPtr, bool> ASTSelectQuery::arrayJoinExpressionList() const
{
const ASTArrayJoin * array_join = getFirstArrayJoin(*this);
if (!array_join)
return {};
is_left = (array_join->kind == ASTArrayJoin::Kind::Left);
return array_join->expression_list;
bool is_left = (array_join->kind == ASTArrayJoin::Kind::Left);
return {array_join->expression_list, is_left};
}
ASTPtr ASTSelectQuery::arrayJoinExpressionList() const
{
bool is_left;
return arrayJoinExpressionList(is_left);
}
const ASTTablesInSelectQueryElement * ASTSelectQuery::join() const
{
return getFirstTableJoin(*this);

View File

@ -123,8 +123,8 @@ public:
/// Compatibility with old parser of tables list. TODO remove
ASTPtr sampleSize() const;
ASTPtr sampleOffset() const;
ASTPtr arrayJoinExpressionList(bool & is_left) const;
ASTPtr arrayJoinExpressionList() const;
std::pair<ASTPtr, bool> arrayJoinExpressionList() const;
const ASTTablesInSelectQueryElement * join() const;
bool final() const;
bool withFill() const;

View File

@ -395,9 +395,14 @@ AggregatingTransform::AggregatingTransform(Block header, AggregatingTransformPar
}
AggregatingTransform::AggregatingTransform(
Block header, AggregatingTransformParamsPtr params_, ManyAggregatedDataPtr many_data_,
size_t current_variant, size_t max_threads_, size_t temporary_data_merge_threads_)
: IProcessor({std::move(header)}, {params_->getHeader()}), params(std::move(params_))
Block header,
AggregatingTransformParamsPtr params_,
ManyAggregatedDataPtr many_data_,
size_t current_variant,
size_t max_threads_,
size_t temporary_data_merge_threads_)
: IProcessor({std::move(header)}, {params_->getHeader()})
, params(std::move(params_))
, key_columns(params->params.keys_size)
, aggregate_columns(params->params.aggregates_size)
, many_data(std::move(many_data_))
@ -525,7 +530,7 @@ void AggregatingTransform::consume(Chunk chunk)
{
auto block = getInputs().front().getHeader().cloneWithColumns(chunk.detachColumns());
block = materializeBlock(block);
if (!params->aggregator.mergeBlock(block, variants, no_more_keys))
if (!params->aggregator.mergeOnBlock(block, variants, no_more_keys))
is_consume_finished = true;
}
else
@ -547,7 +552,7 @@ void AggregatingTransform::initGenerate()
if (variants.empty() && params->params.keys_size == 0 && !params->params.empty_result_for_aggregation_by_empty_set)
{
if (params->only_merge)
params->aggregator.mergeBlock(getInputs().front().getHeader(), variants, no_more_keys);
params->aggregator.mergeOnBlock(getInputs().front().getHeader(), variants, no_more_keys);
else
params->aggregator.executeOnBlock(getInputs().front().getHeader(), variants, key_columns, aggregate_columns, no_more_keys);
}

View File

@ -27,15 +27,38 @@ public:
class IBlockInputStream;
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
using AggregatorList = std::list<Aggregator>;
using AggregatorListPtr = std::shared_ptr<AggregatorList>;
struct AggregatingTransformParams
{
Aggregator::Params params;
Aggregator aggregator;
/// Each params holds a list of aggregators which are used in query. It's needed because we need
/// to use a pointer of aggregator to proper destroy complex aggregation states on exception
/// (See comments in AggregatedDataVariants). However, this pointer might not be valid because
/// we can have two different aggregators at the same time due to mixed pipeline of aggregate
/// projections, and one of them might gets destroyed before used.
AggregatorListPtr aggregator_list_ptr;
Aggregator & aggregator;
bool final;
bool only_merge = false;
AggregatingTransformParams(const Aggregator::Params & params_, bool final_)
: params(params_), aggregator(params), final(final_) {}
: params(params_)
, aggregator_list_ptr(std::make_shared<AggregatorList>())
, aggregator(*aggregator_list_ptr->emplace(aggregator_list_ptr->end(), params))
, final(final_)
{
}
AggregatingTransformParams(const Aggregator::Params & params_, const AggregatorListPtr & aggregator_list_ptr_, bool final_)
: params(params_)
, aggregator_list_ptr(aggregator_list_ptr_)
, aggregator(*aggregator_list_ptr->emplace(aggregator_list_ptr->end(), params))
, final(final_)
{
}
Block getHeader() const { return aggregator.getHeader(final); }
@ -82,9 +105,13 @@ public:
AggregatingTransform(Block header, AggregatingTransformParamsPtr params_);
/// For Parallel aggregating.
AggregatingTransform(Block header, AggregatingTransformParamsPtr params_,
ManyAggregatedDataPtr many_data, size_t current_variant,
size_t max_threads, size_t temporary_data_merge_threads);
AggregatingTransform(
Block header,
AggregatingTransformParamsPtr params_,
ManyAggregatedDataPtr many_data,
size_t current_variant,
size_t max_threads,
size_t temporary_data_merge_threads);
~AggregatingTransform() override;
String getName() const override { return "AggregatingTransform"; }

View File

@ -0,0 +1,144 @@
#include <Storages/MergeTree/BackgroundJobsAssignee.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Common/CurrentMetrics.h>
#include <Common/randomSeed.h>
#include <pcg_random.hpp>
#include <random>
namespace DB
{
BackgroundJobsAssignee::BackgroundJobsAssignee(MergeTreeData & data_, BackgroundJobsAssignee::Type type_, ContextPtr global_context_)
: WithContext(global_context_)
, data(data_)
, sleep_settings(global_context_->getBackgroundMoveTaskSchedulingSettings())
, rng(randomSeed())
, type(type_)
{
}
void BackgroundJobsAssignee::trigger()
{
std::lock_guard lock(holder_mutex);
if (!holder)
return;
no_work_done_count = 0;
/// We have background jobs, schedule task as soon as possible
holder->schedule();
}
void BackgroundJobsAssignee::postpone()
{
std::lock_guard lock(holder_mutex);
if (!holder)
return;
auto no_work_done_times = no_work_done_count.fetch_add(1, std::memory_order_relaxed);
double random_addition = std::uniform_real_distribution<double>(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng);
size_t next_time_to_execute = 1000 * (std::min(
sleep_settings.task_sleep_seconds_when_no_work_max,
sleep_settings.thread_sleep_seconds_if_nothing_to_do * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times))
+ random_addition);
holder->scheduleAfter(next_time_to_execute, false);
}
void BackgroundJobsAssignee::scheduleMergeMutateTask(ExecutableTaskPtr merge_task)
{
bool res = getContext()->getMergeMutateExecutor()->trySchedule(merge_task);
res ? trigger() : postpone();
}
void BackgroundJobsAssignee::scheduleFetchTask(ExecutableTaskPtr fetch_task)
{
bool res = getContext()->getFetchesExecutor()->trySchedule(fetch_task);
res ? trigger() : postpone();
}
void BackgroundJobsAssignee::scheduleMoveTask(ExecutableTaskPtr move_task)
{
bool res = getContext()->getMovesExecutor()->trySchedule(move_task);
res ? trigger() : postpone();
}
String BackgroundJobsAssignee::toString(Type type)
{
switch (type)
{
case Type::DataProcessing:
return "DataProcessing";
case Type::Moving:
return "Moving";
}
__builtin_unreachable();
}
void BackgroundJobsAssignee::start()
{
std::lock_guard lock(holder_mutex);
if (!holder)
holder = getContext()->getSchedulePool().createTask("BackgroundJobsAssignee:" + toString(type), [this]{ threadFunc(); });
holder->activateAndSchedule();
}
void BackgroundJobsAssignee::finish()
{
/// No lock here, because scheduled tasks could call trigger method
if (holder)
{
holder->deactivate();
auto storage_id = data.getStorageID();
getContext()->getMovesExecutor()->removeTasksCorrespondingToStorage(storage_id);
getContext()->getFetchesExecutor()->removeTasksCorrespondingToStorage(storage_id);
getContext()->getMergeMutateExecutor()->removeTasksCorrespondingToStorage(storage_id);
}
}
void BackgroundJobsAssignee::threadFunc()
try
{
bool succeed = false;
switch (type)
{
case Type::DataProcessing:
succeed = data.scheduleDataProcessingJob(*this);
break;
case Type::Moving:
succeed = data.scheduleDataMovingJob(*this);
break;
}
if (!succeed)
postpone();
}
catch (...) /// Catch any exception to avoid thread termination.
{
tryLogCurrentException(__PRETTY_FUNCTION__);
postpone();
}
BackgroundJobsAssignee::~BackgroundJobsAssignee()
{
try
{
finish();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}

View File

@ -0,0 +1,89 @@
#pragma once
#include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
#include <Common/ThreadPool.h>
#include <Core/BackgroundSchedulePool.h>
#include <pcg_random.hpp>
namespace DB
{
/// Settings for background tasks scheduling. Each background assignee has one
/// BackgroundSchedulingPoolTask and depending on execution result may put this
/// task to sleep according to settings. Look at scheduleTask function for details.
struct BackgroundTaskSchedulingSettings
{
double thread_sleep_seconds_random_part = 1.0;
double thread_sleep_seconds_if_nothing_to_do = 0.1;
double task_sleep_seconds_when_no_work_max = 600;
/// For exponential backoff.
double task_sleep_seconds_when_no_work_multiplier = 1.1;
double task_sleep_seconds_when_no_work_random_part = 1.0;
/// Deprecated settings, don't affect background execution
double thread_sleep_seconds = 10;
double task_sleep_seconds_when_no_work_min = 10;
};
class MergeTreeData;
class BackgroundJobsAssignee : public WithContext
{
private:
MergeTreeData & data;
/// Settings for execution control of background scheduling task
BackgroundTaskSchedulingSettings sleep_settings;
/// Useful for random backoff timeouts generation
pcg64 rng;
/// How many times execution of background job failed or we have
/// no new jobs.
std::atomic<size_t> no_work_done_count{0};
/// Scheduling task which assign jobs in background pool
BackgroundSchedulePool::TaskHolder holder;
/// Mutex for thread safety
std::mutex holder_mutex;
public:
/// In case of ReplicatedMergeTree the first assignee will be responsible for
/// polling the replication queue and schedule operations according to the LogEntry type
/// e.g. merges, mutations and fetches. The same will be for Plain MergeTree except there is no
/// replication queue, so we will just scan parts and decide what to do.
/// Moving operations are the same for all types of MergeTree and also have their own timetable.
enum class Type
{
DataProcessing,
Moving
};
Type type{Type::DataProcessing};
void start();
void trigger();
void postpone();
void finish();
void scheduleMergeMutateTask(ExecutableTaskPtr merge_task);
void scheduleFetchTask(ExecutableTaskPtr fetch_task);
void scheduleMoveTask(ExecutableTaskPtr move_task);
/// Just call finish
virtual ~BackgroundJobsAssignee();
BackgroundJobsAssignee(
MergeTreeData & data_,
Type type,
ContextPtr global_context_);
private:
static String toString(Type type);
/// Function that executes in background scheduling pool
void threadFunc();
};
}

View File

@ -1,289 +0,0 @@
#include <Storages/MergeTree/BackgroundJobsExecutor.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Common/CurrentMetrics.h>
#include <Common/randomSeed.h>
#include <pcg_random.hpp>
#include <random>
namespace CurrentMetrics
{
extern const Metric BackgroundPoolTask;
extern const Metric BackgroundMovePoolTask;
extern const Metric BackgroundFetchesPoolTask;
}
namespace DB
{
IBackgroundJobExecutor::IBackgroundJobExecutor(
ContextPtr global_context_,
const BackgroundTaskSchedulingSettings & sleep_settings_,
const std::vector<PoolConfig> & pools_configs_)
: WithContext(global_context_)
, sleep_settings(sleep_settings_)
, rng(randomSeed())
{
for (const auto & pool_config : pools_configs_)
{
const auto max_pool_size = pool_config.get_max_pool_size();
pools.try_emplace(pool_config.pool_type, max_pool_size, 0, max_pool_size, false);
pools_configs.emplace(pool_config.pool_type, pool_config);
}
}
double IBackgroundJobExecutor::getSleepRandomAdd()
{
std::lock_guard random_lock(random_mutex);
return std::uniform_real_distribution<double>(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng);
}
void IBackgroundJobExecutor::runTaskWithoutDelay()
{
no_work_done_count = 0;
/// We have background jobs, schedule task as soon as possible
scheduling_task->schedule();
}
void IBackgroundJobExecutor::scheduleTask(bool with_backoff)
{
size_t next_time_to_execute;
if (with_backoff)
{
auto no_work_done_times = no_work_done_count.fetch_add(1, std::memory_order_relaxed);
next_time_to_execute = 1000 * (std::min(
sleep_settings.task_sleep_seconds_when_no_work_max,
sleep_settings.thread_sleep_seconds_if_nothing_to_do * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times))
+ getSleepRandomAdd());
}
else
{
no_work_done_count = 0;
next_time_to_execute = 1000 * sleep_settings.thread_sleep_seconds_if_nothing_to_do;
}
scheduling_task->scheduleAfter(next_time_to_execute, false);
}
namespace
{
/// Tricky function: we have separate thread pool with max_threads in each background executor for each table
/// But we want total background threads to be less than max_threads value. So we use global atomic counter (BackgroundMetric)
/// to limit total number of background threads.
bool incrementMetricIfLessThanMax(std::atomic<Int64> & atomic_value, Int64 max_value)
{
auto value = atomic_value.load(std::memory_order_relaxed);
while (value < max_value)
{
if (atomic_value.compare_exchange_weak(value, value + 1, std::memory_order_release, std::memory_order_relaxed))
return true;
}
return false;
}
}
/// This is a RAII class which only decrements metric.
/// It is added because after all other fixes a bug non-executing merges was occurred again.
/// Last hypothesis: task was successfully added to pool, however, was not executed because of internal exception in it.
class ParanoidMetricDecrementor
{
public:
explicit ParanoidMetricDecrementor(CurrentMetrics::Metric metric_) : metric(metric_) {}
void alarm() { is_alarmed = true; }
void decrement()
{
if (is_alarmed.exchange(false))
{
CurrentMetrics::values[metric]--;
}
}
~ParanoidMetricDecrementor() { decrement(); }
private:
CurrentMetrics::Metric metric;
std::atomic_bool is_alarmed = false;
};
void IBackgroundJobExecutor::execute(JobAndPool job_and_pool)
try
{
auto & pool_config = pools_configs[job_and_pool.pool_type];
const auto max_pool_size = pool_config.get_max_pool_size();
auto metric_decrementor = std::make_shared<ParanoidMetricDecrementor>(pool_config.tasks_metric);
/// If corresponding pool is not full increment metric and assign new job
if (incrementMetricIfLessThanMax(CurrentMetrics::values[pool_config.tasks_metric], max_pool_size))
{
metric_decrementor->alarm();
try /// this try required because we have to manually decrement metric
{
/// Synchronize pool size, because config could be reloaded
pools[job_and_pool.pool_type].setMaxThreads(max_pool_size);
pools[job_and_pool.pool_type].setQueueSize(max_pool_size);
pools[job_and_pool.pool_type].scheduleOrThrowOnError([this, metric_decrementor, job{std::move(job_and_pool.job)}] ()
{
try /// We don't want exceptions in background pool
{
bool job_success = job();
/// Job done, decrement metric and reset no_work counter
metric_decrementor->decrement();
if (job_success)
{
/// Job done, new empty space in pool, schedule background task
runTaskWithoutDelay();
}
else
{
/// Job done, but failed, schedule with backoff
scheduleTask(/* with_backoff = */ true);
}
}
catch (...)
{
metric_decrementor->decrement();
tryLogCurrentException(__PRETTY_FUNCTION__);
scheduleTask(/* with_backoff = */ true);
}
});
/// We've scheduled task in the background pool and when it will finish we will be triggered again. But this task can be
/// extremely long and we may have a lot of other small tasks to do, so we schedule ourselves here.
runTaskWithoutDelay();
}
catch (...)
{
/// With our Pool settings scheduleOrThrowOnError shouldn't throw exceptions, but for safety catch added here
metric_decrementor->decrement();
tryLogCurrentException(__PRETTY_FUNCTION__);
scheduleTask(/* with_backoff = */ true);
}
}
else /// Pool is full and we have some work to do
{
scheduleTask(/* with_backoff = */ false);
}
}
catch (...) /// Exception while we looking for a task, reschedule
{
tryLogCurrentException(__PRETTY_FUNCTION__);
/// Why do we scheduleTask again?
/// To retry on exception, since it may be some temporary exception.
scheduleTask(/* with_backoff = */ true);
}
void IBackgroundJobExecutor::start()
{
std::lock_guard lock(scheduling_task_mutex);
if (!scheduling_task)
{
scheduling_task = getContext()->getSchedulePool().createTask(
getBackgroundTaskName(), [this]{ backgroundTaskFunction(); });
}
scheduling_task->activateAndSchedule();
}
void IBackgroundJobExecutor::finish()
{
std::lock_guard lock(scheduling_task_mutex);
if (scheduling_task)
{
scheduling_task->deactivate();
for (auto & [pool_type, pool] : pools)
pool.wait();
}
}
void IBackgroundJobExecutor::triggerTask()
{
std::lock_guard lock(scheduling_task_mutex);
if (scheduling_task)
runTaskWithoutDelay();
}
void IBackgroundJobExecutor::backgroundTaskFunction()
try
{
if (!scheduleJob())
scheduleTask(/* with_backoff = */ true);
}
catch (...) /// Catch any exception to avoid thread termination.
{
tryLogCurrentException(__PRETTY_FUNCTION__);
scheduleTask(/* with_backoff = */ true);
}
IBackgroundJobExecutor::~IBackgroundJobExecutor()
{
finish();
}
BackgroundJobsExecutor::BackgroundJobsExecutor(
MergeTreeData & data_,
ContextPtr global_context_)
: IBackgroundJobExecutor(
global_context_,
global_context_->getBackgroundProcessingTaskSchedulingSettings(),
{PoolConfig
{
.pool_type = PoolType::MERGE_MUTATE,
.get_max_pool_size = [global_context_] () { return global_context_->getSettingsRef().background_pool_size; },
.tasks_metric = CurrentMetrics::BackgroundPoolTask
},
PoolConfig
{
.pool_type = PoolType::FETCH,
.get_max_pool_size = [global_context_] () { return global_context_->getSettingsRef().background_fetches_pool_size; },
.tasks_metric = CurrentMetrics::BackgroundFetchesPoolTask
}
})
, data(data_)
{
}
String BackgroundJobsExecutor::getBackgroundTaskName() const
{
return data.getStorageID().getFullTableName() + " (dataProcessingTask)";
}
bool BackgroundJobsExecutor::scheduleJob()
{
return data.scheduleDataProcessingJob(*this);
}
BackgroundMovesExecutor::BackgroundMovesExecutor(
MergeTreeData & data_,
ContextPtr global_context_)
: IBackgroundJobExecutor(
global_context_,
global_context_->getBackgroundMoveTaskSchedulingSettings(),
{PoolConfig
{
.pool_type = PoolType::MOVE,
.get_max_pool_size = [global_context_] () { return global_context_->getSettingsRef().background_move_pool_size; },
.tasks_metric = CurrentMetrics::BackgroundMovePoolTask
}
})
, data(data_)
{
}
String BackgroundMovesExecutor::getBackgroundTaskName() const
{
return data.getStorageID().getFullTableName() + " (dataMovingTask)";
}
bool BackgroundMovesExecutor::scheduleJob()
{
return data.scheduleDataMovingJob(*this);
}
}

View File

@ -1,162 +0,0 @@
#pragma once
#include <Storages/MergeTree/MergeTreeData.h>
#include <Common/ThreadPool.h>
#include <Core/BackgroundSchedulePool.h>
#include <pcg_random.hpp>
namespace DB
{
/// Settings for background tasks scheduling. Each background executor has one
/// BackgroundSchedulingPoolTask and depending on execution result may put this
/// task to sleep according to settings. Look at scheduleTask function for details.
struct BackgroundTaskSchedulingSettings
{
double thread_sleep_seconds_random_part = 1.0;
double thread_sleep_seconds_if_nothing_to_do = 0.1;
double task_sleep_seconds_when_no_work_max = 600;
/// For exponential backoff.
double task_sleep_seconds_when_no_work_multiplier = 1.1;
double task_sleep_seconds_when_no_work_random_part = 1.0;
/// Deprecated settings, don't affect background execution
double thread_sleep_seconds = 10;
double task_sleep_seconds_when_no_work_min = 10;
};
/// Pool type where we must execute new job. Each background executor can have several
/// background pools. When it receives new job it will execute new task in corresponding pool.
enum class PoolType
{
MERGE_MUTATE,
MOVE,
FETCH,
};
using BackgroundJobFunc = std::function<bool()>;
/// Result from background job providers. Function which will be executed in pool and pool type.
struct JobAndPool
{
BackgroundJobFunc job;
PoolType pool_type;
};
/// Background jobs executor which execute heavy-weight background tasks for MergTree tables, like
/// background merges, moves, mutations, fetches and so on.
/// Consists of two important parts:
/// 1) Task in background scheduling pool which receives new jobs from storages and put them into required pool.
/// 2) One or more ThreadPool objects, which execute background jobs.
class IBackgroundJobExecutor : protected WithContext
{
protected:
/// Configuration for single background ThreadPool
struct PoolConfig
{
/// This pool type
PoolType pool_type;
/// Max pool size in threads
const std::function<size_t()> get_max_pool_size;
/// Metric that we have to increment when we execute task in this pool
CurrentMetrics::Metric tasks_metric;
};
private:
/// Name for task in background scheduling pool
String task_name;
/// Settings for execution control of background scheduling task
BackgroundTaskSchedulingSettings sleep_settings;
/// Useful for random backoff timeouts generation
pcg64 rng;
/// How many times execution of background job failed or we have
/// no new jobs.
std::atomic<size_t> no_work_done_count{0};
/// Pools where we execute background jobs
std::unordered_map<PoolType, ThreadPool> pools;
/// Configs for background pools
std::unordered_map<PoolType, PoolConfig> pools_configs;
/// Scheduling task which assign jobs in background pool
BackgroundSchedulePool::TaskHolder scheduling_task;
/// Mutex for thread safety
std::mutex scheduling_task_mutex;
/// Mutex for pcg random generator thread safety
std::mutex random_mutex;
public:
/// These three functions are thread safe
/// Start background task and start to assign jobs
void start();
/// Schedule background task as soon as possible, even if it sleep at this
/// moment for some reason.
void triggerTask();
/// Finish execution: deactivate background task and wait already scheduled jobs
void finish();
/// Executes job in a nested pool
void execute(JobAndPool job_and_pool);
/// Just call finish
virtual ~IBackgroundJobExecutor();
protected:
IBackgroundJobExecutor(
ContextPtr global_context_,
const BackgroundTaskSchedulingSettings & sleep_settings_,
const std::vector<PoolConfig> & pools_configs_);
/// Name for task in background schedule pool
virtual String getBackgroundTaskName() const = 0;
/// Schedules a job in a nested pool in this class.
virtual bool scheduleJob() = 0;
private:
/// Function that executes in background scheduling pool
void backgroundTaskFunction();
/// Recalculate timeouts when we have to check for a new job
void scheduleTask(bool with_backoff);
/// Run background task as fast as possible and reset errors counter
void runTaskWithoutDelay();
/// Return random add for sleep in case of error
double getSleepRandomAdd();
};
/// Main jobs executor: merges, mutations, fetches and so on
class BackgroundJobsExecutor final : public IBackgroundJobExecutor
{
private:
MergeTreeData & data;
public:
BackgroundJobsExecutor(
MergeTreeData & data_,
ContextPtr global_context_);
protected:
String getBackgroundTaskName() const override;
bool scheduleJob() override;
};
/// Move jobs executor, move parts between disks in the background
/// Does nothing in case of default configuration
class BackgroundMovesExecutor final : public IBackgroundJobExecutor
{
private:
MergeTreeData & data;
public:
BackgroundMovesExecutor(
MergeTreeData & data_,
ContextPtr global_context_);
protected:
String getBackgroundTaskName() const override;
bool scheduleJob() override;
};
}

View File

@ -0,0 +1,70 @@
#pragma once
#include <memory>
#include <functional>
#include <common/shared_ptr_helper.h>
#include <Interpreters/StorageID.h>
namespace DB
{
/**
* Generic interface for background operations. Simply this is self-made coroutine.
* The main method is executeStep, which will return true
* if the task wants to execute another 'step' in near future and false otherwise.
*
* Each storage assigns some operations such as merges, mutations, fetches, etc.
* We need to ask a storage or some another entity to try to assign another operation when current operation is completed.
*
* Each task corresponds to a storage, that's why there is a method getStorageID.
* This is needed to correctly shutdown a storage, e.g. we need to wait for all background operations to complete.
*/
class IExecutableTask
{
public:
virtual bool executeStep() = 0;
virtual void onCompleted() = 0;
virtual StorageID getStorageID() = 0;
virtual ~IExecutableTask() = default;
};
using ExecutableTaskPtr = std::shared_ptr<IExecutableTask>;
/**
* Some background operations won't represent a coroutines (don't want to be executed step-by-step). For this we have this wrapper.
*/
class ExecutableLambdaAdapter : public shared_ptr_helper<ExecutableLambdaAdapter>, public IExecutableTask
{
public:
template <typename Job, typename Callback>
explicit ExecutableLambdaAdapter(
Job && job_to_execute_,
Callback && job_result_callback_,
StorageID id_)
: job_to_execute(job_to_execute_)
, job_result_callback(job_result_callback_)
, id(id_) {}
bool executeStep() override
{
res = job_to_execute();
job_to_execute = {};
return false;
}
void onCompleted() override { job_result_callback(!res); }
StorageID getStorageID() override { return id; }
private:
bool res = false;
std::function<bool()> job_to_execute;
std::function<void(bool)> job_result_callback;
StorageID id;
};
}

View File

@ -0,0 +1,185 @@
#include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
#include <algorithm>
#include <Common/setThreadName.h>
#include <Storages/MergeTree/BackgroundJobsAssignee.h>
namespace DB
{
String MergeTreeBackgroundExecutor::toString(Type type)
{
switch (type)
{
case Type::MERGE_MUTATE:
return "MergeMutate";
case Type::FETCH:
return "Fetch";
case Type::MOVE:
return "Move";
}
__builtin_unreachable();
}
void MergeTreeBackgroundExecutor::wait()
{
{
std::lock_guard lock(mutex);
shutdown = true;
has_tasks.notify_all();
}
pool.wait();
}
bool MergeTreeBackgroundExecutor::trySchedule(ExecutableTaskPtr task)
{
std::lock_guard lock(mutex);
if (shutdown)
return false;
auto & value = CurrentMetrics::values[metric];
if (value.load() >= static_cast<int64_t>(max_tasks_count))
return false;
pending.push_back(std::make_shared<TaskRuntimeData>(std::move(task), metric));
has_tasks.notify_one();
return true;
}
void MergeTreeBackgroundExecutor::removeTasksCorrespondingToStorage(StorageID id)
{
std::vector<TaskRuntimeDataPtr> tasks_to_wait;
{
std::lock_guard lock(mutex);
/// Erase storage related tasks from pending and select active tasks to wait for
auto it = std::remove_if(pending.begin(), pending.end(),
[&] (auto item) -> bool { return item->task->getStorageID() == id; });
pending.erase(it, pending.end());
/// Copy items to wait for their completion
std::copy_if(active.begin(), active.end(), std::back_inserter(tasks_to_wait),
[&] (auto item) -> bool { return item->task->getStorageID() == id; });
for (auto & item : tasks_to_wait)
item->is_currently_deleting = true;
}
for (auto & item : tasks_to_wait)
item->is_done.wait();
}
void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item)
{
DENY_ALLOCATIONS_IN_SCOPE;
/// All operations with queues are considered no to do any allocations
auto erase_from_active = [this, item]
{
active.erase(std::remove(active.begin(), active.end(), item), active.end());
};
bool need_execute_again = false;
try
{
ALLOW_ALLOCATIONS_IN_SCOPE;
need_execute_again = item->task->executeStep();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
if (need_execute_again)
{
std::lock_guard guard(mutex);
if (item->is_currently_deleting)
{
erase_from_active();
return;
}
pending.push_back(item);
erase_from_active();
has_tasks.notify_one();
return;
}
{
std::lock_guard guard(mutex);
erase_from_active();
has_tasks.notify_one();
}
try
{
ALLOW_ALLOCATIONS_IN_SCOPE;
/// In a situation of a lack of memory this method can throw an exception,
/// because it may interact somehow with BackgroundSchedulePool, which may allocate memory
/// But it is rather safe, because we have try...catch block here, and another one in ThreadPool.
item->task->onCompleted();
item->task.reset();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void MergeTreeBackgroundExecutor::threadFunction()
{
setThreadName(name.c_str());
DENY_ALLOCATIONS_IN_SCOPE;
while (true)
{
try
{
TaskRuntimeDataPtr item;
{
std::unique_lock lock(mutex);
has_tasks.wait(lock, [this](){ return !pending.empty() || shutdown; });
if (shutdown)
break;
item = std::move(pending.front());
pending.pop_front();
active.push_back(item);
}
routine(item);
/// When storage shutdowns it will wait until all related background tasks
/// are finished, because they may want to interact with its fields
/// and this will cause segfault.
if (item->is_currently_deleting)
item->is_done.set();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}
}

View File

@ -0,0 +1,156 @@
#pragma once
#include <deque>
#include <functional>
#include <atomic>
#include <mutex>
#include <future>
#include <condition_variable>
#include <set>
#include <boost/circular_buffer.hpp>
#include <common/shared_ptr_helper.h>
#include <common/logger_useful.h>
#include <Common/ThreadPool.h>
#include <Common/Stopwatch.h>
#include <Storages/MergeTree/IExecutableTask.h>
namespace DB
{
/**
* Executor for a background MergeTree related operations such as merges, mutations, fetches an so on.
* It can execute only successors of ExecutableTask interface.
* Which is a self-written coroutine. It suspends, when returns true from executeStep() method.
*
* There are two queues of a tasks: pending (main queue for all the tasks) and active (currently executing).
* Pending queue is needed since the number of tasks will be more than thread to execute.
* Pending tasks are tasks that successfully scheduled to an executor or tasks that have some extra steps to execute.
* There is an invariant, that task may occur only in one of these queue. It can occur in both queues only in critical sections.
*
* Pending: Active:
*
* |s| |s| |s| |s| |s| |s| |s| |s| |s| |s| |s|
* |s| |s| |s| |s| |s| |s| |s| |s| |s| |s|
* |s| |s| |s| |s| |s| |s| |s|
* |s| |s| |s| |s|
* |s| |s|
* |s|
*
* Each task is simply a sequence of steps. Heavier tasks have longer sequences.
* When a step of a task is executed, we move tasks to pending queue. And take another from the queue's head.
* With these architecture all small merges / mutations will be executed faster, than bigger ones.
*
* We use boost::circular_buffer as a container for queues not to do any allocations.
*
* Another nuisance that we faces with is than background operations always interact with an associated Storage.
* So, when a Storage want to shutdown, it must wait until all its background operaions are finished.
*/
class MergeTreeBackgroundExecutor : public shared_ptr_helper<MergeTreeBackgroundExecutor>
{
public:
enum class Type
{
MERGE_MUTATE,
FETCH,
MOVE
};
MergeTreeBackgroundExecutor(
Type type_,
size_t threads_count_,
size_t max_tasks_count_,
CurrentMetrics::Metric metric_)
: type(type_)
, threads_count(threads_count_)
, max_tasks_count(max_tasks_count_)
, metric(metric_)
{
name = toString(type);
pending.set_capacity(max_tasks_count);
active.set_capacity(max_tasks_count);
pool.setMaxThreads(std::max(1UL, threads_count));
pool.setMaxFreeThreads(std::max(1UL, threads_count));
pool.setQueueSize(std::max(1UL, threads_count));
for (size_t number = 0; number < threads_count; ++number)
pool.scheduleOrThrowOnError([this] { threadFunction(); });
}
~MergeTreeBackgroundExecutor()
{
wait();
}
bool trySchedule(ExecutableTaskPtr task);
void removeTasksCorrespondingToStorage(StorageID id);
void wait();
size_t activeCount()
{
std::lock_guard lock(mutex);
return active.size();
}
size_t pendingCount()
{
std::lock_guard lock(mutex);
return pending.size();
}
private:
static String toString(Type type);
Type type;
String name;
size_t threads_count{0};
size_t max_tasks_count{0};
CurrentMetrics::Metric metric;
/**
* Has RAII class to determine how many tasks are waiting for the execution and executing at the moment.
* Also has some flags and primitives to wait for current task to be executed.
*/
struct TaskRuntimeData
{
TaskRuntimeData(ExecutableTaskPtr && task_, CurrentMetrics::Metric metric_)
: task(std::move(task_))
, increment(std::move(metric_))
{}
ExecutableTaskPtr task;
CurrentMetrics::Increment increment;
std::atomic_bool is_currently_deleting{false};
/// Actually autoreset=false is needed only for unit test
/// where multiple threads could remove tasks corresponding to the same storage
/// This scenario in not possible in reality.
Poco::Event is_done{/*autoreset=*/false};
};
using TaskRuntimeDataPtr = std::shared_ptr<TaskRuntimeData>;
void routine(TaskRuntimeDataPtr item);
void threadFunction();
/// Initially it will be empty
boost::circular_buffer<TaskRuntimeDataPtr> pending{0};
boost::circular_buffer<TaskRuntimeDataPtr> active{0};
std::mutex mutex;
std::condition_variable has_tasks;
std::atomic_bool shutdown{false};
ThreadPool pool;
};
}

View File

@ -200,6 +200,8 @@ MergeTreeData::MergeTreeData(
, data_parts_by_info(data_parts_indexes.get<TagByInfo>())
, data_parts_by_state_and_info(data_parts_indexes.get<TagByStateAndInfo>())
, parts_mover(this)
, background_operations_assignee(*this, BackgroundJobsAssignee::Type::DataProcessing, getContext())
, background_moves_assignee(*this, BackgroundJobsAssignee::Type::Moving, getContext())
{
const auto settings = getSettings();
allow_nullable_key = attach || settings->allow_nullable_key;
@ -305,6 +307,22 @@ MergeTreeData::MergeTreeData(
if (!canUsePolymorphicParts(*settings, &reason) && !reason.empty())
LOG_WARNING(log, "{} Settings 'min_rows_for_wide_part', 'min_bytes_for_wide_part', "
"'min_rows_for_compact_part' and 'min_bytes_for_compact_part' will be ignored.", reason);
common_assignee_trigger = [this] (bool delay) noexcept
{
if (delay)
background_operations_assignee.postpone();
else
background_operations_assignee.trigger();
};
moves_assignee_trigger = [this] (bool delay) noexcept
{
if (delay)
background_moves_assignee.postpone();
else
background_moves_assignee.trigger();
};
}
StoragePolicyPtr MergeTreeData::getStoragePolicy() const
@ -5011,7 +5029,7 @@ MergeTreeData::CurrentlyMovingPartsTagger::~CurrentlyMovingPartsTagger()
}
}
bool MergeTreeData::scheduleDataMovingJob(IBackgroundJobExecutor & executor)
bool MergeTreeData::scheduleDataMovingJob(BackgroundJobsAssignee & assignee)
{
if (parts_mover.moves_blocker.isCancelled())
return false;
@ -5020,10 +5038,11 @@ bool MergeTreeData::scheduleDataMovingJob(IBackgroundJobExecutor & executor)
if (moving_tagger->parts_to_move.empty())
return false;
executor.execute({[this, moving_tagger] () mutable
{
return moveParts(moving_tagger);
}, PoolType::MOVE});
assignee.scheduleMoveTask(ExecutableLambdaAdapter::create(
[this, moving_tagger] () mutable
{
return moveParts(moving_tagger);
}, moves_assignee_trigger, getStorageID()));
return true;
}

View File

@ -3,6 +3,7 @@
#include <Common/SimpleIncrement.h>
#include <Common/MultiVersion.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/BackgroundJobsAssignee.h>
#include <Storages/MergeTree/MergeTreeIndices.h>
#include <Storages/MergeTree/MergeTreePartInfo.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
@ -57,7 +58,6 @@ class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
using ManyExpressionActions = std::vector<ExpressionActionsPtr>;
class MergeTreeDeduplicationLog;
class IBackgroundJobExecutor;
namespace ErrorCodes
{
@ -827,9 +827,9 @@ public:
PinnedPartUUIDsPtr getPinnedPartUUIDs() const;
/// Schedules background job to like merge/mutate/fetch an executor
virtual bool scheduleDataProcessingJob(IBackgroundJobExecutor & executor) = 0;
virtual bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) = 0;
/// Schedules job to move parts between disks/volumes and so on.
bool scheduleDataMovingJob(IBackgroundJobExecutor & executor);
bool scheduleDataMovingJob(BackgroundJobsAssignee & assignee);
bool areBackgroundMovesNeeded() const;
/// Lock part in zookeeper for shared data in several nodes
@ -923,6 +923,23 @@ protected:
MergeTreePartsMover parts_mover;
/// Executors are common for both ReplicatedMergeTree and plain MergeTree
/// but they are being started and finished in derived classes, so let them be protected.
///
/// Why there are two executors, not one? Or an executor for each kind of operation?
/// It is historically formed.
/// Another explanation is that moving operations are common for Replicated and Plain MergeTree classes.
/// Task that schedules this operations is executed with its own timetable and triggered in a specific places in code.
/// And for ReplicatedMergeTree we don't have LogEntry type for this operation.
BackgroundJobsAssignee background_operations_assignee;
BackgroundJobsAssignee background_moves_assignee;
/// Strongly connected with two fields above.
/// Every task that is finished will ask to assign a new one into an executor.
/// These callbacks will be passed to the constructor of each task.
std::function<void(bool)> common_assignee_trigger;
std::function<void(bool)> moves_assignee_trigger;
using DataPartIteratorByInfo = DataPartsIndexes::index<TagByInfo>::type::iterator;
using DataPartIteratorByStateAndInfo = DataPartsIndexes::index<TagByStateAndInfo>::type::iterator;

View File

@ -267,6 +267,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
auto many_data = std::make_shared<ManyAggregatedData>(projection_pipe.numOutputPorts() + ordinary_pipe.numOutputPorts());
size_t counter = 0;
AggregatorListPtr aggregator_list_ptr = std::make_shared<AggregatorList>();
// TODO apply in_order_optimization here
auto build_aggregate_pipe = [&](Pipe & pipe, bool projection)
{
@ -306,7 +308,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
settings.min_count_to_compile_aggregate_expression,
header_before_aggregation); // The source header is also an intermediate header
transform_params = std::make_shared<AggregatingTransformParams>(std::move(params), query_info.projection->aggregate_final);
transform_params = std::make_shared<AggregatingTransformParams>(
std::move(params), aggregator_list_ptr, query_info.projection->aggregate_final);
/// This part is hacky.
/// We want AggregatingTransform to work with aggregate states instead of normal columns.
@ -336,7 +339,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
settings.compile_aggregate_expressions,
settings.min_count_to_compile_aggregate_expression);
transform_params = std::make_shared<AggregatingTransformParams>(std::move(params), query_info.projection->aggregate_final);
transform_params = std::make_shared<AggregatingTransformParams>(
std::move(params), aggregator_list_ptr, query_info.projection->aggregate_final);
}
pipe.resize(pipe.numOutputPorts(), true, true);

View File

@ -89,6 +89,7 @@ struct Settings;
M(Bool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.", 0) \
M(Seconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.", 0) \
M(Bool, detach_old_local_parts_when_cloning_replica, true, "Do not remove old local parts when repairing lost replica.", 0) \
M(Bool, detach_not_byte_identical_parts, false, "Do not remove non byte-idential parts for ReplicatedMergeTree, instead detach them (maybe useful for further analysis).", 0) \
M(UInt64, max_replicated_fetches_network_bandwidth, 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, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \
\

View File

@ -37,7 +37,7 @@ void MergeTreeSink::consume(Chunk chunk)
PartLog::addNewPart(storage.getContext(), part, watch.elapsed());
/// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'.
storage.background_executor.triggerTask();
storage.background_operations_assignee.trigger();
}
}
}

View File

@ -373,7 +373,7 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr, bool is_final) c
void MergeTreeWhereOptimizer::determineArrayJoinedNames(ASTSelectQuery & select)
{
auto array_join_expression_list = select.arrayJoinExpressionList();
auto [array_join_expression_list, _] = select.arrayJoinExpressionList();
/// much simplified code from ExpressionAnalyzer::getArrayJoinedColumns()
if (!array_join_expression_list)

View File

@ -144,7 +144,10 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper)
updated = true;
}
zookeeper->tryGet(fs::path(replica_path) / "mutation_pointer", mutation_pointer);
{ /// Mutation pointer is a part of "state" and must be updated with state mutex
std::lock_guard lock(state_mutex);
zookeeper->tryGet(fs::path(replica_path) / "mutation_pointer", mutation_pointer);
}
}
updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, {});
@ -624,7 +627,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper
}
}
storage.background_executor.triggerTask();
storage.background_operations_assignee.trigger();
}
return stat.version;
@ -713,7 +716,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C
}
if (some_active_mutations_were_killed)
storage.background_executor.triggerTask();
storage.background_operations_assignee.trigger();
if (!entries_to_load.empty())
{
@ -847,7 +850,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation(
}
if (mutation_was_active)
storage.background_executor.triggerTask();
storage.background_operations_assignee.trigger();
return entry;
}

View File

@ -204,7 +204,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
storage.partial_shutdown_event.reset();
/// Start queue processing
storage.background_executor.start();
storage.background_operations_assignee.start();
storage.queue_updating_task->activateAndSchedule();
storage.mutations_updating_task->activateAndSchedule();
@ -389,7 +389,7 @@ void ReplicatedMergeTreeRestartingThread::partialShutdown()
auto fetch_lock = storage.fetcher.blocker.cancel();
auto merge_lock = storage.merger_mutator.merges_blocker.cancel();
auto move_lock = storage.parts_mover.moves_blocker.cancel();
storage.background_executor.finish();
storage.background_operations_assignee.finish();
}
LOG_TRACE(log, "Threads finished");

View File

@ -0,0 +1,155 @@
#include <gtest/gtest.h>
#include <atomic>
#include <barrier>
#include <memory>
#include <random>
#include <Storages/MergeTree/IExecutableTask.h>
#include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
using namespace DB;
namespace CurrentMetrics
{
extern const Metric BackgroundPoolTask;
}
std::random_device device;
class FakeExecutableTask : public IExecutableTask
{
public:
explicit FakeExecutableTask(String name_) : generator(device()), distribution(0, 5), name(name_)
{
}
bool executeStep() override
{
auto sleep_time = distribution(generator);
std::this_thread::sleep_for(std::chrono::milliseconds(5 * sleep_time));
auto choice = distribution(generator);
if (choice == 0)
throw std::runtime_error("Unlucky...");
return false;
}
StorageID getStorageID() override
{
return {"test", name};
}
void onCompleted() override
{
auto choice = distribution(generator);
if (choice == 0)
throw std::runtime_error("Unlucky...");
}
private:
std::mt19937 generator;
std::uniform_int_distribution<> distribution;
String name;
std::function<void()> on_completed;
};
TEST(Executor, RemoveTasks)
{
const size_t tasks_kinds = 25;
const size_t batch = 100;
auto executor = DB::MergeTreeBackgroundExecutor::create
(
DB::MergeTreeBackgroundExecutor::Type::MERGE_MUTATE,
tasks_kinds,
tasks_kinds * batch,
CurrentMetrics::BackgroundPoolTask
);
for (size_t i = 0; i < batch; ++i)
for (size_t j = 0; j < tasks_kinds; ++j)
ASSERT_TRUE(
executor->trySchedule(std::make_shared<FakeExecutableTask>(std::to_string(j)))
);
std::vector<std::thread> threads(batch);
auto remover_routine = [&] ()
{
for (size_t j = 0; j < tasks_kinds; ++j)
executor->removeTasksCorrespondingToStorage({"test", std::to_string(j)});
};
for (auto & thread : threads)
thread = std::thread(remover_routine);
for (auto & thread : threads)
thread.join();
ASSERT_EQ(executor->activeCount(), 0);
ASSERT_EQ(executor->pendingCount(), 0);
ASSERT_EQ(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask], 0);
executor->wait();
}
TEST(Executor, RemoveTasksStress)
{
const size_t tasks_kinds = 25;
const size_t batch = 100;
const size_t schedulers_count = 5;
const size_t removers_count = 5;
auto executor = DB::MergeTreeBackgroundExecutor::create
(
DB::MergeTreeBackgroundExecutor::Type::MERGE_MUTATE,
tasks_kinds,
tasks_kinds * batch * (schedulers_count + removers_count),
CurrentMetrics::BackgroundPoolTask
);
std::barrier barrier(schedulers_count + removers_count);
auto scheduler_routine = [&] ()
{
barrier.arrive_and_wait();
for (size_t i = 0; i < batch; ++i)
for (size_t j = 0; j < tasks_kinds; ++j)
executor->trySchedule(std::make_shared<FakeExecutableTask>(std::to_string(j)));
};
auto remover_routine = [&] ()
{
barrier.arrive_and_wait();
for (size_t j = 0; j < tasks_kinds; ++j)
executor->removeTasksCorrespondingToStorage({"test", std::to_string(j)});
};
std::vector<std::thread> schedulers(schedulers_count);
for (auto & scheduler : schedulers)
scheduler = std::thread(scheduler_routine);
std::vector<std::thread> removers(removers_count);
for (auto & remover : removers)
remover = std::thread(remover_routine);
for (auto & scheduler : schedulers)
scheduler.join();
for (auto & remover : removers)
remover.join();
for (size_t j = 0; j < tasks_kinds; ++j)
executor->removeTasksCorrespondingToStorage({"test", std::to_string(j)});
ASSERT_EQ(executor->activeCount(), 0);
ASSERT_EQ(executor->pendingCount(), 0);
ASSERT_EQ(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask], 0);
executor->wait();
}

View File

@ -82,9 +82,6 @@ StorageMergeTree::StorageMergeTree(
, reader(*this)
, writer(*this)
, merger_mutator(*this, getContext()->getSettingsRef().background_pool_size)
, background_executor(*this, getContext())
, background_moves_executor(*this, getContext())
{
loadDataParts(has_force_restore_data_flag);
@ -115,7 +112,7 @@ void StorageMergeTree::startup()
try
{
background_executor.start();
background_operations_assignee.start();
startBackgroundMovesIfNeeded();
}
catch (...)
@ -153,8 +150,8 @@ void StorageMergeTree::shutdown()
merger_mutator.merges_blocker.cancelForever();
parts_mover.moves_blocker.cancelForever();
background_executor.finish();
background_moves_executor.finish();
background_operations_assignee.finish();
background_moves_assignee.finish();
try
{
@ -414,7 +411,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String
LOG_INFO(log, "Added mutation: {}", mutation_file_name);
}
background_executor.triggerTask();
background_operations_assignee.trigger();
return version;
}
@ -640,7 +637,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id)
}
/// Maybe there is another mutation that was blocked by the killed one. Try to execute it immediately.
background_executor.triggerTask();
background_operations_assignee.trigger();
return CancellationCode::CancelSent;
}
@ -1044,7 +1041,7 @@ bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_sn
return true;
}
bool StorageMergeTree::scheduleDataProcessingJob(IBackgroundJobExecutor & executor) //-V657
bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) //-V657
{
if (shutdown_called)
return false;
@ -1076,46 +1073,50 @@ bool StorageMergeTree::scheduleDataProcessingJob(IBackgroundJobExecutor & execut
if (merge_entry)
{
executor.execute({[this, metadata_snapshot, merge_entry, share_lock] () mutable
{
return mergeSelectedParts(metadata_snapshot, false, {}, *merge_entry, share_lock);
}, PoolType::MERGE_MUTATE});
assignee.scheduleMergeMutateTask(ExecutableLambdaAdapter::create(
[this, metadata_snapshot, merge_entry, share_lock] () mutable
{
return mergeSelectedParts(metadata_snapshot, false, {}, *merge_entry, share_lock);
}, common_assignee_trigger, getStorageID()));
return true;
}
if (mutate_entry)
{
executor.execute({[this, metadata_snapshot, merge_entry, mutate_entry, share_lock] () mutable
{
assignee.scheduleMergeMutateTask(ExecutableLambdaAdapter::create(
[this, metadata_snapshot, merge_entry, mutate_entry, share_lock] () mutable
{
return mutateSelectedPart(metadata_snapshot, *mutate_entry, share_lock);
}, PoolType::MERGE_MUTATE});
}, common_assignee_trigger, getStorageID()));
return true;
}
bool executed = false;
bool scheduled = false;
if (time_after_previous_cleanup_temporary_directories.compareAndRestartDeferred(getContext()->getSettingsRef().merge_tree_clear_old_temporary_directories_interval_seconds))
{
executor.execute({[this, share_lock] ()
{
clearOldTemporaryDirectories(getSettings()->temporary_directories_lifetime.totalSeconds());
return true;
}, PoolType::MERGE_MUTATE});
executed = true;
assignee.scheduleMergeMutateTask(ExecutableLambdaAdapter::create(
[this, share_lock] ()
{
clearOldTemporaryDirectories(getSettings()->temporary_directories_lifetime.totalSeconds());
return true;
}, common_assignee_trigger, getStorageID()));
scheduled = true;
}
if (time_after_previous_cleanup_parts.compareAndRestartDeferred(getContext()->getSettingsRef().merge_tree_clear_old_parts_interval_seconds))
{
executor.execute({[this, share_lock] ()
{
/// All use relative_data_path which changes during rename
/// so execute under share lock.
clearOldPartsFromFilesystem();
clearOldWriteAheadLogs();
clearOldMutations();
clearEmptyParts();
return true;
}, PoolType::MERGE_MUTATE});
executed = true;
assignee.scheduleMergeMutateTask(ExecutableLambdaAdapter::create(
[this, share_lock] ()
{
/// All use relative_data_path which changes during rename
/// so execute under share lock.
clearOldPartsFromFilesystem();
clearOldWriteAheadLogs();
clearOldMutations();
clearEmptyParts();
return true;
}, common_assignee_trigger, getStorageID()));
scheduled = true;
}
return executed;
return scheduled;
}
Int64 StorageMergeTree::getCurrentMutationVersion(
@ -1566,9 +1567,9 @@ ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type)
void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type)
{
if (action_type == ActionLocks::PartsMerge || action_type == ActionLocks::PartsTTLMerge)
background_executor.triggerTask();
background_operations_assignee.trigger();
else if (action_type == ActionLocks::PartsMove)
background_moves_executor.triggerTask();
background_moves_assignee.trigger();
}
CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context)
@ -1646,7 +1647,7 @@ MutationCommands StorageMergeTree::getFirstAlterMutationCommandsForPart(const Da
void StorageMergeTree::startBackgroundMovesIfNeeded()
{
if (areBackgroundMovesNeeded())
background_moves_executor.start();
background_moves_assignee.start();
}
std::unique_ptr<MergeTreeSettings> StorageMergeTree::getDefaultSettings() const

View File

@ -16,7 +16,6 @@
#include <Disks/StoragePolicy.h>
#include <Common/SimpleIncrement.h>
#include <Storages/MergeTree/BackgroundJobsExecutor.h>
namespace DB
@ -96,9 +95,10 @@ public:
RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override;
bool scheduleDataProcessingJob(IBackgroundJobExecutor & executor) override;
bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override;
MergeTreeDeduplicationLog * getDeduplicationLog() { return deduplication_log.get(); }
private:
/// Mutex and condvar for synchronous mutations wait
@ -108,8 +108,6 @@ private:
MergeTreeDataSelectExecutor reader;
MergeTreeDataWriter writer;
MergeTreeDataMergerMutator merger_mutator;
BackgroundJobsExecutor background_executor;
BackgroundMovesExecutor background_moves_executor;
std::unique_ptr<MergeTreeDeduplicationLog> deduplication_log;

View File

@ -17,6 +17,7 @@
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
#include <Storages/MergeTree/MergeList.h>
#include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
#include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Storages/MergeTree/PinnedPartUUIDs.h>
#include <Storages/MergeTree/PartitionPruner.h>
@ -283,8 +284,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
, merge_strategy_picker(*this)
, queue(*this, merge_strategy_picker)
, fetcher(*this)
, background_executor(*this, getContext())
, background_moves_executor(*this, getContext())
, cleanup_thread(*this)
, part_check_thread(*this)
, restarting_thread(*this)
@ -1811,7 +1810,10 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
write_part_log(ExecutionStatus::fromCurrentException());
tryRemovePartImmediately(std::move(part));
if (storage_settings_ptr->detach_not_byte_identical_parts)
forgetPartAndMoveToDetached(std::move(part), "merge-not-byte-identical");
else
tryRemovePartImmediately(std::move(part));
/// No need to delete the part from ZK because we can be sure that the commit transaction
/// didn't go through.
@ -1935,7 +1937,10 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM
write_part_log(ExecutionStatus::fromCurrentException());
tryRemovePartImmediately(std::move(new_part));
if (storage_settings_ptr->detach_not_byte_identical_parts)
forgetPartAndMoveToDetached(std::move(new_part), "mutate-not-byte-identical");
else
tryRemovePartImmediately(std::move(new_part));
/// No need to delete the part from ZK because we can be sure that the commit transaction
/// didn't go through.
@ -3174,7 +3179,7 @@ bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::Sel
});
}
bool StorageReplicatedMergeTree::scheduleDataProcessingJob(IBackgroundJobExecutor & executor)
bool StorageReplicatedMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assignee)
{
/// If replication queue is stopped exit immediately as we successfully executed the task
if (queue.actions_blocker.isCancelled())
@ -3189,18 +3194,20 @@ bool StorageReplicatedMergeTree::scheduleDataProcessingJob(IBackgroundJobExecuto
/// Depending on entry type execute in fetches (small) pool or big merge_mutate pool
if (selected_entry->log_entry->type == LogEntry::GET_PART)
{
executor.execute({[this, selected_entry] () mutable
{
return processQueueEntry(selected_entry);
}, PoolType::FETCH});
assignee.scheduleFetchTask(ExecutableLambdaAdapter::create(
[this, selected_entry] () mutable
{
return processQueueEntry(selected_entry);
}, common_assignee_trigger, getStorageID()));
return true;
}
else
{
executor.execute({[this, selected_entry] () mutable
{
return processQueueEntry(selected_entry);
}, PoolType::MERGE_MUTATE});
assignee.scheduleMergeMutateTask(ExecutableLambdaAdapter::create(
[this, selected_entry] () mutable
{
return processQueueEntry(selected_entry);
}, common_assignee_trigger, getStorageID()));
return true;
}
}
@ -4336,7 +4343,7 @@ void StorageReplicatedMergeTree::shutdown()
parts_mover.moves_blocker.cancelForever();
restarting_thread.shutdown();
background_executor.finish();
background_operations_assignee.finish();
part_moves_between_shards_orchestrator.shutdown();
{
@ -4346,7 +4353,7 @@ void StorageReplicatedMergeTree::shutdown()
/// MUTATE, etc. query.
queue.pull_log_blocker.cancelForever();
}
background_moves_executor.finish();
background_moves_assignee.finish();
auto data_parts_exchange_ptr = std::atomic_exchange(&data_parts_exchange_endpoint, InterserverIOEndpointPtr{});
if (data_parts_exchange_ptr)
@ -6946,9 +6953,9 @@ void StorageReplicatedMergeTree::onActionLockRemove(StorageActionBlockType actio
if (action_type == ActionLocks::PartsMerge || action_type == ActionLocks::PartsTTLMerge
|| action_type == ActionLocks::PartsFetch || action_type == ActionLocks::PartsSend
|| action_type == ActionLocks::ReplicationQueue)
background_executor.triggerTask();
background_operations_assignee.trigger();
else if (action_type == ActionLocks::PartsMove)
background_moves_executor.triggerTask();
background_moves_assignee.trigger();
}
bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UInt64 max_wait_milliseconds)
@ -6960,7 +6967,7 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI
/// This is significant, because the execution of this task could be delayed at BackgroundPool.
/// And we force it to be executed.
background_executor.triggerTask();
background_operations_assignee.trigger();
Poco::Event target_size_event;
auto callback = [&target_size_event, queue_size] (size_t new_queue_size)
@ -7194,7 +7201,7 @@ MutationCommands StorageReplicatedMergeTree::getFirstAlterMutationCommandsForPar
void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded()
{
if (areBackgroundMovesNeeded())
background_moves_executor.start();
background_moves_assignee.start();
}
std::unique_ptr<MergeTreeSettings> StorageReplicatedMergeTree::getDefaultSettings() const

View File

@ -29,7 +29,7 @@
#include <Common/Throttler.h>
#include <Core/BackgroundSchedulePool.h>
#include <Processors/Pipe.h>
#include <Storages/MergeTree/BackgroundJobsExecutor.h>
#include <Storages/MergeTree/BackgroundJobsAssignee.h>
namespace DB
@ -218,7 +218,7 @@ public:
const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock, Poco::Logger * logger);
/// Schedules job to execute in background pool (merge, mutate, drop range and so on)
bool scheduleDataProcessingJob(IBackgroundJobExecutor & executor) override;
bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override;
/// Checks that fetches are not disabled with action blocker and pool for fetches
/// is not overloaded
@ -350,9 +350,6 @@ private:
int metadata_version = 0;
/// Threads.
BackgroundJobsExecutor background_executor;
BackgroundMovesExecutor background_moves_executor;
/// A task that keeps track of the updates in the logs of all replicas and loads them into the queue.
bool queue_update_in_progress = false;
BackgroundSchedulePool::TaskHolder queue_updating_task;

View File

@ -2,6 +2,8 @@
# You can also regenerate it manually this way:
# execute_process(COMMAND "${CMAKE_CURRENT_SOURCE_DIR}/StorageSystemContributors.sh")
include(${ClickHouse_SOURCE_DIR}/cmake/embed_binary.cmake)
set (CONFIG_BUILD "${CMAKE_CURRENT_BINARY_DIR}/StorageSystemBuildOptions.generated.cpp")
get_property (BUILD_COMPILE_DEFINITIONS DIRECTORY ${ClickHouse_SOURCE_DIR} PROPERTY COMPILE_DEFINITIONS)
get_property (BUILD_INCLUDE_DIRECTORIES DIRECTORY ${ClickHouse_SOURCE_DIR} PROPERTY INCLUDE_DIRECTORIES)
@ -41,7 +43,16 @@ list (APPEND storages_system_sources ${GENERATED_TIMEZONES_SRC})
# Overlength strings
set_source_files_properties(${GENERATED_LICENSES_SRC} PROPERTIES COMPILE_FLAGS -w)
clickhouse_embed_binaries(
TARGET information_schema_metadata
RESOURCE_DIR "${CMAKE_CURRENT_SOURCE_DIR}/InformationSchema/"
RESOURCES schemata.sql tables.sql views.sql columns.sql
)
add_library(clickhouse_storages_system ${storages_system_headers} ${storages_system_sources})
add_dependencies(clickhouse_storages_system information_schema_metadata)
target_link_libraries(clickhouse_storages_system PRIVATE
dbms
common
@ -49,4 +60,5 @@ target_link_libraries(clickhouse_storages_system PRIVATE
clickhouse_common_zookeeper
clickhouse_parsers
Poco::JSON
INTERFACE "-Wl,${WHOLE_ARCHIVE} $<TARGET_FILE:information_schema_metadata> -Wl,${NO_WHOLE_ARCHIVE}"
)

View File

@ -0,0 +1,74 @@
ATTACH VIEW columns
(
`table_catalog` String,
`table_schema` String,
`table_name` String,
`column_name` String,
`ordinal_position` UInt64,
`column_default` String,
`is_nullable` UInt8,
`data_type` String,
`character_maximum_length` Nullable(UInt64),
`character_octet_length` Nullable(UInt64),
`numeric_precision` Nullable(UInt64),
`numeric_precision_radix` Nullable(UInt64),
`numeric_scale` Nullable(UInt64),
`datetime_precision` Nullable(UInt64),
`character_set_catalog` Nullable(String),
`character_set_schema` Nullable(String),
`character_set_name` Nullable(String),
`collation_catalog` Nullable(String),
`collation_schema` Nullable(String),
`collation_name` Nullable(String),
`domain_catalog` Nullable(String),
`domain_schema` Nullable(String),
`domain_name` Nullable(String),
`TABLE_CATALOG` String ALIAS table_catalog,
`TABLE_SCHEMA` String ALIAS table_schema,
`TABLE_NAME` String ALIAS table_name,
`COLUMN_NAME` String ALIAS column_name,
`ORDINAL_POSITION` UInt64 ALIAS ordinal_position,
`COLUMN_DEFAULT` String ALIAS column_default,
`IS_NULLABLE` UInt8 ALIAS is_nullable,
`DATA_TYPE` String ALIAS data_type,
`CHARACTER_MAXIMUM_LENGTH` Nullable(UInt64) ALIAS character_maximum_length,
`CHARACTER_OCTET_LENGTH` Nullable(UInt64) ALIAS character_octet_length,
`NUMERIC_PRECISION` Nullable(UInt64) ALIAS numeric_precision,
`NUMERIC_PRECISION_RADIX` Nullable(UInt64) ALIAS numeric_precision_radix,
`NUMERIC_SCALE` Nullable(UInt64) ALIAS numeric_scale,
`DATETIME_PRECISION` Nullable(UInt64) ALIAS datetime_precision,
`CHARACTER_SET_CATALOG` Nullable(String) ALIAS character_set_catalog,
`CHARACTER_SET_SCHEMA` Nullable(String) ALIAS character_set_schema,
`CHARACTER_SET_NAME` Nullable(String) ALIAS character_set_name,
`COLLATION_CATALOG` Nullable(String) ALIAS collation_catalog,
`COLLATION_SCHEMA` Nullable(String) ALIAS collation_schema,
`COLLATION_NAME` Nullable(String) ALIAS collation_name,
`DOMAIN_CATALOG` Nullable(String) ALIAS domain_catalog,
`DOMAIN_SCHEMA` Nullable(String) ALIAS domain_schema,
`DOMAIN_NAME` Nullable(String) ALIAS domain_name
) AS
SELECT
database AS table_catalog,
database AS table_schema,
table AS table_name,
name AS column_name,
position AS ordinal_position,
default_expression AS column_default,
type LIKE 'Nullable(%)' AS is_nullable,
type AS data_type,
character_octet_length AS character_maximum_length,
character_octet_length,
numeric_precision,
numeric_precision_radix,
numeric_scale,
datetime_precision,
NULL AS character_set_catalog,
NULL AS character_set_schema,
NULL AS character_set_name,
NULL AS collation_catalog,
NULL AS collation_schema,
NULL AS collation_name,
NULL AS domain_catalog,
NULL AS domain_schema,
NULL AS domain_name
FROM system.columns

View File

@ -0,0 +1,26 @@
ATTACH VIEW schemata
(
`catalog_name` String,
`schema_name` String,
`schema_owner` String,
`default_character_set_catalog` Nullable(String),
`default_character_set_schema` Nullable(String),
`default_character_set_name` Nullable(String),
`sql_path` Nullable(String),
`CATALOG_NAME` String ALIAS catalog_name,
`SCHEMA_NAME` String ALIAS schema_name,
`SCHEMA_OWNER` String ALIAS schema_owner,
`DEFAULT_CHARACTER_SET_CATALOG` Nullable(String) ALIAS default_character_set_catalog,
`DEFAULT_CHARACTER_SET_SCHEMA` Nullable(String) ALIAS default_character_set_schema,
`DEFAULT_CHARACTER_SET_NAME` Nullable(String) ALIAS default_character_set_name,
`SQL_PATH` Nullable(String) ALIAS sql_path
) AS
SELECT
name AS catalog_name,
name AS schema_name,
'default' AS schema_owner,
NULL AS default_character_set_catalog,
NULL AS default_character_set_schema,
NULL AS default_character_set_name,
NULL AS sql_path
FROM system.databases

View File

@ -0,0 +1,17 @@
ATTACH VIEW tables
(
`table_catalog` String,
`table_schema` String,
`table_name` String,
`table_type` Enum8('BASE TABLE' = 1, 'VIEW' = 2, 'FOREIGN TABLE' = 3, 'LOCAL TEMPORARY' = 4, 'SYSTEM VIEW' = 5),
`TABLE_CATALOG` String ALIAS table_catalog,
`TABLE_SCHEMA` String ALIAS table_schema,
`TABLE_NAME` String ALIAS table_name,
`TABLE_TYPE` Enum8('BASE TABLE' = 1, 'VIEW' = 2, 'FOREIGN TABLE' = 3, 'LOCAL TEMPORARY' = 4, 'SYSTEM VIEW' = 5) ALIAS table_type
) AS
SELECT
database AS table_catalog,
database AS table_schema,
name AS table_name,
multiIf(is_temporary, 4, engine like '%View', 2, engine LIKE 'System%', 5, has_own_data = 0, 3, 1) AS table_type
FROM system.tables

View File

@ -0,0 +1,36 @@
ATTACH VIEW views
(
`table_catalog` String,
`table_schema` String,
`table_name` String,
`view_definition` String,
`check_option` String,
`is_updatable` Enum8('NO' = 0, 'YES' = 1),
`is_insertable_into` Enum8('NO' = 0, 'YES' = 1),
`is_trigger_updatable` Enum8('NO' = 0, 'YES' = 1),
`is_trigger_deletable` Enum8('NO' = 0, 'YES' = 1),
`is_trigger_insertable_into` Enum8('NO' = 0, 'YES' = 1),
`TABLE_CATALOG` String ALIAS table_catalog,
`TABLE_SCHEMA` String ALIAS table_schema,
`TABLE_NAME` String ALIAS table_name,
`VIEW_DEFINITION` String ALIAS view_definition,
`CHECK_OPTION` String ALIAS check_option,
`IS_UPDATABLE` Enum8('NO' = 0, 'YES' = 1) ALIAS is_updatable,
`IS_INSERTABLE_INTO` Enum8('NO' = 0, 'YES' = 1) ALIAS is_insertable_into,
`IS_TRIGGER_UPDATABLE` Enum8('NO' = 0, 'YES' = 1) ALIAS is_trigger_updatable,
`IS_TRIGGER_DELETABLE` Enum8('NO' = 0, 'YES' = 1) ALIAS is_trigger_deletable,
`IS_TRIGGER_INSERTABLE_INTO` Enum8('NO' = 0, 'YES' = 1) ALIAS is_trigger_insertable_into
) AS
SELECT
database AS table_catalog,
database AS table_schema,
name AS table_name,
as_select AS view_definition,
'NONE' AS check_option,
0 AS is_updatable,
engine = 'MaterializedView' AS is_insertable_into,
0 AS is_trigger_updatable,
0 AS is_trigger_deletable,
0 AS is_trigger_insertable_into
FROM system.tables
WHERE engine LIKE '%View'

View File

@ -5,6 +5,8 @@
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Storages/VirtualColumnUtils.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTSelectQuery.h>
@ -44,6 +46,12 @@ StorageSystemColumns::StorageSystemColumns(const StorageID & table_id_)
{ "is_in_primary_key", std::make_shared<DataTypeUInt8>() },
{ "is_in_sampling_key", std::make_shared<DataTypeUInt8>() },
{ "compression_codec", std::make_shared<DataTypeString>() },
{ "character_octet_length", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()) },
{ "numeric_precision", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()) },
{ "numeric_precision_radix", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()) },
{ "numeric_scale", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()) },
{ "datetime_precision", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()) },
}));
setInMemoryMetadata(storage_metadata);
}
@ -218,6 +226,60 @@ protected:
res_columns[res_index++]->insertDefault();
}
/// character_octet_length makes sense for FixedString only
DataTypePtr not_nullable_type = removeNullable(column.type);
if (columns_mask[src_index++])
{
if (isFixedString(not_nullable_type))
res_columns[res_index++]->insert(not_nullable_type->getSizeOfValueInMemory());
else
res_columns[res_index++]->insertDefault();
}
/// numeric_precision
if (columns_mask[src_index++])
{
if (isInteger(not_nullable_type))
res_columns[res_index++]->insert(not_nullable_type->getSizeOfValueInMemory() * 8); /// radix is 2
else if (isDecimal(not_nullable_type))
res_columns[res_index++]->insert(getDecimalPrecision(*not_nullable_type)); /// radix is 10
else
res_columns[res_index++]->insertDefault();
}
/// numeric_precision_radix
if (columns_mask[src_index++])
{
if (isInteger(not_nullable_type))
res_columns[res_index++]->insert(2);
else if (isDecimal(not_nullable_type))
res_columns[res_index++]->insert(10);
else
res_columns[res_index++]->insertDefault();
}
/// numeric_scale
if (columns_mask[src_index++])
{
if (isInteger(not_nullable_type))
res_columns[res_index++]->insert(0);
else if (isDecimal(not_nullable_type))
res_columns[res_index++]->insert(getDecimalScale(*not_nullable_type));
else
res_columns[res_index++]->insertDefault();
}
/// datetime_precision
if (columns_mask[src_index++])
{
if (isDateTime64(not_nullable_type))
res_columns[res_index++]->insert(assert_cast<const DataTypeDateTime64 &>(*not_nullable_type).getScale());
else if (isDateOrDate32(not_nullable_type) || isDateTime(not_nullable_type) || isDateTime64(not_nullable_type))
res_columns[res_index++]->insert(0);
else
res_columns[res_index++]->insertDefault();
}
++rows_count;
}
}

View File

@ -15,6 +15,7 @@ const char * auto_contributors[] {
"adevyatova",
"ageraab",
"Ahmed Dardery",
"Aimiyoo",
"akazz",
"Akazz",
"akonyaev",
@ -33,6 +34,7 @@ const char * auto_contributors[] {
"alesapin",
"Alex",
"Alexander Avdonkin",
"Alexander Bezpiatov",
"Alexander Burmak",
"Alexander Ermolaev",
"Alexander Gololobov",
@ -43,6 +45,7 @@ const char * auto_contributors[] {
"Alexander Krasheninnikov",
"Alexander Kuranoff",
"Alexander Kuzmenkov",
"Alexander Lodin",
"Alexander Lukin",
"Alexander Makarov",
"Alexander Marshalov",
@ -54,6 +57,7 @@ const char * auto_contributors[] {
"Alexander Sapin",
"Alexander Tokmakov",
"Alexander Tretiakov",
"Alexandra",
"Alexandra Latysheva",
"Alexandre Snarskii",
"Alexandr Kondratev",
@ -63,6 +67,7 @@ const char * auto_contributors[] {
"Alexei Averchenko",
"Alexey",
"Alexey Arno",
"Alexey Boykov",
"Alexey Dushechkin",
"Alexey Elymanov",
"Alexey Ilyukhov",
@ -95,12 +100,14 @@ const char * auto_contributors[] {
"Anatoly Pugachev",
"ana-uvarova",
"AnaUvarova",
"Andr0901",
"Andreas Hunkeler",
"AndreevDm",
"Andrei Bodrov",
"Andrei Chulkov",
"andrei-karpliuk",
"Andrei Nekrashevich",
"ANDREI STAROVEROV",
"Andrew Grigorev",
"Andrew Onyshchuk",
"andrewsg",
@ -115,10 +122,12 @@ const char * auto_contributors[] {
"Andrey Skobtsov",
"Andrey Urusov",
"Andrey Z",
"Andy Liang",
"Andy Yang",
"Anmol Arora",
"Anna",
"Anna Shakhova",
"anneji-dev",
"annvsh",
"anrodigina",
"Anthony N. Simon",
@ -140,6 +149,7 @@ const char * auto_contributors[] {
"aprudaev",
"Ariel Robaldo",
"Arsen Hakobyan",
"Arslan G",
"ArtCorp",
"Artem Andreenko",
"Artemeey",
@ -155,6 +165,7 @@ const char * auto_contributors[] {
"artpaul",
"Artur",
"Artur Beglaryan",
"Artur Filatenkov",
"AsiaKorushkina",
"asiana21",
"atereh",
@ -192,11 +203,14 @@ const char * auto_contributors[] {
"Boris Granveaud",
"Bowen Masco",
"bo zeng",
"Braulio Valdivielso",
"Brett Hoerner",
"BSD_Conqueror",
"bseng",
"Bulat Gaifullin",
"Carbyn",
"caspian",
"Caspian",
"cekc",
"centos7",
"champtar",
@ -263,6 +277,7 @@ const char * auto_contributors[] {
"Dmitrii Raev",
"dmitriiut",
"Dmitriy",
"Dmitriy Lushnikov",
"Dmitry",
"Dmitry Belyavtsev",
"Dmitry Bilunov",
@ -281,6 +296,7 @@ const char * auto_contributors[] {
"Dongdong Yang",
"DoomzD",
"Dr. Strange Looker",
"DuckSoft",
"d.v.semenov",
"eaxdev",
"eejoin",
@ -328,6 +344,7 @@ const char * auto_contributors[] {
"favstovol",
"FawnD2",
"FeehanG",
"feihengye",
"felixoid",
"felixxdu",
"feng lv",
@ -335,6 +352,7 @@ const char * auto_contributors[] {
"fessmage",
"FgoDt",
"fibersel",
"Filatenkov Artur",
"filimonov",
"filipe",
"Filipe Caixeta",
@ -361,6 +379,7 @@ const char * auto_contributors[] {
"George",
"George3d6",
"George G",
"Georgy Ginzburg",
"Gervasio Varela",
"ggerogery",
"giordyb",
@ -386,15 +405,18 @@ const char * auto_contributors[] {
"hchen9",
"hcz",
"heng zhao",
"hermano",
"hexiaoting",
"Hiroaki Nakamura",
"hotid",
"huangzhaowei",
"HuFuwang",
"Hui Wang",
"hustnn",
"huzhichengdd",
"ice1x",
"idfer",
"igomac",
"igor",
"Igor",
"Igor Hatarist",
@ -404,6 +426,7 @@ const char * auto_contributors[] {
"Igr",
"Igr Mineev",
"ikarishinjieva",
"Ikko Ashimine",
"ikopylov",
"Ildar Musin",
"Ildus Kurbangaliev",
@ -437,23 +460,29 @@ const char * auto_contributors[] {
"Ivan Starkov",
"ivanzhukov",
"Ivan Zhukov",
"ivoleg",
"Jack Song",
"JackyWoo",
"Jacob Hayes",
"jakalletti",
"Jakub Kuklis",
"JaosnHsieh",
"jasine",
"Jason",
"Jason Keirstead",
"javartisan",
"javi",
"javi santana",
"Javi Santana",
"Javi santana bot",
"Jean Baptiste Favre",
"Jeffrey Dang",
"jennyma",
"jetgm",
"Jiading Guo",
"Jiang Tao",
"jianmei zhang",
"jkuklis",
"Jochen Schalanda",
"John",
"John Hummel",
@ -467,15 +496,18 @@ const char * auto_contributors[] {
"Keiji Yoshida",
"Ken Chen",
"Kevin Chiang",
"Kevin Michel",
"kevin wan",
"Kiran",
"Kirill Danshin",
"Kirill Ershov",
"kirillikoff",
"Kirill Malev",
"Kirill Shvakov",
"kmeaw",
"Koblikov Mihail",
"KochetovNicolai",
"kolsys",
"Konstantin Grabar",
"Konstantin Lebedev",
"Konstantin Malanchev",
@ -492,6 +524,7 @@ const char * auto_contributors[] {
"Kseniia Sumarokova",
"kshvakov",
"kssenii",
"Ky Li",
"l",
"l1tsolaiki",
"lalex",
@ -502,6 +535,7 @@ const char * auto_contributors[] {
"Leopold Schabel",
"leozhang",
"Lev Borodin",
"levie",
"levushkin aleksej",
"levysh",
"Lewinma",
@ -509,6 +543,7 @@ const char * auto_contributors[] {
"libenwang",
"lichengxiang",
"linceyou",
"listar",
"litao91",
"liu-bov",
"Liu Cong",
@ -521,6 +556,7 @@ const char * auto_contributors[] {
"long2ice",
"Lopatin Konstantin",
"Loud_Scream",
"lthaooo",
"ltybc-coder",
"luc1ph3r",
"Lucid Dreams",
@ -529,12 +565,14 @@ const char * auto_contributors[] {
"Lv Feng",
"M0r64n",
"madianjun",
"MagiaGroz",
"maiha",
"Maksim",
"Maksim Fedotov",
"Maksim Kita",
"Maks Skorokhod",
"malkfilipp",
"Malte",
"manmitya",
"maqroll",
"Marat IDRISOV",
@ -595,6 +633,7 @@ const char * auto_contributors[] {
"Michal Lisowski",
"michon470",
"MicrochipQ",
"Miguel Fernández",
"miha-g",
"Mihail Fandyushin",
"Mikahil Nacharov",
@ -617,10 +656,12 @@ const char * auto_contributors[] {
"MikuSugar",
"Milad Arabi",
"millb",
"Misko Lee",
"mnkonkova",
"Mohammad Hossein Sekhavat",
"morty",
"moscas",
"Mostafa Dahab",
"MovElb",
"Mr.General",
"Murat Kabilov",
@ -634,11 +675,14 @@ const char * auto_contributors[] {
"nauta",
"nautaa",
"Neeke Gao",
"neng.liu",
"Neng Liu",
"NengLiu",
"never lee",
"NeZeD [Mac Pro]",
"nicelulu",
"Nickita",
"Nickita Taranov",
"Nickolay Yastrebov",
"nickzhwang",
"Nicolae Vartolomei",
@ -684,6 +728,9 @@ const char * auto_contributors[] {
"Olga Khvostikova",
"olgarev",
"Olga Revyakina",
"OmarBazaraa",
"Onehr7",
"OnePiece",
"orantius",
"Orivej Desh",
"Oskar Wojciski",
@ -701,6 +748,7 @@ const char * auto_contributors[] {
"Pavel Yakunin",
"Pavlo Bashynskiy",
"Pawel Rog",
"pdv-ru",
"Peng Jian",
"Persiyanov Dmitriy Andreevich",
"Pervakov Grigorii",
@ -728,6 +776,7 @@ const char * auto_contributors[] {
"Ramazan Polat",
"Raúl Marín",
"Ravengg",
"Realist007",
"redclusive",
"RedClusive",
"RegulusZ",
@ -749,12 +798,15 @@ const char * auto_contributors[] {
"Roman Peshkurov",
"Roman Tsisyk",
"romanzhukov",
"Roman Zhukov",
"root",
"roverxu",
"ruct",
"Ruslan",
"Ruslan Savchenko",
"Russ Frank",
"Ruzal Ibragimov",
"ryzuo",
"Sabyanin Maxim",
"SaltTan",
"Sami Kerola",
@ -763,6 +815,7 @@ const char * auto_contributors[] {
"satanson",
"Saulius Valatka",
"sdk2",
"Sean Haynes",
"Sébastien Launay",
"serebrserg",
"Sergei Bocharov",
@ -839,12 +892,15 @@ const char * auto_contributors[] {
"TCeason",
"Tema Novikov",
"templarzq",
"terrylin",
"The-Alchemist",
"Thomas Berdy",
"Tiaonmmn",
"tiger.yan",
"tison",
"TiunovNN",
"Tobias Adamson",
"Tobias Lins",
"Tom Bombadil",
"topvisor",
"Tsarkova Anastasia",
@ -863,6 +919,7 @@ const char * auto_contributors[] {
"VadimPE",
"Vadim Plakhtinskiy",
"Vadim Skipin",
"Vadim Volodin",
"Val",
"Valera Ryaboshapko",
"Vasilyev Nikita",
@ -877,12 +934,15 @@ const char * auto_contributors[] {
"Veloman Yunkan",
"Veniamin Gvozdikov",
"Veselkov Konstantin",
"vgocoder",
"vic",
"vicdashkov",
"vicgao",
"Victor",
"Victor Tarnavsky",
"Viktor Taranenko",
"vinity",
"Vitalii S",
"Vitaliy Fedorchenko",
"Vitaliy Karnienko",
"Vitaliy Kozlovskiy",
@ -891,12 +951,14 @@ const char * auto_contributors[] {
"Vitaly",
"Vitaly Baranov",
"Vitaly Samigullin",
"Vitaly Stoyan",
"vitstn",
"vivarum",
"Vivien Maisonneuve",
"Vlad Arkhipov",
"Vladimir",
"Vladimir Bunchuk",
"Vladimir C",
"Vladimir Ch",
"Vladimir Chebotarev",
"vladimir golovchenko",
@ -925,6 +987,7 @@ const char * auto_contributors[] {
"wzl",
"Xianda Ke",
"Xiang Zhou",
"xiedeyantu",
"xPoSx",
"Yağızcan Değirmenci",
"yang",
@ -938,6 +1001,7 @@ const char * auto_contributors[] {
"Yiğit Konur",
"yiguolei",
"Yingchun Lai",
"Yingfan Chen",
"yingjinghan",
"ylchou",
"Y Lu",
@ -966,6 +1030,8 @@ const char * auto_contributors[] {
"Zhichang Yu",
"Zhichun Wu",
"Zhipeng",
"ZhiYong Wang",
"zhongyuankai",
"zhukai",
"Zijie Lu",
"zlx19950903",
@ -994,6 +1060,7 @@ const char * auto_contributors[] {
"曲正鹏",
"未来星___费",
"极客青年",
"董海镔",
"谢磊",
"贾顺名(Jarvis)",
"陈小玉",

View File

@ -11,7 +11,7 @@ namespace DB
class Context;
/// system.replicated_fetches table. Takes data from context.getReplicatedFetchList()
class StorageSystemReplicatedFetches final : public shared_ptr_helper<StorageSystemReplicatedFetches>, public IStorageSystemOneBlock<StorageSystemReplicatedFetches >
class StorageSystemReplicatedFetches final : public shared_ptr_helper<StorageSystemReplicatedFetches>, public IStorageSystemOneBlock<StorageSystemReplicatedFetches>
{
friend struct shared_ptr_helper<StorageSystemReplicatedFetches>;
public:

View File

@ -47,6 +47,7 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_)
{"dependencies_table", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{"create_table_query", std::make_shared<DataTypeString>()},
{"engine_full", std::make_shared<DataTypeString>()},
{"as_select", std::make_shared<DataTypeString>()},
{"partition_key", std::make_shared<DataTypeString>()},
{"sorting_key", std::make_shared<DataTypeString>()},
{"primary_key", std::make_shared<DataTypeString>()},
@ -57,6 +58,7 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_)
{"lifetime_rows", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
{"lifetime_bytes", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
{"comment", std::make_shared<DataTypeString>()},
{"has_own_data", std::make_shared<DataTypeUInt8>()},
}));
setInMemoryMetadata(storage_metadata);
}
@ -209,6 +211,10 @@ protected:
if (columns_mask[src_index++])
res_columns[res_index++]->insert(table.second->getName());
// as_select
if (columns_mask[src_index++])
res_columns[res_index++]->insertDefault();
// partition_key
if (columns_mask[src_index++])
res_columns[res_index++]->insertDefault();
@ -248,6 +254,10 @@ protected:
// comment
if (columns_mask[src_index++])
res_columns[res_index++]->insertDefault();
// has_own_data
if (columns_mask[src_index++])
res_columns[res_index++]->insertDefault();
}
}
@ -355,15 +365,15 @@ protected:
res_columns[res_index++]->insert(dependencies_table_name_array);
}
if (columns_mask[src_index] || columns_mask[src_index + 1])
if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2])
{
ASTPtr ast = database->tryGetCreateTableQuery(table_name, context);
auto * ast_create = ast ? ast->as<ASTCreateQuery>() : nullptr;
if (ast && !context->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil)
if (ast_create && !context->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil)
{
auto & create = ast->as<ASTCreateQuery &>();
create.uuid = UUIDHelpers::Nil;
create.to_inner_uuid = UUIDHelpers::Nil;
ast_create->uuid = UUIDHelpers::Nil;
ast_create->to_inner_uuid = UUIDHelpers::Nil;
}
if (columns_mask[src_index++])
@ -373,24 +383,28 @@ protected:
{
String engine_full;
if (ast)
if (ast_create && ast_create->storage)
{
const auto & ast_create = ast->as<ASTCreateQuery &>();
if (ast_create.storage)
{
engine_full = queryToString(*ast_create.storage);
engine_full = queryToString(*ast_create->storage);
static const char * const extra_head = " ENGINE = ";
if (startsWith(engine_full, extra_head))
engine_full = engine_full.substr(strlen(extra_head));
}
static const char * const extra_head = " ENGINE = ";
if (startsWith(engine_full, extra_head))
engine_full = engine_full.substr(strlen(extra_head));
}
res_columns[res_index++]->insert(engine_full);
}
if (columns_mask[src_index++])
{
String as_select;
if (ast_create && ast_create->select)
as_select = queryToString(*ast_create->select);
res_columns[res_index++]->insert(as_select);
}
}
else
src_index += 2;
src_index += 3;
StorageMetadataPtr metadata_snapshot;
if (table)
@ -483,6 +497,14 @@ protected:
else
res_columns[res_index++]->insertDefault();
}
if (columns_mask[src_index++])
{
if (table)
res_columns[res_index++]->insert(table->storesDataOnDisk());
else
res_columns[res_index++]->insertDefault();
}
}
}

View File

@ -0,0 +1,57 @@
#include <Databases/DatabaseOnDisk.h>
#include <Storages/System/attachInformationSchemaTables.h>
#include <Storages/System/attachSystemTablesImpl.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <common/getResource.h>
namespace DB
{
/// View structures are taken from http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt
static void createInformationSchemaView(ContextMutablePtr context, IDatabase & database, const String & view_name)
{
try
{
assert(database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA ||
database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE);
if (database.getEngineName() != "Memory")
return;
bool is_uppercase = database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE;
String metadata_resource_name = view_name + ".sql";
auto attach_query = getResource(metadata_resource_name);
if (attach_query.empty())
return;
ParserCreateQuery parser;
ASTPtr ast = parseQuery(parser, attach_query.data(), attach_query.data() + attach_query.size(),
"Attach query from embedded resource " + metadata_resource_name,
DBMS_DEFAULT_MAX_QUERY_SIZE, DBMS_DEFAULT_MAX_PARSER_DEPTH);
auto & ast_create = ast->as<ASTCreateQuery &>();
assert(view_name == ast_create.table);
if (is_uppercase)
ast_create.table = Poco::toUpper(view_name);
StoragePtr view = createTableFromAST(ast_create, database.getDatabaseName(),
database.getTableDataPath(ast_create), context, true).second;
database.createTable(context, ast_create.table, view, ast);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void attachInformationSchema(ContextMutablePtr context, IDatabase & information_schema_database)
{
createInformationSchemaView(context, information_schema_database, "schemata");
createInformationSchemaView(context, information_schema_database, "tables");
createInformationSchemaView(context, information_schema_database, "views");
createInformationSchemaView(context, information_schema_database, "columns");
}
}

View File

@ -0,0 +1,11 @@
#pragma once
#include <Interpreters/Context_fwd.h>
namespace DB
{
class IDatabase;
void attachInformationSchema(ContextMutablePtr context, IDatabase & information_schema_database);
}

View File

@ -9,6 +9,7 @@ namespace DB
template<typename StorageT, typename... StorageArgs>
void attach(IDatabase & system_database, const String & table_name, StorageArgs && ... args)
{
assert(system_database.getDatabaseName() == DatabaseCatalog::SYSTEM_DATABASE);
if (system_database.getUUID() == UUIDHelpers::Nil)
{
/// Attach to Ordinary database

View File

@ -104,7 +104,7 @@ static void check(
SelectQueryInfo query_info;
SelectQueryOptions select_options;
query_info.syntax_analyzer_result
= TreeRewriter(state.context).analyzeSelect(ast, state.getColumns(), select_options, state.getTables(table_num));
= TreeRewriter(state.context).analyzeSelect(ast, DB::TreeRewriterResult(state.getColumns()), select_options, state.getTables(table_num));
query_info.query = ast;
std::string transformed_query = transformQueryForExternalDatabase(
query_info, state.getColumns(), IdentifierQuotingStyle::DoubleQuotes, "test", "table", state.context);

View File

@ -28,7 +28,7 @@ SRCS(
MemorySettings.cpp
MergeTree/ActiveDataPartSet.cpp
MergeTree/AllMergeSelector.cpp
MergeTree/BackgroundJobsExecutor.cpp
MergeTree/BackgroundJobsAssignee.cpp
MergeTree/BoolMask.cpp
MergeTree/DataPartsExchange.cpp
MergeTree/DropPartsRanges.cpp
@ -41,6 +41,7 @@ SRCS(
MergeTree/LevelMergeSelector.cpp
MergeTree/MergeAlgorithm.cpp
MergeTree/MergeList.cpp
MergeTree/MergeTreeBackgroundExecutor.cpp
MergeTree/MergeTreeBaseSelectProcessor.cpp
MergeTree/MergeTreeBlockReadUtils.cpp
MergeTree/MergeTreeData.cpp
@ -217,6 +218,7 @@ SRCS(
System/StorageSystemWarnings.cpp
System/StorageSystemZeros.cpp
System/StorageSystemZooKeeper.cpp
System/attachInformationSchemaTables.cpp
System/attachSystemTables.cpp
TTLDescription.cpp
VirtualColumnUtils.cpp

View File

@ -1,7 +1,17 @@
<yandex>
<metric_log remove="remove"/>
<query_masking_rules remove="remove"/>
<query_thread_log remove="remove"/>
<query_log remove="remove" />
<query_views_log remove="remove" />
<metric_log remove="remove"/>
<text_log remove="remove"/>
<trace_log remove="remove"/>
<asynchronous_metric_log remove="remove" />
<session_log remove="remove" />
<part_log remove="remove" />
<crash_log remove="remove" />
<opentelemetry_span_log remove="remove" />
<!-- just in case it will be enabled by default -->
<zookeeper_log remove="remove" />
</yandex>

View File

@ -0,0 +1,8 @@
<yandex>
<profiles>
<default>
<query_profiler_real_time_period_ns>0</query_profiler_real_time_period_ns>
<query_profiler_cpu_time_period_ns>0</query_profiler_cpu_time_period_ns>
</default>
</profiles>
</yandex>

View File

@ -11,6 +11,8 @@ cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', main_configs=[
'configs/no_system_log.xml',
'configs/asynchronous_metrics_update_period_s.xml',
], user_configs=[
'configs/users.d/overrides.xml',
])
@pytest.fixture(scope='module', autouse=True)
@ -23,8 +25,6 @@ def start_cluster():
query_settings = {
'max_threads': 1,
'query_profiler_real_time_period_ns': 0,
'query_profiler_cpu_time_period_ns': 0,
'log_queries': 0,
}
sample_query = "SELECT groupArray(repeat('a', 1000)) FROM numbers(10000) GROUP BY number%10 FORMAT JSON"

View File

@ -13,6 +13,7 @@ name CHAR
a TINYINT
Result:
tables 1
tables 1
Columns:
a
b

View File

@ -407,24 +407,24 @@ def test_php_client(started_cluster, php_container):
code, (stdout, stderr) = php_container.exec_run(
'php -f test.php {host} {port} default 123'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True)
assert code == 0
assert stdout.decode() == 'tables\n'
assert stdout.decode() == 'tables\ntables\n'
code, (stdout, stderr) = php_container.exec_run(
'php -f test_ssl.php {host} {port} default 123'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True)
assert code == 0
assert stdout.decode() == 'tables\n'
assert stdout.decode() == 'tables\ntables\n'
code, (stdout, stderr) = php_container.exec_run(
'php -f test.php {host} {port} user_with_double_sha1 abacaba'.format(host=started_cluster.get_instance_ip('node'), port=server_port),
demux=True)
assert code == 0
assert stdout.decode() == 'tables\n'
assert stdout.decode() == 'tables\ntables\n'
code, (stdout, stderr) = php_container.exec_run(
'php -f test_ssl.php {host} {port} user_with_double_sha1 abacaba'.format(host=started_cluster.get_instance_ip('node'), port=server_port),
demux=True)
assert code == 0
assert stdout.decode() == 'tables\n'
assert stdout.decode() == 'tables\ntables\n'
def test_mysqljs_client(started_cluster, nodejs_container):

View File

@ -399,9 +399,9 @@ def test_consumption_of_show_tables():
instance.query("SHOW QUOTA"))
def test_consumption_of_show_databases():
assert instance.query("SHOW DATABASES") == "default\nsystem\n"
assert instance.query("SHOW DATABASES") == "INFORMATION_SCHEMA\ndefault\ninformation_schema\nsystem\n"
assert re.match(
"myQuota\\tdefault\\t.*\\t31556952\\t1\\t1000\\t1\\t500\\t0\\t500\\t0\\t\\\\N\\t2\\t\\\\N.*",
"myQuota\\tdefault\\t.*\\t31556952\\t1\\t1000\\t1\\t500\\t0\\t500\\t0\\t\\\\N\\t4\\t\\\\N.*",
instance.query("SHOW QUOTA"))
def test_consumption_of_show_clusters():

View File

@ -25,24 +25,17 @@ insert into joinbug_join (id, id2, val, val2, created)
select id, id2, val, val2, created
from joinbug;
/* expected */
select *
from joinbug;
select * from joinbug;
/* wtf */
select id, id2, val, val2, created
from (
SELECT toUInt64(arrayJoin(range(50))) AS id2
) js1
from ( SELECT toUInt64(arrayJoin(range(50))) AS id2 ) js1
SEMI LEFT JOIN joinbug_join using id2;
/* type conversion */
SELECT * FROM
(
SELECT toUInt32(11) AS id2
) AS js1
SEMI LEFT JOIN joinbug_join USING (id2);
-- type conversion
SELECT * FROM ( SELECT toUInt32(11) AS id2 ) AS js1 SEMI LEFT JOIN joinbug_join USING (id2);
-- can't convert right side in case on storage join
SELECT * FROM ( SELECT toInt64(11) AS id2 ) AS js1 SEMI LEFT JOIN joinbug_join USING (id2); -- { serverError 53 }
DROP TABLE joinbug;
DROP TABLE joinbug_join;

View File

@ -9,3 +9,12 @@
2 2 a2
1 0
2 2 a2
0 0
0 0
0 0
0 0
0
0
0
0
0 Int64 Int64

View File

@ -30,6 +30,22 @@ SELECT did, id, name FROM f LEFT ARRAY JOIN d_ids as did LEFT JOIN d ON d.id = d
-- name clash, doesn't work yet
SELECT id, name FROM f LEFT ARRAY JOIN d_ids as id LEFT JOIN d ON d.id = id ORDER BY id; -- { serverError 403 }
SELECT * FROM ( SELECT [dummy, dummy] AS dummy FROM system.one ) AS x ARRAY JOIN dummy
JOIN system.one AS y ON x.dummy == y.dummy;
SELECT * FROM ( SELECT [dummy, dummy] AS dummy FROM system.one ) AS x ARRAY JOIN dummy
JOIN system.one AS y ON x.dummy + 1 == y.dummy + 1;
SELECT * FROM ( SELECT [dummy, dummy] AS dummy FROM system.one ) AS x ARRAY JOIN dummy
JOIN system.one AS y USING dummy;
SELECT * FROM ( SELECT [toUInt32(dummy), toUInt32(dummy)] AS dummy FROM system.one ) AS x ARRAY JOIN dummy
JOIN (select toInt32(dummy) as dummy from system.one ) AS y USING dummy;
SELECT dummy > 0, toTypeName(any(dummy)), any(toTypeName(dummy))
FROM ( SELECT [toUInt32(dummy), toUInt32(dummy)] AS dummy FROM system.one ) AS x ARRAY JOIN dummy
JOIN ( SELECT toInt32(dummy) AS dummy FROM system.one ) AS y USING dummy GROUP BY (dummy > 0);
DROP TABLE IF EXISTS f;
DROP TABLE IF EXISTS d;

View File

@ -0,0 +1,27 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
THREADS=8
RAND=$(($RANDOM))
LIMIT=10000
function run_selects()
{
thread_num=$1
readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT database || '.' || name FROM system.tables
WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper'
AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num")
for t in "${tables_arr[@]}"
do
${CLICKHOUSE_CLIENT} -q "SELECT * FROM $t LIMIT $LIMIT FORMAT Null" # Suppress style check: database=$CLICKHOUSE_DATABASEs
done
}
for ((i=0; i<THREADS; i++)) do
run_selects "$i" &
done
wait

View File

@ -0,0 +1,27 @@
columns
schemata
tables
views
COLUMNS
SCHEMATA
TABLES
VIEWS
INFORMATION_SCHEMA INFORMATION_SCHEMA default \N \N \N \N
information_schema information_schema default \N \N \N \N
default default mv VIEW
default default t FOREIGN TABLE
default default v VIEW
tmp LOCAL TEMPORARY
default default mv SELECT * FROM system.one NONE NO YES NO NO NO
default default v SELECT n, f FROM default.t NONE NO NO NO NO NO
default default mv dummy 1 0 UInt8 \N \N 8 2 0 \N \N \N \N \N \N \N \N \N \N
default default t n 1 0 UInt64 \N \N 64 2 0 \N \N \N \N \N \N \N \N \N \N
default default t f 2 0 Float32 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N
default default t s 3 0 String \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N
default default t fs 4 0 FixedString(42) 42 42 \N \N \N \N \N \N \N \N \N \N \N \N \N
default default t d 5 0 Decimal(9, 6) \N \N 9 10 6 \N \N \N \N \N \N \N \N \N \N
default default v n 1 1 Nullable(Int32) \N \N 32 2 0 \N \N \N \N \N \N \N \N \N \N
default default v f 2 0 Float64 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N
tmp d 1 0 Date \N \N \N \N \N 0 \N \N \N \N \N \N \N \N \N
tmp dt 2 0 DateTime \N \N \N \N \N 0 \N \N \N \N \N \N \N \N \N
tmp dtms 3 0 DateTime64(3) \N \N \N \N \N 3 \N \N \N \N \N \N \N \N \N

View File

@ -0,0 +1,19 @@
show tables from information_schema;
SHOW TABLES FROM INFORMATION_SCHEMA;
create table t (n UInt64, f Float32, s String, fs FixedString(42), d Decimal(9, 6)) engine=Memory;
create view v (n Nullable(Int32), f Float64) as select n, f from t;
create materialized view mv engine=Null as select * from system.one;
create temporary table tmp (d Date, dt DateTime, dtms DateTime64(3));
-- FIXME #28687
select * from information_schema.schemata where schema_name ilike 'information_schema';
-- SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE (TABLE_SCHEMA=currentDatabase() OR TABLE_SCHEMA='') AND TABLE_NAME NOT LIKE '%inner%';
SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE (table_schema=currentDatabase() OR table_schema='') AND table_name NOT LIKE '%inner%';
select * from information_schema.views where table_schema=currentDatabase();
-- SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE (TABLE_SCHEMA=currentDatabase() OR TABLE_SCHEMA='') AND TABLE_NAME NOT LIKE '%inner%';
SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE (table_schema=currentDatabase() OR table_schema='') AND table_name NOT LIKE '%inner%';
drop table t;
drop view v;
drop view mv;

View File

@ -1,4 +1,4 @@
1
1
1
1
[[1]]
[[1]]
[[1]]
[[1]]

View File

@ -6,7 +6,8 @@ CREATE TABLE table_with_single_pk
value String
)
ENGINE = MergeTree
ORDER BY key;
ORDER BY key
SETTINGS min_compress_block_size=65536, max_compress_block_size=65536;
INSERT INTO table_with_single_pk SELECT number, toString(number % 10) FROM numbers(10000000);
@ -15,9 +16,9 @@ ALTER TABLE table_with_single_pk DELETE WHERE key % 77 = 0 SETTINGS mutations_sy
SYSTEM FLUSH LOGS;
-- Memory usage for all mutations must be almost constant and less than
-- read_bytes.
-- read_bytes
SELECT
DISTINCT read_bytes >= peak_memory_usage
arrayDistinct(groupArray(if (read_bytes >= peak_memory_usage, [1], [read_bytes, peak_memory_usage])))
FROM
system.part_log
WHERE event_type = 'MutatePart' AND table = 'table_with_single_pk' AND database = currentDatabase();
@ -34,7 +35,8 @@ CREATE TABLE table_with_multi_pk
value String
)
ENGINE = MergeTree
ORDER BY (key1, key2, key3);
ORDER BY (key1, key2, key3)
SETTINGS min_compress_block_size=65536, max_compress_block_size=65536;
INSERT INTO table_with_multi_pk SELECT number % 32, number, toDateTime('2019-10-01 00:00:00'), toString(number % 10) FROM numbers(10000000);
@ -43,9 +45,9 @@ ALTER TABLE table_with_multi_pk DELETE WHERE key1 % 77 = 0 SETTINGS mutations_sy
SYSTEM FLUSH LOGS;
-- Memory usage for all mutations must be almost constant and less than
-- read_bytes.
-- read_bytes
SELECT
DISTINCT read_bytes >= peak_memory_usage
arrayDistinct(groupArray(if (read_bytes >= peak_memory_usage, [1], [read_bytes, peak_memory_usage])))
FROM
system.part_log
WHERE event_type = 'MutatePart' AND table = 'table_with_multi_pk' AND database = currentDatabase();
@ -64,7 +66,8 @@ CREATE TABLE table_with_function_pk
value String
)
ENGINE = MergeTree
ORDER BY (cast(value as UInt64), key2);
ORDER BY (cast(value as UInt64), key2)
SETTINGS min_compress_block_size=65536, max_compress_block_size=65536;
INSERT INTO table_with_function_pk SELECT number % 32, number, toDateTime('2019-10-01 00:00:00'), toString(number % 10) FROM numbers(10000000);
@ -73,9 +76,9 @@ ALTER TABLE table_with_function_pk DELETE WHERE key1 % 77 = 0 SETTINGS mutations
SYSTEM FLUSH LOGS;
-- Memory usage for all mutations must be almost constant and less than
-- read_bytes.
-- read_bytes
SELECT
DISTINCT read_bytes >= peak_memory_usage
arrayDistinct(groupArray(if (read_bytes >= peak_memory_usage, [1], [read_bytes, peak_memory_usage])))
FROM
system.part_log
WHERE event_type = 'MutatePart' AND table = 'table_with_function_pk' AND database = currentDatabase();
@ -92,7 +95,8 @@ CREATE TABLE table_without_pk
value String
)
ENGINE = MergeTree
ORDER BY tuple();
ORDER BY tuple()
SETTINGS min_compress_block_size=65536, max_compress_block_size=65536;
INSERT INTO table_without_pk SELECT number % 32, number, toDateTime('2019-10-01 00:00:00'), toString(number % 10) FROM numbers(10000000);
@ -101,9 +105,9 @@ ALTER TABLE table_without_pk DELETE WHERE key1 % 77 = 0 SETTINGS mutations_sync
SYSTEM FLUSH LOGS;
-- Memory usage for all mutations must be almost constant and less than
-- read_bytes.
-- read_bytes
SELECT
DISTINCT read_bytes >= peak_memory_usage
arrayDistinct(groupArray(if (read_bytes >= peak_memory_usage, [1], [read_bytes, peak_memory_usage])))
FROM
system.part_log
WHERE event_type = 'MutatePart' AND table = 'table_without_pk' AND database = currentDatabase();

View File

@ -1,8 +1,8 @@
test_01602a CREATE TEMPORARY TABLE test_01602a (`x` UInt32) ENGINE = Memory Memory Memory 1
test_01602b CREATE TEMPORARY TABLE test_01602b (`y` Float64, `z` String) ENGINE = Memory Memory Memory 1
test_01602a x UInt32 1 0 0 0 0 0 0 0
test_01602b y Float64 1 0 0 0 0 0 0 0
test_01602b z String 2 0 0 0 0 0 0 0
test_01602a x UInt32 1 0 0 0 0 0 0 0 \N 32 2 0 \N
test_01602b y Float64 1 0 0 0 0 0 0 0 \N \N \N \N \N
test_01602b z String 2 0 0 0 0 0 0 0 \N \N \N \N \N
CREATE TEMPORARY TABLE test_01602a\n(\n `x` UInt32\n)\nENGINE = Memory
CREATE TEMPORARY TABLE test_01602b\n(\n `y` Float64,\n `z` String\n)\nENGINE = Memory
0

Some files were not shown because too many files have changed in this diff Show More