Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into testflows_windows_functions_add_lag_and_lead_in_frame_tests

This commit is contained in:
Vitaliy Zakaznikov 2021-06-07 09:43:14 -04:00
commit 9e99e20765
606 changed files with 24930 additions and 13456 deletions

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,129 @@
### ClickHouse release 21.6, 2021-06-05
#### Upgrade Notes
* `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

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

@ -26,8 +26,6 @@
#include <Poco/Observer.h>
#include <Poco/AutoPtr.h>
#include <Poco/PatternFormatter.h>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Poco/Message.h>
#include <Poco/Util/Application.h>
#include <Poco/Exception.h>
@ -59,6 +57,7 @@
#include <Common/getExecutablePath.h>
#include <Common/getHashOfLoadedBinary.h>
#include <Common/Elf.h>
#include <filesystem>
#if !defined(ARCADIA_BUILD)
# include <Common/config_version.h>
@ -70,6 +69,7 @@
#endif
#include <ucontext.h>
namespace fs = std::filesystem;
DB::PipeFDs signal_pipe;
@ -437,11 +437,11 @@ static void sanitizerDeathCallback()
static std::string createDirectory(const std::string & file)
{
auto path = Poco::Path(file).makeParent();
if (path.toString().empty())
fs::path path = fs::path(file).parent_path();
if (path.empty())
return "";
Poco::File(path).createDirectories();
return path.toString();
fs::create_directories(path);
return path;
};
@ -449,7 +449,7 @@ static bool tryCreateDirectories(Poco::Logger * logger, const std::string & path
{
try
{
Poco::File(path).createDirectories();
fs::create_directories(path);
return true;
}
catch (...)
@ -470,7 +470,7 @@ void BaseDaemon::reloadConfiguration()
*/
config_path = config().getString("config-file", getDefaultConfigFileName());
DB::ConfigProcessor config_processor(config_path, false, true);
config_processor.setConfigPath(Poco::Path(config_path).makeParent().toString());
config_processor.setConfigPath(fs::path(config_path).parent_path());
loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true);
if (last_configuration != nullptr)
@ -524,18 +524,20 @@ std::string BaseDaemon::getDefaultConfigFileName() const
void BaseDaemon::closeFDs()
{
#if defined(OS_FREEBSD) || defined(OS_DARWIN)
Poco::File proc_path{"/dev/fd"};
fs::path proc_path{"/dev/fd"};
#else
Poco::File proc_path{"/proc/self/fd"};
fs::path proc_path{"/proc/self/fd"};
#endif
if (proc_path.isDirectory()) /// Hooray, proc exists
if (fs::is_directory(proc_path)) /// Hooray, proc exists
{
std::vector<std::string> fds;
/// in /proc/self/fd directory filenames are numeric file descriptors
proc_path.list(fds);
for (const auto & fd_str : fds)
/// in /proc/self/fd directory filenames are numeric file descriptors.
/// Iterate directory separately from closing fds to avoid closing iterated directory fd.
std::vector<int> fds;
for (const auto & path : fs::directory_iterator(proc_path))
fds.push_back(DB::parse<int>(path.path().filename()));
for (const auto & fd : fds)
{
int fd = DB::parse<int>(fd_str);
if (fd > 2 && fd != signal_pipe.fds_rw[0] && fd != signal_pipe.fds_rw[1])
::close(fd);
}
@ -597,7 +599,7 @@ void BaseDaemon::initialize(Application & self)
{
/** When creating pid file and looking for config, will search for paths relative to the working path of the program when started.
*/
std::string path = Poco::Path(config().getString("application.path")).setFileName("").toString();
std::string path = fs::path(config().getString("application.path")).replace_filename("");
if (0 != chdir(path.c_str()))
throw Poco::Exception("Cannot change directory to " + path);
}
@ -645,7 +647,7 @@ void BaseDaemon::initialize(Application & self)
std::string log_path = config().getString("logger.log", "");
if (!log_path.empty())
log_path = Poco::Path(log_path).setFileName("").toString();
log_path = fs::path(log_path).replace_filename("");
/** Redirect stdout, stderr to separate files in the log directory (or in the specified file).
* Some libraries write to stderr in case of errors in debug mode,
@ -708,8 +710,7 @@ void BaseDaemon::initialize(Application & self)
tryCreateDirectories(&logger(), core_path);
Poco::File cores = core_path;
if (!(cores.exists() && cores.isDirectory()))
if (!(fs::exists(core_path) && fs::is_directory(core_path)))
{
core_path = !log_path.empty() ? log_path : "/opt/";
tryCreateDirectories(&logger(), core_path);

View File

@ -1,6 +1,5 @@
#include <daemon/SentryWriter.h>
#include <Poco/File.h>
#include <Poco/Util/Application.h>
#include <Poco/Util/LayeredConfiguration.h>
@ -25,6 +24,7 @@
# include <stdio.h>
# include <filesystem>
namespace fs = std::filesystem;
namespace
{
@ -53,8 +53,7 @@ void setExtras()
sentry_set_extra("physical_cpu_cores", sentry_value_new_int32(getNumberOfPhysicalCPUCores()));
if (!server_data_path.empty())
sentry_set_extra("disk_free_space", sentry_value_new_string(formatReadableSizeWithBinarySuffix(
Poco::File(server_data_path).freeSpace()).c_str()));
sentry_set_extra("disk_free_space", sentry_value_new_string(formatReadableSizeWithBinarySuffix(fs::space(server_data_path).free).c_str()));
}
void sentry_logger(sentry_level_e level, const char * message, va_list args, void *)
@ -110,12 +109,12 @@ void SentryWriter::initialize(Poco::Util::LayeredConfiguration & config)
if (enabled)
{
server_data_path = config.getString("path", "");
const std::filesystem::path & default_tmp_path = std::filesystem::path(config.getString("tmp_path", Poco::Path::temp())) / "sentry";
const std::filesystem::path & default_tmp_path = fs::path(config.getString("tmp_path", fs::temp_directory_path())) / "sentry";
const std::string & endpoint
= config.getString("send_crash_reports.endpoint");
const std::string & temp_folder_path
= config.getString("send_crash_reports.tmp_path", default_tmp_path);
Poco::File(temp_folder_path).createDirectories();
fs::create_directories(temp_folder_path);
sentry_options_t * options = sentry_options_new(); /// will be freed by sentry_init or sentry_shutdown
sentry_options_set_release(options, VERSION_STRING_SHORT);

View File

@ -6,10 +6,11 @@
#include "OwnFormattingChannel.h"
#include "OwnPatternFormatter.h"
#include <Poco/ConsoleChannel.h>
#include <Poco/File.h>
#include <Poco/Logger.h>
#include <Poco/Net/RemoteSyslogChannel.h>
#include <Poco/Path.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -20,11 +21,11 @@ namespace DB
// TODO: move to libcommon
static std::string createDirectory(const std::string & file)
{
auto path = Poco::Path(file).makeParent();
if (path.toString().empty())
auto path = fs::path(file).parent_path();
if (path.empty())
return "";
Poco::File(path).createDirectories();
return path.toString();
fs::create_directories(path);
return path;
};
void Loggers::setTextLog(std::shared_ptr<DB::TextLog> log, int max_priority)
@ -70,7 +71,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
// Set up two channel chains.
log_file = new Poco::FileChannel;
log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(log_path).absolute().toString());
log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::weakly_canonical(log_path));
log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M"));
log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number");
log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true"));
@ -102,7 +103,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
std::cerr << "Logging errors to " << errorlog_path << std::endl;
error_log_file = new Poco::FileChannel;
error_log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(errorlog_path).absolute().toString());
error_log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::weakly_canonical(errorlog_path));
error_log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M"));
error_log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number");
error_log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true"));

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/NuRaft vendored

@ -1 +1 @@
Subproject commit 95d6bbba579b3a4e4c2dede954f541ff6f3dba51
Subproject commit 2a1bf7d87b4a03561fc66fbb49cee8a288983c5d

2
contrib/cassandra vendored

@ -1 +1 @@
Subproject commit c097fb5c7e63cc430016d9a8b240d8e63fbefa52
Subproject commit eb9b68dadbb4417a2c132ad4a1c2fa76e65e6fc1

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

@ -1,5 +1,5 @@
# docker build -t yandex/clickhouse-integration-tests-runner .
FROM ubuntu:18.04
FROM ubuntu:20.04
RUN apt-get update \
&& env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
@ -14,7 +14,6 @@ RUN apt-get update \
wget \
git \
iproute2 \
module-init-tools \
cgroupfs-mount \
python3-pip \
tzdata \
@ -42,7 +41,6 @@ ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
ENV DOCKER_CHANNEL stable
ENV DOCKER_VERSION 5:19.03.13~3-0~ubuntu-bionic
RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add -
RUN add-apt-repository "deb [arch=amd64] https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}"
@ -66,17 +64,18 @@ RUN python3 -m pip install \
dict2xml \
dicttoxml \
docker \
docker-compose==1.22.0 \
docker-compose==1.28.2 \
grpcio \
grpcio-tools \
kafka-python \
kazoo \
minio \
protobuf \
psycopg2-binary==2.7.5 \
psycopg2-binary==2.8.6 \
pymongo \
pytest \
pytest-timeout \
pytest-xdist \
redis \
tzlocal \
urllib3 \
@ -86,6 +85,7 @@ RUN python3 -m pip install \
COPY modprobe.sh /usr/local/bin/modprobe
COPY dockerd-entrypoint.sh /usr/local/bin/
COPY compose/ /compose/
COPY misc/ /misc/
RUN set -x \
&& addgroup --system dockremap \
@ -94,7 +94,6 @@ RUN set -x \
&& echo 'dockremap:165536:65536' >> /etc/subuid \
&& echo 'dockremap:165536:65536' >> /etc/subgid
VOLUME /var/lib/docker
EXPOSE 2375
ENTRYPOINT ["dockerd-entrypoint.sh"]
CMD ["sh", "-c", "pytest $PYTEST_OPTS"]

View File

@ -1,7 +1,5 @@
version: '2.3'
services:
cassandra1:
image: cassandra
image: cassandra:4.0
restart: always
ports:
- 9043:9042

View File

@ -5,6 +5,10 @@ services:
hostname: hdfs1
restart: always
ports:
- 50075:50075
- 50070:50070
- ${HDFS_NAME_EXTERNAL_PORT}:${HDFS_NAME_INTERNAL_PORT} #50070
- ${HDFS_DATA_EXTERNAL_PORT}:${HDFS_DATA_INTERNAL_PORT} #50075
entrypoint: /etc/bootstrap.sh -d
volumes:
- type: ${HDFS_FS:-tmpfs}
source: ${HDFS_LOGS:-}
target: /usr/local/hadoop/logs

View File

@ -15,10 +15,11 @@ services:
image: confluentinc/cp-kafka:5.2.0
hostname: kafka1
ports:
- "9092:9092"
- ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT}
environment:
KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kafka1:19092
KAFKA_LISTENERS: INSIDE://:9092,OUTSIDE://:19092
KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA_EXTERNAL_PORT},OUTSIDE://kafka1:19092
KAFKA_ADVERTISED_HOST_NAME: kafka1
KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092
KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT
KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE
KAFKA_BROKER_ID: 1
@ -34,7 +35,7 @@ services:
image: confluentinc/cp-schema-registry:5.2.0
hostname: schema-registry
ports:
- "8081:8081"
- ${SCHEMA_REGISTRY_EXTERNAL_PORT}:${SCHEMA_REGISTRY_INTERNAL_PORT}
environment:
SCHEMA_REGISTRY_HOST_NAME: schema-registry
SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT

View File

@ -11,16 +11,18 @@ services:
- ${KERBERIZED_HDFS_DIR}/../../hdfs_configs/bootstrap.sh:/etc/bootstrap.sh:ro
- ${KERBERIZED_HDFS_DIR}/secrets:/usr/local/hadoop/etc/hadoop/conf
- ${KERBERIZED_HDFS_DIR}/secrets/krb_long.conf:/etc/krb5.conf:ro
- type: ${KERBERIZED_HDFS_FS:-tmpfs}
source: ${KERBERIZED_HDFS_LOGS:-}
target: /var/log/hadoop-hdfs
ports:
- 1006:1006
- 50070:50070
- 9010:9010
- ${KERBERIZED_HDFS_NAME_EXTERNAL_PORT}:${KERBERIZED_HDFS_NAME_INTERNAL_PORT} #50070
- ${KERBERIZED_HDFS_DATA_EXTERNAL_PORT}:${KERBERIZED_HDFS_DATA_INTERNAL_PORT} #1006
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

View File

@ -23,13 +23,13 @@ services:
# restart: always
hostname: kerberized_kafka1
ports:
- "9092:9092"
- "9093:9093"
- ${KERBERIZED_KAFKA_EXTERNAL_PORT}:${KERBERIZED_KAFKA_EXTERNAL_PORT}
environment:
KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://:9093
KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:9093
KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://0.0.0.0:${KERBERIZED_KAFKA_EXTERNAL_PORT}
KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:${KERBERIZED_KAFKA_EXTERNAL_PORT}
# KAFKA_LISTENERS: INSIDE://kerberized_kafka1:9092,OUTSIDE://kerberized_kafka1:19092
# KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kerberized_kafka1:19092
KAFKA_ADVERTISED_HOST_NAME: kerberized_kafka1
KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL: GSSAPI
KAFKA_SASL_ENABLED_MECHANISMS: GSSAPI
KAFKA_SASL_KERBEROS_SERVICE_NAME: kafka

View File

@ -6,8 +6,8 @@ services:
volumes:
- data1-1:/data1
- ${MINIO_CERTS_DIR:-}:/certs
ports:
- "9001:9001"
expose:
- ${MINIO_PORT}
environment:
MINIO_ACCESS_KEY: minio
MINIO_SECRET_KEY: minio123
@ -20,14 +20,14 @@ services:
# HTTP proxies for Minio.
proxy1:
image: yandex/clickhouse-s3-proxy
ports:
expose:
- "8080" # Redirect proxy port
- "80" # Reverse proxy port
- "443" # Reverse proxy port (secure)
proxy2:
image: yandex/clickhouse-s3-proxy
ports:
expose:
- "8080"
- "80"
- "443"
@ -35,7 +35,7 @@ services:
# Empty container to run proxy resolver.
resolver:
image: yandex/clickhouse-python-bottle
ports:
expose:
- "8080"
tty: true
depends_on:

View File

@ -7,5 +7,5 @@ services:
MONGO_INITDB_ROOT_USERNAME: root
MONGO_INITDB_ROOT_PASSWORD: clickhouse
ports:
- 27018:27017
- ${MONGO_EXTERNAL_PORT}:${MONGO_INTERNAL_PORT}
command: --profile=2 --verbose

View File

@ -1,10 +1,24 @@
version: '2.3'
services:
mysql1:
mysql57:
image: mysql:5.7
restart: always
environment:
MYSQL_ROOT_PASSWORD: clickhouse
ports:
- 3308:3306
command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency
MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST}
DATADIR: /mysql/
expose:
- ${MYSQL_PORT}
command: --server_id=100
--log-bin='mysql-bin-1.log'
--default-time-zone='+3:00'
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/mysql/error.log
--general-log=ON
--general-log-file=/mysql/general.log
volumes:
- type: ${MYSQL_LOGS_FS:-tmpfs}
source: ${MYSQL_LOGS:-}
target: /mysql/

View File

@ -12,3 +12,10 @@ services:
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/var/log/mysqld/error.log
--general-log=ON
--general-log-file=/var/log/mysqld/general.log
volumes:
- type: ${MYSQL_LOGS_FS:-tmpfs}
source: ${MYSQL_LOGS:-}
target: /var/log/mysqld/

View File

@ -0,0 +1,23 @@
version: '2.3'
services:
mysql80:
image: mysql:8.0
restart: always
environment:
MYSQL_ROOT_PASSWORD: clickhouse
MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST}
DATADIR: /mysql/
expose:
- ${MYSQL8_PORT}
command: --server_id=100 --log-bin='mysql-bin-1.log'
--default_authentication_plugin='mysql_native_password'
--default-time-zone='+3:00' --gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/mysql/error.log
--general-log=ON
--general-log-file=/mysql/general.log
volumes:
- type: ${MYSQL8_LOGS_FS:-tmpfs}
source: ${MYSQL8_LOGS:-}
target: /mysql/

View File

@ -1,15 +0,0 @@
version: '2.3'
services:
mysql8_0:
image: mysql:8.0
restart: 'no'
environment:
MYSQL_ROOT_PASSWORD: clickhouse
ports:
- 3309:3306
command: --server_id=100 --log-bin='mysql-bin-1.log'
--default_authentication_plugin='mysql_native_password'
--default-time-zone='+3:00'
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3

View File

@ -1,6 +1,6 @@
version: '2.3'
services:
mysql1:
mysql_client:
image: mysql:5.7
restart: always
environment:

View File

@ -5,19 +5,64 @@ services:
restart: always
environment:
MYSQL_ROOT_PASSWORD: clickhouse
ports:
- 3348:3306
MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST}
DATADIR: /mysql/
expose:
- ${MYSQL_CLUSTER_PORT}
command: --server_id=100
--log-bin='mysql-bin-2.log'
--default-time-zone='+3:00'
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/mysql/2_error.log
--general-log=ON
--general-log-file=/mysql/2_general.log
volumes:
- type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs}
source: ${MYSQL_CLUSTER_LOGS:-}
target: /mysql/
mysql3:
image: mysql:5.7
restart: always
environment:
MYSQL_ROOT_PASSWORD: clickhouse
ports:
- 3388:3306
MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST}
DATADIR: /mysql/
expose:
- ${MYSQL_CLUSTER_PORT}
command: --server_id=100
--log-bin='mysql-bin-3.log'
--default-time-zone='+3:00'
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/mysql/3_error.log
--general-log=ON
--general-log-file=/mysql/3_general.log
volumes:
- type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs}
source: ${MYSQL_CLUSTER_LOGS:-}
target: /mysql/
mysql4:
image: mysql:5.7
restart: always
environment:
MYSQL_ROOT_PASSWORD: clickhouse
ports:
- 3368:3306
MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST}
DATADIR: /mysql/
expose:
- ${MYSQL_CLUSTER_PORT}
command: --server_id=100
--log-bin='mysql-bin-4.log'
--default-time-zone='+3:00'
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/mysql/4_error.log
--general-log=ON
--general-log-file=/mysql/4_general.log
volumes:
- type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs}
source: ${MYSQL_CLUSTER_LOGS:-}
target: /mysql/

View File

@ -2,12 +2,24 @@ version: '2.3'
services:
postgres1:
image: postgres
command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"]
restart: always
environment:
POSTGRES_PASSWORD: mysecretpassword
ports:
- 5432:5432
expose:
- ${POSTGRES_PORT}
healthcheck:
test: ["CMD-SHELL", "pg_isready -U postgres"]
interval: 10s
timeout: 5s
retries: 5
networks:
default:
aliases:
- postgre-sql.local
default:
aliases:
- postgre-sql.local
environment:
POSTGRES_HOST_AUTH_METHOD: "trust"
POSTGRES_PASSWORD: mysecretpassword
PGDATA: /postgres/data
volumes:
- type: ${POSTGRES_LOGS_FS:-tmpfs}
source: ${POSTGRES_DIR:-}
target: /postgres/

View File

@ -2,22 +2,43 @@ version: '2.3'
services:
postgres2:
image: postgres
command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"]
restart: always
environment:
POSTGRES_HOST_AUTH_METHOD: "trust"
POSTGRES_PASSWORD: mysecretpassword
ports:
- 5421:5432
PGDATA: /postgres/data
expose:
- ${POSTGRES_PORT}
volumes:
- type: ${POSTGRES_LOGS_FS:-tmpfs}
source: ${POSTGRES2_DIR:-}
target: /postgres/
postgres3:
image: postgres
command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"]
restart: always
environment:
POSTGRES_HOST_AUTH_METHOD: "trust"
POSTGRES_PASSWORD: mysecretpassword
ports:
- 5441:5432
PGDATA: /postgres/data
expose:
- ${POSTGRES_PORT}
volumes:
- type: ${POSTGRES_LOGS_FS:-tmpfs}
source: ${POSTGRES3_DIR:-}
target: /postgres/
postgres4:
image: postgres
command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"]
restart: always
environment:
POSTGRES_HOST_AUTH_METHOD: "trust"
POSTGRES_PASSWORD: mysecretpassword
ports:
- 5461:5432
PGDATA: /postgres/data
expose:
- ${POSTGRES_PORT}
volumes:
- type: ${POSTGRES_LOGS_FS:-tmpfs}
source: ${POSTGRES4_DIR:-}
target: /postgres/

View File

@ -2,11 +2,15 @@ version: '2.3'
services:
rabbitmq1:
image: rabbitmq:3-management
image: rabbitmq:3-management-alpine
hostname: rabbitmq1
ports:
- "5672:5672"
- "15672:15672"
expose:
- ${RABBITMQ_PORT}
environment:
RABBITMQ_DEFAULT_USER: "root"
RABBITMQ_DEFAULT_PASS: "clickhouse"
RABBITMQ_LOG_BASE: /rabbitmq_logs/
volumes:
- type: ${RABBITMQ_LOGS_FS:-tmpfs}
source: ${RABBITMQ_LOGS:-}
target: /rabbitmq_logs/

View File

@ -4,5 +4,5 @@ services:
image: redis
restart: always
ports:
- 6380:6379
- ${REDIS_EXTERNAL_PORT}:${REDIS_INTERNAL_PORT}
command: redis-server --requirepass "clickhouse" --databases 32

View File

@ -0,0 +1,75 @@
version: '2.3'
services:
zoo1:
image: zookeeper:3.6.2
restart: always
environment:
ZOO_TICK_TIME: 500
ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181
ZOO_MY_ID: 1
JVMFLAGS: -Dzookeeper.forceSync=no
ZOO_SECURE_CLIENT_PORT: $ZOO_SECURE_CLIENT_PORT
command: ["zkServer.sh", "start-foreground"]
entrypoint: /zookeeper-ssl-entrypoint.sh
volumes:
- type: bind
source: /misc/zookeeper-ssl-entrypoint.sh
target: /zookeeper-ssl-entrypoint.sh
- type: bind
source: /misc/client.crt
target: /clickhouse-config/client.crt
- type: ${ZK_FS:-tmpfs}
source: ${ZK_DATA1:-}
target: /data
- type: ${ZK_FS:-tmpfs}
source: ${ZK_DATA_LOG1:-}
target: /datalog
zoo2:
image: zookeeper:3.6.2
restart: always
environment:
ZOO_TICK_TIME: 500
ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888
ZOO_MY_ID: 2
JVMFLAGS: -Dzookeeper.forceSync=no
ZOO_SECURE_CLIENT_PORT: $ZOO_SECURE_CLIENT_PORT
command: ["zkServer.sh", "start-foreground"]
entrypoint: /zookeeper-ssl-entrypoint.sh
volumes:
- type: bind
source: /misc/zookeeper-ssl-entrypoint.sh
target: /zookeeper-ssl-entrypoint.sh
- type: bind
source: /misc/client.crt
target: /clickhouse-config/client.crt
- type: ${ZK_FS:-tmpfs}
source: ${ZK_DATA2:-}
target: /data
- type: ${ZK_FS:-tmpfs}
source: ${ZK_DATA_LOG2:-}
target: /datalog
zoo3:
image: zookeeper:3.6.2
restart: always
environment:
ZOO_TICK_TIME: 500
ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181
ZOO_MY_ID: 3
JVMFLAGS: -Dzookeeper.forceSync=no
ZOO_SECURE_CLIENT_PORT: $ZOO_SECURE_CLIENT_PORT
command: ["zkServer.sh", "start-foreground"]
entrypoint: /zookeeper-ssl-entrypoint.sh
volumes:
- type: bind
source: /misc/zookeeper-ssl-entrypoint.sh
target: /zookeeper-ssl-entrypoint.sh
- type: bind
source: /misc/client.crt
target: /clickhouse-config/client.crt
- type: ${ZK_FS:-tmpfs}
source: ${ZK_DATA3:-}
target: /data
- type: ${ZK_FS:-tmpfs}
source: ${ZK_DATA_LOG3:-}
target: /datalog

View File

@ -2,17 +2,17 @@
set -e
mkdir -p /etc/docker/
cat > /etc/docker/daemon.json << EOF
{
echo '{
"ipv6": true,
"fixed-cidr-v6": "fd00::/8",
"ip-forward": true,
"log-level": "debug",
"storage-driver": "overlay2",
"insecure-registries" : ["dockerhub-proxy.sas.yp-c.yandex.net:5000"],
"registry-mirrors" : ["http://dockerhub-proxy.sas.yp-c.yandex.net:5000"]
}
EOF
}' | dd of=/etc/docker/daemon.json
dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 &>/var/log/somefile &
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 &
set +e
reties=0

View File

@ -0,0 +1,19 @@
-----BEGIN CERTIFICATE-----
MIIC/TCCAeWgAwIBAgIJANjx1QSR77HBMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV
BAMMCWxvY2FsaG9zdDAgFw0xODA3MzAxODE2MDhaGA8yMjkyMDUxNDE4MTYwOFow
FDESMBAGA1UEAwwJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB
CgKCAQEAs9uSo6lJG8o8pw0fbVGVu0tPOljSWcVSXH9uiJBwlZLQnhN4SFSFohfI
4K8U1tBDTnxPLUo/V1K9yzoLiRDGMkwVj6+4+hE2udS2ePTQv5oaMeJ9wrs+5c9T
4pOtlq3pLAdm04ZMB1nbrEysceVudHRkQbGHzHp6VG29Fw7Ga6YpqyHQihRmEkTU
7UCYNA+Vk7aDPdMS/khweyTpXYZimaK9f0ECU3/VOeG3fH6Sp2X6FN4tUj/aFXEj
sRmU5G2TlYiSIUMF2JPdhSihfk1hJVALrHPTU38SOL+GyyBRWdNcrIwVwbpvsvPg
pryMSNxnpr0AK0dFhjwnupIv5hJIOQIDAQABo1AwTjAdBgNVHQ4EFgQUjPLb3uYC
kcamyZHK4/EV8jAP0wQwHwYDVR0jBBgwFoAUjPLb3uYCkcamyZHK4/EV8jAP0wQw
DAYDVR0TBAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAM/ocuDvfPus/KpMVD51j
4IdlU8R0vmnYLQ+ygzOAo7+hUWP5j0yvq4ILWNmQX6HNvUggCgFv9bjwDFhb/5Vr
85ieWfTd9+LTjrOzTw4avdGwpX9G+6jJJSSq15tw5ElOIFb/qNA9O4dBiu8vn03C
L/zRSXrARhSqTW5w/tZkUcSTT+M5h28+Lgn9ysx4Ff5vi44LJ1NnrbJbEAIYsAAD
+UA+4MBFKx1r6hHINULev8+lCfkpwIaeS8RL+op4fr6kQPxnULw8wT8gkuc8I4+L
P9gg/xDHB44T3ADGZ5Ib6O0DJaNiToO6rnoaaxs0KkotbvDWvRoxEytSbXKoYjYp
0g==
-----END CERTIFICATE-----

View File

@ -81,8 +81,8 @@ if [[ ! -f "$ZOO_DATA_DIR/myid" ]]; then
echo "${ZOO_MY_ID:-1}" > "$ZOO_DATA_DIR/myid"
fi
mkdir -p $(dirname $ZOO_SSL_KEYSTORE_LOCATION)
mkdir -p $(dirname $ZOO_SSL_TRUSTSTORE_LOCATION)
mkdir -p "$(dirname $ZOO_SSL_KEYSTORE_LOCATION)"
mkdir -p "$(dirname $ZOO_SSL_TRUSTSTORE_LOCATION)"
if [[ ! -f "$ZOO_SSL_KEYSTORE_LOCATION" ]]; then
keytool -genkeypair -alias zookeeper -keyalg RSA -validity 365 -keysize 2048 -dname "cn=zookeeper" -keypass password -keystore $ZOO_SSL_KEYSTORE_LOCATION -storepass password -deststoretype pkcs12

View File

@ -90,7 +90,7 @@ clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits"
clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits"
clickhouse-client --query "SHOW TABLES FROM test"
./stress --hung-check --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" \
./stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" \
&& echo -e 'Test script exit code\tOK' >> /test_output/test_results.tsv \
|| echo -e 'Test script failed\tFAIL' >> /test_output/test_results.tsv

View File

@ -19,25 +19,25 @@ def get_skip_list_cmd(path):
def get_options(i):
options = ""
options = []
if 0 < i:
options += " --order=random"
options.append("--order=random")
if i % 3 == 1:
options += " --db-engine=Ordinary"
options.append("--db-engine=Ordinary")
if i % 3 == 2:
options += ''' --db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i)
options.append('''--client-option='allow_experimental_database_replicated=1' --db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i))
# If database name is not specified, new database is created for each functional test.
# Run some threads with one database for all tests.
if i % 2 == 1:
options += " --database=test_{}".format(i)
options.append(" --database=test_{}".format(i))
if i == 13:
options += " --client-option='memory_tracker_fault_probability=0.00001'"
options.append(" --client-option='memory_tracker_fault_probability=0.00001'")
return options
return ' '.join(options)
def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_time_limit):
@ -58,7 +58,11 @@ def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_t
time.sleep(0.5)
return pipes
def prepare_for_hung_check():
def compress_stress_logs(output_path, files_prefix):
cmd = f"cd {output_path} && tar -zcf stress_run_logs.tar.gz {files_prefix}* && rm {files_prefix}*"
check_output(cmd, shell=True)
def prepare_for_hung_check(drop_databases):
# FIXME this function should not exist, but...
# We attach gdb to clickhouse-server before running tests
@ -91,6 +95,17 @@ def prepare_for_hung_check():
# Long query from 00084_external_agregation
call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'" """, shell=True, stderr=STDOUT)
if drop_databases:
# Here we try to drop all databases in async mode. If some queries really hung, than drop will hung too.
# Otherwise we will get rid of queries which wait for background pool. It can take a long time on slow builds (more than 900 seconds).
databases = check_output('clickhouse client -q "SHOW DATABASES"', shell=True).decode('utf-8').strip().split()
for db in databases:
if db == "system":
continue
command = f'clickhouse client -q "DROP DATABASE {db}"'
# we don't wait for drop
Popen(command, shell=True)
# Wait for last queries to finish if any, not longer than 300 seconds
call("""clickhouse client -q "select sleepEachRow((
select maxOrDefault(300 - elapsed) + 1 from system.processes where query not like '%from system.processes%' and elapsed < 300
@ -116,10 +131,14 @@ if __name__ == "__main__":
parser.add_argument("--server-log-folder", default='/var/log/clickhouse-server')
parser.add_argument("--output-folder")
parser.add_argument("--global-time-limit", type=int, default=3600)
parser.add_argument("--num-parallel", default=cpu_count())
parser.add_argument("--num-parallel", type=int, default=cpu_count())
parser.add_argument('--hung-check', action='store_true', default=False)
# make sense only for hung check
parser.add_argument('--drop-databases', action='store_true', default=False)
args = parser.parse_args()
if args.drop_databases and not args.hung_check:
raise Exception("--drop-databases only used in hung check (--hung-check)")
func_pipes = []
func_pipes = run_func_test(args.test_cmd, args.output_folder, args.num_parallel, args.skip_func_tests, args.global_time_limit)
@ -135,8 +154,13 @@ if __name__ == "__main__":
time.sleep(5)
logging.info("All processes finished")
logging.info("Compressing stress logs")
compress_stress_logs(args.output_folder, "stress_test_run_")
logging.info("Logs compressed")
if args.hung_check:
have_long_running_queries = prepare_for_hung_check()
have_long_running_queries = prepare_for_hung_check(args.drop_databases)
logging.info("Checking if some queries hung")
cmd = "{} {} {}".format(args.test_cmd, "--hung-check", "00001_select_1")
res = call(cmd, shell=True, stderr=STDOUT)

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

@ -94,4 +94,6 @@ select * from products limit 1;
└───────────────┴─────────────────┘
```
[Original article](https://clickhouse.tech/docs/en/operations/table_engines/dictionary/) <!--hide-->
**See Also**
- [Dictionary function](../../../sql-reference/table-functions/dictionary.md#dictionary-function)

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

@ -547,7 +547,7 @@ Differs from JSON only in that data rows are output in arrays, not in objects.
Example:
``` json
```
// JSONCompact
{
"meta":
@ -579,7 +579,7 @@ Example:
}
```
```json
```
// JSONCompactString
{
"meta":

View File

@ -42,6 +42,8 @@ toc_title: Client Libraries
- Ruby
- [ClickHouse (Ruby)](https://github.com/shlima/click_house)
- [clickhouse-activerecord](https://github.com/PNixx/clickhouse-activerecord)
- Rust
- [Klickhouse](https://github.com/Protryon/klickhouse)
- R
- [clickhouse-r](https://github.com/hannesmuehleisen/clickhouse-r)
- [RClickHouse](https://github.com/IMSMWU/RClickHouse)

View File

@ -123,6 +123,19 @@ The `Insert` command creates one or more blocks (parts). When inserting into Rep
A large number of `replicated_deduplication_window` slows down `Inserts` because it needs to compare more entries.
The hash sum is calculated from the composition of the field names and types and the data of the inserted part (stream of bytes).
## non_replicated_deduplication_window {#non-replicated-deduplication-window}
The number of the most recently inserted blocks in the non-replicated [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table for which hash sums are stored to check for duplicates.
Possible values:
- Any positive integer.
- 0 (disable deduplication).
Default value: 0.
A deduplication mechanism is used, similar to replicated tables (see [replicated_deduplication_window](#replicated-deduplication-window) setting). The hash sums of the created parts are written to a local file on a disk.
## replicated_deduplication_window_seconds {#replicated-deduplication-window-seconds}
The number of seconds after which the hash sums of the inserted blocks are removed from Zookeeper.

View File

@ -506,3 +506,256 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >=
## sumMapFiltered(keys_to_keep)(keys, values) {#summapfilteredkeys-to-keepkeys-values}
Same behavior as [sumMap](../../sql-reference/aggregate-functions/reference/summap.md#agg_functions-summap) except that an array of keys is passed as a parameter. This can be especially useful when working with a high cardinality of keys.
## sequenceNextNode {#sequenceNextNode}
Returns a value of next event that matched an event chain.
_Experimental function, `SET allow_experimental_funnel_functions = 1` to enable it._
**Syntax**
``` sql
sequenceNextNode(direction, base)(timestamp, event_column, base_condition, event1, event2, event3, ...)
```
**Parameters**
- `direction` - Used to navigate to directions.
- forward : Moving forward
- backward: Moving backward
- `base` - Used to set the base point.
- head : Set the base point to the first event
- tail : Set the base point to the last event
- first_match : Set the base point to the first matched event1
- last_match : Set the base point to the last matched event1
**Arguments**
- `timestamp` — Name of the column containing the timestamp. Data types supported: `Date`, `DateTime` and other unsigned integer types.
- `event_column` — Name of the column containing the value of the next event to be returned. Data types supported: `String` and `Nullable(String)`
- `base_condition` — Condition that the base point must fulfill.
- `cond` — Conditions describing the chain of events. `UInt8`
**Returned value**
- `event_column[next_index]` - if the pattern is matched and next value exists.
- `NULL` - if the pattern isnt matched or next value doesn't exist.
Type: `Nullable(String)`.
**Example**
It can be used when events are A->B->C->E->F and you want to know the event following B->C, which is E.
The query statement searching the event following A->B :
``` sql
CREATE TABLE test_flow (
dt DateTime,
id int,
page String)
ENGINE = MergeTree()
PARTITION BY toYYYYMMDD(dt)
ORDER BY id;
INSERT INTO test_flow VALUES (1, 1, 'A') (2, 1, 'B') (3, 1, 'C') (4, 1, 'E') (5, 1, 'F');
SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'A', page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id;
```
Result:
``` text
┌─id─┬─next_flow─┐
│ 1 │ C │
└────┴───────────┘
```
**Behavior for `forward` and `head`**
```SQL
ALTER TABLE test_flow DELETE WHERE 1 = 1 settings mutations_sync = 1;
INSERT INTO test_flow VALUES (1, 1, 'Home') (2, 1, 'Gift') (3, 1, 'Exit');
INSERT INTO test_flow VALUES (1, 2, 'Home') (2, 2, 'Home') (3, 2, 'Gift') (4, 2, 'Basket');
INSERT INTO test_flow VALUES (1, 3, 'Gift') (2, 3, 'Home') (3, 3, 'Gift') (4, 3, 'Basket');
```
```SQL
SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'Home', page = 'Home', page = 'Gift') FROM test_flow GROUP BY id;
dt id page
1970-01-01 09:00:01 1 Home // Base point, Matched with Home
1970-01-01 09:00:02 1 Gift // Matched with Gift
1970-01-01 09:00:03 1 Exit // The result
1970-01-01 09:00:01 2 Home // Base point, Matched with Home
1970-01-01 09:00:02 2 Home // Unmatched with Gift
1970-01-01 09:00:03 2 Gift
1970-01-01 09:00:04 2 Basket
1970-01-01 09:00:01 3 Gift // Base point, Unmatched with Home
1970-01-01 09:00:02 3 Home
1970-01-01 09:00:03 3 Gift
1970-01-01 09:00:04 3 Basket
```
**Behavior for `backward` and `tail`**
```SQL
SELECT id, sequenceNextNode('backward', 'tail')(dt, page, page = 'Basket', page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id;
dt id page
1970-01-01 09:00:01 1 Home
1970-01-01 09:00:02 1 Gift
1970-01-01 09:00:03 1 Exit // Base point, Unmatched with Basket
1970-01-01 09:00:01 2 Home
1970-01-01 09:00:02 2 Home // The result
1970-01-01 09:00:03 2 Gift // Matched with Gift
1970-01-01 09:00:04 2 Basket // Base point, Matched with Basket
1970-01-01 09:00:01 3 Gift
1970-01-01 09:00:02 3 Home // The result
1970-01-01 09:00:03 3 Gift // Base point, Matched with Gift
1970-01-01 09:00:04 3 Basket // Base point, Matched with Basket
```
**Behavior for `forward` and `first_match`**
```SQL
SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Gift') FROM test_flow GROUP BY id;
dt id page
1970-01-01 09:00:01 1 Home
1970-01-01 09:00:02 1 Gift // Base point
1970-01-01 09:00:03 1 Exit // The result
1970-01-01 09:00:01 2 Home
1970-01-01 09:00:02 2 Home
1970-01-01 09:00:03 2 Gift // Base point
1970-01-01 09:00:04 2 Basket The result
1970-01-01 09:00:01 3 Gift // Base point
1970-01-01 09:00:02 3 Home // Thre result
1970-01-01 09:00:03 3 Gift
1970-01-01 09:00:04 3 Basket
```
```SQL
SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Gift', page = 'Home') FROM test_flow GROUP BY id;
dt id page
1970-01-01 09:00:01 1 Home
1970-01-01 09:00:02 1 Gift // Base point
1970-01-01 09:00:03 1 Exit // Unmatched with Home
1970-01-01 09:00:01 2 Home
1970-01-01 09:00:02 2 Home
1970-01-01 09:00:03 2 Gift // Base point
1970-01-01 09:00:04 2 Basket // Unmatched with Home
1970-01-01 09:00:01 3 Gift // Base point
1970-01-01 09:00:02 3 Home // Matched with Home
1970-01-01 09:00:03 3 Gift // The result
1970-01-01 09:00:04 3 Basket
```
**Behavior for `backward` and `last_match`**
```SQL
SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Gift') FROM test_flow GROUP BY id;
dt id page
1970-01-01 09:00:01 1 Home // The result
1970-01-01 09:00:02 1 Gift // Base point
1970-01-01 09:00:03 1 Exit
1970-01-01 09:00:01 2 Home
1970-01-01 09:00:02 2 Home // The result
1970-01-01 09:00:03 2 Gift // Base point
1970-01-01 09:00:04 2 Basket
1970-01-01 09:00:01 3 Gift
1970-01-01 09:00:02 3 Home // The result
1970-01-01 09:00:03 3 Gift // Base point
1970-01-01 09:00:04 3 Basket
```
```SQL
SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Gift', page = 'Home') FROM test_flow GROUP BY id;
dt id page
1970-01-01 09:00:01 1 Home // Matched with Home, the result is null
1970-01-01 09:00:02 1 Gift // Base point
1970-01-01 09:00:03 1 Exit
1970-01-01 09:00:01 2 Home // The result
1970-01-01 09:00:02 2 Home // Matched with Home
1970-01-01 09:00:03 2 Gift // Base point
1970-01-01 09:00:04 2 Basket
1970-01-01 09:00:01 3 Gift // The result
1970-01-01 09:00:02 3 Home // Matched with Home
1970-01-01 09:00:03 3 Gift // Base point
1970-01-01 09:00:04 3 Basket
```
**Behavior for `base_condition`**
```SQL
CREATE TABLE test_flow_basecond
(
`dt` DateTime,
`id` int,
`page` String,
`ref` String
)
ENGINE = MergeTree
PARTITION BY toYYYYMMDD(dt)
ORDER BY id
INSERT INTO test_flow_basecond VALUES (1, 1, 'A', 'ref4') (2, 1, 'A', 'ref3') (3, 1, 'B', 'ref2') (4, 1, 'B', 'ref1');
```
```SQL
SELECT id, sequenceNextNode('forward', 'head')(dt, page, ref = 'ref1', page = 'A') FROM test_flow_basecond GROUP BY id;
dt id page ref
1970-01-01 09:00:01 1 A ref4 // The head can't be base point becasue the ref column of the head unmatched with 'ref1'.
1970-01-01 09:00:02 1 A ref3
1970-01-01 09:00:03 1 B ref2
1970-01-01 09:00:04 1 B ref1
```
```SQL
SELECT id, sequenceNextNode('backward', 'tail')(dt, page, ref = 'ref4', page = 'B') FROM test_flow_basecond GROUP BY id;
dt id page ref
1970-01-01 09:00:01 1 A ref4
1970-01-01 09:00:02 1 A ref3
1970-01-01 09:00:03 1 B ref2
1970-01-01 09:00:04 1 B ref1 // The tail can't be base point becasue the ref column of the tail unmatched with 'ref4'.
```
```SQL
SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, ref = 'ref3', page = 'A') FROM test_flow_basecond GROUP BY id;
dt id page ref
1970-01-01 09:00:01 1 A ref4 // This row can't be base point becasue the ref column unmatched with 'ref3'.
1970-01-01 09:00:02 1 A ref3 // Base point
1970-01-01 09:00:03 1 B ref2 // The result
1970-01-01 09:00:04 1 B ref1
```
```SQL
SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, ref = 'ref2', page = 'B') FROM test_flow_basecond GROUP BY id;
dt id page ref
1970-01-01 09:00:01 1 A ref4
1970-01-01 09:00:02 1 A ref3 // The result
1970-01-01 09:00:03 1 B ref2 // Base point
1970-01-01 09:00:04 1 B ref1 // This row can't be base point becasue the ref column unmatched with 'ref2'.
```

View File

@ -18,10 +18,10 @@ When using multiple `quantile*` functions with different levels in a query, the
**Syntax**
``` sql
quantileTDigest(level)(expr)
quantileTDigestWeighted(level)(expr, weight)
```
Alias: `medianTDigest`.
Alias: `medianTDigestWeighted`.
**Arguments**

View File

@ -98,6 +98,10 @@ Setting fields:
When dictionary with source `FILE` is created via DDL command (`CREATE DICTIONARY ...`), the source file needs to be located in `user_files` directory, to prevent DB users accessing arbitrary file on ClickHouse node.
**See Also**
- [Dictionary function](../../../sql-reference/table-functions/dictionary.md#dictionary-function)
## Executable File {#dicts-external_dicts_dict_sources-executable}
Working with executable files depends on [how the dictionary is stored in memory](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable files STDIN. Otherwise, ClickHouse starts executable file and treats its output as dictionary data.

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

@ -16,6 +16,7 @@ The following operations with [partitions](../../../engines/table-engines/merget
- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) — Resets the value of a specified column in a partition.
- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) — Resets the specified secondary index in a partition.
- [FREEZE PARTITION](#alter_freeze-partition) — Creates a backup of a partition.
- [UNFREEZE PARTITION](#alter_unfreeze-partition) — Removes a backup of a partition.
- [FETCH PARTITION\|PART](#alter_fetch-partition) — Downloads a part or partition from another server.
- [MOVE PARTITION\|PART](#alter_move-partition) — Move partition/data part to another disk or volume.
@ -160,7 +161,7 @@ This query creates a local backup of a specified partition. If the `PARTITION` c
!!! note "Note"
The entire backup process is performed without stopping the server.
Note that for old-styled tables you can specify the prefix of the partition name (for example, 2019) - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
Note that for old-styled tables you can specify the prefix of the partition name (for example, `2019`) - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
At the time of execution, for a data snapshot, the query creates hardlinks to a table data. Hardlinks are placed in the directory `/var/lib/clickhouse/shadow/N/...`, where:
@ -170,7 +171,7 @@ At the time of execution, for a data snapshot, the query creates hardlinks to a
!!! note "Note"
If you use [a set of disks for data storage in a table](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes), the `shadow/N` directory appears on every disk, storing data parts that matched by the `PARTITION` expression.
The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs chmod for all files, forbidding writing into them.
The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs `chmod` for all files, forbidding writing into them.
After creating the backup, you can copy the data from `/var/lib/clickhouse/shadow/` to the remote server and then delete it from the local server. Note that the `ALTER t FREEZE PARTITION` query is not replicated. It creates a local backup only on the local server.
@ -188,6 +189,14 @@ Restoring from a backup does not require stopping the server.
For more information about backups and restoring data, see the [Data Backup](../../../operations/backup.md) section.
## UNFREEZE PARTITION {#alter_unfreeze-partition}
``` sql
ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name'
```
Removes `freezed` partitions with the specified name from the disk. If the `PARTITION` clause is omitted, the query removes the backup of all partitions at once.
## CLEAR INDEX IN PARTITION {#alter_clear-index-partition}
``` sql

