Merge remote-tracking branch 'qoega/better-hdfs-tests' into remove-enable-dictionaries

This commit is contained in:
Yatsishin Ilya 2021-06-09 17:08:02 +03:00
commit 0c2380e4ac
253 changed files with 10732 additions and 2104 deletions

5
.gitignore vendored
View File

@ -14,6 +14,11 @@
/build-*
/tests/venv
# logs
*.log
*.stderr
*.stdout
/docs/build
/docs/publish
/docs/edit

1
.gitmodules vendored
View File

@ -228,7 +228,6 @@
[submodule "contrib/datasketches-cpp"]
path = contrib/datasketches-cpp
url = https://github.com/ClickHouse-Extras/datasketches-cpp.git
[submodule "contrib/yaml-cpp"]
path = contrib/yaml-cpp
url = https://github.com/ClickHouse-Extras/yaml-cpp.git

View File

@ -1,3 +1,130 @@
### ClickHouse release 21.6, 2021-06-05
#### Upgrade Notes
* Do not upgrade if you have partition key with `UUID`.
* `zstd` compression library is updated to v1.5.0. You may get messages about "checksum does not match" in replication. These messages are expected due to update of compression algorithm and you can ignore them. These messages are informational and do not indicate any kinds of undesired behaviour.
* The setting `compile_expressions` is enabled by default. Although it has been heavily tested on variety of scenarios, if you find some undesired behaviour on your servers, you can try turning this setting off.
* Values of `UUID` type cannot be compared with integer. For example, instead of writing `uuid != 0` type `uuid != '00000000-0000-0000-0000-000000000000'`.
#### New Feature
* Add Postgres-like cast operator (`::`). E.g.: `[1, 2]::Array(UInt8)`, `0.1::Decimal(4, 4)`, `number::UInt16`. [#23871](https://github.com/ClickHouse/ClickHouse/pull/23871) ([Anton Popov](https://github.com/CurtizJ)).
* Make big integers production ready. Add support for `UInt128` data type. Fix known issues with the `Decimal256` data type. Support big integers in dictionaries. Support `gcd`/`lcm` functions for big integers. Support big integers in array search and conditional functions. Support `LowCardinality(UUID)`. Support big integers in `generateRandom` table function and `clickhouse-obfuscator`. Fix error with returning `UUID` from scalar subqueries. This fixes [#7834](https://github.com/ClickHouse/ClickHouse/issues/7834). This fixes [#23936](https://github.com/ClickHouse/ClickHouse/issues/23936). This fixes [#4176](https://github.com/ClickHouse/ClickHouse/issues/4176). This fixes [#24018](https://github.com/ClickHouse/ClickHouse/issues/24018). Backward incompatible change: values of `UUID` type cannot be compared with integer. For example, instead of writing `uuid != 0` type `uuid != '00000000-0000-0000-0000-000000000000'`. [#23631](https://github.com/ClickHouse/ClickHouse/pull/23631) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Support `Array` data type for inserting and selecting data in `Arrow`, `Parquet` and `ORC` formats. [#21770](https://github.com/ClickHouse/ClickHouse/pull/21770) ([taylor12805](https://github.com/taylor12805)).
* Implement table comments. Closes [#23225](https://github.com/ClickHouse/ClickHouse/issues/23225). [#23548](https://github.com/ClickHouse/ClickHouse/pull/23548) ([flynn](https://github.com/ucasFL)).
* Support creating dictionaries with DDL queries in `clickhouse-local`. Closes [#22354](https://github.com/ClickHouse/ClickHouse/issues/22354). Added support for `DETACH DICTIONARY PERMANENTLY`. Added support for `EXCHANGE DICTIONARIES` for `Atomic` database engine. Added support for moving dictionaries between databases using `RENAME DICTIONARY`. [#23436](https://github.com/ClickHouse/ClickHouse/pull/23436) ([Maksim Kita](https://github.com/kitaisreal)).
* Add aggregate function `uniqTheta` to support [Theta Sketch](https://datasketches.apache.org/docs/Theta/ThetaSketchFramework.html) in ClickHouse. [#23894](https://github.com/ClickHouse/ClickHouse/pull/23894). [#22609](https://github.com/ClickHouse/ClickHouse/pull/22609) ([Ping Yu](https://github.com/pingyu)).
* Add function `splitByRegexp`. [#24077](https://github.com/ClickHouse/ClickHouse/pull/24077) ([abel-cheng](https://github.com/abel-cheng)).
* Add function `arrayProduct` which accept an array as the parameter, and return the product of all the elements in array. Closes [#21613](https://github.com/ClickHouse/ClickHouse/issues/21613). [#23782](https://github.com/ClickHouse/ClickHouse/pull/23782) ([Maksim Kita](https://github.com/kitaisreal)).
* Add `thread_name` column in `system.stack_trace`. This closes [#23256](https://github.com/ClickHouse/ClickHouse/issues/23256). [#24124](https://github.com/ClickHouse/ClickHouse/pull/24124) ([abel-cheng](https://github.com/abel-cheng)).
* If `insert_null_as_default` = 1, insert default values instead of NULL in `INSERT ... SELECT` and `INSERT ... SELECT ... UNION ALL ...` queries. Closes [#22832](https://github.com/ClickHouse/ClickHouse/issues/22832). [#23524](https://github.com/ClickHouse/ClickHouse/pull/23524) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add support for progress indication in `clickhouse-local` with `--progress` option. [#23196](https://github.com/ClickHouse/ClickHouse/pull/23196) ([Egor Savin](https://github.com/Amesaru)).
* Add support for HTTP compression (determined by `Content-Encoding` HTTP header) in `http` dictionary source. This fixes [#8912](https://github.com/ClickHouse/ClickHouse/issues/8912). [#23946](https://github.com/ClickHouse/ClickHouse/pull/23946) ([FArthur-cmd](https://github.com/FArthur-cmd)).
* Added `SYSTEM QUERY RELOAD MODEL`, `SYSTEM QUERY RELOAD MODELS`. Closes [#18722](https://github.com/ClickHouse/ClickHouse/issues/18722). [#23182](https://github.com/ClickHouse/ClickHouse/pull/23182) ([Maksim Kita](https://github.com/kitaisreal)).
* Add setting `json` (boolean, 0 by default) for `EXPLAIN PLAN` query. When enabled, query output will be a single `JSON` row. It is recommended to use `TSVRaw` format to avoid unnecessary escaping. [#23082](https://github.com/ClickHouse/ClickHouse/pull/23082) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Add setting `indexes` (boolean, disabled by default) to `EXPLAIN PIPELINE` query. When enabled, shows used indexes, number of filtered parts and granules for every index applied. Supported for `MergeTree*` tables. [#22352](https://github.com/ClickHouse/ClickHouse/pull/22352) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* LDAP: implemented user DN detection functionality to use when mapping Active Directory groups to ClickHouse roles. [#22228](https://github.com/ClickHouse/ClickHouse/pull/22228) ([Denis Glazachev](https://github.com/traceon)).
* New aggregate function `deltaSumTimestamp` for summing the difference between consecutive rows while maintaining ordering during merge by storing timestamps. [#21888](https://github.com/ClickHouse/ClickHouse/pull/21888) ([Russ Frank](https://github.com/rf)).
* Added less secure IMDS credentials provider for S3 which works under docker correctly. [#21852](https://github.com/ClickHouse/ClickHouse/pull/21852) ([Vladimir Chebotarev](https://github.com/excitoon)).
* Add back `indexHint` function. This is for [#21238](https://github.com/ClickHouse/ClickHouse/issues/21238). This reverts [#9542](https://github.com/ClickHouse/ClickHouse/pull/9542). This fixes [#9540](https://github.com/ClickHouse/ClickHouse/issues/9540). [#21304](https://github.com/ClickHouse/ClickHouse/pull/21304) ([Amos Bird](https://github.com/amosbird)).
#### Experimental Feature
* Add `PROJECTION` support for `MergeTree*` tables. [#20202](https://github.com/ClickHouse/ClickHouse/pull/20202) ([Amos Bird](https://github.com/amosbird)).
#### Performance Improvement
* Enable `compile_expressions` setting by default. When this setting enabled, compositions of simple functions and operators will be compiled to native code with LLVM at runtime. [#8482](https://github.com/ClickHouse/ClickHouse/pull/8482) ([Maksim Kita](https://github.com/kitaisreal), [alexey-milovidov](https://github.com/alexey-milovidov)). Note: if you feel in trouble, turn this option off.
* Update `re2` library. Performance of regular expressions matching is improved. Also this PR adds compatibility with gcc-11. [#24196](https://github.com/ClickHouse/ClickHouse/pull/24196) ([Raúl Marín](https://github.com/Algunenano)).
* ORC input format reading by stripe instead of reading entire table into memory by once which is cost memory when file size is huge. [#23102](https://github.com/ClickHouse/ClickHouse/pull/23102) ([Chao Ma](https://github.com/godliness)).
* Fusion of aggregate functions `sum`, `count` and `avg` in a query into single aggregate function. The optimization is controlled with the `optimize_fuse_sum_count_avg` setting. This is implemented with a new aggregate function `sumCount`. This function returns a tuple of two fields: `sum` and `count`. [#21337](https://github.com/ClickHouse/ClickHouse/pull/21337) ([hexiaoting](https://github.com/hexiaoting)).
* Update `zstd` to v1.5.0. The performance of compression is improved for single digits percentage. [#24135](https://github.com/ClickHouse/ClickHouse/pull/24135) ([Raúl Marín](https://github.com/Algunenano)). Note: you may get messages about "checksum does not match" in replication. These messages are expected due to update of compression algorithm and you can ignore them.
* Improved performance of `Buffer` tables: do not acquire lock for total_bytes/total_rows for `Buffer` engine. [#24066](https://github.com/ClickHouse/ClickHouse/pull/24066) ([Azat Khuzhin](https://github.com/azat)).
* Preallocate support for hashed/sparse_hashed dictionaries is returned. [#23979](https://github.com/ClickHouse/ClickHouse/pull/23979) ([Azat Khuzhin](https://github.com/azat)).
* Enable `async_socket_for_remote` by default (lower amount of threads in querying Distributed tables with large fanout). [#23683](https://github.com/ClickHouse/ClickHouse/pull/23683) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
#### Improvement
* Add `_partition_value` virtual column to MergeTree table family. It can be used to prune partition in a deterministic way. It's needed to implement partition matcher for mutations. [#23673](https://github.com/ClickHouse/ClickHouse/pull/23673) ([Amos Bird](https://github.com/amosbird)).
* Added `region` parameter for S3 storage and disk. [#23846](https://github.com/ClickHouse/ClickHouse/pull/23846) ([Vladimir Chebotarev](https://github.com/excitoon)).
* Allow configuring different log levels for different logging channels. Closes [#19569](https://github.com/ClickHouse/ClickHouse/issues/19569). [#23857](https://github.com/ClickHouse/ClickHouse/pull/23857) ([filimonov](https://github.com/filimonov)).
* Keep default timezone on `DateTime` operations if it was not provided explicitly. For example, if you add one second to a value of `DateTime` type without timezone it will remain `DateTime` without timezone. In previous versions the value of default timezone was placed to the returned data type explicitly so it becomes DateTime('something'). This closes [#4854](https://github.com/ClickHouse/ClickHouse/issues/4854). [#23392](https://github.com/ClickHouse/ClickHouse/pull/23392) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Allow user to specify empty string instead of database name for `MySQL` storage. Default database will be used for queries. In previous versions it was working for SELECT queries and not support for INSERT was also added. This closes [#19281](https://github.com/ClickHouse/ClickHouse/issues/19281). This can be useful working with `Sphinx` or other MySQL-compatible foreign databases. [#23319](https://github.com/ClickHouse/ClickHouse/pull/23319) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Fixed `quantile(s)TDigest`. Added special handling of singleton centroids according to tdunning/t-digest 3.2+. Also a bug with over-compression of centroids in implementation of earlier version of the algorithm was fixed. [#23314](https://github.com/ClickHouse/ClickHouse/pull/23314) ([Vladimir Chebotarev](https://github.com/excitoon)).
* Function `now64` now supports optional timezone argument. [#24091](https://github.com/ClickHouse/ClickHouse/pull/24091) ([Vasily Nemkov](https://github.com/Enmk)).
* Fix the case when a progress bar in interactive mode in clickhouse-client that appear in the middle of the data may rewrite some parts of visible data in terminal. This closes [#19283](https://github.com/ClickHouse/ClickHouse/issues/19283). [#23050](https://github.com/ClickHouse/ClickHouse/pull/23050) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Fix crash when memory allocation fails in simdjson. https://github.com/simdjson/simdjson/pull/1567 . Mark as improvement because it's a very rare bug. [#24147](https://github.com/ClickHouse/ClickHouse/pull/24147) ([Amos Bird](https://github.com/amosbird)).
* Preserve dictionaries until storage shutdown (this will avoid possible `external dictionary 'DICT' not found` errors at server shutdown during final flush of the `Buffer` engine). [#24068](https://github.com/ClickHouse/ClickHouse/pull/24068) ([Azat Khuzhin](https://github.com/azat)).
* Flush `Buffer` tables before shutting down tables (within one database), to avoid discarding blocks due to underlying table had been already detached (and `Destination table default.a_data_01870 doesn't exist. Block of data is discarded` error in the log). [#24067](https://github.com/ClickHouse/ClickHouse/pull/24067) ([Azat Khuzhin](https://github.com/azat)).
* Now `prefer_column_name_to_alias = 1` will also favor column names for `group by`, `having` and `order by`. This fixes [#23882](https://github.com/ClickHouse/ClickHouse/issues/23882). [#24022](https://github.com/ClickHouse/ClickHouse/pull/24022) ([Amos Bird](https://github.com/amosbird)).
* Add support for `ORDER BY WITH FILL` with `DateTime64`. [#24016](https://github.com/ClickHouse/ClickHouse/pull/24016) ([kevin wan](https://github.com/MaxWk)).
* Enable `DateTime64` to be a version column in `ReplacingMergeTree`. [#23992](https://github.com/ClickHouse/ClickHouse/pull/23992) ([kevin wan](https://github.com/MaxWk)).
* Log information about OS name, kernel version and CPU architecture on server startup. [#23988](https://github.com/ClickHouse/ClickHouse/pull/23988) ([Azat Khuzhin](https://github.com/azat)).
* Support specifying table schema for `postgresql` dictionary source. Closes [#23958](https://github.com/ClickHouse/ClickHouse/issues/23958). [#23980](https://github.com/ClickHouse/ClickHouse/pull/23980) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add hints for names of `Enum` elements (suggest names in case of typos). Closes [#17112](https://github.com/ClickHouse/ClickHouse/issues/17112). [#23919](https://github.com/ClickHouse/ClickHouse/pull/23919) ([flynn](https://github.com/ucasFL)).
* Measure found rate (the percentage for which the value was found) for dictionaries (see `found_rate` in `system.dictionaries`). [#23916](https://github.com/ClickHouse/ClickHouse/pull/23916) ([Azat Khuzhin](https://github.com/azat)).
* Allow to add specific queue settings via table settng `rabbitmq_queue_settings_list`. (Closes [#23737](https://github.com/ClickHouse/ClickHouse/issues/23737) and [#23918](https://github.com/ClickHouse/ClickHouse/issues/23918)). Allow user to control all RabbitMQ setup: if table setting `rabbitmq_queue_consume` is set to `1` - RabbitMQ table engine will only connect to specified queue and will not perform any RabbitMQ consumer-side setup like declaring exchange, queues, bindings. (Closes [#21757](https://github.com/ClickHouse/ClickHouse/issues/21757)). Add proper cleanup when RabbitMQ table is dropped - delete queues, which the table has declared and all bound exchanges - if they were created by the table. [#23887](https://github.com/ClickHouse/ClickHouse/pull/23887) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add `broken_data_files`/`broken_data_compressed_bytes` into `system.distribution_queue`. Add metric for number of files for asynchronous insertion into Distributed tables that has been marked as broken (`BrokenDistributedFilesToInsert`). [#23885](https://github.com/ClickHouse/ClickHouse/pull/23885) ([Azat Khuzhin](https://github.com/azat)).
* Querying `system.tables` does not go to ZooKeeper anymore. [#23793](https://github.com/ClickHouse/ClickHouse/pull/23793) ([Fuwang Hu](https://github.com/fuwhu)).
* Respect `lock_acquire_timeout_for_background_operations` for `OPTIMIZE` queries. [#23623](https://github.com/ClickHouse/ClickHouse/pull/23623) ([Azat Khuzhin](https://github.com/azat)).
* Possibility to change `S3` disk settings in runtime via new `SYSTEM RESTART DISK` SQL command. [#23429](https://github.com/ClickHouse/ClickHouse/pull/23429) ([Pavel Kovalenko](https://github.com/Jokser)).
* If user applied a misconfiguration by mistakenly setting `max_distributed_connections` to value zero, every query to a `Distributed` table will throw exception with a message containing "logical error". But it's really an expected behaviour, not a logical error, so the exception message was slightly incorrect. It also triggered checks in our CI enviroment that ensures that no logical errors ever happen. Instead we will treat `max_distributed_connections` misconfigured to zero as the minimum possible value (one). [#23348](https://github.com/ClickHouse/ClickHouse/pull/23348) ([Azat Khuzhin](https://github.com/azat)).
* Disable `min_bytes_to_use_mmap_io` by default. [#23322](https://github.com/ClickHouse/ClickHouse/pull/23322) ([Azat Khuzhin](https://github.com/azat)).
* Support `LowCardinality` nullability with `join_use_nulls`, close [#15101](https://github.com/ClickHouse/ClickHouse/issues/15101). [#23237](https://github.com/ClickHouse/ClickHouse/pull/23237) ([vdimir](https://github.com/vdimir)).
* Added possibility to restore `MergeTree` parts to `detached` directory for `S3` disk. [#23112](https://github.com/ClickHouse/ClickHouse/pull/23112) ([Pavel Kovalenko](https://github.com/Jokser)).
* Retries on HTTP connection drops in S3. [#22988](https://github.com/ClickHouse/ClickHouse/pull/22988) ([Vladimir Chebotarev](https://github.com/excitoon)).
* Add settings `external_storage_max_read_rows` and `external_storage_max_read_rows` for MySQL table engine, dictionary source and MaterializeMySQL minor data fetches. [#22697](https://github.com/ClickHouse/ClickHouse/pull/22697) ([TCeason](https://github.com/TCeason)).
* `MaterializeMySQL` (experimental feature): Previously, MySQL 5.7.9 was not supported due to SQL incompatibility. Now leave MySQL parameter verification to the MaterializeMySQL. [#23413](https://github.com/ClickHouse/ClickHouse/pull/23413) ([TCeason](https://github.com/TCeason)).
* Enable reading of subcolumns for distributed tables. [#24472](https://github.com/ClickHouse/ClickHouse/pull/24472) ([Anton Popov](https://github.com/CurtizJ)).
* Fix usage of tuples in `CREATE .. AS SELECT` queries. [#24464](https://github.com/ClickHouse/ClickHouse/pull/24464) ([Anton Popov](https://github.com/CurtizJ)).
* Support for `Parquet` format in `Kafka` tables. [#23412](https://github.com/ClickHouse/ClickHouse/pull/23412) ([Chao Ma](https://github.com/godliness)).
#### Bug Fix
* Use old modulo function version when used in partition key and primary key. Closes [#23508](https://github.com/ClickHouse/ClickHouse/issues/23508). [#24157](https://github.com/ClickHouse/ClickHouse/pull/24157) ([Kseniia Sumarokova](https://github.com/kssenii)). It was a source of backward incompatibility in previous releases.
* Fixed the behavior when query `SYSTEM RESTART REPLICA` or `SYSTEM SYNC REPLICA` is being processed infinitely. This was detected on server with extremely little amount of RAM. [#24457](https://github.com/ClickHouse/ClickHouse/pull/24457) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Fix incorrect monotonicity of `toWeek` function. This fixes [#24422](https://github.com/ClickHouse/ClickHouse/issues/24422) . This bug was introduced in [#5212](https://github.com/ClickHouse/ClickHouse/pull/5212), and was exposed later by smarter partition pruner. [#24446](https://github.com/ClickHouse/ClickHouse/pull/24446) ([Amos Bird](https://github.com/amosbird)).
* Fix drop partition with intersect fake parts. In rare cases there might be parts with mutation version greater than current block number. [#24321](https://github.com/ClickHouse/ClickHouse/pull/24321) ([Amos Bird](https://github.com/amosbird)).
* Fixed a bug in moving Materialized View from Ordinary to Atomic database (`RENAME TABLE` query). Now inner table is moved to new database together with Materialized View. Fixes [#23926](https://github.com/ClickHouse/ClickHouse/issues/23926). [#24309](https://github.com/ClickHouse/ClickHouse/pull/24309) ([tavplubix](https://github.com/tavplubix)).
* Allow empty HTTP headers in client requests. Fixes [#23901](https://github.com/ClickHouse/ClickHouse/issues/23901). [#24285](https://github.com/ClickHouse/ClickHouse/pull/24285) ([Ivan](https://github.com/abyss7)).
* Set `max_threads = 1` to fix mutation fail of `Memory` tables. Closes [#24274](https://github.com/ClickHouse/ClickHouse/issues/24274). [#24275](https://github.com/ClickHouse/ClickHouse/pull/24275) ([flynn](https://github.com/ucasFL)).
* Fix typo in implementation of `Memory` tables, this bug was introduced at [#15127](https://github.com/ClickHouse/ClickHouse/issues/15127). Closes [#24192](https://github.com/ClickHouse/ClickHouse/issues/24192). [#24193](https://github.com/ClickHouse/ClickHouse/pull/24193) ([张中南](https://github.com/plugine)).
* Fix abnormal server termination due to `HDFS` becoming not accessible during query execution. Closes [#24117](https://github.com/ClickHouse/ClickHouse/issues/24117). [#24191](https://github.com/ClickHouse/ClickHouse/pull/24191) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix crash on updating of `Nested` column with const condition. [#24183](https://github.com/ClickHouse/ClickHouse/pull/24183) ([hexiaoting](https://github.com/hexiaoting)).
* Fix race condition which could happen in RBAC under a heavy load. This PR fixes [#24090](https://github.com/ClickHouse/ClickHouse/issues/24090), [#24134](https://github.com/ClickHouse/ClickHouse/issues/24134),. [#24176](https://github.com/ClickHouse/ClickHouse/pull/24176) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fix a rare bug that could lead to a partially initialized table that can serve write requests (insert/alter/so on). Now such tables will be in readonly mode. [#24122](https://github.com/ClickHouse/ClickHouse/pull/24122) ([alesapin](https://github.com/alesapin)).
* Fix an issue: `EXPLAIN PIPELINE` with `SELECT xxx FINAL` showed a wrong pipeline. ([hexiaoting](https://github.com/hexiaoting)).
* Fixed using const `DateTime` value vs `DateTime64` column in `WHERE`. [#24100](https://github.com/ClickHouse/ClickHouse/pull/24100) ([Vasily Nemkov](https://github.com/Enmk)).
* Fix crash in merge JOIN, closes [#24010](https://github.com/ClickHouse/ClickHouse/issues/24010). [#24013](https://github.com/ClickHouse/ClickHouse/pull/24013) ([vdimir](https://github.com/vdimir)).
* Some `ALTER PARTITION` queries might cause `Part A intersects previous part B` and `Unexpected merged part C intersecting drop range D` errors in replication queue. It's fixed. Fixes [#23296](https://github.com/ClickHouse/ClickHouse/issues/23296). [#23997](https://github.com/ClickHouse/ClickHouse/pull/23997) ([tavplubix](https://github.com/tavplubix)).
* Fix SIGSEGV for external GROUP BY and overflow row (i.e. queries like `SELECT FROM GROUP BY WITH TOTALS SETTINGS max_bytes_before_external_group_by>0, max_rows_to_group_by>0, group_by_overflow_mode='any', totals_mode='before_having'`). [#23962](https://github.com/ClickHouse/ClickHouse/pull/23962) ([Azat Khuzhin](https://github.com/azat)).
* Fix keys metrics accounting for `CACHE` dictionary with duplicates in the source (leads to `DictCacheKeysRequestedMiss` overflows). [#23929](https://github.com/ClickHouse/ClickHouse/pull/23929) ([Azat Khuzhin](https://github.com/azat)).
* Fix implementation of connection pool of `PostgreSQL` engine. Closes [#23897](https://github.com/ClickHouse/ClickHouse/issues/23897). [#23909](https://github.com/ClickHouse/ClickHouse/pull/23909) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix `distributed_group_by_no_merge = 2` with `GROUP BY` and aggregate function wrapped into regular function (had been broken in [#23546](https://github.com/ClickHouse/ClickHouse/issues/23546)). Throw exception in case of someone trying to use `distributed_group_by_no_merge = 2` with window functions. Disable `optimize_distributed_group_by_sharding_key` for queries with window functions. [#23906](https://github.com/ClickHouse/ClickHouse/pull/23906) ([Azat Khuzhin](https://github.com/azat)).
* A fix for `s3` table function: better handling of HTTP errors. Response bodies of HTTP errors were being ignored earlier. [#23844](https://github.com/ClickHouse/ClickHouse/pull/23844) ([Vladimir Chebotarev](https://github.com/excitoon)).
* A fix for `s3` table function: better handling of URI's. Fixed an incompatibility with URLs containing `+` symbol, data with such keys could not be read previously. [#23822](https://github.com/ClickHouse/ClickHouse/pull/23822) ([Vladimir Chebotarev](https://github.com/excitoon)).
* Fix error `Can't initialize pipeline with empty pipe` for queries with `GLOBAL IN/JOIN` and `use_hedged_requests`. Fixes [#23431](https://github.com/ClickHouse/ClickHouse/issues/23431). [#23805](https://github.com/ClickHouse/ClickHouse/pull/23805) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix `CLEAR COLUMN` does not work when it is referenced by materialized view. Close [#23764](https://github.com/ClickHouse/ClickHouse/issues/23764). [#23781](https://github.com/ClickHouse/ClickHouse/pull/23781) ([flynn](https://github.com/ucasFL)).
* Fix heap use after free when reading from HDFS if `Values` format is used. [#23761](https://github.com/ClickHouse/ClickHouse/pull/23761) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Avoid possible "Cannot schedule a task" error (in case some exception had been occurred) on INSERT into Distributed. [#23744](https://github.com/ClickHouse/ClickHouse/pull/23744) ([Azat Khuzhin](https://github.com/azat)).
* Fixed a bug in recovery of staled `ReplicatedMergeTree` replica. Some metadata updates could be ignored by staled replica if `ALTER` query was executed during downtime of the replica. [#23742](https://github.com/ClickHouse/ClickHouse/pull/23742) ([tavplubix](https://github.com/tavplubix)).
* Fix a bug with `Join` and `WITH TOTALS`, close [#17718](https://github.com/ClickHouse/ClickHouse/issues/17718). [#23549](https://github.com/ClickHouse/ClickHouse/pull/23549) ([vdimir](https://github.com/vdimir)).
* Fix possible `Block structure mismatch` error for queries with `UNION` which could possibly happen after filter-pushdown optimization. Fixes [#23029](https://github.com/ClickHouse/ClickHouse/issues/23029). [#23359](https://github.com/ClickHouse/ClickHouse/pull/23359) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Add type conversion when the setting `optimize_skip_unused_shards_rewrite_in` is enabled. This fixes MSan report. [#23219](https://github.com/ClickHouse/ClickHouse/pull/23219) ([Azat Khuzhin](https://github.com/azat)).
* Add a missing check when updating nested subcolumns, close issue: [#22353](https://github.com/ClickHouse/ClickHouse/issues/22353). [#22503](https://github.com/ClickHouse/ClickHouse/pull/22503) ([hexiaoting](https://github.com/hexiaoting)).
#### Build/Testing/Packaging Improvement
* Support building on Illumos. [#24144](https://github.com/ClickHouse/ClickHouse/pull/24144). Adds support for building on Solaris-derived operating systems. [#23746](https://github.com/ClickHouse/ClickHouse/pull/23746) ([bnaecker](https://github.com/bnaecker)).
* Add more benchmarks for hash tables, including the Swiss Table from Google (that appeared to be slower than ClickHouse hash map in our specific usage scenario). [#24111](https://github.com/ClickHouse/ClickHouse/pull/24111) ([Maksim Kita](https://github.com/kitaisreal)).
* Update librdkafka 1.6.0-RC3 to 1.6.1. [#23874](https://github.com/ClickHouse/ClickHouse/pull/23874) ([filimonov](https://github.com/filimonov)).
* Always enable `asynchronous-unwind-tables` explicitly. It may fix query profiler on AArch64. [#23602](https://github.com/ClickHouse/ClickHouse/pull/23602) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Avoid possible build dependency on locale and filesystem order. This allows reproducible builds. [#23600](https://github.com/ClickHouse/ClickHouse/pull/23600) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Remove a source of nondeterminism from build. Now builds at different point of time will produce byte-identical binaries. Partially addressed [#22113](https://github.com/ClickHouse/ClickHouse/issues/22113). [#23559](https://github.com/ClickHouse/ClickHouse/pull/23559) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Add simple tool for benchmarking (Zoo)Keeper. [#23038](https://github.com/ClickHouse/ClickHouse/pull/23038) ([alesapin](https://github.com/alesapin)).
## ClickHouse release 21.5, 2021-05-20
#### Backward Incompatible Change
@ -637,6 +764,7 @@
* Allow using extended integer types (`Int128`, `Int256`, `UInt256`) in `avg` and `avgWeighted` functions. Also allow using different types (integer, decimal, floating point) for value and for weight in `avgWeighted` function. This is a backward-incompatible change: now the `avg` and `avgWeighted` functions always return `Float64` (as documented). Before this change the return type for `Decimal` arguments was also `Decimal`. [#15419](https://github.com/ClickHouse/ClickHouse/pull/15419) ([Mike](https://github.com/myrrc)).
* Expression `toUUID(N)` no longer works. Replace with `toUUID('00000000-0000-0000-0000-000000000000')`. This change is motivated by non-obvious results of `toUUID(N)` where N is non zero.
* SSL Certificates with incorrect "key usage" are rejected. In previous versions they are used to work. See [#19262](https://github.com/ClickHouse/ClickHouse/issues/19262).
* `incl` references to substitutions file (`/etc/metrika.xml`) were removed from the default config (`<remote_servers>`, `<zookeeper>`, `<macros>`, `<compression>`, `<networks>`). If you were using substitutions file and were relying on those implicit references, you should put them back manually and explicitly by adding corresponding sections with `incl="..."` attributes before the update. See [#18740](https://github.com/ClickHouse/ClickHouse/pull/18740) ([alexey-milovidov](https://github.com/alexey-milovidov)).
#### New Feature

View File

@ -528,7 +528,6 @@ include (cmake/find/libpqxx.cmake)
include (cmake/find/nuraft.cmake)
include (cmake/find/yaml-cpp.cmake)
if(NOT USE_INTERNAL_PARQUET_LIBRARY)
set (ENABLE_ORC OFF CACHE INTERNAL "")
endif()

View File

@ -4,12 +4,14 @@
#include <Core/Block.h>
#include <Interpreters/InternalTextLogsQueue.h>
#include <Interpreters/TextLog.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <sys/time.h>
#include <Poco/Message.h>
#include <Common/CurrentThread.h>
#include <Common/DNSResolver.h>
#include <common/getThreadId.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/IO.h>
namespace DB
{
@ -26,16 +28,48 @@ void OwnSplitChannel::log(const Poco::Message & msg)
auto matches = masker->wipeSensitiveData(message_text);
if (matches > 0)
{
logSplit({msg, message_text}); // we will continue with the copy of original message with text modified
tryLogSplit({msg, message_text}); // we will continue with the copy of original message with text modified
return;
}
}
logSplit(msg);
tryLogSplit(msg);
}
void OwnSplitChannel::tryLogSplit(const Poco::Message & msg)
{
try
{
logSplit(msg);
}
/// It is better to catch the errors here in order to avoid
/// breaking some functionality because of unexpected "File not
/// found" (or similar) error.
///
/// For example StorageDistributedDirectoryMonitor will mark batch
/// as broken, some MergeTree code can also be affected.
///
/// Also note, that we cannot log the exception here, since this
/// will lead to recursion, using regular tryLogCurrentException().
/// but let's log it into the stderr at least.
catch (...)
{
MemoryTracker::LockExceptionInThread lock_memory_tracker(VariableContext::Global);
const std::string & exception_message = getCurrentExceptionMessage(true);
const std::string & message = msg.getText();
/// NOTE: errors are ignored, since nothing can be done.
writeRetry(STDERR_FILENO, "Cannot add message to the log: ");
writeRetry(STDERR_FILENO, message.data(), message.size());
writeRetry(STDERR_FILENO, "\n");
writeRetry(STDERR_FILENO, exception_message.data(), exception_message.size());
writeRetry(STDERR_FILENO, "\n");
}
}
void OwnSplitChannel::logSplit(const Poco::Message & msg)
{
ExtendedLogMessage msg_ext = ExtendedLogMessage::getFrom(msg);

View File

@ -24,6 +24,7 @@ public:
private:
void logSplit(const Poco::Message & msg);
void tryLogSplit(const Poco::Message & msg);
using ChannelPtr = Poco::AutoPtr<Poco::Channel>;
/// Handler and its pointer casted to extended interface

View File

@ -4,6 +4,6 @@ if (NOT USE_YAML_CPP)
return()
endif()
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/yaml-cpp")
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/yaml-cpp/README.md")
message (ERROR "submodule contrib/yaml-cpp is missing. to fix try run: \n git submodule update --init --recursive")
endif()

View File

@ -61,7 +61,6 @@ endif()
add_subdirectory (poco-cmake)
add_subdirectory (croaring-cmake)
# TODO: refactor the contrib libraries below this comment.
if (USE_INTERNAL_ZSTD_LIBRARY)

2
contrib/avro vendored

@ -1 +1 @@
Subproject commit 1ee16d8c5a7808acff5cf0475f771195d9aa3faa
Subproject commit e43c46e87fd32eafdc09471e95344555454c5ef8

View File

@ -1,6 +1,6 @@
if (SANITIZE OR NOT (
((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_ARM OR ARCH_PPC64LE)) OR
(OS_DARWIN AND CMAKE_BUILD_TYPE STREQUAL "RelWithDebInfo")
(OS_DARWIN AND (CMAKE_BUILD_TYPE STREQUAL "RelWithDebInfo" OR CMAKE_BUILD_TYPE STREQUAL "Debug"))
))
if (ENABLE_JEMALLOC)
message (${RECONFIGURE_MESSAGE_LEVEL}

View File

@ -229,6 +229,7 @@ status()
case "$1" in
status)
status
exit 0
;;
esac

View File

@ -34,7 +34,7 @@ fi
CLICKHOUSE_CONFIG="${CLICKHOUSE_CONFIG:-/etc/clickhouse-server/config.xml}"
if ! $gosu test -f "$CLICKHOUSE_CONFIG" -a -r "$CLICKHOUSE_CONFIG"; then
echo "Configuration file '$dir' isn't readable by user with id '$USER'"
echo "Configuration file '$CLICKHOUSE_CONFIG' isn't readable by user with id '$USER'"
exit 1
fi

View File

@ -378,6 +378,9 @@ function run_tests
01852_jit_if
01865_jit_comparison_constant_result
01871_merge_tree_compile_expressions
# needs psql
01889_postgresql_protocol_null_fields
)
time clickhouse-test --hung-check -j 8 --order=random --use-skip-list \

View File

@ -4,9 +4,9 @@ services:
image: sequenceiq/hadoop-docker:2.7.0
hostname: hdfs1
restart: always
ports:
- ${HDFS_NAME_EXTERNAL_PORT}:${HDFS_NAME_INTERNAL_PORT} #50070
- ${HDFS_DATA_EXTERNAL_PORT}:${HDFS_DATA_INTERNAL_PORT} #50075
expose:
- ${HDFS_NAME_PORT}
- ${HDFS_DATA_PORT}
entrypoint: /etc/bootstrap.sh -d
volumes:
- type: ${HDFS_FS:-tmpfs}

View File

@ -14,18 +14,18 @@ services:
- type: ${KERBERIZED_HDFS_FS:-tmpfs}
source: ${KERBERIZED_HDFS_LOGS:-}
target: /var/log/hadoop-hdfs
ports:
- ${KERBERIZED_HDFS_NAME_EXTERNAL_PORT}:${KERBERIZED_HDFS_NAME_INTERNAL_PORT} #50070
- ${KERBERIZED_HDFS_DATA_EXTERNAL_PORT}:${KERBERIZED_HDFS_DATA_INTERNAL_PORT} #1006
expose:
- ${KERBERIZED_HDFS_NAME_PORT}
- ${KERBERIZED_HDFS_DATA_PORT}
depends_on:
- hdfskerberos
entrypoint: /etc/bootstrap.sh -d
hdfskerberos:
image: yandex/clickhouse-kerberos-kdc:${DOCKER_KERBEROS_KDC_TAG}
image: yandex/clickhouse-kerberos-kdc:${DOCKER_KERBEROS_KDC_TAG:-latest}
hostname: hdfskerberos
volumes:
- ${KERBERIZED_HDFS_DIR}/secrets:/tmp/keytab
- ${KERBERIZED_HDFS_DIR}/../../kerberos_image_config.sh:/config.sh
- /dev/urandom:/dev/random
ports: [88, 749]
expose: [88, 749]

View File

@ -10,7 +10,7 @@ echo '{
"storage-driver": "overlay2",
"insecure-registries" : ["dockerhub-proxy.sas.yp-c.yandex.net:5000"],
"registry-mirrors" : ["http://dockerhub-proxy.sas.yp-c.yandex.net:5000"]
}' | dd of=/etc/docker/daemon.json
}' | dd of=/etc/docker/daemon.json 2>/dev/null
dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log &

View File

@ -8,6 +8,7 @@ RUN apt-get update -y && \
python3-wheel \
brotli \
netcat-openbsd \
postgresql-client \
zstd
RUN python3 -m pip install \

View File

@ -5,7 +5,7 @@ toc_title: MySQL
# MySQL {#mysql}
The MySQL engine allows you to perform `SELECT` queries on data that is stored on a remote MySQL server.
The MySQL engine allows you to perform `SELECT` and `INSERT` queries on data that is stored on a remote MySQL server.
## Creating a Table {#creating-a-table}

View File

@ -10,7 +10,7 @@ This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ec
## Create Table {#creating-a-table}
``` sql
CREATE TABLE s3_engine_table (name String, value UInt32)
CREATE TABLE s3_engine_table (name String, value UInt32)
ENGINE = S3(path, [aws_access_key_id, aws_secret_access_key,] format, [compression])
```
@ -130,6 +130,7 @@ The following settings can be set before query execution or placed into configur
- `s3_max_single_part_upload_size` — The maximum size of object to upload using singlepart upload to S3. Default value is `64Mb`.
- `s3_min_upload_part_size` — The minimum size of part to upload during multipart upload to [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Default value is `512Mb`.
- `s3_max_redirects` — Max number of S3 redirects hops allowed. Default value is `10`.
- `s3_single_read_retries` — The maximum number of attempts during single read. Default value is `4`.
Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration.
@ -144,6 +145,7 @@ The following settings can be specified in configuration file for given endpoint
- `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Optional, default value is `false`.
- `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be speficied multiple times.
- `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional.
- `max_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Optional.
**Example:**
@ -158,13 +160,14 @@ The following settings can be specified in configuration file for given endpoint
<!-- <use_insecure_imds_request>false</use_insecure_imds_request> -->
<!-- <header>Authorization: Bearer SOME-TOKEN</header> -->
<!-- <server_side_encryption_customer_key_base64>BASE64-ENCODED-KEY</server_side_encryption_customer_key_base64> -->
<!-- <max_single_read_retries>4</max_single_read_retries> -->
</endpoint-name>
</s3>
```
## Usage {#usage-examples}
Suppose we have several files in TSV format with the following URIs on HDFS:
Suppose we have several files in CSV format with the following URIs on S3:
- 'https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_1.csv'
- 'https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_2.csv'
@ -177,21 +180,21 @@ Suppose we have several files in TSV format with the following URIs on HDFS:
1. There are several ways to make a table consisting of all six files:
``` sql
CREATE TABLE table_with_range (name String, value UInt32)
CREATE TABLE table_with_range (name String, value UInt32)
ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/some_file_{1..3}', 'CSV');
```
2. Another way:
``` sql
CREATE TABLE table_with_question_mark (name String, value UInt32)
CREATE TABLE table_with_question_mark (name String, value UInt32)
ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/some_file_?', 'CSV');
```
3. Table consists of all the files in both directories (all files should satisfy format and schema described in query):
``` sql
CREATE TABLE table_with_asterisk (name String, value UInt32)
CREATE TABLE table_with_asterisk (name String, value UInt32)
ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/*', 'CSV');
```
@ -201,7 +204,7 @@ ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_p
4. Create table with files named `file-000.csv`, `file-001.csv`, … , `file-999.csv`:
``` sql
CREATE TABLE big_table (name String, value UInt32)
CREATE TABLE big_table (name String, value UInt32)
ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/big_prefix/file-{000..999}.csv', 'CSV');
```

View File

@ -45,10 +45,10 @@ ORDER BY expr
[PARTITION BY expr]
[PRIMARY KEY expr]
[SAMPLE BY expr]
[TTL expr
[TTL expr
[DELETE|TO DISK 'xxx'|TO VOLUME 'xxx' [, ...] ]
[WHERE conditions]
[GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ]
[WHERE conditions]
[GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ]
[SETTINGS name=value, ...]
```
@ -105,7 +105,7 @@ For a description of parameters, see the [CREATE query description](../../../sql
- `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](../../../operations/settings/settings.md#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting.
- `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](../../../operations/settings/settings.md#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting.
- `max_partitions_to_read` — Limits the maximum number of partitions that can be accessed in one query. You can also specify setting [max_partitions_to_read](../../../operations/settings/merge-tree-settings.md#max-partitions-to-read) in the global setting.
**Example of Sections Setting**
``` sql
@ -158,7 +158,7 @@ When data is inserted in a table, separate data parts are created and each of th
Data belonging to different partitions are separated into different parts. In the background, ClickHouse merges data parts for more efficient storage. Parts belonging to different partitions are not merged. The merge mechanism does not guarantee that all rows with the same primary key will be in the same data part.
Data parts can be stored in `Wide` or `Compact` format. In `Wide` format each column is stored in a separate file in a filesystem, in `Compact` format all columns are stored in one file. `Compact` format can be used to increase performance of small and frequent inserts.
Data parts can be stored in `Wide` or `Compact` format. In `Wide` format each column is stored in a separate file in a filesystem, in `Compact` format all columns are stored in one file. `Compact` format can be used to increase performance of small and frequent inserts.
Data storing format is controlled by the `min_bytes_for_wide_part` and `min_rows_for_wide_part` settings of the table engine. If the number of bytes or rows in a data part is less then the corresponding setting's value, the part is stored in `Compact` format. Otherwise it is stored in `Wide` format. If none of these settings is set, data parts are stored in `Wide` format.
@ -457,10 +457,10 @@ ALTER TABLE example_table
Table can have an expression for removal of expired rows, and multiple expressions for automatic move of parts between [disks or volumes](#table_engine-mergetree-multiple-volumes). When rows in the table expire, ClickHouse deletes all corresponding rows. For parts moving feature, all rows of a part must satisfy the movement expression criteria.
``` sql
TTL expr
TTL expr
[DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'][, DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'] ...
[WHERE conditions]
[GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ]
[WHERE conditions]
[GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ]
```
Type of TTL rule may follow each TTL expression. It affects an action which is to be done once the expression is satisfied (reaches current time):
@ -472,7 +472,7 @@ Type of TTL rule may follow each TTL expression. It affects an action which is t
With `WHERE` clause you may specify which of the expired rows to delete or aggregate (it cannot be applied to moves).
`GROUP BY` expression must be a prefix of the table primary key.
`GROUP BY` expression must be a prefix of the table primary key.
If a column is not part of the `GROUP BY` expression and is not set explicitely in the `SET` clause, in result row it contains an occasional value from the grouped rows (as if aggregate function `any` is applied to it).
@ -506,7 +506,7 @@ Creating a table, where the rows are expired after one month. The expired rows w
``` sql
CREATE TABLE table_with_where
(
d DateTime,
d DateTime,
a Int
)
ENGINE = MergeTree
@ -520,10 +520,10 @@ Creating a table, where expired rows are aggregated. In result rows `x` contains
``` sql
CREATE TABLE table_for_aggregation
(
d DateTime,
k1 Int,
k2 Int,
x Int,
d DateTime,
k1 Int,
k2 Int,
x Int,
y Int
)
ENGINE = MergeTree
@ -657,7 +657,7 @@ Cofiguration examples:
</volumes>
<move_factor>0.2</move_factor>
</moving_from_ssd_to_hdd>
<small_jbod_with_external_no_merges>
<volumes>
<main>
@ -748,6 +748,7 @@ Configuration markup:
<connect_timeout_ms>10000</connect_timeout_ms>
<request_timeout_ms>5000</request_timeout_ms>
<retry_attempts>10</retry_attempts>
<single_read_retries>4</single_read_retries>
<min_bytes_for_seek>1000</min_bytes_for_seek>
<metadata_path>/var/lib/clickhouse/disks/s3/</metadata_path>
<cache_enabled>true</cache_enabled>
@ -761,21 +762,22 @@ Configuration markup:
Required parameters:
- `endpoint` — S3 endpoint url in `path` or `virtual hosted` [styles](https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html). Endpoint url should contain bucket and root path to store data.
- `access_key_id` — S3 access key id.
- `access_key_id` — S3 access key id.
- `secret_access_key` — S3 secret access key.
Optional parameters:
Optional parameters:
- `region` — S3 region name.
- `use_environment_credentials` — Reads AWS credentials from the Environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY and AWS_SESSION_TOKEN if they exist. Default value is `false`.
- `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Default value is `false`.
- `proxy` — Proxy configuration for S3 endpoint. Each `uri` element inside `proxy` block should contain a proxy URL.
- `connect_timeout_ms` — Socket connect timeout in milliseconds. Default value is `10 seconds`.
- `request_timeout_ms` — Request timeout in milliseconds. Default value is `5 seconds`.
- `retry_attempts` — Number of retry attempts in case of failed request. Default value is `10`.
- `min_bytes_for_seek` — Minimal number of bytes to use seek operation instead of sequential read. Default value is `1 Mb`.
- `metadata_path` — Path on local FS to store metadata files for S3. Default value is `/var/lib/clickhouse/disks/<disk_name>/`.
- `cache_enabled` — Allows to cache mark and index files on local FS. Default value is `true`.
- `cache_path` — Path on local FS where to store cached mark and index files. Default value is `/var/lib/clickhouse/disks/<disk_name>/cache/`.
- `proxy` — Proxy configuration for S3 endpoint. Each `uri` element inside `proxy` block should contain a proxy URL.
- `connect_timeout_ms` — Socket connect timeout in milliseconds. Default value is `10 seconds`.
- `request_timeout_ms` — Request timeout in milliseconds. Default value is `5 seconds`.
- `retry_attempts` — Number of retry attempts in case of failed request. Default value is `10`.
- `single_read_retries` — Number of retry attempts in case of connection drop during read. Default value is `4`.
- `min_bytes_for_seek` — Minimal number of bytes to use seek operation instead of sequential read. Default value is `1 Mb`.
- `metadata_path` — Path on local FS to store metadata files for S3. Default value is `/var/lib/clickhouse/disks/<disk_name>/`.
- `cache_enabled` — Allows to cache mark and index files on local FS. Default value is `true`.
- `cache_path` — Path on local FS where to store cached mark and index files. Default value is `/var/lib/clickhouse/disks/<disk_name>/cache/`.
- `skip_access_check` — If true, disk access checks will not be performed on disk start-up. Default value is `false`.
- `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set.
@ -816,6 +818,6 @@ S3 disk can be configured as `main` or `cold` storage:
</storage_configuration>
```
In case of `cold` option a data can be moved to S3 if local disk free size will be smaller than `move_factor * disk_size` or by TTL move rule.
In case of `cold` option a data can be moved to S3 if local disk free size will be smaller than `move_factor * disk_size` or by TTL move rule.
[Original article](https://clickhouse.tech/docs/ru/operations/table_engines/mergetree/) <!--hide-->

View File

@ -32,7 +32,7 @@ CREATE TABLE `ontime`
`Reporting_Airline` String,
`DOT_ID_Reporting_Airline` Int32,
`IATA_CODE_Reporting_Airline` String,
`Tail_Number` Int32,
`Tail_Number` String,
`Flight_Number_Reporting_Airline` String,
`OriginAirportID` Int32,
`OriginAirportSeqID` Int32,

View File

@ -18,6 +18,8 @@ To apply a CatBoost model in ClickHouse:
For more information about training CatBoost models, see [Training and applying models](https://catboost.ai/docs/features/training.html#training).
You can reload CatBoost models if the configuration was updated without restarting the server using [RELOAD MODEL](../sql-reference/statements/system.md#query_language-system-reload-model) and [RELOAD MODELS](../sql-reference/statements/system.md#query_language-system-reload-models) system queries.
## Prerequisites {#prerequisites}
If you do not have the [Docker](https://docs.docker.com/install/) yet, install it.

View File

@ -0,0 +1,107 @@
---
toc_priority: 146
toc_title: intervalLengthSum
---
# intervalLengthSum {#agg_function-intervallengthsum}
Calculates the total length of union of all ranges (segments on numeric axis).
**Syntax**
``` sql
intervalLengthSum(start, end)
```
**Arguments**
- `start` — The starting value of the interval. [Int32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Int64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Float32](../../../sql-reference/data-types/float.md#float32-float64), [Float64](../../../sql-reference/data-types/float.md#float32-float64), [DateTime](../../../sql-reference/data-types/datetime.md#data_type-datetime) or [Date](../../../sql-reference/data-types/date.md#data_type-date).
- `end` — The ending value of the interval. [Int32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Int64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Float32](../../../sql-reference/data-types/float.md#float32-float64), [Float64](../../../sql-reference/data-types/float.md#float32-float64), [DateTime](../../../sql-reference/data-types/datetime.md#data_type-datetime) or [Date](../../../sql-reference/data-types/date.md#data_type-date).
!!! info "Note"
Arguments must be of the same data type. Otherwise, an exception will be thrown.
**Returned value**
- Total length of union of all ranges (segments on numeric axis). Depending on the type of the argument, the return value may be [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64) or [Float64](../../../sql-reference/data-types/float.md#float32-float64) type.
**Examples**
1. Input table:
``` text
┌─id─┬─start─┬─end─┐
│ a │ 1.1 │ 2.9 │
│ a │ 2.5 │ 3.2 │
│ a │ 4 │ 5 │
└────┴───────┴─────┘
```
In this example, the arguments of the Float32 type are used. The function returns a value of the Float64 type.
Result is the sum of lengths of intervals `[1.1, 3.2]` (union of `[1.1, 2.9]` and `[2.5, 3.2]`) and `[4, 5]`
Query:
``` sql
SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM fl_interval GROUP BY id ORDER BY id;
```
Result:
``` text
┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐
│ a │ 3.1 │ Float64 │
└────┴───────────────────────────────┴───────────────────────────────────────────┘
```
2. Input table:
``` text
┌─id─┬───────────────start─┬─────────────────end─┐
│ a │ 2020-01-01 01:12:30 │ 2020-01-01 02:10:10 │
│ a │ 2020-01-01 02:05:30 │ 2020-01-01 02:50:31 │
│ a │ 2020-01-01 03:11:22 │ 2020-01-01 03:23:31 │
└────┴─────────────────────┴─────────────────────┘
```
In this example, the arguments of the DateTime type are used. The function returns a value in seconds.
Query:
``` sql
SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM dt_interval GROUP BY id ORDER BY id;
```
Result:
``` text
┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐
│ a │ 6610 │ UInt64 │
└────┴───────────────────────────────┴───────────────────────────────────────────┘
```
3. Input table:
``` text
┌─id─┬──────start─┬────────end─┐
│ a │ 2020-01-01 │ 2020-01-04 │
│ a │ 2020-01-12 │ 2020-01-18 │
└────┴────────────┴────────────┘
```
In this example, the arguments of the Date type are used. The function returns a value in days.
Query:
``` sql
SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM date_interval GROUP BY id ORDER BY id;
```
Result:
``` text
┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐
│ a │ 9 │ UInt64 │
└────┴───────────────────────────────┴───────────────────────────────────────────┘
```

View File

@ -159,7 +159,7 @@ Configuration fields:
| Tag | Description | Required |
|------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------|
| `name` | Column name. | Yes |
| `type` | ClickHouse data type.<br/>ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.<br/>[Nullable](../../../sql-reference/data-types/nullable.md) is currently supported for [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md), [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache) dictionaries. In [IPTrie](external-dicts-dict-layout.md#ip-trie) dictionaries `Nullable` types are not supported. | Yes |
| `type` | ClickHouse data type: [UInt8](../../../sql-reference/data-types/int-uint.md), [UInt16](../../../sql-reference/data-types/int-uint.md), [UInt32](../../../sql-reference/data-types/int-uint.md), [UInt64](../../../sql-reference/data-types/int-uint.md), [Int8](../../../sql-reference/data-types/int-uint.md), [Int16](../../../sql-reference/data-types/int-uint.md), [Int32](../../../sql-reference/data-types/int-uint.md), [Int64](../../../sql-reference/data-types/int-uint.md), [Float32](../../../sql-reference/data-types/float.md), [Float64](../../../sql-reference/data-types/float.md), [UUID](../../../sql-reference/data-types/uuid.md), [Decimal32](../../../sql-reference/data-types/decimal.md), [Decimal64](../../../sql-reference/data-types/decimal.md), [Decimal128](../../../sql-reference/data-types/decimal.md), [Decimal256](../../../sql-reference/data-types/decimal.md), [String](../../../sql-reference/data-types/string.md).<br/>ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.<br/>[Nullable](../../../sql-reference/data-types/nullable.md) is currently supported for [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md), [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache) dictionaries. In [IPTrie](external-dicts-dict-layout.md#ip-trie) dictionaries `Nullable` types are not supported. | Yes |
| `null_value` | Default value for a non-existing element.<br/>In the example, it is an empty string. [NULL](../../syntax.md#null-literal) value can be used only for the `Nullable` types (see the previous line with types description). | Yes |
| `expression` | [Expression](../../../sql-reference/syntax.md#syntax-expressions) that ClickHouse executes on the value.<br/>The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.<br/><br/>Default value: no expression. | No |
| <a name="hierarchical-dict-attr"></a> `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md).<br/><br/>Default value: `false`. | No |

View File

@ -265,7 +265,7 @@ Result:
```
!!! attention "Attention"
The return type `toStartOf*` functions described below is `Date` or `DateTime`. Though these functions can take `DateTime64` as an argument, passing them a `DateTime64` that is out of normal range (years 1970 - 2105) will give incorrect result.
The return type `toStartOf*` functions described below is `Date` or `DateTime`. Though these functions can take `DateTime64` as an argument, passing them a `DateTime64` that is out of the normal range (years 1925 - 2283) will give an incorrect result.
## toStartOfYear {#tostartofyear}

View File

@ -6,7 +6,7 @@ toc_title: RENAME
# RENAME Statement {#misc_operations-rename}
## RENAME DATABASE {#misc_operations-rename_database}
Renames database, support only for Atomic database engine
Renames database, it is supported only for Atomic database engine.
```
RENAME DATABASE atomic_database1 TO atomic_database2 [ON CLUSTER cluster]

View File

@ -10,6 +10,8 @@ The list of available `SYSTEM` statements:
- [RELOAD EMBEDDED DICTIONARIES](#query_language-system-reload-emdedded-dictionaries)
- [RELOAD DICTIONARIES](#query_language-system-reload-dictionaries)
- [RELOAD DICTIONARY](#query_language-system-reload-dictionary)
- [RELOAD MODELS](#query_language-system-reload-models)
- [RELOAD MODEL](#query_language-system-reload-model)
- [DROP DNS CACHE](#query_language-system-drop-dns-cache)
- [DROP MARK CACHE](#query_language-system-drop-mark-cache)
- [DROP UNCOMPRESSED CACHE](#query_language-system-drop-uncompressed-cache)
@ -60,6 +62,26 @@ The status of the dictionary can be checked by querying the `system.dictionaries
SELECT name, status FROM system.dictionaries;
```
## RELOAD MODELS {#query_language-system-reload-models}
Reloads all [CatBoost](../../guides/apply-catboost-model.md#applying-catboost-model-in-clickhouse) models if the configuration was updated without restarting the server.
**Syntax**
```sql
SYSTEM RELOAD MODELS
```
## RELOAD MODEL {#query_language-system-reload-model}
Completely reloads a CatBoost model `model_name` if the configuration was updated without restarting the server.
**Syntax**
```sql
SYSTEM RELOAD MODEL <model_name>
```
## DROP DNS CACHE {#query_language-system-drop-dns-cache}
Resets ClickHouses internal DNS cache. Sometimes (for old ClickHouse versions) it is necessary to use this command when changing the infrastructure (changing the IP address of another ClickHouse server or the server used by dictionaries).

View File

@ -114,14 +114,14 @@ FROM s3('https://storage.yandexcloud.net/my-test-bucket-768/big_prefix/file-{000
Insert data into file `test-data.csv.gz`:
``` sql
INSERT INTO s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip')
INSERT INTO FUNCTION s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip')
VALUES ('test-data', 1), ('test-data-2', 2);
```
Insert data into file `test-data.csv.gz` from existing table:
``` sql
INSERT INTO s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip')
INSERT INTO FUNCTION s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip')
SELECT name, value FROM existing_table;
```

View File

@ -6,9 +6,7 @@ toc_title: Window Functions
# [experimental] Window Functions
!!! warning "Warning"
This is an experimental feature that is currently in development and is not ready
for general use. It will change in unpredictable backwards-incompatible ways in
the future releases. Set `allow_experimental_window_functions = 1` to enable it.
This is an experimental feature that is currently in development and is not ready for general use. It will change in unpredictable backwards-incompatible ways in the future releases. Set `allow_experimental_window_functions = 1` to enable it.
ClickHouse supports the standard grammar for defining windows and window functions. The following features are currently supported:

View File

@ -5,7 +5,7 @@ toc_title: MySQL
# MySQL {#mysql}
Движок MySQL позволяет выполнять запросы `SELECT` над данными, хранящимися на удалённом MySQL сервере.
Движок MySQL позволяет выполнять запросы `SELECT` и `INSERT` над данными, хранящимися на удалённом MySQL сервере.
## Создание таблицы {#sozdanie-tablitsy}

View File

@ -10,7 +10,7 @@ toc_title: S3
## Создание таблицы {#creating-a-table}
``` sql
CREATE TABLE s3_engine_table (name String, value UInt32)
CREATE TABLE s3_engine_table (name String, value UInt32)
ENGINE = S3(path, [aws_access_key_id, aws_secret_access_key,] format, [compression])
```
@ -19,12 +19,12 @@ ENGINE = S3(path, [aws_access_key_id, aws_secret_access_key,] format, [compressi
- `path` — URL-адрес бакета с указанием пути к файлу. Поддерживает следующие подстановочные знаки в режиме "только чтение": `*`, `?`, `{abc,def}` и `{N..M}` где `N`, `M` — числа, `'abc'`, `'def'` — строки. Подробнее смотри [ниже](#wildcards-in-path).
- `format` — [формат](../../../interfaces/formats.md#formats) файла.
- `aws_access_key_id`, `aws_secret_access_key` - данные пользователя учетной записи [AWS](https://aws.amazon.com/ru/). Вы можете использовать их для аутентификации ваших запросов. Необязательный параметр. Если параметры учетной записи не указаны, то используются данные из конфигурационного файла. Смотрите подробнее [Использование сервиса S3 для хранения данных](../mergetree-family/mergetree.md#table_engine-mergetree-s3).
- `compression` — тип сжатия. Возможные значения: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Необязательный параметр. Если не указано, то тип сжатия определяется автоматически по расширению файла.
- `compression` — тип сжатия. Возможные значения: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Необязательный параметр. Если не указано, то тип сжатия определяется автоматически по расширению файла.
**Пример**
``` sql
CREATE TABLE s3_engine_table (name String, value UInt32)
CREATE TABLE s3_engine_table (name String, value UInt32)
ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'gzip');
INSERT INTO s3_engine_table VALUES ('one', 1), ('two', 2), ('three', 3);
SELECT * FROM s3_engine_table LIMIT 2;
@ -65,11 +65,12 @@ SELECT * FROM s3_engine_table LIMIT 2;
## Настройки движка S3 {#s3-settings}
Перед выполнением запроса или в конфигурационном файле могут быть установлены следующие настройки:
Перед выполнением запроса или в конфигурационном файле могут быть установлены следующие настройки:
- `s3_max_single_part_upload_size` — максимальный размер объекта для загрузки с использованием однокомпонентной загрузки в S3. Значение по умолчанию — `64 Mб`.
- `s3_max_single_part_upload_size` — максимальный размер объекта для загрузки с использованием однокомпонентной загрузки в S3. Значение по умолчанию — `64 Mб`.
- `s3_min_upload_part_size` — минимальный размер объекта для загрузки при многокомпонентной загрузке в [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Значение по умолчанию — `512 Mб`.
- `s3_max_redirects` — максимальное количество разрешенных переадресаций S3. Значение по умолчанию — `10`.
- `s3_max_redirects` — максимальное количество разрешенных переадресаций S3. Значение по умолчанию — `10`.
- `s3_single_read_retries` — максимальное количество попыток запроса при единичном чтении. Значение по умолчанию — `4`.
Соображение безопасности: если злонамеренный пользователь попробует указать произвольные URL-адреса S3, параметр `s3_max_redirects` должен быть установлен в ноль, чтобы избежать атак [SSRF] (https://en.wikipedia.org/wiki/Server-side_request_forgery). Как альтернатива, в конфигурации сервера должен быть указан `remote_host_filter`.
@ -78,7 +79,7 @@ SELECT * FROM s3_engine_table LIMIT 2;
Для точки приема запроса (которая соответствует точному префиксу URL-адреса) в конфигурационном файле могут быть заданы следующие настройки:
Обязательная настройка:
- `endpoint` — указывает префикс точки приема запроса.
- `endpoint` — указывает префикс точки приема запроса.
Необязательные настройки:
- `access_key_id` и `secret_access_key` — указывают учетные данные для использования с данной точкой приема запроса.
@ -86,7 +87,8 @@ SELECT * FROM s3_engine_table LIMIT 2;
- `use_insecure_imds_request` — признак использования менее безопасного соединения при выполнении запроса к IMDS при получении учётных данных из метаданных Amazon EC2. Значение по умолчанию — `false`.
- `region` — название региона S3.
- `header` — добавляет указанный HTTP-заголовок к запросу на заданную точку приема запроса. Может быть определен несколько раз.
- `server_side_encryption_customer_key_base64` — устанавливает необходимые заголовки для доступа к объектам S3 с шифрованием SSE-C.
- `server_side_encryption_customer_key_base64` — устанавливает необходимые заголовки для доступа к объектам S3 с шифрованием SSE-C.
- `single_read_retries` — Максимальное количество попыток запроса при единичном чтении. Значение по умолчанию — `4`.
**Пример**
@ -101,13 +103,14 @@ SELECT * FROM s3_engine_table LIMIT 2;
<!-- <use_insecure_imds_request>false</use_insecure_imds_request> -->
<!-- <header>Authorization: Bearer SOME-TOKEN</header> -->
<!-- <server_side_encryption_customer_key_base64>BASE64-ENCODED-KEY</server_side_encryption_customer_key_base64> -->
<!-- <single_read_retries>4</single_read_retries> -->
</endpoint-name>
</s3>
```
## Примеры использования {#usage-examples}
Предположим, у нас есть несколько файлов в формате TSV со следующими URL-адресами в S3:
Предположим, у нас есть несколько файлов в формате CSV со следующими URL-адресами в S3:
- 'https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_1.csv'
- 'https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_2.csv'
@ -119,21 +122,21 @@ SELECT * FROM s3_engine_table LIMIT 2;
1. Существует несколько способов создать таблицу, включающую в себя все шесть файлов:
``` sql
CREATE TABLE table_with_range (name String, value UInt32)
CREATE TABLE table_with_range (name String, value UInt32)
ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/some_file_{1..3}', 'CSV');
```
2. Другой способ:
``` sql
CREATE TABLE table_with_question_mark (name String, value UInt32)
CREATE TABLE table_with_question_mark (name String, value UInt32)
ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/some_file_?', 'CSV');
```
3. Таблица содержит все файлы в обоих каталогах (все файлы должны соответствовать формату и схеме, описанным в запросе):
``` sql
CREATE TABLE table_with_asterisk (name String, value UInt32)
CREATE TABLE table_with_asterisk (name String, value UInt32)
ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/*', 'CSV');
```
@ -142,7 +145,7 @@ ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_p
4. Создание таблицы из файлов с именами `file-000.csv`, `file-001.csv`, … , `file-999.csv`:
``` sql
CREATE TABLE big_table (name String, value UInt32)
CREATE TABLE big_table (name String, value UInt32)
ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/big_prefix/file-{000..999}.csv', 'CSV');
```

View File

@ -37,10 +37,10 @@ ORDER BY expr
[PARTITION BY expr]
[PRIMARY KEY expr]
[SAMPLE BY expr]
[TTL expr
[TTL expr
[DELETE|TO DISK 'xxx'|TO VOLUME 'xxx' [, ...] ]
[WHERE conditions]
[GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ]
[WHERE conditions]
[GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ]
[SETTINGS name=value, ...]
```
@ -51,31 +51,31 @@ ORDER BY expr
- `ENGINE` — имя и параметры движка. `ENGINE = MergeTree()`. `MergeTree` не имеет параметров.
- `ORDER BY` — ключ сортировки.
Кортеж столбцов или произвольных выражений. Пример: `ORDER BY (CounterID, EventDate)`.
ClickHouse использует ключ сортировки в качестве первичного ключа, если первичный ключ не задан в секции `PRIMARY KEY`.
Чтобы отключить сортировку, используйте синтаксис `ORDER BY tuple()`. Смотрите [выбор первичного ключа](#primary-keys-and-indexes-in-queries).
Чтобы отключить сортировку, используйте синтаксис `ORDER BY tuple()`. Смотрите [выбор первичного ключа](#primary-keys-and-indexes-in-queries).
- `PARTITION BY` — [ключ партиционирования](custom-partitioning-key.md). Необязательный параметр.
Для партиционирования по месяцам используйте выражение `toYYYYMM(date_column)`, где `date_column` — столбец с датой типа [Date](../../../engines/table-engines/mergetree-family/mergetree.md). В этом случае имена партиций имеют формат `"YYYYMM"`.
- `PRIMARY KEY` — первичный ключ, если он [отличается от ключа сортировки](#choosing-a-primary-key-that-differs-from-the-sorting-key). Необязательный параметр.
По умолчанию первичный ключ совпадает с ключом сортировки (который задаётся секцией `ORDER BY`.) Поэтому в большинстве случаев секцию `PRIMARY KEY` отдельно указывать не нужно.
- `SAMPLE BY` — выражение для сэмплирования. Необязательный параметр.
Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Пример: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`.
- `TTL` — список правил, определяющих длительности хранения строк, а также задающих правила перемещения частей на определённые тома или диски. Необязательный параметр.
Выражение должно возвращать столбец `Date` или `DateTime`. Пример: `TTL date + INTERVAL 1 DAY`.
Выражение должно возвращать столбец `Date` или `DateTime`. Пример: `TTL date + INTERVAL 1 DAY`.
Тип правила `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'|GROUP BY` указывает действие, которое будет выполнено с частью: удаление строк (прореживание), перемещение (при выполнении условия для всех строк части) на определённый диск (`TO DISK 'xxx'`) или том (`TO VOLUME 'xxx'`), или агрегирование данных в устаревших строках. Поведение по умолчанию соответствует удалению строк (`DELETE`). В списке правил может быть указано только одно выражение с поведением `DELETE`.
Дополнительные сведения смотрите в разделе [TTL для столбцов и таблиц](#table_engine-mergetree-ttl)
- `SETTINGS` — дополнительные параметры, регулирующие поведение `MergeTree` (необязательные):
@ -149,7 +149,7 @@ MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID)
Данные, относящиеся к разным партициям, разбиваются на разные куски. В фоновом режиме ClickHouse выполняет слияния (merge) кусков данных для более эффективного хранения. Куски, относящиеся к разным партициям не объединяются. Механизм слияния не гарантирует, что все строки с одинаковым первичным ключом окажутся в одном куске.
Куски данных могут храниться в формате `Wide` или `Compact`. В формате `Wide` каждый столбец хранится в отдельном файле, а в формате `Compact` все столбцы хранятся в одном файле. Формат `Compact` может быть полезен для повышения производительности при частом добавлении небольших объемов данных.
Куски данных могут храниться в формате `Wide` или `Compact`. В формате `Wide` каждый столбец хранится в отдельном файле, а в формате `Compact` все столбцы хранятся в одном файле. Формат `Compact` может быть полезен для повышения производительности при частом добавлении небольших объемов данных.
Формат хранения определяется настройками движка `min_bytes_for_wide_part` и `min_rows_for_wide_part`. Если число байт или строк в куске данных меньше значения, указанного в соответствующей настройке, тогда этот кусок данных хранится в формате `Compact`. В противном случае кусок данных хранится в формате `Wide`. Если ни одна из настроек не задана, куски данных хранятся в формате `Wide`.
@ -211,7 +211,7 @@ ClickHouse не требует уникального первичного кл
Вы можете создать таблицу без первичного ключа, используя синтаксис `ORDER BY tuple()`. В этом случае ClickHouse хранит данные в порядке вставки. Если вы хотите сохранить порядок данных при вставке данных с помощью запросов `INSERT ... SELECT`, установите [max_insert_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads).
Чтобы выбрать данные в первоначальном порядке, используйте
Чтобы выбрать данные в первоначальном порядке, используйте
[однопоточные](../../../operations/settings/settings.md#settings-max_threads) запросы `SELECT.
@ -323,7 +323,7 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
Необязательный параметр `false_positive` — это вероятность получения ложноположительного срабатывания. Возможные значения: (0, 1). Значение по умолчанию: 0.025.
Поддержанные типы данных: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`.
Фильтром могут пользоваться функции: [equals](../../../engines/table-engines/mergetree-family/mergetree.md), [notEquals](../../../engines/table-engines/mergetree-family/mergetree.md), [in](../../../engines/table-engines/mergetree-family/mergetree.md), [notIn](../../../engines/table-engines/mergetree-family/mergetree.md).
**Примеры**
@ -447,10 +447,10 @@ ALTER TABLE example_table
Для таблицы можно задать одно выражение для устаревания данных, а также несколько выражений, по срабатывании которых данные переместятся на [некоторый диск или том](#table_engine-mergetree-multiple-volumes). Когда некоторые данные в таблице устаревают, ClickHouse удаляет все соответствующие строки.
``` sql
TTL expr
TTL expr
[DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'][, DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'] ...
[WHERE conditions]
[GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ]
[WHERE conditions]
[GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ]
```
За каждым TTL выражением может следовать тип действия, которое выполняется после достижения времени, соответствующего результату TTL выражения:
@ -462,13 +462,13 @@ TTL expr
В секции `WHERE` можно задать условие удаления или агрегирования устаревших строк (для перемещения условие `WHERE` не применимо).
Колонки, по которым агрегируются данные в `GROUP BY`, должны являться префиксом первичного ключа таблицы.
Колонки, по которым агрегируются данные в `GROUP BY`, должны являться префиксом первичного ключа таблицы.
Если колонка не является частью выражения `GROUP BY` и не задается напрямую в секции `SET`, в результирующих строках она будет содержать случайное значение, взятое из одной из сгруппированных строк (как будто к ней применяется агрегирующая функция `any`).
**Примеры**
Создание таблицы с TTL:
Создание таблицы с TTL:
``` sql
CREATE TABLE example_table
@ -496,7 +496,7 @@ ALTER TABLE example_table
``` sql
CREATE TABLE table_with_where
(
d DateTime,
d DateTime,
a Int
)
ENGINE = MergeTree
@ -510,10 +510,10 @@ TTL d + INTERVAL 1 MONTH DELETE WHERE toDayOfWeek(d) = 1;
``` sql
CREATE TABLE table_for_aggregation
(
d DateTime,
k1 Int,
k2 Int,
x Int,
d DateTime,
k1 Int,
k2 Int,
x Int,
y Int
)
ENGINE = MergeTree
@ -735,6 +735,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd'
<connect_timeout_ms>10000</connect_timeout_ms>
<request_timeout_ms>5000</request_timeout_ms>
<retry_attempts>10</retry_attempts>
<single_read_retries>4</single_read_retries>
<min_bytes_for_seek>1000</min_bytes_for_seek>
<metadata_path>/var/lib/clickhouse/disks/s3/</metadata_path>
<cache_enabled>true</cache_enabled>
@ -749,22 +750,23 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd'
Обязательные параметры:
- `endpoint` — URL точки приема запроса на стороне S3 в [форматах](https://docs.aws.amazon.com/AmazonS3/latest/userguide/VirtualHosting.html) `path` или `virtual hosted`. URL точки должен содержать бакет и путь к корневой директории на сервере, где хранятся данные.
- `access_key_id` — id ключа доступа к S3.
- `access_key_id` — id ключа доступа к S3.
- `secret_access_key` — секретный ключ доступа к S3.
Необязательные параметры:
Необязательные параметры:
- `region` — название региона S3.
- `use_environment_credentials` — признак, нужно ли считывать учетные данные AWS из сетевого окружения, а также из переменных окружения `AWS_ACCESS_KEY_ID`, `AWS_SECRET_ACCESS_KEY` и `AWS_SESSION_TOKEN`, если они есть. Значение по умолчанию: `false`.
- `use_insecure_imds_request` — признак, нужно ли использовать менее безопасное соединение при выполнении запроса к IMDS при получении учётных данных из метаданных Amazon EC2. Значение по умолчанию: `false`.
- `proxy` — конфигурация прокси-сервера для конечной точки S3. Каждый элемент `uri` внутри блока `proxy` должен содержать URL прокси-сервера.
- `connect_timeout_ms` — таймаут подключения к сокету в миллисекундах. Значение по умолчанию: 10 секунд.
- `request_timeout_ms` — таймаут выполнения запроса в миллисекундах. Значение по умолчанию: 5 секунд.
- `retry_attempts` — число попыток выполнения запроса в случае возникновения ошибки. Значение по умолчанию: `10`.
- `min_bytes_for_seek` — минимальное количество байтов, которые используются для операций поиска вместо последовательного чтения. Значение по умолчанию: 1 МБайт.
- `metadata_path` — путь к локальному файловому хранилищу для хранения файлов с метаданными для S3. Значение по умолчанию: `/var/lib/clickhouse/disks/<disk_name>/`.
- `cache_enabled` — признак, разрешено ли хранение кэша засечек и файлов индекса в локальной файловой системе. Значение по умолчанию: `true`.
- `cache_path` — путь в локальной файловой системе, где будут храниться кэш засечек и файлы индекса. Значение по умолчанию: `/var/lib/clickhouse/disks/<disk_name>/cache/`.
- `proxy` — конфигурация прокси-сервера для конечной точки S3. Каждый элемент `uri` внутри блока `proxy` должен содержать URL прокси-сервера.
- `connect_timeout_ms` — таймаут подключения к сокету в миллисекундах. Значение по умолчанию: 10 секунд.
- `request_timeout_ms` — таймаут выполнения запроса в миллисекундах. Значение по умолчанию: 5 секунд.
- `retry_attempts` — число попыток выполнения запроса в случае возникновения ошибки. Значение по умолчанию: `10`.
- `single_read_retries` — число попыток выполнения запроса в случае возникновения ошибки в процессе чтения. Значение по умолчанию: `4`.
- `min_bytes_for_seek` — минимальное количество байтов, которые используются для операций поиска вместо последовательного чтения. Значение по умолчанию: 1 МБайт.
- `metadata_path` — путь к локальному файловому хранилищу для хранения файлов с метаданными для S3. Значение по умолчанию: `/var/lib/clickhouse/disks/<disk_name>/`.
- `cache_enabled` — признак, разрешено ли хранение кэша засечек и файлов индекса в локальной файловой системе. Значение по умолчанию: `true`.
- `cache_path` — путь в локальной файловой системе, где будут храниться кэш засечек и файлы индекса. Значение по умолчанию: `/var/lib/clickhouse/disks/<disk_name>/cache/`.
- `skip_access_check` — признак, выполнять ли проверку доступов при запуске диска. Если установлено значение `true`, то проверка не выполняется. Значение по умолчанию: `false`.
@ -805,6 +807,4 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd'
</storage_configuration>
```
Если диск сконфигурирован как `cold`, данные будут переноситься в S3 при срабатывании правил TTL или когда свободное место на локальном диске станет меньше порогового значения, которое определяется как `move_factor * disk_size`.
Если диск сконфигурирован как `cold`, данные будут переноситься в S3 при срабатывании правил TTL или когда свободное место на локальном диске станет меньше порогового значения, которое определяется как `move_factor * disk_size`.

View File

@ -18,6 +18,8 @@ toc_title: "Применение модели CatBoost в ClickHouse"
Подробнее об обучении моделей в CatBoost, см. [Обучение и применение моделей](https://catboost.ai/docs/features/training.html#training).
Вы можете перегрузить модели CatBoost, если их конфигурация была обновлена, без перезагрузки сервера. Для этого используйте системные запросы [RELOAD MODEL](../sql-reference/statements/system.md#query_language-system-reload-model) и [RELOAD MODELS](../sql-reference/statements/system.md#query_language-system-reload-models).
## Перед началом работы {#prerequisites}
Если у вас еще нет [Docker](https://docs.docker.com/install/), установите его.

View File

@ -0,0 +1,107 @@
---
toc_priority: 146
toc_title: intervalLengthSum
---
# intervalLengthSum {#agg_function-intervallengthsum}
Вычисляет длину объединения интервалов (отрезков на числовой оси).
**Синтаксис**
``` sql
intervalLengthSum(start, end)
```
**Аргументы**
- `start` — начальное значение интервала. [Int32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Int64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Float32](../../../sql-reference/data-types/float.md#float32-float64), [Float64](../../../sql-reference/data-types/float.md#float32-float64), [DateTime](../../../sql-reference/data-types/datetime.md#data_type-datetime) или [Date](../../../sql-reference/data-types/date.md#data_type-date).
- `end` — конечное значение интервала. [Int32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Int64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Float32](../../../sql-reference/data-types/float.md#float32-float64), [Float64](../../../sql-reference/data-types/float.md#float32-float64), [DateTime](../../../sql-reference/data-types/datetime.md#data_type-datetime) или [Date](../../../sql-reference/data-types/date.md#data_type-date).
!!! info "Примечание"
Аргументы должны быть одного типа. В противном случае ClickHouse сгенерирует исключение.
**Возвращаемое значение**
- Длина объединения всех интервалов (отрезков на числовой оси). В зависимости от типа аргумента возвращаемое значение может быть типа [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64) или [Float64](../../../sql-reference/data-types/float.md#float32-float64).
**Примеры**
1. Входная таблица:
``` text
┌─id─┬─start─┬─end─┐
│ a │ 1.1 │ 2.9 │
│ a │ 2.5 │ 3.2 │
│ a │ 4 │ 5 │
└────┴───────┴─────┘
```
В этом примере используются аргументы типа Float32. Функция возвращает значение типа Float64.
Результатом функции будет сумма длин интервалов `[1.1, 3.2]` (объединение `[1.1, 2.9]` и `[2.5, 3.2]`) и `[4, 5]`
Запрос:
``` sql
SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM fl_interval GROUP BY id ORDER BY id;
```
Результат:
``` text
┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐
│ a │ 3.1 │ Float64 │
└────┴───────────────────────────────┴───────────────────────────────────────────┘
```
2. Входная таблица:
``` text
┌─id─┬───────────────start─┬─────────────────end─┐
│ a │ 2020-01-01 01:12:30 │ 2020-01-01 02:10:10 │
│ a │ 2020-01-01 02:05:30 │ 2020-01-01 02:50:31 │
│ a │ 2020-01-01 03:11:22 │ 2020-01-01 03:23:31 │
└────┴─────────────────────┴─────────────────────┘
```
В этом примере используются аргументы типа DateTime. Функция возвращает значение, выраженное в секундах.
Запрос:
``` sql
SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM dt_interval GROUP BY id ORDER BY id;
```
Результат:
``` text
┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐
│ a │ 6610 │ UInt64 │
└────┴───────────────────────────────┴───────────────────────────────────────────┘
```
3. Входная таблица:
``` text
┌─id─┬──────start─┬────────end─┐
│ a │ 2020-01-01 │ 2020-01-04 │
│ a │ 2020-01-12 │ 2020-01-18 │
└────┴────────────┴────────────┘
```
В этом примере используются аргументы типа Date. Функция возвращает значение, выраженное в днях.
Запрос:
``` sql
SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM date_interval GROUP BY id ORDER BY id;
```
Результат:
``` text
┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐
│ a │ 9 │ UInt64 │
└────┴───────────────────────────────┴───────────────────────────────────────────┘
```

View File

@ -159,7 +159,7 @@ CREATE DICTIONARY somename (
| Тег | Описание | Обязательный |
|------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------|
| `name` | Имя столбца. | Да |
| `type` | Тип данных ClickHouse.<br/>ClickHouse пытается привести значение из словаря к заданному типу данных. Например, в случае MySQL, в таблице-источнике поле может быть `TEXT`, `VARCHAR`, `BLOB`, но загружено может быть как `String`. <br/>[Nullable](../../../sql-reference/data-types/nullable.md) в настоящее время поддерживается для словарей [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md), [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache). Для словарей [IPTrie](external-dicts-dict-layout.md#ip-trie) `Nullable`-типы не поддерживаются. | Да |
| `type` | Тип данных ClickHouse: [UInt8](../../../sql-reference/data-types/int-uint.md), [UInt16](../../../sql-reference/data-types/int-uint.md), [UInt32](../../../sql-reference/data-types/int-uint.md), [UInt64](../../../sql-reference/data-types/int-uint.md), [Int8](../../../sql-reference/data-types/int-uint.md), [Int16](../../../sql-reference/data-types/int-uint.md), [Int32](../../../sql-reference/data-types/int-uint.md), [Int64](../../../sql-reference/data-types/int-uint.md), [Float32](../../../sql-reference/data-types/float.md), [Float64](../../../sql-reference/data-types/float.md), [UUID](../../../sql-reference/data-types/uuid.md), [Decimal32](../../../sql-reference/data-types/decimal.md), [Decimal64](../../../sql-reference/data-types/decimal.md), [Decimal128](../../../sql-reference/data-types/decimal.md), [Decimal256](../../../sql-reference/data-types/decimal.md), [String](../../../sql-reference/data-types/string.md).<br/>ClickHouse пытается привести значение из словаря к заданному типу данных. Например, в случае MySQL, в таблице-источнике поле может быть `TEXT`, `VARCHAR`, `BLOB`, но загружено может быть как `String`. <br/>[Nullable](../../../sql-reference/data-types/nullable.md) в настоящее время поддерживается для словарей [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md), [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache). Для словарей [IPTrie](external-dicts-dict-layout.md#ip-trie) `Nullable`-типы не поддерживаются. | Да |
| `null_value` | Значение по умолчанию для несуществующего элемента.<br/>В примере это пустая строка. Значение [NULL](../../syntax.md#null-literal) можно указывать только для типов `Nullable` (см. предыдущую строку с описанием типов). | Да |
| `expression` | [Выражение](../../syntax.md#syntax-expressions), которое ClickHouse выполняет со значением.<br/>Выражением может быть имя столбца в удаленной SQL базе. Таким образом, вы можете использовать его для создания псевдонима удаленного столбца.<br/><br/>Значение по умолчанию: нет выражения. | Нет |
| <a name="hierarchical-dict-attr"></a> `hierarchical` | Если `true`, то атрибут содержит ключ предка для текущего элемента. Смотрите [Иерархические словари](external-dicts-dict-hierarchical.md).<br/><br/>Значение по умолчанию: `false`. | Нет |

View File

@ -264,6 +264,9 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp;
└────────────────┘
```
!!! attention "Attention"
`Date` или `DateTime` это возвращаемый тип функций `toStartOf*`, который описан ниже. Несмотря на то, что эти функции могут принимать `DateTime64` в качестве аргумента, если переданное значение типа `DateTime64` выходит за пределы нормального диапазона (с 1925 по 2283 год), то это даст неверный результат.
## toStartOfYear {#tostartofyear}
Округляет дату или дату-с-временем вниз до первого дня года.

View File

@ -6,7 +6,7 @@ toc_title: RENAME
# RENAME Statement {#misc_operations-rename}
## RENAME DATABASE {#misc_operations-rename_database}
Переименование базы данных
Переименовывает базу данных, поддерживается только для движка базы данных Atomic.
```
RENAME DATABASE atomic_database1 TO atomic_database2 [ON CLUSTER cluster]

View File

@ -8,6 +8,8 @@ toc_title: SYSTEM
- [RELOAD EMBEDDED DICTIONARIES](#query_language-system-reload-emdedded-dictionaries)
- [RELOAD DICTIONARIES](#query_language-system-reload-dictionaries)
- [RELOAD DICTIONARY](#query_language-system-reload-dictionary)
- [RELOAD MODELS](#query_language-system-reload-models)
- [RELOAD MODEL](#query_language-system-reload-model)
- [DROP DNS CACHE](#query_language-system-drop-dns-cache)
- [DROP MARK CACHE](#query_language-system-drop-mark-cache)
- [DROP UNCOMPRESSED CACHE](#query_language-system-drop-uncompressed-cache)
@ -37,7 +39,7 @@ toc_title: SYSTEM
- [RESTART REPLICAS](#query_language-system-restart-replicas)
## RELOAD EMBEDDED DICTIONARIES] {#query_language-system-reload-emdedded-dictionaries}
Перегружет все [Встроенные словари](../dictionaries/internal-dicts.md).
Перегружает все [Встроенные словари](../dictionaries/internal-dicts.md).
По умолчанию встроенные словари выключены.
Всегда возвращает `Ok.`, вне зависимости от результата обновления встроенных словарей.
@ -57,6 +59,26 @@ toc_title: SYSTEM
SELECT name, status FROM system.dictionaries;
```
## RELOAD MODELS {#query_language-system-reload-models}
Перегружает все модели [CatBoost](../../guides/apply-catboost-model.md#applying-catboost-model-in-clickhouse), если их конфигурация была обновлена, без перезагрузки сервера.
**Синтаксис**
```sql
SYSTEM RELOAD MODELS
```
## RELOAD MODEL {#query_language-system-reload-model}
Полностью перегружает модель [CatBoost](../../guides/apply-catboost-model.md#applying-catboost-model-in-clickhouse) `model_name`, если ее конфигурация была обновлена, без перезагрузки сервера.
**Синтаксис**
```sql
SYSTEM RELOAD MODEL <model_name>
```
## DROP DNS CACHE {#query_language-system-drop-dns-cache}
Сбрасывает внутренний DNS кеш ClickHouse. Иногда (для старых версий ClickHouse) необходимо использовать эту команду при изменении инфраструктуры (смене IP адреса у другого ClickHouse сервера или сервера, используемого словарями).

File diff suppressed because it is too large Load Diff

View File

@ -18,12 +18,13 @@ public:
ClusterCopier(const String & task_path_,
const String & host_id_,
const String & proxy_database_name_,
ContextMutablePtr context_)
ContextMutablePtr context_,
Poco::Logger * log_)
: WithMutableContext(context_),
task_zookeeper_path(task_path_),
host_id(host_id_),
working_database_name(proxy_database_name_),
log(&Poco::Logger::get("ClusterCopier")) {}
log(log_) {}
void init();
@ -117,14 +118,14 @@ protected:
TaskStatus tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name);
/// Removes MATERIALIZED and ALIAS columns from create table query
static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast);
static ASTPtr removeAliasMaterializedAndTTLColumnsFromCreateQuery(const ASTPtr & query_ast, bool allow_to_copy_alias_and_materialized_columns);
bool tryDropPartitionPiece(ShardPartition & task_partition, size_t current_piece_number,
const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock);
static constexpr UInt64 max_table_tries = 3;
static constexpr UInt64 max_shard_partition_tries = 3;
static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 3;
static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 10;
bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table);
@ -189,9 +190,7 @@ protected:
const ClusterPtr & cluster,
const String & query,
const Settings & current_settings,
PoolMode pool_mode = PoolMode::GET_ALL,
ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_SHARD,
UInt64 max_successful_executions_per_shard = 0) const;
ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_SHARD) const;
private:
String task_zookeeper_path;
@ -208,7 +207,6 @@ private:
ConfigurationPtr task_cluster_initial_config;
ConfigurationPtr task_cluster_current_config;
Coordination::Stat task_description_current_stat{};
std::unique_ptr<TaskCluster> task_cluster;

View File

@ -22,8 +22,9 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self)
config_xml_path = config().getString("config-file");
task_path = config().getString("task-path");
log_level = config().getString("log-level", "trace");
log_level = config().getString("log-level", "info");
is_safe_mode = config().has("safe-mode");
is_status_mode = config().has("status");
if (config().has("copy-fault-probability"))
copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0);
if (config().has("move-fault-probability"))
@ -97,6 +98,7 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options)
.argument("base-dir").binding("base-dir"));
options.addOption(Poco::Util::Option("experimental-use-sample-offset", "", "Use SAMPLE OFFSET query instead of cityHash64(PRIMARY KEY) % n == k")
.argument("experimental-use-sample-offset").binding("experimental-use-sample-offset"));
options.addOption(Poco::Util::Option("status", "", "Get for status for current execution").binding("status"));
using Me = std::decay_t<decltype(*this)>;
options.addOption(Poco::Util::Option("help", "", "produce this help message").binding("help")
@ -106,6 +108,25 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options)
void ClusterCopierApp::mainImpl()
{
/// Status command
{
if (is_status_mode)
{
SharedContextHolder shared_context = Context::createShared();
auto context = Context::createGlobal(shared_context.get());
context->makeGlobalContext();
SCOPE_EXIT_SAFE(context->shutdown());
auto zookeeper = context->getZooKeeper();
auto status_json = zookeeper->get(task_path + "/status");
LOG_INFO(&logger(), "{}", status_json);
std::cout << status_json << std::endl;
context->resetZooKeeper();
return;
}
}
StatusFile status_file(process_path + "/status", StatusFile::write_full_info);
ThreadStatus thread_status;
@ -136,7 +157,7 @@ void ClusterCopierApp::mainImpl()
/// Initialize query scope just in case.
CurrentThread::QueryScope query_scope(context);
auto copier = std::make_unique<ClusterCopier>(task_path, host_id, default_database, context);
auto copier = std::make_unique<ClusterCopier>(task_path, host_id, default_database, context, log);
copier->setSafeMode(is_safe_mode);
copier->setCopyFaultProbability(copy_fault_probability);
copier->setMoveFaultProbability(move_fault_probability);

View File

@ -76,8 +76,9 @@ private:
std::string config_xml_path;
std::string task_path;
std::string log_level = "trace";
std::string log_level = "info";
bool is_safe_mode = false;
bool is_status_mode = false;
double copy_fault_probability = 0.0;
double move_fault_probability = 0.0;
bool is_help = false;

View File

@ -0,0 +1,65 @@
#pragma once
#include <Poco/JSON/Parser.h>
#include <Poco/JSON/JSON.h>
#include <Poco/JSON/Object.h>
#include <Poco/JSON/Stringifier.h>
#include <unordered_map>
#include <memory>
#include <string>
#include <iostream>
namespace DB
{
class StatusAccumulator
{
public:
struct TableStatus
{
size_t all_partitions_count;
size_t processed_partitions_count;
};
using Map = std::unordered_map<std::string, TableStatus>;
using MapPtr = std::shared_ptr<Map>;
static MapPtr fromJSON(std::string state_json)
{
Poco::JSON::Parser parser;
auto state = parser.parse(state_json).extract<Poco::JSON::Object::Ptr>();
MapPtr result_ptr = std::make_shared<Map>();
for (const auto & table_name : state->getNames())
{
auto table_status_json = state->getValue<std::string>(table_name);
auto table_status = parser.parse(table_status_json).extract<Poco::JSON::Object::Ptr>();
/// Map entry will be created if it is absent
auto & map_table_status = (*result_ptr)[table_name];
map_table_status.all_partitions_count += table_status->getValue<size_t>("all_partitions_count");
map_table_status.processed_partitions_count += table_status->getValue<size_t>("processed_partitions_count");
}
return result_ptr;
}
static std::string serializeToJSON(MapPtr statuses)
{
Poco::JSON::Object result_json;
for (const auto & [table_name, table_status] : *statuses)
{
Poco::JSON::Object status_json;
status_json.set("all_partitions_count", table_status.all_partitions_count);
status_json.set("processed_partitions_count", table_status.processed_partitions_count);
result_json.set(table_name, status_json);
}
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
oss.exceptions(std::ios::failbit);
Poco::JSON::Stringifier::stringify(result_json, oss);
auto result = oss.str();
return result;
}
};
}

View File

@ -77,6 +77,8 @@ inline void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfigurat
if (config.has(prefix + "settings"))
settings_common.loadSettingsFromConfig(prefix + "settings", config);
settings_common.prefer_localhost_replica = 0;
settings_pull = settings_common;
if (config.has(prefix + "settings_pull"))
settings_pull.loadSettingsFromConfig(prefix + "settings_pull", config);
@ -92,11 +94,15 @@ inline void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfigurat
/// Override important settings
settings_pull.readonly = 1;
settings_push.insert_distributed_sync = 1;
settings_pull.prefer_localhost_replica = false;
settings_push.insert_distributed_sync = true;
settings_push.prefer_localhost_replica = false;
set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME);
set_default_value(settings_pull.max_threads, 1);
set_default_value(settings_pull.max_block_size, 8192UL);
set_default_value(settings_pull.preferred_block_size_bytes, 0);
set_default_value(settings_push.insert_distributed_timeout, 0);
set_default_value(settings_push.replication_alter_partitions_sync, 2);
}

View File

@ -36,27 +36,33 @@ struct TaskTable
String getPartitionAttachIsDonePath(const String & partition_name) const;
String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const;
String getPartitionPiecePath(const String & partition_name, size_t piece_number) const;
String getCertainPartitionIsDirtyPath(const String & partition_name) const;
String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const;
String getCertainPartitionPieceIsDirtyPath(const String & partition_name, size_t piece_number) const;
String getCertainPartitionIsCleanedPath(const String & partition_name) const;
String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const;
String getCertainPartitionPieceIsCleanedPath(const String & partition_name, size_t piece_number) const;
String getCertainPartitionTaskStatusPath(const String & partition_name) const;
String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const;
String getCertainPartitionPieceTaskStatusPath(const String & partition_name, size_t piece_number) const;
bool isReplicatedTable() const { return is_replicated_table; }
/// These nodes are used for check-status option
String getStatusAllPartitionCount() const;
String getStatusProcessedPartitionsCount() const;
/// Partitions will be split into number-of-splits pieces.
/// Each piece will be copied independently. (10 by default)
size_t number_of_splits;
bool allow_to_copy_alias_and_materialized_columns{false};
bool allow_to_drop_target_partitions{false};
String name_in_config;
/// Used as task ID
@ -83,7 +89,7 @@ struct TaskTable
String engine_push_zk_path;
bool is_replicated_table;
ASTPtr rewriteReplicatedCreateQueryToPlain();
ASTPtr rewriteReplicatedCreateQueryToPlain() const;
/*
* A Distributed table definition used to split data
@ -181,6 +187,7 @@ struct TaskShard
/// Last CREATE TABLE query of the table of the shard
ASTPtr current_pull_table_create_query;
ASTPtr current_push_table_create_query;
/// Internal distributed tables
DatabaseAndTableName table_read_shard;
@ -242,6 +249,16 @@ inline String TaskTable::getCertainPartitionPieceTaskStatusPath(const String & p
return getPartitionPiecePath(partition_name, piece_number) + "/shards";
}
inline String TaskTable::getStatusAllPartitionCount() const
{
return task_cluster.task_zookeeper_path + "/status/all_partitions_count";
}
inline String TaskTable::getStatusProcessedPartitionsCount() const
{
return task_cluster.task_zookeeper_path + "/status/processed_partitions_count";
}
inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config,
const String & prefix_, const String & table_key)
: task_cluster(parent)
@ -250,7 +267,10 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf
name_in_config = table_key;
number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10);
number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 3);
allow_to_copy_alias_and_materialized_columns = config.getBool(table_prefix + "allow_to_copy_alias_and_materialized_columns", false);
allow_to_drop_target_partitions = config.getBool(table_prefix + "allow_to_drop_target_partitions", false);
cluster_pull_name = config.getString(table_prefix + "cluster_pull");
cluster_push_name = config.getString(table_prefix + "cluster_push");
@ -343,7 +363,7 @@ inline void TaskTable::initShards(RandomEngine && random_engine)
std::uniform_int_distribution<UInt8> get_urand(0, std::numeric_limits<UInt8>::max());
// Compute the priority
for (auto & shard_info : cluster_pull->getShardsInfo())
for (const auto & shard_info : cluster_pull->getShardsInfo())
{
TaskShardPtr task_shard = std::make_shared<TaskShard>(*this, shard_info);
const auto & replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster());
@ -369,7 +389,7 @@ inline void TaskTable::initShards(RandomEngine && random_engine)
local_shards.assign(all_shards.begin(), it_first_remote);
}
inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain()
inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain() const
{
ASTPtr prev_engine_push_ast = engine_push_ast->clone();
@ -383,9 +403,15 @@ inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain()
{
auto & replicated_table_arguments = new_engine_ast.arguments->children;
/// Delete first two arguments of Replicated...MergeTree() table.
replicated_table_arguments.erase(replicated_table_arguments.begin());
replicated_table_arguments.erase(replicated_table_arguments.begin());
/// In some cases of Atomic database engine usage ReplicatedMergeTree tables
/// could be created without arguments.
if (!replicated_table_arguments.empty())
{
/// Delete first two arguments of Replicated...MergeTree() table.
replicated_table_arguments.erase(replicated_table_arguments.begin());
replicated_table_arguments.erase(replicated_table_arguments.begin());
}
}
return new_storage_ast.clone();
@ -400,7 +426,7 @@ inline String DB::TaskShard::getDescription() const
inline String DB::TaskShard::getHostNameExample() const
{
auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster());
const auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster());
return replicas.at(0).readableString();
}

View File

@ -19,6 +19,7 @@
#include <Common/StringUtils/StringUtils.h>
#include <Common/getHashOfLoadedBinary.h>
#include <Common/IO.h>
#include <common/phdr_cache.h>
#include <ext/scope_guard.h>
@ -172,11 +173,11 @@ enum class InstructionFail
AVX512 = 8
};
std::pair<const char *, size_t> instructionFailToString(InstructionFail fail)
auto instructionFailToString(InstructionFail fail)
{
switch (fail)
{
#define ret(x) return std::make_pair(x, ARRAY_SIZE(x) - 1)
#define ret(x) return std::make_tuple(STDERR_FILENO, x, ARRAY_SIZE(x) - 1)
case InstructionFail::NONE:
ret("NONE");
case InstructionFail::SSE3:
@ -260,28 +261,12 @@ void checkRequiredInstructionsImpl(volatile InstructionFail & fail)
fail = InstructionFail::NONE;
}
/// This function is safe to use in static initializers.
void writeErrorLen(const char * data, size_t size)
{
while (size != 0)
{
ssize_t res = ::write(STDERR_FILENO, data, size);
if ((-1 == res || 0 == res) && errno != EINTR)
_Exit(1);
if (res > 0)
{
data += res;
size -= res;
}
}
}
/// Macros to avoid using strlen(), since it may fail if SSE is not supported.
#define writeError(data) do \
{ \
static_assert(__builtin_constant_p(data)); \
writeErrorLen(data, ARRAY_SIZE(data) - 1); \
if (!writeRetry(STDERR_FILENO, data, ARRAY_SIZE(data) - 1)) \
_Exit(1); \
} while (false)
/// Check SSE and others instructions availability. Calls exit on fail.
@ -310,7 +295,8 @@ void checkRequiredInstructions()
if (sigsetjmp(jmpbuf, 1))
{
writeError("Instruction check fail. The CPU does not support ");
std::apply(writeErrorLen, instructionFailToString(fail));
if (!std::apply(writeRetry, instructionFailToString(fail)))
_Exit(1);
writeError(" instruction set.\n");
_Exit(1);
}

View File

@ -1038,7 +1038,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
attachSystemTablesServer(*database_catalog.getSystemDatabase(), has_zookeeper);
/// We load temporary database first, because projections need it.
database_catalog.loadTemporaryDatabase();
database_catalog.initializeAndLoadTemporaryDatabase();
/// Then, load remaining databases
loadMetadata(global_context, default_database);
database_catalog.loadDatabases();

View File

@ -705,6 +705,8 @@
"yandex.ru" -> "yandex.ru:443", "yandex.ru:80" etc. is allowed, but "yandex.ru:80" -> only "yandex.ru:80" is allowed.
If the host is specified as IP address, it is checked as specified in URL. Example: "[2a02:6b8:a::a]".
If there are redirects and support for redirects is enabled, every redirect (the Location field) is checked.
Host should be specified using the host xml tag:
<host>yandex.ru</host>
-->
<!-- Regular expression can be specified. RE2 engine is used for regexps.

View File

@ -1,5 +1,5 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionSegmentLengthSum.h>
#include <AggregateFunctions/AggregateFunctionIntervalLengthSum.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeDate.h>
@ -22,7 +22,7 @@ namespace
{
template <template <typename> class Data>
AggregateFunctionPtr
createAggregateFunctionSegmentLengthSum(const std::string & name, const DataTypes & arguments, const Array &, const Settings *)
createAggregateFunctionIntervalLengthSum(const std::string & name, const DataTypes & arguments, const Array &, const Settings *)
{
if (arguments.size() != 2)
throw Exception(
@ -32,35 +32,35 @@ namespace
if (WhichDataType{args.begin()[0]}.idx != WhichDataType{args.begin()[1]}.idx)
throw Exception(
"Illegal type " + args.begin()[0]->getName() + " and " + args.begin()[1]->getName() + " of arguments of aggregate function "
+ name + ", there two arguments should have same DataType",
"Illegal types " + args.begin()[0]->getName() + " and " + args.begin()[1]->getName() + " of arguments of aggregate function "
+ name + ", both arguments should have same data type",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
for (const auto & arg : args)
{
if (!isNativeNumber(arg) && !isDate(arg) && !isDateTime(arg) && !isDateTime64(arg))
if (!isNativeNumber(arg) && !isDate(arg) && !isDateTime(arg))
throw Exception(
"Illegal type " + arg->getName() + " of argument of aggregate function " + name
+ ", must be Number, Date, DateTime or DateTime64",
+ ", must be native integral type, Date/DateTime or Float",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
AggregateFunctionPtr res(createWithBasicNumberOrDateOrDateTime<AggregateFunctionSegmentLengthSum, Data>(*arguments[0], arguments));
AggregateFunctionPtr res(createWithBasicNumberOrDateOrDateTime<AggregateFunctionIntervalLengthSum, Data>(*arguments[0], arguments));
if (res)
return res;
throw Exception(
"Illegal type " + arguments.front().get()->getName() + " of first argument of aggregate function " + name
+ ", must be Native Unsigned Number",
"Illegal type " + arguments.front().get()->getName() + " of argument of aggregate function " + name
+ ", must be native integral type, Date/DateTime or Float",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
void registerAggregateFunctionSegmentLengthSum(AggregateFunctionFactory & factory)
void registerAggregateFunctionIntervalLengthSum(AggregateFunctionFactory & factory)
{
factory.registerFunction("segmentLengthSum", createAggregateFunctionSegmentLengthSum<AggregateFunctionSegmentLengthSumData>);
factory.registerFunction("intervalLengthSum", createAggregateFunctionIntervalLengthSum<AggregateFunctionIntervalLengthSumData>);
}
}

View File

@ -1,22 +1,39 @@
#pragma once
#include <unordered_set>
#include <AggregateFunctions/AggregateFunctionNull.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/ArenaAllocator.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/AggregateFunctionNull.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <unordered_set>
namespace DB
{
template <typename T>
struct AggregateFunctionSegmentLengthSumData
namespace ErrorCodes
{
extern const int TOO_LARGE_ARRAY_SIZE;
}
/**
* Calculate total length of intervals without intersections. Each interval is the pair of numbers [begin, end];
* Return UInt64 for integral types (UInt/Int*, Date/DateTime) and return Float64 for Float*.
*
* Implementation simply stores intervals sorted by beginning and sums lengths at final.
*/
template <typename T>
struct AggregateFunctionIntervalLengthSumData
{
constexpr static size_t MAX_ARRAY_SIZE = 0xFFFFFF;
using Segment = std::pair<T, T>;
using Segments = PODArrayWithStackMemory<Segment, 64>;
@ -24,18 +41,16 @@ struct AggregateFunctionSegmentLengthSumData
Segments segments;
size_t size() const { return segments.size(); }
void add(T start, T end)
void add(T begin, T end)
{
if (sorted && segments.size() > 0)
if (sorted && !segments.empty())
{
sorted = segments.back().first <= start;
sorted = segments.back().first <= begin;
}
segments.emplace_back(start, end);
segments.emplace_back(begin, end);
}
void merge(const AggregateFunctionSegmentLengthSumData & other)
void merge(const AggregateFunctionIntervalLengthSumData & other)
{
if (other.segments.empty())
return;
@ -46,7 +61,9 @@ struct AggregateFunctionSegmentLengthSumData
/// either sort whole container or do so partially merging ranges afterwards
if (!sorted && !other.sorted)
std::stable_sort(std::begin(segments), std::end(segments));
{
std::sort(std::begin(segments), std::end(segments));
}
else
{
const auto begin = std::begin(segments);
@ -54,10 +71,10 @@ struct AggregateFunctionSegmentLengthSumData
const auto end = std::end(segments);
if (!sorted)
std::stable_sort(begin, middle);
std::sort(begin, middle);
if (!other.sorted)
std::stable_sort(middle, end);
std::sort(middle, end);
std::inplace_merge(begin, middle, end);
}
@ -69,7 +86,7 @@ struct AggregateFunctionSegmentLengthSumData
{
if (!sorted)
{
std::stable_sort(std::begin(segments), std::end(segments));
std::sort(std::begin(segments), std::end(segments));
sorted = true;
}
}
@ -93,28 +110,30 @@ struct AggregateFunctionSegmentLengthSumData
size_t size;
readBinary(size, buf);
if (unlikely(size > MAX_ARRAY_SIZE))
throw Exception("Too large array size", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
segments.clear();
segments.reserve(size);
T start, end;
Segment segment;
for (size_t i = 0; i < size; ++i)
{
readBinary(start, buf);
readBinary(end, buf);
segments.emplace_back(start, end);
readBinary(segment.first, buf);
readBinary(segment.second, buf);
segments.emplace_back(segment);
}
}
};
template <typename T, typename Data>
class AggregateFunctionSegmentLengthSum final : public IAggregateFunctionDataHelper<Data, AggregateFunctionSegmentLengthSum<T, Data>>
class AggregateFunctionIntervalLengthSum final : public IAggregateFunctionDataHelper<Data, AggregateFunctionIntervalLengthSum<T, Data>>
{
private:
template <typename TResult>
TResult getSegmentLengthSum(Data & data) const
TResult getIntervalLengthSum(Data & data) const
{
if (data.size() == 0)
if (data.segments.empty())
return 0;
data.sort();
@ -123,8 +142,9 @@ private:
typename Data::Segment cur_segment = data.segments[0];
for (size_t i = 1; i < data.segments.size(); ++i)
for (size_t i = 1, sz = data.segments.size(); i < sz; ++i)
{
/// Check if current interval intersect with next one then add length, otherwise advance interval end
if (cur_segment.second < data.segments[i].first)
{
res += cur_segment.second - cur_segment.first;
@ -140,10 +160,10 @@ private:
}
public:
String getName() const override { return "segmentLengthSum"; }
String getName() const override { return "intervalLengthSum"; }
explicit AggregateFunctionSegmentLengthSum(const DataTypes & arguments)
: IAggregateFunctionDataHelper<Data, AggregateFunctionSegmentLengthSum<T, Data>>(arguments, {})
explicit AggregateFunctionIntervalLengthSum(const DataTypes & arguments)
: IAggregateFunctionDataHelper<Data, AggregateFunctionIntervalLengthSum<T, Data>>(arguments, {})
{
}
@ -167,9 +187,9 @@ public:
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
auto start = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
auto begin = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
auto end = assert_cast<const ColumnVector<T> *>(columns[1])->getData()[row_num];
this->data(place).add(start, end);
this->data(place).add(begin, end);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
@ -190,9 +210,9 @@ public:
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
if constexpr (std::is_floating_point_v<T>)
assert_cast<ColumnFloat64 &>(to).getData().push_back(getSegmentLengthSum<Float64>(this->data(place)));
assert_cast<ColumnFloat64 &>(to).getData().push_back(getIntervalLengthSum<Float64>(this->data(place)));
else
assert_cast<ColumnUInt64 &>(to).getData().push_back(getSegmentLengthSum<UInt64>(this->data(place)));
assert_cast<ColumnUInt64 &>(to).getData().push_back(getIntervalLengthSum<UInt64>(this->data(place)));
}
};

View File

@ -637,7 +637,7 @@ struct AggregateFunctionAnyLastData : Data
template <typename Data>
struct AggregateFunctionAnyHeavyData : Data
{
size_t counter = 0;
UInt64 counter = 0;
using Self = AggregateFunctionAnyHeavyData;

View File

@ -5,11 +5,8 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeUUID.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -19,6 +16,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -40,6 +38,12 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types);
const WhichDataType t(argument_types[0]);
if (t.isAggregateFunction())
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (argument_types.size() == 1)
{
const IDataType & argument_type = *argument_types[0];
@ -82,6 +86,12 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const WhichDataType t(argument_types[0]);
if (t.isAggregateFunction())
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
/// We use exact hash function if the user wants it;
/// or if the arguments are not contiguous in memory, because only exact hash function have support for this case.
bool use_exact_hash_function = is_exact || !isAllArgumentsContiguousInMemory(argument_types);

View File

@ -19,6 +19,7 @@ namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
@ -104,6 +105,11 @@ namespace
if (argument_types.empty())
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const WhichDataType t(argument_types[0]);
if (t.isAggregateFunction())
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
switch (precision)
{

View File

@ -16,6 +16,7 @@ namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -46,6 +47,11 @@ AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, c
if (argument_types.empty())
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const WhichDataType t(argument_types[0]);
if (t.isAggregateFunction())
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types);

View File

@ -64,7 +64,7 @@ void registerAggregateFunctionCombinatorDistinct(AggregateFunctionCombinatorFact
void registerWindowFunctions(AggregateFunctionFactory & factory);
void registerAggregateFunctionSegmentLengthSum(AggregateFunctionFactory &);
void registerAggregateFunctionIntervalLengthSum(AggregateFunctionFactory &);
void registerAggregateFunctions()
{
@ -116,7 +116,7 @@ void registerAggregateFunctions()
registerWindowFunctions(factory);
registerAggregateFunctionSegmentLengthSum(factory);
registerAggregateFunctionIntervalLengthSum(factory);
}
{

View File

@ -31,6 +31,7 @@ SRCS(
AggregateFunctionGroupUniqArray.cpp
AggregateFunctionHistogram.cpp
AggregateFunctionIf.cpp
AggregateFunctionIntervalLengthSum.cpp
AggregateFunctionMLMethod.cpp
AggregateFunctionMannWhitney.cpp
AggregateFunctionMax.cpp
@ -43,7 +44,6 @@ SRCS(
AggregateFunctionRankCorrelation.cpp
AggregateFunctionResample.cpp
AggregateFunctionRetention.cpp
AggregateFunctionSegmentLengthSum.cpp
AggregateFunctionSequenceMatch.cpp
AggregateFunctionSequenceNextNode.cpp
AggregateFunctionSimpleLinearRegression.cpp

View File

@ -424,7 +424,7 @@ void Connection::sendQuery(
if (method == "ZSTD")
level = settings->network_zstd_compression_level;
CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs);
CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs);
compression_codec = CompressionCodecFactory::instance().get(method, level);
}
else

27
src/Common/IO.cpp Normal file
View File

@ -0,0 +1,27 @@
#include <Common/IO.h>
#include <unistd.h>
#include <errno.h>
#include <cstring>
bool writeRetry(int fd, const char * data, size_t size)
{
if (!size)
size = strlen(data);
while (size != 0)
{
ssize_t res = ::write(fd, data, size);
if ((-1 == res || 0 == res) && errno != EINTR)
return false;
if (res > 0)
{
data += res;
size -= res;
}
}
return true;
}

13
src/Common/IO.h Normal file
View File

@ -0,0 +1,13 @@
#pragma once
#include <cstddef>
/// IO helpers
/// Write loop with EINTR handling.
///
/// This function is safe to use in static initializers.
///
/// @param size - len of @data or 0 to use strlen()
/// @return true if write was succeed, otherwise false.
bool writeRetry(int fd, const char * data, size_t size = 0);

View File

@ -14,6 +14,6 @@
#cmakedefine01 USE_SENTRY
#cmakedefine01 USE_GRPC
#cmakedefine01 USE_STATS
#cmakedefine01 CLICKHOUSE_SPLIT_BINARY
#cmakedefine01 USE_DATASKETCHES
#cmakedefine01 USE_YAML_CPP
#cmakedefine01 CLICKHOUSE_SPLIT_BINARY

View File

@ -46,6 +46,7 @@ SRCS(
ExternalLoaderStatus.cpp
FieldVisitors.cpp
FileChecker.cpp
IO.cpp
IPv6ToBinary.cpp
IntervalKind.cpp
JSONBuilder.cpp

View File

@ -1,4 +1,4 @@
#include <Compression/CompressionCodecDelta.h>
#include <Compression/ICompressionCodec.h>
#include <Compression/CompressionInfo.h>
#include <Compression/CompressionFactory.h>
#include <common/unaligned.h>
@ -11,6 +11,29 @@
namespace DB
{
class CompressionCodecDelta : public ICompressionCodec
{
public:
explicit CompressionCodecDelta(UInt8 delta_bytes_size_);
uint8_t getMethodByte() const override;
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override { return uncompressed_size + 2; }
bool isCompression() const override { return false; }
bool isGenericCompression() const override { return false; }
private:
UInt8 delta_bytes_size;
};
namespace ErrorCodes
{
extern const int CANNOT_COMPRESS;

View File

@ -1,32 +0,0 @@
#pragma once
#include <Compression/ICompressionCodec.h>
namespace DB
{
class CompressionCodecDelta : public ICompressionCodec
{
public:
CompressionCodecDelta(UInt8 delta_bytes_size_);
uint8_t getMethodByte() const override;
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override { return uncompressed_size + 2; }
bool isCompression() const override { return false; }
bool isGenericCompression() const override { return false; }
private:
UInt8 delta_bytes_size;
};
}

View File

@ -1,4 +1,4 @@
#include <Compression/CompressionCodecDoubleDelta.h>
#include <Compression/ICompressionCodec.h>
#include <Compression/CompressionInfo.h>
#include <Compression/CompressionFactory.h>
#include <common/unaligned.h>
@ -15,9 +15,126 @@
#include <type_traits>
#include <limits>
namespace DB
{
/** NOTE DoubleDelta is surprisingly bad name. The only excuse is that it comes from an academic paper.
* Most people will think that "double delta" is just applying delta transform twice.
* But in fact it is something more than applying delta transform twice.
*/
/** DoubleDelta column codec implementation.
*
* Based on Gorilla paper: http://www.vldb.org/pvldb/vol8/p1816-teller.pdf, which was extended
* to support 64bit types. The drawback is 1 extra bit for 32-byte wide deltas: 5-bit prefix
* instead of 4-bit prefix.
*
* This codec is best used against monotonic integer sequences with constant (or almost constant)
* stride, like event timestamp for some monitoring application.
*
* Given input sequence a: [a0, a1, ... an]:
*
* First, write number of items (sizeof(int32)*8 bits): n
* Then write first item as is (sizeof(a[0])*8 bits): a[0]
* Second item is written as delta (sizeof(a[0])*8 bits): a[1] - a[0]
* Loop over remaining items and calculate double delta:
* double_delta = a[i] - 2 * a[i - 1] + a[i - 2]
* Write it in compact binary form with `BitWriter`
* if double_delta == 0:
* write 1bit: 0
* else if -63 < double_delta < 64:
* write 2 bit prefix: 10
* write sign bit (1 if signed): x
* write 7-1 bits of abs(double_delta - 1): xxxxxx
* else if -255 < double_delta < 256:
* write 3 bit prefix: 110
* write sign bit (1 if signed): x
* write 9-1 bits of abs(double_delta - 1): xxxxxxxx
* else if -2047 < double_delta < 2048:
* write 4 bit prefix: 1110
* write sign bit (1 if signed): x
* write 12-1 bits of abs(double_delta - 1): xxxxxxxxxxx
* else if double_delta fits into 32-bit int:
* write 5 bit prefix: 11110
* write sign bit (1 if signed): x
* write 32-1 bits of abs(double_delta - 1): xxxxxxxxxxx...
* else
* write 5 bit prefix: 11111
* write sign bit (1 if signed): x
* write 64-1 bits of abs(double_delta - 1): xxxxxxxxxxx...
*
* @example sequence of UInt8 values [1, 2, 3, 4, 5, 6, 7, 8, 9 10] is encoded as (codec header is omitted):
*
* .- 4-byte little-endian sequence length (10 == 0xa)
* | .- 1 byte (sizeof(UInt8) a[0] : 0x01
* | | .- 1 byte of delta: a[1] - a[0] = 2 - 1 = 1 : 0x01
* | | | .- 8 zero bits since double delta for remaining 8 elements was 0 : 0x00
* v_______________v___v___v___
* \x0a\x00\x00\x00\x01\x01\x00
*
* @example sequence of Int16 values [-10, 10, -20, 20, -40, 40] is encoded as:
*
* .- 4-byte little endian sequence length = 6 : 0x00000006
* | .- 2 bytes (sizeof(Int16) a[0] as UInt16 = -10 : 0xfff6
* | | .- 2 bytes of delta: a[1] - a[0] = 10 - (-10) = 20 : 0x0014
* | | | .- 4 encoded double deltas (see below)
* v_______________ v______ v______ v______________________
* \x06\x00\x00\x00\xf6\xff\x14\x00\xb8\xe2\x2e\xb1\xe4\x58
*
* 4 binary encoded double deltas (\xb8\xe2\x2e\xb1\xe4\x58):
* double_delta (DD) = -20 - 2 * 10 + (-10) = -50
* .- 2-bit prefix : 0b10
* | .- sign-bit : 0b1
* | |.- abs(DD - 1) = 49 : 0b110001
* | ||
* | || DD = 20 - 2 * (-20) + 10 = 70
* | || .- 3-bit prefix : 0b110
* | || | .- sign bit : 0b0
* | || | |.- abs(DD - 1) = 69 : 0b1000101
* | || | ||
* | || | || DD = -40 - 2 * 20 + (-20) = -100
* | || | || .- 3-bit prefix : 0b110
* | || | || | .- sign-bit : 0b0
* | || | || | |.- abs(DD - 1) = 99 : 0b1100011
* | || | || | ||
* | || | || | || DD = 40 - 2 * (-40) + 20 = 140
* | || | || | || .- 3-bit prefix : 0b110
* | || | || | || | .- sign bit : 0b0
* | || | || | || | |.- abs(DD - 1) = 139 : 0b10001011
* | || | || | || | ||
* V_vv______V__vv________V____vv_______V__vv________,- padding bits
* 10111000 11100010 00101110 10110001 11100100 01011000
*
* Please also see unit tests for:
* * Examples on what output `BitWriter` produces on predefined input.
* * Compatibility tests solidifying encoded binary output on set of predefined sequences.
*/
class CompressionCodecDoubleDelta : public ICompressionCodec
{
public:
explicit CompressionCodecDoubleDelta(UInt8 data_bytes_size_);
uint8_t getMethodByte() const override;
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
private:
UInt8 data_bytes_size;
};
namespace ErrorCodes
{
extern const int CANNOT_COMPRESS;

View File

@ -1,118 +0,0 @@
#pragma once
#include <Compression/ICompressionCodec.h>
namespace DB
{
/** DoubleDelta column codec implementation.
*
* Based on Gorilla paper: http://www.vldb.org/pvldb/vol8/p1816-teller.pdf, which was extended
* to support 64bit types. The drawback is 1 extra bit for 32-byte wide deltas: 5-bit prefix
* instead of 4-bit prefix.
*
* This codec is best used against monotonic integer sequences with constant (or almost constant)
* stride, like event timestamp for some monitoring application.
*
* Given input sequence a: [a0, a1, ... an]:
*
* First, write number of items (sizeof(int32)*8 bits): n
* Then write first item as is (sizeof(a[0])*8 bits): a[0]
* Second item is written as delta (sizeof(a[0])*8 bits): a[1] - a[0]
* Loop over remaining items and calculate double delta:
* double_delta = a[i] - 2 * a[i - 1] + a[i - 2]
* Write it in compact binary form with `BitWriter`
* if double_delta == 0:
* write 1bit: 0
* else if -63 < double_delta < 64:
* write 2 bit prefix: 10
* write sign bit (1 if signed): x
* write 7-1 bits of abs(double_delta - 1): xxxxxx
* else if -255 < double_delta < 256:
* write 3 bit prefix: 110
* write sign bit (1 if signed): x
* write 9-1 bits of abs(double_delta - 1): xxxxxxxx
* else if -2047 < double_delta < 2048:
* write 4 bit prefix: 1110
* write sign bit (1 if signed): x
* write 12-1 bits of abs(double_delta - 1): xxxxxxxxxxx
* else if double_delta fits into 32-bit int:
* write 5 bit prefix: 11110
* write sign bit (1 if signed): x
* write 32-1 bits of abs(double_delta - 1): xxxxxxxxxxx...
* else
* write 5 bit prefix: 11111
* write sign bit (1 if signed): x
* write 64-1 bits of abs(double_delta - 1): xxxxxxxxxxx...
*
* @example sequence of UInt8 values [1, 2, 3, 4, 5, 6, 7, 8, 9 10] is encoded as (codec header is omitted):
*
* .- 4-byte little-endian sequence length (10 == 0xa)
* | .- 1 byte (sizeof(UInt8) a[0] : 0x01
* | | .- 1 byte of delta: a[1] - a[0] = 2 - 1 = 1 : 0x01
* | | | .- 8 zero bits since double delta for remaining 8 elements was 0 : 0x00
* v_______________v___v___v___
* \x0a\x00\x00\x00\x01\x01\x00
*
* @example sequence of Int16 values [-10, 10, -20, 20, -40, 40] is encoded as:
*
* .- 4-byte little endian sequence length = 6 : 0x00000006
* | .- 2 bytes (sizeof(Int16) a[0] as UInt16 = -10 : 0xfff6
* | | .- 2 bytes of delta: a[1] - a[0] = 10 - (-10) = 20 : 0x0014
* | | | .- 4 encoded double deltas (see below)
* v_______________ v______ v______ v______________________
* \x06\x00\x00\x00\xf6\xff\x14\x00\xb8\xe2\x2e\xb1\xe4\x58
*
* 4 binary encoded double deltas (\xb8\xe2\x2e\xb1\xe4\x58):
* double_delta (DD) = -20 - 2 * 10 + (-10) = -50
* .- 2-bit prefix : 0b10
* | .- sign-bit : 0b1
* | |.- abs(DD - 1) = 49 : 0b110001
* | ||
* | || DD = 20 - 2 * (-20) + 10 = 70
* | || .- 3-bit prefix : 0b110
* | || | .- sign bit : 0b0
* | || | |.- abs(DD - 1) = 69 : 0b1000101
* | || | ||
* | || | || DD = -40 - 2 * 20 + (-20) = -100
* | || | || .- 3-bit prefix : 0b110
* | || | || | .- sign-bit : 0b0
* | || | || | |.- abs(DD - 1) = 99 : 0b1100011
* | || | || | ||
* | || | || | || DD = 40 - 2 * (-40) + 20 = 140
* | || | || | || .- 3-bit prefix : 0b110
* | || | || | || | .- sign bit : 0b0
* | || | || | || | |.- abs(DD - 1) = 139 : 0b10001011
* | || | || | || | ||
* V_vv______V__vv________V____vv_______V__vv________,- padding bits
* 10111000 11100010 00101110 10110001 11100100 01011000
*
* Please also see unit tests for:
* * Examples on what output `BitWriter` produces on predefined input.
* * Compatibility tests solidifying encoded binary output on set of predefined sequences.
*/
class CompressionCodecDoubleDelta : public ICompressionCodec
{
public:
CompressionCodecDoubleDelta(UInt8 data_bytes_size_);
uint8_t getMethodByte() const override;
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
private:
UInt8 data_bytes_size;
};
}

View File

@ -1,4 +1,4 @@
#include <Compression/CompressionCodecGorilla.h>
#include <Compression/ICompressionCodec.h>
#include <Compression/CompressionInfo.h>
#include <Compression/CompressionFactory.h>
#include <common/unaligned.h>
@ -14,9 +14,118 @@
#include <bitset>
namespace DB
{
/** Gorilla column codec implementation.
*
* Based on Gorilla paper: http://www.vldb.org/pvldb/vol8/p1816-teller.pdf
*
* This codec is best used against monotonic floating sequences, like CPU usage percentage
* or any other gauge.
*
* Given input sequence a: [a0, a1, ... an]
*
* First, write number of items (sizeof(int32)*8 bits): n
* Then write first item as is (sizeof(a[0])*8 bits): a[0]
* Loop over remaining items and calculate xor_diff:
* xor_diff = a[i] ^ a[i - 1] (e.g. 00000011'10110100)
* Write it in compact binary form with `BitWriter`
* if xor_diff == 0:
* write 1 bit: 0
* else:
* calculate leading zero bits (lzb)
* and trailing zero bits (tzb) of xor_diff,
* compare to lzb and tzb of previous xor_diff
* (X = sizeof(a[i]) * 8, e.g. X = 16, lzb = 6, tzb = 2)
* if lzb >= prev_lzb && tzb >= prev_tzb:
* (e.g. prev_lzb=4, prev_tzb=1)
* write 2 bit prefix: 0b10
* write xor_diff >> prev_tzb (X - prev_lzb - prev_tzb bits):0b00111011010
* (where X = sizeof(a[i]) * 8, e.g. 16)
* else:
* write 2 bit prefix: 0b11
* write 5 bits of lzb: 0b00110
* write 6 bits of (X - lzb - tzb)=(16-6-2)=8: 0b001000
* write (X - lzb - tzb) non-zero bits of xor_diff: 0b11101101
* prev_lzb = lzb
* prev_tzb = tzb
*
* @example sequence of Float32 values [0.1, 0.1, 0.11, 0.2, 0.1] is encoded as:
*
* .- 4-byte little endian sequence length: 5 : 0x00000005
* | .- 4 byte (sizeof(Float32) a[0] as UInt32 : -10 : 0xcdcccc3d
* | | .- 4 encoded xor diffs (see below)
* v_______________ v______________ v__________________________________________________
* \x05\x00\x00\x00\xcd\xcc\xcc\x3d\x6a\x5a\xd8\xb6\x3c\xcd\x75\xb1\x6c\x77\x00\x00\x00
*
* 4 binary encoded xor diffs (\x6a\x5a\xd8\xb6\x3c\xcd\x75\xb1\x6c\x77\x00\x00\x00):
*
* ...........................................
* a[i-1] = 00111101110011001100110011001101
* a[i] = 00111101110011001100110011001101
* xor_diff = 00000000000000000000000000000000
* .- 1-bit prefix : 0b0
* |
* | ...........................................
* | a[i-1] = 00111101110011001100110011001101
* ! a[i] = 00111101111000010100011110101110
* | xor_diff = 00000000001011011000101101100011
* | lzb = 10
* | tzb = 0
* |.- 2-bit prefix : 0b11
* || .- lzb (10) : 0b1010
* || | .- data length (32-10-0): 22 : 0b010110
* || | | .- data : 0b1011011000101101100011
* || | | |
* || | | | ...........................................
* || | | | a[i-1] = 00111101111000010100011110101110
* || | | | a[i] = 00111110010011001100110011001101
* || | | | xor_diff = 00000011101011011000101101100011
* || | | | .- 2-bit prefix : 0b11
* || | | | | .- lzb = 6 : 0b00110
* || | | | | | .- data length = (32 - 6) = 26 : 0b011010
* || | | | | | | .- data : 0b11101011011000101101100011
* || | | | | | | |
* || | | | | | | | ...........................................
* || | | | | | | | a[i-1] = 00111110010011001100110011001101
* || | | | | | | | a[i] = 00111101110011001100110011001101
* || | | | | | | | xor_diff = 00000011100000000000000000000000
* || | | | | | | | .- 2-bit prefix : 0b10
* || | | | | | | | | .- data : 0b11100000000000000000000000
* VV_v____ v_____v________________________V_v_____v______v____________________________V_v_____________________________
* 01101010 01011010 11011000 10110110 00111100 11001101 01110101 10110001 01101100 01110111 00000000 00000000 00000000
*
* Please also see unit tests for:
* * Examples on what output `BitWriter` produces on predefined input.
* * Compatibility tests solidifying encoded binary output on set of predefined sequences.
*/
class CompressionCodecGorilla : public ICompressionCodec
{
public:
explicit CompressionCodecGorilla(UInt8 data_bytes_size_);
uint8_t getMethodByte() const override;
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
private:
UInt8 data_bytes_size;
};
namespace ErrorCodes
{
extern const int CANNOT_COMPRESS;

View File

@ -1,115 +0,0 @@
#pragma once
#include <Compression/ICompressionCodec.h>
namespace DB
{
/** Gorilla column codec implementation.
*
* Based on Gorilla paper: http://www.vldb.org/pvldb/vol8/p1816-teller.pdf
*
* This codec is best used against monotonic floating sequences, like CPU usage percentage
* or any other gauge.
*
* Given input sequence a: [a0, a1, ... an]
*
* First, write number of items (sizeof(int32)*8 bits): n
* Then write first item as is (sizeof(a[0])*8 bits): a[0]
* Loop over remaining items and calculate xor_diff:
* xor_diff = a[i] ^ a[i - 1] (e.g. 00000011'10110100)
* Write it in compact binary form with `BitWriter`
* if xor_diff == 0:
* write 1 bit: 0
* else:
* calculate leading zero bits (lzb)
* and trailing zero bits (tzb) of xor_diff,
* compare to lzb and tzb of previous xor_diff
* (X = sizeof(a[i]) * 8, e.g. X = 16, lzb = 6, tzb = 2)
* if lzb >= prev_lzb && tzb >= prev_tzb:
* (e.g. prev_lzb=4, prev_tzb=1)
* write 2 bit prefix: 0b10
* write xor_diff >> prev_tzb (X - prev_lzb - prev_tzb bits):0b00111011010
* (where X = sizeof(a[i]) * 8, e.g. 16)
* else:
* write 2 bit prefix: 0b11
* write 5 bits of lzb: 0b00110
* write 6 bits of (X - lzb - tzb)=(16-6-2)=8: 0b001000
* write (X - lzb - tzb) non-zero bits of xor_diff: 0b11101101
* prev_lzb = lzb
* prev_tzb = tzb
*
* @example sequence of Float32 values [0.1, 0.1, 0.11, 0.2, 0.1] is encoded as:
*
* .- 4-byte little endian sequence length: 5 : 0x00000005
* | .- 4 byte (sizeof(Float32) a[0] as UInt32 : -10 : 0xcdcccc3d
* | | .- 4 encoded xor diffs (see below)
* v_______________ v______________ v__________________________________________________
* \x05\x00\x00\x00\xcd\xcc\xcc\x3d\x6a\x5a\xd8\xb6\x3c\xcd\x75\xb1\x6c\x77\x00\x00\x00
*
* 4 binary encoded xor diffs (\x6a\x5a\xd8\xb6\x3c\xcd\x75\xb1\x6c\x77\x00\x00\x00):
*
* ...........................................
* a[i-1] = 00111101110011001100110011001101
* a[i] = 00111101110011001100110011001101
* xor_diff = 00000000000000000000000000000000
* .- 1-bit prefix : 0b0
* |
* | ...........................................
* | a[i-1] = 00111101110011001100110011001101
* ! a[i] = 00111101111000010100011110101110
* | xor_diff = 00000000001011011000101101100011
* | lzb = 10
* | tzb = 0
* |.- 2-bit prefix : 0b11
* || .- lzb (10) : 0b1010
* || | .- data length (32-10-0): 22 : 0b010110
* || | | .- data : 0b1011011000101101100011
* || | | |
* || | | | ...........................................
* || | | | a[i-1] = 00111101111000010100011110101110
* || | | | a[i] = 00111110010011001100110011001101
* || | | | xor_diff = 00000011101011011000101101100011
* || | | | .- 2-bit prefix : 0b11
* || | | | | .- lzb = 6 : 0b00110
* || | | | | | .- data length = (32 - 6) = 26 : 0b011010
* || | | | | | | .- data : 0b11101011011000101101100011
* || | | | | | | |
* || | | | | | | | ...........................................
* || | | | | | | | a[i-1] = 00111110010011001100110011001101
* || | | | | | | | a[i] = 00111101110011001100110011001101
* || | | | | | | | xor_diff = 00000011100000000000000000000000
* || | | | | | | | .- 2-bit prefix : 0b10
* || | | | | | | | | .- data : 0b11100000000000000000000000
* VV_v____ v_____v________________________V_v_____v______v____________________________V_v_____________________________
* 01101010 01011010 11011000 10110110 00111100 11001101 01110101 10110001 01101100 01110111 00000000 00000000 00000000
*
* Please also see unit tests for:
* * Examples on what output `BitWriter` produces on predefined input.
* * Compatibility tests solidifying encoded binary output on set of predefined sequences.
*/
class CompressionCodecGorilla : public ICompressionCodec
{
public:
CompressionCodecGorilla(UInt8 data_bytes_size_);
uint8_t getMethodByte() const override;
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
private:
UInt8 data_bytes_size;
};
}

View File

@ -1,7 +1,7 @@
#include "CompressionCodecLZ4.h"
#include <lz4.h>
#include <lz4hc.h>
#include <Compression/ICompressionCodec.h>
#include <Compression/CompressionInfo.h>
#include <Compression/CompressionFactory.h>
#include <Compression/LZ4_decompress_faster.h>
@ -9,7 +9,9 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/BufferWithOwnMemory.h>
#pragma GCC diagnostic ignored "-Wold-style-cast"
@ -17,11 +19,51 @@
namespace DB
{
class CompressionCodecLZ4 : public ICompressionCodec
{
public:
explicit CompressionCodecLZ4();
uint8_t getMethodByte() const override;
UInt32 getAdditionalSizeAtTheEndOfBuffer() const override { return LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER; }
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return true; }
private:
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
mutable LZ4::PerformanceStatistics lz4_stat;
ASTPtr codec_desc;
};
class CompressionCodecLZ4HC : public CompressionCodecLZ4
{
public:
explicit CompressionCodecLZ4HC(int level_);
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
private:
const int level;
};
namespace ErrorCodes
{
extern const int CANNOT_COMPRESS;
extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
extern const int ILLEGAL_CODEC_PARAMETER;
extern const int CANNOT_COMPRESS;
extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
extern const int ILLEGAL_CODEC_PARAMETER;
}
CompressionCodecLZ4::CompressionCodecLZ4()

View File

@ -1,52 +0,0 @@
#pragma once
#include <IO/WriteBuffer.h>
#include <Compression/ICompressionCodec.h>
#include <IO/BufferWithOwnMemory.h>
#include <Parsers/StringRange.h>
#include <Compression/LZ4_decompress_faster.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
class CompressionCodecLZ4 : public ICompressionCodec
{
public:
CompressionCodecLZ4();
uint8_t getMethodByte() const override;
UInt32 getAdditionalSizeAtTheEndOfBuffer() const override { return LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER; }
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return true; }
private:
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
mutable LZ4::PerformanceStatistics lz4_stat;
ASTPtr codec_desc;
};
class CompressionCodecLZ4HC : public CompressionCodecLZ4
{
public:
CompressionCodecLZ4HC(int level_);
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
private:
const int level;
};
}

View File

@ -1,6 +1,6 @@
#include <cstring>
#include <Compression/CompressionCodecT64.h>
#include <Compression/ICompressionCodec.h>
#include <Compression/CompressionFactory.h>
#include <common/unaligned.h>
#include <Parsers/IAST.h>
@ -8,18 +8,63 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <IO/WriteHelpers.h>
#include <Core/Types.h>
namespace DB
{
/// Get 64 integer values, makes 64x64 bit matrix, transpose it and crop unused bits (most significant zeroes).
/// In example, if we have UInt8 with only 0 and 1 inside 64xUInt8 would be compressed into 1xUInt64.
/// It detects unused bits by calculating min and max values of data part, saving them in header in compression phase.
/// There's a special case with signed integers parts with crossing zero data. Here it stores one more bit to detect sign of value.
class CompressionCodecT64 : public ICompressionCodec
{
public:
static constexpr UInt32 HEADER_SIZE = 1 + 2 * sizeof(UInt64);
static constexpr UInt32 MAX_COMPRESSED_BLOCK_SIZE = sizeof(UInt64) * 64;
/// There're 2 compression variants:
/// Byte - transpose bit matrix by bytes (only the last not full byte is transposed by bits). It's default.
/// Bits - full bit-transpose of the bit matrix. It uses more resources and leads to better compression with ZSTD (but worse with LZ4).
enum class Variant
{
Byte,
Bit
};
CompressionCodecT64(TypeIndex type_idx_, Variant variant_);
uint8_t getMethodByte() const override;
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * src, UInt32 src_size, char * dst) const override;
void doDecompressData(const char * src, UInt32 src_size, char * dst, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override
{
/// uncompressed_size - (uncompressed_size % (sizeof(T) * 64)) + sizeof(UInt64) * sizeof(T) + header_size
return uncompressed_size + MAX_COMPRESSED_BLOCK_SIZE + HEADER_SIZE;
}
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
private:
TypeIndex type_idx;
Variant variant;
};
namespace ErrorCodes
{
extern const int CANNOT_COMPRESS;
extern const int CANNOT_DECOMPRESS;
extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
extern const int ILLEGAL_CODEC_PARAMETER;
extern const int LOGICAL_ERROR;
extern const int CANNOT_COMPRESS;
extern const int CANNOT_DECOMPRESS;
extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
extern const int ILLEGAL_CODEC_PARAMETER;
extern const int LOGICAL_ERROR;
}
namespace

View File

@ -1,53 +0,0 @@
#pragma once
#include <Core/Types.h>
#include <Compression/ICompressionCodec.h>
namespace DB
{
/// Get 64 integer values, makes 64x64 bit matrix, transpose it and crop unused bits (most significant zeroes).
/// In example, if we have UInt8 with only 0 and 1 inside 64xUInt8 would be compressed into 1xUInt64.
/// It detects unused bits by calculating min and max values of data part, saving them in header in compression phase.
/// There's a special case with signed integers parts with crossing zero data. Here it stores one more bit to detect sign of value.
class CompressionCodecT64 : public ICompressionCodec
{
public:
static constexpr UInt32 HEADER_SIZE = 1 + 2 * sizeof(UInt64);
static constexpr UInt32 MAX_COMPRESSED_BLOCK_SIZE = sizeof(UInt64) * 64;
/// There're 2 compression variants:
/// Byte - transpose bit matrix by bytes (only the last not full byte is transposed by bits). It's default.
/// Bits - full bit-transpose of the bit matrix. It uses more resources and leads to better compression with ZSTD (but worse with LZ4).
enum class Variant
{
Byte,
Bit
};
CompressionCodecT64(TypeIndex type_idx_, Variant variant_);
uint8_t getMethodByte() const override;
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * src, UInt32 src_size, char * dst) const override;
void doDecompressData(const char * src, UInt32 src_size, char * dst, UInt32 uncompressed_size) const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override
{
/// uncompressed_size - (uncompressed_size % (sizeof(T) * 64)) + sizeof(UInt64) * sizeof(T) + header_size
return uncompressed_size + MAX_COMPRESSED_BLOCK_SIZE + HEADER_SIZE;
}
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return false; }
private:
TypeIndex type_idx;
Variant variant;
};
}

View File

@ -1,4 +1,4 @@
#include <Compression/CompressionCodecZSTD.h>
#include <Compression/ICompressionCodec.h>
#include <Compression/CompressionInfo.h>
#include <Compression/CompressionFactory.h>
#include <zstd.h>
@ -7,11 +7,44 @@
#include <Parsers/ASTFunction.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBuffer.h>
#include <IO/BufferWithOwnMemory.h>
namespace DB
{
class CompressionCodecZSTD : public ICompressionCodec
{
public:
static constexpr auto ZSTD_DEFAULT_LEVEL = 1;
static constexpr auto ZSTD_DEFAULT_LOG_WINDOW = 24;
explicit CompressionCodecZSTD(int level_);
CompressionCodecZSTD(int level_, int window_log);
uint8_t getMethodByte() const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return true; }
private:
const int level;
const bool enable_long_range;
const int window_log;
};
namespace ErrorCodes
{
extern const int CANNOT_COMPRESS;

View File

@ -1,42 +0,0 @@
#pragma once
#include <IO/WriteBuffer.h>
#include <Compression/ICompressionCodec.h>
#include <IO/BufferWithOwnMemory.h>
#include <Parsers/StringRange.h>
namespace DB
{
class CompressionCodecZSTD : public ICompressionCodec
{
public:
static constexpr auto ZSTD_DEFAULT_LEVEL = 1;
static constexpr auto ZSTD_DEFAULT_LOG_WINDOW = 24;
CompressionCodecZSTD(int level_);
CompressionCodecZSTD(int level_, int window_log);
uint8_t getMethodByte() const override;
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
void updateHash(SipHash & hash) const override;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
bool isCompression() const override { return true; }
bool isGenericCompression() const override { return true; }
private:
const int level;
const bool enable_long_range;
const int window_log;
};
}

View File

@ -1,3 +1,7 @@
#if !defined(ARCADIA_BUILD)
# include "config_core.h"
#endif
#include <Compression/CompressionFactory.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
@ -13,6 +17,7 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
@ -34,8 +39,8 @@ CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std
{
if (level)
{
auto literal = std::make_shared<ASTLiteral>(static_cast<UInt64>(*level));
return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), {});
auto level_literal = std::make_shared<ASTLiteral>(static_cast<UInt64>(*level));
return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), level_literal)), {});
}
else
{
@ -44,7 +49,8 @@ CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std
}
}
void CompressionCodecFactory::validateCodec(const String & family_name, std::optional<int> level, bool sanity_check) const
void CompressionCodecFactory::validateCodec(
const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs) const
{
if (family_name.empty())
throw Exception("Compression codec name cannot be empty", ErrorCodes::BAD_ARGUMENTS);
@ -52,16 +58,19 @@ void CompressionCodecFactory::validateCodec(const String & family_name, std::opt
if (level)
{
auto literal = std::make_shared<ASTLiteral>(static_cast<UInt64>(*level));
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), {}, sanity_check);
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)),
{}, sanity_check, allow_experimental_codecs);
}
else
{
auto identifier = std::make_shared<ASTIdentifier>(Poco::toUpper(family_name));
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), {}, sanity_check);
validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier),
{}, sanity_check, allow_experimental_codecs);
}
}
ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check) const
ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const
{
if (const auto * func = ast->as<ASTFunction>())
{
@ -72,7 +81,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr
std::optional<size_t> generic_compression_codec_pos;
bool can_substitute_codec_arguments = true;
for (size_t i = 0; i < func->arguments->children.size(); ++i)
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
{
const auto & inner_codec_ast = func->arguments->children[i];
String codec_family_name;
@ -107,7 +116,8 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr
if (column_type)
{
CompressionCodecPtr prev_codec;
IDataType::StreamCallbackWithType callback = [&](const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type)
IDataType::StreamCallbackWithType callback = [&](
const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type)
{
if (ISerialization::isSpecialCompressionAllowed(substream_path))
{
@ -132,6 +142,12 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr
result_codec = getImpl(codec_family_name, codec_arguments, nullptr);
}
if (!allow_experimental_codecs && result_codec->isExperimental())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Codec {} is experimental and not meant to be used in production."
" You can enable it with the 'allow_experimental_codecs' setting.",
codec_family_name);
codecs_descriptions->children.emplace_back(result_codec->getCodecDesc());
}
@ -172,6 +188,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS);
}
/// For columns with nested types like Tuple(UInt32, UInt64) we
/// obviously cannot substitute parameters for codecs which depend on
/// data type, because for the first column Delta(4) is suitable and
@ -195,7 +212,9 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr
throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC);
}
CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default, bool only_generic) const
CompressionCodecPtr CompressionCodecFactory::get(
const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default, bool only_generic) const
{
if (current_default == nullptr)
current_default = default_codec;
@ -246,6 +265,7 @@ CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, const IData
throw Exception("Unexpected AST structure for compression codec: " + queryToString(ast), ErrorCodes::UNEXPECTED_AST_STRUCTURE);
}
CompressionCodecPtr CompressionCodecFactory::get(const uint8_t byte_code) const
{
const auto family_code_and_creator = family_code_with_codec.find(byte_code);
@ -303,7 +323,7 @@ void CompressionCodecFactory::registerSimpleCompressionCodec(
registerCompressionCodec(family_name, byte_code, [family_name, creator](const ASTPtr & ast)
{
if (ast)
throw Exception("Compression codec " + family_name + " cannot have arguments", ErrorCodes::DATA_TYPE_CANNOT_HAVE_ARGUMENTS);
throw Exception(ErrorCodes::DATA_TYPE_CANNOT_HAVE_ARGUMENTS, "Compression codec {} cannot have arguments", family_name);
return creator();
});
}

View File

@ -38,16 +38,16 @@ public:
CompressionCodecPtr getDefaultCodec() const;
/// Validate codecs AST specified by user and parses codecs description (substitute default parameters)
ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check) const;
ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const;
/// Just wrapper for previous method.
ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check) const
ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs) const
{
return validateCodecAndGetPreprocessedAST(ast, column_type.get(), sanity_check);
return validateCodecAndGetPreprocessedAST(ast, column_type.get(), sanity_check, allow_experimental_codecs);
}
/// Validate codecs AST specified by user
void validateCodec(const String & family_name, std::optional<int> level, bool sanity_check) const;
void validateCodec(const String & family_name, std::optional<int> level, bool sanity_check, bool allow_experimental_codecs) const;
/// Get codec by AST and possible column_type. Some codecs can use
/// information about type to improve inner settings, but every codec should

View File

@ -41,8 +41,8 @@ enum class CompressionMethodByte : uint8_t
Multiple = 0x91,
Delta = 0x92,
T64 = 0x93,
DoubleDelta = 0x94,
Gorilla = 0x95,
DoubleDelta = 0x94,
Gorilla = 0x95,
};
}

View File

@ -73,6 +73,10 @@ public:
/// Is it a generic compression algorithm like lz4, zstd. Usually it does not make sense to apply generic compression more than single time.
virtual bool isGenericCompression() const = 0;
/// It is a codec available only for evaluation purposes and not meant to be used in production.
/// It will not be allowed to use unless the user will turn off the safety switch.
virtual bool isExperimental() const { return false; }
/// If it does nothing.
virtual bool isNone() const { return false; }

View File

@ -724,8 +724,9 @@ public:
Int32 size() const override
{
Int32 sz = 4 + 2; // size of message + number of fields
/// If values is NULL, field size is -1 and data not added.
for (const std::shared_ptr<ISerializable> & field : row)
sz += 4 + field->size();
sz += 4 + (field->size() > 0 ? field->size() : 0);
return sz;
}

View File

@ -80,7 +80,7 @@ class IColumn;
M(UInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \
M(UInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \
M(UInt64, background_fetches_pool_size, 8, "Number of threads performing background fetches for replicated tables. Only has meaning at server startup.", 0) \
M(UInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables, dns cache updates. Only has meaning at server startup.", 0) \
M(UInt64, background_schedule_pool_size, 128, "Number of threads performing background tasks for replicated tables, dns cache updates. Only has meaning at server startup.", 0) \
M(UInt64, background_message_broker_schedule_pool_size, 16, "Number of threads performing background tasks for message streaming. Only has meaning at server startup.", 0) \
M(UInt64, background_distributed_schedule_pool_size, 16, "Number of threads performing background tasks for distributed sends. Only has meaning at server startup.", 0) \
M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited. Only has meaning at server startup.", 0) \
@ -240,6 +240,7 @@ class IColumn;
M(Bool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \
M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \
M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \
M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \
M(UInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.", 0) \
M(UInt64, query_profiler_real_time_period_ns, 1000000000, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
M(UInt64, query_profiler_cpu_time_period_ns, 1000000000, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
@ -402,6 +403,7 @@ class IColumn;
M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \
M(Bool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \
M(Bool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 0) \
M(Bool, optimize_functions_to_subcolumns, false, "Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ", 0) \
M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \
M(Bool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \
M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \

View File

@ -121,25 +121,25 @@ public:
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
const String & bucket_,
DiskS3::Metadata metadata_,
size_t s3_max_single_read_retries_,
size_t max_single_read_retries_,
size_t buf_size_)
: ReadIndirectBufferFromRemoteFS<ReadBufferFromS3>(metadata_)
, client_ptr(std::move(client_ptr_))
, bucket(bucket_)
, s3_max_single_read_retries(s3_max_single_read_retries_)
, max_single_read_retries(max_single_read_retries_)
, buf_size(buf_size_)
{
}
std::unique_ptr<ReadBufferFromS3> createReadBuffer(const String & path) override
{
return std::make_unique<ReadBufferFromS3>(client_ptr, bucket, metadata.remote_fs_root_path + path, s3_max_single_read_retries, buf_size);
return std::make_unique<ReadBufferFromS3>(client_ptr, bucket, metadata.remote_fs_root_path + path, max_single_read_retries, buf_size);
}
private:
std::shared_ptr<Aws::S3::S3Client> client_ptr;
const String & bucket;
size_t s3_max_single_read_retries;
UInt64 max_single_read_retries;
size_t buf_size;
};
@ -937,7 +937,7 @@ void DiskS3::applyNewSettings(const Poco::Util::AbstractConfiguration & config,
DiskS3Settings::DiskS3Settings(
const std::shared_ptr<Aws::S3::S3Client> & client_,
size_t s3_max_single_read_retries_,
UInt64 s3_max_single_read_retries_,
size_t s3_min_upload_part_size_,
size_t s3_max_single_part_upload_size_,
size_t min_bytes_for_seek_,

View File

@ -7,7 +7,6 @@ PEERDIR(
clickhouse/src/Common
)
SRCS(
DiskCacheWrapper.cpp
DiskDecorator.cpp
@ -16,13 +15,21 @@ SRCS(
DiskMemory.cpp
DiskRestartProxy.cpp
DiskSelector.cpp
HDFS/DiskHDFS.cpp
IDisk.cpp
IDiskRemote.cpp
IVolume.cpp
LocalDirectorySyncGuard.cpp
ReadIndirectBufferFromRemoteFS.cpp
S3/DiskS3.cpp
S3/ProxyListConfiguration.cpp
S3/ProxyResolverConfiguration.cpp
S3/registerDiskS3.cpp
SingleDiskVolume.cpp
StoragePolicy.cpp
VolumeJBOD.cpp
VolumeRAID1.cpp
WriteIndirectBufferFromRemoteFS.cpp
createVolume.cpp
registerDisks.cpp

View File

@ -6,7 +6,7 @@ PEERDIR(
clickhouse/src/Common
)
SRCS(
<? find . -name '*.cpp' | grep -v -F tests | grep -v -F examples | grep -v -F 'S3|HDFS' | sed 's/^\.\// /' | sort ?>
)

View File

@ -182,18 +182,20 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
const ColumnMap * col_map = typeid_cast<const ColumnMap *>(arguments[0].column.get());
bool is_const = isColumnConst(*arguments[0].column);
const ColumnMap * col_map = is_const ? checkAndGetColumnConstData<ColumnMap>(arguments[0].column.get()) : checkAndGetColumn<ColumnMap>(arguments[0].column.get());
if (!col_map)
return nullptr;
throw Exception{"First argument for function " + getName() + " must be a map", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
const auto & nested_column = col_map->getNestedColumn();
const auto & keys_data = col_map->getNestedData().getColumn(0);
/// Prepare arguments to call arrayIndex for check has the array element.
ColumnPtr column_array = ColumnArray::create(keys_data.getPtr(), nested_column.getOffsetsPtr());
ColumnsWithTypeAndName new_arguments =
{
{
ColumnArray::create(keys_data.getPtr(), nested_column.getOffsetsPtr()),
is_const ? ColumnConst::create(std::move(column_array), keys_data.size()) : std::move(column_array),
std::make_shared<DataTypeArray>(result_type),
""
},

View File

@ -31,12 +31,12 @@ namespace ErrorCodes
ReadBufferFromS3::ReadBufferFromS3(
std::shared_ptr<Aws::S3::S3Client> client_ptr_, const String & bucket_, const String & key_, UInt64 s3_max_single_read_retries_, size_t buffer_size_)
std::shared_ptr<Aws::S3::S3Client> client_ptr_, const String & bucket_, const String & key_, UInt64 max_single_read_retries_, size_t buffer_size_)
: SeekableReadBuffer(nullptr, 0)
, client_ptr(std::move(client_ptr_))
, bucket(bucket_)
, key(key_)
, s3_max_single_read_retries(s3_max_single_read_retries_)
, max_single_read_retries(max_single_read_retries_)
, buffer_size(buffer_size_)
{
}
@ -51,12 +51,10 @@ bool ReadBufferFromS3::nextImpl()
Stopwatch watch;
bool next_result = false;
auto sleep_time_with_backoff_milliseconds = std::chrono::milliseconds(100);
for (Int64 attempt = static_cast<Int64>(s3_max_single_read_retries); attempt >= 0; --attempt)
for (size_t attempt = 0; attempt < max_single_read_retries; ++attempt)
{
if (!impl)
impl = initialize();
try
{
next_result = impl->next();
@ -70,14 +68,15 @@ bool ReadBufferFromS3::nextImpl()
{
ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors, 1);
LOG_INFO(log, "Caught exception while reading S3 object. Bucket: {}, Key: {}, Offset: {}, Remaining attempts: {}, Message: {}",
LOG_INFO(log, "Caught exception while reading S3 object. Bucket: {}, Key: {}, Offset: {}, Attempt: {}, Message: {}",
bucket, key, getPosition(), attempt, e.message());
impl.reset();
if (!attempt)
throw;
impl = initialize();
}
std::this_thread::sleep_for(sleep_time_with_backoff_milliseconds);
sleep_time_with_backoff_milliseconds *= 2;
}
watch.stop();

View File

@ -27,7 +27,7 @@ private:
std::shared_ptr<Aws::S3::S3Client> client_ptr;
String bucket;
String key;
UInt64 s3_max_single_read_retries;
UInt64 max_single_read_retries;
size_t buffer_size;
off_t offset = 0;
Aws::S3::Model::GetObjectResult read_result;
@ -40,7 +40,7 @@ public:
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
const String & bucket_,
const String & key_,
UInt64 s3_max_single_read_retries_,
UInt64 max_single_read_retries_,
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE);
bool nextImpl() override;

View File

@ -428,8 +428,6 @@ public:
/// EC2MetadataService throttles by delaying the response so the service client should set a large read timeout.
/// EC2MetadataService delay is in order of seconds so it only make sense to retry after a couple of seconds.
aws_client_configuration.connectTimeoutMs = 1000;
/// FIXME. Somehow this timeout does not work in docker without --net=host.
aws_client_configuration.requestTimeoutMs = 1000;
aws_client_configuration.retryStrategy = std::make_shared<Aws::Client::DefaultRetryStrategy>(1, 1000);

View File

@ -1894,11 +1894,11 @@ void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
res = place;
}
if (block.rows() > 0)
for (size_t row = 0, rows = block.rows(); row < rows; ++row)
{
/// Adding Values
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->merge(res + offsets_of_aggregate_states[i], (*aggregate_columns[i])[0], result.aggregates_pool);
aggregate_functions[i]->merge(res + offsets_of_aggregate_states[i], (*aggregate_columns[i])[row], result.aggregates_pool);
}
/// Early release memory.

View File

@ -83,6 +83,17 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr c
}
}
if (settings.offset)
{
new_settings.offset = 0;
new_settings.offset.changed = false;
}
if (settings.limit)
{
new_settings.limit = 0;
new_settings.limit.changed = false;
}
auto new_context = Context::createCopy(context);
new_context->setSettings(new_settings);
return new_context;

View File

@ -132,7 +132,7 @@ StoragePtr TemporaryTableHolder::getTable() const
}
void DatabaseCatalog::loadTemporaryDatabase()
void DatabaseCatalog::initializeAndLoadTemporaryDatabase()
{
drop_delay_sec = getContext()->getConfigRef().getInt("database_atomic_delay_before_drop_table_sec", default_drop_delay_sec);

View File

@ -127,7 +127,7 @@ public:
static DatabaseCatalog & instance();
static void shutdown();
void loadTemporaryDatabase();
void initializeAndLoadTemporaryDatabase();
void loadDatabases();
/// Get an object that protects the table from concurrently executing multiple DDL operations.

View File

@ -136,13 +136,16 @@ ExpressionAnalyzer::ExpressionAnalyzer(
ContextPtr context_,
size_t subquery_depth_,
bool do_global,
SubqueriesForSets subqueries_for_sets_)
SubqueriesForSets subqueries_for_sets_,
PreparedSets prepared_sets_)
: WithContext(context_)
, query(query_), settings(getContext()->getSettings())
, subquery_depth(subquery_depth_)
, syntax(syntax_analyzer_result_)
{
/// Cache prepared sets because we might run analysis multiple times
subqueries_for_sets = std::move(subqueries_for_sets_);
prepared_sets = std::move(prepared_sets_);
/// external_tables, subqueries_for_sets for global subqueries.
/// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers.
@ -395,8 +398,7 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
getRootActions(left_in_operand, true, temp_actions);
if (temp_actions->tryFindInIndex(left_in_operand->getColumnName()))
makeExplicitSet(func, *temp_actions, true, getContext(),
settings.size_limits_for_set, prepared_sets);
makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, prepared_sets);
}
}
}

View File

@ -96,12 +96,10 @@ private:
public:
/// Ctor for non-select queries. Generally its usage is:
/// auto actions = ExpressionAnalyzer(query, syntax, context).getActions();
ExpressionAnalyzer(
const ASTPtr & query_,
const TreeRewriterResultPtr & syntax_analyzer_result_,
ContextPtr context_)
: ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, 0, false, {})
{}
ExpressionAnalyzer(const ASTPtr & query_, const TreeRewriterResultPtr & syntax_analyzer_result_, ContextPtr context_)
: ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, 0, false, {}, {})
{
}
~ExpressionAnalyzer();
@ -125,6 +123,8 @@ public:
*/
SubqueriesForSets & getSubqueriesForSets() { return subqueries_for_sets; }
PreparedSets & getPreparedSets() { return prepared_sets; }
/// Get intermediates for tests
const ExpressionAnalyzerData & getAnalyzedData() const { return *this; }
@ -153,7 +153,8 @@ protected:
ContextPtr context_,
size_t subquery_depth_,
bool do_global_,
SubqueriesForSets subqueries_for_sets_);
SubqueriesForSets subqueries_for_sets_,
PreparedSets prepared_sets_);
ASTPtr query;
const ExtractedSettings settings;
@ -285,8 +286,16 @@ public:
const NameSet & required_result_columns_ = {},
bool do_global_ = false,
const SelectQueryOptions & options_ = {},
SubqueriesForSets subqueries_for_sets_ = {})
: ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, options_.subquery_depth, do_global_, std::move(subqueries_for_sets_))
SubqueriesForSets subqueries_for_sets_ = {},
PreparedSets prepared_sets_ = {})
: ExpressionAnalyzer(
query_,
syntax_analyzer_result_,
context_,
options_.subquery_depth,
do_global_,
std::move(subqueries_for_sets_),
std::move(prepared_sets_))
, metadata_snapshot(metadata_snapshot_)
, required_result_columns(required_result_columns_)
, query_options(options_)

View File

@ -66,8 +66,15 @@ DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const std
std::string resolved_name = resolveDictionaryName(dictionary_name, query_context->getCurrentDatabase());
auto load_result = getLoadResult(resolved_name);
if (load_result.object)
{
const auto dictionary = std::static_pointer_cast<const IDictionary>(load_result.object);
return dictionary->getStructure();
}
if (!load_result.config)
throw Exception("Dictionary " + backQuote(dictionary_name) + " config not found", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary {} config not found", backQuote(dictionary_name));
return ExternalDictionariesLoader::getDictionaryStructure(*load_result.config);
}
@ -128,7 +135,7 @@ std::string ExternalDictionariesLoader::resolveDictionaryNameFromDatabaseCatalog
DictionaryStructure
ExternalDictionariesLoader::getDictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config)
{
return {config, key_in_config};
return DictionaryStructure(config, key_in_config);
}
DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const ObjectConfig & config)

View File

@ -85,7 +85,8 @@ public:
/// If this is already an external table, you do not need to add anything. Just remember its presence.
auto temporary_table_name = getIdentifierName(subquery_or_table_name);
bool exists_in_local_map = external_tables.end() != external_tables.find(temporary_table_name);
bool exists_in_context = getContext()->tryResolveStorageID(StorageID("", temporary_table_name), Context::ResolveExternal);
bool exists_in_context = static_cast<bool>(getContext()->tryResolveStorageID(
StorageID("", temporary_table_name), Context::ResolveExternal));
if (exists_in_local_map || exists_in_context)
return;
}

View File

@ -447,6 +447,8 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
defaults_sample_block = validateColumnsDefaultsAndGetSampleBlock(default_expr_list, column_names_and_types, context_);
bool sanity_check_compression_codecs = !attach && !context_->getSettingsRef().allow_suspicious_codecs;
bool allow_experimental_codecs = attach || context_->getSettingsRef().allow_experimental_codecs;
ColumnsDescription res;
auto name_type_it = column_names_and_types.begin();
for (auto ast_it = columns_ast.children.begin(); ast_it != columns_ast.children.end(); ++ast_it, ++name_type_it)
@ -481,7 +483,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
if (col_decl.default_specifier == "ALIAS")
throw Exception{"Cannot specify codec for column type ALIAS", ErrorCodes::BAD_ARGUMENTS};
column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(
col_decl.codec, column.type, sanity_check_compression_codecs);
col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs);
}
if (col_decl.ttl)

View File

@ -44,7 +44,8 @@ BlockInputStreamPtr InterpreterExistsQuery::executeImpl()
{
if (exists_query->temporary)
{
result = getContext()->tryResolveStorageID({"", exists_query->table}, Context::ResolveExternal);
result = static_cast<bool>(getContext()->tryResolveStorageID(
{"", exists_query->table}, Context::ResolveExternal));
}
else
{

View File

@ -242,9 +242,11 @@ static void checkAccessRightsForSelect(
if (access->isGranted(AccessType::SELECT, table_id.database_name, table_id.table_name, column.name))
return;
}
throw Exception(context->getUserName() + ": Not enough privileges. "
"To execute this query it's necessary to have grant SELECT for at least one column on " + table_id.getFullTableName(),
ErrorCodes::ACCESS_DENIED);
throw Exception(
ErrorCodes::ACCESS_DENIED,
"{}: Not enough privileges. To execute this query it's necessary to have grant SELECT for at least one column on {}",
context->getUserName(),
table_id.getFullTableName());
}
/// General check.
@ -369,7 +371,9 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (storage)
view = dynamic_cast<StorageView *>(storage.get());
/// Reuse already built sets for multiple passes of analysis
SubqueriesForSets subquery_for_sets;
PreparedSets prepared_sets;
auto analyze = [&] (bool try_move_to_prewhere)
{
@ -429,9 +433,15 @@ InterpreterSelectQuery::InterpreterSelectQuery(
}
query_analyzer = std::make_unique<SelectQueryExpressionAnalyzer>(
query_ptr, syntax_analyzer_result, context, metadata_snapshot,
NameSet(required_result_column_names.begin(), required_result_column_names.end()),
!options.only_analyze, options, std::move(subquery_for_sets));
query_ptr,
syntax_analyzer_result,
context,
metadata_snapshot,
NameSet(required_result_column_names.begin(), required_result_column_names.end()),
!options.only_analyze,
options,
std::move(subquery_for_sets),
std::move(prepared_sets));
if (!options.only_analyze)
{
@ -439,10 +449,14 @@ InterpreterSelectQuery::InterpreterSelectQuery(
throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED);
if (query.final() && (input || input_pipe || !storage || !storage->supportsFinal()))
throw Exception((!input && !input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support FINAL" : "Illegal FINAL", ErrorCodes::ILLEGAL_FINAL);
throw Exception(
(!input && !input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support FINAL" : "Illegal FINAL",
ErrorCodes::ILLEGAL_FINAL);
if (query.prewhere() && (input || input_pipe || !storage || !storage->supportsPrewhere()))
throw Exception((!input && !input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support PREWHERE" : "Illegal PREWHERE", ErrorCodes::ILLEGAL_PREWHERE);
throw Exception(
(!input && !input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support PREWHERE" : "Illegal PREWHERE",
ErrorCodes::ILLEGAL_PREWHERE);
/// Save the new temporary tables in the query context
for (const auto & it : query_analyzer->getExternalTables())
@ -515,8 +529,10 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (need_analyze_again)
{
LOG_TRACE(log, "Running 'analyze' second time");
query_analyzer->getSubqueriesForSets().clear();
subquery_for_sets = SubqueriesForSets();
/// Reuse already built sets for multiple passes of analysis
subquery_for_sets = std::move(query_analyzer->getSubqueriesForSets());
prepared_sets = std::move(query_analyzer->getPreparedSets());
/// Do not try move conditions to PREWHERE for the second time.
/// Otherwise, we won't be able to fallback from inefficient PREWHERE to WHERE later.
@ -565,10 +581,10 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan)
if (!options.ignore_aggregation && !blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header))
{
auto convert_actions_dag = ActionsDAG::makeConvertingActions(
query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(),
result_header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Name,
true);
query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(),
result_header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Name,
true);
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), convert_actions_dag);
query_plan.addStep(std::move(converting));
@ -583,8 +599,7 @@ BlockIO InterpreterSelectQuery::execute()
buildQueryPlan(query_plan);
res.pipeline = std::move(*query_plan.buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(context),
BuildQueryPipelineSettings::fromContext(context)));
QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)));
return res;
}
@ -623,13 +638,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
&& options.to_stage > QueryProcessingStage::WithMergeableState;
analysis_result = ExpressionAnalysisResult(
*query_analyzer,
metadata_snapshot,
first_stage,
second_stage,
options.only_analyze,
filter_info,
source_header);
*query_analyzer, metadata_snapshot, first_stage, second_stage, options.only_analyze, filter_info, source_header);
if (options.to_stage == QueryProcessingStage::Enum::FetchColumns)
{
@ -668,9 +677,8 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
// with this code. See
// https://github.com/ClickHouse/ClickHouse/issues/19857 for details.
if (analysis_result.before_window)
{
return analysis_result.before_window->getResultColumns();
}
return analysis_result.before_order_by->getResultColumns();
}
@ -701,9 +709,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
// It's different from selected_columns, see the comment above for
// WithMergeableState stage.
if (analysis_result.before_window)
{
return analysis_result.before_window->getResultColumns();
}
return analysis_result.before_order_by->getResultColumns();
}
@ -782,8 +788,7 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP
if (order_by_elem.with_fill)
{
FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context);
order_descr.emplace_back(name, order_by_elem.direction,
order_by_elem.nulls_direction, collator, true, fill_desc);
order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, true, fill_desc);
}
else
order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator);
@ -811,11 +816,14 @@ static UInt64 getLimitUIntValue(const ASTPtr & node, ContextPtr context, const s
const auto & [field, type] = evaluateConstantExpression(node, context);
if (!isNativeNumber(type))
throw Exception("Illegal type " + type->getName() + " of " + expr + " expression, must be numeric type", ErrorCodes::INVALID_LIMIT_EXPRESSION);
throw Exception(
"Illegal type " + type->getName() + " of " + expr + " expression, must be numeric type", ErrorCodes::INVALID_LIMIT_EXPRESSION);
Field converted = convertFieldToType(field, DataTypeUInt64());
if (converted.isNull())
throw Exception("The value " + applyVisitor(FieldVisitorToString(), field) + " of " + expr + " expression is not representable as UInt64", ErrorCodes::INVALID_LIMIT_EXPRESSION);
throw Exception(
"The value " + applyVisitor(FieldVisitorToString(), field) + " of " + expr + " expression is not representable as UInt64",
ErrorCodes::INVALID_LIMIT_EXPRESSION);
return converted.safeGet<UInt64>();
}
@ -962,10 +970,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
}
auto prewhere_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
expressions.prewhere_info->prewhere_actions,
expressions.prewhere_info->prewhere_column_name,
expressions.prewhere_info->remove_prewhere_column);
query_plan.getCurrentDataStream(),
expressions.prewhere_info->prewhere_actions,
expressions.prewhere_info->prewhere_column_name,
expressions.prewhere_info->remove_prewhere_column);
prewhere_step->setStepDescription("PREWHERE");
query_plan.addStep(std::move(prewhere_step));
@ -976,8 +984,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
if (expressions.prewhere_info->remove_columns_actions)
{
auto remove_columns = std::make_unique<ExpressionStep>(
query_plan.getCurrentDataStream(),
expressions.prewhere_info->remove_columns_actions);
query_plan.getCurrentDataStream(), expressions.prewhere_info->remove_columns_actions);
remove_columns->setStepDescription("Remove unnecessary columns after PREWHERE");
query_plan.addStep(std::move(remove_columns));
@ -988,8 +995,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
{
if (prepared_input)
{
auto prepared_source_step = std::make_unique<ReadFromPreparedSource>(
Pipe(std::make_shared<SourceFromInputStream>(prepared_input)), context);
auto prepared_source_step
= std::make_unique<ReadFromPreparedSource>(Pipe(std::make_shared<SourceFromInputStream>(prepared_input)), context);
query_plan.addStep(std::move(prepared_source_step));
}
else if (prepared_pipe)
@ -1073,10 +1080,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
if (!query_info.projection && expressions.filter_info)
{
auto row_level_security_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
expressions.filter_info->actions,
expressions.filter_info->column_name,
expressions.filter_info->do_remove_column);
query_plan.getCurrentDataStream(),
expressions.filter_info->actions,
expressions.filter_info->column_name,
expressions.filter_info->do_remove_column);
row_level_security_step->setStepDescription("Row-level security filter");
query_plan.addStep(std::move(row_level_security_step));
@ -1084,18 +1091,16 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
if (expressions.before_array_join)
{
QueryPlanStepPtr before_array_join_step = std::make_unique<ExpressionStep>(
query_plan.getCurrentDataStream(),
expressions.before_array_join);
QueryPlanStepPtr before_array_join_step
= std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), expressions.before_array_join);
before_array_join_step->setStepDescription("Before ARRAY JOIN");
query_plan.addStep(std::move(before_array_join_step));
}
if (expressions.array_join)
{
QueryPlanStepPtr array_join_step = std::make_unique<ArrayJoinStep>(
query_plan.getCurrentDataStream(),
expressions.array_join);
QueryPlanStepPtr array_join_step
= std::make_unique<ArrayJoinStep>(query_plan.getCurrentDataStream(), expressions.array_join);
array_join_step->setStepDescription("ARRAY JOIN");
query_plan.addStep(std::move(array_join_step));
@ -1228,7 +1233,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
if (query.group_by_with_totals)
{
bool final = !query.group_by_with_rollup && !query.group_by_with_cube;
executeTotalsAndHaving(query_plan, expressions.hasHaving(), expressions.before_having, aggregate_overflow_row, final);
executeTotalsAndHaving(
query_plan, expressions.hasHaving(), expressions.before_having, aggregate_overflow_row, final);
}
if (query.group_by_with_rollup)
@ -1239,7 +1245,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
if ((query.group_by_with_rollup || query.group_by_with_cube) && expressions.hasHaving())
{
if (query.group_by_with_totals)
throw Exception("WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(
"WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING",
ErrorCodes::NOT_IMPLEMENTED);
executeHaving(query_plan, expressions.before_having);
}
}
@ -1259,7 +1267,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
if (from_aggregation_stage)
{
if (query_analyzer->hasWindow())
throw Exception("Window functions does not support processing from WithMergeableStateAfterAggregation", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(
"Window functions does not support processing from WithMergeableStateAfterAggregation",
ErrorCodes::NOT_IMPLEMENTED);
}
else if (expressions.need_aggregate)
{
@ -1384,8 +1394,7 @@ static StreamLocalLimits getLimitsForStorage(const Settings & settings, const Se
{
StreamLocalLimits limits;
limits.mode = LimitsMode::LIMITS_TOTAL;
limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read,
settings.read_overflow_mode);
limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read, settings.read_overflow_mode);
limits.speed_limits.max_execution_time = settings.max_execution_time;
limits.timeout_overflow_mode = settings.timeout_overflow_mode;
@ -1446,11 +1455,11 @@ static void executeMergeAggregatedImpl(
auto transform_params = std::make_shared<AggregatingTransformParams>(params, final);
auto merging_aggregated = std::make_unique<MergingAggregatedStep>(
query_plan.getCurrentDataStream(),
std::move(transform_params),
settings.distributed_aggregation_memory_efficient && is_remote_storage,
settings.max_threads,
settings.aggregation_memory_efficient_merge_threads);
query_plan.getCurrentDataStream(),
std::move(transform_params),
settings.distributed_aggregation_memory_efficient && is_remote_storage,
settings.max_threads,
settings.aggregation_memory_efficient_merge_threads);
query_plan.addStep(std::move(merging_aggregated));
}
@ -1467,33 +1476,22 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(
if (prewhere_info.alias_actions)
{
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ExpressionTransform>(
header,
prewhere_info.alias_actions);
});
pipe.addSimpleTransform(
[&](const Block & header) { return std::make_shared<ExpressionTransform>(header, prewhere_info.alias_actions); });
}
if (prewhere_info.row_level_filter)
{
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<FilterTransform>(
header,
prewhere_info.row_level_filter,
prewhere_info.row_level_column_name,
true);
return std::make_shared<FilterTransform>(header, prewhere_info.row_level_filter, prewhere_info.row_level_column_name, true);
});
}
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<FilterTransform>(
header,
prewhere_info.prewhere_actions,
prewhere_info.prewhere_column_name,
prewhere_info.remove_prewhere_column);
header, prewhere_info.prewhere_actions, prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column);
});
// To remove additional columns
@ -1502,12 +1500,8 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(
// This leads to mismatched header in distributed table
if (prewhere_info.remove_columns_actions)
{
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ExpressionTransform>(
header,
prewhere_info.remove_columns_actions);
});
pipe.addSimpleTransform(
[&](const Block & header) { return std::make_shared<ExpressionTransform>(header, prewhere_info.remove_columns_actions); });
}
}
@ -1703,12 +1697,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions()
prewhere_info->remove_prewhere_column = false;
/// Remove columns which will be added by prewhere.
required_columns.erase(
std::remove_if(
required_columns.begin(),
required_columns.end(),
[&](const String & name) { return required_columns_after_prewhere_set.count(name) != 0; }),
required_columns.end());
std::erase_if(required_columns, [&](const String & name) { return required_columns_after_prewhere_set.count(name) != 0; });
if (prewhere_info)
{
@ -1813,10 +1802,11 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
/// Limitation on the number of columns to read.
/// It's not applied in 'only_analyze' mode, because the query could be analyzed without removal of unnecessary columns.
if (!options.only_analyze && settings.max_columns_to_read && required_columns.size() > settings.max_columns_to_read)
throw Exception("Limit for number of columns to read exceeded. "
"Requested: " + toString(required_columns.size())
+ ", maximum: " + settings.max_columns_to_read.toString(),
ErrorCodes::TOO_MANY_COLUMNS);
throw Exception(
ErrorCodes::TOO_MANY_COLUMNS,
"Limit for number of columns to read exceeded. Requested: {}, maximum: {}",
required_columns.size(),
settings.max_columns_to_read);
/// General limit for the number of threads.
size_t max_threads_execute_query = settings.max_threads;
@ -1909,14 +1899,18 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
auto actions_settings = ExpressionActionsSettings::fromContext(context, CompileExpressions::yes);
query_info.prewhere_info = std::make_shared<PrewhereInfo>();
query_info.prewhere_info->prewhere_actions = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions, actions_settings);
query_info.prewhere_info->prewhere_actions
= std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions, actions_settings);
if (prewhere_info->row_level_filter_actions)
query_info.prewhere_info->row_level_filter = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter_actions, actions_settings);
query_info.prewhere_info->row_level_filter
= std::make_shared<ExpressionActions>(prewhere_info->row_level_filter_actions, actions_settings);
if (prewhere_info->alias_actions)
query_info.prewhere_info->alias_actions = std::make_shared<ExpressionActions>(prewhere_info->alias_actions, actions_settings);
query_info.prewhere_info->alias_actions
= std::make_shared<ExpressionActions>(prewhere_info->alias_actions, actions_settings);
if (prewhere_info->remove_columns_actions)
query_info.prewhere_info->remove_columns_actions = std::make_shared<ExpressionActions>(prewhere_info->remove_columns_actions, actions_settings);
query_info.prewhere_info->remove_columns_actions
= std::make_shared<ExpressionActions>(prewhere_info->remove_columns_actions, actions_settings);
query_info.prewhere_info->prewhere_column_name = prewhere_info->prewhere_column_name;
query_info.prewhere_info->remove_prewhere_column = prewhere_info->remove_prewhere_column;
@ -1976,15 +1970,13 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
if (!options.ignore_limits)
{
limits = getLimitsForStorage(settings, options);
leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, settings.max_bytes_to_read_leaf,
settings.read_overflow_mode_leaf);
leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, settings.max_bytes_to_read_leaf, settings.read_overflow_mode_leaf);
}
if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete))
quota = context->getQuota();
storage->read(query_plan, required_columns, metadata_snapshot,
query_info, context, processing_stage, max_block_size, max_streams);
storage->read(query_plan, required_columns, metadata_snapshot, query_info, context, processing_stage, max_block_size, max_streams);
if (context->hasQueryContext() && !options.is_internal)
{
@ -2009,13 +2001,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
/// Extend lifetime of context, table lock, storage. Set limits and quota.
auto adding_limits_and_quota = std::make_unique<SettingQuotaAndLimitsStep>(
query_plan.getCurrentDataStream(),
storage,
std::move(table_lock),
limits,
leaf_limits,
std::move(quota),
context);
query_plan.getCurrentDataStream(), storage, std::move(table_lock), limits, leaf_limits, std::move(quota), context);
adding_limits_and_quota->setStepDescription("Set limits and quota after reading from storage");
query_plan.addStep(std::move(adding_limits_and_quota));
}
@ -2044,10 +2030,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter)
{
auto where_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
expression,
getSelectQuery().where()->getColumnName(),
remove_filter);
query_plan.getCurrentDataStream(), expression, getSelectQuery().where()->getColumnName(), remove_filter);
where_step->setStepDescription("WHERE");
query_plan.addStep(std::move(where_step));
@ -2076,15 +2059,20 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
const Settings & settings = context->getSettingsRef();
Aggregator::Params params(header_before_aggregation, keys, aggregates,
overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
settings.group_by_two_level_threshold,
settings.group_by_two_level_threshold_bytes,
settings.max_bytes_before_external_group_by,
settings.empty_result_for_aggregation_by_empty_set,
context->getTemporaryVolume(),
settings.max_threads,
settings.min_free_disk_space_for_temporary_data);
Aggregator::Params params(
header_before_aggregation,
keys,
aggregates,
overflow_row,
settings.max_rows_to_group_by,
settings.group_by_overflow_mode,
settings.group_by_two_level_threshold,
settings.group_by_two_level_threshold_bytes,
settings.max_bytes_before_external_group_by,
settings.empty_result_for_aggregation_by_empty_set,
context->getTemporaryVolume(),
settings.max_threads,
settings.min_free_disk_space_for_temporary_data);
SortDescription group_by_sort_description;
@ -2095,20 +2083,21 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
auto merge_threads = max_streams;
auto temporary_data_merge_threads = settings.aggregation_memory_efficient_merge_threads
? static_cast<size_t>(settings.aggregation_memory_efficient_merge_threads)
: static_cast<size_t>(settings.max_threads);
? static_cast<size_t>(settings.aggregation_memory_efficient_merge_threads)
: static_cast<size_t>(settings.max_threads);
bool storage_has_evenly_distributed_read = storage && storage->hasEvenlyDistributedRead();
auto aggregating_step = std::make_unique<AggregatingStep>(
query_plan.getCurrentDataStream(),
params, final,
settings.max_block_size,
merge_threads,
temporary_data_merge_threads,
storage_has_evenly_distributed_read,
std::move(group_by_info),
std::move(group_by_sort_description));
query_plan.getCurrentDataStream(),
params,
final,
settings.max_block_size,
merge_threads,
temporary_data_merge_threads,
storage_has_evenly_distributed_read,
std::move(group_by_info),
std::move(group_by_sort_description));
query_plan.addStep(std::move(aggregating_step));
}
@ -2135,24 +2124,27 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool
void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression)
{
auto having_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
expression, getSelectQuery().having()->getColumnName(), false);
auto having_step
= std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(), false);
having_step->setStepDescription("HAVING");
query_plan.addStep(std::move(having_step));
}
void InterpreterSelectQuery::executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool overflow_row, bool final)
void InterpreterSelectQuery::executeTotalsAndHaving(
QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool overflow_row, bool final)
{
const Settings & settings = context->getSettingsRef();
auto totals_having_step = std::make_unique<TotalsHavingStep>(
query_plan.getCurrentDataStream(),
overflow_row, expression,
has_having ? getSelectQuery().having()->getColumnName() : "",
settings.totals_mode, settings.totals_auto_threshold, final);
query_plan.getCurrentDataStream(),
overflow_row,
expression,
has_having ? getSelectQuery().having()->getColumnName() : "",
settings.totals_mode,
settings.totals_auto_threshold,
final);
query_plan.addStep(std::move(totals_having_step));
}
@ -2169,10 +2161,20 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific
const Settings & settings = context->getSettingsRef();
Aggregator::Params params(header_before_transform, keys, query_analyzer->aggregates(),
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0,
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
context->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data);
Aggregator::Params params(
header_before_transform,
keys,
query_analyzer->aggregates(),
false,
settings.max_rows_to_group_by,
settings.group_by_overflow_mode,
0,
0,
settings.max_bytes_before_external_group_by,
settings.empty_result_for_aggregation_by_empty_set,
context->getTemporaryVolume(),
settings.max_threads,
settings.min_free_disk_space_for_temporary_data);
auto transform_params = std::make_shared<AggregatingTransformParams>(params, true);
@ -2189,9 +2191,7 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific
void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description)
{
if (!expression)
{
return;
}
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), expression);
@ -2199,8 +2199,7 @@ void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const Act
query_plan.addStep(std::move(expression_step));
}
static bool windowDescriptionComparator(const WindowDescription * _left,
const WindowDescription * _right)
static bool windowDescriptionComparator(const WindowDescription * _left, const WindowDescription * _right)
{
const auto & left = _left->full_sort_description;
const auto & right = _right->full_sort_description;
@ -2208,37 +2207,21 @@ static bool windowDescriptionComparator(const WindowDescription * _left,
for (size_t i = 0; i < std::min(left.size(), right.size()); ++i)
{
if (left[i].column_name < right[i].column_name)
{
return true;
}
else if (left[i].column_name > right[i].column_name)
{
return false;
}
else if (left[i].column_number < right[i].column_number)
{
return true;
}
else if (left[i].column_number > right[i].column_number)
{
return false;
}
else if (left[i].direction < right[i].direction)
{
return true;
}
else if (left[i].direction > right[i].direction)
{
return false;
}
else if (left[i].nulls_direction < right[i].nulls_direction)
{
return true;
}
else if (left[i].nulls_direction > right[i].nulls_direction)
{
return false;
}
assert(left[i] == right[i]);
}
@ -2255,16 +2238,12 @@ static bool sortIsPrefix(const WindowDescription & _prefix,
const auto & full = _full.full_sort_description;
if (prefix.size() > full.size())
{
return false;
}
for (size_t i = 0; i < prefix.size(); ++i)
{
if (full[i] != prefix[i])
{
return false;
}
}
return true;
@ -2276,12 +2255,9 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan)
// sort description goes first, and all window that use its prefixes follow.
std::vector<const WindowDescription *> windows_sorted;
for (const auto & [_, w] : query_analyzer->windowDescriptions())
{
windows_sorted.push_back(&w);
}
std::sort(windows_sorted.begin(), windows_sorted.end(),
windowDescriptionComparator);
std::sort(windows_sorted.begin(), windows_sorted.end(), windowDescriptionComparator);
const Settings & settings = context->getSettingsRef();
for (size_t i = 0; i < windows_sorted.size(); ++i)
@ -2292,17 +2268,14 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan)
// has suitable sorting. Also don't create sort steps when there are no
// columns to sort by, because the sort nodes are confused by this. It
// happens in case of `over ()`.
if (!w.full_sort_description.empty()
&& (i == 0 || !sortIsPrefix(w, *windows_sorted[i - 1])))
if (!w.full_sort_description.empty() && (i == 0 || !sortIsPrefix(w, *windows_sorted[i - 1])))
{
auto partial_sorting = std::make_unique<PartialSortingStep>(
query_plan.getCurrentDataStream(),
w.full_sort_description,
0 /* LIMIT */,
SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort,
settings.sort_overflow_mode));
partial_sorting->setStepDescription("Sort each block for window '"
+ w.window_name + "'");
SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode));
partial_sorting->setStepDescription("Sort each block for window '" + w.window_name + "'");
query_plan.addStep(std::move(partial_sorting));
auto merge_sorting_step = std::make_unique<MergeSortingStep>(
@ -2315,8 +2288,7 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan)
settings.max_bytes_before_external_sort,
context->getTemporaryVolume(),
settings.min_free_disk_space_for_temporary_data);
merge_sorting_step->setStepDescription(
"Merge sorted blocks for window '" + w.window_name + "'");
merge_sorting_step->setStepDescription("Merge sorted blocks for window '" + w.window_name + "'");
query_plan.addStep(std::move(merge_sorting_step));
// First MergeSorted, now MergingSorted.
@ -2325,17 +2297,12 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan)
w.full_sort_description,
settings.max_block_size,
0 /* LIMIT */);
merging_sorted->setStepDescription(
"Merge sorted streams for window '" + w.window_name + "'");
merging_sorted->setStepDescription("Merge sorted streams for window '" + w.window_name + "'");
query_plan.addStep(std::move(merging_sorted));
}
auto window_step = std::make_unique<WindowStep>(
query_plan.getCurrentDataStream(),
w,
w.window_functions);
window_step->setStepDescription("Window step for window '"
+ w.window_name + "'");
auto window_step = std::make_unique<WindowStep>(query_plan.getCurrentDataStream(), w, w.window_functions);
window_step->setStepDescription("Window step for window '" + w.window_name + "'");
query_plan.addStep(std::move(window_step));
}
@ -2347,11 +2314,7 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input
const Settings & settings = context->getSettingsRef();
auto finish_sorting_step = std::make_unique<FinishSortingStep>(
query_plan.getCurrentDataStream(),
input_sorting_info->order_key_prefix_descr,
output_order_descr,
settings.max_block_size,
limit);
query_plan.getCurrentDataStream(), input_sorting_info->order_key_prefix_descr, output_order_descr, settings.max_block_size, limit);
query_plan.addStep(std::move(finish_sorting_step));
}
@ -2377,25 +2340,25 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo
const Settings & settings = context->getSettingsRef();
auto partial_sorting = std::make_unique<PartialSortingStep>(
query_plan.getCurrentDataStream(),
output_order_descr,
limit,
SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode));
query_plan.getCurrentDataStream(),
output_order_descr,
limit,
SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode));
partial_sorting->setStepDescription("Sort each block for ORDER BY");
query_plan.addStep(std::move(partial_sorting));
/// Merge the sorted blocks.
auto merge_sorting_step = std::make_unique<MergeSortingStep>(
query_plan.getCurrentDataStream(),
output_order_descr,
settings.max_block_size,
limit,
settings.max_bytes_before_remerge_sort,
settings.remerge_sort_lowered_memory_bytes_ratio,
settings.max_bytes_before_external_sort,
context->getTemporaryVolume(),
settings.min_free_disk_space_for_temporary_data);
query_plan.getCurrentDataStream(),
output_order_descr,
settings.max_block_size,
limit,
settings.max_bytes_before_remerge_sort,
settings.remerge_sort_lowered_memory_bytes_ratio,
settings.max_bytes_before_external_sort,
context->getTemporaryVolume(),
settings.min_free_disk_space_for_temporary_data);
merge_sorting_step->setStepDescription("Merge sorted blocks for ORDER BY");
query_plan.addStep(std::move(merge_sorting_step));
@ -2418,10 +2381,8 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const So
{
const Settings & settings = context->getSettingsRef();
auto merging_sorted = std::make_unique<MergingSortedStep>(
query_plan.getCurrentDataStream(),
sort_description,
settings.max_block_size, limit);
auto merging_sorted
= std::make_unique<MergingSortedStep>(query_plan.getCurrentDataStream(), sort_description, settings.max_block_size, limit);
merging_sorted->setStepDescription("Merge sorted streams " + description);
query_plan.addStep(std::move(merging_sorted));
@ -2453,9 +2414,8 @@ void InterpreterSelectQuery::executeDistinct(QueryPlan & query_plan, bool before
SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode);
auto distinct_step = std::make_unique<DistinctStep>(
query_plan.getCurrentDataStream(),
limits, limit_for_distinct, columns, pre_distinct);
auto distinct_step
= std::make_unique<DistinctStep>(query_plan.getCurrentDataStream(), limits, limit_for_distinct, columns, pre_distinct);
if (pre_distinct)
distinct_step->setStepDescription("Preliminary DISTINCT");

View File

@ -61,10 +61,10 @@ public:
/// Read data not from the table specified in the query, but from the prepared pipe `input`.
InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
Pipe input_pipe_,
const SelectQueryOptions & = {});
const ASTPtr & query_ptr_,
ContextPtr context_,
Pipe input_pipe_,
const SelectQueryOptions & = {});
/// Read data not from the table specified in the query, but from the specified `storage_`.
InterpreterSelectQuery(

View File

@ -80,6 +80,9 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
/// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column").
auto it_alias = data.aliases.find(node.name());
if (!data.allow_self_aliases && current_alias == node.name())
throw Exception(ErrorCodes::CYCLIC_ALIASES, "Self referencing of {} to {}. Cyclic alias", backQuote(current_alias), backQuote(node.name()));
if (it_alias != data.aliases.end() && current_alias != node.name())
{
if (!IdentifierSemantic::canBeAlias(node))

View File

@ -48,18 +48,22 @@ public:
MapOfASTs finished_asts; /// already processed vertices (and by what they replaced)
SetOfASTs current_asts; /// vertices in the current call stack of this method
std::string current_alias; /// the alias referencing to the ancestor of ast (the deepest ancestor with aliases)
bool ignore_alias; /// normalize query without any aliases
const bool ignore_alias; /// normalize query without any aliases
Data(const Aliases & aliases_, const NameSet & source_columns_set_, bool ignore_alias_, ExtractedSettings && settings_)
/// It's Ok to have "c + 1 AS c" in queries, but not in table definition
const bool allow_self_aliases; /// for constructs like "SELECT column + 1 AS column"
Data(const Aliases & aliases_, const NameSet & source_columns_set_, bool ignore_alias_, ExtractedSettings && settings_, bool allow_self_aliases_)
: aliases(aliases_)
, source_columns_set(source_columns_set_)
, settings(settings_)
, level(0)
, ignore_alias(ignore_alias_)
, allow_self_aliases(allow_self_aliases_)
{}
};
QueryNormalizer(Data & data)
explicit QueryNormalizer(Data & data)
: visitor_data(data)
{}

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