Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into interactive-mode-for-clickhouse-local

This commit is contained in:
kssenii 2021-09-09 12:43:12 +00:00
commit 1838a8672d
232 changed files with 5051 additions and 1759 deletions

View File

@ -203,3 +203,5 @@ CheckOptions:
value: CamelCase
- key: readability-identifier-naming.UsingCase
value: CamelCase
- key: modernize-loop-convert.UseCxx20ReverseRanges
value: false

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

@ -42,6 +42,7 @@ namespace
} while (false)
#define LOG_TEST(logger, ...) LOG_IMPL(logger, DB::LogsLevel::test, Poco::Message::PRIO_TEST, __VA_ARGS__)
#define LOG_TRACE(logger, ...) LOG_IMPL(logger, DB::LogsLevel::trace, Poco::Message::PRIO_TRACE, __VA_ARGS__)
#define LOG_DEBUG(logger, ...) LOG_IMPL(logger, DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG, __VA_ARGS__)
#define LOG_INFO(logger, ...) LOG_IMPL(logger, DB::LogsLevel::information, Poco::Message::PRIO_INFORMATION, __VA_ARGS__)

View File

@ -7,10 +7,22 @@
#endif
#include <mysqlxx/Pool.h>
#include <common/sleep.h>
#include <Poco/Util/LayeredConfiguration.h>
#include <ctime>
namespace
{
inline uint64_t clock_gettime_ns(clockid_t clock_type = CLOCK_MONOTONIC)
{
struct timespec ts;
clock_gettime(clock_type, &ts);
return uint64_t(ts.tv_sec * 1000000000LL + ts.tv_nsec);
}
}
namespace mysqlxx
@ -124,10 +136,15 @@ Pool::~Pool()
}
Pool::Entry Pool::get()
Pool::Entry Pool::get(uint64_t wait_timeout)
{
std::unique_lock<std::mutex> lock(mutex);
uint64_t deadline = 0;
/// UINT64_MAX -- wait indefinitely
if (wait_timeout && wait_timeout != UINT64_MAX)
deadline = clock_gettime_ns() + wait_timeout * 1'000'000'000;
initialize();
for (;;)
{
@ -153,6 +170,12 @@ Pool::Entry Pool::get()
logger.trace("(%s): Unable to create a new connection: Max number of connections has been reached.", getDescription());
}
if (!wait_timeout)
throw Poco::Exception("mysqlxx::Pool is full (wait is disabled, see connection_wait_timeout setting)");
if (deadline && clock_gettime_ns() >= deadline)
throw Poco::Exception("mysqlxx::Pool is full (connection_wait_timeout is exceeded)");
lock.unlock();
logger.trace("(%s): Sleeping for %d seconds.", getDescription(), MYSQLXX_POOL_SLEEP_ON_CONNECT_FAIL);
sleepForSeconds(MYSQLXX_POOL_SLEEP_ON_CONNECT_FAIL);

View File

@ -189,7 +189,7 @@ public:
~Pool();
/// Allocates connection.
Entry get();
Entry get(uint64_t wait_timeout);
/// Allocates connection.
/// If database is not accessible, returns empty Entry object.

View File

@ -21,8 +21,9 @@ PoolWithFailover::PoolWithFailover(
const unsigned max_connections_,
const size_t max_tries_)
: max_tries(max_tries_)
, shareable(config_.getBool(config_name_ + ".share_connection", false))
, wait_timeout(UINT64_MAX)
{
shareable = config_.getBool(config_name_ + ".share_connection", false);
if (config_.has(config_name_ + ".replica"))
{
Poco::Util::AbstractConfiguration::Keys replica_keys;
@ -80,9 +81,11 @@ PoolWithFailover::PoolWithFailover(
const std::string & password,
unsigned default_connections_,
unsigned max_connections_,
size_t max_tries_)
size_t max_tries_,
uint64_t wait_timeout_)
: max_tries(max_tries_)
, shareable(false)
, wait_timeout(wait_timeout_)
{
/// Replicas have the same priority, but traversed replicas are moved to the end of the queue.
for (const auto & [host, port] : addresses)
@ -101,6 +104,7 @@ PoolWithFailover::PoolWithFailover(
PoolWithFailover::PoolWithFailover(const PoolWithFailover & other)
: max_tries{other.max_tries}
, shareable{other.shareable}
, wait_timeout(other.wait_timeout)
{
if (shareable)
{
@ -140,7 +144,7 @@ PoolWithFailover::Entry PoolWithFailover::get()
try
{
Entry entry = shareable ? pool->get() : pool->tryGet();
Entry entry = shareable ? pool->get(wait_timeout) : pool->tryGet();
if (!entry.isNull())
{
@ -172,7 +176,7 @@ PoolWithFailover::Entry PoolWithFailover::get()
if (full_pool)
{
app.logger().error("All connections failed, trying to wait on a full pool " + (*full_pool)->getDescription());
return (*full_pool)->get();
return (*full_pool)->get(wait_timeout);
}
std::stringstream message;

View File

@ -80,6 +80,8 @@ namespace mysqlxx
std::mutex mutex;
/// Can the Pool be shared
bool shareable;
/// Timeout for waiting free connection.
uint64_t wait_timeout = 0;
public:
using Entry = Pool::Entry;
@ -96,6 +98,7 @@ namespace mysqlxx
* default_connections Number of connection in pool to each replica at start.
* max_connections Maximum number of connections in pool to each replica.
* max_tries_ Max number of connection tries.
* wait_timeout_ Timeout for waiting free connection.
*/
PoolWithFailover(
const std::string & config_name_,
@ -117,7 +120,8 @@ namespace mysqlxx
const std::string & password,
unsigned default_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
unsigned max_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS,
size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES,
uint64_t wait_timeout_ = UINT64_MAX);
PoolWithFailover(const PoolWithFailover & other);

View File

@ -6,7 +6,7 @@ if (ENABLE_CLANG_TIDY)
message(FATAL_ERROR "clang-tidy requires CMake version at least 3.6.")
endif()
find_program (CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-11" "clang-tidy-10" "clang-tidy-9" "clang-tidy-8")
find_program (CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-12" "clang-tidy-11" "clang-tidy-10" "clang-tidy-9" "clang-tidy-8")
if (CLANG_TIDY_PATH)
message(STATUS

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

View File

@ -10,7 +10,7 @@ set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) # disable linkage check - it
set (CMAKE_AR "/usr/bin/ar" CACHE FILEPATH "" FORCE)
set (CMAKE_RANLIB "/usr/bin/ranlib" CACHE FILEPATH "" FORCE)
set (LINKER_NAME "lld" CACHE STRING "" FORCE)
set (LINKER_NAME "ld.lld" CACHE STRING "" FORCE)
set (CMAKE_EXE_LINKER_FLAGS_INIT "-fuse-ld=lld")
set (CMAKE_SHARED_LINKER_FLAGS_INIT "-fuse-ld=lld")

View File

@ -13,7 +13,7 @@ set (CMAKE_C_FLAGS_INIT "${CMAKE_C_FLAGS} --gcc-toolchain=${CMAKE_CURRENT_LIST_D
set (CMAKE_CXX_FLAGS_INIT "${CMAKE_CXX_FLAGS} --gcc-toolchain=${CMAKE_CURRENT_LIST_DIR}/../toolchain/linux-aarch64")
set (CMAKE_ASM_FLAGS_INIT "${CMAKE_ASM_FLAGS} --gcc-toolchain=${CMAKE_CURRENT_LIST_DIR}/../toolchain/linux-aarch64")
set (LINKER_NAME "lld" CACHE STRING "" FORCE)
set (LINKER_NAME "ld.lld" CACHE STRING "" FORCE)
set (CMAKE_EXE_LINKER_FLAGS_INIT "-fuse-ld=lld")
set (CMAKE_SHARED_LINKER_FLAGS_INIT "-fuse-ld=lld")

View File

@ -79,8 +79,9 @@ endif ()
if (LINKER_NAME)
if (COMPILER_CLANG AND (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER 12.0.0 OR CMAKE_CXX_COMPILER_VERSION VERSION_EQUAL 12.0.0))
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} --ld-path=${LINKER_NAME}")
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} --ld-path=${LINKER_NAME}")
find_program (LLD_PATH NAMES ${LINKER_NAME})
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} --ld-path=${LLD_PATH}")
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} --ld-path=${LLD_PATH}")
else ()
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=${LINKER_NAME}")
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=${LINKER_NAME}")

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 ()

2
contrib/poco vendored

@ -1 +1 @@
Subproject commit b687c17bc2be36b6333a1d7cfffbf9eab65509a9
Subproject commit 46c80daf1b015aa10474ce82e3d24b578c6ae422

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,6 +1,6 @@
FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=12
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list

View File

@ -4,7 +4,7 @@ set -e
#ccache -s # uncomment to display CCache statistics
mkdir -p /server/build_docker
cd /server/build_docker
cmake -G Ninja /server "-DCMAKE_C_COMPILER=$(command -v clang-11)" "-DCMAKE_CXX_COMPILER=$(command -v clang++-11)"
cmake -G Ninja /server "-DCMAKE_C_COMPILER=$(command -v clang-12)" "-DCMAKE_CXX_COMPILER=$(command -v clang++-12)"
# Set the number of build jobs to the half of number of virtual CPU cores (rounded up).
# By default, ninja use all virtual CPU cores, that leads to very high memory consumption without much improvement in build time.

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

@ -1,7 +1,7 @@
# docker build -t yandex/clickhouse-binary-builder .
FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=12
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
@ -39,8 +39,6 @@ RUN apt-get update \
bash \
build-essential \
ccache \
clang-11 \
clang-tidy-11 \
cmake \
curl \
g++-10 \
@ -50,9 +48,13 @@ RUN apt-get update \
gperf \
libicu-dev \
libreadline-dev \
lld-11 \
llvm-11 \
llvm-11-dev \
clang-12 \
clang-tidy-12 \
lld-12 \
llvm-12 \
llvm-12-dev \
libicu-dev \
libreadline-dev \
moreutils \
ninja-build \
pigz \

View File

@ -4,7 +4,6 @@ set -x -e
mkdir -p build/cmake/toolchain/darwin-x86_64
tar xJf MacOSX11.0.sdk.tar.xz -C build/cmake/toolchain/darwin-x86_64 --strip-components=1
ln -sf darwin-x86_64 build/cmake/toolchain/darwin-aarch64
mkdir -p build/cmake/toolchain/linux-aarch64
@ -23,6 +22,7 @@ cd build/build_docker
rm -f CMakeCache.txt
# Read cmake arguments into array (possibly empty)
read -ra CMAKE_FLAGS <<< "${CMAKE_FLAGS:-}"
env
cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" ..
ccache --show-config ||:

View File

@ -1,7 +1,7 @@
# docker build -t yandex/clickhouse-deb-builder .
FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=12
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
@ -37,17 +37,17 @@ RUN curl -O https://clickhouse-datasets.s3.yandex.net/utils/1/dpkg-deb \
RUN apt-get update \
&& apt-get install \
alien \
clang-11 \
clang-tidy-11 \
clang-12 \
clang-tidy-12 \
cmake \
debhelper \
devscripts \
gdb \
git \
gperf \
lld-11 \
llvm-11 \
llvm-11-dev \
lld-12 \
llvm-12 \
llvm-12-dev \
moreutils \
ninja-build \
perl \

View File

@ -75,7 +75,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ
# Explicitly use LLD with Clang by default.
# Don't force linker for cross-compilation.
if is_clang and not is_cross_compile:
cmake_flags.append("-DLINKER_NAME=lld")
cmake_flags.append("-DLINKER_NAME=ld.lld")
if is_cross_darwin:
cc = compiler[:-len(DARWIN_SUFFIX)]
@ -204,7 +204,8 @@ if __name__ == "__main__":
parser.add_argument("--output-dir", required=True)
parser.add_argument("--build-type", choices=("debug", ""), default="")
parser.add_argument("--compiler", choices=("clang-11", "clang-11-darwin", "clang-11-darwin-aarch64", "clang-11-aarch64",
"clang-11-freebsd", "gcc-10"), default="clang-11")
"clang-12", "clang-12-darwin", "clang-12-darwin-aarch64", "clang-12-aarch64",
"clang-11-freebsd", "clang-12-freebsd", "gcc-10"), default="clang-12")
parser.add_argument("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="")
parser.add_argument("--unbundled", action="store_true")
parser.add_argument("--split-binary", action="store_true")

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

@ -1,7 +1,7 @@
# docker build -t yandex/clickhouse-test-base .
FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=12
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list

View File

@ -11,7 +11,7 @@ RUN apt-get update && apt-get --yes --allow-unauthenticated install clang-9 libl
# https://github.com/ClickHouse-Extras/woboq_codebrowser/commit/37e15eaf377b920acb0b48dbe82471be9203f76b
RUN git clone https://github.com/ClickHouse-Extras/woboq_codebrowser
RUN cd woboq_codebrowser && cmake . -DCMAKE_BUILD_TYPE=Release -DCMAKE_CXX_COMPILER=clang\+\+-9 -DCMAKE_C_COMPILER=clang-9 && make -j
RUN cd woboq_codebrowser && cmake . -DCMAKE_BUILD_TYPE=Release -DCMAKE_CXX_COMPILER=clang\+\+-12 -DCMAKE_C_COMPILER=clang-12 && make -j
ENV CODEGEN=/woboq_codebrowser/generator/codebrowser_generator
ENV CODEINDEX=/woboq_codebrowser/indexgenerator/codebrowser_indexgenerator
@ -24,7 +24,7 @@ ENV SHA=nosha
ENV DATA="data"
CMD mkdir -p $BUILD_DIRECTORY && cd $BUILD_DIRECTORY && \
cmake $SOURCE_DIRECTORY -DCMAKE_CXX_COMPILER=/usr/bin/clang\+\+-11 -DCMAKE_C_COMPILER=/usr/bin/clang-11 -DCMAKE_EXPORT_COMPILE_COMMANDS=ON -DENABLE_EMBEDDED_COMPILER=0 -DENABLE_S3=0 && \
cmake $SOURCE_DIRECTORY -DCMAKE_CXX_COMPILER=/usr/bin/clang\+\+-12 -DCMAKE_C_COMPILER=/usr/bin/clang-12 -DCMAKE_EXPORT_COMPILE_COMMANDS=ON -DENABLE_EMBEDDED_COMPILER=0 -DENABLE_S3=0 && \
mkdir -p $HTML_RESULT_DIRECTORY && \
$CODEGEN -b $BUILD_DIRECTORY -a -o $HTML_RESULT_DIRECTORY -p ClickHouse:$SOURCE_DIRECTORY:$SHA -d $DATA | ts '%Y-%m-%d %H:%M:%S' && \
cp -r $STATIC_DATA $HTML_RESULT_DIRECTORY/ &&\

View File

@ -1,7 +1,7 @@
# docker build -t yandex/clickhouse-fasttest .
FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=12
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list

View File

@ -9,7 +9,7 @@ trap 'kill $(jobs -pr) ||:' EXIT
stage=${stage:-}
# Compiler version, normally set by Dockerfile
export LLVM_VERSION=${LLVM_VERSION:-11}
export LLVM_VERSION=${LLVM_VERSION:-12}
# A variable to pass additional flags to CMake.
# Here we explicitly default it to nothing so that bash doesn't complain about
@ -401,6 +401,9 @@ function run_tests
# depends on Go
02013_zlib_read_after_eof
# Accesses CH via mysql table function (which is unavailable)
01747_system_session_log_long
)
time clickhouse-test --hung-check -j 8 --order=random --use-skip-list \

View File

@ -12,7 +12,7 @@ stage=${stage:-}
script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
echo "$script_dir"
repo_dir=ch
BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-11_debug_none_bundled_unsplitted_disable_False_binary"}
BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-12_debug_none_bundled_unsplitted_disable_False_binary"}
function clone
{

View File

@ -2,7 +2,7 @@
set -euo pipefail
CLICKHOUSE_PACKAGE=${CLICKHOUSE_PACKAGE:="https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-11_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse"}
CLICKHOUSE_PACKAGE=${CLICKHOUSE_PACKAGE:="https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-12_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse"}
CLICKHOUSE_REPO_PATH=${CLICKHOUSE_REPO_PATH:=""}

View File

@ -28,7 +28,7 @@ RUN apt-get update --yes \
ENV PKG_VERSION="pvs-studio-latest"
RUN set -x \
&& export PUBKEY_HASHSUM="686e5eb8b3c543a5c54442c39ec876b6c2d912fe8a729099e600017ae53c877dda3368fe38ed7a66024fe26df6b5892a" \
&& export PUBKEY_HASHSUM="ad369a2e9d8b8c30f5a9f2eb131121739b79c78e03fef0f016ea51871a5f78cd4e6257b270dca0ac3be3d1f19d885516" \
&& wget -nv https://files.viva64.com/etc/pubkey.txt -O /tmp/pubkey.txt \
&& echo "${PUBKEY_HASHSUM} /tmp/pubkey.txt" | sha384sum -c \
&& apt-key add /tmp/pubkey.txt \
@ -38,7 +38,7 @@ RUN set -x \
&& dpkg -i "${PKG_VERSION}.deb"
CMD echo "Running PVS version $PKG_VERSION" && cd /repo_folder && pvs-studio-analyzer credentials $LICENCE_NAME $LICENCE_KEY -o ./licence.lic \
&& cmake . -D"ENABLE_EMBEDDED_COMPILER"=OFF -D"USE_INTERNAL_PROTOBUF_LIBRARY"=OFF -D"USE_INTERNAL_GRPC_LIBRARY"=OFF \
&& cmake . -D"ENABLE_EMBEDDED_COMPILER"=OFF -D"USE_INTERNAL_PROTOBUF_LIBRARY"=OFF -D"USE_INTERNAL_GRPC_LIBRARY"=OFF -DCMAKE_C_COMPILER=clang-12 -DCMAKE_CXX_COMPILER=clang\+\+-12 \
&& ninja re2_st clickhouse_grpc_protos \
&& pvs-studio-analyzer analyze -o pvs-studio.log -e contrib -j 4 -l ./licence.lic; \
cp /repo_folder/pvs-studio.log /test_output; \

View File

@ -23,7 +23,7 @@ $ sudo apt-get install git cmake python ninja-build
Or cmake3 instead of cmake on older systems.
### Install clang-11 (recommended) {#install-clang-11}
### Install clang-12 (recommended) {#install-clang-12}
On Ubuntu/Debian you can use the automatic installation script (check [official webpage](https://apt.llvm.org/))
@ -33,11 +33,11 @@ sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)"
For other Linux distribution - check the availability of the [prebuild packages](https://releases.llvm.org/download.html) or build clang [from sources](https://clang.llvm.org/get_started.html).
#### Use clang-11 for Builds
#### Use clang-12 for Builds
``` bash
$ export CC=clang-11
$ export CXX=clang++-11
$ export CC=clang-12
$ export CXX=clang++-12
```
Gcc can also be used though it is discouraged.

View File

@ -31,6 +31,10 @@ ENGINE = MaterializedPostgreSQL('host:port', ['database' | database], 'user', 'p
- [materialized_postgresql_allow_automatic_update](../../operations/settings/settings.md#materialized-postgresql-allow-automatic-update)
- [materialized_postgresql_replication_slot](../../operations/settings/settings.md#materialized-postgresql-replication-slot)
- [materialized_postgresql_snapshot](../../operations/settings/settings.md#materialized-postgresql-snapshot)
``` sql
CREATE DATABASE database1
ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password')
@ -73,7 +77,7 @@ WHERE oid = 'postgres_table'::regclass;
!!! warning "Warning"
Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. The default value for the data type will be used.
## Example of Use {#example-of-use}
``` sql
@ -82,3 +86,11 @@ ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres
SELECT * FROM postgresql_db.postgres_table;
```
## Notes {#notes}
- Failover of the logical replication slot.
Logical Replication Slots which exist on the primary are not available on standby replicas.
So if there is a failover, new primary (the old physical standby) wont be aware of any slots which were existing with old primary. This will lead to a broken replication from PostgreSQL.
A solution to this is to manage replication slots yourself and define a permanent replication slot (some information can be found [here](https://patroni.readthedocs.io/en/latest/SETTINGS.html)). You'll need to pass slot name via `materialized_postgresql_replication_slot` setting, and it has to be exported with `EXPORT SNAPSHOT` option. The snapshot identifier needs to be passed via `materialized_postgresql_snapshot` setting.

View File

@ -19,6 +19,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
SETTINGS
[connection_pool_size=16, ]
[connection_max_tries=3, ]
[connection_wait_timeout=5, ] /* 0 -- do not wait */
[connection_auto_close=true ]
;
```

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
@ -1270,6 +1270,8 @@ You can insert Parquet data from a file into ClickHouse table by the following c
$ cat {filename} | clickhouse-client --query="INSERT INTO {some_table} FORMAT Parquet"
```
To insert data into [Nested](../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs values you must switch on the [input_format_parquet_import_nested](../operations/settings/settings.md#input_format_parquet_import_nested) setting.
You can select data from a ClickHouse table and save them into some file in the Parquet format by the following command:
``` bash
@ -1328,6 +1330,8 @@ You can insert Arrow data from a file into ClickHouse table by the following com
$ cat filename.arrow | clickhouse-client --query="INSERT INTO some_table FORMAT Arrow"
```
To insert data into [Nested](../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs values you must switch on the [input_format_arrow_import_nested](../operations/settings/settings.md#input_format_arrow_import_nested) setting.
### Selecting Data {#selecting-data-arrow}
You can select data from a ClickHouse table and save them into some file in the Arrow format by the following command:
@ -1384,6 +1388,8 @@ You can insert ORC data from a file into ClickHouse table by the following comma
$ cat filename.orc | clickhouse-client --query="INSERT INTO some_table FORMAT ORC"
```
To insert data into [Nested](../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs values you must switch on the [input_format_orc_import_nested](../operations/settings/settings.md#input_format_orc_import_nested) setting.
### Selecting Data {#selecting-data-2}
You can select data from a ClickHouse table and save them into some file in the ORC format by the following command:

View File

@ -1253,7 +1253,7 @@ If this section is specified, the path from [users_config](../../operations/serv
The `user_directories` section can contain any number of items, the order of the items means their precedence (the higher the item the higher the precedence).
**Example**
**Examples**
``` xml
<user_directories>
@ -1263,13 +1263,23 @@ The `user_directories` section can contain any number of items, the order of the
<local_directory>
<path>/var/lib/clickhouse/access/</path>
</local_directory>
</user_directories>
```
Users, roles, row policies, quotas, and profiles can be also stored in ZooKeeper:
``` xml
<user_directories>
<users_xml>
<path>/etc/clickhouse-server/users.xml</path>
</users_xml>
<replicated>
<zookeeper_path>/clickhouse/access/</zookeeper_path>
</replicated>
</user_directories>
```
You can also specify settings `memory` — means storing information only in memory, without writing to disk, and `ldap` — means storing information on an LDAP server.
You can also define sections `memory` — means storing information only in memory, without writing to disk, and `ldap` — means storing information on an LDAP server.
To add an LDAP server as a remote user directory of users that are not defined locally, define a single `ldap` section with a following parameters:
- `server` — one of LDAP server names defined in `ldap_servers` config section. This parameter is mandatory and cannot be empty.

View File

@ -260,6 +260,39 @@ If an error occurred while reading rows but the error counter is still less than
If both `input_format_allow_errors_num` and `input_format_allow_errors_ratio` are exceeded, ClickHouse throws an exception.
## input_format_parquet_import_nested {#input_format_parquet_import_nested}
Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [Parquet](../../interfaces/formats.md#data-format-parquet) input format.
Possible values:
- 0 — Data can not be inserted into `Nested` columns as an array of structs.
- 1 — Data can be inserted into `Nested` columns as an array of structs.
Default value: `0`.
## input_format_arrow_import_nested {#input_format_arrow_import_nested}
Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [Arrow](../../interfaces/formats.md#data_types-matching-arrow) input format.
Possible values:
- 0 — Data can not be inserted into `Nested` columns as an array of structs.
- 1 — Data can be inserted into `Nested` columns as an array of structs.
Default value: `0`.
## input_format_orc_import_nested {#input_format_orc_import_nested}
Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [ORC](../../interfaces/formats.md#data-format-orc) input format.
Possible values:
- 0 — Data can not be inserted into `Nested` columns as an array of structs.
- 1 — Data can be inserted into `Nested` columns as an array of structs.
Default value: `0`.
## input_format_values_interpret_expressions {#settings-input_format_values_interpret_expressions}
Enables or disables the full SQL parser if the fast stream parser cant parse the data. This setting is used only for the [Values](../../interfaces/formats.md#data-format-values) format at the data insertion. For more information about syntax parsing, see the [Syntax](../../sql-reference/syntax.md) section.
@ -3436,6 +3469,14 @@ Possible values:
Default value: `0`.
## materialized_postgresql_replication_slot {#materialized-postgresql-replication-slot}
Allows to have user-managed replication slots. Must be used together with `materialized_postgresql_snapshot`.
## materialized_postgresql_replication_slot {#materialized-postgresql-replication-slot}
A text string identifying a snapshot, from which initial dump of tables will be performed. Must be used together with `materialized_postgresql_replication_slot`.
## allow_experimental_projection_optimization {#allow-experimental-projection-optimization}
Enables or disables [projection](../../engines/table-engines/mergetree-family/mergetree.md#projections) optimization when processing `SELECT` queries.
@ -3449,7 +3490,7 @@ Default value: `0`.
## force_optimize_projection {#force-optimize-projection}
Enables or disables the obligatory use of [projections](../../engines/table-engines/mergetree-family/mergetree.md#projections) in `SELECT` queries, when projection optimization is enabled (see [allow_experimental_projection_optimization](#allow-experimental-projection-optimization) setting).
Enables or disables the obligatory use of [projections](../../engines/table-engines/mergetree-family/mergetree.md#projections) in `SELECT` queries, when projection optimization is enabled (see [allow_experimental_projection_optimization](#allow-experimental-projection-optimization) setting).
Possible values:
@ -3457,3 +3498,13 @@ Possible values:
- 1 — Projection optimization is obligatory.
Default value: `0`.
## regexp_max_matches_per_row {#regexp-max-matches-per-row}
Sets the maximum number of matches for a single regular expression per row. Use it to protect against memory overload when using greedy regular expression in the [extractAllGroupsHorizontal](../../sql-reference/functions/string-search-functions.md#extractallgroups-horizontal) function.
Possible values:
- Positive integer.
Default value: `1000`.

View File

@ -1,44 +0,0 @@
# system.views {#system-views}
Contains the dependencies of all views and the type to which the view belongs. The metadata of the view comes from the [system.tables](tables.md).
Columns:
- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database the view is in.
- `name` ([String](../../sql-reference/data-types/string.md)) — Name of the view.
- `main_dependency_database` ([String](../../sql-reference/data-types/string.md)) — The name of the database on which the view depends.
- `main_dependency_table` ([String](../../sql-reference/data-types/string.md)) - The name of the table on which the view depends.
- `view_type` ([Enum8](../../sql-reference/data-types/enum.md)) — Type of the view. Values:
- `'Default' = 1` — [Default views](../../sql-reference/statements/create/view.md#normal). Should not appear in this log.
- `'Materialized' = 2` — [Materialized views](../../sql-reference/statements/create/view.md#materialized).
- `'Live' = 3` — [Live views](../../sql-reference/statements/create/view.md#live-view).
**Example**
```sql
SELECT * FROM system.views LIMIT 2 FORMAT Vertical;
```
```text
Row 1:
──────
database: default
name: live_view
main_dependency_database: default
main_dependency_table: view_source_tb
view_type: Live
Row 2:
──────
database: default
name: materialized_view
main_dependency_database: default
main_dependency_table: view_source_tb
view_type: Materialized
```
[Original article](https://clickhouse.tech/docs/en/operations/system-tables/views) <!--hide-->

View File

@ -3,7 +3,9 @@ toc_priority: 57
toc_title: Nested(Name1 Type1, Name2 Type2, ...)
---
# Nested(name1 Type1, Name2 Type2, …) {#nestedname1-type1-name2-type2}
# Nested {#nested}
## Nested(name1 Type1, Name2 Type2, …) {#nestedname1-type1-name2-type2}
A nested data structure is like a table inside a cell. The parameters of a nested data structure the column names and types are specified the same way as in a [CREATE TABLE](../../../sql-reference/statements/create/table.md) query. Each table row can correspond to any number of rows in a nested data structure.

View File

@ -1438,9 +1438,9 @@ Result:
└───────────────────────────────────────────┘
```
## snowflakeToDateTime {#snowflakeToDateTime}
## snowflakeToDateTime {#snowflaketodatetime}
Extract time from snowflake id as DateTime format.
Extracts time from [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as [DateTime](../data-types/datetime.md) format.
**Syntax**
@ -1450,12 +1450,12 @@ snowflakeToDateTime(value [, time_zone])
**Parameters**
- `value``snowflake id`, Int64 value.
- `value`Snowflake ID. [Int64](../data-types/int-uint.md).
- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md).
**Returned value**
- value converted to the `DateTime` data type.
- Input value converted to the [DateTime](../data-types/datetime.md) data type.
**Example**
@ -1474,9 +1474,9 @@ Result:
└──────────────────────────────────────────────────────────────────┘
```
## snowflakeToDateTime64 {#snowflakeToDateTime64}
## snowflakeToDateTime64 {#snowflaketodatetime64}
Extract time from snowflake id as DateTime64 format.
Extracts time from [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as [DateTime64](../data-types/datetime64.md) format.
**Syntax**
@ -1486,12 +1486,12 @@ snowflakeToDateTime64(value [, time_zone])
**Parameters**
- `value``snowflake id`, Int64 value.
- `value`Snowflake ID. [Int64](../data-types/int-uint.md).
- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md).
**Returned value**
- value converted to the `DateTime64` data type.
- Input value converted to the [DateTime64](../data-types/datetime64.md) data type.
**Example**
@ -1510,9 +1510,9 @@ Result:
└────────────────────────────────────────────────────────────────────┘
```
## dateTimeToSnowflake {#dateTimeToSnowflake}
## dateTimeToSnowflake {#datetimetosnowflake}
Convert DateTime to the first snowflake id at the giving time.
Converts [DateTime](../data-types/datetime.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time.
**Syntax**
@ -1524,33 +1524,29 @@ dateTimeToSnowflake(value)
- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md).
**Returned value**
- `value` converted to the `Int64` data type as the first snowflake id at that time.
- Input value converted to the [Int64](../data-types/int-uint.md) data type as the first Snowflake ID at that time.
**Example**
Query:
``` sql
WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt
SELECT dateTimeToSnowflake(dt);
WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt SELECT dateTimeToSnowflake(dt);
```
Result:
``` text
┌─dateTimeToSnowflake(dt)─┐
│ 1426860702823350272 │
└─────────────────────────┘
```
## dateTime64ToSnowflake {#datetime64tosnowflake}
## dateTime64ToSnowflake {#dateTime64ToSnowflake}
Convert DateTime64 to the first snowflake id at the giving time.
Convert [DateTime64](../data-types/datetime64.md) to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time.
**Syntax**
@ -1562,18 +1558,16 @@ dateTime64ToSnowflake(value)
- `value` — Date and time. [DateTime64](../../sql-reference/data-types/datetime64.md).
**Returned value**
- `value` converted to the `Int64` data type as the first snowflake id at that time.
- Input value converted to the [Int64](../data-types/int-uint.md) data type as the first Snowflake ID at that time.
**Example**
Query:
``` sql
WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS dt64
SELECT dateTime64ToSnowflake(dt64);
WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS dt64 SELECT dateTime64ToSnowflake(dt64);
```
Result:
@ -1582,4 +1576,4 @@ Result:
┌─dateTime64ToSnowflake(dt64)─┐
│ 1426860704886947840 │
└─────────────────────────────┘
```
```

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

@ -1180,7 +1180,7 @@ ClickHouse поддерживает настраиваемую точность
Типы данных столбцов в ClickHouse могут отличаться от типов данных соответствующих полей файла в формате Parquet. При вставке данных ClickHouse интерпретирует типы данных в соответствии с таблицей выше, а затем [приводит](../sql-reference/functions/type-conversion-functions/#type_conversion_function-cast) данные к тому типу, который установлен для столбца таблицы.
### Вставка и выборка данных {#vstavka-i-vyborka-dannykh}
### Вставка и выборка данных {#inserting-and-selecting-data}
Чтобы вставить в ClickHouse данные из файла в формате Parquet, выполните команду следующего вида:
@ -1188,6 +1188,8 @@ ClickHouse поддерживает настраиваемую точность
$ cat {filename} | clickhouse-client --query="INSERT INTO {some_table} FORMAT Parquet"
```
Чтобы вставить данные в колонки типа [Nested](../sql-reference/data-types/nested-data-structures/nested.md) в виде массива структур, нужно включить настройку [input_format_parquet_import_nested](../operations/settings/settings.md#input_format_parquet_import_nested).
Чтобы получить данные из таблицы ClickHouse и сохранить их в файл формата Parquet, используйте команду следующего вида:
``` bash
@ -1246,6 +1248,8 @@ ClickHouse поддерживает настраиваемую точность
$ cat filename.arrow | clickhouse-client --query="INSERT INTO some_table FORMAT Arrow"
```
Чтобы вставить данные в колонки типа [Nested](../sql-reference/data-types/nested-data-structures/nested.md) в виде массива структур, нужно включить настройку [input_format_arrow_import_nested](../operations/settings/settings.md#input_format_arrow_import_nested).
### Вывод данных {#selecting-data-arrow}
Чтобы получить данные из таблицы ClickHouse и сохранить их в файл формата Arrow, используйте команду следующего вида:
@ -1294,7 +1298,7 @@ ClickHouse поддерживает настраиваемую точность
Типы данных столбцов в таблицах ClickHouse могут отличаться от типов данных для соответствующих полей ORC. При вставке данных ClickHouse интерпретирует типы данных ORC согласно таблице соответствия, а затем [приводит](../sql-reference/functions/type-conversion-functions/#type_conversion_function-cast) данные к типу, установленному для столбца таблицы ClickHouse.
### Вставка данных {#vstavka-dannykh-1}
### Вставка данных {#inserting-data-2}
Чтобы вставить в ClickHouse данные из файла в формате ORC, используйте команду следующего вида:
@ -1302,7 +1306,9 @@ ClickHouse поддерживает настраиваемую точность
$ cat filename.orc | clickhouse-client --query="INSERT INTO some_table FORMAT ORC"
```
### Вывод данных {#vyvod-dannykh-1}
Чтобы вставить данные в колонки типа [Nested](../sql-reference/data-types/nested-data-structures/nested.md) в виде массива структур, нужно включить настройку [input_format_orc_import_nested](../operations/settings/settings.md#input_format_orc_import_nested).
### Вывод данных {#selecting-data-2}
Чтобы получить данные из таблицы ClickHouse и сохранить их в файл формата ORC, используйте команду следующего вида:

View File

@ -1200,12 +1200,13 @@ ClickHouse использует ZooKeeper для хранения метадан
Секция конфигурационного файла,которая содержит настройки:
- Путь к конфигурационному файлу с предустановленными пользователями.
- Путь к файлу, в котором содержатся пользователи, созданные при помощи SQL команд.
- Путь к узлу ZooKeeper, где хранятся и реплицируются пользователи, созданные с помощью команд SQL (экспериментальная функциональность).
Если эта секция определена, путь из [users_config](../../operations/server-configuration-parameters/settings.md#users-config) и [access_control_path](../../operations/server-configuration-parameters/settings.md#access_control_path) не используется.
Секция `user_directories` может содержать любое количество элементов, порядок расположения элементов обозначает их приоритет (чем выше элемент, тем выше приоритет).
**Пример**
**Примеры**
``` xml
<user_directories>
@ -1218,7 +1219,20 @@ ClickHouse использует ZooKeeper для хранения метадан
</user_directories>
```
Также вы можете указать настройку `memory` — означает хранение информации только в памяти, без записи на диск, и `ldap` — означает хранения информации на [LDAP-сервере](https://en.wikipedia.org/wiki/Lightweight_Directory_Access_Protocol).
Пользователи, роли, политики доступа к строкам, квоты и профили могут храниться в ZooKeeper:
``` xml
<user_directories>
<users_xml>
<path>/etc/clickhouse-server/users.xml</path>
</users_xml>
<replicated>
<zookeeper_path>/clickhouse/access/</zookeeper_path>
</replicated>
</user_directories>
```
Также вы можете добавить секции `memory` — означает хранение информации только в памяти, без записи на диск, и `ldap` — означает хранения информации на [LDAP-сервере](https://en.wikipedia.org/wiki/Lightweight_Directory_Access_Protocol).
Чтобы добавить LDAP-сервер в качестве удаленного каталога пользователей, которые не определены локально, определите один раздел `ldap` со следующими параметрами:
- `server` — имя одного из LDAP-серверов, определенных в секции `ldap_servers` конфигурациионного файла. Этот параметр явялется необязательным и может быть пустым.

View File

@ -237,6 +237,39 @@ ClickHouse применяет настройку в тех случаях, ко
В случае превышения `input_format_allow_errors_ratio` ClickHouse генерирует исключение.
## input_format_parquet_import_nested {#input_format_parquet_import_nested}
Включает или отключает возможность вставки данных в колонки типа [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) в виде массива структур в формате ввода [Parquet](../../interfaces/formats.md#data-format-parquet).
Возможные значения:
- 0 — данные не могут быть вставлены в колонки типа `Nested` в виде массива структур.
- 0 — данные могут быть вставлены в колонки типа `Nested` в виде массива структур.
Значение по умолчанию: `0`.
## input_format_arrow_import_nested {#input_format_arrow_import_nested}
Включает или отключает возможность вставки данных в колонки типа [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) в виде массива структур в формате ввода [Arrow](../../interfaces/formats.md#data_types-matching-arrow).
Возможные значения:
- 0 — данные не могут быть вставлены в колонки типа `Nested` в виде массива структур.
- 0 — данные могут быть вставлены в колонки типа `Nested` в виде массива структур.
Значение по умолчанию: `0`.
## input_format_orc_import_nested {#input_format_orc_import_nested}
Включает или отключает возможность вставки данных в колонки типа [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) в виде массива структур в формате ввода [ORC](../../interfaces/formats.md#data-format-orc).
Возможные значения:
- 0 — данные не могут быть вставлены в колонки типа `Nested` в виде массива структур.
- 0 — данные могут быть вставлены в колонки типа `Nested` в виде массива структур.
Значение по умолчанию: `0`.
## input_format_values_interpret_expressions {#settings-input_format_values_interpret_expressions}
Включает или отключает парсер SQL, если потоковый парсер не может проанализировать данные. Этот параметр используется только для формата [Values](../../interfaces/formats.md#data-format-values) при вставке данных. Дополнительные сведения о парсерах читайте в разделе [Синтаксис](../../sql-reference/syntax.md).
@ -3273,4 +3306,14 @@ SETTINGS index_granularity = 8192 │
- 0 — Проекции используются опционально.
- 1 — Проекции обязательно используются.
Значение по умолчанию: `0`.
Значение по умолчанию: `0`.
## regexp_max_matches_per_row {#regexp-max-matches-per-row}
Задает максимальное количество совпадений для регулярного выражения. Настройка применяется для защиты памяти от перегрузки при использовании "жадных" квантификаторов в регулярном выражении для функции [extractAllGroupsHorizontal](../../sql-reference/functions/string-search-functions.md#extractallgroups-horizontal).
Возможные значения:
- Положительное целое число.
Значение по умолчанию: `1000`.

View File

@ -1,4 +1,6 @@
# Nested(Name1 Type1, Name2 Type2, …) {#nestedname1-type1-name2-type2}
# Nested {#nested}
## Nested(Name1 Type1, Name2 Type2, …) {#nestedname1-type1-name2-type2}
Вложенная структура данных - это как будто вложенная таблица. Параметры вложенной структуры данных - имена и типы столбцов, указываются так же, как у запроса CREATE. Каждой строке таблицы может соответствовать произвольное количество строк вложенной структуры данных.
@ -95,4 +97,3 @@ LIMIT 10
При запросе DESCRIBE, столбцы вложенной структуры данных перечисляются так же по отдельности.
Работоспособность запроса ALTER для элементов вложенных структур данных, является сильно ограниченной.

View File

@ -1436,3 +1436,144 @@ FROM numbers(3);
│ 2,"good" │
└───────────────────────────────────────────┘
```
## snowflakeToDateTime {#snowflaketodatetime}
Извлекает время из [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) в формате [DateTime](../data-types/datetime.md).
**Синтаксис**
``` sql
snowflakeToDateTime(value [, time_zone])
```
**Аргументы**
- `value` — Snowflake ID. [Int64](../data-types/int-uint.md).
- `time_zone` — [временная зона сервера](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). Функция распознает `time_string` в соответствии с часовым поясом. Необязательный. [String](../../sql-reference/data-types/string.md).
**Возвращаемое значение**
- Значение, преобразованное в фомат [DateTime](../data-types/datetime.md).
**Пример**
Запрос:
``` sql
SELECT snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC');
```
Результат:
``` text
┌─snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC')─┐
│ 2021-08-15 10:57:56 │
└──────────────────────────────────────────────────────────────────┘
```
## snowflakeToDateTime64 {#snowflaketodatetime64}
Извлекает время из [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) в формате [DateTime64](../data-types/datetime64.md).
**Синтаксис**
``` sql
snowflakeToDateTime64(value [, time_zone])
```
**Аргументы**
- `value` — Snowflake ID. [Int64](../data-types/int-uint.md).
- `time_zone` — [временная зона сервера](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). Функция распознает `time_string` в соответствии с часовым поясом. Необязательный. [String](../../sql-reference/data-types/string.md).
**Возвращаемое значение**
- Значение, преобразованное в фомат [DateTime64](../data-types/datetime64.md).
**Пример**
Запрос:
``` sql
SELECT snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC');
```
Результат:
``` text
┌─snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC')─┐
│ 2021-08-15 10:58:19.841 │
└────────────────────────────────────────────────────────────────────┘
```
## dateTimeToSnowflake {#datetimetosnowflake}
Преобразует значение [DateTime](../data-types/datetime.md) в первый идентификатор [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) на текущий момент.
**Syntax**
``` sql
dateTimeToSnowflake(value)
```
**Аргументы**
- `value` — дата и время. [DateTime](../../sql-reference/data-types/datetime.md).
**Возвращаемое значение**
- Значение, преобразованное в [Int64](../data-types/int-uint.md), как первый идентификатор Snowflake ID в момент выполнения.
**Пример**
Запрос:
``` sql
WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt SELECT dateTimeToSnowflake(dt);
```
Результат:
``` text
┌─dateTimeToSnowflake(dt)─┐
│ 1426860702823350272 │
└─────────────────────────┘
```
## dateTime64ToSnowflake {#datetime64tosnowflake}
Преобразует значение [DateTime64](../data-types/datetime64.md) в первый идентификатор [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) на текущий момент.
**Синтаксис**
``` sql
dateTime64ToSnowflake(value)
```
**Аргументы**
- `value` — дата и время. [DateTime64](../data-types/datetime64.md).
**Возвращаемое значение**
- Значение, преобразованное в [Int64](../data-types/int-uint.md), как первый идентификатор Snowflake ID в момент выполнения.
**Пример**
Запрос:
``` sql
WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS dt64 SELECT dateTime64ToSnowflake(dt64);
```
Результат:
``` text
┌─dateTime64ToSnowflake(dt64)─┐
│ 1426860704886947840 │
└─────────────────────────────┘
```

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

@ -247,6 +247,7 @@ CREATE TABLE codec_example
)
ENGINE = MergeTree()
```
## Временные таблицы {#temporary-tables}
ClickHouse поддерживает временные таблицы со следующими характеристиками:

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>
@ -241,6 +242,19 @@ void LocalServer::initialize(Poco::Util::Application & self)
}
static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const String & database_name)
{
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>(database_name, context);
DatabaseCatalog::instance().attachDatabase(database_name, system_database);
}
return system_database;
}
/// If path is specified and not empty, will try to setup server environment and load existing metadata
void LocalServer::tryInitPath()
{
@ -313,17 +327,16 @@ 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;
}
@ -559,6 +572,8 @@ void LocalServer::processConfig()
/// Sets external authenticators config (LDAP, Kerberos).
global_context->setExternalAuthenticatorsConfig(config());
global_context->initializeBackgroundExecutors();
setupUsers();
/// Limit on total number of concurrently executing queries.
@ -615,7 +630,9 @@ void LocalServer::processConfig()
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();
@ -623,7 +640,9 @@ void LocalServer::processConfig()
}
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));
}
server_display_name = config().getString("display_name", getFQDNOrHostName());

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>
@ -45,19 +45,18 @@
#include <IO/UseSSL.h>
#include <Interpreters/AsynchronousMetrics.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/ExternalModelsLoader.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/loadMetadata.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Interpreters/InterserverCredentials.h>
#include <Interpreters/UserDefinedObjectsLoader.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
#include <Interpreters/UserDefinedObjectsLoader.h>
#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>
@ -549,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");
@ -1131,6 +1132,12 @@ 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.
database_catalog.loadMarkedAsDroppedTables();
/// Then, load remaining databases
loadMetadata(global_context, default_database);
database_catalog.loadDatabases();
@ -1506,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

@ -18,6 +18,7 @@
- information
- debug
- trace
- test (not for production usage)
[1]: https://github.com/pocoproject/poco/blob/poco-1.9.4-release/Foundation/include/Poco/Logger.h#L105-L114
-->
@ -964,6 +965,14 @@
<flush_interval_milliseconds>1000</flush_interval_milliseconds>
</crash_log>
<!-- Session log. Stores user log in (successful or not) and log out events. -->
<session_log>
<database>system</database>
<table>session_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</session_log>
<!-- Parameters for embedded dictionaries, used in Yandex.Metrica.
See https://clickhouse.yandex/docs/en/dicts/internal_dicts/

View File

@ -0,0 +1 @@
../../../tests/config/users.d/session_log_test.xml

View File

@ -143,10 +143,11 @@ public:
std::vector<QuotaUsage> getAllQuotasUsage() const;
std::shared_ptr<const EnabledSettings> getEnabledSettings(const UUID & user_id,
const SettingsProfileElements & settings_from_user,
const boost::container::flat_set<UUID> & enabled_roles,
const SettingsProfileElements & settings_from_enabled_roles) const;
std::shared_ptr<const EnabledSettings> getEnabledSettings(
const UUID & user_id,
const SettingsProfileElements & settings_from_user,
const boost::container::flat_set<UUID> & enabled_roles,
const SettingsProfileElements & settings_from_enabled_roles) const;
std::shared_ptr<const SettingsProfilesInfo> getSettingsProfileInfo(const UUID & profile_id);

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

@ -36,6 +36,16 @@ struct SettingsProfilesInfo
friend bool operator ==(const SettingsProfilesInfo & lhs, const SettingsProfilesInfo & rhs);
friend bool operator !=(const SettingsProfilesInfo & lhs, const SettingsProfilesInfo & rhs) { return !(lhs == rhs); }
Strings getProfileNames() const
{
Strings result;
result.reserve(profiles.size());
for (const auto & profile_id : profiles)
result.push_back(names_of_profiles.at(profile_id));
return result;
}
private:
const AccessControlManager & manager;
};

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

@ -27,7 +27,12 @@ private:
size_t stack_size;
size_t page_size = 0;
public:
static constexpr size_t default_stack_size = 128 * 1024; /// 64KB was not enough for tests
/// NOTE: If you see random segfaults in CI and stack starts from boost::context::...fiber...
/// 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 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_)
{
@ -43,6 +48,8 @@ public:
if (MAP_FAILED == vp)
DB::throwFromErrno(fmt::format("FiberStack: Cannot mmap {}.", ReadableSize(num_bytes)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
/// TODO: make reports on illegal guard page access more clear.
/// Currently we will see segfault and almost random stacktrace.
if (-1 == ::mprotect(vp, page_size, PROT_NONE))
{
::munmap(vp, num_bytes);

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

@ -6,6 +6,8 @@
#include <Poco/Logger.h>
#include <Poco/AutoPtr.h>
#include <Poco/NullChannel.h>
#include <Poco/StreamChannel.h>
#include <sstream>
TEST(Logger, Log)
@ -17,3 +19,34 @@ TEST(Logger, Log)
/// This test checks that we don't pass this string to fmtlib, because it is the only argument.
EXPECT_NO_THROW(LOG_INFO(log, "Hello {} World"));
}
TEST(Logger, TestLog)
{
{ /// Test logs visible for test level
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
auto my_channel = Poco::AutoPtr<Poco::StreamChannel>(new Poco::StreamChannel(oss));
auto * log = &Poco::Logger::create("TestLogger", my_channel.get());
log->setLevel("test");
LOG_TEST(log, "Hello World");
EXPECT_EQ(oss.str(), "Hello World\n");
Poco::Logger::destroy("TestLogger");
}
{ /// Test logs invisible for other levels
for (const auto & level : {"trace", "debug", "information", "warning", "error", "fatal"})
{
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
auto my_channel = Poco::AutoPtr<Poco::StreamChannel>(new Poco::StreamChannel(oss));
auto * log = &Poco::Logger::create(std::string{level} + "_Logger", my_channel.get());
log->setLevel(level);
LOG_TEST(log, "Hello World");
EXPECT_EQ(oss.str(), "");
Poco::Logger::destroy(std::string{level} + "_Logger");
}
}
}

View File

@ -113,7 +113,8 @@ namespace DB
std::string CompressionCodecEncrypted::deriveKey(const std::string_view & master_key)
{
std::string_view salt(""); // No salt: derive keys in a deterministic manner.
/// No salt: derive keys in a deterministic manner.
std::string_view salt(""); // NOLINT
std::string_view info("Codec Encrypted('AES-128-GCM-SIV') key generation key");
std::array<char, 32> result;

View File

@ -122,6 +122,10 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
}
else
{
LOG_TEST(log, "Commit request for session {} with type {}, log id {}{}",
request_for_session.session_id, toString(request_for_session.request->getOpNum()), log_idx,
request_for_session.request->getPath().empty() ? "" : ", path " + request_for_session.request->getPath());
std::lock_guard lock(storage_and_responses_lock);
KeeperStorage::ResponsesForSessions responses_for_sessions = storage->processRequest(request_for_session.request, request_for_session.session_id, log_idx);
for (auto & response_for_session : responses_for_sessions)

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

@ -900,8 +900,7 @@ public:
Messaging::MessageTransport & mt,
const Poco::Net::SocketAddress & address)
{
Authentication::Type user_auth_type = session.getAuthenticationType(user_name);
const Authentication::Type user_auth_type = session.getAuthenticationTypeOrLogInFailure(user_name);
if (type_to_method.find(user_auth_type) != type_to_method.end())
{
type_to_method[user_auth_type]->authenticate(user_name, session, mt, address);

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) \
@ -70,8 +70,8 @@ class IColumn;
M(UInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.", 0) \
M(UInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.", 0) \
M(UInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \
M(UInt64, s3_min_upload_part_size, 512*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \
M(UInt64, s3_max_single_part_upload_size, 64*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \
M(UInt64, s3_min_upload_part_size, 32*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \
M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \
M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \
M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \
M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \

View File

@ -79,8 +79,8 @@ IMPLEMENT_SETTING_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS,
{"warning", LogsLevel::warning},
{"information", LogsLevel::information},
{"debug", LogsLevel::debug},
{"trace", LogsLevel::trace}})
{"trace", LogsLevel::trace},
{"test", LogsLevel::test}})
IMPLEMENT_SETTING_ENUM_WITH_RENAME(LogQueriesType, ErrorCodes::BAD_ARGUMENTS,
{{"QUERY_START", QUERY_START},

View File

@ -94,6 +94,7 @@ enum class LogsLevel
information,
debug,
trace,
test,
};
DECLARE_SETTING_ENUM(LogsLevel)

View File

@ -3,8 +3,11 @@
#include <memory>
#include <common/logger_useful.h>
#include <common/BorrowedObjectPool.h>
#include <Common/ShellCommand.h>
#include <Common/ThreadPool.h>
#include <IO/ReadHelpers.h>
#include <Formats/FormatFactory.h>
#include <Processors/ISimpleTransform.h>
@ -17,44 +20,85 @@
namespace DB
{
/** A stream, that runs child process and sends data to its stdin in background thread,
* and receives data from its stdout.
/** A stream, that get child process and sends data using tasks in background threads.
* For each send data task background thread is created. Send data task must send data to process input pipes.
* ShellCommandPoolSource receives data from process stdout.
*
* If process_pool is passed in constructor then after source is destroyed process is returned to pool.
*/
using ProcessPool = BorrowedObjectPool<std::unique_ptr<ShellCommand>>;
struct ShellCommandSourceConfiguration
{
/// Read fixed number of rows from command output
bool read_fixed_number_of_rows = false;
/// Valid only if read_fixed_number_of_rows = true
bool read_number_of_rows_from_process_output = false;
/// Valid only if read_fixed_number_of_rows = true
size_t number_of_rows_to_read = 0;
/// Max block size
size_t max_block_size = DBMS_DEFAULT_BUFFER_SIZE;
};
class ShellCommandSource final : public SourceWithProgress
{
public:
using SendDataTask = std::function<void (void)>;
using SendDataTask = std::function<void(void)>;
ShellCommandSource(
ContextPtr context,
const std::string & format,
const Block & sample_block,
std::unique_ptr<ShellCommand> command_,
std::unique_ptr<ShellCommand> && command_,
Poco::Logger * log_,
std::vector<SendDataTask> && send_data_tasks,
size_t max_block_size = DEFAULT_BLOCK_SIZE)
std::vector<SendDataTask> && send_data_tasks = {},
const ShellCommandSourceConfiguration & configuration_ = {},
std::shared_ptr<ProcessPool> process_pool_ = nullptr)
: SourceWithProgress(sample_block)
, command(std::move(command_))
, configuration(configuration_)
, log(log_)
, process_pool(process_pool_)
{
for (auto && send_data_task : send_data_tasks)
send_data_threads.emplace_back([task = std::move(send_data_task)]() { task(); });
{
send_data_threads.emplace_back([task = std::move(send_data_task), this]()
{
try
{
task();
}
catch (...)
{
std::lock_guard<std::mutex> lock(send_data_lock);
exception_during_send_data = std::current_exception();
}
});
}
pipeline.init(Pipe(FormatFactory::instance().getInput(format, command->out, sample_block, context, max_block_size)));
executor = std::make_unique<PullingPipelineExecutor>(pipeline);
}
size_t max_block_size = configuration.max_block_size;
if (configuration.read_fixed_number_of_rows)
{
/** Currently parallel parsing input format cannot read exactly max_block_size rows from input,
* so it will be blocked on ReadBufferFromFileDescriptor because this file descriptor represent pipe that does not have eof.
*/
auto context_for_reading = Context::createCopy(context);
context_for_reading->setSetting("input_format_parallel_parsing", false);
context = context_for_reading;
if (configuration.read_number_of_rows_from_process_output)
{
readText(configuration.number_of_rows_to_read, command->out);
char dummy;
readChar(dummy, command->out);
}
max_block_size = configuration.number_of_rows_to_read;
}
ShellCommandSource(
ContextPtr context,
const std::string & format,
const Block & sample_block,
std::unique_ptr<ShellCommand> command_,
Poco::Logger * log_,
size_t max_block_size = DEFAULT_BLOCK_SIZE)
: SourceWithProgress(sample_block)
, command(std::move(command_))
, log(log_)
{
pipeline.init(Pipe(FormatFactory::instance().getInput(format, command->out, sample_block, context, max_block_size)));
executor = std::make_unique<PullingPipelineExecutor>(pipeline);
}
@ -64,33 +108,50 @@ public:
for (auto & thread : send_data_threads)
if (thread.joinable())
thread.join();
if (command && process_pool)
process_pool->returnObject(std::move(command));
}
protected:
Chunk generate() override
{
rethrowExceptionDuringSendDataIfNeeded();
if (configuration.read_fixed_number_of_rows && configuration.number_of_rows_to_read == current_read_rows)
return {};
Chunk chunk;
executor->pull(chunk);
try
{
if (!executor->pull(chunk))
return {};
current_read_rows += chunk.getNumRows();
}
catch (...)
{
tryLogCurrentException(log);
command = nullptr;
throw;
}
return chunk;
}
public:
Status prepare() override
{
auto status = SourceWithProgress::prepare();
if (status == Status::Finished)
{
std::string err;
readStringUntilEOF(err, command->err);
if (!err.empty())
LOG_ERROR(log, "Having stderr: {}", err);
for (auto & thread : send_data_threads)
if (thread.joinable())
thread.join();
command->wait();
rethrowExceptionDuringSendDataIfNeeded();
}
return status;
@ -100,11 +161,30 @@ public:
private:
void rethrowExceptionDuringSendDataIfNeeded()
{
std::lock_guard<std::mutex> lock(send_data_lock);
if (exception_during_send_data)
{
command = nullptr;
std::rethrow_exception(exception_during_send_data);
}
}
std::unique_ptr<ShellCommand> command;
ShellCommandSourceConfiguration configuration;
size_t current_read_rows = 0;
Poco::Logger * log;
std::shared_ptr<ProcessPool> process_pool;
QueryPipeline pipeline;
std::unique_ptr<PullingPipelineExecutor> executor;
std::unique_ptr<ShellCommand> command;
std::vector<ThreadFromGlobalPool> send_data_threads;
Poco::Logger * log;
};
std::vector<ThreadFromGlobalPool> send_data_threads;
std::mutex send_data_lock;
std::exception_ptr exception_during_send_data;
};
}

View File

@ -247,7 +247,7 @@ void MaterializedMySQLSyncThread::assertMySQLAvailable()
{
try
{
checkMySQLVariables(pool.get(), getContext()->getSettingsRef());
checkMySQLVariables(pool.get(/* wait_timeout= */ UINT64_MAX), getContext()->getSettingsRef());
}
catch (const mysqlxx::ConnectionFailed & e)
{
@ -729,7 +729,7 @@ void MaterializedMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPt
{
/// Some behaviors(such as changing the value of "binlog_checksum") rotate the binlog file.
/// To ensure that the synchronization continues, we need to handle these events
metadata.fetchMasterVariablesValue(pool.get());
metadata.fetchMasterVariablesValue(pool.get(/* wait_timeout= */ UINT64_MAX));
client.setBinlogChecksum(metadata.binlog_checksum);
}
else if (receive_event->header.type != HEARTBEAT_EVENT)

View File

@ -61,10 +61,8 @@ void DatabaseMaterializedPostgreSQL::startSynchronization()
connection_info,
getContext(),
is_attach,
settings->materialized_postgresql_max_block_size.value,
settings->materialized_postgresql_allow_automatic_update,
/* is_materialized_postgresql_database = */ true,
settings->materialized_postgresql_tables_list.value);
*settings,
/* is_materialized_postgresql_database = */ true);
postgres::Connection connection(connection_info);
NameSet tables_to_replicate;

View File

@ -1,6 +1,5 @@
#include "ExecutableDictionarySource.h"
#include <functional>
#include <common/logger_useful.h>
#include <common/LocalDateTime.h>
#include <Common/ShellCommand.h>
@ -125,8 +124,7 @@ Pipe ExecutableDictionarySource::getStreamForBlock(const Block & block)
formatBlock(output_stream, block);
out.close();
}};
std::vector<ShellCommandSource::SendDataTask> tasks = {task};
std::vector<ShellCommandSource::SendDataTask> tasks = {std::move(task)};
Pipe pipe(std::make_unique<ShellCommandSource>(context, configuration.format, sample_block, std::move(process), log, std::move(tasks)));

View File

@ -1,11 +1,12 @@
#pragma once
#include "DictionaryStructure.h"
#include "IDictionarySource.h"
#include <common/logger_useful.h>
#include <Core/Block.h>
#include <Interpreters/Context.h>
namespace Poco { class Logger; }
#include <Dictionaries/IDictionarySource.h>
#include <Dictionaries/DictionaryStructure.h>
namespace DB

View File

@ -1,24 +1,18 @@
#include "ExecutablePoolDictionarySource.h"
#include <functional>
#include <common/scope_guard.h>
#include <Processors/Sources/SourceWithProgress.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/QueryPipeline.h>
#include <common/logger_useful.h>
#include <common/LocalDateTime.h>
#include <Common/ShellCommand.h>
#include <DataStreams/formatBlock.h>
#include <Interpreters/Context.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/IInputFormat.h>
#include <Common/ShellCommand.h>
#include <Common/ThreadPool.h>
#include <common/logger_useful.h>
#include <common/LocalDateTime.h>
#include "DictionarySourceFactory.h"
#include "DictionarySourceHelpers.h"
#include "DictionaryStructure.h"
#include "registerDictionaries.h"
#include <Dictionaries/DictionarySourceFactory.h>
#include <Dictionaries/DictionarySourceHelpers.h>
#include <Dictionaries/DictionaryStructure.h>
namespace DB
@ -37,13 +31,13 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource(
const Configuration & configuration_,
Block & sample_block_,
ContextPtr context_)
: log(&Poco::Logger::get("ExecutablePoolDictionarySource"))
, dict_struct{dict_struct_}
, configuration{configuration_}
, sample_block{sample_block_}
, context{context_}
: dict_struct(dict_struct_)
, configuration(configuration_)
, sample_block(sample_block_)
, context(context_)
/// If pool size == 0 then there is no size restrictions. Poco max size of semaphore is integer type.
, process_pool{std::make_shared<ProcessPool>(configuration.pool_size == 0 ? std::numeric_limits<int>::max() : configuration.pool_size)}
, process_pool(std::make_shared<ProcessPool>(configuration.pool_size == 0 ? std::numeric_limits<int>::max() : configuration.pool_size))
, log(&Poco::Logger::get("ExecutablePoolDictionarySource"))
{
/// Remove keys from sample_block for implicit_key dictionary because
/// these columns will not be returned from source
@ -62,13 +56,12 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource(
}
ExecutablePoolDictionarySource::ExecutablePoolDictionarySource(const ExecutablePoolDictionarySource & other)
: log(&Poco::Logger::get("ExecutablePoolDictionarySource"))
, update_time{other.update_time}
, dict_struct{other.dict_struct}
, configuration{other.configuration}
, sample_block{other.sample_block}
, context{Context::createCopy(other.context)}
, process_pool{std::make_shared<ProcessPool>(configuration.pool_size)}
: dict_struct(other.dict_struct)
, configuration(other.configuration)
, sample_block(other.sample_block)
, context(Context::createCopy(other.context))
, process_pool(std::make_shared<ProcessPool>(configuration.pool_size))
, log(&Poco::Logger::get("ExecutablePoolDictionarySource"))
{
}
@ -82,123 +75,6 @@ Pipe ExecutablePoolDictionarySource::loadUpdatedAll()
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutablePoolDictionarySource does not support loadUpdatedAll method");
}
namespace
{
/** A stream, that runs child process and sends data to its stdin in background thread,
* and receives data from its stdout.
*/
class PoolSourceWithBackgroundThread final : public SourceWithProgress
{
public:
PoolSourceWithBackgroundThread(
std::shared_ptr<ProcessPool> process_pool_,
std::unique_ptr<ShellCommand> && command_,
Pipe pipe,
size_t read_rows_,
Poco::Logger * log_,
std::function<void(WriteBufferFromFile &)> && send_data_)
: SourceWithProgress(pipe.getHeader())
, process_pool(process_pool_)
, command(std::move(command_))
, rows_to_read(read_rows_)
, log(log_)
, send_data(std::move(send_data_))
, thread([this]
{
try
{
send_data(command->in);
}
catch (...)
{
std::lock_guard<std::mutex> lck(exception_during_read_lock);
exception_during_read = std::current_exception();
}
})
{
pipeline.init(std::move(pipe));
executor = std::make_unique<PullingPipelineExecutor>(pipeline);
}
~PoolSourceWithBackgroundThread() override
{
if (thread.joinable())
thread.join();
if (command)
process_pool->returnObject(std::move(command));
}
protected:
Chunk generate() override
{
rethrowExceptionDuringReadIfNeeded();
if (current_read_rows == rows_to_read)
return {};
Chunk chunk;
try
{
if (!executor->pull(chunk))
return {};
current_read_rows += chunk.getNumRows();
}
catch (...)
{
tryLogCurrentException(log);
command = nullptr;
throw;
}
return chunk;
}
public:
Status prepare() override
{
auto status = SourceWithProgress::prepare();
if (status == Status::Finished)
{
if (thread.joinable())
thread.join();
rethrowExceptionDuringReadIfNeeded();
}
return status;
}
void rethrowExceptionDuringReadIfNeeded()
{
std::lock_guard<std::mutex> lck(exception_during_read_lock);
if (exception_during_read)
{
command = nullptr;
std::rethrow_exception(exception_during_read);
}
}
String getName() const override { return "PoolWithBackgroundThread"; }
std::shared_ptr<ProcessPool> process_pool;
std::unique_ptr<ShellCommand> command;
QueryPipeline pipeline;
std::unique_ptr<PullingPipelineExecutor> executor;
size_t rows_to_read;
Poco::Logger * log;
std::function<void(WriteBufferFromFile &)> send_data;
ThreadFromGlobalPool thread;
size_t current_read_rows = 0;
std::mutex exception_during_read_lock;
std::exception_ptr exception_during_read;
};
}
Pipe ExecutablePoolDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
@ -228,19 +104,23 @@ Pipe ExecutablePoolDictionarySource::getStreamForBlock(const Block & block)
if (!result)
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED,
"Could not get process from pool, max command execution timeout exceeded ({}) seconds",
"Could not get process from pool, max command execution timeout exceeded {} seconds",
configuration.max_command_execution_time);
size_t rows_to_read = block.rows();
auto format = FormatFactory::instance().getInput(configuration.format, process->out, sample_block, context, rows_to_read);
auto * process_in = &process->in;
ShellCommandSource::SendDataTask task = [process_in, block, this]() mutable
{
auto & out = *process_in;
auto output_stream = context->getOutputStream(configuration.format, out, block.cloneEmpty());
formatBlock(output_stream, block);
};
std::vector<ShellCommandSource::SendDataTask> tasks = {std::move(task)};
Pipe pipe(std::make_unique<PoolSourceWithBackgroundThread>(
process_pool, std::move(process), Pipe(std::move(format)), rows_to_read, log,
[block, this](WriteBufferFromFile & out) mutable
{
auto output_stream = context->getOutputStream(configuration.format, out, block.cloneEmpty());
formatBlock(output_stream, block);
}));
ShellCommandSourceConfiguration command_configuration;
command_configuration.read_fixed_number_of_rows = true;
command_configuration.number_of_rows_to_read = rows_to_read;
Pipe pipe(std::make_unique<ShellCommandSource>(context, configuration.format, sample_block, std::move(process), log, std::move(tasks), command_configuration, process_pool));
if (configuration.implicit_key)
pipe.addTransform(std::make_shared<TransformWithAdditionalColumns>(block, pipe.getHeader()));
@ -294,11 +174,6 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory)
ContextMutablePtr context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix);
/** Currently parallel parsing input format cannot read exactly max_block_size rows from input,
* so it will be blocked on ReadBufferFromFileDescriptor because this file descriptor represent pipe that does not have eof.
*/
context->setSetting("input_format_parallel_parsing", false);
String settings_config_prefix = config_prefix + ".executable_pool";
size_t max_command_execution_time = config.getUInt64(settings_config_prefix + ".max_command_execution_time", 10);

View File

@ -1,20 +1,18 @@
#pragma once
#include <common/BorrowedObjectPool.h>
#include <common/logger_useful.h>
#include <Core/Block.h>
#include <Interpreters/Context.h>
#include "IDictionarySource.h"
#include "DictionaryStructure.h"
namespace Poco { class Logger; }
#include <Dictionaries/IDictionarySource.h>
#include <Dictionaries/DictionaryStructure.h>
#include <DataStreams/ShellCommandSource.h>
namespace DB
{
using ProcessPool = BorrowedObjectPool<std::unique_ptr<ShellCommand>>;
/** ExecutablePoolDictionarySource allows loading data from pool of processes.
* When client requests ids or keys source get process from ProcessPool
@ -73,14 +71,13 @@ public:
Pipe getStreamForBlock(const Block & block);
private:
Poco::Logger * log;
time_t update_time = 0;
const DictionaryStructure dict_struct;
const Configuration configuration;
Block sample_block;
ContextPtr context;
std::shared_ptr<ProcessPool> process_pool;
Poco::Logger * log;
};
}

View File

@ -104,9 +104,11 @@ public:
const auto * attr_name_col = checkAndGetColumnConst<ColumnString>(sample_columns.getByPosition(1).column.get());
if (!attr_name_col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument of function dictGet... must be a constant string");
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument of function dictGet must be a constant string");
return getDictionary(dict_name_col->getValue<String>())->isInjective(attr_name_col->getValue<String>());
const auto dictionary_name = dict_name_col->getValue<String>();
const auto attribute_name = attr_name_col->getValue<String>();
return getDictionary(dictionary_name)->isInjective(attribute_name);
}
DictionaryStructure getDictionaryStructure(const String & dictionary_name) const

View File

@ -41,7 +41,7 @@ public:
return name;
}
size_t getNumberOfArguments() const override { return 4; }
size_t getNumberOfArguments() const override { return 3; }
bool useDefaultImplementationForConstants() const override { return true; }

View File

@ -41,7 +41,7 @@ public:
return name;
}
size_t getNumberOfArguments() const override { return 4; }
size_t getNumberOfArguments() const override { return 3; }
bool useDefaultImplementationForConstants() const override { return true; }

View File

@ -32,7 +32,7 @@ WriteBufferFromFile::WriteBufferFromFile(
mode_t mode,
char * existing_memory,
size_t alignment)
: WriteBufferFromFileDescriptor(-1, buf_size, existing_memory, alignment), file_name(file_name_)
: WriteBufferFromFileDescriptor(-1, buf_size, existing_memory, alignment, file_name_)
{
ProfileEvents::increment(ProfileEvents::FileOpen);
@ -65,9 +65,7 @@ WriteBufferFromFile::WriteBufferFromFile(
size_t buf_size,
char * existing_memory,
size_t alignment)
:
WriteBufferFromFileDescriptor(fd_, buf_size, existing_memory, alignment),
file_name(original_file_name.empty() ? "(fd = " + toString(fd_) + ")" : original_file_name)
: WriteBufferFromFileDescriptor(fd_, buf_size, existing_memory, alignment, original_file_name)
{
fd_ = -1;
}

View File

@ -25,7 +25,6 @@ namespace DB
class WriteBufferFromFile : public WriteBufferFromFileDescriptor
{
protected:
std::string file_name;
CurrentMetrics::Increment metric_increment{CurrentMetrics::OpenFileForWrite};
public:

View File

@ -61,7 +61,12 @@ void WriteBufferFromFileDescriptor::nextImpl()
if ((-1 == res || 0 == res) && errno != EINTR)
{
ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWriteFailed);
throwFromErrnoWithPath("Cannot write to file " + getFileName(), getFileName(),
/// Don't use getFileName() here because this method can be called from destructor
String error_file_name = file_name;
if (error_file_name.empty())
error_file_name = "(fd = " + toString(fd) + ")";
throwFromErrnoWithPath("Cannot write to file " + error_file_name, error_file_name,
ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR);
}
@ -73,20 +78,20 @@ void WriteBufferFromFileDescriptor::nextImpl()
ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWriteBytes, bytes_written);
}
/// Name or some description of file.
std::string WriteBufferFromFileDescriptor::getFileName() const
{
return "(fd = " + toString(fd) + ")";
}
/// NOTE: This class can be used as a very low-level building block, for example
/// in trace collector. In such places allocations of memory can be dangerous,
/// so don't allocate anything in this constructor.
WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor(
int fd_,
size_t buf_size,
char * existing_memory,
size_t alignment)
: WriteBufferFromFileBase(buf_size, existing_memory, alignment), fd(fd_) {}
size_t alignment,
std::string file_name_)
: WriteBufferFromFileBase(buf_size, existing_memory, alignment)
, fd(fd_)
, file_name(std::move(file_name_))
{
}
WriteBufferFromFileDescriptor::~WriteBufferFromFileDescriptor()
@ -115,7 +120,7 @@ void WriteBufferFromFileDescriptor::sync()
}
off_t WriteBufferFromFileDescriptor::seek(off_t offset, int whence)
off_t WriteBufferFromFileDescriptor::seek(off_t offset, int whence) // NOLINT
{
off_t res = lseek(fd, offset, whence);
if (-1 == res)
@ -125,7 +130,7 @@ off_t WriteBufferFromFileDescriptor::seek(off_t offset, int whence)
}
void WriteBufferFromFileDescriptor::truncate(off_t length)
void WriteBufferFromFileDescriptor::truncate(off_t length) // NOLINT
{
int res = ftruncate(fd, length);
if (-1 == res)
@ -133,7 +138,7 @@ void WriteBufferFromFileDescriptor::truncate(off_t length)
}
off_t WriteBufferFromFileDescriptor::size()
off_t WriteBufferFromFileDescriptor::size() const
{
struct stat buf;
int res = fstat(fd, &buf);
@ -142,4 +147,13 @@ off_t WriteBufferFromFileDescriptor::size()
return buf.st_size;
}
std::string WriteBufferFromFileDescriptor::getFileName() const
{
if (file_name.empty())
return "(fd = " + toString(fd) + ")";
return file_name;
}
}

View File

@ -13,17 +13,17 @@ class WriteBufferFromFileDescriptor : public WriteBufferFromFileBase
protected:
int fd;
/// If file has name contains filename, otherwise contains string "(fd=...)"
std::string file_name;
void nextImpl() override;
/// Name or some description of file.
std::string getFileName() const override;
public:
WriteBufferFromFileDescriptor(
int fd_ = -1,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,
size_t alignment = 0);
size_t alignment = 0,
std::string file_name_ = "");
/** Could be used before initialization if needed 'fd' was not passed to constructor.
* It's not possible to change 'fd' during work.
@ -42,10 +42,15 @@ public:
void sync() override;
off_t seek(off_t offset, int whence);
void truncate(off_t length);
/// clang-tidy wants these methods to be const, but
/// they are not const semantically
off_t seek(off_t offset, int whence); // NOLINT
void truncate(off_t length); // NOLINT
off_t size();
/// Name or some description of file.
std::string getFileName() const override;
off_t size() const;
};
}

View File

@ -28,41 +28,49 @@ ZstdInflatingReadBuffer::~ZstdInflatingReadBuffer()
bool ZstdInflatingReadBuffer::nextImpl()
{
if (eof)
return false;
if (input.pos >= input.size)
do
{
in->nextIfAtEnd();
input.src = reinterpret_cast<unsigned char *>(in->position());
input.pos = 0;
input.size = in->buffer().end() - in->position();
}
// If it is known that end of file was reached, return false
if (eof)
return false;
output.dst = reinterpret_cast<unsigned char *>(internal_buffer.begin());
output.size = internal_buffer.size();
output.pos = 0;
/// If end was reached, get next part
if (input.pos >= input.size)
{
in->nextIfAtEnd();
input.src = reinterpret_cast<unsigned char *>(in->position());
input.pos = 0;
input.size = in->buffer().end() - in->position();
}
size_t ret = ZSTD_decompressStream(dctx, &output, &input);
if (ZSTD_isError(ret))
throw Exception(
ErrorCodes::ZSTD_DECODER_FAILED, "Zstd stream decoding failed: error code: {}; zstd version: {}", ret, ZSTD_VERSION_STRING);
/// fill output
output.dst = reinterpret_cast<unsigned char *>(internal_buffer.begin());
output.size = internal_buffer.size();
output.pos = 0;
in->position() = in->buffer().begin() + input.pos;
working_buffer.resize(output.pos);
/// Decompress data and check errors.
size_t ret = ZSTD_decompressStream(dctx, &output, &input);
if (ZSTD_isError(ret))
throw Exception(
ErrorCodes::ZSTD_DECODER_FAILED, "Zstd stream decoding failed: error code: {}; zstd version: {}", ret, ZSTD_VERSION_STRING);
if (in->eof())
{
eof = true;
return !working_buffer.empty();
}
else if (output.pos == 0)
{
/// Check that something has changed after decompress (input or output position)
assert(output.pos > 0 || in->position() < in->buffer().begin() + input.pos);
/// move position to the end of read data
in->position() = in->buffer().begin() + input.pos;
working_buffer.resize(output.pos);
/// If end of file is reached, fill eof variable and return true if there is some data in buffer, otherwise return false
if (in->eof())
{
eof = true;
return !working_buffer.empty();
}
/// It is possible, that input buffer is not at eof yet, but nothing was decompressed in current iteration.
/// But there are cases, when such behaviour is not allowed - i.e. if input buffer is not eof, then
/// it has to be guaranteed that working_buffer is not empty. So if it is empty, continue.
return nextImpl();
}
} while (output.pos == 0);
return true;
}

View File

@ -5,6 +5,7 @@ LIBRARY()
ADDINCL(
contrib/libs/zstd/include
contrib/libs/lz4
contrib/restricted/fast_float/include
)
@ -14,10 +15,10 @@ PEERDIR(
contrib/libs/brotli/enc
contrib/libs/poco/NetSSL_OpenSSL
contrib/libs/zstd
contrib/libs/lz4
contrib/restricted/fast_float
)
SRCS(
AIO.cpp
AsynchronousReadBufferFromFile.cpp

View File

@ -4,6 +4,7 @@ LIBRARY()
ADDINCL(
contrib/libs/zstd/include
contrib/libs/lz4
contrib/restricted/fast_float/include
)
@ -13,10 +14,10 @@ PEERDIR(
contrib/libs/brotli/enc
contrib/libs/poco/NetSSL_OpenSSL
contrib/libs/zstd
contrib/libs/lz4
contrib/restricted/fast_float
)
SRCS(
<? find . -name '*.cpp' | grep -v -F tests | grep -v -F examples | grep -v -F fuzzers | grep -v -P 'S3|HDFS' | sed 's/^\.\// /' | sort ?>
)

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

@ -57,6 +57,7 @@
#include <Interpreters/Cluster.h>
#include <Interpreters/InterserverIOHandler.h>
#include <Interpreters/SystemLog.h>
#include <Interpreters/SessionLog.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/DDLTask.h>
@ -77,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>
@ -100,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
@ -222,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
@ -291,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);
@ -641,7 +661,6 @@ ConfigurationPtr Context::getUsersConfig()
return shared->users_config;
}
void Context::setUser(const UUID & user_id_)
{
auto lock = getLock();
@ -2071,6 +2090,16 @@ std::shared_ptr<OpenTelemetrySpanLog> Context::getOpenTelemetrySpanLog() const
return shared->system_logs->opentelemetry_span_log;
}
std::shared_ptr<SessionLog> Context::getSessionLog() const
{
auto lock = getLock();
if (!shared->system_logs)
return {};
return shared->system_logs->session_log;
}
std::shared_ptr<ZooKeeperLog> Context::getZooKeeperLog() const
{
@ -2708,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

@ -74,6 +74,7 @@ class MetricLog;
class AsynchronousMetricLog;
class OpenTelemetrySpanLog;
class ZooKeeperLog;
class SessionLog;
struct MergeTreeSettings;
class StorageS3Settings;
class IDatabase;
@ -100,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;
@ -600,6 +603,7 @@ public:
bool hasSessionContext() const { return !session_context.expired(); }
ContextMutablePtr getGlobalContext() const;
bool hasGlobalContext() const { return !global_context.expired(); }
bool isGlobalContext() const
{
@ -735,6 +739,7 @@ public:
std::shared_ptr<AsynchronousMetricLog> getAsynchronousMetricLog() const;
std::shared_ptr<OpenTelemetrySpanLog> getOpenTelemetrySpanLog() const;
std::shared_ptr<ZooKeeperLog> getZooKeeperLog() const;
std::shared_ptr<SessionLog> getSessionLog() const;
/// Returns an object used to log operations with parts if it possible.
/// Provide table name to make required checks.
@ -827,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

@ -146,7 +146,6 @@ void DatabaseCatalog::initializeAndLoadTemporaryDatabase()
void DatabaseCatalog::loadDatabases()
{
loadMarkedAsDroppedTables();
auto task_holder = getContext()->getSchedulePool().createTask("DatabaseCatalog", [this](){ this->dropTableDataTask(); });
drop_task = std::make_unique<BackgroundSchedulePoolTaskHolder>(std::move(task_holder));
(*drop_task)->activate();
@ -618,12 +617,6 @@ Dependencies DatabaseCatalog::getDependencies(const StorageID & from) const
return Dependencies(iter->second.begin(), iter->second.end());
}
ViewDependencies DatabaseCatalog::getViewDependencies() const
{
std::lock_guard lock{databases_mutex};
return ViewDependencies(view_dependencies.begin(), view_dependencies.end());
}
void
DatabaseCatalog::updateDependency(const StorageID & old_from, const StorageID & old_where, const StorageID & new_from,
const StorageID & new_where)

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();
@ -130,6 +132,7 @@ public:
void initializeAndLoadTemporaryDatabase();
void loadDatabases();
void loadMarkedAsDroppedTables();
/// Get an object that protects the table from concurrently executing multiple DDL operations.
DDLGuardPtr getDDLGuard(const String & database, const String & table);
@ -174,7 +177,6 @@ public:
void addDependency(const StorageID & from, const StorageID & where);
void removeDependency(const StorageID & from, const StorageID & where);
Dependencies getDependencies(const StorageID & from) const;
ViewDependencies getViewDependencies() const;
/// For Materialized and Live View
void updateDependency(const StorageID & old_from, const StorageID & old_where,const StorageID & new_from, const StorageID & new_where);
@ -241,7 +243,6 @@ private:
};
using TablesMarkedAsDropped = std::list<TableMarkedAsDropped>;
void loadMarkedAsDroppedTables();
void dropTableDataTask();
void dropTableFinally(const TableMarkedAsDropped & table);

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);

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