Merge branch 'master' into copier-stuck

This commit is contained in:
mergify[bot] 2021-06-05 01:40:23 +00:00 committed by GitHub
commit d8ed474062
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
57 changed files with 6877 additions and 1235 deletions

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

@ -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

@ -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,6 +160,7 @@ 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>
```
@ -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

@ -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,6 +103,7 @@ 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>
```
@ -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

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

View File

@ -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

@ -57,7 +57,7 @@ void processNode(const YAML::Node & node, Poco::XML::Element & parent_xml_elemen
{
case YAML::NodeType::Scalar:
{
auto value = node.as<std::string>();
std::string value = node.as<std::string>();
Poco::AutoPtr<Poco::XML::Text> xml_value = xml_document->createTextNode(value);
parent_xml_element.appendChild(xml_value);
break;
@ -110,13 +110,13 @@ void processNode(const YAML::Node & node, Poco::XML::Element & parent_xml_elemen
{
const auto & key_node = key_value_pair.first;
const auto & value_node = key_value_pair.second;
auto key = key_node.as<std::string>();
std::string key = key_node.as<std::string>();
bool is_attribute = (key.starts_with(YAML_ATTRIBUTE_PREFIX) && value_node.IsScalar());
if (is_attribute)
{
/// we use substr(1) here to remove YAML_ATTRIBUTE_PREFIX from key
auto attribute_name = key.substr(1);
auto value = value_node.as<std::string>();
std::string value = value_node.as<std::string>();
parent_xml_element.setAttribute(attribute_name, value);
}
else
@ -148,7 +148,7 @@ Poco::AutoPtr<Poco::XML::Document> YAMLParser::parse(const String& path)
catch (const YAML::ParserException& e)
{
/// yaml-cpp cannot parse the file because its contents are incorrect
throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "Unable to parse YAML configuration file {}", path, e.what());
throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "Unable to parse YAML configuration file {}, {}", path, e.what());
}
catch (const YAML::BadFile&)
{
@ -158,7 +158,14 @@ Poco::AutoPtr<Poco::XML::Document> YAMLParser::parse(const String& path)
Poco::AutoPtr<Poco::XML::Document> xml = new Document;
Poco::AutoPtr<Poco::XML::Element> root_node = xml->createElement("yandex");
xml->appendChild(root_node);
processNode(node_yml, *root_node);
try
{
processNode(node_yml, *root_node);
}
catch (const YAML::TypedBadConversion<std::string>&)
{
throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with key or value which cannot be represented as string and cannot continue parsing of the file");
}
return xml;
}

View File

@ -80,3 +80,8 @@ target_link_libraries (average PRIVATE clickhouse_common_io)
add_executable (shell_command_inout shell_command_inout.cpp)
target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io)
if (ENABLE_FUZZING)
add_executable(YAML_fuzzer YAML_fuzzer.cpp ${SRCS})
target_link_libraries(YAML_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE})
endif ()

View File

@ -0,0 +1,39 @@
#include <iostream>
#include <fstream>
#include <string>
#include <cstdio>
#include <time.h>
#include <filesystem>
extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size)
{
/// How to test:
/// build ClickHouse with YAML_fuzzer.cpp
/// ./YAML_fuzzer YAML_CORPUS
/// where YAML_CORPUS is a directory with different YAML configs for libfuzzer
char file_name[L_tmpnam];
if (!std::tmpnam(file_name))
{
std::cerr << "Cannot create temp file!\n";
return 1;
}
std::string input = std::string(reinterpret_cast<const char*>(data), size);
DB::YAMLParser parser;
{
std::ofstream temp_file(file_name);
temp_file << input;
}
try
{
DB::YAMLParser::parse(std::string(file_name));
}
catch (...)
{
std::cerr << "YAML_fuzzer failed: " << getCurrentExceptionMessage() << std::endl;
return 1;
}
return 0;
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -169,7 +169,7 @@ StorageS3Source::StorageS3Source(
ContextPtr context_,
const ColumnsDescription & columns_,
UInt64 max_block_size_,
UInt64 s3_max_single_read_retries_,
UInt64 max_single_read_retries_,
const String compression_hint_,
const std::shared_ptr<Aws::S3::S3Client> & client_,
const String & bucket_,
@ -181,7 +181,7 @@ StorageS3Source::StorageS3Source(
, format(format_)
, columns_desc(columns_)
, max_block_size(max_block_size_)
, s3_max_single_read_retries(s3_max_single_read_retries_)
, max_single_read_retries(max_single_read_retries_)
, compression_hint(compression_hint_)
, client(client_)
, sample_block(sample_block_)
@ -202,7 +202,7 @@ bool StorageS3Source::initialize()
file_path = fs::path(bucket) / current_key;
read_buf = wrapReadBufferWithCompressionMethod(
std::make_unique<ReadBufferFromS3>(client, bucket, current_key, s3_max_single_read_retries), chooseCompressionMethod(current_key, compression_hint));
std::make_unique<ReadBufferFromS3>(client, bucket, current_key, max_single_read_retries), chooseCompressionMethod(current_key, compression_hint));
auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, getContext(), max_block_size);
reader = std::make_shared<InputStreamFromInputFormat>(input_format);
@ -326,7 +326,7 @@ StorageS3::StorageS3(
const String & secret_access_key_,
const StorageID & table_id_,
const String & format_name_,
UInt64 s3_max_single_read_retries_,
UInt64 max_single_read_retries_,
UInt64 min_upload_part_size_,
UInt64 max_single_part_upload_size_,
UInt64 max_connections_,
@ -339,7 +339,7 @@ StorageS3::StorageS3(
: IStorage(table_id_)
, client_auth{uri_, access_key_id_, secret_access_key_, max_connections_, {}, {}} /// Client and settings will be updated later
, format_name(format_name_)
, s3_max_single_read_retries(s3_max_single_read_retries_)
, max_single_read_retries(max_single_read_retries_)
, min_upload_part_size(min_upload_part_size_)
, max_single_part_upload_size(max_single_part_upload_size_)
, compression_method(compression_method_)
@ -407,7 +407,7 @@ Pipe StorageS3::read(
local_context,
metadata_snapshot->getColumns(),
max_block_size,
s3_max_single_read_retries,
max_single_read_retries,
compression_method,
client_auth.client,
client_auth.uri.bucket,
@ -494,7 +494,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory)
secret_access_key = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
}
UInt64 s3_max_single_read_retries = args.getLocalContext()->getSettingsRef().s3_max_single_read_retries;
UInt64 max_single_read_retries = args.getLocalContext()->getSettingsRef().s3_max_single_read_retries;
UInt64 min_upload_part_size = args.getLocalContext()->getSettingsRef().s3_min_upload_part_size;
UInt64 max_single_part_upload_size = args.getLocalContext()->getSettingsRef().s3_max_single_part_upload_size;
UInt64 max_connections = args.getLocalContext()->getSettingsRef().s3_max_connections;
@ -518,7 +518,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory)
secret_access_key,
args.table_id,
format_name,
s3_max_single_read_retries,
max_single_read_retries,
min_upload_part_size,
max_single_part_upload_size,
max_connections,

View File

@ -55,7 +55,7 @@ public:
ContextPtr context_,
const ColumnsDescription & columns_,
UInt64 max_block_size_,
UInt64 s3_max_single_read_retries_,
UInt64 max_single_read_retries_,
const String compression_hint_,
const std::shared_ptr<Aws::S3::S3Client> & client_,
const String & bucket,
@ -72,7 +72,7 @@ private:
String format;
ColumnsDescription columns_desc;
UInt64 max_block_size;
UInt64 s3_max_single_read_retries;
UInt64 max_single_read_retries;
String compression_hint;
std::shared_ptr<Aws::S3::S3Client> client;
Block sample_block;
@ -103,7 +103,7 @@ public:
const String & secret_access_key,
const StorageID & table_id_,
const String & format_name_,
UInt64 s3_max_single_read_retries_,
UInt64 max_single_read_retries_,
UInt64 min_upload_part_size_,
UInt64 max_single_part_upload_size_,
UInt64 max_connections_,
@ -150,7 +150,7 @@ private:
ClientAuthentificaiton client_auth;
String format_name;
UInt64 s3_max_single_read_retries;
UInt64 max_single_read_retries;
size_t min_upload_part_size;
size_t max_single_part_upload_size;
String compression_method;

View File

@ -12,6 +12,7 @@
#include <Parsers/ASTLiteral.h>
#include "registerTableFunctions.h"
namespace DB
{
@ -83,7 +84,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context
{
Poco::URI uri (filename);
S3::URI s3_uri (uri);
UInt64 s3_max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries;
UInt64 max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries;
UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size;
UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size;
UInt64 max_connections = context->getSettingsRef().s3_max_connections;
@ -94,7 +95,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context
secret_access_key,
StorageID(getDatabaseName(), table_name),
format,
s3_max_single_read_retries,
max_single_read_retries,
min_upload_part_size,
max_single_part_upload_size,
max_connections,

View File

@ -28,6 +28,7 @@
#include <memory>
#include <thread>
namespace DB
{
@ -109,7 +110,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl(
Poco::URI uri (filename);
S3::URI s3_uri (uri);
/// Actually this parameters are not used
UInt64 s3_max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries;
UInt64 max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries;
UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size;
UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size;
UInt64 max_connections = context->getSettingsRef().s3_max_connections;
@ -119,7 +120,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl(
secret_access_key,
StorageID(getDatabaseName(), table_name),
format,
s3_max_single_read_retries,
max_single_read_retries,
min_upload_part_size,
max_single_part_upload_size,
max_connections,

View File

@ -235,7 +235,7 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std
# Normalize randomized database names in stdout, stderr files.
os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stdout_file))
if not args.show_db_name:
if args.hide_db_name:
os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stderr_file))
if args.replicated_database:
os.system("LC_ALL=C sed -i -e 's|/auto_{{shard}}||g' {file}".format(file=stdout_file))
@ -1045,7 +1045,7 @@ if __name__ == '__main__':
parser.add_argument('--force-color', action='store_true', default=False)
parser.add_argument('--database', help='Database for tests (random name test_XXXXXX by default)')
parser.add_argument('--no-drop-if-fail', action='store_true', help='Do not drop database for test if test has failed')
parser.add_argument('--show-db-name', action='store_true', help='Do not replace random database name with "default"')
parser.add_argument('--hide-db-name', action='store_true', help='Replace random database name with "default" in stderr')
parser.add_argument('--parallel', default='1/1', help='One parallel test run number/total')
parser.add_argument('-j', '--jobs', default=1, nargs='?', type=int, help='Run all tests in parallel')
parser.add_argument('--test-runs', default=1, nargs='?', type=int, help='Run each test many times (useful for e.g. flaky check)')

View File

@ -1,12 +0,0 @@
<yandex>
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/log.log</log>
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
<size>1000M</size>
<count>10</count>
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
</logger>
</yandex>

View File

@ -1,4 +0,0 @@
<yandex>
<merge_tree replace="replace">
</merge_tree>
</yandex>

View File

@ -1,68 +0,0 @@
<yandex>
<remote_servers>
<test_cluster>
<shard>
<!-- both latest version -->
<internal_replication>true</internal_replication>
<replica>
<default_database>test</default_database>
<host>node1</host>
<port>9000</port>
</replica>
<replica>
<default_database>test</default_database>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
<test_cluster>
<shard>
<!-- 19.6 + latest version -->
<internal_replication>true</internal_replication>
<replica>
<default_database>test</default_database>
<host>node3</host>
<port>9000</port>
</replica>
<replica>
<default_database>test</default_database>
<host>node4</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
<test_cluster>
<shard>
<!-- 19.1 + latest version -->
<internal_replication>true</internal_replication>
<replica>
<default_database>test</default_database>
<host>node5</host>
<port>9000</port>
</replica>
<replica>
<default_database>test</default_database>
<host>node6</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
<test_cluster>
<shard>
<!-- both 19.1 and update to new version -->
<internal_replication>true</internal_replication>
<replica>
<default_database>test</default_database>
<host>node11</host>
<port>9000</port>
</replica>
<replica>
<default_database>test</default_database>
<host>node12</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</yandex>

View File

@ -1,6 +0,0 @@
<yandex>
<merge_tree>
<min_rows_for_wide_part>0</min_rows_for_wide_part>
<min_bytes_for_wide_part>0</min_bytes_for_wide_part>
</merge_tree>
</yandex>

View File

@ -1,408 +0,0 @@
import time
import logging
import pytest
from helpers.client import QueryRuntimeException, QueryTimeoutExceedException
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server', tag='19.6.3.18', with_installed_binary=True)
node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/wide_parts_only.xml'], with_zookeeper=True)
node5 = cluster.add_instance('node5', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server', tag='19.1.15', with_installed_binary=True)
node6 = cluster.add_instance('node6', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/wide_parts_only.xml'], with_zookeeper=True)
node7 = cluster.add_instance('node7', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server', tag='19.6.3.18', stay_alive=True, with_installed_binary=True)
node8 = cluster.add_instance('node8', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server', tag='19.1.15', stay_alive=True, with_installed_binary=True)
node9 = cluster.add_instance('node9', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server', tag='19.1.15', stay_alive=True, with_installed_binary=True)
node10 = cluster.add_instance('node10', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server', tag='19.6.3.18', stay_alive=True, with_installed_binary=True)
node11 = cluster.add_instance('node11', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server', tag='19.1.15', stay_alive=True, with_installed_binary=True)
node12 = cluster.add_instance('node12', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server', tag='19.1.15', stay_alive=True, with_installed_binary=True)
def prepare_single_pair_with_setting(first_node, second_node, group):
for node in (first_node, second_node):
node.query("CREATE DATABASE IF NOT EXISTS test")
first_node.query("DROP TABLE IF EXISTS table_by_default")
second_node.query("DROP TABLE IF EXISTS table_by_default")
first_node.query("DROP TABLE IF EXISTS table_with_fixed_granularity")
second_node.query("DROP TABLE IF EXISTS table_with_fixed_granularity")
# Two tables with adaptive granularity
first_node.query(
'''
CREATE TABLE table_by_default(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_by_default', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 10485760
'''.format(g=group))
second_node.query(
'''
CREATE TABLE table_by_default(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_by_default', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 10485760
'''.format(g=group))
# Two tables with fixed granularity
first_node.query(
'''
CREATE TABLE table_with_fixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_fixed_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0
'''.format(g=group))
second_node.query(
'''
CREATE TABLE table_with_fixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_fixed_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0
'''.format(g=group))
# Two tables with different granularity
with pytest.raises(QueryRuntimeException):
first_node.query(
'''
CREATE TABLE table_with_different_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_different_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 10485760
'''.format(g=group))
second_node.query(
'''
CREATE TABLE table_with_different_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_different_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0
'''.format(g=group))
# Two tables with different granularity, but enabled mixed parts
first_node.query(
'''
CREATE TABLE table_with_mixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_mixed_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 10485760, enable_mixed_granularity_parts=1
'''.format(g=group))
second_node.query(
'''
CREATE TABLE table_with_mixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_mixed_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0, enable_mixed_granularity_parts=1
'''.format(g=group))
def prepare_single_pair_without_setting(first_node, second_node, group):
for node in (first_node, second_node):
node.query("CREATE DATABASE IF NOT EXISTS test")
# Two tables with fixed granularity
first_node.query(
'''
CREATE TABLE table_with_fixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_fixed_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
'''.format(g=group))
second_node.query(
'''
CREATE TABLE table_with_fixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_fixed_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0
'''.format(g=group))
@pytest.fixture(scope="module")
def start_static_cluster():
try:
cluster.start()
prepare_single_pair_with_setting(node1, node2, "shard1")
prepare_single_pair_with_setting(node3, node4, "shard2")
prepare_single_pair_without_setting(node5, node6, "shard3")
yield cluster
finally:
cluster.shutdown()
@pytest.mark.parametrize(
('first_node', 'second_node', 'table'),
[
(node1, node2, 'table_by_default'),
(node1, node2, 'table_with_fixed_granularity'),
(node3, node4, 'table_by_default'),
(node3, node4, 'table_with_fixed_granularity'),
(node5, node6, 'table_with_fixed_granularity'),
]
)
def test_different_versions_cluster(start_static_cluster, first_node, second_node, table):
counter = 1
for n1, n2 in ((first_node, second_node), (second_node, first_node)):
n1.query("INSERT INTO {tbl} VALUES (toDate('2018-10-01'), {c1}, 333), (toDate('2018-10-02'), {c2}, 444)".format(
tbl=table, c1=counter * 2, c2=counter * 2 + 1))
n2.query("SYSTEM SYNC REPLICA {tbl}".format(tbl=table))
assert_eq_with_retry(n2, "SELECT count() from {tbl}".format(tbl=table), str(counter * 2))
n1.query("DETACH TABLE {tbl}".format(tbl=table))
n2.query("DETACH TABLE {tbl}".format(tbl=table))
n1.query("ATTACH TABLE {tbl}".format(tbl=table))
n2.query("ATTACH TABLE {tbl}".format(tbl=table))
assert_eq_with_retry(n1, "SELECT count() from {tbl}".format(tbl=table), str(counter * 2))
assert_eq_with_retry(n2, "SELECT count() from {tbl}".format(tbl=table), str(counter * 2))
n1.query("OPTIMIZE TABLE {tbl} FINAL".format(tbl=table))
n2.query("SYSTEM SYNC REPLICA {tbl}".format(tbl=table))
assert_eq_with_retry(n1, "SELECT count() from {tbl}".format(tbl=table), str(counter * 2))
assert_eq_with_retry(n2, "SELECT count() from {tbl}".format(tbl=table), str(counter * 2))
counter += 1
@pytest.fixture(scope="module")
def start_dynamic_cluster():
try:
cluster.start()
node7.query(
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/7/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node7.query(
'''
CREATE TABLE table_with_adaptive_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/7/table_with_adaptive_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes=10485760
''')
node8.query(
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/8/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node9.query(
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/9/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node10.query(
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/10/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node11.query(
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard11/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node12.query(
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard11/table_with_default_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
yield cluster
finally:
cluster.shutdown()
@pytest.mark.parametrize(
('n', 'tables'),
[
(node7, ['table_with_default_granularity', 'table_with_adaptive_default_granularity']),
(node8, ['table_with_default_granularity']),
]
)
def test_version_single_node_update(start_dynamic_cluster, n, tables):
for table in tables:
n.query(
"INSERT INTO {tbl} VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)".format(tbl=table))
n.restart_with_latest_version()
for table in tables:
assert n.query("SELECT count() from {tbl}".format(tbl=table)) == '2\n'
n.query(
"INSERT INTO {tbl} VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)".format(tbl=table))
assert n.query("SELECT count() from {tbl}".format(tbl=table)) == '4\n'
@pytest.mark.parametrize(
('node',),
[
(node9,),
(node10,)
]
)
def test_mixed_granularity_single_node(start_dynamic_cluster, node):
node.query(
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)")
node.query(
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-09-01'), 1, 333), (toDate('2018-09-02'), 2, 444)")
path_to_part = node.query(
"SELECT path FROM system.parts WHERE table = 'table_with_default_granularity' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
result = node.exec_in_container(["bash", "-c", "find {p} -name '*.mrk*'".format(
p=path_to_part)]) # check that we have non adaptive files
logging.info(f"path {path_to_part} result\n {result}")
def callback(n):
new_config = """
<yandex><merge_tree>
<enable_mixed_granularity_parts>1</enable_mixed_granularity_parts>
<min_bytes_for_wide_part>0</min_bytes_for_wide_part>
</merge_tree></yandex>"""
n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", new_config)
n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", new_config)
node.restart_with_latest_version(callback_onstop=callback)
node.query("SYSTEM RELOAD CONFIG")
assert_eq_with_retry(node,
"SELECT value FROM system.merge_tree_settings WHERE name='enable_mixed_granularity_parts'",
'1')
assert node.query("SELECT count() from table_with_default_granularity") == '4\n'
node.query(
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)")
assert node.query("SELECT count() from table_with_default_granularity") == '6\n'
node.query("OPTIMIZE TABLE table_with_default_granularity PARTITION 201810 FINAL")
assert node.query("SELECT count() from table_with_default_granularity") == '6\n'
path_to_merged_part = node.query(
"SELECT path FROM system.parts WHERE table = 'table_with_default_granularity' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
node.exec_in_container(["bash", "-c", "find {p} -name '*.mrk2' | grep '.*'".format(
p=path_to_merged_part)]) # check that we have adaptive files
path_to_old_part = node.query(
"SELECT path FROM system.parts WHERE table = 'table_with_default_granularity' AND active=1 ORDER BY partition ASC LIMIT 1").strip()
node.exec_in_container(["bash", "-c", "find {p} -name '*.mrk' | grep '.*'".format(
p=path_to_old_part)]) # check that we have non adaptive files
node.query("ALTER TABLE table_with_default_granularity UPDATE dummy = dummy + 1 WHERE 1")
# still works
assert node.query("SELECT count() from table_with_default_granularity") == '6\n'
node.query("ALTER TABLE table_with_default_granularity MODIFY COLUMN dummy String")
node.query("ALTER TABLE table_with_default_granularity ADD COLUMN dummy2 Float64")
# still works
assert node.query("SELECT count() from table_with_default_granularity") == '6\n'
@pytest.mark.skip(reason="flaky")
def test_version_update_two_nodes(start_dynamic_cluster):
node11.query(
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)")
node12.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=20)
assert node12.query("SELECT COUNT() FROM table_with_default_granularity") == '2\n'
def callback(n):
new_config = """
<yandex><merge_tree>
<enable_mixed_granularity_parts>0</enable_mixed_granularity_parts>
<min_bytes_for_wide_part>0</min_bytes_for_wide_part>
</merge_tree></yandex>"""
n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", new_config)
n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", new_config)
node12.restart_with_latest_version(callback_onstop=callback)
node12.query(
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)")
node11.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=20)
assert node11.query("SELECT COUNT() FROM table_with_default_granularity") == '4\n'
node12.query(
'''
CREATE TABLE table_with_default_granularity_new(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard11/table_with_default_granularity_new', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node11.query(
'''
CREATE TABLE table_with_default_granularity_new(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard11/table_with_default_granularity_new', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node12.query(
"INSERT INTO table_with_default_granularity_new VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)")
with pytest.raises(QueryTimeoutExceedException):
node11.query("SYSTEM SYNC REPLICA table_with_default_granularity_new", timeout=20)
node12.query(
"INSERT INTO table_with_default_granularity_new VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)")
node11.restart_with_latest_version(callback_onstop=callback) # just to be sure
for i in range(3):
try:
node11.query("SYSTEM SYNC REPLICA table_with_default_granularity_new", timeout=120)
node12.query("SYSTEM SYNC REPLICA table_with_default_granularity_new", timeout=120)
break
except Exception as ex:
print(("Exception during replica sync", ex))
node11.query("SYSTEM RESTART REPLICA table_with_default_granularity_new")
node12.query("SYSTEM RESTART REPLICA table_with_default_granularity_new")
time.sleep(2 * i)
assert node11.query("SELECT COUNT() FROM table_with_default_granularity_new") == "4\n"
assert node12.query("SELECT COUNT() FROM table_with_default_granularity_new") == "4\n"
node11.query(
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 5, 333), (toDate('2018-10-02'), 6, 444)")
for i in range(3):
try:
node12.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=120)
break
except Exception as ex:
print(("Exception during replica sync", ex))
node11.query("SYSTEM RESTART REPLICA table_with_default_granularity")
node12.query("SYSTEM RESTART REPLICA table_with_default_granularity")
time.sleep(2 * i)
assert node12.query("SELECT COUNT() FROM table_with_default_granularity") == '6\n'

View File

@ -1,53 +0,0 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_zookeeper=True)
node2 = cluster.add_instance('node2', with_zookeeper=True)
# no adaptive granularity by default
node3 = cluster.add_instance('node3', image='yandex/clickhouse-server', tag='19.9.5.36', with_installed_binary=True,
stay_alive=True)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_attach_detach(start_cluster):
node1.query("""
CREATE TABLE test (key UInt64)
ENGINE = ReplicatedMergeTree('/clickhouse/test', '1')
ORDER BY tuple()
SETTINGS index_granularity_bytes = 0""")
node1.query("INSERT INTO test VALUES (1), (2)")
node2.query("""
CREATE TABLE test (key UInt64)
ENGINE = ReplicatedMergeTree('/clickhouse/test', '2')
ORDER BY tuple() SETTINGS enable_mixed_granularity_parts = 0""")
node2.query("INSERT INTO test VALUES (3), (4)")
node1.query_with_retry("SYSTEM SYNC REPLICA test", timeout=10)
node2.query_with_retry("SYSTEM SYNC REPLICA test", timeout=10)
assert node1.query("SELECT COUNT() FROM test") == "4\n"
assert node2.query("SELECT COUNT() FROM test") == "4\n"
node1.query("DETACH TABLE test")
node2.query("DETACH TABLE test")
node1.query("ATTACH TABLE test")
node2.query("ATTACH TABLE test")
assert node1.query("SELECT COUNT() FROM test") == "4\n"
assert node2.query("SELECT COUNT() FROM test") == "4\n"

View File

@ -1,104 +0,0 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_zookeeper=True)
node2 = cluster.add_instance('node2', with_zookeeper=True)
node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server', tag='19.1.14',
with_installed_binary=True)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
time.sleep(1)
yield cluster
finally:
cluster.shutdown()
def test_creating_table_different_setting(start_cluster):
node1.query(
"CREATE TABLE t1 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t1', '1') ORDER BY tuple(c1) SETTINGS index_granularity_bytes = 0")
node1.query("INSERT INTO t1 VALUES('x', 'y')")
node2.query(
"CREATE TABLE t1 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t1', '2') ORDER BY tuple(c1) SETTINGS enable_mixed_granularity_parts = 0")
node1.query("INSERT INTO t1 VALUES('a', 'b')")
node2.query("SYSTEM SYNC REPLICA t1", timeout=5)
node1.query("SELECT count() FROM t1") == "2\n"
node2.query("SELECT count() FROM t1") == "1\n"
node2.query("INSERT INTO t1 VALUES('c', 'd')")
node1.query("SYSTEM SYNC REPLICA t1", timeout=5)
# replication works
node1.query("SELECT count() FROM t1") == "3\n"
node2.query("SELECT count() FROM t1") == "2\n"
# OPTIMIZE also works correctly
node2.query("OPTIMIZE TABLE t1 FINAL") == "3\n"
node1.query("SYSTEM SYNC REPLICA t1", timeout=5)
node1.query("SELECT count() FROM t1") == "3\n"
node2.query("SELECT count() FROM t1") == "2\n"
path_part = node1.query(
"SELECT path FROM system.parts WHERE table = 't1' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
with pytest.raises(Exception): # check that we have no adaptive files
node1.exec_in_container(["bash", "-c", "find {p} -name '*.mrk2' | grep '.*'".format(p=path_part)])
path_part = node2.query(
"SELECT path FROM system.parts WHERE table = 't1' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
with pytest.raises(Exception): # check that we have no adaptive files
node2.exec_in_container(["bash", "-c", "find {p} -name '*.mrk2' | grep '.*'".format(p=path_part)])
def test_old_node_with_new_node(start_cluster):
node3.query(
"CREATE TABLE t2 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t2', '3') ORDER BY tuple(c1)")
node3.query("INSERT INTO t2 VALUES('x', 'y')")
node2.query(
"CREATE TABLE t2 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t2', '2') ORDER BY tuple(c1) SETTINGS enable_mixed_granularity_parts = 0")
node3.query("INSERT INTO t2 VALUES('a', 'b')")
node2.query("SYSTEM SYNC REPLICA t2", timeout=5)
node3.query("SELECT count() FROM t2") == "2\n"
node2.query("SELECT count() FROM t2") == "1\n"
node2.query("INSERT INTO t2 VALUES('c', 'd')")
node3.query("SYSTEM SYNC REPLICA t2", timeout=5)
# replication works
node3.query("SELECT count() FROM t2") == "3\n"
node2.query("SELECT count() FROM t2") == "2\n"
# OPTIMIZE also works correctly
node3.query("OPTIMIZE table t2 FINAL")
node2.query("SYSTEM SYNC REPLICA t2", timeout=5)
node3.query("SELECT count() FROM t2") == "3\n"
node2.query("SELECT count() FROM t2") == "2\n"
path_part = node3.query(
"SELECT path FROM system.parts WHERE table = 't2' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
with pytest.raises(Exception): # check that we have no adaptive files
node3.exec_in_container(["bash", "-c", "find {p} -name '*.mrk2' | grep '.*'".format(p=path_part)])
path_part = node2.query(
"SELECT path FROM system.parts WHERE table = 't2' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
with pytest.raises(Exception): # check that we have no adaptive files
node2.exec_in_container(["bash", "-c", "find {p} -name '*.mrk2' | grep '.*'".format(p=path_part)])

View File

@ -58,6 +58,8 @@ def test_inconsistent_parts_if_drop_while_replica_not_active(start_cluster):
# DROP_RANGE will be removed from the replication log and the first replica will be lost
for i in range(20):
node2.query("INSERT INTO test_table VALUES ('2019-08-16', {})".format(20 + i))
assert_eq_with_retry(node2, "SELECT value FROM system.zookeeper WHERE path='/clickhouse/tables/test1/replicated/replicas/node1' AND name='is_lost'", "1")
# the first replica will be cloned from the second
pm.heal_all()

View File

@ -40,7 +40,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler):
self.end_bytes = len(lines)
self.size = self.end_bytes
self.send_block_size = 256
self.stop_at = random.randint(900000, 1200000) // self.send_block_size # Block size is 1024**2.
self.stop_at = random.randint(900000, 1300000) // self.send_block_size # Block size is 1024**2.
if "Range" in self.headers:
cr = self.headers["Range"]

View File

@ -0,0 +1,20 @@
#!/usr/bin/env bash
export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
function check()
{
$CLICKHOUSE_CLIENT -q "SELECT id, $1(s) FROM ( SELECT number % 10 as id, uniqState(number) as s FROM ( SELECT number FROM system.numbers LIMIT 100 ) GROUP BY number ) GROUP BY id" 2>&1 | grep -v -P '^(Received exception from server|Code: 43)' ||:
}
stateFunctions=("uniqState" "uniqExactState" "uniqHLL12State" "uniqCombinedState" "uniqUpToState") # "uniqThetaState" not tested because its availability depends on compilation options
for i1 in "${stateFunctions[@]}"
do
check "$i1"
done

View File

@ -0,0 +1,5 @@
NULL
------
(1 row)

View File

@ -0,0 +1,12 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
echo "
DROP USER IF EXISTS postgresql_user;
CREATE USER postgresql_user HOST IP '127.0.0.1' IDENTIFIED WITH no_password;
" | $CLICKHOUSE_CLIENT -n
psql --host localhost --port ${CLICKHOUSE_PORT_POSTGRESQL} ${CLICKHOUSE_DATABASE} --user postgresql_user -c "SELECT NULL;"

View File

@ -406,7 +406,8 @@
"01053_ssd_dictionary",
"01280_ssd_complex_key_dictionary",
"01684_ssd_cache_dictionary_simple_key",
"01685_ssd_cache_dictionary_complex_key"
"01685_ssd_cache_dictionary_complex_key",
"01889_postgresql_protocol_null_fields"
],
"parallel":
[
@ -728,6 +729,7 @@
"01821_table_comment",
"01710_projection_fetch",
"01870_modulo_partition_key",
"01870_buffer_flush" // creates database
"01870_buffer_flush", // creates database
"01889_postgresql_protocol_null_fields"
]
}

File diff suppressed because it is too large Load Diff

3
utils/changelog/.gitignore vendored Normal file
View File

@ -0,0 +1,3 @@
*.md
*.txt
*.json

View File

@ -0,0 +1,8 @@
## Generate changelog
Usage example:
```
export GITHUB_USER=... GITHUB_TOKEN=ghp_...
./changelog.sh v21.5.6.6-stable v21.6.2.7-prestable
```

View File

@ -18,7 +18,6 @@ then
echo Some commits will be missed, review these manually.
fi
# NOTE keep in sync with ./backport.sh.
# Search for PR numbers in commit messages. First variant is normal merge, and second
# variant is squashed. Next are some backport message variants.
find_prs=(sed -n "s/^.*merg[eding]*.*#\([[:digit:]]\+\).*$/\1/Ip;
@ -39,6 +38,8 @@ function github_download()
local file=${2}
if ! [ -f "$file" ]
then
echo "curl -u \"$GITHUB_USER:***\" -sSf \"$url\" > \"$file\""
if ! curl -u "$GITHUB_USER:$GITHUB_TOKEN" \
-sSf "$url" \
> "$file"

View File

@ -1,3 +1,4 @@
v21.6.3.14-stable 2021-06-04
v21.5.6.6-stable 2021-05-29
v21.5.5.12-stable 2021-05-20
v21.4.7.3-stable 2021-05-19

1 v21.5.6.6-stable v21.6.3.14-stable 2021-05-29 2021-06-04
1 v21.6.3.14-stable 2021-06-04
2 v21.5.6.6-stable v21.5.6.6-stable 2021-05-29 2021-05-29
3 v21.5.5.12-stable v21.5.5.12-stable 2021-05-20 2021-05-20
4 v21.4.7.3-stable v21.4.7.3-stable 2021-05-19 2021-05-19

View File

@ -1 +0,0 @@
*

View File

@ -1,107 +0,0 @@
# Упрощённый скрипт для бекпортирования
Это упрощённый скрипт для бекпортирования. Он определяет, какие пулреквесты ещё не бекпортировали из мастера в указанную ветку. Запускать скрипт нужно из папки, где он лежит, указав ему название ветки. Он предполагает, что ваш апстримный remote называется origin.
```
cd my-clickhouse-repo/utils/simple-backport
git fetch origin
time GITHUB_TOKEN=<my github token> ./backport.sh 20.1
```
Скрипт выведет примитивный отчёт:
```
$ time GITHUB_TOKEN=<my github token> ~/backport.sh 20.3
144 PRs differ between 20.3 and master.
backport https://github.com/ClickHouse/ClickHouse/pull/10135
backport https://github.com/ClickHouse/ClickHouse/pull/10121
...
backport https://github.com/ClickHouse/ClickHouse/pull/9808
backport https://github.com/ClickHouse/ClickHouse/pull/9410
real 0m1.213s
user 0m1.065s
sys 0m0.311s
```
Также в рабочей папке сгенерируется отчёт `<ваша-ветка>-report.tsv`:
```
$ cat 20.3-report.tsv
skip 10153 https://github.com/ClickHouse/ClickHouse/pull/10153 pr10153.json
skip 10147 https://github.com/ClickHouse/ClickHouse/pull/10147 pr10147.json
no-backport 10138 https://github.com/ClickHouse/ClickHouse/pull/10138 pr10138.json
backport 10135 https://github.com/ClickHouse/ClickHouse/pull/10135 pr10135.json
skip 10134 https://github.com/ClickHouse/ClickHouse/pull/10134 pr10134.json
...
```
Можно кликать по ссылкам прям из консоли, а можно ещё проще:
```
$ cat <ветка>-report.tsv | grep ^backport | cut -f3
$ cat <ветка>-report.tsv | grep ^backport | cut -f3 | xargs -n1 xdg-open
```
Такая команда откроет в браузере все пулреквесты, которые надо бекпортировать. Есть и другие статусы, посмотрите какие:
```
$ cat 20.1-report.tsv | cut -f1 | sort | uniq -c | sort -rn
446 skip
38 done
25 conflict
18 backport
10 no-backport
```
### Как разметить пулреквест?
По умолчанию бекпортируются все пулреквесты, у которых в описании указана
категория чейнжлога Bug fix. Если этого недостаточно, используйте теги:
* v20.1-no-backport -- в ветку 20.1 бекпортировать не нужно.
* pr-no-backport -- ни в какие ветки бекпортировать не нужно.
* v20.1-conflicts -- при бекпорте в 20.1 произошёл конфликт. Такие пулреквесты
скрипт пропускает, к ним можно потом вернуться.
* pr-must-backport -- нужно бекпортировать в поддерживаемые ветки.
* v20.1-must-backport -- нужно бекпортировать в 20.1.
### Я бекпортировал, почему скрипт не видит?
* Сообщение коммита должно содержать текст backport/cherry-pick #12345, или
иметь вид стандартного гитхабовского мерж-коммита для ПР #12345.
* Коммит должен быть достижим по `git log --first-parent my-branch`. Возможно,
в ветке сделали pull с merge, от чего некоторые коммиты из ветки становятся
недоступны по `--first-parent`.
В качестве обхода, добавьте в ветку пустой коммит с текстом вроде "backport
#12345 -- real backport commit is <sha>".
### Я поправил пулреквест, почему скрипт не видит?
В процессе работы скрипт кеширует данные о пулреквестах в текущей папке, чтобы
экономить квоту гитхаба. Удалите закешированные файлы, например, для всех
реквестов, которые не помечены как пропущенные:
```
$ cat <ваша-ветка>-report.tsv | grep -v "^skip" | cut -f4
$ cat <ваша-ветка>-report.tsv | grep -v "^skip" | cut -f4 | xargs rm
```
## Как сформировать change log
В этой же папке запустите:
```
$ time GITHUB_TOKEN=... ./changelog.sh v20.3.4.10-stable v20.3.5.21-stable
9 PRs added between v20.3.4.10-stable and v20.3.5.21-stable.
### ClickHouse release v20.3.5.21-stable FIXME as compared to v20.3.4.10-stable
#### Bug Fix
* Fix 'Different expressions with the same alias' error when query has PREWHERE
and WHERE on distributed table and `SET distributed_product_mode = 'local'`.
[#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem
Zuikov](https://github.com/4ertus2)).
...
```
Скрипт выведет changelog на экран, а также сохранит его в `./changelog.md`.
Скопируйте этот текст в большой changelog, проверьте и поправьте версию и дату
релиза, вычитайте сообщения. Если сообщения неправильные, обязательно исправьте
их на гитхабе -- это поможет при последующей генерации changelog для других
версий, содержащих этот пулреквест. Чтобы скрипт подтянул изменения с гитхаба,
удалите соответствующие файлы `./pr12345.json`. Если вы часто видите
неправильно оформленные пулреквесты, это повод подумать об улучшении проверки
Description check в CI.

View File

@ -1,112 +0,0 @@
#!/bin/bash
set -e
branch="$1"
merge_base=$(git merge-base origin/master "origin/$branch")
master_git_cmd=(git log "$merge_base..origin/master" --first-parent)
# The history in back branches shouldn't be too crazy, and sometimes we have a PR
# that merges several backport commits there (3f2cba6824fddf31c30bde8c6f4f860572f4f580),
# so don't use --first-parent
branch_git_cmd=(git log "$merge_base..origin/$branch")
# Make lists of PRs that were merged into each branch. Use first parent here, or else
# we'll get weird things like seeing older master that was merged into a PR branch
# that was then merged into master.
"${master_git_cmd[@]}" > master-log.txt
"${branch_git_cmd[@]}" > "$branch-log.txt"
# Check for diamond merges.
diamonds_in_master=$("${master_git_cmd[@]}" --oneline --grep "Merge branch '")
diamonds_in_branch=$("${branch_git_cmd[@]}" --oneline --grep "Merge branch '")
if [ "$diamonds_in_master" != "" ] || [ "$diamonds_in_branch" != "" ]
then
echo "$diamonds_in_master"
echo "$diamonds_in_branch"
# DO NOT ADD automated handling of diamond merges to this script.
# It is an unsustainable way to work with git, and it MUST be visible.
echo Warning: suspected diamond merges above.
echo Some commits will be missed, review these manually.
fi
# NOTE keep in sync with ./backport.sh.
# Search for PR numbers in commit messages. First variant is normal merge, and second
# variant is squashed. Next are some backport message variants.
find_prs=(sed -n "s/^.*merg[eding]*.*#\([[:digit:]]\+\).*$/\1/Ip;
s/^.*#\([[:digit:]]\+\))$/\1/p;
s/^.*back[- ]*port[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip;
s/^.*cherry[- ]*pick[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip")
# awk is to filter out small task numbers from different task tracker, which are
# referenced by documentation commits like '* DOCSUP-824: query log (#115)'.
"${find_prs[@]}" master-log.txt | sort -rn | uniq | awk '$0 > 1000 { print $0 }' > master-prs.txt
"${find_prs[@]}" "$branch-log.txt" | sort -rn | uniq | awk '$0 > 1000 { print $0 }' > "$branch-prs.txt"
# Find all master PRs that are not in branch by calculating differences of two PR lists.
grep -f "$branch-prs.txt" -F -x -v master-prs.txt > "$branch-diff-prs.txt"
echo "$(wc -l < "$branch-diff-prs".txt) PRs differ between $branch and master."
function github_download()
{
local url=${1}
local file=${2}
if ! [ -f "$file" ]
then
if ! curl -H "Authorization: token $GITHUB_TOKEN" \
-sSf "$url" \
> "$file"
then
>&2 echo "Failed to download '$url' to '$file'. Contents: '$(cat "$file")'."
rm "$file"
return 1
fi
sleep 0.1
fi
}
rm "$branch-report.tsv" &> /dev/null ||:
for pr in $(cat "$branch-diff-prs.txt")
do
# Download PR info from github.
file="pr$pr.json"
github_download "https://api.github.com/repos/ClickHouse/ClickHouse/pulls/$pr" "$file" || continue
if ! [ "$pr" == "$(jq -r .number "$file")" ]
then
>&2 echo "Got wrong data for PR #$pr (please check and remove '$file')."
continue
fi
action="skip"
# First, check the changelog category. We port all bugfixes.
if jq -r .body "$file" | grep -i "^- bug[ -]*fix" > /dev/null
then
action="backport"
fi
# Next, check the tag. They might override the decision. Checks are ordered by priority.
labels="$(jq -r .labels[].name "$file")"
if echo "$labels" | grep -x "pr-must-backport\|v$branch-must-backport" > /dev/null; then action="backport"; fi
if echo "$labels" | grep -x "v$branch-conflicts" > /dev/null; then action="conflict"; fi
if echo "$labels" | grep -x "pr-no-backport\|v$branch-no-backport" > /dev/null; then action="no-backport"; fi
# FIXME Ignore "backported" labels for now. If we can't find the backport commit,
# this means that the changelog script also won't be able to. An alternative
# way to mark PR as backported is to add an empty commit with text like
# "backported #12345", so that it can be found between tags and put in proper
# place in changelog.
#if echo "$labels" | grep -x "v$branch\|v$branch-backported" > /dev/null; then action="done"; fi
# Find merge commit SHA for convenience
merge_sha="$(jq -r .merge_commit_sha "$file")"
url="https://github.com/ClickHouse/ClickHouse/pull/$pr"
printf "%s\t%s\t%s\t%s\t%s\n" "$action" "$pr" "$url" "$file" "$merge_sha" >> "$branch-report.tsv"
if [ "$action" == "backport" ]
then
printf "%s\t%s\t%s\n" "$action" "$url" "$merge_sha"
fi
done
echo "Done."