View File

@ -31,15 +31,17 @@ When `OPTIMIZE` is used with the [ReplicatedMergeTree](../../engines/table-engin
If you want to perform deduplication on custom set of columns rather than on all, you can specify list of columns explicitly or use any combination of [`*`](../../sql-reference/statements/select/index.md#asterisk), [`COLUMNS`](../../sql-reference/statements/select/index.md#columns-expression) or [`EXCEPT`](../../sql-reference/statements/select/index.md#except-modifier) expressions. The explictly written or implicitly expanded list of columns must include all columns specified in row ordering expression (both primary and sorting keys) and partitioning expression (partitioning key).
!!! note "Note"
Notice that `*` behaves just like in `SELECT`: `MATERIALIZED` and `ALIAS` columns are not used for expansion.
Also, it is an error to specify empty list of columns, or write an expression that results in an empty list of columns, or deduplicate by an ALIAS column.
Notice that `*` behaves just like in `SELECT`: [MATERIALIZED](../../sql-reference/statements/create/table.md#materialized) and [ALIAS](../../sql-reference/statements/create/table.md#alias) columns are not used for expansion.
Also, it is an error to specify empty list of columns, or write an expression that results in an empty list of columns, or deduplicate by an `ALIAS` column.
**Syntax**
``` sql
OPTIMIZE TABLE table DEDUPLICATE; -- the old one
OPTIMIZE TABLE table DEDUPLICATE BY *; -- not the same as the old one, excludes MATERIALIZED columns (see the note above)
OPTIMIZE TABLE table DEDUPLICATE; -- all columns
OPTIMIZE TABLE table DEDUPLICATE BY *; -- excludes MATERIALIZED and ALIAS columns
OPTIMIZE TABLE table DEDUPLICATE BY colX,colY,colZ;
OPTIMIZE TABLE table DEDUPLICATE BY * EXCEPT colX;
OPTIMIZE TABLE table DEDUPLICATE BY * EXCEPT (colX, colY);
OPTIMIZE TABLE table DEDUPLICATE BY col1,col2,col3;
OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex');
OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT colX;
OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT (colX, colY);
@ -47,7 +49,7 @@ OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT (c
**Examples**
Create a table:
Consider the table:
``` sql
CREATE TABLE example (
@ -62,32 +64,129 @@ CREATE TABLE example (
PARTITION BY partition_key
ORDER BY (primary_key, secondary_key);
```
``` sql
INSERT INTO example (primary_key, secondary_key, value, partition_key)
VALUES (0, 0, 0, 0), (0, 0, 0, 0), (1, 1, 2, 2), (1, 1, 2, 3), (1, 1, 3, 3);
```
``` sql
SELECT * FROM example;
```
Result:
```
The 'old' deduplicate, all columns are taken into account, i.e. row is removed only if all values in all columns are equal to corresponding values in previous row.
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 0 │ 0 │ 0 │ 0 │
│ 0 │ 0 │ 0 │ 0 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 2 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 3 │
│ 1 │ 1 │ 3 │ 3 │
└─────────────┴───────────────┴───────┴───────────────┘
```
When columns for deduplication are not specified, all of them are taken into account. Row is removed only if all values in all columns are equal to corresponding values in previous row:
``` sql
OPTIMIZE TABLE example FINAL DEDUPLICATE;
```
Deduplicate by all columns that are not `ALIAS` or `MATERIALIZED`: `primary_key`, `secondary_key`, `value`, `partition_key`, and `materialized_value` columns.
``` sql
SELECT * FROM example;
```
Result:
```
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 2 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 0 │ 0 │ 0 │ 0 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 3 │
│ 1 │ 1 │ 3 │ 3 │
└─────────────┴───────────────┴───────┴───────────────┘
```
When columns are specified implicitly, the table is deduplicated by all columns that are not `ALIAS` or `MATERIALIZED`. Considering the table above, these are `primary_key`, `secondary_key`, `value`, and `partition_key` columns:
```sql
OPTIMIZE TABLE example FINAL DEDUPLICATE BY *;
```
Deduplicate by all columns that are not `ALIAS` or `MATERIALIZED` and explicitly not `materialized_value`: `primary_key`, `secondary_key`, `value`, and `partition_key` columns.
``` sql
OPTIMIZE TABLE example FINAL DEDUPLICATE BY * EXCEPT materialized_value;
SELECT * FROM example;
```
Result:
```
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 2 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 0 │ 0 │ 0 │ 0 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 3 │
│ 1 │ 1 │ 3 │ 3 │
└─────────────┴───────────────┴───────┴───────────────┘
```
Deduplicate explicitly by `primary_key`, `secondary_key`, and `partition_key` columns.
Deduplicate by all columns that are not `ALIAS` or `MATERIALIZED` and explicitly not `value`: `primary_key`, `secondary_key`, and `partition_key` columns.
``` sql
OPTIMIZE TABLE example FINAL DEDUPLICATE BY * EXCEPT value;
```
``` sql
SELECT * FROM example;
```
Result:
```
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 2 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 0 │ 0 │ 0 │ 0 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 3 │
└─────────────┴───────────────┴───────┴───────────────┘
```
Deduplicate explicitly by `primary_key`, `secondary_key`, and `partition_key` columns:
```sql
OPTIMIZE TABLE example FINAL DEDUPLICATE BY primary_key, secondary_key, partition_key;
```
Deduplicate by any column matching a regex: `primary_key`, `secondary_key`, and `partition_key` columns.
``` sql
SELECT * FROM example;
```
Result:
```
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 2 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 0 │ 0 │ 0 │ 0 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 3 │
└─────────────┴───────────────┴───────┴───────────────┘
```
Deduplicate by any column matching a regex: `primary_key`, `secondary_key`, and `partition_key` columns:
```sql
OPTIMIZE TABLE example FINAL DEDUPLICATE BY COLUMNS('.*_key');
```
``` sql
SELECT * FROM example;
```
Result:
```
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 0 │ 0 │ 0 │ 0 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 2 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 3 │
└─────────────┴───────────────┴───────┴───────────────┘
```

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

@ -0,0 +1,59 @@
---
toc_priority: 54
toc_title: dictionary function
---
# dictionary {#dictionary-function}
Displays the [dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. Works the same way as [Dictionary](../../engines/table-engines/special/dictionary.md) engine.
**Syntax**
``` sql
dictionary('dict')
```
**Arguments**
- `dict` — A dictionary name. [String](../../sql-reference/data-types/string.md).
**Returned value**
A ClickHouse table.
**Example**
Input table `dictionary_source_table`:
``` text
┌─id─┬─value─┐
│ 0 │ 0 │
│ 1 │ 1 │
└────┴───────┘
```
Create a dictionary:
``` sql
CREATE DICTIONARY new_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table')) LAYOUT(DIRECT());
```
Query:
``` sql
SELECT * FROM dictionary('new_dictionary');
```
Result:
``` text
┌─id─┬─value─┐
│ 0 │ 0 │
│ 1 │ 1 │
└────┴───────┘
```
**See Also**
- [Dictionary engine](../../engines/table-engines/special/dictionary.md#dictionary)

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

@ -90,3 +90,6 @@ select * from products limit 1;
└───────────────┴─────────────────┘
```
**Смотрите также**
- [Функция dictionary](../../../sql-reference/table-functions/dictionary.md#dictionary-function)

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

@ -120,6 +120,19 @@ Eсли суммарное число активных кусков во все
Команда `Insert` создает один или несколько блоков (кусков). При вставке в Replicated таблицы ClickHouse для [дедупликации вставок](../../engines/table-engines/mergetree-family/replication.md) записывает в Zookeeper хеш-суммы созданных кусков. Но хранятся хеш-суммы не всех кусков, а только последние `replicated_deduplication_window`. Наиболее старые хеш-суммы удаляются из Zookeeper.
Большое число `replicated_deduplication_window` замедляет `Insert`-ы. Хеш-сумма рассчитывается от композиции имен и типов полей, а также данных вставленного куска (потока байт).
## non_replicated_deduplication_window {#non-replicated-deduplication-window}
Количество последних вставленных блоков в нереплицированной [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) таблице, для которых хранятся хеш-суммы для проверки дубликатов.
Возможные значения:
- Положительное целое число.
- 0 (дедупликация отключена).
Значение по умолчанию: 0.
Используется механизм дедупликации, аналогичный реплицированным таблицам (см. описание настройки [replicated_deduplication_window](#replicated-deduplication-window)). Хеш-суммы вставленных кусков записываются в локальный файл на диске.
## replicated_deduplication_window_seconds {#replicated-deduplication-window-seconds}
Число секунд, после которых хеш-суммы вставленных блоков удаляются из Zookeeper.

View File

@ -97,6 +97,10 @@ SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
Если словарь с источником `FILE` создается с помощью DDL-команды (`CREATE DICTIONARY ...`), источник словаря должен быть расположен в каталоге `user_files`. Иначе пользователи базы данных будут иметь доступ к произвольному файлу на узле ClickHouse.
**Смотрите также**
- [Функция dictionary](../../../sql-reference/table-functions/dictionary.md#dictionary-function)
## Исполняемый файл {#dicts-external_dicts_dict_sources-executable}
Работа с исполняемым файлом зависит от [размещения словаря в памяти](external-dicts-dict-layout.md). Если тип размещения словаря `cache` и `complex_key_cache`, то ClickHouse запрашивает необходимые ключи, отправляя запрос в `STDIN` исполняемого файла.

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

@ -16,6 +16,7 @@ toc_title: PARTITION
- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) — удалить все значения в столбце для заданной партиции;
- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) — очистить построенные вторичные индексы для заданной партиции;
- [FREEZE PARTITION](#alter_freeze-partition) — создать резервную копию партиции;
- [UNFREEZE PARTITION](#alter_unfreeze-partition) — удалить резервную копию партиции;
- [FETCH PARTITION](#alter_fetch-partition) — скачать партицию с другого сервера;
- [MOVE PARTITION\|PART](#alter_move-partition) — переместить партицию/кускок на другой диск или том.
@ -170,9 +171,9 @@ ALTER TABLE table_name FREEZE [PARTITION partition_expr]
!!! note "Примечание"
Создание резервной копии не требует остановки сервера.
Для таблиц старого стиля имя партиций можно задавать в виде префикса (например, 2019). В этом случае резервные копии будут созданы для всех соответствующих партиций. Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr).
Для таблиц старого стиля имя партиций можно задавать в виде префикса (например, `2019`). В этом случае, резервные копии будут созданы для всех соответствующих партиций. Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr).
Запрос делает следующее — для текущего состояния таблицы он формирует жесткие ссылки на данные в этой таблице. Ссылки размещаются в директории `/var/lib/clickhouse/shadow/N/...`, где:
Запрос формирует для текущего состояния таблицы жесткие ссылки на данные в этой таблице. Ссылки размещаются в директории `/var/lib/clickhouse/shadow/N/...`, где:
- `/var/lib/clickhouse/` — рабочая директория ClickHouse, заданная в конфигурационном файле;
- `N` — инкрементальный номер резервной копии.
@ -180,11 +181,11 @@ ALTER TABLE table_name FREEZE [PARTITION partition_expr]
!!! note "Примечание"
При использовании [нескольких дисков для хранения данных таблицы](../../statements/alter/index.md#table_engine-mergetree-multiple-volumes) директория `shadow/N` появляется на каждом из дисков, на которых были куски, попавшие под выражение `PARTITION`.
Структура директорий внутри резервной копии такая же, как внутри `/var/lib/clickhouse/`. Запрос выполнит chmod для всех файлов, запрещая запись в них.
Структура директорий внутри резервной копии такая же, как внутри `/var/lib/clickhouse/`. Запрос выполнит `chmod` для всех файлов, запрещая запись в них.
Обратите внимание, запрос `ALTER TABLE t FREEZE PARTITION` не реплицируется. Он создает резервную копию только на локальном сервере. После создания резервной копии данные из `/var/lib/clickhouse/shadow/` можно скопировать на удалённый сервер, а локальную копию удалить.
Резервная копия создается почти мгновенно (однако сначала запрос дожидается завершения всех запросов, которые выполняются для соответствующей таблицы).
Резервная копия создается почти мгновенно (однако, сначала запрос дожидается завершения всех запросов, которые выполняются для соответствующей таблицы).
`ALTER TABLE t FREEZE PARTITION` копирует только данные, но не метаданные таблицы. Чтобы сделать резервную копию метаданных таблицы, скопируйте файл `/var/lib/clickhouse/metadata/database/table.sql`
@ -198,6 +199,14 @@ ALTER TABLE table_name FREEZE [PARTITION partition_expr]
Подробнее о резервном копировании и восстановлении данных читайте в разделе [Резервное копирование данных](../../../operations/backup.md).
## UNFREEZE PARTITION {#alter_unfreeze-partition}
``` sql
ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name'
```
Удаляет с диска "замороженные" партиции с указанным именем. Если секция `PARTITION` опущена, запрос удаляет резервную копию всех партиций сразу.
## FETCH PARTITION {#alter_fetch-partition}
``` sql

View File

@ -32,10 +32,22 @@ ClickHouse не оповещает клиента. Чтобы включить
Список столбцов для дедупликации должен включать все столбцы, указанные в условиях сортировки (первичный ключ и ключ сортировки), а также в условиях партиционирования (ключ партиционирования).
!!! note "Примечание"
Обратите внимание, что символ подстановки `*` обрабатывается так же, как и в запросах `SELECT`: столбцы `MATERIALIZED` и `ALIAS` не включаются в результат.
Если указать пустой список или выражение, которое возвращает пустой список, или дедуплицировать столбец по псевдониму (`ALIAS`), то сервер вернет ошибку.
!!! note "Примечание"
Обратите внимание, что символ подстановки `*` обрабатывается так же, как и в запросах `SELECT`: столбцы [MATERIALIZED](../../sql-reference/statements/create/table.md#materialized) и [ALIAS](../../sql-reference/statements/create/table.md#alias) не включаются в результат.
Если указать пустой список или выражение, которое возвращает пустой список, то сервер вернет ошибку. Запрос вида `DEDUPLICATE BY aliased_value` также вернет ошибку.
**Синтаксис**
``` sql
OPTIMIZE TABLE table DEDUPLICATE; -- по всем столбцам
OPTIMIZE TABLE table DEDUPLICATE BY *; -- исключаются столбцы MATERIALIZED и ALIAS
OPTIMIZE TABLE table DEDUPLICATE BY colX,colY,colZ;
OPTIMIZE TABLE table DEDUPLICATE BY * EXCEPT colX;
OPTIMIZE TABLE table DEDUPLICATE BY * EXCEPT (colX, colY);
OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex');
OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT colX;
OPTIMIZE TABLE table DEDUPLICATE BY COLUMNS('column-matched-by-regex') EXCEPT (colX, colY);
```
**Примеры**
@ -50,38 +62,131 @@ CREATE TABLE example (
materialized_value UInt32 MATERIALIZED 12345,
aliased_value UInt32 ALIAS 2,
PRIMARY KEY primary_key
) ENGINE=MergeTree
PARTITION BY partition_key;
) ENGINE=MergeTree 
PARTITION BY partition_key
ORDER BY (primary_key, secondary_key);
```
``` sql
INSERT INTO example (primary_key, secondary_key, value, partition_key)
VALUES (0, 0, 0, 0), (0, 0, 0, 0), (1, 1, 2, 2), (1, 1, 2, 3), (1, 1, 3, 3);
```
``` sql
SELECT * FROM example;
```
Результат:
```
Прежний способ дедупликации, когда учитываются все столбцы. Строка удаляется только в том случае, если все значения во всех столбцах равны соответствующим значениям в предыдущей строке.
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 0 │ 0 │ 0 │ 0 │
│ 0 │ 0 │ 0 │ 0 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 2 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 3 │
│ 1 │ 1 │ 3 │ 3 │
└─────────────┴───────────────┴───────┴───────────────┘
```
Если в запросе не указаны столбцы, по которым нужно дедуплицировать, то учитываются все столбцы таблицы. Строка удаляется только в том случае, если все значения во всех столбцах равны соответствующим значениям в другой строке.
``` sql
OPTIMIZE TABLE example FINAL DEDUPLICATE;
```
Дедупликация по всем столбцам, кроме `ALIAS` и `MATERIALIZED`: `primary_key`, `secondary_key`, `value`, `partition_key` и `materialized_value`.
``` sql
OPTIMIZE TABLE example FINAL DEDUPLICATE BY *;
SELECT * FROM example;
```
Результат:
```
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 2 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 0 │ 0 │ 0 │ 0 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 3 │
│ 1 │ 1 │ 3 │ 3 │
└─────────────┴───────────────┴───────┴───────────────┘
```
Дедупликация по всем столбцам, кроме `ALIAS`, `MATERIALIZED` и `materialized_value`: столбцы `primary_key`, `secondary_key`, `value` и `partition_key`.
Если столбцы в запросе указаны через `*`, то дедупликация пройдет по всем столбцам, кроме `ALIAS` и `MATERIALIZED`. Для таблицы `example` будут учтены: `primary_key`, `secondary_key`, `value` и `partition_key`.
```sql
OPTIMIZE TABLE example FINAL DEDUPLICATE BY *;
```
``` sql
OPTIMIZE TABLE example FINAL DEDUPLICATE BY * EXCEPT materialized_value;
SELECT * FROM example;
```
Результат:
```
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 2 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 0 │ 0 │ 0 │ 0 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 3 │
│ 1 │ 1 │ 3 │ 3 │
└─────────────┴───────────────┴───────┴───────────────┘
```
Дедупликация по всем столбцам, кроме `ALIAS` и `MATERIALIZED` (`BY *`), и с исключением столбца `value`: `primary_key`, `secondary_key` и `partition_key`.
``` sql
OPTIMIZE TABLE example FINAL DEDUPLICATE BY * EXCEPT value;
```
``` sql
SELECT * FROM example;
```
Результат:
```
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 2 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 0 │ 0 │ 0 │ 0 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 3 │
└─────────────┴───────────────┴───────┴───────────────┘
```
Дедупликация по столбцам `primary_key`, `secondary_key` и `partition_key`.
``` sql
```sql
OPTIMIZE TABLE example FINAL DEDUPLICATE BY primary_key, secondary_key, partition_key;
```
Дедупликация по любому столбцу, соответствующему регулярному выражению: столбцам `primary_key`, `secondary_key` и `partition_key`.
``` sql
SELECT * FROM example;
```
Результат:
```
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 2 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 0 │ 0 │ 0 │ 0 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 3 │
└─────────────┴───────────────┴───────┴───────────────┘
```
Дедупликация по любому столбцу, который соответствует регулярному выражению `.*_key`: `primary_key`, `secondary_key` и `partition_key`.
```sql
OPTIMIZE TABLE example FINAL DEDUPLICATE BY COLUMNS('.*_key');
```
``` sql
SELECT * FROM example;
```
Результат:
```
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 0 │ 0 │ 0 │ 0 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 2 │
└─────────────┴───────────────┴───────┴───────────────┘
┌─primary_key─┬─secondary_key─┬─value─┬─partition_key─┐
│ 1 │ 1 │ 2 │ 3 │
└─────────────┴───────────────┴───────┴───────────────┘
```

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 сервера или сервера, используемого словарями).

View File

@ -94,7 +94,7 @@ INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def')
Обозначает, что значение отсутствует.
Чтобы в поле таблицы можно было хранить `NULL`, оно должно быть типа [Nullable](../sql-reference/syntax.md).
Чтобы в поле таблицы можно было хранить `NULL`, оно должно быть типа [Nullable](../sql-reference/data-types/nullable.md).
В зависимости от формата данных (входных или выходных) `NULL` может иметь различное представление. Подробнее смотрите в документации для [форматов данных](../interfaces/formats.md#formats).

View File

@ -0,0 +1,59 @@
---
toc_priority: 54
toc_title: dictionary
---
# dictionary {#dictionary-function}
Отображает данные [словаря](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) как таблицу ClickHouse. Работает аналогично движку [Dictionary](../../engines/table-engines/special/dictionary.md).
**Синтаксис**
``` sql
dictionary('dict')
```
**Аргументы**
- `dict` — имя словаря. [String](../../sql-reference/data-types/string.md).
**Возвращаемое значение**
Таблица ClickHouse.
**Пример**
Входная таблица `dictionary_source_table`:
``` text
┌─id─┬─value─┐
│ 0 │ 0 │
│ 1 │ 1 │
└────┴───────┘
```
Создаем словарь:
``` sql
CREATE DICTIONARY new_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table')) LAYOUT(DIRECT());
```
Запрос:
``` sql
SELECT * FROM dictionary('new_dictionary');
```
Результат:
``` text
┌─id─┬─value─┐
│ 0 │ 0 │
│ 1 │ 1 │
└────┴───────┘
```
**Смотрите также**
- [Движок Dictionary](../../engines/table-engines/special/dictionary.md#dictionary)

View File

@ -27,12 +27,12 @@ pymdown-extensions==8.0
python-slugify==4.0.1
PyYAML==5.4.1
repackage==0.7.3
requests==2.24.0
requests==2.25.1
singledispatch==3.4.0.3
six==1.15.0
soupsieve==2.0.1
termcolor==1.1.0
tornado==6.1
Unidecode==1.1.1
urllib3==1.25.10
urllib3>=1.26.5
Pygments>=2.7.4

View File

@ -8,7 +8,6 @@
#include <iomanip>
#include <random>
#include <pcg_random.hpp>
#include <Poco/File.h>
#include <Poco/Util/Application.h>
#include <Common/Stopwatch.h>
#include <Common/ThreadPool.h>
@ -36,7 +35,9 @@
#include <Common/Config/configReadClient.h>
#include <Common/TerminalSize.h>
#include <Common/StudentTTest.h>
#include <filesystem>
namespace fs = std::filesystem;
/** A tool for evaluating ClickHouse performance.
* The tool emulates a case with fixed amount of simultaneously executing queries.
@ -119,8 +120,8 @@ public:
int main(const std::vector<std::string> &) override
{
if (!json_path.empty() && Poco::File(json_path).exists()) /// Clear file with previous results
Poco::File(json_path).remove();
if (!json_path.empty() && fs::exists(json_path)) /// Clear file with previous results
fs::remove(json_path);
readQueries();
runBenchmark();
@ -159,7 +160,7 @@ private:
bool print_stacktrace;
const Settings & settings;
SharedContextHolder shared_context;
ContextPtr global_context;
ContextMutablePtr global_context;
QueryProcessingStage::Enum query_processing_stage;
/// Don't execute new queries after timelimit or SIGINT or exception

View File

@ -25,7 +25,6 @@
#include <boost/program_options.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <Poco/String.h>
#include <Poco/File.h>
#include <Poco/Util/Application.h>
#include <common/find_symbols.h>
#include <common/LineReader.h>
@ -87,6 +86,8 @@
#include <Common/TerminalSize.h>
#include <Common/UTF8Helpers.h>
#include <Common/ProgressBar.h>
#include <filesystem>
#include <Common/filesystemHelpers.h>
#if !defined(ARCADIA_BUILD)
# include <Common/config_version.h>
@ -96,6 +97,7 @@
#pragma GCC optimize("-fno-var-tracking-assignments")
#endif
namespace fs = std::filesystem;
namespace DB
{
@ -181,7 +183,7 @@ private:
bool has_vertical_output_suffix = false; /// Is \G present at the end of the query string?
SharedContextHolder shared_context = Context::createShared();
ContextPtr context = Context::createGlobal(shared_context.get());
ContextMutablePtr context = Context::createGlobal(shared_context.get());
/// Buffer that reads from stdin in batch mode.
ReadBufferFromFileDescriptor std_in{STDIN_FILENO};
@ -276,7 +278,7 @@ private:
/// Set path for format schema files
if (config().has("format_schema_path"))
context->setFormatSchemaPath(Poco::Path(config().getString("format_schema_path")).toString());
context->setFormatSchemaPath(fs::weakly_canonical(config().getString("format_schema_path")));
/// Initialize query_id_formats if any
if (config().has("query_id_formats"))
@ -633,8 +635,8 @@ private:
history_file = home_path + "/.clickhouse-client-history";
}
if (!history_file.empty() && !Poco::File(history_file).exists())
Poco::File(history_file).createFile();
if (!history_file.empty() && !fs::exists(history_file))
FS::createFile(history_file);
LineReader::Patterns query_extenders = {"\\"};
LineReader::Patterns query_delimiters = {";", "\\G"};

View File

@ -3,6 +3,9 @@
#include <Core/Settings.h>
#include <Columns/ColumnString.h>
#include <Common/typeid_cast.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
namespace DB
{
@ -90,7 +93,7 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo
/// NOTE: Once you will update the completion list,
/// do not forget to update 01676_clickhouse_client_autocomplete.sh
std::stringstream query; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
WriteBufferFromOwnString query;
query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM ("
"SELECT name FROM system.functions"
" UNION ALL "

View File

@ -12,14 +12,14 @@
namespace DB
{
class ClusterCopier : WithContext
class ClusterCopier : WithMutableContext
{
public:
ClusterCopier(const String & task_path_,
const String & host_id_,
const String & proxy_database_name_,
ContextPtr context_)
: WithContext(context_),
ContextMutablePtr context_)
: WithMutableContext(context_),
task_zookeeper_path(task_path_),
host_id(host_id_),
working_database_name(proxy_database_name_),

View File

@ -5,7 +5,9 @@
#include <Formats/registerFormats.h>
#include <ext/scope_guard_safe.h>
#include <unistd.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -26,7 +28,7 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self)
copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0);
if (config().has("move-fault-probability"))
move_fault_probability = std::max(std::min(config().getDouble("move-fault-probability"), 1.0), 0.0);
base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current();
base_dir = (config().has("base-dir")) ? config().getString("base-dir") : fs::current_path().string();
if (config().has("experimental-use-sample-offset"))
@ -38,18 +40,18 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self)
process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid);
host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id;
process_path = Poco::Path(base_dir + "/clickhouse-copier_" + process_id).absolute().toString();
Poco::File(process_path).createDirectories();
process_path = fs::weakly_canonical(fs::path(base_dir) / ("clickhouse-copier_" + process_id));
fs::create_directories(process_path);
/// Override variables for BaseDaemon
if (config().has("log-level"))
config().setString("logger.level", config().getString("log-level"));
if (config().has("base-dir") || !config().has("logger.log"))
config().setString("logger.log", process_path + "/log.log");
config().setString("logger.log", fs::path(process_path) / "log.log");
if (config().has("base-dir") || !config().has("logger.errorlog"))
config().setString("logger.errorlog", process_path + "/log.err.log");
config().setString("logger.errorlog", fs::path(process_path) / "log.err.log");
Base::initialize(self);
}

View File

@ -8,7 +8,6 @@
#include <Poco/FormattingChannel.h>
#include <Poco/PatternFormatter.h>
#include <Poco/UUIDGenerator.h>
#include <Poco/File.h>
#include <Poco/Process.h>
#include <Poco/FileChannel.h>
#include <Poco/SplitterChannel.h>

View File

@ -288,7 +288,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
bool is_symlink = fs::is_symlink(symlink_path);
fs::path points_to;
if (is_symlink)
points_to = fs::absolute(fs::read_symlink(symlink_path));
points_to = fs::weakly_canonical(fs::read_symlink(symlink_path));
if (is_symlink && points_to == main_bin_path)
{

View File

@ -32,7 +32,7 @@ public:
return BaseDaemon::logger();
}
ContextPtr context() const override
ContextMutablePtr context() const override
{
return global_context;
}
@ -58,7 +58,7 @@ protected:
std::string getDefaultConfigFileName() const override;
private:
ContextPtr global_context;
ContextMutablePtr global_context;
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const;

View File

@ -42,9 +42,9 @@
#include <common/argsToConfig.h>
#include <Common/TerminalSize.h>
#include <Common/randomSeed.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -72,11 +72,11 @@ void LocalServer::initialize(Poco::Util::Application & self)
Poco::Util::Application::initialize(self);
/// Load config files if exists
if (config().has("config-file") || Poco::File("config.xml").exists())
if (config().has("config-file") || fs::exists("config.xml"))
{
const auto config_path = config().getString("config-file", "config.xml");
ConfigProcessor config_processor(config_path, false, true);
config_processor.setConfigPath(Poco::Path(config_path).makeParent().toString());
config_processor.setConfigPath(fs::path(config_path).parent_path());
auto loaded_config = config_processor.loadConfig();
config_processor.savePreprocessedConfig(loaded_config, loaded_config.configuration->getString("path", "."));
config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false);
@ -100,7 +100,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
}
}
void LocalServer::applyCmdSettings(ContextPtr context)
void LocalServer::applyCmdSettings(ContextMutablePtr context)
{
context->applySettingsChanges(cmd_settings.changes());
}
@ -287,8 +287,8 @@ try
status.emplace(path + "status", StatusFile::write_full_info);
LOG_DEBUG(log, "Loading metadata from {}", path);
Poco::File(path + "data/").createDirectories();
Poco::File(path + "metadata/").createDirectories();
fs::create_directories(fs::path(path) / "data/");
fs::create_directories(fs::path(path) / "metadata/");
loadMetadataSystem(global_context);
attachSystemTables(global_context);
loadMetadata(global_context);
@ -479,7 +479,7 @@ void LocalServer::setupUsers()
{
ConfigurationPtr users_config;
if (config().has("users_config") || config().has("config-file") || Poco::File("config.xml").exists())
if (config().has("users_config") || config().has("config-file") || fs::exists("config.xml"))
{
const auto users_config_path = config().getString("users_config", config().getString("config-file", "config.xml"));
ConfigProcessor config_processor(users_config_path);
@ -645,7 +645,7 @@ void LocalServer::init(int argc, char ** argv)
argsToConfig(arguments, config(), 100);
}
void LocalServer::applyCmdOptions(ContextPtr context)
void LocalServer::applyCmdOptions(ContextMutablePtr context)
{
context->setDefaultFormat(config().getString("output-format", config().getString("format", "TSV")));
applyCmdSettings(context);

View File

@ -36,8 +36,8 @@ private:
std::string getInitialCreateTableQuery();
void tryInitPath();
void applyCmdOptions(ContextPtr context);
void applyCmdSettings(ContextPtr context);
void applyCmdOptions(ContextMutablePtr context);
void applyCmdSettings(ContextMutablePtr context);
void processQueries();
void setupUsers();
void cleanup();
@ -45,7 +45,7 @@ private:
protected:
SharedContextHolder shared_context;
ContextPtr global_context;
ContextMutablePtr global_context;
/// Settings specified via command line args
Settings cmd_settings;

View File

@ -1133,7 +1133,7 @@ try
}
SharedContextHolder shared_context = Context::createShared();
ContextPtr context = Context::createGlobal(shared_context.get());
auto context = Context::createGlobal(shared_context.get());
context->makeGlobalContext();
ReadBufferFromFileDescriptor file_in(STDIN_FILENO);

View File

@ -74,6 +74,7 @@
#include <Server/PostgreSQLHandlerFactory.h>
#include <Server/ProtocolServerAdapter.h>
#include <Server/HTTP/HTTPServer.h>
#include <filesystem>
#if !defined(ARCADIA_BUILD)
@ -117,6 +118,8 @@ namespace CurrentMetrics
extern const Metric MaxDDLEntryID;
}
namespace fs = std::filesystem;
#if USE_JEMALLOC
static bool jemallocOptionEnabled(const char *name)
{
@ -183,19 +186,19 @@ void setupTmpPath(Poco::Logger * log, const std::string & path)
{
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
Poco::File(path).createDirectories();
fs::create_directories(path);
/// Clearing old temporary files.
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator it(path); it != dir_end; ++it)
fs::directory_iterator dir_end;
for (fs::directory_iterator it(path); it != dir_end; ++it)
{
if (it->isFile() && startsWith(it.name(), "tmp"))
if (it->is_regular_file() && startsWith(it->path().filename(), "tmp"))
{
LOG_DEBUG(log, "Removing old temporary file {}", it->path());
it->remove();
LOG_DEBUG(log, "Removing old temporary file {}", it->path().string());
fs::remove(it->path());
}
else
LOG_DEBUG(log, "Skipped file in temporary path {}", it->path());
LOG_DEBUG(log, "Skipped file in temporary path {}", it->path().string());
}
}
@ -678,37 +681,38 @@ int Server::main(const std::vector<std::string> & /*args*/)
* Examples: do repair of local data; clone all replicated tables from replica.
*/
{
Poco::File(path + "flags/").createDirectories();
global_context->setFlagsPath(path + "flags/");
auto flags_path = fs::path(path) / "flags/";
fs::create_directories(flags_path);
global_context->setFlagsPath(flags_path);
}
/** Directory with user provided files that are usable by 'file' table function.
*/
{
std::string user_files_path = config().getString("user_files_path", path + "user_files/");
std::string user_files_path = config().getString("user_files_path", fs::path(path) / "user_files/");
global_context->setUserFilesPath(user_files_path);
Poco::File(user_files_path).createDirectories();
fs::create_directories(user_files_path);
}
{
std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", path + "dictionaries_lib/");
std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", fs::path(path) / "dictionaries_lib/");
global_context->setDictionariesLibPath(dictionaries_lib_path);
Poco::File(dictionaries_lib_path).createDirectories();
fs::create_directories(dictionaries_lib_path);
}
/// top_level_domains_lists
{
const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/") + "/";
TLDListsHolder::getInstance().parseConfig(top_level_domains_path, config());
const std::string & top_level_domains_path = config().getString("top_level_domains_path", fs::path(path) / "top_level_domains/");
TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config());
}
{
Poco::File(path + "data/").createDirectories();
Poco::File(path + "metadata/").createDirectories();
fs::create_directories(fs::path(path) / "data/");
fs::create_directories(fs::path(path) / "metadata/");
/// Directory with metadata of tables, which was marked as dropped by Atomic database
Poco::File(path + "metadata_dropped/").createDirectories();
fs::create_directories(fs::path(path) / "metadata_dropped/");
}
if (config().has("interserver_http_port") && config().has("interserver_https_port"))
@ -891,9 +895,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
#endif
/// Set path for format schema files
auto format_schema_path = Poco::File(config().getString("format_schema_path", path + "format_schemas/"));
global_context->setFormatSchemaPath(format_schema_path.path());
format_schema_path.createDirectories();
fs::path format_schema_path(config().getString("format_schema_path", fs::path(path) / "format_schemas/"));
global_context->setFormatSchemaPath(format_schema_path);
fs::create_directories(format_schema_path);
/// Check sanity of MergeTreeSettings on server startup
global_context->getMergeTreeSettings().sanityCheck(settings);
@ -1034,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

@ -40,7 +40,7 @@ public:
return BaseDaemon::logger();
}
ContextPtr context() const override
ContextMutablePtr context() const override
{
return global_context;
}
@ -64,7 +64,7 @@ protected:
std::string getDefaultCorePath() const override;
private:
ContextPtr global_context;
ContextMutablePtr global_context;
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const;
using CreateServerFunc = std::function<void(UInt16)>;

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

@ -7,7 +7,9 @@
#include <boost/range/algorithm/find.hpp>
#include <boost/range/algorithm_ext/erase.hpp>
#include <boost/algorithm/string/predicate.hpp>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -198,9 +200,9 @@ inline String AllowedClientHosts::IPSubnet::toString() const
if (isMaskAllBitsOne())
return prefix.toString();
else if (IPAddress{prefix_length, mask.family()} == mask)
return prefix.toString() + "/" + std::to_string(prefix_length);
return fs::path(prefix.toString()) / std::to_string(prefix_length);
else
return prefix.toString() + "/" + mask.toString();
return fs::path(prefix.toString()) / mask.toString();
}
inline bool AllowedClientHosts::IPSubnet::isMaskAllBitsOne() const

View File

@ -51,7 +51,7 @@ private:
void setAllNoLock(const std::vector<std::pair<UUID, AccessEntityPtr>> & all_entities, Notifications & notifications);
void prepareNotifications(const Entry & entry, bool remove, Notifications & notifications) const;
mutable std::mutex mutex;
mutable std::recursive_mutex mutex;
std::unordered_map<UUID, Entry> entries_by_id; /// We want to search entries both by ID and by the pair of name and type.
std::unordered_map<String, Entry *> entries_by_name_and_type[static_cast<size_t>(EntityType::MAX)];
mutable std::list<OnChangedHandler> handlers_by_type[static_cast<size_t>(EntityType::MAX)];

View File

@ -29,11 +29,11 @@ AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp(
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!isInteger(arguments[0]) && !isFloat(arguments[0]) && !isDateOrDateTime(arguments[0]))
if (!isInteger(arguments[0]) && !isFloat(arguments[0]) && !isDate(arguments[0]) && !isDateTime(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " +
name + ", must be Int, Float, Date, DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!isInteger(arguments[1]) && !isFloat(arguments[1]) && !isDateOrDateTime(arguments[1]))
if (!isInteger(arguments[1]) && !isFloat(arguments[1]) && !isDate(arguments[1]) && !isDateTime(arguments[1]))
throw Exception("Illegal type " + arguments[1]->getName() + " of argument for aggregate function " +
name + ", must be Int, Float, Date, DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

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) && !isDateOrDateTime(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

@ -53,7 +53,7 @@ public:
{
WhichDataType which{arguments.back()};
if (which.isNativeUInt() || which.isDateOrDateTime())
if (which.isNativeUInt() || which.isDate() || which.isDateTime() || which.isDateTime64())
{
UInt64 begin = params[params.size() - 3].safeGet<UInt64>();
UInt64 end = params[params.size() - 2].safeGet<UInt64>();

View File

@ -0,0 +1,144 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionSequenceNextNode.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <Core/Settings.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeNullable.h>
#include <Interpreters/Context.h>
#include <Common/CurrentThread.h>
#include <ext/range.h>
namespace DB
{
constexpr size_t max_events_size = 64;
constexpr size_t min_required_args = 3;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_AGGREGATE_FUNCTION;
}
namespace
{
template <typename T>
inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(
const DataTypePtr data_type, const DataTypes & argument_types, SequenceDirection direction, SequenceBase base)
{
return std::make_shared<SequenceNextNodeImpl<T, NodeString<max_events_size>>>(
data_type, argument_types, base, direction, min_required_args);
}
AggregateFunctionPtr
createAggregateFunctionSequenceNode(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
if (settings == nullptr || !settings->allow_experimental_funnel_functions)
{
throw Exception(
"Aggregate function " + name + " is experimental. Set `allow_experimental_funnel_functions` setting to enable it",
ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION);
}
if (parameters.size() < 2)
throw Exception("Aggregate function '" + name + "' requires 2 parameters (direction, head)",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
auto expected_param_type = Field::Types::Which::String;
if (parameters.at(0).getType() != expected_param_type || parameters.at(1).getType() != expected_param_type)
throw Exception("Aggregate function '" + name + "' requires 'String' parameters",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
String param_dir = parameters.at(0).safeGet<String>();
std::unordered_map<std::string, SequenceDirection> seq_dir_mapping{
{"forward", SequenceDirection::Forward},
{"backward", SequenceDirection::Backward},
};
if (!seq_dir_mapping.contains(param_dir))
throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_dir, ErrorCodes::BAD_ARGUMENTS};
SequenceDirection direction = seq_dir_mapping[param_dir];
String param_base = parameters.at(1).safeGet<String>();
std::unordered_map<std::string, SequenceBase> seq_base_mapping{
{"head", SequenceBase::Head},
{"tail", SequenceBase::Tail},
{"first_match", SequenceBase::FirstMatch},
{"last_match", SequenceBase::LastMatch},
};
if (!seq_base_mapping.contains(param_base))
throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_base, ErrorCodes::BAD_ARGUMENTS};
SequenceBase base = seq_base_mapping[param_base];
if ((base == SequenceBase::Head && direction == SequenceDirection::Backward) ||
(base == SequenceBase::Tail && direction == SequenceDirection::Forward))
throw Exception(fmt::format(
"Invalid argument combination of '{}' with '{}'", param_base, param_dir), ErrorCodes::BAD_ARGUMENTS);
if (argument_types.size() < min_required_args)
throw Exception("Aggregate function " + name + " requires at least " + toString(min_required_args) + " arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
bool is_base_match_type = base == SequenceBase::FirstMatch || base == SequenceBase::LastMatch;
if (is_base_match_type && argument_types.size() < min_required_args + 1)
throw Exception(
"Aggregate function " + name + " requires at least " + toString(min_required_args + 1) + " arguments when base is first_match or last_match.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (argument_types.size() > max_events_size + min_required_args)
throw Exception(fmt::format(
"Aggregate function '{}' requires at most {} (timestamp, value_column, ...{} events) arguments.",
name, max_events_size + min_required_args, max_events_size), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (const auto * cond_arg = argument_types[2].get(); cond_arg && !isUInt8(cond_arg))
throw Exception("Illegal type " + cond_arg->getName() + " of third argument of aggregate function "
+ name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
for (const auto i : ext::range(min_required_args, argument_types.size()))
{
const auto * cond_arg = argument_types[i].get();
if (!isUInt8(cond_arg))
throw Exception(fmt::format(
"Illegal type '{}' of {} argument of aggregate function '{}', must be UInt8", cond_arg->getName(), i + 1, name),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
if (WhichDataType(argument_types[1].get()).idx != TypeIndex::String)
throw Exception{"Illegal type " + argument_types[1].get()->getName()
+ " of second argument of aggregate function " + name + ", must be String",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
DataTypePtr data_type = makeNullable(argument_types[1]);
WhichDataType timestamp_type(argument_types[0].get());
if (timestamp_type.idx == TypeIndex::UInt8)
return createAggregateFunctionSequenceNodeImpl<UInt8>(data_type, argument_types, direction, base);
if (timestamp_type.idx == TypeIndex::UInt16)
return createAggregateFunctionSequenceNodeImpl<UInt16>(data_type, argument_types, direction, base);
if (timestamp_type.idx == TypeIndex::UInt32)
return createAggregateFunctionSequenceNodeImpl<UInt32>(data_type, argument_types, direction, base);
if (timestamp_type.idx == TypeIndex::UInt64)
return createAggregateFunctionSequenceNodeImpl<UInt64>(data_type, argument_types, direction, base);
if (timestamp_type.isDate())
return createAggregateFunctionSequenceNodeImpl<DataTypeDate::FieldType>(data_type, argument_types, direction, base);
if (timestamp_type.isDateTime())
return createAggregateFunctionSequenceNodeImpl<DataTypeDateTime::FieldType>(data_type, argument_types, direction, base);
throw Exception{"Illegal type " + argument_types.front().get()->getName()
+ " of first argument of aggregate function " + name + ", must be Unsigned Number, Date, DateTime",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
}
void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory)
{
AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false };
factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNode, properties });
}
}

View File

@ -0,0 +1,426 @@
#pragma once
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnNullable.h>
#include <Common/ArenaAllocator.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/AggregateFunctionNull.h>
#include <type_traits>
#include <bitset>
namespace DB
{
struct Settings;
enum class SequenceDirection
{
Forward,
Backward,
};
enum SequenceBase
{
Head,
Tail,
FirstMatch,
LastMatch,
};
/// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl
template <typename Node, size_t MaxEventsSize>
struct NodeBase
{
UInt64 size; /// size of payload
DataTypeDateTime::FieldType event_time;
std::bitset<MaxEventsSize> events_bitset;
bool can_be_base;
char * data() { return reinterpret_cast<char *>(this) + sizeof(Node); }
const char * data() const { return reinterpret_cast<const char *>(this) + sizeof(Node); }
Node * clone(Arena * arena) const
{
return reinterpret_cast<Node *>(
const_cast<char *>(arena->alignedInsert(reinterpret_cast<const char *>(this), sizeof(Node) + size, alignof(Node))));
}
void write(WriteBuffer & buf) const
{
writeVarUInt(size, buf);
buf.write(data(), size);
writeBinary(event_time, buf);
UInt64 ulong_bitset = events_bitset.to_ulong();
writeBinary(ulong_bitset, buf);
writeBinary(can_be_base, buf);
}
static Node * read(ReadBuffer & buf, Arena * arena)
{
UInt64 size;
readVarUInt(size, buf);
Node * node = reinterpret_cast<Node *>(arena->alignedAlloc(sizeof(Node) + size, alignof(Node)));
node->size = size;
buf.read(node->data(), size);
readBinary(node->event_time, buf);
UInt64 ulong_bitset;
readBinary(ulong_bitset, buf);
node->events_bitset = ulong_bitset;
readBinary(node->can_be_base, buf);
return node;
}
};
/// It stores String, timestamp, bitset of matched events.
template <size_t MaxEventsSize>
struct NodeString : public NodeBase<NodeString<MaxEventsSize>, MaxEventsSize>
{
using Node = NodeString<MaxEventsSize>;
static Node * allocate(const IColumn & column, size_t row_num, Arena * arena)
{
StringRef string = assert_cast<const ColumnString &>(column).getDataAt(row_num);
Node * node = reinterpret_cast<Node *>(arena->alignedAlloc(sizeof(Node) + string.size, alignof(Node)));
node->size = string.size;
memcpy(node->data(), string.data, string.size);
return node;
}
void insertInto(IColumn & column)
{
assert_cast<ColumnString &>(column).insertData(this->data(), this->size);
}
bool compare(const Node * rhs) const
{
auto cmp = strncmp(this->data(), rhs->data(), std::min(this->size, rhs->size));
return (cmp == 0) ? this->size < rhs->size : cmp < 0;
}
};
/// TODO : Support other types than string
template <typename Node>
struct SequenceNextNodeGeneralData
{
using Allocator = MixedAlignedArenaAllocator<alignof(Node *), 4096>;
using Array = PODArray<Node *, 32, Allocator>;
Array value;
bool sorted = false;
struct Comparator final
{
bool operator()(const Node * lhs, const Node * rhs) const
{
return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time < rhs->event_time;
}
};
void sort()
{
if (!sorted)
{
std::stable_sort(std::begin(value), std::end(value), Comparator{});
sorted = true;
}
}
};
/// Implementation of sequenceFirstNode
template <typename T, typename Node>
class SequenceNextNodeImpl final
: public IAggregateFunctionDataHelper<SequenceNextNodeGeneralData<Node>, SequenceNextNodeImpl<T, Node>>
{
using Self = SequenceNextNodeImpl<T, Node>;
using Data = SequenceNextNodeGeneralData<Node>;
static Data & data(AggregateDataPtr place) { return *reinterpret_cast<Data *>(place); }
static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast<const Data *>(place); }
static constexpr size_t base_cond_column_idx = 2;
static constexpr size_t event_column_idx = 1;
SequenceBase seq_base_kind;
SequenceDirection seq_direction;
const size_t min_required_args;
DataTypePtr & data_type;
UInt8 events_size;
UInt64 max_elems;
public:
SequenceNextNodeImpl(
const DataTypePtr & data_type_,
const DataTypes & arguments,
SequenceBase seq_base_kind_,
SequenceDirection seq_direction_,
size_t min_required_args_,
UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: IAggregateFunctionDataHelper<SequenceNextNodeGeneralData<Node>, Self>({data_type_}, {})
, seq_base_kind(seq_base_kind_)
, seq_direction(seq_direction_)
, min_required_args(min_required_args_)
, data_type(this->argument_types[0])
, events_size(arguments.size() - min_required_args)
, max_elems(max_elems_)
{
}
String getName() const override { return "sequenceNextNode"; }
DataTypePtr getReturnType() const override { return data_type; }
AggregateFunctionPtr getOwnNullAdapter(
const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params,
const AggregateFunctionProperties &) const override
{
/// Even though some values are mapped to aggregating key, it could return nulls for the below case.
/// aggregated events: [A -> B -> C]
/// events to find: [C -> D]
/// [C -> D] is not matched to 'A -> B -> C' so that it returns null.
return std::make_shared<AggregateFunctionNullVariadic<false, false, true>>(nested_function, arguments, params);
}
void insert(Data & a, const Node * v, Arena * arena) const
{
++a.total_values;
a.value.push_back(v->clone(arena), arena);
}
void create(AggregateDataPtr place) const override
{
new (place) Data;
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Node * node = Node::allocate(*columns[event_column_idx], row_num, arena);
const auto timestamp = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
/// The events_bitset variable stores matched events in the form of bitset.
/// Each Nth-bit indicates that the Nth-event are matched.
/// For example, event1 and event3 is matched then the values of events_bitset is 0x00000005.
/// 0x00000000
/// + 1 (bit of event1)
/// + 4 (bit of event3)
node->events_bitset.reset();
for (UInt8 i = 0; i < events_size; ++i)
if (assert_cast<const ColumnVector<UInt8> *>(columns[min_required_args + i])->getData()[row_num])
node->events_bitset.set(i);
node->event_time = timestamp;
node->can_be_base = assert_cast<const ColumnVector<UInt8> *>(columns[base_cond_column_idx])->getData()[row_num];
data(place).value.push_back(node, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
if (data(rhs).value.empty())
return;
if (data(place).value.size() >= max_elems)
return;
auto & a = data(place).value;
auto & b = data(rhs).value;
const auto a_size = a.size();
const UInt64 new_elems = std::min(data(rhs).value.size(), static_cast<size_t>(max_elems) - data(place).value.size());
for (UInt64 i = 0; i < new_elems; ++i)
a.push_back(b[i]->clone(arena), arena);
/// Either sort whole container or do so partially merging ranges afterwards
using Comparator = typename SequenceNextNodeGeneralData<Node>::Comparator;
if (!data(place).sorted && !data(rhs).sorted)
std::stable_sort(std::begin(a), std::end(a), Comparator{});
else
{
const auto begin = std::begin(a);
const auto middle = std::next(begin, a_size);
const auto end = std::end(a);
if (!data(place).sorted)
std::stable_sort(begin, middle, Comparator{});
if (!data(rhs).sorted)
std::stable_sort(middle, end, Comparator{});
std::inplace_merge(begin, middle, end, Comparator{});
}
data(place).sorted = true;
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override
{
/// Temporarily do a const_cast to sort the values. It helps to reduce the computational burden on the initiator node.
this->data(const_cast<AggregateDataPtr>(place)).sort();
writeBinary(data(place).sorted, buf);
auto & value = data(place).value;
size_t size = std::min(static_cast<size_t>(events_size + 1), value.size());
switch (seq_base_kind)
{
case SequenceBase::Head:
writeVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
value[i]->write(buf);
break;
case SequenceBase::Tail:
writeVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
value[value.size() - size + i]->write(buf);
break;
case SequenceBase::FirstMatch:
case SequenceBase::LastMatch:
writeVarUInt(value.size(), buf);
for (auto & node : value)
node->write(buf);
break;
}
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override
{
readBinary(data(place).sorted, buf);
UInt64 size;
readVarUInt(size, buf);
if (unlikely(size == 0))
return;
auto & value = data(place).value;
value.resize(size, arena);
for (UInt64 i = 0; i < size; ++i)
value[i] = Node::read(buf, arena);
}
inline std::optional<size_t> getBaseIndex(Data & data) const
{
if (data.value.size() == 0)
return {};
switch (seq_base_kind)
{
case SequenceBase::Head:
if (data.value[0]->can_be_base)
return 0;
break;
case SequenceBase::Tail:
if (data.value[data.value.size() - 1]->can_be_base)
return data.value.size() - 1;
break;
case SequenceBase::FirstMatch:
for (size_t i = 0; i < data.value.size(); ++i)
{
if (data.value[i]->events_bitset.test(0) && data.value[i]->can_be_base)
return i;
}
break;
case SequenceBase::LastMatch:
for (size_t i = 0; i < data.value.size(); ++i)
{
auto reversed_i = data.value.size() - i - 1;
if (data.value[reversed_i]->events_bitset.test(0) && data.value[reversed_i]->can_be_base)
return reversed_i;
}
break;
}
return {};
}
/// This method returns an index of next node that matched the events.
/// matched events in the chain of events are represented as a bitmask.
/// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on.
UInt32 getNextNodeIndex(Data & data) const
{
const UInt32 unmatched_idx = data.value.size();
if (data.value.size() <= events_size)
return unmatched_idx;
data.sort();
std::optional<size_t> base_opt = getBaseIndex(data);
if (!base_opt.has_value())
return unmatched_idx;
UInt32 base = static_cast<UInt32>(base_opt.value());
if (events_size == 0)
return data.value.size() > 0 ? base : unmatched_idx;
UInt32 i = 0;
switch (seq_direction)
{
case SequenceDirection::Forward:
for (i = 0; i < events_size && base + i < data.value.size(); ++i)
if (!data.value[base + i]->events_bitset.test(i))
break;
return (i == events_size) ? base + i : unmatched_idx;
case SequenceDirection::Backward:
for (i = 0; i < events_size && i < base; ++i)
if (!data.value[base - i]->events_bitset.test(i))
break;
return (i == events_size) ? base - i : unmatched_idx;
}
__builtin_unreachable();
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & value = data(place).value;
UInt32 event_idx = getNextNodeIndex(this->data(place));
if (event_idx < value.size())
{
ColumnNullable & to_concrete = assert_cast<ColumnNullable &>(to);
value[event_idx]->insertInto(to_concrete.getNestedColumn());
to_concrete.getNullMapData().push_back(0);
}
else
{
to.insertDefault();
}
}
bool allocatesMemoryInArena() const override { return true; }
};
}

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

@ -17,7 +17,6 @@ namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int UNKNOWN_AGGREGATE_FUNCTION;
}
namespace
@ -25,15 +24,8 @@ namespace
template <template <typename> class Data>
AggregateFunctionPtr
createAggregateFunctionWindowFunnel(const std::string & name, const DataTypes & arguments, const Array & params, const Settings * settings)
createAggregateFunctionWindowFunnel(const std::string & name, const DataTypes & arguments, const Array & params, const Settings *)
{
if (settings == nullptr || !settings->allow_experimental_funnel_functions)
{
throw Exception(
"Aggregate function " + name + " is experimental. Set `allow_experimental_funnel_functions` setting to enable it",
ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION);
}
if (params.empty())
throw Exception{"Aggregate function " + name + " requires at least one parameter: <window>, [option, [option, ...]]", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};

View File

@ -48,6 +48,7 @@ void registerAggregateFunctionRankCorrelation(AggregateFunctionFactory &);
void registerAggregateFunctionMannWhitney(AggregateFunctionFactory &);
void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &);
void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &);
void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory &);
class AggregateFunctionCombinatorFactory;
void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &);
@ -63,7 +64,7 @@ void registerAggregateFunctionCombinatorDistinct(AggregateFunctionCombinatorFact
void registerWindowFunctions(AggregateFunctionFactory & factory);
void registerAggregateFunctionSegmentLengthSum(AggregateFunctionFactory &);
void registerAggregateFunctionIntervalLengthSum(AggregateFunctionFactory &);
void registerAggregateFunctions()
{
@ -109,12 +110,13 @@ void registerAggregateFunctions()
registerAggregateFunctionAggThrow(factory);
registerAggregateFunctionRankCorrelation(factory);
registerAggregateFunctionMannWhitney(factory);
registerAggregateFunctionSequenceNextNode(factory);
registerAggregateFunctionWelchTTest(factory);
registerAggregateFunctionStudentTTest(factory);
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,8 +44,8 @@ SRCS(
AggregateFunctionRankCorrelation.cpp
AggregateFunctionResample.cpp
AggregateFunctionRetention.cpp
AggregateFunctionSegmentLengthSum.cpp
AggregateFunctionSequenceMatch.cpp
AggregateFunctionSequenceNextNode.cpp
AggregateFunctionSimpleLinearRegression.cpp
AggregateFunctionSimpleState.cpp
AggregateFunctionState.cpp

View File

@ -3,9 +3,10 @@
#include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/ReadHelpers.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Path.h>
#include <Poco/URI.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -87,10 +88,10 @@ std::unique_ptr<ShellCommand> IBridgeHelper::startBridgeCommand() const
const auto & config = getConfig();
/// Path to executable folder
Poco::Path path{config.getString("application.dir", "/usr/bin")};
fs::path path(config.getString("application.dir", "/usr/bin"));
std::vector<std::string> cmd_args;
path.setFileName(serviceFileName());
path /= serviceFileName();
cmd_args.push_back("--http-port");
cmd_args.push_back(std::to_string(config.getUInt(configPrefix() + ".port", getDefaultPort())));
@ -126,7 +127,7 @@ std::unique_ptr<ShellCommand> IBridgeHelper::startBridgeCommand() const
LOG_TRACE(getLog(), "Starting {}", serviceAlias());
return ShellCommand::executeDirect(path.toString(), cmd_args, ShellCommandDestructorStrategy(true));
return ShellCommand::executeDirect(path.string(), cmd_args, ShellCommandDestructorStrategy(true));
}
}

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