diff --git a/.gitmodules b/.gitmodules index 2ccce88e5e4..ab7c8a7c94d 100644 --- a/.gitmodules +++ b/.gitmodules @@ -228,7 +228,6 @@ [submodule "contrib/datasketches-cpp"] path = contrib/datasketches-cpp url = https://github.com/ClickHouse-Extras/datasketches-cpp.git - [submodule "contrib/yaml-cpp"] path = contrib/yaml-cpp url = https://github.com/ClickHouse-Extras/yaml-cpp.git diff --git a/CHANGELOG.md b/CHANGELOG.md index 2eaecaa4c9b..cad82fd1e28 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -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 diff --git a/CMakeLists.txt b/CMakeLists.txt index 9c62748ff95..ce0f58e2521 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -528,7 +528,6 @@ include (cmake/find/libpqxx.cmake) include (cmake/find/nuraft.cmake) include (cmake/find/yaml-cpp.cmake) - if(NOT USE_INTERNAL_PARQUET_LIBRARY) set (ENABLE_ORC OFF CACHE INTERNAL "") endif() diff --git a/base/loggers/OwnSplitChannel.cpp b/base/loggers/OwnSplitChannel.cpp index fc167ec3bf6..eb881e4a661 100644 --- a/base/loggers/OwnSplitChannel.cpp +++ b/base/loggers/OwnSplitChannel.cpp @@ -4,12 +4,14 @@ #include #include #include +#include #include #include #include #include #include #include +#include namespace DB { @@ -26,16 +28,48 @@ void OwnSplitChannel::log(const Poco::Message & msg) auto matches = masker->wipeSensitiveData(message_text); if (matches > 0) { - logSplit({msg, message_text}); // we will continue with the copy of original message with text modified + tryLogSplit({msg, message_text}); // we will continue with the copy of original message with text modified return; } } - logSplit(msg); + tryLogSplit(msg); } +void OwnSplitChannel::tryLogSplit(const Poco::Message & msg) +{ + try + { + logSplit(msg); + } + /// It is better to catch the errors here in order to avoid + /// breaking some functionality because of unexpected "File not + /// found" (or similar) error. + /// + /// For example StorageDistributedDirectoryMonitor will mark batch + /// as broken, some MergeTree code can also be affected. + /// + /// Also note, that we cannot log the exception here, since this + /// will lead to recursion, using regular tryLogCurrentException(). + /// but let's log it into the stderr at least. + catch (...) + { + MemoryTracker::LockExceptionInThread lock_memory_tracker(VariableContext::Global); + + const std::string & exception_message = getCurrentExceptionMessage(true); + const std::string & message = msg.getText(); + + /// NOTE: errors are ignored, since nothing can be done. + writeRetry(STDERR_FILENO, "Cannot add message to the log: "); + writeRetry(STDERR_FILENO, message.data(), message.size()); + writeRetry(STDERR_FILENO, "\n"); + writeRetry(STDERR_FILENO, exception_message.data(), exception_message.size()); + writeRetry(STDERR_FILENO, "\n"); + } +} + void OwnSplitChannel::logSplit(const Poco::Message & msg) { ExtendedLogMessage msg_ext = ExtendedLogMessage::getFrom(msg); diff --git a/base/loggers/OwnSplitChannel.h b/base/loggers/OwnSplitChannel.h index ac313b383bb..03ff7b57745 100644 --- a/base/loggers/OwnSplitChannel.h +++ b/base/loggers/OwnSplitChannel.h @@ -24,6 +24,7 @@ public: private: void logSplit(const Poco::Message & msg); + void tryLogSplit(const Poco::Message & msg); using ChannelPtr = Poco::AutoPtr; /// Handler and its pointer casted to extended interface diff --git a/cmake/find/yaml-cpp.cmake b/cmake/find/yaml-cpp.cmake index 9b9d9bd39d6..4633d559980 100644 --- a/cmake/find/yaml-cpp.cmake +++ b/cmake/find/yaml-cpp.cmake @@ -4,6 +4,6 @@ if (NOT USE_YAML_CPP) return() endif() -if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/yaml-cpp") +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/yaml-cpp/README.md") message (ERROR "submodule contrib/yaml-cpp is missing. to fix try run: \n git submodule update --init --recursive") endif() diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index c499da9d087..710c8c7fca5 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -61,7 +61,6 @@ endif() add_subdirectory (poco-cmake) add_subdirectory (croaring-cmake) - # TODO: refactor the contrib libraries below this comment. if (USE_INTERNAL_ZSTD_LIBRARY) diff --git a/contrib/NuRaft b/contrib/NuRaft index 95d6bbba579..2a1bf7d87b4 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 95d6bbba579b3a4e4c2dede954f541ff6f3dba51 +Subproject commit 2a1bf7d87b4a03561fc66fbb49cee8a288983c5d diff --git a/contrib/avro b/contrib/avro index 1ee16d8c5a7..e43c46e87fd 160000 --- a/contrib/avro +++ b/contrib/avro @@ -1 +1 @@ -Subproject commit 1ee16d8c5a7808acff5cf0475f771195d9aa3faa +Subproject commit e43c46e87fd32eafdc09471e95344555454c5ef8 diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 140b7eb370b..9531a5a4f9e 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ if (SANITIZE OR NOT ( ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_ARM OR ARCH_PPC64LE)) OR - (OS_DARWIN AND CMAKE_BUILD_TYPE STREQUAL "RelWithDebInfo") + (OS_DARWIN AND (CMAKE_BUILD_TYPE STREQUAL "RelWithDebInfo" OR CMAKE_BUILD_TYPE STREQUAL "Debug")) )) if (ENABLE_JEMALLOC) message (${RECONFIGURE_MESSAGE_LEVEL} diff --git a/debian/clickhouse-server.init b/debian/clickhouse-server.init index 3e4e888eacd..d7d87c6d53c 100755 --- a/debian/clickhouse-server.init +++ b/debian/clickhouse-server.init @@ -229,6 +229,7 @@ status() case "$1" in status) status + exit 0 ;; esac diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index 4486b0d9d7f..c93017bd0d3 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -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 diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index fc73a0df0ee..cc6aeff357f 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -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 \ diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index cff49a805fa..0665ab7560f 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -1,5 +1,5 @@ # docker build -t yandex/clickhouse-integration-tests-runner . -FROM ubuntu:18.04 +FROM ubuntu:20.04 RUN apt-get update \ && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ @@ -14,7 +14,6 @@ RUN apt-get update \ wget \ git \ iproute2 \ - module-init-tools \ cgroupfs-mount \ python3-pip \ tzdata \ @@ -42,7 +41,6 @@ ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ENV DOCKER_CHANNEL stable -ENV DOCKER_VERSION 5:19.03.13~3-0~ubuntu-bionic RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - RUN add-apt-repository "deb [arch=amd64] https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}" @@ -66,17 +64,18 @@ RUN python3 -m pip install \ dict2xml \ dicttoxml \ docker \ - docker-compose==1.22.0 \ + docker-compose==1.28.2 \ grpcio \ grpcio-tools \ kafka-python \ kazoo \ minio \ protobuf \ - psycopg2-binary==2.7.5 \ + psycopg2-binary==2.8.6 \ pymongo \ pytest \ pytest-timeout \ + pytest-xdist \ redis \ tzlocal \ urllib3 \ @@ -86,6 +85,7 @@ RUN python3 -m pip install \ COPY modprobe.sh /usr/local/bin/modprobe COPY dockerd-entrypoint.sh /usr/local/bin/ COPY compose/ /compose/ +COPY misc/ /misc/ RUN set -x \ && addgroup --system dockremap \ @@ -94,7 +94,6 @@ RUN set -x \ && echo 'dockremap:165536:65536' >> /etc/subuid \ && echo 'dockremap:165536:65536' >> /etc/subgid -VOLUME /var/lib/docker EXPOSE 2375 ENTRYPOINT ["dockerd-entrypoint.sh"] CMD ["sh", "-c", "pytest $PYTEST_OPTS"] diff --git a/docker/test/integration/runner/compose/docker_compose_cassandra.yml b/docker/test/integration/runner/compose/docker_compose_cassandra.yml index c5cdfac5ce7..b6190a11d73 100644 --- a/docker/test/integration/runner/compose/docker_compose_cassandra.yml +++ b/docker/test/integration/runner/compose/docker_compose_cassandra.yml @@ -1,7 +1,5 @@ version: '2.3' services: cassandra1: - image: cassandra + image: cassandra:4.0 restart: always - ports: - - 9043:9042 diff --git a/docker/test/integration/runner/compose/docker_compose_hdfs.yml b/docker/test/integration/runner/compose/docker_compose_hdfs.yml index 43dd1aa43d3..12bdc292b2f 100644 --- a/docker/test/integration/runner/compose/docker_compose_hdfs.yml +++ b/docker/test/integration/runner/compose/docker_compose_hdfs.yml @@ -5,6 +5,10 @@ services: hostname: hdfs1 restart: always ports: - - 50075:50075 - - 50070:50070 + - ${HDFS_NAME_EXTERNAL_PORT}:${HDFS_NAME_INTERNAL_PORT} #50070 + - ${HDFS_DATA_EXTERNAL_PORT}:${HDFS_DATA_INTERNAL_PORT} #50075 entrypoint: /etc/bootstrap.sh -d + volumes: + - type: ${HDFS_FS:-tmpfs} + source: ${HDFS_LOGS:-} + target: /usr/local/hadoop/logs \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index b77542f7e11..4552e6f0696 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -15,10 +15,11 @@ services: image: confluentinc/cp-kafka:5.2.0 hostname: kafka1 ports: - - "9092:9092" + - ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT} environment: - KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kafka1:19092 - KAFKA_LISTENERS: INSIDE://:9092,OUTSIDE://:19092 + KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA_EXTERNAL_PORT},OUTSIDE://kafka1:19092 + KAFKA_ADVERTISED_HOST_NAME: kafka1 + KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092 KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE KAFKA_BROKER_ID: 1 @@ -34,7 +35,7 @@ services: image: confluentinc/cp-schema-registry:5.2.0 hostname: schema-registry ports: - - "8081:8081" + - ${SCHEMA_REGISTRY_EXTERNAL_PORT}:${SCHEMA_REGISTRY_INTERNAL_PORT} environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml index e2e15975e22..b2e142c25a8 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml @@ -11,16 +11,18 @@ services: - ${KERBERIZED_HDFS_DIR}/../../hdfs_configs/bootstrap.sh:/etc/bootstrap.sh:ro - ${KERBERIZED_HDFS_DIR}/secrets:/usr/local/hadoop/etc/hadoop/conf - ${KERBERIZED_HDFS_DIR}/secrets/krb_long.conf:/etc/krb5.conf:ro + - type: ${KERBERIZED_HDFS_FS:-tmpfs} + source: ${KERBERIZED_HDFS_LOGS:-} + target: /var/log/hadoop-hdfs ports: - - 1006:1006 - - 50070:50070 - - 9010:9010 + - ${KERBERIZED_HDFS_NAME_EXTERNAL_PORT}:${KERBERIZED_HDFS_NAME_INTERNAL_PORT} #50070 + - ${KERBERIZED_HDFS_DATA_EXTERNAL_PORT}:${KERBERIZED_HDFS_DATA_INTERNAL_PORT} #1006 depends_on: - hdfskerberos entrypoint: /etc/bootstrap.sh -d hdfskerberos: - image: yandex/clickhouse-kerberos-kdc:${DOCKER_KERBEROS_KDC_TAG} + image: yandex/clickhouse-kerberos-kdc:${DOCKER_KERBEROS_KDC_TAG:-latest} hostname: hdfskerberos volumes: - ${KERBERIZED_HDFS_DIR}/secrets:/tmp/keytab diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml index 64a3ef3e956..081b90c4f27 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -23,13 +23,13 @@ services: # restart: always hostname: kerberized_kafka1 ports: - - "9092:9092" - - "9093:9093" + - ${KERBERIZED_KAFKA_EXTERNAL_PORT}:${KERBERIZED_KAFKA_EXTERNAL_PORT} environment: - KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://:9093 - KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:9093 + KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://0.0.0.0:${KERBERIZED_KAFKA_EXTERNAL_PORT} + KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:${KERBERIZED_KAFKA_EXTERNAL_PORT} # KAFKA_LISTENERS: INSIDE://kerberized_kafka1:9092,OUTSIDE://kerberized_kafka1:19092 # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kerberized_kafka1:19092 + KAFKA_ADVERTISED_HOST_NAME: kerberized_kafka1 KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL: GSSAPI KAFKA_SASL_ENABLED_MECHANISMS: GSSAPI KAFKA_SASL_KERBEROS_SERVICE_NAME: kafka diff --git a/docker/test/integration/runner/compose/docker_compose_minio.yml b/docker/test/integration/runner/compose/docker_compose_minio.yml index dbb29f9711a..96a5f8bdc31 100644 --- a/docker/test/integration/runner/compose/docker_compose_minio.yml +++ b/docker/test/integration/runner/compose/docker_compose_minio.yml @@ -6,8 +6,8 @@ services: volumes: - data1-1:/data1 - ${MINIO_CERTS_DIR:-}:/certs - ports: - - "9001:9001" + expose: + - ${MINIO_PORT} environment: MINIO_ACCESS_KEY: minio MINIO_SECRET_KEY: minio123 @@ -20,14 +20,14 @@ services: # HTTP proxies for Minio. proxy1: image: yandex/clickhouse-s3-proxy - ports: + expose: - "8080" # Redirect proxy port - "80" # Reverse proxy port - "443" # Reverse proxy port (secure) proxy2: image: yandex/clickhouse-s3-proxy - ports: + expose: - "8080" - "80" - "443" @@ -35,7 +35,7 @@ services: # Empty container to run proxy resolver. resolver: image: yandex/clickhouse-python-bottle - ports: + expose: - "8080" tty: true depends_on: diff --git a/docker/test/integration/runner/compose/docker_compose_mongo.yml b/docker/test/integration/runner/compose/docker_compose_mongo.yml index 6c98fde2303..e794966bd08 100644 --- a/docker/test/integration/runner/compose/docker_compose_mongo.yml +++ b/docker/test/integration/runner/compose/docker_compose_mongo.yml @@ -7,5 +7,5 @@ services: MONGO_INITDB_ROOT_USERNAME: root MONGO_INITDB_ROOT_PASSWORD: clickhouse ports: - - 27018:27017 + - ${MONGO_EXTERNAL_PORT}:${MONGO_INTERNAL_PORT} command: --profile=2 --verbose diff --git a/docker/test/integration/runner/compose/docker_compose_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql.yml index 5b15d517f37..c9b45af6563 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql.yml @@ -1,10 +1,24 @@ version: '2.3' services: - mysql1: + mysql57: image: mysql:5.7 restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - ports: - - 3308:3306 - command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency + MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} + DATADIR: /mysql/ + expose: + - ${MYSQL_PORT} + command: --server_id=100 + --log-bin='mysql-bin-1.log' + --default-time-zone='+3:00' + --gtid-mode="ON" + --enforce-gtid-consistency + --log-error-verbosity=3 + --log-error=/mysql/error.log + --general-log=ON + --general-log-file=/mysql/general.log + volumes: + - type: ${MYSQL_LOGS_FS:-tmpfs} + source: ${MYSQL_LOGS:-} + target: /mysql/ \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml index 5aa13ba91c7..ba693fd9fb4 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml @@ -12,3 +12,10 @@ services: --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 + --log-error=/var/log/mysqld/error.log + --general-log=ON + --general-log-file=/var/log/mysqld/general.log + volumes: + - type: ${MYSQL_LOGS_FS:-tmpfs} + source: ${MYSQL_LOGS:-} + target: /var/log/mysqld/ \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml new file mode 100644 index 00000000000..e13076c4e2e --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml @@ -0,0 +1,23 @@ +version: '2.3' +services: + mysql80: + image: mysql:8.0 + restart: always + environment: + MYSQL_ROOT_PASSWORD: clickhouse + MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} + DATADIR: /mysql/ + expose: + - ${MYSQL8_PORT} + command: --server_id=100 --log-bin='mysql-bin-1.log' + --default_authentication_plugin='mysql_native_password' + --default-time-zone='+3:00' --gtid-mode="ON" + --enforce-gtid-consistency + --log-error-verbosity=3 + --log-error=/mysql/error.log + --general-log=ON + --general-log-file=/mysql/general.log + volumes: + - type: ${MYSQL8_LOGS_FS:-tmpfs} + source: ${MYSQL8_LOGS:-} + target: /mysql/ \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml deleted file mode 100644 index 93bfee35caf..00000000000 --- a/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml +++ /dev/null @@ -1,15 +0,0 @@ -version: '2.3' -services: - mysql8_0: - image: mysql:8.0 - restart: 'no' - environment: - MYSQL_ROOT_PASSWORD: clickhouse - ports: - - 3309:3306 - command: --server_id=100 --log-bin='mysql-bin-1.log' - --default_authentication_plugin='mysql_native_password' - --default-time-zone='+3:00' - --gtid-mode="ON" - --enforce-gtid-consistency - --log-error-verbosity=3 diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_client.yml b/docker/test/integration/runner/compose/docker_compose_mysql_client.yml index 5e4565d64c3..5b37b6e6c09 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_client.yml @@ -1,6 +1,6 @@ version: '2.3' services: - mysql1: + mysql_client: image: mysql:5.7 restart: always environment: diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml b/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml index d0674362709..6ced7536812 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml @@ -5,19 +5,64 @@ services: restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - ports: - - 3348:3306 + MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST} + DATADIR: /mysql/ + expose: + - ${MYSQL_CLUSTER_PORT} + command: --server_id=100 + --log-bin='mysql-bin-2.log' + --default-time-zone='+3:00' + --gtid-mode="ON" + --enforce-gtid-consistency + --log-error-verbosity=3 + --log-error=/mysql/2_error.log + --general-log=ON + --general-log-file=/mysql/2_general.log + volumes: + - type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs} + source: ${MYSQL_CLUSTER_LOGS:-} + target: /mysql/ mysql3: image: mysql:5.7 restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - ports: - - 3388:3306 + MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST} + DATADIR: /mysql/ + expose: + - ${MYSQL_CLUSTER_PORT} + command: --server_id=100 + --log-bin='mysql-bin-3.log' + --default-time-zone='+3:00' + --gtid-mode="ON" + --enforce-gtid-consistency + --log-error-verbosity=3 + --log-error=/mysql/3_error.log + --general-log=ON + --general-log-file=/mysql/3_general.log + volumes: + - type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs} + source: ${MYSQL_CLUSTER_LOGS:-} + target: /mysql/ mysql4: image: mysql:5.7 restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - ports: - - 3368:3306 + MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST} + DATADIR: /mysql/ + expose: + - ${MYSQL_CLUSTER_PORT} + command: --server_id=100 + --log-bin='mysql-bin-4.log' + --default-time-zone='+3:00' + --gtid-mode="ON" + --enforce-gtid-consistency + --log-error-verbosity=3 + --log-error=/mysql/4_error.log + --general-log=ON + --general-log-file=/mysql/4_general.log + volumes: + - type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs} + source: ${MYSQL_CLUSTER_LOGS:-} + target: /mysql/ \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_postgres.yml b/docker/test/integration/runner/compose/docker_compose_postgres.yml index 5657352e1b3..c4a506ad356 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres.yml @@ -2,12 +2,24 @@ version: '2.3' services: postgres1: image: postgres + command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"] restart: always - environment: - POSTGRES_PASSWORD: mysecretpassword - ports: - - 5432:5432 + expose: + - ${POSTGRES_PORT} + healthcheck: + test: ["CMD-SHELL", "pg_isready -U postgres"] + interval: 10s + timeout: 5s + retries: 5 networks: - default: - aliases: - - postgre-sql.local + default: + aliases: + - postgre-sql.local + environment: + POSTGRES_HOST_AUTH_METHOD: "trust" + POSTGRES_PASSWORD: mysecretpassword + PGDATA: /postgres/data + volumes: + - type: ${POSTGRES_LOGS_FS:-tmpfs} + source: ${POSTGRES_DIR:-} + target: /postgres/ \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml b/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml index d04c8a2f3a6..94b941b74da 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml @@ -2,22 +2,43 @@ version: '2.3' services: postgres2: image: postgres + command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"] restart: always environment: + POSTGRES_HOST_AUTH_METHOD: "trust" POSTGRES_PASSWORD: mysecretpassword - ports: - - 5421:5432 + PGDATA: /postgres/data + expose: + - ${POSTGRES_PORT} + volumes: + - type: ${POSTGRES_LOGS_FS:-tmpfs} + source: ${POSTGRES2_DIR:-} + target: /postgres/ postgres3: image: postgres + command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"] restart: always environment: + POSTGRES_HOST_AUTH_METHOD: "trust" POSTGRES_PASSWORD: mysecretpassword - ports: - - 5441:5432 + PGDATA: /postgres/data + expose: + - ${POSTGRES_PORT} + volumes: + - type: ${POSTGRES_LOGS_FS:-tmpfs} + source: ${POSTGRES3_DIR:-} + target: /postgres/ postgres4: image: postgres + command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"] restart: always environment: + POSTGRES_HOST_AUTH_METHOD: "trust" POSTGRES_PASSWORD: mysecretpassword - ports: - - 5461:5432 + PGDATA: /postgres/data + expose: + - ${POSTGRES_PORT} + volumes: + - type: ${POSTGRES_LOGS_FS:-tmpfs} + source: ${POSTGRES4_DIR:-} + target: /postgres/ \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_postgesql.yml b/docker/test/integration/runner/compose/docker_compose_postgresql.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_postgesql.yml rename to docker/test/integration/runner/compose/docker_compose_postgresql.yml diff --git a/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml b/docker/test/integration/runner/compose/docker_compose_postgresql_java_client.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml rename to docker/test/integration/runner/compose/docker_compose_postgresql_java_client.yml diff --git a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml index 1e9c3777505..99e0ea8e280 100644 --- a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml +++ b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml @@ -2,11 +2,15 @@ version: '2.3' services: rabbitmq1: - image: rabbitmq:3-management + image: rabbitmq:3-management-alpine hostname: rabbitmq1 - ports: - - "5672:5672" - - "15672:15672" + expose: + - ${RABBITMQ_PORT} environment: RABBITMQ_DEFAULT_USER: "root" RABBITMQ_DEFAULT_PASS: "clickhouse" + RABBITMQ_LOG_BASE: /rabbitmq_logs/ + volumes: + - type: ${RABBITMQ_LOGS_FS:-tmpfs} + source: ${RABBITMQ_LOGS:-} + target: /rabbitmq_logs/ diff --git a/docker/test/integration/runner/compose/docker_compose_redis.yml b/docker/test/integration/runner/compose/docker_compose_redis.yml index 3d834aadaa4..1bf67a6c44e 100644 --- a/docker/test/integration/runner/compose/docker_compose_redis.yml +++ b/docker/test/integration/runner/compose/docker_compose_redis.yml @@ -4,5 +4,5 @@ services: image: redis restart: always ports: - - 6380:6379 + - ${REDIS_EXTERNAL_PORT}:${REDIS_INTERNAL_PORT} command: redis-server --requirepass "clickhouse" --databases 32 diff --git a/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml b/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml new file mode 100644 index 00000000000..7a1c32e0023 --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml @@ -0,0 +1,75 @@ +version: '2.3' +services: + zoo1: + image: zookeeper:3.6.2 + restart: always + environment: + ZOO_TICK_TIME: 500 + ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181 + ZOO_MY_ID: 1 + JVMFLAGS: -Dzookeeper.forceSync=no + ZOO_SECURE_CLIENT_PORT: $ZOO_SECURE_CLIENT_PORT + command: ["zkServer.sh", "start-foreground"] + entrypoint: /zookeeper-ssl-entrypoint.sh + volumes: + - type: bind + source: /misc/zookeeper-ssl-entrypoint.sh + target: /zookeeper-ssl-entrypoint.sh + - type: bind + source: /misc/client.crt + target: /clickhouse-config/client.crt + - type: ${ZK_FS:-tmpfs} + source: ${ZK_DATA1:-} + target: /data + - type: ${ZK_FS:-tmpfs} + source: ${ZK_DATA_LOG1:-} + target: /datalog + zoo2: + image: zookeeper:3.6.2 + restart: always + environment: + ZOO_TICK_TIME: 500 + ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888 + ZOO_MY_ID: 2 + JVMFLAGS: -Dzookeeper.forceSync=no + ZOO_SECURE_CLIENT_PORT: $ZOO_SECURE_CLIENT_PORT + + command: ["zkServer.sh", "start-foreground"] + entrypoint: /zookeeper-ssl-entrypoint.sh + volumes: + - type: bind + source: /misc/zookeeper-ssl-entrypoint.sh + target: /zookeeper-ssl-entrypoint.sh + - type: bind + source: /misc/client.crt + target: /clickhouse-config/client.crt + - type: ${ZK_FS:-tmpfs} + source: ${ZK_DATA2:-} + target: /data + - type: ${ZK_FS:-tmpfs} + source: ${ZK_DATA_LOG2:-} + target: /datalog + zoo3: + image: zookeeper:3.6.2 + restart: always + environment: + ZOO_TICK_TIME: 500 + ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181 + ZOO_MY_ID: 3 + JVMFLAGS: -Dzookeeper.forceSync=no + ZOO_SECURE_CLIENT_PORT: $ZOO_SECURE_CLIENT_PORT + command: ["zkServer.sh", "start-foreground"] + entrypoint: /zookeeper-ssl-entrypoint.sh + volumes: + - type: bind + source: /misc/zookeeper-ssl-entrypoint.sh + target: /zookeeper-ssl-entrypoint.sh + - type: bind + source: /misc/client.crt + target: /clickhouse-config/client.crt + - type: ${ZK_FS:-tmpfs} + source: ${ZK_DATA3:-} + target: /data + - type: ${ZK_FS:-tmpfs} + source: ${ZK_DATA_LOG3:-} + target: /datalog diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 4d25ca60c5e..626b6883b91 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -2,17 +2,17 @@ set -e mkdir -p /etc/docker/ -cat > /etc/docker/daemon.json << EOF -{ +echo '{ "ipv6": true, "fixed-cidr-v6": "fd00::/8", "ip-forward": true, + "log-level": "debug", + "storage-driver": "overlay2", "insecure-registries" : ["dockerhub-proxy.sas.yp-c.yandex.net:5000"], "registry-mirrors" : ["http://dockerhub-proxy.sas.yp-c.yandex.net:5000"] -} -EOF +}' | dd of=/etc/docker/daemon.json -dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 &>/var/log/somefile & +dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log & set +e reties=0 diff --git a/docker/test/integration/runner/misc/client.crt b/docker/test/integration/runner/misc/client.crt new file mode 100644 index 00000000000..7ade2d96273 --- /dev/null +++ b/docker/test/integration/runner/misc/client.crt @@ -0,0 +1,19 @@ +-----BEGIN CERTIFICATE----- +MIIC/TCCAeWgAwIBAgIJANjx1QSR77HBMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV +BAMMCWxvY2FsaG9zdDAgFw0xODA3MzAxODE2MDhaGA8yMjkyMDUxNDE4MTYwOFow +FDESMBAGA1UEAwwJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB +CgKCAQEAs9uSo6lJG8o8pw0fbVGVu0tPOljSWcVSXH9uiJBwlZLQnhN4SFSFohfI +4K8U1tBDTnxPLUo/V1K9yzoLiRDGMkwVj6+4+hE2udS2ePTQv5oaMeJ9wrs+5c9T +4pOtlq3pLAdm04ZMB1nbrEysceVudHRkQbGHzHp6VG29Fw7Ga6YpqyHQihRmEkTU +7UCYNA+Vk7aDPdMS/khweyTpXYZimaK9f0ECU3/VOeG3fH6Sp2X6FN4tUj/aFXEj +sRmU5G2TlYiSIUMF2JPdhSihfk1hJVALrHPTU38SOL+GyyBRWdNcrIwVwbpvsvPg +pryMSNxnpr0AK0dFhjwnupIv5hJIOQIDAQABo1AwTjAdBgNVHQ4EFgQUjPLb3uYC +kcamyZHK4/EV8jAP0wQwHwYDVR0jBBgwFoAUjPLb3uYCkcamyZHK4/EV8jAP0wQw +DAYDVR0TBAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAM/ocuDvfPus/KpMVD51j +4IdlU8R0vmnYLQ+ygzOAo7+hUWP5j0yvq4ILWNmQX6HNvUggCgFv9bjwDFhb/5Vr +85ieWfTd9+LTjrOzTw4avdGwpX9G+6jJJSSq15tw5ElOIFb/qNA9O4dBiu8vn03C +L/zRSXrARhSqTW5w/tZkUcSTT+M5h28+Lgn9ysx4Ff5vi44LJ1NnrbJbEAIYsAAD ++UA+4MBFKx1r6hHINULev8+lCfkpwIaeS8RL+op4fr6kQPxnULw8wT8gkuc8I4+L +P9gg/xDHB44T3ADGZ5Ib6O0DJaNiToO6rnoaaxs0KkotbvDWvRoxEytSbXKoYjYp +0g== +-----END CERTIFICATE----- diff --git a/tests/integration/helpers/zookeeper-ssl-entrypoint.sh b/docker/test/integration/runner/misc/zookeeper-ssl-entrypoint.sh similarity index 97% rename from tests/integration/helpers/zookeeper-ssl-entrypoint.sh rename to docker/test/integration/runner/misc/zookeeper-ssl-entrypoint.sh index 3ddb21881d6..9748a5e8ce9 100755 --- a/tests/integration/helpers/zookeeper-ssl-entrypoint.sh +++ b/docker/test/integration/runner/misc/zookeeper-ssl-entrypoint.sh @@ -81,8 +81,8 @@ if [[ ! -f "$ZOO_DATA_DIR/myid" ]]; then echo "${ZOO_MY_ID:-1}" > "$ZOO_DATA_DIR/myid" fi -mkdir -p $(dirname $ZOO_SSL_KEYSTORE_LOCATION) -mkdir -p $(dirname $ZOO_SSL_TRUSTSTORE_LOCATION) +mkdir -p "$(dirname $ZOO_SSL_KEYSTORE_LOCATION)" +mkdir -p "$(dirname $ZOO_SSL_TRUSTSTORE_LOCATION)" if [[ ! -f "$ZOO_SSL_KEYSTORE_LOCATION" ]]; then keytool -genkeypair -alias zookeeper -keyalg RSA -validity 365 -keysize 2048 -dname "cn=zookeeper" -keypass password -keystore $ZOO_SSL_KEYSTORE_LOCATION -storepass password -deststoretype pkcs12 diff --git a/docker/test/sqlancer/Dockerfile b/docker/test/sqlancer/Dockerfile index b08b1b0bf0d..253ca1b729a 100644 --- a/docker/test/sqlancer/Dockerfile +++ b/docker/test/sqlancer/Dockerfile @@ -3,7 +3,6 @@ FROM ubuntu:20.04 RUN apt-get update --yes && env DEBIAN_FRONTEND=noninteractive apt-get install wget unzip git openjdk-14-jdk maven python3 --yes --no-install-recommends RUN wget https://github.com/sqlancer/sqlancer/archive/master.zip -O /sqlancer.zip - RUN mkdir /sqlancer && \ cd /sqlancer && \ unzip /sqlancer.zip diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 43a92fdeebe..8016b2c59f3 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -90,7 +90,7 @@ clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" clickhouse-client --query "SHOW TABLES FROM test" -./stress --hung-check --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" \ +./stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" \ && echo -e 'Test script exit code\tOK' >> /test_output/test_results.tsv \ || echo -e 'Test script failed\tFAIL' >> /test_output/test_results.tsv diff --git a/docker/test/stress/stress b/docker/test/stress/stress index e59c87a8b68..59c8dd07743 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -19,25 +19,25 @@ def get_skip_list_cmd(path): def get_options(i): - options = "" + options = [] if 0 < i: - options += " --order=random" + options.append("--order=random") if i % 3 == 1: - options += " --db-engine=Ordinary" + options.append("--db-engine=Ordinary") if i % 3 == 2: - options += ''' --client-option='allow_experimental_database_replicated=1' --db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i) + options.append('''--client-option='allow_experimental_database_replicated=1' --db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i)) # If database name is not specified, new database is created for each functional test. # Run some threads with one database for all tests. if i % 2 == 1: - options += " --database=test_{}".format(i) + options.append(" --database=test_{}".format(i)) if i == 13: - options += " --client-option='memory_tracker_fault_probability=0.00001'" + options.append(" --client-option='memory_tracker_fault_probability=0.00001'") - return options + return ' '.join(options) def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_time_limit): @@ -62,7 +62,7 @@ def compress_stress_logs(output_path, files_prefix): cmd = f"cd {output_path} && tar -zcf stress_run_logs.tar.gz {files_prefix}* && rm {files_prefix}*" check_output(cmd, shell=True) -def prepare_for_hung_check(): +def prepare_for_hung_check(drop_databases): # FIXME this function should not exist, but... # We attach gdb to clickhouse-server before running tests @@ -95,6 +95,17 @@ def prepare_for_hung_check(): # Long query from 00084_external_agregation call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'" """, shell=True, stderr=STDOUT) + if drop_databases: + # Here we try to drop all databases in async mode. If some queries really hung, than drop will hung too. + # Otherwise we will get rid of queries which wait for background pool. It can take a long time on slow builds (more than 900 seconds). + databases = check_output('clickhouse client -q "SHOW DATABASES"', shell=True).decode('utf-8').strip().split() + for db in databases: + if db == "system": + continue + command = f'clickhouse client -q "DROP DATABASE {db}"' + # we don't wait for drop + Popen(command, shell=True) + # Wait for last queries to finish if any, not longer than 300 seconds call("""clickhouse client -q "select sleepEachRow(( select maxOrDefault(300 - elapsed) + 1 from system.processes where query not like '%from system.processes%' and elapsed < 300 @@ -122,8 +133,12 @@ if __name__ == "__main__": parser.add_argument("--global-time-limit", type=int, default=3600) parser.add_argument("--num-parallel", type=int, default=cpu_count()) parser.add_argument('--hung-check', action='store_true', default=False) + # make sense only for hung check + parser.add_argument('--drop-databases', action='store_true', default=False) args = parser.parse_args() + if args.drop_databases and not args.hung_check: + raise Exception("--drop-databases only used in hung check (--hung-check)") func_pipes = [] func_pipes = run_func_test(args.test_cmd, args.output_folder, args.num_parallel, args.skip_func_tests, args.global_time_limit) @@ -145,7 +160,7 @@ if __name__ == "__main__": logging.info("Logs compressed") if args.hung_check: - have_long_running_queries = prepare_for_hung_check() + have_long_running_queries = prepare_for_hung_check(args.drop_databases) logging.info("Checking if some queries hung") cmd = "{} {} {}".format(args.test_cmd, "--hung-check", "00001_select_1") res = call(cmd, shell=True, stderr=STDOUT) diff --git a/docs/en/engines/table-engines/integrations/mysql.md b/docs/en/engines/table-engines/integrations/mysql.md index 9bd12e97dd8..013add6c249 100644 --- a/docs/en/engines/table-engines/integrations/mysql.md +++ b/docs/en/engines/table-engines/integrations/mysql.md @@ -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} diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index a27308b9b3f..e763d940752 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -10,7 +10,7 @@ This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ec ## Create Table {#creating-a-table} ``` sql -CREATE TABLE s3_engine_table (name String, value UInt32) +CREATE TABLE s3_engine_table (name String, value UInt32) ENGINE = S3(path, [aws_access_key_id, aws_secret_access_key,] format, [compression]) ``` @@ -130,6 +130,7 @@ The following settings can be set before query execution or placed into configur - `s3_max_single_part_upload_size` — The maximum size of object to upload using singlepart upload to S3. Default value is `64Mb`. - `s3_min_upload_part_size` — The minimum size of part to upload during multipart upload to [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Default value is `512Mb`. - `s3_max_redirects` — Max number of S3 redirects hops allowed. Default value is `10`. +- `s3_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration. @@ -144,6 +145,7 @@ The following settings can be specified in configuration file for given endpoint - `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Optional, default value is `false`. - `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be speficied multiple times. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional. +- `max_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Optional. **Example:** @@ -158,13 +160,14 @@ The following settings can be specified in configuration file for given endpoint + ``` ## Usage {#usage-examples} -Suppose we have several files in TSV format with the following URIs on HDFS: +Suppose we have several files in CSV format with the following URIs on S3: - 'https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_1.csv' - 'https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_2.csv' @@ -177,21 +180,21 @@ Suppose we have several files in TSV format with the following URIs on HDFS: 1. There are several ways to make a table consisting of all six files: ``` sql -CREATE TABLE table_with_range (name String, value UInt32) +CREATE TABLE table_with_range (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/some_file_{1..3}', 'CSV'); ``` 2. Another way: ``` sql -CREATE TABLE table_with_question_mark (name String, value UInt32) +CREATE TABLE table_with_question_mark (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/some_file_?', 'CSV'); ``` 3. Table consists of all the files in both directories (all files should satisfy format and schema described in query): ``` sql -CREATE TABLE table_with_asterisk (name String, value UInt32) +CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/*', 'CSV'); ``` @@ -201,7 +204,7 @@ ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_p 4. Create table with files named `file-000.csv`, `file-001.csv`, … , `file-999.csv`: ``` sql -CREATE TABLE big_table (name String, value UInt32) +CREATE TABLE big_table (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/big_prefix/file-{000..999}.csv', 'CSV'); ``` diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index e385b234cd8..eb87dbc6580 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -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: 0.2 - +
@@ -748,6 +748,7 @@ Configuration markup: 10000 5000 10 + 4 1000 /var/lib/clickhouse/disks/s3/ true @@ -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//`. -- `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//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//`. +- `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//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: ``` -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/) diff --git a/docs/en/engines/table-engines/special/dictionary.md b/docs/en/engines/table-engines/special/dictionary.md index a6d6f296673..d76adebe01e 100644 --- a/docs/en/engines/table-engines/special/dictionary.md +++ b/docs/en/engines/table-engines/special/dictionary.md @@ -94,4 +94,6 @@ select * from products limit 1; └───────────────┴─────────────────┘ ``` -[Original article](https://clickhouse.tech/docs/en/operations/table_engines/dictionary/) +**See Also** + +- [Dictionary function](../../../sql-reference/table-functions/dictionary.md#dictionary-function) diff --git a/docs/en/guides/apply-catboost-model.md b/docs/en/guides/apply-catboost-model.md index ec3ecc92141..b354a42a843 100644 --- a/docs/en/guides/apply-catboost-model.md +++ b/docs/en/guides/apply-catboost-model.md @@ -18,6 +18,8 @@ To apply a CatBoost model in ClickHouse: For more information about training CatBoost models, see [Training and applying models](https://catboost.ai/docs/features/training.html#training). +You can reload CatBoost models if the configuration was updated without restarting the server using [RELOAD MODEL](../sql-reference/statements/system.md#query_language-system-reload-model) and [RELOAD MODELS](../sql-reference/statements/system.md#query_language-system-reload-models) system queries. + ## Prerequisites {#prerequisites} If you do not have the [Docker](https://docs.docker.com/install/) yet, install it. diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index c616d843173..4dc0fa1cff0 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -547,7 +547,7 @@ Differs from JSON only in that data rows are output in arrays, not in objects. Example: -``` json +``` // JSONCompact { "meta": @@ -579,7 +579,7 @@ Example: } ``` -```json +``` // JSONCompactString { "meta": diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 10ea46098d4..fc5c887c92e 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -123,6 +123,19 @@ The `Insert` command creates one or more blocks (parts). When inserting into Rep A large number of `replicated_deduplication_window` slows down `Inserts` because it needs to compare more entries. The hash sum is calculated from the composition of the field names and types and the data of the inserted part (stream of bytes). +## non_replicated_deduplication_window {#non-replicated-deduplication-window} + +The number of the most recently inserted blocks in the non-replicated [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table for which hash sums are stored to check for duplicates. + +Possible values: + +- Any positive integer. +- 0 (disable deduplication). + +Default value: 0. + +A deduplication mechanism is used, similar to replicated tables (see [replicated_deduplication_window](#replicated-deduplication-window) setting). The hash sums of the created parts are written to a local file on a disk. + ## replicated_deduplication_window_seconds {#replicated-deduplication-window-seconds} The number of seconds after which the hash sums of the inserted blocks are removed from Zookeeper. diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 8e97171d31b..e82cb4882a0 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -506,3 +506,256 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= ## sumMapFiltered(keys_to_keep)(keys, values) {#summapfilteredkeys-to-keepkeys-values} Same behavior as [sumMap](../../sql-reference/aggregate-functions/reference/summap.md#agg_functions-summap) except that an array of keys is passed as a parameter. This can be especially useful when working with a high cardinality of keys. + +## sequenceNextNode {#sequenceNextNode} + +Returns a value of next event that matched an event chain. + +_Experimental function, `SET allow_experimental_funnel_functions = 1` to enable it._ + +**Syntax** + +``` sql +sequenceNextNode(direction, base)(timestamp, event_column, base_condition, event1, event2, event3, ...) +``` + +**Parameters** +- `direction` - Used to navigate to directions. + - forward : Moving forward + - backward: Moving backward + +- `base` - Used to set the base point. + - head : Set the base point to the first event + - tail : Set the base point to the last event + - first_match : Set the base point to the first matched event1 + - last_match : Set the base point to the last matched event1 + +**Arguments** +- `timestamp` — Name of the column containing the timestamp. Data types supported: `Date`, `DateTime` and other unsigned integer types. +- `event_column` — Name of the column containing the value of the next event to be returned. Data types supported: `String` and `Nullable(String)` +- `base_condition` — Condition that the base point must fulfill. +- `cond` — Conditions describing the chain of events. `UInt8` + +**Returned value** +- `event_column[next_index]` - if the pattern is matched and next value exists. +- `NULL` - if the pattern isn’t matched or next value doesn't exist. + +Type: `Nullable(String)`. + +**Example** + +It can be used when events are A->B->C->E->F and you want to know the event following B->C, which is E. + +The query statement searching the event following A->B : + +``` sql +CREATE TABLE test_flow ( + dt DateTime, + id int, + page String) +ENGINE = MergeTree() +PARTITION BY toYYYYMMDD(dt) +ORDER BY id; + +INSERT INTO test_flow VALUES (1, 1, 'A') (2, 1, 'B') (3, 1, 'C') (4, 1, 'E') (5, 1, 'F'); + +SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'A', page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id; +``` + +Result: + +``` text +┌─id─┬─next_flow─┐ +│ 1 │ C │ +└────┴───────────┘ +``` + +**Behavior for `forward` and `head`** + +```SQL +ALTER TABLE test_flow DELETE WHERE 1 = 1 settings mutations_sync = 1; + +INSERT INTO test_flow VALUES (1, 1, 'Home') (2, 1, 'Gift') (3, 1, 'Exit'); +INSERT INTO test_flow VALUES (1, 2, 'Home') (2, 2, 'Home') (3, 2, 'Gift') (4, 2, 'Basket'); +INSERT INTO test_flow VALUES (1, 3, 'Gift') (2, 3, 'Home') (3, 3, 'Gift') (4, 3, 'Basket'); +``` + +```SQL +SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'Home', page = 'Home', page = 'Gift') FROM test_flow GROUP BY id; + + dt id page + 1970-01-01 09:00:01 1 Home // Base point, Matched with Home + 1970-01-01 09:00:02 1 Gift // Matched with Gift + 1970-01-01 09:00:03 1 Exit // The result + + 1970-01-01 09:00:01 2 Home // Base point, Matched with Home + 1970-01-01 09:00:02 2 Home // Unmatched with Gift + 1970-01-01 09:00:03 2 Gift + 1970-01-01 09:00:04 2 Basket + + 1970-01-01 09:00:01 3 Gift // Base point, Unmatched with Home + 1970-01-01 09:00:02 3 Home + 1970-01-01 09:00:03 3 Gift + 1970-01-01 09:00:04 3 Basket +``` + +**Behavior for `backward` and `tail`** + +```SQL +SELECT id, sequenceNextNode('backward', 'tail')(dt, page, page = 'Basket', page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home +1970-01-01 09:00:02 1 Gift +1970-01-01 09:00:03 1 Exit // Base point, Unmatched with Basket + +1970-01-01 09:00:01 2 Home +1970-01-01 09:00:02 2 Home // The result +1970-01-01 09:00:03 2 Gift // Matched with Gift +1970-01-01 09:00:04 2 Basket // Base point, Matched with Basket + +1970-01-01 09:00:01 3 Gift +1970-01-01 09:00:02 3 Home // The result +1970-01-01 09:00:03 3 Gift // Base point, Matched with Gift +1970-01-01 09:00:04 3 Basket // Base point, Matched with Basket +``` + + +**Behavior for `forward` and `first_match`** + +```SQL +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Gift') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home +1970-01-01 09:00:02 1 Gift // Base point +1970-01-01 09:00:03 1 Exit // The result + +1970-01-01 09:00:01 2 Home +1970-01-01 09:00:02 2 Home +1970-01-01 09:00:03 2 Gift // Base point +1970-01-01 09:00:04 2 Basket The result + +1970-01-01 09:00:01 3 Gift // Base point +1970-01-01 09:00:02 3 Home // Thre result +1970-01-01 09:00:03 3 Gift +1970-01-01 09:00:04 3 Basket +``` + +```SQL +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home +1970-01-01 09:00:02 1 Gift // Base point +1970-01-01 09:00:03 1 Exit // Unmatched with Home + +1970-01-01 09:00:01 2 Home +1970-01-01 09:00:02 2 Home +1970-01-01 09:00:03 2 Gift // Base point +1970-01-01 09:00:04 2 Basket // Unmatched with Home + +1970-01-01 09:00:01 3 Gift // Base point +1970-01-01 09:00:02 3 Home // Matched with Home +1970-01-01 09:00:03 3 Gift // The result +1970-01-01 09:00:04 3 Basket +``` + + +**Behavior for `backward` and `last_match`** + +```SQL +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Gift') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home // The result +1970-01-01 09:00:02 1 Gift // Base point +1970-01-01 09:00:03 1 Exit + +1970-01-01 09:00:01 2 Home +1970-01-01 09:00:02 2 Home // The result +1970-01-01 09:00:03 2 Gift // Base point +1970-01-01 09:00:04 2 Basket + +1970-01-01 09:00:01 3 Gift +1970-01-01 09:00:02 3 Home // The result +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket +``` + +```SQL +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home // Matched with Home, the result is null +1970-01-01 09:00:02 1 Gift // Base point +1970-01-01 09:00:03 1 Exit + +1970-01-01 09:00:01 2 Home // The result +1970-01-01 09:00:02 2 Home // Matched with Home +1970-01-01 09:00:03 2 Gift // Base point +1970-01-01 09:00:04 2 Basket + +1970-01-01 09:00:01 3 Gift // The result +1970-01-01 09:00:02 3 Home // Matched with Home +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket +``` + + +**Behavior for `base_condition`** + +```SQL +CREATE TABLE test_flow_basecond +( + `dt` DateTime, + `id` int, + `page` String, + `ref` String +) +ENGINE = MergeTree +PARTITION BY toYYYYMMDD(dt) +ORDER BY id + +INSERT INTO test_flow_basecond VALUES (1, 1, 'A', 'ref4') (2, 1, 'A', 'ref3') (3, 1, 'B', 'ref2') (4, 1, 'B', 'ref1'); +``` + +```SQL +SELECT id, sequenceNextNode('forward', 'head')(dt, page, ref = 'ref1', page = 'A') FROM test_flow_basecond GROUP BY id; + + dt id page ref + 1970-01-01 09:00:01 1 A ref4 // The head can't be base point becasue the ref column of the head unmatched with 'ref1'. + 1970-01-01 09:00:02 1 A ref3 + 1970-01-01 09:00:03 1 B ref2 + 1970-01-01 09:00:04 1 B ref1 + ``` + +```SQL +SELECT id, sequenceNextNode('backward', 'tail')(dt, page, ref = 'ref4', page = 'B') FROM test_flow_basecond GROUP BY id; + + dt id page ref + 1970-01-01 09:00:01 1 A ref4 + 1970-01-01 09:00:02 1 A ref3 + 1970-01-01 09:00:03 1 B ref2 + 1970-01-01 09:00:04 1 B ref1 // The tail can't be base point becasue the ref column of the tail unmatched with 'ref4'. +``` + +```SQL +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, ref = 'ref3', page = 'A') FROM test_flow_basecond GROUP BY id; + + dt id page ref + 1970-01-01 09:00:01 1 A ref4 // This row can't be base point becasue the ref column unmatched with 'ref3'. + 1970-01-01 09:00:02 1 A ref3 // Base point + 1970-01-01 09:00:03 1 B ref2 // The result + 1970-01-01 09:00:04 1 B ref1 +``` + +```SQL +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, ref = 'ref2', page = 'B') FROM test_flow_basecond GROUP BY id; + + dt id page ref + 1970-01-01 09:00:01 1 A ref4 + 1970-01-01 09:00:02 1 A ref3 // The result + 1970-01-01 09:00:03 1 B ref2 // Base point + 1970-01-01 09:00:04 1 B ref1 // This row can't be base point becasue the ref column unmatched with 'ref2'. +``` diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 5a7efa37fd1..41da56362e7 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -98,6 +98,10 @@ Setting fields: When dictionary with source `FILE` is created via DDL command (`CREATE DICTIONARY ...`), the source file needs to be located in `user_files` directory, to prevent DB users accessing arbitrary file on ClickHouse node. +**See Also** + +- [Dictionary function](../../../sql-reference/table-functions/dictionary.md#dictionary-function) + ## Executable File {#dicts-external_dicts_dict_sources-executable} Working with executable files depends on [how the dictionary is stored in memory](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts executable file and treats its output as dictionary data. diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index afbaed2b413..34f8ff470fe 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -265,7 +265,7 @@ Result: ``` !!! attention "Attention" - The return type `toStartOf*` functions described below is `Date` or `DateTime`. Though these functions can take `DateTime64` as an argument, passing them a `DateTime64` that is out of normal range (years 1970 - 2105) will give incorrect result. + The return type `toStartOf*` functions described below is `Date` or `DateTime`. Though these functions can take `DateTime64` as an argument, passing them a `DateTime64` that is out of the normal range (years 1925 - 2283) will give an incorrect result. ## toStartOfYear {#tostartofyear} diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index b38643d6027..86381d3c6a4 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -16,6 +16,7 @@ The following operations with [partitions](../../../engines/table-engines/merget - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) — Resets the value of a specified column in a partition. - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) — Resets the specified secondary index in a partition. - [FREEZE PARTITION](#alter_freeze-partition) — Creates a backup of a partition. +- [UNFREEZE PARTITION](#alter_unfreeze-partition) — Removes a backup of a partition. - [FETCH PARTITION\|PART](#alter_fetch-partition) — Downloads a part or partition from another server. - [MOVE PARTITION\|PART](#alter_move-partition) — Move partition/data part to another disk or volume. @@ -160,7 +161,7 @@ This query creates a local backup of a specified partition. If the `PARTITION` c !!! note "Note" The entire backup process is performed without stopping the server. -Note that for old-styled tables you can specify the prefix of the partition name (for example, ‘2019’) - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). +Note that for old-styled tables you can specify the prefix of the partition name (for example, `2019`) - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). At the time of execution, for a data snapshot, the query creates hardlinks to a table data. Hardlinks are placed in the directory `/var/lib/clickhouse/shadow/N/...`, where: @@ -170,7 +171,7 @@ At the time of execution, for a data snapshot, the query creates hardlinks to a !!! note "Note" If you use [a set of disks for data storage in a table](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes), the `shadow/N` directory appears on every disk, storing data parts that matched by the `PARTITION` expression. -The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs ‘chmod’ for all files, forbidding writing into them. +The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs `chmod` for all files, forbidding writing into them. After creating the backup, you can copy the data from `/var/lib/clickhouse/shadow/` to the remote server and then delete it from the local server. Note that the `ALTER t FREEZE PARTITION` query is not replicated. It creates a local backup only on the local server. @@ -188,6 +189,14 @@ Restoring from a backup does not require stopping the server. For more information about backups and restoring data, see the [Data Backup](../../../operations/backup.md) section. +## UNFREEZE PARTITION {#alter_unfreeze-partition} + +``` sql +ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name' +``` + +Removes `freezed` partitions with the specified name from the disk. If the `PARTITION` clause is omitted, the query removes the backup of all partitions at once. + ## CLEAR INDEX IN PARTITION {#alter_clear-index-partition} ``` sql diff --git a/docs/en/sql-reference/statements/rename.md b/docs/en/sql-reference/statements/rename.md index 4f454626493..3591c187e93 100644 --- a/docs/en/sql-reference/statements/rename.md +++ b/docs/en/sql-reference/statements/rename.md @@ -6,7 +6,7 @@ toc_title: RENAME # RENAME Statement {#misc_operations-rename} ## RENAME DATABASE {#misc_operations-rename_database} -Renames database, support only for Atomic database engine +Renames database, it is supported only for Atomic database engine. ``` RENAME DATABASE atomic_database1 TO atomic_database2 [ON CLUSTER cluster] diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 9397d7002fd..a17c87d2326 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -10,6 +10,8 @@ The list of available `SYSTEM` statements: - [RELOAD EMBEDDED DICTIONARIES](#query_language-system-reload-emdedded-dictionaries) - [RELOAD DICTIONARIES](#query_language-system-reload-dictionaries) - [RELOAD DICTIONARY](#query_language-system-reload-dictionary) +- [RELOAD MODELS](#query_language-system-reload-models) +- [RELOAD MODEL](#query_language-system-reload-model) - [DROP DNS CACHE](#query_language-system-drop-dns-cache) - [DROP MARK CACHE](#query_language-system-drop-mark-cache) - [DROP UNCOMPRESSED CACHE](#query_language-system-drop-uncompressed-cache) @@ -60,6 +62,26 @@ The status of the dictionary can be checked by querying the `system.dictionaries SELECT name, status FROM system.dictionaries; ``` +## RELOAD MODELS {#query_language-system-reload-models} + +Reloads all [CatBoost](../../guides/apply-catboost-model.md#applying-catboost-model-in-clickhouse) models if the configuration was updated without restarting the server. + +**Syntax** + +```sql +SYSTEM RELOAD MODELS +``` + +## RELOAD MODEL {#query_language-system-reload-model} + +Completely reloads a CatBoost model `model_name` if the configuration was updated without restarting the server. + +**Syntax** + +```sql +SYSTEM RELOAD MODEL +``` + ## DROP DNS CACHE {#query_language-system-drop-dns-cache} Resets ClickHouse’s internal DNS cache. Sometimes (for old ClickHouse versions) it is necessary to use this command when changing the infrastructure (changing the IP address of another ClickHouse server or the server used by dictionaries). diff --git a/docs/en/sql-reference/table-functions/dictionary.md b/docs/en/sql-reference/table-functions/dictionary.md new file mode 100644 index 00000000000..675fcb5bfdd --- /dev/null +++ b/docs/en/sql-reference/table-functions/dictionary.md @@ -0,0 +1,59 @@ +--- +toc_priority: 54 +toc_title: dictionary function +--- + +# dictionary {#dictionary-function} + +Displays the [dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) data as a ClickHouse table. Works the same way as [Dictionary](../../engines/table-engines/special/dictionary.md) engine. + +**Syntax** + +``` sql +dictionary('dict') +``` + +**Arguments** + +- `dict` — A dictionary name. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +A ClickHouse table. + +**Example** + +Input table `dictionary_source_table`: + +``` text +┌─id─┬─value─┐ +│ 0 │ 0 │ +│ 1 │ 1 │ +└────┴───────┘ +``` + +Create a dictionary: + +``` sql +CREATE DICTIONARY new_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table')) LAYOUT(DIRECT()); +``` + +Query: + +``` sql +SELECT * FROM dictionary('new_dictionary'); +``` + +Result: + +``` text +┌─id─┬─value─┐ +│ 0 │ 0 │ +│ 1 │ 1 │ +└────┴───────┘ +``` + +**See Also** + +- [Dictionary engine](../../engines/table-engines/special/dictionary.md#dictionary) diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 285ec862aab..d84edb3f46e 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -114,14 +114,14 @@ FROM s3('https://storage.yandexcloud.net/my-test-bucket-768/big_prefix/file-{000 Insert data into file `test-data.csv.gz`: ``` sql -INSERT INTO s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip') +INSERT INTO FUNCTION s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip') VALUES ('test-data', 1), ('test-data-2', 2); ``` Insert data into file `test-data.csv.gz` from existing table: ``` sql -INSERT INTO s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip') +INSERT INTO FUNCTION s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip') SELECT name, value FROM existing_table; ``` diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index a646347ea60..dcab019c9d5 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -6,9 +6,7 @@ toc_title: Window Functions # [experimental] Window Functions !!! warning "Warning" -This is an experimental feature that is currently in development and is not ready -for general use. It will change in unpredictable backwards-incompatible ways in -the future releases. Set `allow_experimental_window_functions = 1` to enable it. + This is an experimental feature that is currently in development and is not ready for general use. It will change in unpredictable backwards-incompatible ways in the future releases. Set `allow_experimental_window_functions = 1` to enable it. ClickHouse supports the standard grammar for defining windows and window functions. The following features are currently supported: diff --git a/docs/ru/engines/table-engines/integrations/mysql.md b/docs/ru/engines/table-engines/integrations/mysql.md index 5011c8a93c6..784c941c173 100644 --- a/docs/ru/engines/table-engines/integrations/mysql.md +++ b/docs/ru/engines/table-engines/integrations/mysql.md @@ -5,7 +5,7 @@ toc_title: MySQL # MySQL {#mysql} -Движок MySQL позволяет выполнять запросы `SELECT` над данными, хранящимися на удалённом MySQL сервере. +Движок MySQL позволяет выполнять запросы `SELECT` и `INSERT` над данными, хранящимися на удалённом MySQL сервере. ## Создание таблицы {#sozdanie-tablitsy} diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index 177d69dc3e0..4c910db4b92 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -10,7 +10,7 @@ toc_title: S3 ## Создание таблицы {#creating-a-table} ``` sql -CREATE TABLE s3_engine_table (name String, value UInt32) +CREATE TABLE s3_engine_table (name String, value UInt32) ENGINE = S3(path, [aws_access_key_id, aws_secret_access_key,] format, [compression]) ``` @@ -19,12 +19,12 @@ ENGINE = S3(path, [aws_access_key_id, aws_secret_access_key,] format, [compressi - `path` — URL-адрес бакета с указанием пути к файлу. Поддерживает следующие подстановочные знаки в режиме "только чтение": `*`, `?`, `{abc,def}` и `{N..M}` где `N`, `M` — числа, `'abc'`, `'def'` — строки. Подробнее смотри [ниже](#wildcards-in-path). - `format` — [формат](../../../interfaces/formats.md#formats) файла. - `aws_access_key_id`, `aws_secret_access_key` - данные пользователя учетной записи [AWS](https://aws.amazon.com/ru/). Вы можете использовать их для аутентификации ваших запросов. Необязательный параметр. Если параметры учетной записи не указаны, то используются данные из конфигурационного файла. Смотрите подробнее [Использование сервиса S3 для хранения данных](../mergetree-family/mergetree.md#table_engine-mergetree-s3). -- `compression` — тип сжатия. Возможные значения: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Необязательный параметр. Если не указано, то тип сжатия определяется автоматически по расширению файла. +- `compression` — тип сжатия. Возможные значения: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Необязательный параметр. Если не указано, то тип сжатия определяется автоматически по расширению файла. **Пример** ``` sql -CREATE TABLE s3_engine_table (name String, value UInt32) +CREATE TABLE s3_engine_table (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'gzip'); INSERT INTO s3_engine_table VALUES ('one', 1), ('two', 2), ('three', 3); SELECT * FROM s3_engine_table LIMIT 2; @@ -65,11 +65,12 @@ SELECT * FROM s3_engine_table LIMIT 2; ## Настройки движка S3 {#s3-settings} -Перед выполнением запроса или в конфигурационном файле могут быть установлены следующие настройки: +Перед выполнением запроса или в конфигурационном файле могут быть установлены следующие настройки: -- `s3_max_single_part_upload_size` — максимальный размер объекта для загрузки с использованием однокомпонентной загрузки в S3. Значение по умолчанию — `64 Mб`. +- `s3_max_single_part_upload_size` — максимальный размер объекта для загрузки с использованием однокомпонентной загрузки в S3. Значение по умолчанию — `64 Mб`. - `s3_min_upload_part_size` — минимальный размер объекта для загрузки при многокомпонентной загрузке в [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Значение по умолчанию — `512 Mб`. -- `s3_max_redirects` — максимальное количество разрешенных переадресаций S3. Значение по умолчанию — `10`. +- `s3_max_redirects` — максимальное количество разрешенных переадресаций S3. Значение по умолчанию — `10`. +- `s3_single_read_retries` — максимальное количество попыток запроса при единичном чтении. Значение по умолчанию — `4`. Соображение безопасности: если злонамеренный пользователь попробует указать произвольные URL-адреса S3, параметр `s3_max_redirects` должен быть установлен в ноль, чтобы избежать атак [SSRF] (https://en.wikipedia.org/wiki/Server-side_request_forgery). Как альтернатива, в конфигурации сервера должен быть указан `remote_host_filter`. @@ -78,7 +79,7 @@ SELECT * FROM s3_engine_table LIMIT 2; Для точки приема запроса (которая соответствует точному префиксу URL-адреса) в конфигурационном файле могут быть заданы следующие настройки: Обязательная настройка: -- `endpoint` — указывает префикс точки приема запроса. +- `endpoint` — указывает префикс точки приема запроса. Необязательные настройки: - `access_key_id` и `secret_access_key` — указывают учетные данные для использования с данной точкой приема запроса. @@ -86,7 +87,8 @@ SELECT * FROM s3_engine_table LIMIT 2; - `use_insecure_imds_request` — признак использования менее безопасного соединения при выполнении запроса к IMDS при получении учётных данных из метаданных Amazon EC2. Значение по умолчанию — `false`. - `region` — название региона S3. - `header` — добавляет указанный HTTP-заголовок к запросу на заданную точку приема запроса. Может быть определен несколько раз. -- `server_side_encryption_customer_key_base64` — устанавливает необходимые заголовки для доступа к объектам S3 с шифрованием SSE-C. +- `server_side_encryption_customer_key_base64` — устанавливает необходимые заголовки для доступа к объектам S3 с шифрованием SSE-C. +- `single_read_retries` — Максимальное количество попыток запроса при единичном чтении. Значение по умолчанию — `4`. **Пример** @@ -101,13 +103,14 @@ SELECT * FROM s3_engine_table LIMIT 2; + ``` ## Примеры использования {#usage-examples} -Предположим, у нас есть несколько файлов в формате TSV со следующими URL-адресами в S3: +Предположим, у нас есть несколько файлов в формате CSV со следующими URL-адресами в S3: - 'https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_1.csv' - 'https://storage.yandexcloud.net/my-test-bucket-768/some_prefix/some_file_2.csv' @@ -119,21 +122,21 @@ SELECT * FROM s3_engine_table LIMIT 2; 1. Существует несколько способов создать таблицу, включающую в себя все шесть файлов: ``` sql -CREATE TABLE table_with_range (name String, value UInt32) +CREATE TABLE table_with_range (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/some_file_{1..3}', 'CSV'); ``` 2. Другой способ: ``` sql -CREATE TABLE table_with_question_mark (name String, value UInt32) +CREATE TABLE table_with_question_mark (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/some_file_?', 'CSV'); ``` 3. Таблица содержит все файлы в обоих каталогах (все файлы должны соответствовать формату и схеме, описанным в запросе): ``` sql -CREATE TABLE table_with_asterisk (name String, value UInt32) +CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefix/*', 'CSV'); ``` @@ -142,7 +145,7 @@ ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_p 4. Создание таблицы из файлов с именами `file-000.csv`, `file-001.csv`, … , `file-999.csv`: ``` sql -CREATE TABLE big_table (name String, value UInt32) +CREATE TABLE big_table (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/big_prefix/file-{000..999}.csv', 'CSV'); ``` diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 4cff6fcfb80..b8f06c0fde0 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -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' 10000 5000 10 + 4 1000 /var/lib/clickhouse/disks/s3/ true @@ -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//`. -- `cache_enabled` — признак, разрешено ли хранение кэша засечек и файлов индекса в локальной файловой системе. Значение по умолчанию: `true`. -- `cache_path` — путь в локальной файловой системе, где будут храниться кэш засечек и файлы индекса. Значение по умолчанию: `/var/lib/clickhouse/disks//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//`. +- `cache_enabled` — признак, разрешено ли хранение кэша засечек и файлов индекса в локальной файловой системе. Значение по умолчанию: `true`. +- `cache_path` — путь в локальной файловой системе, где будут храниться кэш засечек и файлы индекса. Значение по умолчанию: `/var/lib/clickhouse/disks//cache/`. - `skip_access_check` — признак, выполнять ли проверку доступов при запуске диска. Если установлено значение `true`, то проверка не выполняется. Значение по умолчанию: `false`. @@ -805,6 +807,4 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' ``` -Если диск сконфигурирован как `cold`, данные будут переноситься в S3 при срабатывании правил TTL или когда свободное место на локальном диске станет меньше порогового значения, которое определяется как `move_factor * disk_size`. - - +Если диск сконфигурирован как `cold`, данные будут переноситься в S3 при срабатывании правил TTL или когда свободное место на локальном диске станет меньше порогового значения, которое определяется как `move_factor * disk_size`. diff --git a/docs/ru/engines/table-engines/special/dictionary.md b/docs/ru/engines/table-engines/special/dictionary.md index 243fd5395c0..15d32419472 100644 --- a/docs/ru/engines/table-engines/special/dictionary.md +++ b/docs/ru/engines/table-engines/special/dictionary.md @@ -90,3 +90,6 @@ select * from products limit 1; └───────────────┴─────────────────┘ ``` +**Смотрите также** + +- [Функция dictionary](../../../sql-reference/table-functions/dictionary.md#dictionary-function) diff --git a/docs/ru/guides/apply-catboost-model.md b/docs/ru/guides/apply-catboost-model.md index db2be63692f..a9cba1d7f70 100644 --- a/docs/ru/guides/apply-catboost-model.md +++ b/docs/ru/guides/apply-catboost-model.md @@ -18,6 +18,8 @@ toc_title: "Применение модели CatBoost в ClickHouse" Подробнее об обучении моделей в CatBoost, см. [Обучение и применение моделей](https://catboost.ai/docs/features/training.html#training). +Вы можете перегрузить модели CatBoost, если их конфигурация была обновлена, без перезагрузки сервера. Для этого используйте системные запросы [RELOAD MODEL](../sql-reference/statements/system.md#query_language-system-reload-model) и [RELOAD MODELS](../sql-reference/statements/system.md#query_language-system-reload-models). + ## Перед началом работы {#prerequisites} Если у вас еще нет [Docker](https://docs.docker.com/install/), установите его. diff --git a/docs/ru/operations/settings/merge-tree-settings.md b/docs/ru/operations/settings/merge-tree-settings.md index 2af99bb8026..4ef811eb1dc 100644 --- a/docs/ru/operations/settings/merge-tree-settings.md +++ b/docs/ru/operations/settings/merge-tree-settings.md @@ -120,6 +120,19 @@ Eсли суммарное число активных кусков во все Команда `Insert` создает один или несколько блоков (кусков). При вставке в Replicated таблицы ClickHouse для [дедупликации вставок](../../engines/table-engines/mergetree-family/replication.md) записывает в Zookeeper хеш-суммы созданных кусков. Но хранятся хеш-суммы не всех кусков, а только последние `replicated_deduplication_window`. Наиболее старые хеш-суммы удаляются из Zookeeper. Большое число `replicated_deduplication_window` замедляет `Insert`-ы. Хеш-сумма рассчитывается от композиции имен и типов полей, а также данных вставленного куска (потока байт). +## non_replicated_deduplication_window {#non-replicated-deduplication-window} + +Количество последних вставленных блоков в нереплицированной [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) таблице, для которых хранятся хеш-суммы для проверки дубликатов. + +Возможные значения: + +- Положительное целое число. +- 0 (дедупликация отключена). + +Значение по умолчанию: 0. + +Используется механизм дедупликации, аналогичный реплицированным таблицам (см. описание настройки [replicated_deduplication_window](#replicated-deduplication-window)). Хеш-суммы вставленных кусков записываются в локальный файл на диске. + ## replicated_deduplication_window_seconds {#replicated-deduplication-window-seconds} Число секунд, после которых хеш-суммы вставленных блоков удаляются из Zookeeper. diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index ff83eb425d0..a0378251ece 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -97,6 +97,10 @@ SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated')) Если словарь с источником `FILE` создается с помощью DDL-команды (`CREATE DICTIONARY ...`), источник словаря должен быть расположен в каталоге `user_files`. Иначе пользователи базы данных будут иметь доступ к произвольному файлу на узле ClickHouse. +**Смотрите также** + +- [Функция dictionary](../../../sql-reference/table-functions/dictionary.md#dictionary-function) + ## Исполняемый файл {#dicts-external_dicts_dict_sources-executable} Работа с исполняемым файлом зависит от [размещения словаря в памяти](external-dicts-dict-layout.md). Если тип размещения словаря `cache` и `complex_key_cache`, то ClickHouse запрашивает необходимые ключи, отправляя запрос в `STDIN` исполняемого файла. diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index b442a782100..69f9a89f4cb 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -264,6 +264,9 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp; └────────────────┘ ``` +!!! attention "Attention" + `Date` или `DateTime` это возвращаемый тип функций `toStartOf*`, который описан ниже. Несмотря на то, что эти функции могут принимать `DateTime64` в качестве аргумента, если переданное значение типа `DateTime64` выходит за пределы нормального диапазона (с 1925 по 2283 год), то это даст неверный результат. + ## toStartOfYear {#tostartofyear} Округляет дату или дату-с-временем вниз до первого дня года. diff --git a/docs/ru/sql-reference/statements/alter/partition.md b/docs/ru/sql-reference/statements/alter/partition.md index 02a87406e86..79242e7bbf3 100644 --- a/docs/ru/sql-reference/statements/alter/partition.md +++ b/docs/ru/sql-reference/statements/alter/partition.md @@ -16,6 +16,7 @@ toc_title: PARTITION - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) — удалить все значения в столбце для заданной партиции; - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) — очистить построенные вторичные индексы для заданной партиции; - [FREEZE PARTITION](#alter_freeze-partition) — создать резервную копию партиции; +- [UNFREEZE PARTITION](#alter_unfreeze-partition) — удалить резервную копию партиции; - [FETCH PARTITION](#alter_fetch-partition) — скачать партицию с другого сервера; - [MOVE PARTITION\|PART](#alter_move-partition) — переместить партицию/кускок на другой диск или том. @@ -170,9 +171,9 @@ ALTER TABLE table_name FREEZE [PARTITION partition_expr] !!! note "Примечание" Создание резервной копии не требует остановки сервера. -Для таблиц старого стиля имя партиций можно задавать в виде префикса (например, ‘2019’). В этом случае резервные копии будут созданы для всех соответствующих партиций. Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr). +Для таблиц старого стиля имя партиций можно задавать в виде префикса (например, `2019`). В этом случае, резервные копии будут созданы для всех соответствующих партиций. Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr). -Запрос делает следующее — для текущего состояния таблицы он формирует жесткие ссылки на данные в этой таблице. Ссылки размещаются в директории `/var/lib/clickhouse/shadow/N/...`, где: +Запрос формирует для текущего состояния таблицы жесткие ссылки на данные в этой таблице. Ссылки размещаются в директории `/var/lib/clickhouse/shadow/N/...`, где: - `/var/lib/clickhouse/` — рабочая директория ClickHouse, заданная в конфигурационном файле; - `N` — инкрементальный номер резервной копии. @@ -180,11 +181,11 @@ ALTER TABLE table_name FREEZE [PARTITION partition_expr] !!! note "Примечание" При использовании [нескольких дисков для хранения данных таблицы](../../statements/alter/index.md#table_engine-mergetree-multiple-volumes) директория `shadow/N` появляется на каждом из дисков, на которых были куски, попавшие под выражение `PARTITION`. -Структура директорий внутри резервной копии такая же, как внутри `/var/lib/clickhouse/`. Запрос выполнит ‘chmod’ для всех файлов, запрещая запись в них. +Структура директорий внутри резервной копии такая же, как внутри `/var/lib/clickhouse/`. Запрос выполнит `chmod` для всех файлов, запрещая запись в них. Обратите внимание, запрос `ALTER TABLE t FREEZE PARTITION` не реплицируется. Он создает резервную копию только на локальном сервере. После создания резервной копии данные из `/var/lib/clickhouse/shadow/` можно скопировать на удалённый сервер, а локальную копию удалить. -Резервная копия создается почти мгновенно (однако сначала запрос дожидается завершения всех запросов, которые выполняются для соответствующей таблицы). +Резервная копия создается почти мгновенно (однако, сначала запрос дожидается завершения всех запросов, которые выполняются для соответствующей таблицы). `ALTER TABLE t FREEZE PARTITION` копирует только данные, но не метаданные таблицы. Чтобы сделать резервную копию метаданных таблицы, скопируйте файл `/var/lib/clickhouse/metadata/database/table.sql` @@ -198,6 +199,14 @@ ALTER TABLE table_name FREEZE [PARTITION partition_expr] Подробнее о резервном копировании и восстановлении данных читайте в разделе [Резервное копирование данных](../../../operations/backup.md). +## UNFREEZE PARTITION {#alter_unfreeze-partition} + +``` sql +ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name' +``` + +Удаляет с диска "замороженные" партиции с указанным именем. Если секция `PARTITION` опущена, запрос удаляет резервную копию всех партиций сразу. + ## FETCH PARTITION {#alter_fetch-partition} ``` sql diff --git a/docs/ru/sql-reference/statements/rename.md b/docs/ru/sql-reference/statements/rename.md index 192426dbafa..b78505ce9c4 100644 --- a/docs/ru/sql-reference/statements/rename.md +++ b/docs/ru/sql-reference/statements/rename.md @@ -6,7 +6,7 @@ toc_title: RENAME # RENAME Statement {#misc_operations-rename} ## RENAME DATABASE {#misc_operations-rename_database} -Переименование базы данных +Переименовывает базу данных, поддерживается только для движка базы данных Atomic. ``` RENAME DATABASE atomic_database1 TO atomic_database2 [ON CLUSTER cluster] diff --git a/docs/ru/sql-reference/statements/system.md b/docs/ru/sql-reference/statements/system.md index 8c82cacdc43..2589408b8fa 100644 --- a/docs/ru/sql-reference/statements/system.md +++ b/docs/ru/sql-reference/statements/system.md @@ -8,6 +8,8 @@ toc_title: SYSTEM - [RELOAD EMBEDDED DICTIONARIES](#query_language-system-reload-emdedded-dictionaries) - [RELOAD DICTIONARIES](#query_language-system-reload-dictionaries) - [RELOAD DICTIONARY](#query_language-system-reload-dictionary) +- [RELOAD MODELS](#query_language-system-reload-models) +- [RELOAD MODEL](#query_language-system-reload-model) - [DROP DNS CACHE](#query_language-system-drop-dns-cache) - [DROP MARK CACHE](#query_language-system-drop-mark-cache) - [DROP UNCOMPRESSED CACHE](#query_language-system-drop-uncompressed-cache) @@ -37,7 +39,7 @@ toc_title: SYSTEM - [RESTART REPLICAS](#query_language-system-restart-replicas) ## RELOAD EMBEDDED DICTIONARIES] {#query_language-system-reload-emdedded-dictionaries} -Перегружет все [Встроенные словари](../dictionaries/internal-dicts.md). +Перегружает все [Встроенные словари](../dictionaries/internal-dicts.md). По умолчанию встроенные словари выключены. Всегда возвращает `Ok.`, вне зависимости от результата обновления встроенных словарей. @@ -57,6 +59,26 @@ toc_title: SYSTEM SELECT name, status FROM system.dictionaries; ``` +## RELOAD MODELS {#query_language-system-reload-models} + +Перегружает все модели [CatBoost](../../guides/apply-catboost-model.md#applying-catboost-model-in-clickhouse), если их конфигурация была обновлена, без перезагрузки сервера. + +**Синтаксис** + +```sql +SYSTEM RELOAD MODELS +``` + +## RELOAD MODEL {#query_language-system-reload-model} + +Полностью перегружает модель [CatBoost](../../guides/apply-catboost-model.md#applying-catboost-model-in-clickhouse) `model_name`, если ее конфигурация была обновлена, без перезагрузки сервера. + +**Синтаксис** + +```sql +SYSTEM RELOAD MODEL +``` + ## DROP DNS CACHE {#query_language-system-drop-dns-cache} Сбрасывает внутренний DNS кеш ClickHouse. Иногда (для старых версий ClickHouse) необходимо использовать эту команду при изменении инфраструктуры (смене IP адреса у другого ClickHouse сервера или сервера, используемого словарями). diff --git a/docs/ru/sql-reference/syntax.md b/docs/ru/sql-reference/syntax.md index dbbf5f92612..488e327dd31 100644 --- a/docs/ru/sql-reference/syntax.md +++ b/docs/ru/sql-reference/syntax.md @@ -94,7 +94,7 @@ INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') Обозначает, что значение отсутствует. -Чтобы в поле таблицы можно было хранить `NULL`, оно должно быть типа [Nullable](../sql-reference/syntax.md). +Чтобы в поле таблицы можно было хранить `NULL`, оно должно быть типа [Nullable](../sql-reference/data-types/nullable.md). В зависимости от формата данных (входных или выходных) `NULL` может иметь различное представление. Подробнее смотрите в документации для [форматов данных](../interfaces/formats.md#formats). diff --git a/docs/ru/sql-reference/table-functions/dictionary.md b/docs/ru/sql-reference/table-functions/dictionary.md new file mode 100644 index 00000000000..d4909bf5d9f --- /dev/null +++ b/docs/ru/sql-reference/table-functions/dictionary.md @@ -0,0 +1,59 @@ +--- +toc_priority: 54 +toc_title: dictionary +--- + +# dictionary {#dictionary-function} + +Отображает данные [словаря](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) как таблицу ClickHouse. Работает аналогично движку [Dictionary](../../engines/table-engines/special/dictionary.md). + +**Синтаксис** + +``` sql +dictionary('dict') +``` + +**Аргументы** + +- `dict` — имя словаря. [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +Таблица ClickHouse. + +**Пример** + +Входная таблица `dictionary_source_table`: + +``` text +┌─id─┬─value─┐ +│ 0 │ 0 │ +│ 1 │ 1 │ +└────┴───────┘ +``` + +Создаем словарь: + +``` sql +CREATE DICTIONARY new_dictionary(id UInt64, value UInt64 DEFAULT 0) PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table')) LAYOUT(DIRECT()); +``` + +Запрос: + +``` sql +SELECT * FROM dictionary('new_dictionary'); +``` + +Результат: + +``` text +┌─id─┬─value─┐ +│ 0 │ 0 │ +│ 1 │ 1 │ +└────┴───────┘ +``` + +**Смотрите также** + +- [Движок Dictionary](../../engines/table-engines/special/dictionary.md#dictionary) diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index 8d4c0fdbd5a..d4db49a1598 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -3,6 +3,9 @@ #include #include #include +#include +#include + namespace DB { @@ -90,7 +93,7 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo /// NOTE: Once you will update the completion list, /// do not forget to update 01676_clickhouse_client_autocomplete.sh - std::stringstream query; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + WriteBufferFromOwnString query; query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" "SELECT name FROM system.functions" " UNION ALL " diff --git a/programs/main.cpp b/programs/main.cpp index ccdf4d50fb4..c5df2596422 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -19,6 +19,7 @@ #include #include +#include #include #include @@ -172,11 +173,11 @@ enum class InstructionFail AVX512 = 8 }; -std::pair instructionFailToString(InstructionFail fail) +auto instructionFailToString(InstructionFail fail) { switch (fail) { -#define ret(x) return std::make_pair(x, ARRAY_SIZE(x) - 1) +#define ret(x) return std::make_tuple(STDERR_FILENO, x, ARRAY_SIZE(x) - 1) case InstructionFail::NONE: ret("NONE"); case InstructionFail::SSE3: @@ -260,28 +261,12 @@ void checkRequiredInstructionsImpl(volatile InstructionFail & fail) fail = InstructionFail::NONE; } -/// This function is safe to use in static initializers. -void writeErrorLen(const char * data, size_t size) -{ - while (size != 0) - { - ssize_t res = ::write(STDERR_FILENO, data, size); - - if ((-1 == res || 0 == res) && errno != EINTR) - _Exit(1); - - if (res > 0) - { - data += res; - size -= res; - } - } -} /// Macros to avoid using strlen(), since it may fail if SSE is not supported. #define writeError(data) do \ { \ static_assert(__builtin_constant_p(data)); \ - writeErrorLen(data, ARRAY_SIZE(data) - 1); \ + if (!writeRetry(STDERR_FILENO, data, ARRAY_SIZE(data) - 1)) \ + _Exit(1); \ } while (false) /// Check SSE and others instructions availability. Calls exit on fail. @@ -310,7 +295,8 @@ void checkRequiredInstructions() if (sigsetjmp(jmpbuf, 1)) { writeError("Instruction check fail. The CPU does not support "); - std::apply(writeErrorLen, instructionFailToString(fail)); + if (!std::apply(writeRetry, instructionFailToString(fail))) + _Exit(1); writeError(" instruction set.\n"); _Exit(1); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 4d73e033c31..3ec0e9fdac4 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1047,7 +1047,7 @@ int Server::main(const std::vector & /*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(); diff --git a/programs/server/config.xml b/programs/server/config.xml index 75647b10416..dd50a693403 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -705,6 +705,8 @@ "yandex.ru" -> "yandex.ru:443", "yandex.ru:80" etc. is allowed, but "yandex.ru:80" -> only "yandex.ru:80" is allowed. If the host is specified as IP address, it is checked as specified in URL. Example: "[2a02:6b8:a::a]". If there are redirects and support for redirects is enabled, every redirect (the Location field) is checked. + Host should be specified using the host xml tag: + yandex.ru --> - true - - test - node1 - 9000 - - - test - node2 - 9000 - - - - - - - true - - test - node3 - 9000 - - - test - node4 - 9000 - - - - - - - true - - test - node5 - 9000 - - - test - node6 - 9000 - - - - - - - true - - test - node11 - 9000 - - - test - node12 - 9000 - - - - - diff --git a/tests/integration/test_adaptive_granularity/configs/wide_parts_only.xml b/tests/integration/test_adaptive_granularity/configs/wide_parts_only.xml deleted file mode 100644 index 42e2173f718..00000000000 --- a/tests/integration/test_adaptive_granularity/configs/wide_parts_only.xml +++ /dev/null @@ -1,6 +0,0 @@ - - - 0 - 0 - - diff --git a/tests/integration/test_adaptive_granularity/test.py b/tests/integration/test_adaptive_granularity/test.py deleted file mode 100644 index 12bfc22d7d9..00000000000 --- a/tests/integration/test_adaptive_granularity/test.py +++ /dev/null @@ -1,394 +0,0 @@ -import time - -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") - - # 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)") - - def callback(n): - new_config = """ - - 1 - 0 -""" - - 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 = """ - - 0 - 0 -""" - - 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' diff --git a/tests/integration/test_adaptive_granularity_different_settings/__init__.py b/tests/integration/test_adaptive_granularity_different_settings/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_adaptive_granularity_different_settings/test.py b/tests/integration/test_adaptive_granularity_different_settings/test.py deleted file mode 100644 index ce8f32b6ec6..00000000000 --- a/tests/integration/test_adaptive_granularity_different_settings/test.py +++ /dev/null @@ -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("SYSTEM SYNC REPLICA test", timeout=10) - node2.query("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" diff --git a/tests/integration/test_adaptive_granularity_replicated/__init__.py b/tests/integration/test_adaptive_granularity_replicated/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_adaptive_granularity_replicated/test.py b/tests/integration/test_adaptive_granularity_replicated/test.py deleted file mode 100644 index 5903cb85603..00000000000 --- a/tests/integration/test_adaptive_granularity_replicated/test.py +++ /dev/null @@ -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)]) diff --git a/tests/integration/test_allowed_client_hosts/test.py b/tests/integration/test_allowed_client_hosts/test.py index cf83718e25c..7b803fd50f3 100644 --- a/tests/integration/test_allowed_client_hosts/test.py +++ b/tests/integration/test_allowed_client_hosts/test.py @@ -37,8 +37,9 @@ def query(node, query): def setup_nodes(): try: cluster.start() - query(server, "CREATE TABLE test_table (x Int32) ENGINE = MergeTree() ORDER BY tuple()") - query(server, "INSERT INTO test_table VALUES (5)") + query(server, "DROP TABLE IF EXISTS test_allowed_client_hosts") + query(server, "CREATE TABLE test_allowed_client_hosts (x Int32) ENGINE = MergeTree() ORDER BY tuple()") + query(server, "INSERT INTO test_allowed_client_hosts VALUES (5)") yield cluster @@ -57,8 +58,8 @@ def test_allowed_host(): # expected_to_fail.extend([clientC3, clientD2]) for client_node in expected_to_pass: - assert query_from_one_node_to_another(client_node, server, "SELECT * FROM test_table") == "5\n" + assert query_from_one_node_to_another(client_node, server, "SELECT * FROM test_allowed_client_hosts") == "5\n" for client_node in expected_to_fail: with pytest.raises(Exception, match=r'default: Authentication failed'): - query_from_one_node_to_another(client_node, server, "SELECT * FROM test_table") + query_from_one_node_to_another(client_node, server, "SELECT * FROM test_allowed_client_hosts") diff --git a/tests/integration/test_allowed_url_from_config/test.py b/tests/integration/test_allowed_url_from_config/test.py index 8001af35913..756a352545e 100644 --- a/tests/integration/test_allowed_url_from_config/test.py +++ b/tests/integration/test_allowed_url_from_config/test.py @@ -115,8 +115,10 @@ def test_table_function_remote(start_cluster): def test_redirect(start_cluster): - start_cluster.hdfs_api.write_data("/simple_storage", "1\t\n") - assert start_cluster.hdfs_api.read_data("/simple_storage") == "1\t\n" + hdfs_api = start_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage", "1\t\n") + assert hdfs_api.read_data("/simple_storage") == "1\t\n" node7.query( "CREATE TABLE table_test_7_1 (word String) ENGINE=URL('http://hdfs1:50070/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', CSV)") assert "not allowed" in node7.query_and_get_error("SET max_http_get_redirects=1; SELECT * from table_test_7_1") diff --git a/tests/integration/test_alter_codec/test.py b/tests/integration/test_alter_codec/test.py index f51dc9a54ff..2117893af5b 100644 --- a/tests/integration/test_alter_codec/test.py +++ b/tests/integration/test_alter_codec/test.py @@ -7,9 +7,6 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', main_configs=['configs/logs_config.xml']) -node2 = cluster.add_instance('node2', - main_configs=['configs/logs_config.xml']) - @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_alter_on_mixed_type_cluster/test.py b/tests/integration/test_alter_on_mixed_type_cluster/test.py index 269072f239a..c22626cb379 100644 --- a/tests/integration/test_alter_on_mixed_type_cluster/test.py +++ b/tests/integration/test_alter_on_mixed_type_cluster/test.py @@ -1,7 +1,6 @@ import pytest from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) @@ -18,19 +17,19 @@ def started_cluster(): cluster.start() for node in [node1, node2]: - node.query(''' - CREATE TABLE test_table_replicated(date Date, id UInt32, value Int32) + node.query_with_retry(''' + CREATE TABLE IF NOT EXISTS test_table_replicated(date Date, id UInt32, value Int32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '{replica}') ORDER BY id; '''.format(replica=node.name)) - node.query('''CREATE TABLE test_table(date Date, id UInt32, value Int32) ENGINE=MergeTree ORDER BY id''') + node.query_with_retry('''CREATE TABLE IF NOT EXISTS test_table(date Date, id UInt32, value Int32) ENGINE=MergeTree ORDER BY id''') for node in [node3, node4]: - node.query(''' - CREATE TABLE test_table_replicated(date Date, id UInt32, value Int32) + node.query_with_retry(''' + CREATE TABLE IF NOT EXISTS test_table_replicated(date Date, id UInt32, value Int32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/1/someotable', '{replica}') ORDER BY id; '''.format(replica=node.name)) - node.query('''CREATE TABLE test_table(date Date, id UInt32, value Int32) ENGINE=MergeTree ORDER BY id''') + node.query_with_retry('''CREATE TABLE IF NOT EXISTS test_table(date Date, id UInt32, value Int32) ENGINE=MergeTree ORDER BY id''') yield cluster @@ -79,7 +78,7 @@ def test_alter_replicated_on_cluster(started_cluster): assert node3.query("SELECT date FROM test_table_replicated") == '2019-10-01 00:00:00\n' assert node4.query("SELECT date FROM test_table_replicated") == '2019-10-01 00:00:00\n' - node3.query("ALTER TABLE test_table_replicated ON CLUSTER 'test_cluster_mixed' MODIFY COLUMN value String", settings={"replication_alter_partitions_sync": "2"}) + node3.query_with_retry("ALTER TABLE test_table_replicated ON CLUSTER 'test_cluster_mixed' MODIFY COLUMN value String", settings={"replication_alter_partitions_sync": "2"}) for node in [node2, node4]: node.query("INSERT INTO test_table_replicated VALUES(toDateTime('2019-10-02 00:00:00'), 2, 'Hello')") diff --git a/tests/integration/test_always_fetch_merged/test.py b/tests/integration/test_always_fetch_merged/test.py index 7ba8e05129b..e3b2d5ca392 100644 --- a/tests/integration/test_always_fetch_merged/test.py +++ b/tests/integration/test_always_fetch_merged/test.py @@ -22,15 +22,15 @@ def started_cluster(): def test_replica_always_download(started_cluster): - node1.query(""" - CREATE TABLE test_table( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS test_table( key UInt64, value String ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_table/replicated', '1') ORDER BY tuple() """) - node2.query(""" - CREATE TABLE test_table( + node2.query_with_retry(""" + CREATE TABLE IF NOT EXISTS test_table( key UInt64, value String ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_table/replicated', '2') @@ -42,12 +42,12 @@ def test_replica_always_download(started_cluster): node1.query("SYSTEM STOP MERGES") for i in range(0, 10): - node1.query("INSERT INTO test_table VALUES ({}, '{}')".format(i, i)) + node1.query_with_retry("INSERT INTO test_table VALUES ({}, '{}')".format(i, i)) assert node1.query("SELECT COUNT() FROM test_table") == "10\n" assert_eq_with_retry(node2, "SELECT COUNT() FROM test_table", "10\n") - time.sleep(3) + time.sleep(5) # Nothing is merged assert node1.query("SELECT COUNT() FROM system.parts WHERE table = 'test_table' and active=1") == "10\n" diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index 8b0c1ffbc5c..605ca6a4f51 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -1,5 +1,6 @@ import time import pytest +import logging from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry @@ -7,9 +8,9 @@ from helpers.network import PartitionManager from helpers.corrupt_part_data_on_disk import corrupt_part_data_by_path def fill_node(node): - node.query( + node.query_with_retry( ''' - CREATE TABLE test(n UInt32) + CREATE TABLE IF NOT EXISTS test(n UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test', '{replica}') ORDER BY n PARTITION BY n % 10; '''.format(replica=node.name)) @@ -39,9 +40,9 @@ def start_cluster(): def check_data(nodes, detached_parts): for node in nodes: print("> Replication queue for", node.name, "\n> table\treplica_name\tsource_replica\ttype\tposition\n", - node.query("SELECT table, replica_name, source_replica, type, position FROM system.replication_queue")) + node.query_with_retry("SELECT table, replica_name, source_replica, type, position FROM system.replication_queue")) - node.query("SYSTEM SYNC REPLICA test") + node.query_with_retry("SYSTEM SYNC REPLICA test") print("> Checking data integrity for", node.name) @@ -56,7 +57,7 @@ def check_data(nodes, detached_parts): for other in nodes: if other != node: - print("> Checking data consistency,", other.name, "vs", node.name) + logging.debug(f"> Checking data consistency, {other.name} vs {node.name}") assert_eq_with_retry(other, "SELECT * FROM test ORDER BY n", res) @@ -92,11 +93,12 @@ def test_attach_without_fetching(start_cluster): # 3. Break the part data on the second node to corrupt the checksums. # Replica 3 should download the data from replica 1 as there is no local data. # Replica 2 should also download the data from 1 as the checksums won't match. - print("Checking attach with corrupted part data with files missing") + logging.debug("Checking attach with corrupted part data with files missing") - print("Before deleting:", node_2.exec_in_container(['bash', '-c', + to_delete = node_2.exec_in_container(['bash', '-c', 'cd {p} && ls *.bin'.format( - p="/var/lib/clickhouse/data/default/test/detached/2_0_0_0")], privileged=True)) + p="/var/lib/clickhouse/data/default/test/detached/2_0_0_0")], privileged=True) + logging.debug(f"Before deleting: {to_delete}") node_2.exec_in_container(['bash', '-c', 'cd {p} && rm -fr *.bin'.format( diff --git a/tests/integration/test_backup_restore/test.py b/tests/integration/test_backup_restore/test.py index 170266aaaea..d395b55a2a6 100644 --- a/tests/integration/test_backup_restore/test.py +++ b/tests/integration/test_backup_restore/test.py @@ -1,12 +1,12 @@ import os.path import pytest +import logging from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance') -q = instance.query +instance = cluster.add_instance('node') path_to_data = '/var/lib/clickhouse/' @@ -14,7 +14,19 @@ path_to_data = '/var/lib/clickhouse/' def started_cluster(): try: cluster.start() - q('CREATE DATABASE test ENGINE = Ordinary') # Different path in shadow/ with Atomic + instance.query('CREATE DATABASE test ENGINE = Ordinary') # Different path in shadow/ with Atomic + instance.query("DROP TABLE IF EXISTS test.tbl") + instance.query("CREATE TABLE test.tbl (p Date, k Int8) ENGINE = MergeTree PARTITION BY toYYYYMM(p) ORDER BY p") + for i in range(1, 4): + instance.query('INSERT INTO test.tbl (p, k) VALUES(toDate({}), {})'.format(i, i)) + for i in range(31, 34): + instance.query('INSERT INTO test.tbl (p, k) VALUES(toDate({}), {})'.format(i, i)) + + expected = TSV('1970-01-02\t1\n1970-01-03\t2\n1970-01-04\t3\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33') + res = instance.query("SELECT * FROM test.tbl ORDER BY p") + assert (TSV(res) == expected) + + instance.query("ALTER TABLE test.tbl FREEZE") yield cluster @@ -22,131 +34,106 @@ def started_cluster(): cluster.shutdown() -def exec_bash(cmd): - cmd = '/bin/bash -c "{}"'.format(cmd.replace('"', '\\"')) - return instance.exec_in_container(cmd) - - -def copy_backup_to_detached(database, src_table, dst_table): +def copy_backup_to_detached(instance, database, src_table, dst_table): fp_increment = os.path.join(path_to_data, 'shadow/increment.txt') - increment = exec_bash('cat ' + fp_increment).strip() + increment = instance.exec_in_container(['cat', fp_increment]).strip() fp_backup = os.path.join(path_to_data, 'shadow', increment, 'data', database, src_table) fp_detached = os.path.join(path_to_data, 'data', database, dst_table, 'detached') - exec_bash('cp -r {}/* {}/'.format(fp_backup, fp_detached)) + logging.debug(f'copy from {fp_backup} to {fp_detached}. increment {fp_increment}') + instance.exec_in_container(['cp', '-r', f'{fp_backup}', '-T' , f'{fp_detached}']) +def test_restore(started_cluster): + instance.query("CREATE TABLE test.tbl1 AS test.tbl") -@pytest.fixture -def backup_restore(started_cluster): - q("DROP TABLE IF EXISTS test.tbl") - q("CREATE TABLE test.tbl (p Date, k Int8) ENGINE = MergeTree PARTITION BY toYYYYMM(p) ORDER BY p") - for i in range(1, 4): - q('INSERT INTO test.tbl (p, k) VALUES(toDate({}), {})'.format(i, i)) - for i in range(31, 34): - q('INSERT INTO test.tbl (p, k) VALUES(toDate({}), {})'.format(i, i)) - - expected = TSV('1970-01-02\t1\n1970-01-03\t2\n1970-01-04\t3\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33') - res = q("SELECT * FROM test.tbl ORDER BY p") - assert (TSV(res) == expected) - - q("ALTER TABLE test.tbl FREEZE") - - yield - - q("DROP TABLE IF EXISTS test.tbl") - - -def test_restore(backup_restore): - q("CREATE TABLE test.tbl1 AS test.tbl") - - copy_backup_to_detached('test', 'tbl', 'tbl1') + copy_backup_to_detached(started_cluster.instances['node'], 'test', 'tbl', 'tbl1') # The data_version of parts to be attached are larger than the newly created table's data_version. - q("ALTER TABLE test.tbl1 ATTACH PARTITION 197001") - q("ALTER TABLE test.tbl1 ATTACH PARTITION 197002") - q("SELECT sleep(2)") + instance.query("ALTER TABLE test.tbl1 ATTACH PARTITION 197001") + instance.query("ALTER TABLE test.tbl1 ATTACH PARTITION 197002") + instance.query("SELECT sleep(2)") # Validate the attached parts are identical to the backup. expected = TSV('1970-01-02\t1\n1970-01-03\t2\n1970-01-04\t3\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33') - res = q("SELECT * FROM test.tbl1 ORDER BY p") + res = instance.query("SELECT * FROM test.tbl1 ORDER BY p") assert (TSV(res) == expected) - q("ALTER TABLE test.tbl1 UPDATE k=10 WHERE 1") - q("SELECT sleep(2)") + instance.query("ALTER TABLE test.tbl1 UPDATE k=10 WHERE 1") + instance.query("SELECT sleep(2)") # Validate mutation has been applied to all attached parts. expected = TSV('1970-01-02\t10\n1970-01-03\t10\n1970-01-04\t10\n1970-02-01\t10\n1970-02-02\t10\n1970-02-03\t10') - res = q("SELECT * FROM test.tbl1 ORDER BY p") + res = instance.query("SELECT * FROM test.tbl1 ORDER BY p") assert (TSV(res) == expected) - q("DROP TABLE IF EXISTS test.tbl1") + instance.query("DROP TABLE IF EXISTS test.tbl1") -def test_attach_partition(backup_restore): - q("CREATE TABLE test.tbl2 AS test.tbl") +def test_attach_partition(started_cluster): + instance.query("CREATE TABLE test.tbl2 AS test.tbl") for i in range(3, 5): - q('INSERT INTO test.tbl2(p, k) VALUES(toDate({}), {})'.format(i, i)) + instance.query('INSERT INTO test.tbl2(p, k) VALUES(toDate({}), {})'.format(i, i)) for i in range(33, 35): - q('INSERT INTO test.tbl2(p, k) VALUES(toDate({}), {})'.format(i, i)) + instance.query('INSERT INTO test.tbl2(p, k) VALUES(toDate({}), {})'.format(i, i)) expected = TSV('1970-01-04\t3\n1970-01-05\t4\n1970-02-03\t33\n1970-02-04\t34') - res = q("SELECT * FROM test.tbl2 ORDER BY p") + res = instance.query("SELECT * FROM test.tbl2 ORDER BY p") assert (TSV(res) == expected) - copy_backup_to_detached('test', 'tbl', 'tbl2') + copy_backup_to_detached(started_cluster.instances['node'], 'test', 'tbl', 'tbl2') # The data_version of parts to be attached # - may be less than, equal to or larger than the current table's data_version. # - may intersect with the existing parts of a partition. - q("ALTER TABLE test.tbl2 ATTACH PARTITION 197001") - q("ALTER TABLE test.tbl2 ATTACH PARTITION 197002") - q("SELECT sleep(2)") + instance.query("ALTER TABLE test.tbl2 ATTACH PARTITION 197001") + instance.query("ALTER TABLE test.tbl2 ATTACH PARTITION 197002") + instance.query("SELECT sleep(2)") expected = TSV( '1970-01-02\t1\n1970-01-03\t2\n1970-01-04\t3\n1970-01-04\t3\n1970-01-05\t4\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33\n1970-02-03\t33\n1970-02-04\t34') - res = q("SELECT * FROM test.tbl2 ORDER BY p") + res = instance.query("SELECT * FROM test.tbl2 ORDER BY p") assert (TSV(res) == expected) - q("ALTER TABLE test.tbl2 UPDATE k=10 WHERE 1") - q("SELECT sleep(2)") + instance.query("ALTER TABLE test.tbl2 UPDATE k=10 WHERE 1") + instance.query("SELECT sleep(2)") # Validate mutation has been applied to all attached parts. expected = TSV( '1970-01-02\t10\n1970-01-03\t10\n1970-01-04\t10\n1970-01-04\t10\n1970-01-05\t10\n1970-02-01\t10\n1970-02-02\t10\n1970-02-03\t10\n1970-02-03\t10\n1970-02-04\t10') - res = q("SELECT * FROM test.tbl2 ORDER BY p") + res = instance.query("SELECT * FROM test.tbl2 ORDER BY p") assert (TSV(res) == expected) - q("DROP TABLE IF EXISTS test.tbl2") + instance.query("DROP TABLE IF EXISTS test.tbl2") -def test_replace_partition(backup_restore): - q("CREATE TABLE test.tbl3 AS test.tbl") +def test_replace_partition(started_cluster): + instance.query("CREATE TABLE test.tbl3 AS test.tbl") for i in range(3, 5): - q('INSERT INTO test.tbl3(p, k) VALUES(toDate({}), {})'.format(i, i)) + instance.query('INSERT INTO test.tbl3(p, k) VALUES(toDate({}), {})'.format(i, i)) for i in range(33, 35): - q('INSERT INTO test.tbl3(p, k) VALUES(toDate({}), {})'.format(i, i)) + instance.query('INSERT INTO test.tbl3(p, k) VALUES(toDate({}), {})'.format(i, i)) expected = TSV('1970-01-04\t3\n1970-01-05\t4\n1970-02-03\t33\n1970-02-04\t34') - res = q("SELECT * FROM test.tbl3 ORDER BY p") + res = instance.query("SELECT * FROM test.tbl3 ORDER BY p") assert (TSV(res) == expected) - copy_backup_to_detached('test', 'tbl', 'tbl3') + copy_backup_to_detached(started_cluster.instances['node'], 'test', 'tbl', 'tbl3') # The data_version of parts to be copied # - may be less than, equal to or larger than the current table data_version. # - may intersect with the existing parts of a partition. - q("ALTER TABLE test.tbl3 REPLACE PARTITION 197002 FROM test.tbl") - q("SELECT sleep(2)") + instance.query("ALTER TABLE test.tbl3 REPLACE PARTITION 197002 FROM test.tbl") + instance.query("SELECT sleep(2)") expected = TSV('1970-01-04\t3\n1970-01-05\t4\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33') - res = q("SELECT * FROM test.tbl3 ORDER BY p") + res = instance.query("SELECT * FROM test.tbl3 ORDER BY p") assert (TSV(res) == expected) - q("ALTER TABLE test.tbl3 UPDATE k=10 WHERE 1") - q("SELECT sleep(2)") + instance.query("ALTER TABLE test.tbl3 UPDATE k=10 WHERE 1") + instance.query("SELECT sleep(2)") # Validate mutation has been applied to all copied parts. expected = TSV('1970-01-04\t10\n1970-01-05\t10\n1970-02-01\t10\n1970-02-02\t10\n1970-02-03\t10') - res = q("SELECT * FROM test.tbl3 ORDER BY p") + res = instance.query("SELECT * FROM test.tbl3 ORDER BY p") assert (TSV(res) == expected) - q("DROP TABLE IF EXISTS test.tbl3") + instance.query("DROP TABLE IF EXISTS test.tbl3") diff --git a/tests/integration/test_backward_compatibility/test.py b/tests/integration/test_backward_compatibility/test.py index d6a6520ed34..71aedb78e5b 100644 --- a/tests/integration/test_backward_compatibility/test.py +++ b/tests/integration/test_backward_compatibility/test.py @@ -12,7 +12,7 @@ def start_cluster(): try: cluster.start() for i, node in enumerate([node1, node2]): - node.query( + node.query_with_retry( '''CREATE TABLE t(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/t', '{}') PARTITION BY toYYYYMM(date) diff --git a/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py b/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py index 3b35c112887..5ed97e7a9a5 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="aggregate_state") node1 = cluster.add_instance('node1', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', stay_alive=True, with_installed_binary=True) diff --git a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py index 91a0a87b6e2..463fadc36e8 100644 --- a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="short_strings") node1 = cluster.add_instance('node1', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', stay_alive=True, with_installed_binary=True) node2 = cluster.add_instance('node2', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', diff --git a/tests/integration/test_check_table/test.py b/tests/integration/test_check_table/test.py index d204f6c5810..b184813d24f 100644 --- a/tests/integration/test_check_table/test.py +++ b/tests/integration/test_check_table/test.py @@ -122,9 +122,9 @@ def test_check_replicated_table_simple(started_cluster): def test_check_replicated_table_corruption(started_cluster): for node in [node1, node2]: - node.query("DROP TABLE IF EXISTS replicated_mt_1") + node.query_with_retry("DROP TABLE IF EXISTS replicated_mt_1") - node.query(''' + node.query_with_retry(''' CREATE TABLE replicated_mt_1(date Date, id UInt32, value Int32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt_1', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; '''.format(replica=node.name)) @@ -136,7 +136,7 @@ def test_check_replicated_table_corruption(started_cluster): assert node1.query("SELECT count() from replicated_mt_1") == "4\n" assert node2.query("SELECT count() from replicated_mt_1") == "4\n" - part_name = node1.query( + part_name = node1.query_with_retry( "SELECT name from system.parts where table = 'replicated_mt_1' and partition_id = '201901' and active = 1").strip() corrupt_data_part_on_disk(node1, "replicated_mt_1", part_name) @@ -144,7 +144,7 @@ def test_check_replicated_table_corruption(started_cluster): "check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and will try to fetch.\n".format( p=part_name) - node1.query("SYSTEM SYNC REPLICA replicated_mt_1") + node1.query_with_retry("SYSTEM SYNC REPLICA replicated_mt_1") assert node1.query("CHECK TABLE replicated_mt_1 PARTITION 201901", settings={"check_query_single_value_result": 0}) == "{}\t1\t\n".format(part_name) assert node1.query("SELECT count() from replicated_mt_1") == "4\n" diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 57f9d150c8d..c6068e3a6e9 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -277,10 +277,10 @@ class Task_self_copy: instance.query("DROP DATABASE db2 SYNC") -def execute_task(task, cmd_options): +def execute_task(started_cluster, task, cmd_options): task.start() - zk = cluster.get_kazoo_client('zoo1') + zk = started_cluster.get_kazoo_client('zoo1') print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) try: @@ -293,7 +293,7 @@ def execute_task(task, cmd_options): zk.create(zk_task_path + "/description", task.copier_task_config.encode()) # Run cluster-copier processes on each node - docker_api = docker.from_env().api + docker_api = started_cluster.docker_client.api copiers_exec_ids = [] cmd = ['/usr/bin/clickhouse', 'copier', @@ -305,7 +305,7 @@ def execute_task(task, cmd_options): copiers = random.sample(list(cluster.instances.keys()), 3) for instance_name in copiers: - instance = cluster.instances[instance_name] + instance = started_cluster.instances[instance_name] container = instance.get_docker_handle() instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") @@ -318,7 +318,7 @@ def execute_task(task, cmd_options): # Wait for copiers stopping and check their return codes for exec_id, instance_name in zip(copiers_exec_ids, copiers): - instance = cluster.instances[instance_name] + instance = started_cluster.instances[instance_name] while True: res = docker_api.exec_inspect(exec_id) if not res['Running']: @@ -344,9 +344,9 @@ def execute_task(task, cmd_options): ) def test_copy_simple(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(Task1(started_cluster), ['--experimental-use-sample-offset', '1']) + execute_task(started_cluster, Task1(started_cluster), ['--experimental-use-sample-offset', '1']) else: - execute_task(Task1(started_cluster), []) + execute_task(started_cluster, Task1(started_cluster), []) @pytest.mark.parametrize( @@ -358,10 +358,10 @@ def test_copy_simple(started_cluster, use_sample_offset): ) def test_copy_with_recovering(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), + execute_task(started_cluster, Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(started_cluster, Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.parametrize( @@ -373,45 +373,45 @@ def test_copy_with_recovering(started_cluster, use_sample_offset): ) def test_copy_with_recovering_after_move_faults(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), + execute_task(started_cluster, Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(started_cluster, Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) @pytest.mark.timeout(600) def test_copy_month_to_week_partition(started_cluster): - execute_task(Task2(started_cluster, "test1"), []) + execute_task(started_cluster, Task2(started_cluster, "test1"), []) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering(started_cluster): - execute_task(Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(started_cluster, Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster): - execute_task(Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(started_cluster, Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) def test_block_size(started_cluster): - execute_task(Task_test_block_size(started_cluster), []) + execute_task(started_cluster, Task_test_block_size(started_cluster), []) def test_no_index(started_cluster): - execute_task(Task_no_index(started_cluster), []) + execute_task(started_cluster, Task_no_index(started_cluster), []) def test_no_arg(started_cluster): - execute_task(Task_no_arg(started_cluster), []) + execute_task(started_cluster, Task_no_arg(started_cluster), []) def test_non_partitioned_table(started_cluster): - execute_task(Task_non_partitioned_table(started_cluster), []) + execute_task(started_cluster, Task_non_partitioned_table(started_cluster), []) def test_self_copy(started_cluster): - execute_task(Task_self_copy(started_cluster), []) + execute_task(started_cluster, Task_self_copy(started_cluster), []) if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: diff --git a/tests/integration/test_cluster_copier/trivial_test.py b/tests/integration/test_cluster_copier/trivial_test.py index 8a43440ac90..717ff9d8d34 100644 --- a/tests/integration/test_cluster_copier/trivial_test.py +++ b/tests/integration/test_cluster_copier/trivial_test.py @@ -79,10 +79,10 @@ class TaskTrivial: node.query("DROP TABLE trivial") -def execute_task(task, cmd_options): +def execute_task(started_cluster, task, cmd_options): task.start() - zk = cluster.get_kazoo_client('zoo1') + zk = started_cluster.get_kazoo_client('zoo1') print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) zk_task_path = task.zk_task_path @@ -90,7 +90,7 @@ def execute_task(task, cmd_options): zk.create(zk_task_path + "/description", task.copier_task_config) # Run cluster-copier processes on each node - docker_api = docker.from_env().api + docker_api = started_cluster.docker_client.api copiers_exec_ids = [] cmd = ['/usr/bin/clickhouse', 'copier', @@ -101,7 +101,7 @@ def execute_task(task, cmd_options): print(cmd) - for instance_name, instance in cluster.instances.items(): + for instance_name, instance in started_cluster.instances.items(): container = instance.get_docker_handle() exec_id = docker_api.exec_create(container.id, cmd, stderr=True) docker_api.exec_start(exec_id, detach=True) @@ -110,7 +110,7 @@ def execute_task(task, cmd_options): print("Copier for {} ({}) has started".format(instance.name, instance.ip_address)) # Wait for copiers stopping and check their return codes - for exec_id, instance in zip(copiers_exec_ids, iter(cluster.instances.values())): + for exec_id, instance in zip(copiers_exec_ids, iter(started_cluster.instances.values())): while True: res = docker_api.exec_inspect(exec_id) if not res['Running']: @@ -137,10 +137,10 @@ def execute_task(task, cmd_options): ) def test_trivial_copy(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--experimental-use-sample-offset', '1']) + execute_task(started_cluster, TaskTrivial(started_cluster, use_sample_offset), ['--experimental-use-sample-offset', '1']) else: print("AAAAA") - execute_task(TaskTrivial(started_cluster, use_sample_offset), []) + execute_task(started_cluster, TaskTrivial(started_cluster, use_sample_offset), []) @pytest.mark.parametrize( @@ -152,10 +152,10 @@ def test_trivial_copy(started_cluster, use_sample_offset): ) def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), + execute_task(started_cluster, TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(started_cluster, TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.parametrize( @@ -167,10 +167,10 @@ def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset): ) def test_trivial_copy_with_move_fault(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), + execute_task(started_cluster, TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(started_cluster, TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) if __name__ == '__main__': diff --git a/tests/integration/test_compression_codec_read/test.py b/tests/integration/test_compression_codec_read/test.py index 0eb1f5aa867..35ae60f05ea 100644 --- a/tests/integration/test_compression_codec_read/test.py +++ b/tests/integration/test_compression_codec_read/test.py @@ -17,6 +17,8 @@ def start_cluster(): cluster.shutdown() def test_default_codec_read(start_cluster): + node1.query("DROP TABLE IF EXISTS test_18340") + node1.query(""" CREATE TABLE test_18340 ( diff --git a/tests/integration/test_concurrent_ttl_merges/test.py b/tests/integration/test_concurrent_ttl_merges/test.py index ba5ed9f0758..f89d024842c 100644 --- a/tests/integration/test_concurrent_ttl_merges/test.py +++ b/tests/integration/test_concurrent_ttl_merges/test.py @@ -55,6 +55,7 @@ def count_running_mutations(node, table): # but it revealed a bug when we assign different merges to the same part # on the borders of partitions. def test_no_ttl_merges_in_busy_pool(started_cluster): + node1.query("DROP TABLE IF EXISTS test_ttl") node1.query( "CREATE TABLE test_ttl (d DateTime, key UInt64, data UInt64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0, number_of_free_entries_in_pool_to_execute_mutation = 0") @@ -87,6 +88,7 @@ def test_no_ttl_merges_in_busy_pool(started_cluster): def test_limited_ttl_merges_in_empty_pool(started_cluster): + node1.query("DROP TABLE IF EXISTS test_ttl_v2") node1.query( "CREATE TABLE test_ttl_v2 (d DateTime, key UInt64, data UInt64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0") @@ -110,13 +112,14 @@ def test_limited_ttl_merges_in_empty_pool(started_cluster): def test_limited_ttl_merges_in_empty_pool_replicated(started_cluster): + node1.query("DROP TABLE IF EXISTS replicated_ttl") node1.query( "CREATE TABLE replicated_ttl (d DateTime, key UInt64, data UInt64) ENGINE = ReplicatedMergeTree('/test/t', '1') ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0") node1.query("SYSTEM STOP TTL MERGES") for i in range(100): - node1.query("INSERT INTO replicated_ttl SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(1)".format(i)) + node1.query_with_retry("INSERT INTO replicated_ttl SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(1)".format(i)) assert node1.query("SELECT COUNT() FROM replicated_ttl") == "100\n" @@ -138,6 +141,9 @@ def test_limited_ttl_merges_in_empty_pool_replicated(started_cluster): def test_limited_ttl_merges_two_replicas(started_cluster): # Actually this test quite fast and often we cannot catch any merges. # To check for sure just add some sleeps in mergePartsToTemporaryPart + node1.query("DROP TABLE IF EXISTS replicated_ttl_2") + node2.query("DROP TABLE IF EXISTS replicated_ttl_2") + node1.query( "CREATE TABLE replicated_ttl_2 (d DateTime, key UInt64, data UInt64) ENGINE = ReplicatedMergeTree('/test/t2', '1') ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0") node2.query( @@ -147,7 +153,7 @@ def test_limited_ttl_merges_two_replicas(started_cluster): node2.query("SYSTEM STOP TTL MERGES") for i in range(100): - node1.query( + node1.query_with_retry( "INSERT INTO replicated_ttl_2 SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(10000)".format(i)) node2.query("SYSTEM SYNC REPLICA replicated_ttl_2", timeout=10) diff --git a/tests/integration/test_consistent_parts_after_clone_replica/test.py b/tests/integration/test_consistent_parts_after_clone_replica/test.py index 784f94397af..35a42b6fb12 100644 --- a/tests/integration/test_consistent_parts_after_clone_replica/test.py +++ b/tests/integration/test_consistent_parts_after_clone_replica/test.py @@ -3,6 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager from helpers.test_tools import assert_eq_with_retry +import time def fill_nodes(nodes, shard): @@ -37,12 +38,17 @@ def start_cluster(): def test_inconsistent_parts_if_drop_while_replica_not_active(start_cluster): with PartitionManager() as pm: # insert into all replicas - for i in range(50): + for i in range(10): node1.query("INSERT INTO test_table VALUES ('2019-08-16', {})".format(i)) assert_eq_with_retry(node2, "SELECT count(*) FROM test_table", node1.query("SELECT count(*) FROM test_table")) - # disable network on the first replica + # partition the first replica from the second one and (later) from zk pm.partition_instances(node1, node2) + + # insert some parts on the second replica only, we will drop these parts + for i in range(10): + node2.query("INSERT INTO test_table VALUES ('2019-08-16', {})".format(10 + i)) + pm.drop_instance_zk_connections(node1) # drop all parts on the second replica @@ -51,9 +57,22 @@ def test_inconsistent_parts_if_drop_while_replica_not_active(start_cluster): # insert into the second replica # DROP_RANGE will be removed from the replication log and the first replica will be lost - for i in range(50): - node2.query("INSERT INTO test_table VALUES ('2019-08-16', {})".format(50 + i)) + 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") + + for i in range(30): + if node2.contains_in_log("Will mark replica node1 as lost"): + break + time.sleep(0.5) # the first replica will be cloned from the second pm.heal_all() assert_eq_with_retry(node1, "SELECT count(*) FROM test_table", node2.query("SELECT count(*) FROM test_table")) + + # ensure replica was cloned + assert node1.contains_in_log("Will mimic node2") + # queue must be empty (except some merges that are possibly executing right now) + assert node1.query("SELECT count() FROM system.replication_queue WHERE type != 'MERGE_PARTS'") == "0\n" + assert node2.query("SELECT count() FROM system.replication_queue WHERE type != 'MERGE_PARTS'") == "0\n" diff --git a/tests/integration/test_cross_replication/test.py b/tests/integration/test_cross_replication/test.py index 8a118934c93..cc5618e04e6 100644 --- a/tests/integration/test_cross_replication/test.py +++ b/tests/integration/test_cross_replication/test.py @@ -45,7 +45,7 @@ CREATE TABLE distributed(date Date, id UInt32, shard_id UInt32) 2017-06-16 333 2 ''' node1.query("INSERT INTO distributed FORMAT TSV", stdin=to_insert) - time.sleep(0.5) + time.sleep(5) yield cluster diff --git a/tests/integration/test_ddl_worker_non_leader/test.py b/tests/integration/test_ddl_worker_non_leader/test.py index b64f99d5345..172fc03c005 100644 --- a/tests/integration/test_ddl_worker_non_leader/test.py +++ b/tests/integration/test_ddl_worker_non_leader/test.py @@ -20,14 +20,14 @@ def started_cluster(): def test_non_leader_replica(started_cluster): - node1.query('''CREATE TABLE sometable(id UInt32, value String) + node1.query_with_retry('''CREATE TABLE IF NOT EXISTS sometable(id UInt32, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '1') ORDER BY tuple()''') - node2.query('''CREATE TABLE sometable(id UInt32, value String) + node2.query_with_retry('''CREATE TABLE IF NOT EXISTS sometable(id UInt32, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '2') ORDER BY tuple() SETTINGS replicated_can_become_leader = 0''') node1.query("INSERT INTO sometable SELECT number, toString(number) FROM numbers(100)") - node2.query("SYSTEM SYNC REPLICA sometable", timeout=10) + node2.query_with_retry("SYSTEM SYNC REPLICA sometable", timeout=10) assert node1.query("SELECT COUNT() FROM sometable") == "100\n" assert node2.query("SELECT COUNT() FROM sometable") == "100\n" diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py index ef6d133893a..a3d0e8a019b 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py @@ -1,4 +1,5 @@ import os +import shutil from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout @@ -89,34 +90,54 @@ VALUES = { SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -DICT_CONFIG_PATH = os.path.join(SCRIPT_DIR, 'configs/dictionaries') +DICT_CONFIG_PATH = os.path.join(SCRIPT_DIR, 'configs', 'dictionaries') -def get_dict(source, layout, fields, suffix_name=''): - global DICT_CONFIG_PATH - structure = DictionaryStructure(layout, fields) - dict_name = source.name + "_" + layout.name + '_' + suffix_name - dict_path = os.path.join(DICT_CONFIG_PATH, dict_name + '.xml') - dictionary = Dictionary(dict_name, structure, source, dict_path, "table_" + dict_name, fields) - dictionary.generate_config() - return dictionary +class BaseLayoutTester: + def __init__(self, test_name): + self.test_name = test_name + self.layouts = [] -class SimpleLayoutTester: - def __init__(self): - self.fields = KEY_FIELDS["simple"] + START_FIELDS["simple"] + MIDDLE_FIELDS + END_FIELDS["simple"] - self.values = VALUES["simple"] - self.data = [Row(self.fields, vals) for vals in self.values] - self.layout_to_dictionary = dict() + def get_dict_directory(self): + return os.path.join(DICT_CONFIG_PATH, self.test_name) + + def cleanup(self): + shutil.rmtree(self.get_dict_directory(), ignore_errors=True) + os.makedirs(self.get_dict_directory()) + + def list_dictionaries(self): + dictionaries = [] + directory = self.get_dict_directory() + for fname in os.listdir(directory): + dictionaries.append(os.path.join(directory, fname)) + return dictionaries def create_dictionaries(self, source_): - for layout in LAYOUTS_SIMPLE: + for layout in self.layouts: if source_.compatible_with_layout(Layout(layout)): - self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) + self.layout_to_dictionary[layout] = self.get_dict(source_, Layout(layout), self.fields) def prepare(self, cluster_): for _, dictionary in list(self.layout_to_dictionary.items()): dictionary.prepare_source(cluster_) dictionary.load_data(self.data) + def get_dict(self, source, layout, fields, suffix_name=''): + structure = DictionaryStructure(layout, fields) + dict_name = source.name + "_" + layout.name + '_' + suffix_name + dict_path = os.path.join(self.get_dict_directory(), dict_name + '.xml') + dictionary = Dictionary(dict_name, structure, source, dict_path, "table_" + dict_name, fields) + dictionary.generate_config() + return dictionary + +class SimpleLayoutTester(BaseLayoutTester): + def __init__(self, test_name): + self.fields = KEY_FIELDS["simple"] + START_FIELDS["simple"] + MIDDLE_FIELDS + END_FIELDS["simple"] + self.values = VALUES["simple"] + self.data = [Row(self.fields, vals) for vals in self.values] + self.layout_to_dictionary = dict() + self.test_name = test_name + self.layouts = LAYOUTS_SIMPLE + def execute(self, layout_name, node): if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) @@ -157,22 +178,14 @@ class SimpleLayoutTester: assert node.query(query) == str(answer) + '\n' -class ComplexLayoutTester: - def __init__(self): +class ComplexLayoutTester(BaseLayoutTester): + def __init__(self, test_name): self.fields = KEY_FIELDS["complex"] + START_FIELDS["complex"] + MIDDLE_FIELDS + END_FIELDS["complex"] self.values = VALUES["complex"] self.data = [Row(self.fields, vals) for vals in self.values] self.layout_to_dictionary = dict() - - def create_dictionaries(self, source_): - for layout in LAYOUTS_COMPLEX: - if source_.compatible_with_layout(Layout(layout)): - self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) - - def prepare(self, cluster_): - for _, dictionary in list(self.layout_to_dictionary.items()): - dictionary.prepare_source(cluster_) - dictionary.load_data(self.data) + self.test_name = test_name + self.layouts = LAYOUTS_COMPLEX def execute(self, layout_name, node): if layout_name not in self.layout_to_dictionary: @@ -200,22 +213,14 @@ class ComplexLayoutTester: assert node.query(query) == str(answer) + '\n' -class RangedLayoutTester: - def __init__(self): +class RangedLayoutTester(BaseLayoutTester): + def __init__(self, test_name): self.fields = KEY_FIELDS["ranged"] + START_FIELDS["ranged"] + MIDDLE_FIELDS + END_FIELDS["ranged"] self.values = VALUES["ranged"] self.data = [Row(self.fields, vals) for vals in self.values] self.layout_to_dictionary = dict() - - def create_dictionaries(self, source_): - for layout in LAYOUTS_RANGED: - if source_.compatible_with_layout(Layout(layout)): - self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) - - def prepare(self, cluster_): - for _, dictionary in list(self.layout_to_dictionary.items()): - dictionary.prepare_source(cluster_) - dictionary.load_data(self.data) + self.test_name = test_name + self.layouts = LAYOUTS_RANGED def execute(self, layout_name, node): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py index 81f9db1964b..65080ab40dd 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py @@ -8,14 +8,13 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceCassandra -SOURCE = SourceCassandra("Cassandra", "localhost", "9043", "cassandra1", "9042", "", "") - +SOURCE = None cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "cassandra" def setup_module(module): global cluster @@ -24,37 +23,32 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) + cluster = ClickHouseCluster(__file__, name=test_name) - simple_tester = SimpleLayoutTester() + SOURCE = SourceCassandra("Cassandra", None, cluster.cassandra_port, cluster.cassandra_host, cluster.cassandra_port, "", "") + + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) - - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) main_configs.append(os.path.join('configs', 'log_conf.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + node = cluster.add_instance('cass_node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py index 3d56746be6e..051b4ff3086 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceClickHouse -SOURCE = SourceClickHouse("LocalClickHouse", "localhost", "9000", "node", "9000", "default", "") +SOURCE = SourceClickHouse("LocalClickHouse", "localhost", "9000", "local_node", "9000", "default", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "local" def setup_module(module): global cluster @@ -24,35 +24,29 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + + dictionaries = simple_tester.list_dictionaries() + + node = cluster.add_instance('local_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + simple_tester.cleanup() @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py index 374e620e1c3..3ed335a1987 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceClickHouse -SOURCE = SourceClickHouse("RemoteClickHouse", "localhost", "9000", "clickhouse1", "9000", "default", "") +SOURCE = SourceClickHouse("RemoteClickHouse", "localhost", "9000", "clickhouse_remote", "9000", "default", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "remote" def setup_module(module): global cluster @@ -24,37 +24,31 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - cluster.add_instance('clickhouse1', main_configs=main_configs) + cluster.add_instance('clickhouse_remote', main_configs=main_configs) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + node = cluster.add_instance('remote_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + simple_tester.cleanup() @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py index 1d741d5271c..5d694bc09a2 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceExecutableCache -SOURCE = SourceExecutableCache("ExecutableCache", "localhost", "9000", "node", "9000", "", "") +SOURCE = SourceExecutableCache("ExecutableCache", "localhost", "9000", "cache_node", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "cache" def setup_module(module): global cluster @@ -24,36 +24,29 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + dictionaries = simple_tester.list_dictionaries() + + node = cluster.add_instance('cache_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py index dfcc35c54f8..8c0e6f8b878 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py @@ -8,13 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceExecutableHashed -SOURCE = SourceExecutableHashed("ExecutableHashed", "localhost", "9000", "node", "9000", "", "") +SOURCE = SourceExecutableHashed("ExecutableHashed", "localhost", "9000", "hashed_node", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None +test_name = "hashed" def setup_module(module): @@ -24,36 +25,29 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + dictionaries = simple_tester.list_dictionaries() + + node = cluster.add_instance('hashed_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py index aa81cca466b..97a06fadc5e 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceFile -SOURCE = SourceFile("File", "localhost", "9000", "node", "9000", "", "") +SOURCE = SourceFile("File", "localhost", "9000", "file_node", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "file" def setup_module(module): global cluster @@ -24,36 +24,29 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + node = cluster.add_instance('file_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py index 7c8b5a41b01..c8c73011f61 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceHTTP -SOURCE = SourceHTTP("SourceHTTP", "localhost", "9000", "clickhouse1", "9000", "", "") +SOURCE = SourceHTTP("SourceHTTP", "localhost", "9000", "clickhouse_h", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "http" def setup_module(module): global cluster @@ -24,38 +24,31 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - cluster.add_instance('clickhouse1', main_configs=main_configs) + cluster.add_instance('clickhouse_h', main_configs=main_configs) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + node = cluster.add_instance('http_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py index 44950f013b3..42f33e3da3c 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceHTTPS -SOURCE = SourceHTTPS("SourceHTTPS", "localhost", "9000", "clickhouse1", "9000", "", "") +SOURCE = SourceHTTPS("SourceHTTPS", "localhost", "9000", "clickhouse_hs", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "https" def setup_module(module): global cluster @@ -24,38 +24,31 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - cluster.add_instance('clickhouse1', main_configs=main_configs) + cluster.add_instance('clickhouse_hs', main_configs=main_configs) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + node = cluster.add_instance('https_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py index 7d808845854..deaaf044bce 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py @@ -8,14 +8,13 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceMongo -SOURCE = SourceMongo("MongoDB", "localhost", "27018", "mongo1", "27017", "root", "clickhouse") - +SOURCE = None cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "mongo" def setup_module(module): global cluster @@ -24,36 +23,30 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) + cluster = ClickHouseCluster(__file__, name=test_name) + SOURCE = SourceMongo("MongoDB", "localhost", cluster.mongo_port, cluster.mongo_host, "27017", "root", "clickhouse") - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) - - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py index 5c09627d0b9..fdf4826cb63 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py @@ -8,13 +8,13 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceMongoURI -SOURCE = SourceMongoURI("MongoDB_URI", "localhost", "27018", "mongo1", "27017", "root", "clickhouse") - +SOURCE = None cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None +test_name = "mongo_uri" def setup_module(module): @@ -24,36 +24,31 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) + cluster = ClickHouseCluster(__file__, name=test_name) - simple_tester = SimpleLayoutTester() + SOURCE = SourceMongoURI("MongoDB", "localhost", cluster.mongo_port, cluster.mongo_host, "27017", "root", "clickhouse") + + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) - - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) + node = cluster.add_instance('uri_node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py index 7a6b0b7ce8d..7feba20f3a1 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py @@ -8,13 +8,13 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceMySQL -SOURCE = SourceMySQL("MySQL", "localhost", "3308", "mysql1", "3306", "root", "clickhouse") - +SOURCE = None cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None +test_name = "mysql" def setup_module(module): @@ -24,43 +24,38 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) + cluster = ClickHouseCluster(__file__, name=test_name) - simple_tester = SimpleLayoutTester() + SOURCE = SourceMySQL("MySQL", None, cluster.mysql_port, cluster.mysql_host, cluster.mysql_port, "root", "clickhouse") + + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) - - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) main_configs.append(os.path.join('configs', 'log_conf.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - + dictionaries = simple_tester.list_dictionaries() + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): try: cluster.start() - + simple_tester.prepare(cluster) complex_tester.prepare(cluster) ranged_tester.prepare(cluster) diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml index c8fdbcbe0ef..45d9d5f8e15 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml +++ b/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml @@ -41,7 +41,9 @@ 16777216 131072 1048576 - /etc/clickhouse/dictionaries/radars + /tmp/dictionaries/radars + + 1 diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml index 46d148ad9b9..8a3d6704670 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml +++ b/tests/integration/test_dictionaries_complex_key_cache_string/configs/enable_dictionaries.xml @@ -1,5 +1,4 @@ - /etc/clickhouse-server/config.d/complex_key_cache_string.xml - /etc/clickhouse-server/config.d/ssd_complex_key_cache_string.xml + /etc/clickhouse-server/config.d/*.xml diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/test.py b/tests/integration/test_dictionaries_complex_key_cache_string/test.py index a01e60af47d..306ebcefc75 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/test.py +++ b/tests/integration/test_dictionaries_complex_key_cache_string/test.py @@ -3,30 +3,30 @@ import os import pytest from helpers.cluster import ClickHouseCluster +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +cluster = ClickHouseCluster(__file__) +node_memory = cluster.add_instance('node_memory', main_configs=['configs/enable_dictionaries.xml', + 'configs/dictionaries/complex_key_cache_string.xml']) +node_ssd = cluster.add_instance('node_ssd', main_configs=['configs/enable_dictionaries.xml', + 'configs/dictionaries/ssd_complex_key_cache_string.xml']) -@pytest.fixture(scope="function") -def cluster(request): - SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - cluster = ClickHouseCluster(__file__) +@pytest.fixture() +def started_cluster(): try: - if request.param == "memory": - node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', - 'configs/dictionaries/complex_key_cache_string.xml']) - if request.param == "ssd": - node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', - 'configs/dictionaries/ssd_complex_key_cache_string.xml']) cluster.start() - node.query( + node_memory.query( + "create table radars_table (radar_id String, radar_ip String, client_id String) engine=MergeTree() order by radar_id") + node_ssd.query( "create table radars_table (radar_id String, radar_ip String, client_id String) engine=MergeTree() order by radar_id") yield cluster finally: cluster.shutdown() - -@pytest.mark.parametrize("cluster", ["memory", "ssd"], indirect=True) -def test_memory_consumption(cluster): - node = cluster.instances['node'] +@pytest.mark.skip(reason="SSD cache test can run on disk only") +@pytest.mark.parametrize("type", ["memory", "ssd"]) +def test_memory_consumption(started_cluster, type): + node = started_cluster.instances[f'node_{type}'] node.query( "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format('w' * 8)) node.query( diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index d96d6864ba3..84374fde246 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -4,6 +4,7 @@ import time import pymysql import pytest +import logging from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster @@ -23,6 +24,7 @@ node4 = cluster.add_instance('node4', user_configs=['configs/user_admin.xml', 'c def create_mysql_conn(user, password, hostname, port): + logging.debug("Created MySQL connection user:{}, password:{}, host:{}, port{}".format(user, password, hostname, port)) return pymysql.connect( user=user, password=password, @@ -31,6 +33,7 @@ def create_mysql_conn(user, password, hostname, port): def execute_mysql_query(connection, query): + logging.debug("Execute MySQL query:{}".format(query)) with warnings.catch_warnings(): warnings.simplefilter("ignore") with connection.cursor() as cursor: @@ -57,14 +60,15 @@ def started_cluster(): @pytest.mark.parametrize("clickhouse,name,layout", [ - (node1, 'complex_node1_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'), - (node1, 'complex_node1_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), - (node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'), - (node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), + pytest.param(node1, 'complex_node1_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())', id="complex_node1_hashed"), + pytest.param(node1, 'complex_node1_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))', id="complex_node1_cache"), + pytest.param(node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())', id="complex_node2_hashed"), + pytest.param(node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))', id="complex_node2_cache"), ]) def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): - mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308) - execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse") + mysql_conn = create_mysql_conn("root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port) + execute_mysql_query(mysql_conn, "DROP DATABASE IF EXISTS clickhouse") + execute_mysql_query(mysql_conn, "CREATE DATABASE clickhouse") execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.{} (key_field1 int, key_field2 bigint, value1 text, value2 float, PRIMARY KEY (key_field1, key_field2))".format( name)) @@ -87,7 +91,7 @@ def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): DB 'clickhouse' TABLE '{}' REPLICA(PRIORITY 1 HOST '127.0.0.1' PORT 3333) - REPLICA(PRIORITY 2 HOST 'mysql1' PORT 3306) + REPLICA(PRIORITY 2 HOST 'mysql57' PORT 3306) )) {} LIFETIME(MIN 1 MAX 3) @@ -241,7 +245,7 @@ def test_file_dictionary_restrictions(started_cluster): def test_dictionary_with_where(started_cluster): - mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308) + mysql_conn = create_mysql_conn("root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port) execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse") execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.special_table (key_field1 int, value1 text, PRIMARY KEY (key_field1))") @@ -258,7 +262,7 @@ def test_dictionary_with_where(started_cluster): PASSWORD 'clickhouse' DB 'clickhouse' TABLE 'special_table' - REPLICA(PRIORITY 1 HOST 'mysql1' PORT 3306) + REPLICA(PRIORITY 1 HOST 'mysql57' PORT 3306) WHERE 'value1 = \\'qweqwe\\' OR value1 = \\'\\\\u3232\\'' )) LAYOUT(FLAT()) diff --git a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml index 514c73f3be2..2242088ebc4 100644 --- a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml +++ b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml @@ -5,7 +5,7 @@ test - mysql1 + mysql57 3306 root clickhouse @@ -41,7 +41,7 @@ test - mysql1 + mysql57 3306 root clickhouse diff --git a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml index 91506481cc9..278fad49d03 100644 --- a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml +++ b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml @@ -5,7 +5,7 @@ test - mysql1 + mysql57 3306 root clickhouse @@ -42,7 +42,7 @@ test - mysql1 + mysql57 3306 root clickhouse @@ -78,7 +78,7 @@ test - mysql1 + mysql57 3306 root clickhouse diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index 16e432c6425..c18f3d6cf70 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -2,6 +2,8 @@ import pymysql.cursors import pytest from helpers.cluster import ClickHouseCluster +import time +import logging CONFIG_FILES = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml', 'configs/remote_servers.xml'] @@ -29,7 +31,7 @@ def started_cluster(): cluster.start() # Create a MySQL database - mysql_connection = get_mysql_conn() + mysql_connection = get_mysql_conn(cluster) create_mysql_db(mysql_connection, 'test') mysql_connection.close() @@ -37,7 +39,7 @@ def started_cluster(): instance.query("CREATE DATABASE IF NOT EXISTS test") # Create database in ClickChouse using MySQL protocol (will be used for data insertion) - instance.query("CREATE DATABASE clickhouse_mysql ENGINE = MySQL('mysql1:3306', 'test', 'root', 'clickhouse')") + instance.query("CREATE DATABASE clickhouse_mysql ENGINE = MySQL('mysql57:3306', 'test', 'root', 'clickhouse')") yield cluster @@ -52,7 +54,7 @@ def test_load_mysql_dictionaries(started_cluster): for n in range(0, 5): # Create MySQL tables, fill them and create CH dict tables - prepare_mysql_table('test', str(n)) + prepare_mysql_table(started_cluster, 'test', str(n)) # Check dictionaries are loaded and have correct number of elements for n in range(0, 100): @@ -66,11 +68,12 @@ def test_load_mysql_dictionaries(started_cluster): def create_mysql_db(mysql_connection, name): with mysql_connection.cursor() as cursor: - cursor.execute("CREATE DATABASE IF NOT EXISTS {} DEFAULT CHARACTER SET 'utf8'".format(name)) + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) + cursor.execute("CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) -def prepare_mysql_table(table_name, index): - mysql_connection = get_mysql_conn() +def prepare_mysql_table(started_cluster, table_name, index): + mysql_connection = get_mysql_conn(started_cluster) # Create table create_mysql_table(mysql_connection, table_name + str(index)) @@ -86,11 +89,22 @@ def prepare_mysql_table(table_name, index): # Create CH Dictionary tables based on MySQL tables query(create_clickhouse_dictionary_table_template.format(table_name + str(index), 'dict' + str(index))) - -def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.10', port=3308) - return conn - +def get_mysql_conn(started_cluster): + errors = [] + conn = None + for _ in range(5): + try: + if conn is None: + conn = pymysql.connect(user='root', password='clickhouse', host=started_cluster.mysql_ip, port=started_cluster.mysql_port) + else: + conn.ping(reconnect=True) + logging.debug(f"MySQL Connection establised: {started_cluster.mysql_ip}:{started_cluster.mysql_port}") + return conn + except Exception as e: + errors += [str(e)] + time.sleep(1) + + raise Exception("Connection not establised, {}".format(errors)) def create_mysql_table(conn, table_name): with conn.cursor() as cursor: diff --git a/tests/integration/test_dictionaries_postgresql/test.py b/tests/integration/test_dictionaries_postgresql/test.py index 2f7c0594438..f0d18909166 100644 --- a/tests/integration/test_dictionaries_postgresql/test.py +++ b/tests/integration/test_dictionaries_postgresql/test.py @@ -22,11 +22,11 @@ click_dict_table_template = """ ) ENGINE = Dictionary({}) """ -def get_postgres_conn(port=5432, database=False): +def get_postgres_conn(ip, port, database=False): if database == True: - conn_string = "host='localhost' port={} dbname='clickhouse' user='postgres' password='mysecretpassword'".format(port) + conn_string = "host={} port={} dbname='clickhouse' user='postgres' password='mysecretpassword'".format(ip, port) else: - conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(port) + conn_string = "host={} port={} user='postgres' password='mysecretpassword'".format(ip, port) conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) @@ -40,7 +40,7 @@ def create_postgres_db(conn, name): def create_postgres_table(cursor, table_name): cursor.execute(postgres_dict_table_template.format(table_name)) -def create_and_fill_postgres_table(cursor, table_name, host='postgres1', port=5432): +def create_and_fill_postgres_table(cursor, table_name, port, host): create_postgres_table(cursor, table_name) # Fill postgres table using clickhouse postgres table function and check table_func = '''postgresql('{}:{}', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(host, port, table_name) @@ -59,11 +59,11 @@ def started_cluster(): cluster.start() node1.query("CREATE DATABASE IF NOT EXISTS test") - postgres_conn = get_postgres_conn(port=5432) + postgres_conn = get_postgres_conn(ip=cluster.postgres_ip, port=cluster.postgres_port) print("postgres1 connected") create_postgres_db(postgres_conn, 'clickhouse') - postgres_conn = get_postgres_conn(port=5421) + postgres_conn = get_postgres_conn(ip=cluster.postgres2_ip, port=cluster.postgres_port) print("postgres2 connected") create_postgres_db(postgres_conn, 'clickhouse') @@ -74,10 +74,10 @@ def started_cluster(): def test_load_dictionaries(started_cluster): - conn = get_postgres_conn(database=True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, database=True, port=started_cluster.postgres_port) cursor = conn.cursor() table_name = 'test0' - create_and_fill_postgres_table(cursor, table_name) + create_and_fill_postgres_table(cursor, table_name, port=started_cluster.postgres_port, host=started_cluster.postgres_ip) create_dict(table_name) dict_name = 'dict0' @@ -92,10 +92,10 @@ def test_load_dictionaries(started_cluster): def test_invalidate_query(started_cluster): - conn = get_postgres_conn(database=True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, database=True, port=started_cluster.postgres_port) cursor = conn.cursor() table_name = 'test0' - create_and_fill_postgres_table(cursor, table_name) + create_and_fill_postgres_table(cursor, table_name, port=started_cluster.postgres_port, host=started_cluster.postgres_ip) # invalidate query: SELECT value FROM test0 WHERE id = 0 dict_name = 'dict0' @@ -130,9 +130,9 @@ def test_invalidate_query(started_cluster): def test_dictionary_with_replicas(started_cluster): - conn1 = get_postgres_conn(port=5432, database=True) + conn1 = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) cursor1 = conn1.cursor() - conn2 = get_postgres_conn(port=5421, database=True) + conn2 = get_postgres_conn(ip=started_cluster.postgres2_ip, port=started_cluster.postgres_port, database=True) cursor2 = conn2.cursor() create_postgres_table(cursor1, 'test1') @@ -160,7 +160,7 @@ def test_dictionary_with_replicas(started_cluster): def test_postgres_scema(started_cluster): - conn = get_postgres_conn(port=5432, database=True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) cursor = conn.cursor() cursor.execute('CREATE SCHEMA test_schema') diff --git a/tests/integration/test_dictionaries_redis/configs/.gitignore b/tests/integration/test_dictionaries_redis/configs/.gitignore new file mode 100644 index 00000000000..12657916b22 --- /dev/null +++ b/tests/integration/test_dictionaries_redis/configs/.gitignore @@ -0,0 +1,2 @@ +dictionaries/* +!.gitignore diff --git a/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitkeep b/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitkeep deleted file mode 100644 index c693f138c81..00000000000 --- a/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitkeep +++ /dev/null @@ -1 +0,0 @@ -keep \ No newline at end of file diff --git a/tests/integration/test_dictionaries_redis/test.py b/tests/integration/test_dictionaries_redis/test.py index d7d7e0ee3ad..e5a51bcb88a 100644 --- a/tests/integration/test_dictionaries_redis/test.py +++ b/tests/integration/test_dictionaries_redis/test.py @@ -1,5 +1,5 @@ import os - +import shutil import pytest from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout @@ -93,15 +93,18 @@ def setup_module(module): global node global dict_configs_path - for f in os.listdir(dict_configs_path): - os.remove(os.path.join(dict_configs_path, f)) + cluster = ClickHouseCluster(__file__) + + if os.path.exists(dict_configs_path): + shutil.rmtree(dict_configs_path) + os.mkdir(dict_configs_path) for i, field in enumerate(FIELDS): DICTIONARIES.append([]) sources = [] - sources.append(SourceRedis("RedisSimple", "localhost", "6380", "redis1", "6379", "", "clickhouse", i * 2, + sources.append(SourceRedis("RedisSimple", "localhost", cluster.redis_port, cluster.redis_host, "6379", "", "clickhouse", i * 2, storage_type="simple")) - sources.append(SourceRedis("RedisHash", "localhost", "6380", "redis1", "6379", "", "clickhouse", i * 2 + 1, + sources.append(SourceRedis("RedisHash", "localhost", cluster.redis_port, cluster.redis_host, "6379", "", "clickhouse", i * 2 + 1, storage_type="hash_map")) for source in sources: for layout in LAYOUTS: @@ -117,7 +120,6 @@ def setup_module(module): for fname in os.listdir(dict_configs_path): dictionaries.append(os.path.join(dict_configs_path, fname)) - cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_redis=True) diff --git a/tests/integration/test_dictionaries_update_and_reload/test.py b/tests/integration/test_dictionaries_update_and_reload/test.py index 533a29dc245..68d9767a78c 100644 --- a/tests/integration/test_dictionaries_update_and_reload/test.py +++ b/tests/integration/test_dictionaries_update_and_reload/test.py @@ -60,7 +60,7 @@ def get_loading_duration(dictionary_name): def replace_in_file_in_container(file_name, what, replace_with): - instance.exec_in_container('sed -i "s/' + what + '/' + replace_with + '/g" ' + file_name) + instance.exec_in_container(['sed', '-i', f's/{what}/{replace_with}/g', file_name]) def test_reload_while_loading(started_cluster): @@ -180,7 +180,7 @@ def test_reload_after_fail_by_system_reload(started_cluster): assert get_status("no_file") == "LOADED" # Removing the file source should not spoil the loaded dictionary. - instance.exec_in_container("rm /etc/clickhouse-server/config.d/no_file.txt") + instance.exec_in_container(["rm", "/etc/clickhouse-server/config.d/no_file.txt"]) assert no_such_file_error in instance.query_and_get_error("SYSTEM RELOAD DICTIONARY 'no_file'") query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n" assert get_status("no_file") == "LOADED" @@ -206,13 +206,13 @@ def test_reload_after_fail_by_timer(started_cluster): # Check that file appears in container and wait if needed. while not instance.file_exists("/etc/clickhouse-server/config.d/no_file_2.txt"): time.sleep(1) - assert("9\t10\n" == instance.exec_in_container("cat /etc/clickhouse-server/config.d/no_file_2.txt")) + assert("9\t10\n" == instance.exec_in_container(["cat", "/etc/clickhouse-server/config.d/no_file_2.txt"])) instance.query("SYSTEM RELOAD DICTIONARY no_file_2") instance.query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n" assert get_status("no_file_2") == "LOADED" # Removing the file source should not spoil the loaded dictionary. - instance.exec_in_container("rm /etc/clickhouse-server/config.d/no_file_2.txt") + instance.exec_in_container(["rm", "/etc/clickhouse-server/config.d/no_file_2.txt"]) time.sleep(6); instance.query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n" assert get_status("no_file_2") == "LOADED" diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py index 0c801ce3f12..46820fb4a89 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py @@ -7,7 +7,7 @@ from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="reading") dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py index caabdf12c66..12cde30d8ed 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py @@ -9,8 +9,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="string") dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py index e794ffa5a37..789d4aeb8b3 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py @@ -7,7 +7,7 @@ from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="default") dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', diff --git a/tests/integration/test_disabled_mysql_server/test.py b/tests/integration/test_disabled_mysql_server/test.py index a2cbcb17534..d7977404c73 100644 --- a/tests/integration/test_disabled_mysql_server/test.py +++ b/tests/integration/test_disabled_mysql_server/test.py @@ -22,10 +22,10 @@ def started_cluster(): class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=3308): + def __init__(self, started_cluster, user='root', password='clickhouse'): self.user = user - self.port = port - self.hostname = hostname + self.port = cluster.mysql_port + self.hostname = cluster.mysql_ip self.password = password self.mysql_connection = None # lazy init @@ -45,16 +45,17 @@ class MySQLNodeInstance: def test_disabled_mysql_server(started_cluster): - with contextlib.closing(MySQLNodeInstance()) as mysql_node: - mysql_node.query("CREATE DATABASE test_db;") - mysql_node.query("CREATE TABLE test_db.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") + with contextlib.closing(MySQLNodeInstance(started_cluster)) as mysql_node: + mysql_node.query("DROP DATABASE IF EXISTS test_db_disabled;") + mysql_node.query("CREATE DATABASE test_db_disabled;") + mysql_node.query("CREATE TABLE test_db_disabled.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") with PartitionManager() as pm: - clickhouse_node.query("CREATE DATABASE test_db ENGINE = MySQL('mysql1:3306', 'test_db', 'root', 'clickhouse')") + clickhouse_node.query("CREATE DATABASE test_db_disabled ENGINE = MySQL('mysql57:3306', 'test_db_disabled', 'root', 'clickhouse')") pm._add_rule({'source': clickhouse_node.ip_address, 'destination_port': 3306, 'action': 'DROP'}) clickhouse_node.query("SELECT * FROM system.parts") clickhouse_node.query("SELECT * FROM system.mutations") clickhouse_node.query("SELECT * FROM system.graphite_retentions") - clickhouse_node.query("DROP DATABASE test_db") + clickhouse_node.query("DROP DATABASE test_db_disabled") diff --git a/tests/integration/test_distributed_ddl/cluster.py b/tests/integration/test_distributed_ddl/cluster.py index 24f11fec547..f90d15ebd08 100644 --- a/tests/integration/test_distributed_ddl/cluster.py +++ b/tests/integration/test_distributed_ddl/cluster.py @@ -8,7 +8,6 @@ from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager from helpers.test_tools import TSV - class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): def __init__(self, base_path, config_dir, testcase_name): ClickHouseCluster.__init__(self, base_path, name=testcase_name) @@ -83,7 +82,7 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): assert codes[0] == "0", "\n" + tsv_content def ddl_check_query(self, instance, query, num_hosts=None, settings=None): - contents = instance.query(query, settings=settings) + contents = instance.query_with_retry(query, settings=settings) self.check_all_hosts_successfully_executed(contents, num_hosts) return contents diff --git a/tests/integration/test_distributed_ddl/configs/config.d/zookeeper_session_timeout.xml b/tests/integration/test_distributed_ddl/configs/config.d/zookeeper_session_timeout.xml index 071725b5391..3b878bbf4de 100644 --- a/tests/integration/test_distributed_ddl/configs/config.d/zookeeper_session_timeout.xml +++ b/tests/integration/test_distributed_ddl/configs/config.d/zookeeper_session_timeout.xml @@ -1,6 +1,6 @@ - 3000 + 10000 diff --git a/tests/integration/test_distributed_ddl/configs_secure/config.d/zookeeper_session_timeout.xml b/tests/integration/test_distributed_ddl/configs_secure/config.d/zookeeper_session_timeout.xml index 071725b5391..3b878bbf4de 100644 --- a/tests/integration/test_distributed_ddl/configs_secure/config.d/zookeeper_session_timeout.xml +++ b/tests/integration/test_distributed_ddl/configs_secure/config.d/zookeeper_session_timeout.xml @@ -1,6 +1,6 @@ - 3000 + 10000 diff --git a/tests/integration/test_distributed_ddl/test.py b/tests/integration/test_distributed_ddl/test.py index 68d8a2dab53..87e793a7acb 100755 --- a/tests/integration/test_distributed_ddl/test.py +++ b/tests/integration/test_distributed_ddl/test.py @@ -98,7 +98,7 @@ def _test_on_connection_losses(test_cluster, zk_timeout): with PartitionManager() as pm: pm.drop_instance_zk_connections(kill_instance) - request = instance.get_query_request("DROP TABLE IF EXISTS test.__nope__ ON CLUSTER 'cluster'", timeout=10) + request = instance.get_query_request("DROP TABLE IF EXISTS test.__nope__ ON CLUSTER 'cluster'", timeout=20) time.sleep(zk_timeout) pm.restore_instance_zk_connections(kill_instance) @@ -106,11 +106,11 @@ def _test_on_connection_losses(test_cluster, zk_timeout): def test_on_connection_loss(test_cluster): - _test_on_connection_losses(test_cluster, 1.5) # connection loss will occur only (3 sec ZK timeout in config) + _test_on_connection_losses(test_cluster, 5) # connection loss will occur only (3 sec ZK timeout in config) def test_on_session_expired(test_cluster): - _test_on_connection_losses(test_cluster, 4) # session should be expired (3 sec ZK timeout in config) + _test_on_connection_losses(test_cluster, 15) # session should be expired (3 sec ZK timeout in config) def test_simple_alters(test_cluster): diff --git a/tests/integration/test_distributed_over_distributed/test.py b/tests/integration/test_distributed_over_distributed/test.py index 410a03a6af1..ae86a70f31b 100644 --- a/tests/integration/test_distributed_over_distributed/test.py +++ b/tests/integration/test_distributed_over_distributed/test.py @@ -2,7 +2,6 @@ # (just in case, with real separate instances). - import pytest from helpers.cluster import ClickHouseCluster @@ -51,9 +50,13 @@ def started_cluster(): cluster.shutdown() -@pytest.mark.parametrize("node", list(NODES.values())) -@pytest.mark.parametrize("source", - ["distributed_over_distributed_table", "cluster('test_cluster', default, distributed_table)"]) +@pytest.mark.parametrize("node,source", [ + pytest.param(NODES["node1"], "distributed_over_distributed_table", id="dod_node1"), + pytest.param(NODES["node1"], "cluster('test_cluster', default, distributed_table)", id="cluster_node1"), + pytest.param(NODES["node2"], "distributed_over_distributed_table", id="dod_node2"), + pytest.param(NODES["node2"], "cluster('test_cluster', default, distributed_table)", id="cluster_node2"), +] +) class TestDistributedOverDistributedSuite: def test_select_with_order_by_node(self, started_cluster, node, source): assert node.query("SELECT * FROM {source} ORDER BY node, key".format(source=source)) \ diff --git a/tests/integration/test_distributed_storage_configuration/test.py b/tests/integration/test_distributed_storage_configuration/test.py index 976fc5211a7..94beb7b57ca 100644 --- a/tests/integration/test_distributed_storage_configuration/test.py +++ b/tests/integration/test_distributed_storage_configuration/test.py @@ -17,7 +17,7 @@ node = cluster.add_instance('node', def start_cluster(): try: cluster.start() - node.query('CREATE DATABASE test ENGINE=Ordinary') # Different paths with Atomic + node.query('CREATE DATABASE IF NOT EXISTS test ENGINE=Ordinary') # Different paths with Atomic yield cluster finally: cluster.shutdown() diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index cc006801735..23e2d8d8c47 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -3,15 +3,12 @@ import logging import avro.schema import pytest +from confluent_kafka.avro.cached_schema_registry_client import CachedSchemaRegistryClient from confluent_kafka.avro.serializer.message_serializer import MessageSerializer from helpers.cluster import ClickHouseCluster, ClickHouseInstance -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - - @pytest.fixture(scope="module") -def cluster(): +def started_cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance("dummy", with_kafka=True) @@ -37,10 +34,10 @@ def run_query(instance, query, data=None, settings=None): return result -def test_select(cluster): +def test_select(started_cluster): # type: (ClickHouseCluster) -> None - schema_registry_client = cluster.schema_registry_client + schema_registry_client = CachedSchemaRegistryClient('http://localhost:{}'.format(started_cluster.schema_registry_port)) serializer = MessageSerializer(schema_registry_client) schema = avro.schema.make_avsc_object({ @@ -62,10 +59,10 @@ def test_select(cluster): buf.write(message) data = buf.getvalue() - instance = cluster.instances["dummy"] # type: ClickHouseInstance + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}".format( - cluster.schema_registry_host, - cluster.schema_registry_port + started_cluster.schema_registry_host, + 8081 ) run_query(instance, "create table avro_data(value Int64) engine = Memory()") diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index d8604276281..ee7e94bad1d 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -133,7 +133,7 @@ def start_cluster(): @pytest.fixture(autouse=True) def reset_after_test(): yield - query("DROP TABLE IF EXISTS t") + node.query_with_retry("DROP TABLE IF EXISTS t") # Actual tests diff --git a/tests/integration/test_insert_into_distributed/test.py b/tests/integration/test_insert_into_distributed/test.py index 47200df4a85..e2af59903bd 100644 --- a/tests/integration/test_insert_into_distributed/test.py +++ b/tests/integration/test_insert_into_distributed/test.py @@ -121,7 +121,7 @@ def test_reconnect(started_cluster): pm.heal_all() time.sleep(1) instance.query("INSERT INTO distributed VALUES (3)") - time.sleep(1) + time.sleep(5) assert remote.query("SELECT count(*) FROM local1").strip() == '3' diff --git a/tests/integration/test_join_set_family_s3/test.py b/tests/integration/test_join_set_family_s3/test.py index 2a20c8eade9..625cac48dee 100644 --- a/tests/integration/test_join_set_family_s3/test.py +++ b/tests/integration/test_join_set_family_s3/test.py @@ -4,9 +4,6 @@ import sys import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_log_family_hdfs/test.py b/tests/integration/test_log_family_hdfs/test.py index 44f6904e8ea..3d067f0febe 100644 --- a/tests/integration/test_log_family_hdfs/test.py +++ b/tests/integration/test_log_family_hdfs/test.py @@ -8,7 +8,7 @@ from pyhdfs import HdfsClient @pytest.fixture(scope="module") -def cluster(): +def started_cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance("node", @@ -18,7 +18,7 @@ def cluster(): cluster.start() logging.info("Cluster started") - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) fs.mkdirs('/clickhouse') yield cluster @@ -26,8 +26,8 @@ def cluster(): cluster.shutdown() -def assert_objects_count(cluster, objects_count, path='data/'): - fs = HdfsClient(hosts='localhost') +def assert_objects_count(started_cluster, objects_count, path='data/'): + fs = HdfsClient(hosts=started_cluster.hdfs_ip) hdfs_objects = fs.listdir('/clickhouse') assert objects_count == len(hdfs_objects) @@ -35,25 +35,25 @@ def assert_objects_count(cluster, objects_count, path='data/'): @pytest.mark.parametrize( "log_engine,files_overhead,files_overhead_per_insert", [("TinyLog", 1, 1), ("Log", 2, 1), ("StripeLog", 1, 2)]) -def test_log_family_hdfs(cluster, log_engine, files_overhead, files_overhead_per_insert): - node = cluster.instances["node"] +def test_log_family_hdfs(started_cluster, log_engine, files_overhead, files_overhead_per_insert): + node = started_cluster.instances["node"] node.query("CREATE TABLE hdfs_test (id UInt64) ENGINE={} SETTINGS disk = 'hdfs'".format(log_engine)) node.query("INSERT INTO hdfs_test SELECT number FROM numbers(5)") assert node.query("SELECT * FROM hdfs_test") == "0\n1\n2\n3\n4\n" - assert_objects_count(cluster, files_overhead_per_insert + files_overhead) + assert_objects_count(started_cluster, files_overhead_per_insert + files_overhead) node.query("INSERT INTO hdfs_test SELECT number + 5 FROM numbers(3)") assert node.query("SELECT * FROM hdfs_test order by id") == "0\n1\n2\n3\n4\n5\n6\n7\n" - assert_objects_count(cluster, files_overhead_per_insert * 2 + files_overhead) + assert_objects_count(started_cluster, files_overhead_per_insert * 2 + files_overhead) node.query("INSERT INTO hdfs_test SELECT number + 8 FROM numbers(1)") assert node.query("SELECT * FROM hdfs_test order by id") == "0\n1\n2\n3\n4\n5\n6\n7\n8\n" - assert_objects_count(cluster, files_overhead_per_insert * 3 + files_overhead) + assert_objects_count(started_cluster, files_overhead_per_insert * 3 + files_overhead) node.query("TRUNCATE TABLE hdfs_test") - assert_objects_count(cluster, 0) + assert_objects_count(started_cluster, 0) node.query("DROP TABLE hdfs_test") diff --git a/tests/integration/test_log_family_s3/test.py b/tests/integration/test_log_family_s3/test.py index c23e7545b27..115ec47871b 100644 --- a/tests/integration/test_log_family_s3/test.py +++ b/tests/integration/test_log_family_s3/test.py @@ -4,9 +4,6 @@ import sys import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_match_process_uid_against_data_owner/test.py b/tests/integration/test_match_process_uid_against_data_owner/test.py index c9f87e683c8..754ebc731c1 100644 --- a/tests/integration/test_match_process_uid_against_data_owner/test.py +++ b/tests/integration/test_match_process_uid_against_data_owner/test.py @@ -20,7 +20,7 @@ def test_different_user(): cluster.start() - docker_api = docker.from_env().api + docker_api = cluster.docker_client.api container = node.get_docker_handle() container.stop() container.start() @@ -31,7 +31,8 @@ def test_different_user(): with open(os.path.join(node.path, 'logs/clickhouse-server.err.log')) as log: expected_message = "Effective user of the process \(.*\) does not match the owner of the data \(.*\)\. Run under 'sudo -u .*'\." - last_message = log.readlines()[-1].strip() + + last_message = [row for row in log.readlines() if "Effective" in row][-1] if re.search(expected_message, last_message) is None: pytest.fail( diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 813a654add3..4091de45ac2 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -3,7 +3,7 @@ import time import pymysql.cursors import pytest from helpers.network import PartitionManager -import pytest +import logging from helpers.client import QueryRuntimeException from helpers.cluster import get_docker_compose_path, run_and_check import random @@ -12,7 +12,7 @@ import threading from multiprocessing.dummy import Pool from helpers.test_tools import assert_eq_with_retry -def check_query(clickhouse_node, query, result_set, retry_count=60, interval_seconds=3): +def check_query(clickhouse_node, query, result_set, retry_count=10, interval_seconds=3): lastest_result = '' for i in range(retry_count): @@ -21,22 +21,22 @@ def check_query(clickhouse_node, query, result_set, retry_count=60, interval_sec if result_set == lastest_result: return - print(lastest_result) + logging.debug(f"latest_result{lastest_result}") time.sleep(interval_seconds) except Exception as e: - print(("check_query retry {} exception {}".format(i + 1, e))) + logging.debug(f"check_query retry {i+1} exception {e}") time.sleep(interval_seconds) else: assert clickhouse_node.query(query) == result_set def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_dml") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_dml") + mysql_node.query("CREATE DATABASE test_database_dml DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created - mysql_node.query("CREATE TABLE test_database.test_table_1 (" + mysql_node.query("CREATE TABLE test_database_dml.test_table_1 (" "`key` INT NOT NULL PRIMARY KEY, " "unsigned_tiny_int TINYINT UNSIGNED, tiny_int TINYINT, " "unsigned_small_int SMALLINT UNSIGNED, small_int SMALLINT, " @@ -53,68 +53,68 @@ def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_nam # it already has some data mysql_node.query(""" - INSERT INTO test_database.test_table_1 VALUES(1, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', 'binary', + INSERT INTO test_database_dml.test_table_1 VALUES(1, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', 'binary', '2020-01-01', '2020-01-01 00:00:00', '2020-01-01 00:00:00', true); """) clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_dml ENGINE = MaterializeMySQL('{}:3306', 'test_database_dml', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") + assert "test_database_dml" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT * FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t" "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n") mysql_node.query(""" - INSERT INTO test_database.test_table_1 VALUES(2, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', 'binary', + INSERT INTO test_database_dml.test_table_1 VALUES(2, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', 'binary', '2020-01-01', '2020-01-01 00:00:00', '2020-01-01 00:00:00', false); """) - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT * FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t" "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" "varchar\tchar\tbinary\\0\\0\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\t0\n") - mysql_node.query("UPDATE test_database.test_table_1 SET unsigned_tiny_int = 2 WHERE `key` = 1") + mysql_node.query("UPDATE test_database_dml.test_table_1 SET unsigned_tiny_int = 2 WHERE `key` = 1") check_query(clickhouse_node, """ SELECT key, unsigned_tiny_int, tiny_int, unsigned_small_int, small_int, unsigned_medium_int, medium_int, unsigned_int, _int, unsigned_integer, _integer, unsigned_bigint, _bigint, unsigned_float, _float, unsigned_double, _double, _varchar, _char, binary_col, _date, _datetime, /* exclude it, because ON UPDATE CURRENT_TIMESTAMP _timestamp, */ - _bool FROM test_database.test_table_1 ORDER BY key FORMAT TSV + _bool FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV """, "1\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t" "2020-01-01 00:00:00\t1\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" "varchar\tchar\tbinary\\0\\0\t2020-01-01\t2020-01-01 00:00:00\t0\n") # update primary key - mysql_node.query("UPDATE test_database.test_table_1 SET `key` = 3 WHERE `unsigned_tiny_int` = 2") + mysql_node.query("UPDATE test_database_dml.test_table_1 SET `key` = 3 WHERE `unsigned_tiny_int` = 2") check_query(clickhouse_node, "SELECT key, unsigned_tiny_int, tiny_int, unsigned_small_int," " small_int, unsigned_medium_int, medium_int, unsigned_int, _int, unsigned_integer, _integer, " " unsigned_bigint, _bigint, unsigned_float, _float, unsigned_double, _double, _varchar, _char, binary_col, " " _date, _datetime, /* exclude it, because ON UPDATE CURRENT_TIMESTAMP _timestamp, */ " - " _bool FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + " _bool FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" "varchar\tchar\tbinary\\0\\0\t2020-01-01\t2020-01-01 00:00:00\t0\n3\t2\t-1\t2\t-2\t3\t-3\t" "4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t2020-01-01 00:00:00\t1\n") - mysql_node.query('DELETE FROM test_database.test_table_1 WHERE `key` = 2') + mysql_node.query('DELETE FROM test_database_dml.test_table_1 WHERE `key` = 2') check_query(clickhouse_node, "SELECT key, unsigned_tiny_int, tiny_int, unsigned_small_int," " small_int, unsigned_medium_int, medium_int, unsigned_int, _int, unsigned_integer, _integer, " " unsigned_bigint, _bigint, unsigned_float, _float, unsigned_double, _double, _varchar, _char, binary_col, " " _date, _datetime, /* exclude it, because ON UPDATE CURRENT_TIMESTAMP _timestamp, */ " - " _bool FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + " _bool FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "3\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t" "2020-01-01 00:00:00\t1\n") - mysql_node.query('DELETE FROM test_database.test_table_1 WHERE `unsigned_tiny_int` = 2') - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", "") + mysql_node.query('DELETE FROM test_database_dml.test_table_1 WHERE `unsigned_tiny_int` = 2') + check_query(clickhouse_node, "SELECT * FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_dml") + mysql_node.query("DROP DATABASE test_database_dml") def materialize_mysql_database_with_views(clickhouse_node, mysql_node, service_name): @@ -157,275 +157,275 @@ def materialize_mysql_database_with_views(clickhouse_node, mysql_node, service_n def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") + mysql_node.query("DROP DATABASE IF EXISTS test_database_dt") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_dt") + mysql_node.query("CREATE DATABASE test_database_dt DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database_dt.test_table_1 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database_dt.test_table_1 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database_dt.test_table_1 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") + mysql_node.query("INSERT INTO test_database_dt.test_table_1 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database_dt.test_table_1 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") + clickhouse_node.query("CREATE DATABASE test_database_dt ENGINE = MaterializeMySQL('{}:3306', 'test_database_dt', 'root', 'clickhouse')".format(service_name)) + assert "test_database_dt" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT * FROM test_database_dt.test_table_1 ORDER BY key FORMAT TSV", "1\t2020-01-01 01:02:03.999999\t2020-01-01 01:02:03.999\t" + ('9' * 35) + "." + ('9' * 30) + "\n" "2\t2020-01-01 01:02:03.000000\t2020-01-01 01:02:03.000\t0." + ('0' * 29) + "1\n" "3\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:03.990\t-" + ('9' * 35) + "." + ('9' * 30) + "\n" "4\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:04.000\t-0." + ('0' * 29) + "1\n") - mysql_node.query("CREATE TABLE test_database.test_table_2 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") + mysql_node.query("CREATE TABLE test_database_dt.test_table_2 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database_dt.test_table_2 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database_dt.test_table_2 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") + mysql_node.query("INSERT INTO test_database_dt.test_table_2 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database_dt.test_table_2 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT * FROM test_database_dt.test_table_2 ORDER BY key FORMAT TSV", "1\t2020-01-01 01:02:03.999999\t2020-01-01 01:02:03.999\t" + ('9' * 35) + "." + ('9' * 30) + "\n" "2\t2020-01-01 01:02:03.000000\t2020-01-01 01:02:03.000\t0." + ('0' * 29) + "1\n" "3\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:03.990\t-" + ('9' * 35) + "." + ('9' * 30) + "\n" "4\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:04.000\t-0." + ('0' * 29) + "1\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") - + clickhouse_node.query("DROP DATABASE test_database_dt") + mysql_node.query("DROP DATABASE test_database_dt") def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("DROP DATABASE IF EXISTS test_database_drop") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_drop") + mysql_node.query("CREATE DATABASE test_database_drop DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database_drop.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("DROP TABLE test_database.test_table_1;") + mysql_node.query("DROP TABLE test_database_drop.test_table_1;") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("CREATE TABLE test_database_drop.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") + mysql_node.query("TRUNCATE TABLE test_database_drop.test_table_2;") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_drop ENGINE = MaterializeMySQL('{}:3306', 'test_database_drop', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + assert "test_database_drop" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SELECT * FROM test_database_drop.test_table_2 ORDER BY id FORMAT TSV", "") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5), (6)") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", + mysql_node.query("INSERT INTO test_database_drop.test_table_2 VALUES(1), (2), (3), (4), (5), (6)") + mysql_node.query("CREATE TABLE test_database_drop.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_drop FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "SELECT * FROM test_database_drop.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6\n") - mysql_node.query("DROP TABLE test_database.test_table_1;") - mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2\n") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + mysql_node.query("DROP TABLE test_database_drop.test_table_1;") + mysql_node.query("TRUNCATE TABLE test_database_drop.test_table_2;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_drop FORMAT TSV", "test_table_2\n") + check_query(clickhouse_node, "SELECT * FROM test_database_drop.test_table_2 ORDER BY id FORMAT TSV", "") + + clickhouse_node.query("DROP DATABASE test_database_drop") + mysql_node.query("DROP DATABASE test_database_drop") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_create") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_create") + mysql_node.query("CREATE DATABASE test_database_create DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("CREATE TABLE test_database_create.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") # it already has some data - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (5), (6), (7);") + mysql_node.query("INSERT INTO test_database_create.test_table_1 VALUES(1), (2), (3), (5), (6), (7);") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_create ENGINE = MaterializeMySQL('{}:3306', 'test_database_create', 'root', 'clickhouse')".format( service_name)) # Check for pre-existing status - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY id FORMAT TSV", + assert "test_database_create" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SELECT * FROM test_database_create.test_table_1 ORDER BY id FORMAT TSV", "1\n2\n3\n5\n6\n7\n") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5), (6);") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", + mysql_node.query("CREATE TABLE test_database_create.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database_create.test_table_2 VALUES(1), (2), (3), (4), (5), (6);") + check_query(clickhouse_node, "SELECT * FROM test_database_create.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_create") + mysql_node.query("DROP DATABASE test_database_create") def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("DROP DATABASE IF EXISTS test_database_rename") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_rename") + mysql_node.query("CREATE DATABASE test_database_rename DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database_rename.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("RENAME TABLE test_database.test_table_1 TO test_database.test_table_2") + mysql_node.query("RENAME TABLE test_database_rename.test_table_1 TO test_database_rename.test_table_2") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_rename ENGINE = MaterializeMySQL('{}:3306', 'test_database_rename', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2\n") - mysql_node.query("RENAME TABLE test_database.test_table_2 TO test_database.test_table_1") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") + assert "test_database_rename" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename FORMAT TSV", "test_table_2\n") + mysql_node.query("RENAME TABLE test_database_rename.test_table_2 TO test_database_rename.test_table_1") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename FORMAT TSV", "test_table_1\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_rename") + mysql_node.query("DROP DATABASE test_database_rename") def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("DROP DATABASE IF EXISTS test_database_add") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_add") + mysql_node.query("CREATE DATABASE test_database_add DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database_add.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( - "0" if service_name == "mysql1" else "(id)")) + mysql_node.query("ALTER TABLE test_database_add.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") + mysql_node.query("ALTER TABLE test_database_add.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") + mysql_node.query("ALTER TABLE test_database_add.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") + mysql_node.query("ALTER TABLE test_database_add.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( + "0" if service_name == "mysql57" else "(id)")) # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_add ENGINE = MaterializeMySQL('{}:3306', 'test_database_add', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + assert "test_database_add" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "DESC test_database_add.test_table_1 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("CREATE TABLE test_database_add.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_add FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database_add.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") + "ALTER TABLE test_database_add.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query( - "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( - "0" if service_name == "mysql1" else "(id)")) + "ALTER TABLE test_database_add.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( + "0" if service_name == "mysql57" else "(id)")) - default_expression = "DEFAULT\t0" if service_name == "mysql1" else "DEFAULT\tid" - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + default_expression = "DEFAULT\t0" if service_name == "mysql57" else "DEFAULT\tid" + check_query(clickhouse_node, "DESC test_database_add.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t" + default_expression + "\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2, 3, 4, 5), (6, 7, 8, 9, 10)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", + mysql_node.query("INSERT INTO test_database_add.test_table_2 VALUES(1, 2, 3, 4, 5), (6, 7, 8, 9, 10)") + check_query(clickhouse_node, "SELECT * FROM test_database_add.test_table_2 ORDER BY id FORMAT TSV", "1\t2\t3\t4\t5\n6\t7\t8\t9\t10\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_add") + mysql_node.query("DROP DATABASE test_database_add") def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_alter_drop") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_alter_drop") + mysql_node.query("CREATE DATABASE test_database_alter_drop DEFAULT CHARACTER SET 'utf8'") mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") + "CREATE TABLE test_database_alter_drop.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column") + mysql_node.query("ALTER TABLE test_database_alter_drop.test_table_1 DROP COLUMN drop_column") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_alter_drop ENGINE = MaterializeMySQL('{}:3306', 'test_database_alter_drop', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + assert "test_database_alter_drop" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_alter_drop FORMAT TSV", "test_table_1\n") + check_query(clickhouse_node, "DESC test_database_alter_drop.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + "CREATE TABLE test_database_alter_drop.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_alter_drop FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database_alter_drop.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 DROP COLUMN drop_column") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_drop.test_table_2 DROP COLUMN drop_column") + check_query(clickhouse_node, "DESC test_database_alter_drop.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") + mysql_node.query("INSERT INTO test_database_alter_drop.test_table_2 VALUES(1), (2), (3), (4), (5)") + check_query(clickhouse_node, "SELECT * FROM test_database_alter_drop.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_alter_drop") + mysql_node.query("DROP DATABASE test_database_alter_drop") def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_alter_rename") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_alter_rename") + mysql_node.query("CREATE DATABASE test_database_alter_rename DEFAULT CHARACTER SET 'utf8'") # maybe should test rename primary key? mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") + "CREATE TABLE test_database_alter_rename.test_table_1 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 RENAME COLUMN rename_column TO new_column_name") + mysql_node.query("ALTER TABLE test_database_alter_rename.test_table_1 RENAME COLUMN rename_column TO new_column_name") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_alter_rename ENGINE = MaterializeMySQL('{}:3306', 'test_database_alter_rename', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + assert "test_database_alter_rename" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "DESC test_database_alter_rename.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + "CREATE TABLE test_database_alter_rename.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "DESC test_database_alter_rename.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nrename_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 RENAME COLUMN rename_column TO new_column_name") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_rename.test_table_2 RENAME COLUMN rename_column TO new_column_name") + check_query(clickhouse_node, "DESC test_database_alter_rename.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2), (3, 4), (5, 6), (7, 8), (9, 10)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", + mysql_node.query("INSERT INTO test_database_alter_rename.test_table_2 VALUES(1, 2), (3, 4), (5, 6), (7, 8), (9, 10)") + check_query(clickhouse_node, "SELECT * FROM test_database_alter_rename.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t4\n5\t6\n7\t8\n9\t10\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_alter_rename") + mysql_node.query("DROP DATABASE test_database_alter_rename") def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_alter_modify") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_alter_modify") + mysql_node.query("CREATE DATABASE test_database_alter_modify DEFAULT CHARACTER SET 'utf8'") # maybe should test rename primary key? mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") + "CREATE TABLE test_database_alter_modify.test_table_1 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") + mysql_node.query("ALTER TABLE test_database_alter_modify.test_table_1 MODIFY COLUMN modify_column INT") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_alter_modify ENGINE = MaterializeMySQL('{}:3306', 'test_database_alter_modify', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + assert "test_database_alter_modify" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_alter_modify FORMAT TSV", "test_table_1\n") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + "CREATE TABLE test_database_alter_modify.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_alter_modify FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_modify.test_table_2 MODIFY COLUMN modify_column INT") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT FIRST") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_modify.test_table_2 MODIFY COLUMN modify_column INT FIRST") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_2 FORMAT TSV", "modify_column\tNullable(Int32)\t\t\t\t\t\nid\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT AFTER id") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_modify.test_table_2 MODIFY COLUMN modify_column INT AFTER id") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2), (3, NULL)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t\\N\n") + mysql_node.query("INSERT INTO test_database_alter_modify.test_table_2 VALUES(1, 2), (3, NULL)") + check_query(clickhouse_node, "SELECT * FROM test_database_alter_modify.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t\\N\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_alter_modify") + mysql_node.query("DROP DATABASE test_database_alter_modify") # TODO: need ClickHouse support ALTER TABLE table_name ADD COLUMN column_name, RENAME COLUMN column_name TO new_column_name; @@ -433,53 +433,53 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n # pass def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_rename_table") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_rename_table") + mysql_node.query("CREATE DATABASE test_database_rename_table DEFAULT CHARACTER SET 'utf8'") mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") + "CREATE TABLE test_database_rename_table.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") mysql_node.query( - "ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_database.test_table_2, RENAME TO test_database.test_table_3") + "ALTER TABLE test_database_rename_table.test_table_1 DROP COLUMN drop_column, RENAME TO test_database_rename_table.test_table_2, RENAME TO test_database_rename_table.test_table_3") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_rename_table ENGINE = MaterializeMySQL('{}:3306', 'test_database_rename_table', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_3\n") - check_query(clickhouse_node, "DESC test_database.test_table_3 FORMAT TSV", + assert "test_database_rename_table" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename_table FORMAT TSV", "test_table_3\n") + check_query(clickhouse_node, "DESC test_database_rename_table.test_table_3 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_3\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + "CREATE TABLE test_database_rename_table.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename_table FORMAT TSV", "test_table_1\ntest_table_3\n") + check_query(clickhouse_node, "DESC test_database_rename_table.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_database.test_table_2, RENAME TO test_database.test_table_4") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_3\ntest_table_4\n") - check_query(clickhouse_node, "DESC test_database.test_table_4 FORMAT TSV", + "ALTER TABLE test_database_rename_table.test_table_1 DROP COLUMN drop_column, RENAME TO test_database_rename_table.test_table_2, RENAME TO test_database_rename_table.test_table_4") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename_table FORMAT TSV", "test_table_3\ntest_table_4\n") + check_query(clickhouse_node, "DESC test_database_rename_table.test_table_4 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_4 VALUES(1), (2), (3), (4), (5)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_4 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") + mysql_node.query("INSERT INTO test_database_rename_table.test_table_4 VALUES(1), (2), (3), (4), (5)") + check_query(clickhouse_node, "SELECT * FROM test_database_rename_table.test_table_4 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_rename_table") + mysql_node.query("DROP DATABASE test_database_rename_table") def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database") + mysql_node.query("DROP DATABASE IF EXISTS test_database_event") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_event") + mysql_node.query("CREATE DATABASE test_database_event") mysql_node.query("RESET MASTER") - mysql_node.query("CREATE TABLE test_database.t1(a INT NOT NULL PRIMARY KEY, b VARCHAR(255) DEFAULT 'BEGIN')") - mysql_node.query("INSERT INTO test_database.t1(a) VALUES(1)") + mysql_node.query("CREATE TABLE test_database_event.t1(a INT NOT NULL PRIMARY KEY, b VARCHAR(255) DEFAULT 'BEGIN')") + mysql_node.query("INSERT INTO test_database_event.t1(a) VALUES(1)") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_event ENGINE = MaterializeMySQL('{}:3306', 'test_database_event', 'root', 'clickhouse')".format( service_name)) # Reject one empty GTID QUERY event with 'BEGIN' and 'COMMIT' @@ -495,13 +495,13 @@ def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name # Reject one 'BEGIN' QUERY event and 'COMMIT' XID event. mysql_node.query("/* start */ begin /* end */") - mysql_node.query("INSERT INTO test_database.t1(a) VALUES(2)") + mysql_node.query("INSERT INTO test_database_event.t1(a) VALUES(2)") mysql_node.query("/* start */ commit /* end */") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "t1\n") - check_query(clickhouse_node, "SELECT * FROM test_database.t1 ORDER BY a FORMAT TSV", "1\tBEGIN\n2\tBEGIN\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_event FORMAT TSV", "t1\n") + check_query(clickhouse_node, "SELECT * FROM test_database_event.t1 ORDER BY a FORMAT TSV", "1\tBEGIN\n2\tBEGIN\n") + clickhouse_node.query("DROP DATABASE test_database_event") + mysql_node.query("DROP DATABASE test_database_event") def select_without_columns(clickhouse_node, mysql_node, service_name): @@ -631,23 +631,23 @@ def drop_instance_mysql_connections(clickhouse_node, pm, action='REJECT'): time.sleep(5) def network_partition_test(clickhouse_node, mysql_node, service_name): - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_network") clickhouse_node.query("DROP DATABASE IF EXISTS test") - mysql_node.query("DROP DATABASE IF EXISTS test_database") + mysql_node.query("DROP DATABASE IF EXISTS test_database_network") mysql_node.query("DROP DATABASE IF EXISTS test") - mysql_node.query("CREATE DATABASE test_database;") - mysql_node.query("CREATE TABLE test_database.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") + mysql_node.query("CREATE DATABASE test_database_network;") + mysql_node.query("CREATE TABLE test_database_network.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") mysql_node.query("CREATE DATABASE test;") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) - check_query(clickhouse_node, "SELECT * FROM test_database.test_table", '') + "CREATE DATABASE test_database_network ENGINE = MaterializeMySQL('{}:3306', 'test_database_network', 'root', 'clickhouse')".format(service_name)) + check_query(clickhouse_node, "SELECT * FROM test_database_network.test_table", '') with PartitionManager() as pm: drop_instance_mysql_connections(clickhouse_node, pm) - mysql_node.query('INSERT INTO test_database.test_table VALUES(1)') - check_query(clickhouse_node, "SELECT * FROM test_database.test_table", '') + mysql_node.query('INSERT INTO test_database_network.test_table VALUES(1)') + check_query(clickhouse_node, "SELECT * FROM test_database_network.test_table", '') with pytest.raises(QueryRuntimeException) as exception: clickhouse_node.query( @@ -657,18 +657,18 @@ def network_partition_test(clickhouse_node, mysql_node, service_name): restore_instance_mysql_connections(clickhouse_node, pm) - check_query(clickhouse_node, "SELECT * FROM test_database.test_table FORMAT TSV", '1\n') + check_query(clickhouse_node, "SELECT * FROM test_database_network.test_table FORMAT TSV", '1\n') clickhouse_node.query( "CREATE DATABASE test ENGINE = MaterializeMySQL('{}:3306', 'test', 'root', 'clickhouse')".format(service_name)) - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table\n") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_network FORMAT TSV", "test_table\n") mysql_node.query("CREATE TABLE test.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test FORMAT TSV", "test\n") - clickhouse_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_network") clickhouse_node.query("DROP DATABASE test") - mysql_node.query("DROP DATABASE test_database") + mysql_node.query("DROP DATABASE test_database_network") mysql_node.query("DROP DATABASE test") @@ -732,9 +732,11 @@ def mysql_kill_sync_thread_restore_test(clickhouse_node, mysql_node, service_nam def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS kill_mysql_while_insert") + clickhouse_node.query("DROP DATABASE IF EXISTS kill_mysql_while_insert") mysql_node.query("CREATE DATABASE kill_mysql_while_insert") mysql_node.query("CREATE TABLE kill_mysql_while_insert.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") - clickhouse_node.query("CREATE DATABASE kill_mysql_while_insert ENGINE = MaterializeMySQL('{}:3306', 'kill_mysql_while_insert', 'root', 'clickhouse')".format(service_name)) + clickhouse_node.query("CREATE DATABASE kill_mysql_while_insert ENGINE = MaterializeMySQL('{}:3306', 'kill_mysql_while_insert', 'root', 'clickhouse') SETTINGS max_wait_time_when_mysql_unavailable=-1".format(service_name)) check_query(clickhouse_node, "SHOW TABLES FROM kill_mysql_while_insert FORMAT TSV", 'test\n') try: @@ -746,16 +748,13 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): t = threading.Thread(target=insert, args=(10000,)) t.start() - run_and_check( - ['docker-compose', '-p', mysql_node.project_name, '-f', mysql_node.docker_compose, 'stop']) + clickhouse_node.cluster.restart_service(service_name) finally: with pytest.raises(QueryRuntimeException) as exception: - time.sleep(5) + time.sleep(2) clickhouse_node.query("SELECT count() FROM kill_mysql_while_insert.test") - run_and_check( - ['docker-compose', '-p', mysql_node.project_name, '-f', mysql_node.docker_compose, 'start']) - mysql_node.wait_mysql_to_start(120) + mysql_node.alloc_connection() clickhouse_node.query("DETACH DATABASE kill_mysql_while_insert") clickhouse_node.query("ATTACH DATABASE kill_mysql_while_insert") @@ -770,6 +769,7 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): def clickhouse_killed_while_insert(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS kill_clickhouse_while_insert") mysql_node.query("CREATE DATABASE kill_clickhouse_while_insert") mysql_node.query("CREATE TABLE kill_clickhouse_while_insert.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") clickhouse_node.query("CREATE DATABASE kill_clickhouse_while_insert ENGINE = MaterializeMySQL('{}:3306', 'kill_clickhouse_while_insert', 'root', 'clickhouse')".format(service_name)) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 19067db3eca..e720bdfd5f5 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -7,15 +7,18 @@ import pymysql.cursors import pytest from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check import docker +import logging from . import materialize_with_ddl DOCKER_COMPOSE_PATH = get_docker_compose_path() cluster = ClickHouseCluster(__file__) +mysql_node = None +mysql8_node = None -node_db_ordinary = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=False, stay_alive=True) -node_db_atomic = cluster.add_instance('node2', user_configs=["configs/users_db_atomic.xml"], with_mysql=False, stay_alive=True) +node_db_ordinary = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=True, stay_alive=True) +node_db_atomic = cluster.add_instance('node2', user_configs=["configs/users_db_atomic.xml"], with_mysql8=True, stay_alive=True) node_disable_bytes_settings = cluster.add_instance('node3', user_configs=["configs/users_disable_bytes_settings.xml"], with_mysql=False, stay_alive=True) node_disable_rows_settings = cluster.add_instance('node4', user_configs=["configs/users_disable_rows_settings.xml"], with_mysql=False, stay_alive=True) @@ -28,32 +31,29 @@ def started_cluster(): cluster.shutdown() -class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', ip_address='127.0.0.1', port=3308, docker_compose=None, project_name=cluster.project_name): +class MySQLConnection: + def __init__(self, port, user='root', password='clickhouse', ip_address=None, docker_compose=None, project_name=cluster.project_name): self.user = user self.port = port self.ip_address = ip_address self.password = password self.mysql_connection = None # lazy init - self.docker_compose = docker_compose - self.project_name = project_name - - self.base_dir = p.dirname(__file__) - self.instances_dir = p.join(self.base_dir, '_instances_mysql') - if not os.path.exists(self.instances_dir): - os.mkdir(self.instances_dir) - self.docker_logs_path = p.join(self.instances_dir, 'docker_mysql.log') - self.start_up = False def alloc_connection(self): - if self.mysql_connection is None: - self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.ip_address, - port=self.port, autocommit=True) - else: - if self.mysql_connection.ping(): - self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.ip_address, - port=self.port, autocommit=True) - return self.mysql_connection + errors = [] + for _ in range(5): + try: + if self.mysql_connection is None: + self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.ip_address, + port=self.port, autocommit=True) + else: + self.mysql_connection.ping(reconnect=True) + logging.debug("MySQL Connection establised: {}:{}".format(self.ip_address, self.port)) + return self.mysql_connection + except Exception as e: + errors += [str(e)] + time.sleep(1) + raise Exception("Connection not establised, {}".format(errors)) def query(self, execution_query): with self.alloc_connection().cursor() as cursor: @@ -78,226 +78,148 @@ class MySQLNodeInstance: cursor.execute(executio_query) return cursor.fetchall() - def start_and_wait(self): - if self.start_up: - return - - run_and_check(['docker-compose', - '-p', cluster.project_name, - '-f', self.docker_compose, - 'up', '--no-recreate', '-d', - ]) - self.wait_mysql_to_start(120) - self.start_up = True - def close(self): if self.mysql_connection is not None: self.mysql_connection.close() - with open(self.docker_logs_path, "w+") as f: - try: - run_and_check([ - 'docker-compose', - '-p', cluster.project_name, - '-f', self.docker_compose, 'logs', - ], stdout=f) - except Exception as e: - print("Unable to get logs from docker mysql.") - - self.start_up = False - - def wait_mysql_to_start(self, timeout=60): - start = time.time() - while time.time() - start < timeout: - try: - self.alloc_connection() - print("Mysql Started") - return - except Exception as ex: - print("Can't connect to MySQL " + str(ex)) - time.sleep(0.5) - - run_and_check(['docker-compose', 'ps', '--services', 'all']) - raise Exception("Cannot wait MySQL container") - - -mysql_5_7_docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_5_7_for_materialize_mysql.yml') -mysql_5_7_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308, mysql_5_7_docker_compose) - -mysql_8_0_docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_8_0_for_materialize_mysql.yml') -mysql_8_0_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3309, mysql_8_0_docker_compose) - - @pytest.fixture(scope="module") def started_mysql_5_7(): - try: - mysql_5_7_node.start_and_wait() - yield mysql_5_7_node - finally: - mysql_5_7_node.close() - run_and_check(['docker-compose', '-p', cluster.project_name, '-f', mysql_5_7_docker_compose, 'down', '--volumes', '--remove-orphans']) - + mysql_node = MySQLConnection(cluster.mysql_port, 'root', 'clickhouse', cluster.mysql_ip) + yield mysql_node @pytest.fixture(scope="module") def started_mysql_8_0(): - try: - mysql_8_0_node.start_and_wait() - yield mysql_8_0_node - finally: - mysql_8_0_node.close() - run_and_check(['docker-compose', '-p', cluster.project_name, '-f', mysql_8_0_docker_compose, 'down', '--volumes', '--remove-orphans']) + mysql8_node = MySQLConnection(cluster.mysql8_port, 'root', 'clickhouse', cluster.mysql8_ip) + yield mysql8_node - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.materialize_mysql_database_with_views(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.move_to_prewhere_and_column_filtering(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.materialize_mysql_database_with_views(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.move_to_prewhere_and_column_filtering(clickhouse_node, started_mysql_5_7, "mysql57") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.materialize_mysql_database_with_views(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.move_to_prewhere_and_column_filtering(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.materialize_mysql_database_with_views(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.move_to_prewhere_and_column_filtering(clickhouse_node, started_mysql_8_0, "mysql80") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") # mysql 5.7 cannot support alter rename column - # materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + # materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") - materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") - materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") - materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") - materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") + materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_ddl_with_empty_transaction_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7, "mysql57") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_ddl_with_empty_transaction_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_select_without_columns_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_select_without_columns_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_insert_with_modify_binlog_checksum_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_insert_with_modify_binlog_checksum_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_err_sync_user_privs_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_network_partition_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_5_7, "mysql57") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_network_partition_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_8_0, "mysql80") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_mysql_kill_sync_thread_restore_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_5_7, "mysql57") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_mysql_kill_sync_thread_restore_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_8_0, "mysql80") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_mysql_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql57") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_mysql_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql80") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_clickhouse_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql57") - -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_clickhouse_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql8_0") - + materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_utf8mb4(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_8_0, "mysql8_0") - + materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_system_parts_table(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.system_parts_test(clickhouse_node, started_mysql_8_0, "mysql8_0") - + materialize_with_ddl.system_parts_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_multi_table_update(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_8_0, "mysql8_0") - + materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_system_tables_table(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_disable_bytes_settings, node_disable_rows_settings]) def test_mysql_settings(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.mysql_settings_test(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.mysql_settings_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.mysql_settings_test(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.mysql_settings_test(clickhouse_node, started_mysql_8_0, "mysql80") diff --git a/tests/integration/test_merge_tree_hdfs/test.py b/tests/integration/test_merge_tree_hdfs/test.py index 2d0d9d9fb1e..0984e4d288a 100644 --- a/tests/integration/test_merge_tree_hdfs/test.py +++ b/tests/integration/test_merge_tree_hdfs/test.py @@ -64,7 +64,7 @@ def cluster(): cluster.start() logging.info("Cluster started") - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) fs.mkdirs('/clickhouse') logging.info("Created HDFS directory") @@ -75,7 +75,7 @@ def cluster(): def wait_for_delete_hdfs_objects(cluster, expected, num_tries=30): - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) while num_tries > 0: num_hdfs_objects = len(fs.listdir('/clickhouse')) if num_hdfs_objects == expected: @@ -89,7 +89,7 @@ def wait_for_delete_hdfs_objects(cluster, expected, num_tries=30): def drop_table(cluster): node = cluster.instances["node"] - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) hdfs_objects = fs.listdir('/clickhouse') print('Number of hdfs objects to delete:', len(hdfs_objects), sep=' ') @@ -116,7 +116,7 @@ def test_simple_insert_select(cluster, min_rows_for_wide_part, files_per_part): node.query("INSERT INTO hdfs_test VALUES {}".format(values1)) assert node.query("SELECT * FROM hdfs_test order by dt, id FORMAT Values") == values1 - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) hdfs_objects = fs.listdir('/clickhouse') print(hdfs_objects) @@ -136,7 +136,7 @@ def test_alter_table_columns(cluster): create_table(cluster, "hdfs_test") node = cluster.instances["node"] - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096))) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096, -1))) @@ -165,7 +165,7 @@ def test_attach_detach_partition(cluster): create_table(cluster, "hdfs_test") node = cluster.instances["node"] - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096))) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096))) @@ -204,7 +204,7 @@ def test_move_partition_to_another_disk(cluster): create_table(cluster, "hdfs_test") node = cluster.instances["node"] - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096))) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096))) @@ -230,7 +230,7 @@ def test_table_manipulations(cluster): create_table(cluster, "hdfs_test") node = cluster.instances["node"] - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096))) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096))) @@ -262,7 +262,7 @@ def test_move_replace_partition_to_another_table(cluster): create_table(cluster, "hdfs_test") node = cluster.instances["node"] - fs = HdfsClient(hosts='localhost') + fs = HdfsClient(hosts=cluster.hdfs_ip) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096))) node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096))) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index c0c05355def..4b30287ca1e 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -8,9 +8,6 @@ import os import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - # By default the exceptions that was throwed in threads will be ignored # (they will not mark the test as failed, only printed to stderr). diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index 11a1e464cb6..5af9582363a 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -7,9 +7,6 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - # Runs custom python-based S3 endpoint. def run_endpoint(cluster): @@ -20,11 +17,12 @@ def run_endpoint(cluster): cluster.exec_in_container(container_id, ["python", "endpoint.py"], detach=True) # Wait for S3 endpoint start - for attempt in range(10): + num_attempts = 100 + for attempt in range(num_attempts): ping_response = cluster.exec_in_container(cluster.get_container_id('resolver'), ["curl", "-s", "http://resolver:8080/"], nothrow=True) if ping_response != 'OK': - if attempt == 9: + if attempt == num_attempts - 1: assert ping_response == 'OK', 'Expected "OK", but got "{}"'.format(ping_response) else: time.sleep(1) @@ -159,13 +157,13 @@ def test_move_failover(cluster): # There should be 2 attempts to move part. assert node.query(""" - SELECT count(*) FROM system.part_log + SELECT count(*) FROM system.part_log WHERE event_type='MovePart' AND table='s3_failover_test' """) == '2\n' # First attempt should be failed with expected error. exception = node.query(""" - SELECT exception FROM system.part_log + SELECT exception FROM system.part_log WHERE event_type='MovePart' AND table='s3_failover_test' AND notEmpty(exception) ORDER BY event_time LIMIT 1 diff --git a/tests/integration/test_merge_tree_s3_restore/test.py b/tests/integration/test_merge_tree_s3_restore/test.py index 1ad49eec7d2..b7543388791 100644 --- a/tests/integration/test_merge_tree_s3_restore/test.py +++ b/tests/integration/test_merge_tree_s3_restore/test.py @@ -7,10 +7,6 @@ import time import pytest from helpers.cluster import ClickHouseCluster - -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) NOT_RESTORABLE_CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node_not_restorable/configs/config.d/storage_conf_not_restorable.xml') COMMON_CONFIGS = ["configs/config.d/bg_processing_pool_conf.xml", "configs/config.d/log_conf.xml", "configs/config.d/clusters.xml"] diff --git a/tests/integration/test_merge_tree_s3_with_cache/test.py b/tests/integration/test_merge_tree_s3_with_cache/test.py index d5d6db2fb77..7022f90a2b9 100644 --- a/tests/integration/test_merge_tree_s3_with_cache/test.py +++ b/tests/integration/test_merge_tree_s3_with_cache/test.py @@ -3,9 +3,6 @@ import logging import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index c01289c8114..69ce3da17ac 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -244,7 +244,7 @@ def test_query_parser(start_cluster): try: with pytest.raises(QueryRuntimeException): node1.query(""" - CREATE TABLE table_with_absent_policy ( + CREATE TABLE IF NOT EXISTS table_with_absent_policy ( d UInt64 ) ENGINE = MergeTree() ORDER BY d @@ -253,7 +253,7 @@ def test_query_parser(start_cluster): with pytest.raises(QueryRuntimeException): node1.query(""" - CREATE TABLE table_with_absent_policy ( + CREATE TABLE IF NOT EXISTS table_with_absent_policy ( d UInt64 ) ENGINE = MergeTree() ORDER BY d @@ -261,7 +261,7 @@ def test_query_parser(start_cluster): """) node1.query(""" - CREATE TABLE table_with_normal_policy ( + CREATE TABLE IF NOT EXISTS table_with_normal_policy ( d UInt64 ) ENGINE = MergeTree() ORDER BY d @@ -290,13 +290,13 @@ def test_query_parser(start_cluster): @pytest.mark.parametrize("name,engine", [ - ("test_alter_policy", "MergeTree()"), - ("replicated_test_alter_policy", "ReplicatedMergeTree('/clickhouse/test_alter_policy', '1')",), + pytest.param("test_alter_policy", "MergeTree()", id="mt"), + pytest.param("replicated_test_alter_policy", "ReplicatedMergeTree('/clickhouse/test_alter_policy', '1')", id="replicated"), ]) def test_alter_policy(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( d UInt64 ) ENGINE = {engine} ORDER BY d @@ -314,7 +314,7 @@ def test_alter_policy(start_cluster, name, engine): assert node1.query("""SELECT storage_policy FROM system.tables WHERE name = '{name}'""".format( name=name)) == "small_jbod_with_external\n" - node1.query("""ALTER TABLE {name} MODIFY SETTING storage_policy='jbods_with_external'""".format(name=name)) + node1.query_with_retry("""ALTER TABLE {name} MODIFY SETTING storage_policy='jbods_with_external'""".format(name=name)) assert node1.query("""SELECT storage_policy FROM system.tables WHERE name = '{name}'""".format( name=name)) == "jbods_with_external\n" @@ -327,7 +327,7 @@ def test_alter_policy(start_cluster, name, engine): name=name)) == "jbods_with_external\n" finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") def get_random_string(length): @@ -349,13 +349,13 @@ def test_no_warning_about_zero_max_data_part_size(start_cluster): for node in (node1, node2): node.query(""" - CREATE TABLE default.test_warning_table ( + CREATE TABLE IF NOT EXISTS default.test_warning_table ( s String ) ENGINE = MergeTree ORDER BY tuple() SETTINGS storage_policy='small_jbod_with_external' """) - node.query("DROP TABLE default.test_warning_table SYNC") + node.query("DROP TABLE IF EXISTS default.test_warning_table SYNC") log = get_log(node) assert not re.search("Warning.*Volume.*special_warning_zero_volume", log) assert not re.search("Warning.*Volume.*special_warning_default_volume", log) @@ -364,13 +364,13 @@ def test_no_warning_about_zero_max_data_part_size(start_cluster): @pytest.mark.parametrize("name,engine", [ - ("mt_on_jbod", "MergeTree()"), - ("replicated_mt_on_jbod", "ReplicatedMergeTree('/clickhouse/replicated_mt_on_jbod', '1')",), + pytest.param("mt_on_jbod", "MergeTree()", id="mt"), + pytest.param("replicated_mt_on_jbod", "ReplicatedMergeTree('/clickhouse/replicated_mt_on_jbod', '1')", id="replicated"), ]) def test_round_robin(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( d UInt64 ) ENGINE = {engine} ORDER BY d @@ -378,17 +378,17 @@ def test_round_robin(start_cluster, name, engine): """.format(name=name, engine=engine)) # first should go to the jbod1 - node1.query("insert into {} select * from numbers(10000)".format(name)) + node1.query_with_retry("insert into {} select * from numbers(10000)".format(name)) used_disk = get_used_disks_for_table(node1, name) assert len(used_disk) == 1, 'More than one disk used for single insert' - node1.query("insert into {} select * from numbers(10000, 10000)".format(name)) + node1.query_with_retry("insert into {} select * from numbers(10000, 10000)".format(name)) used_disks = get_used_disks_for_table(node1, name) assert len(used_disks) == 2, 'Two disks should be used for two parts' assert used_disks[0] != used_disks[1], "Should write to different disks" - node1.query("insert into {} select * from numbers(20000, 10000)".format(name)) + node1.query_with_retry("insert into {} select * from numbers(20000, 10000)".format(name)) used_disks = get_used_disks_for_table(node1, name) # jbod1 -> jbod2 -> jbod1 -> jbod2 ... etc @@ -396,19 +396,19 @@ def test_round_robin(start_cluster, name, engine): assert used_disks[0] != used_disks[1] assert used_disks[2] == used_disks[0] finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ - ("mt_with_huge_part", "MergeTree()"), - ("replicated_mt_with_huge_part", "ReplicatedMergeTree('/clickhouse/replicated_mt_with_huge_part', '1')",), + pytest.param("mt_with_huge_part", "MergeTree()", id="mt"), + pytest.param("replicated_mt_with_huge_part", "ReplicatedMergeTree('/clickhouse/replicated_mt_with_huge_part', '1')", id="replicated"), ]) def test_max_data_part_size(start_cluster, name, engine): try: assert int(*node1.query("""SELECT max_data_part_size FROM system.storage_policies WHERE policy_name = 'jbods_with_external' AND volume_name = 'main'""").splitlines()) == 10*1024*1024 - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( s1 String ) ENGINE = {engine} ORDER BY tuple() @@ -418,22 +418,22 @@ def test_max_data_part_size(start_cluster, name, engine): for i in range(10): data.append(get_random_string(1024 * 1024)) # 1MB row - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) used_disks = get_used_disks_for_table(node1, name) assert len(used_disks) == 1 assert used_disks[0] == 'external' finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ - ("mt_with_overflow", "MergeTree()"), - ("replicated_mt_with_overflow", "ReplicatedMergeTree('/clickhouse/replicated_mt_with_overflow', '1')",), + pytest.param("mt_with_overflow", "MergeTree()", id="mt"), + pytest.param("replicated_mt_with_overflow", "ReplicatedMergeTree('/clickhouse/replicated_mt_with_overflow', '1')", id="replicated"), ]) def test_jbod_overflow(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( s1 String ) ENGINE = {engine} ORDER BY tuple() @@ -447,7 +447,7 @@ def test_jbod_overflow(start_cluster, name, engine): data = [] # 5MB in total for i in range(5): data.append(get_random_string(1024 * 1024)) # 1MB row - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) used_disks = get_used_disks_for_table(node1, name) assert used_disks == tuple('jbod1' for _ in used_disks) @@ -457,7 +457,7 @@ def test_jbod_overflow(start_cluster, name, engine): for i in range(10): data.append(get_random_string(1024 * 1024)) # 1MB row - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) used_disks = get_used_disks_for_table(node1, name) @@ -466,7 +466,7 @@ def test_jbod_overflow(start_cluster, name, engine): node1.query(f"SYSTEM START MERGES {name}") time.sleep(1) - node1.query("OPTIMIZE TABLE {} FINAL".format(name)) + node1.query_with_retry("OPTIMIZE TABLE {} FINAL".format(name)) time.sleep(2) disks_for_merges = tuple(node1.query( @@ -476,17 +476,17 @@ def test_jbod_overflow(start_cluster, name, engine): assert disks_for_merges == tuple('external' for _ in disks_for_merges) finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ - ("moving_mt", "MergeTree()"), - ("moving_replicated_mt", "ReplicatedMergeTree('/clickhouse/moving_replicated_mt', '1')",), + pytest.param("moving_mt", "MergeTree()", id="mt"), + pytest.param("moving_replicated_mt", "ReplicatedMergeTree('/clickhouse/moving_replicated_mt', '1')", id="replicated"), ]) def test_background_move(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( s1 String ) ENGINE = {engine} ORDER BY tuple() @@ -500,7 +500,7 @@ def test_background_move(start_cluster, name, engine): for i in range(5): data.append(get_random_string(1024 * 1024)) # 1MB row # small jbod size is 40MB, so lets insert 5MB batch 5 times - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) used_disks = get_used_disks_for_table(node1, name) @@ -526,25 +526,25 @@ def test_background_move(start_cluster, name, engine): node1.query(f"SYSTEM START MERGES {name}") finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ - ("stopped_moving_mt", "MergeTree()"), - ("stopped_moving_replicated_mt", "ReplicatedMergeTree('/clickhouse/stopped_moving_replicated_mt', '1')",), + pytest.param("stopped_moving_mt", "MergeTree()", id="mt"), + pytest.param("stopped_moving_replicated_mt", "ReplicatedMergeTree('/clickhouse/stopped_moving_replicated_mt', '1')", id="replicated"), ]) def test_start_stop_moves(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( s1 String ) ENGINE = {engine} ORDER BY tuple() SETTINGS storage_policy='moving_jbod_with_external' """.format(name=name, engine=engine)) - node1.query("INSERT INTO {} VALUES ('HELLO')".format(name)) - node1.query("INSERT INTO {} VALUES ('WORLD')".format(name)) + node1.query_with_retry("INSERT INTO {} VALUES ('HELLO')".format(name)) + node1.query_with_retry("INSERT INTO {} VALUES ('WORLD')".format(name)) used_disks = get_used_disks_for_table(node1, name) assert all(d == "jbod1" for d in used_disks), "All writes shoud go to jbods" @@ -571,7 +571,7 @@ def test_start_stop_moves(start_cluster, name, engine): assert disk == "external" - node1.query("TRUNCATE TABLE {}".format(name)) + node1.query_with_retry("TRUNCATE TABLE {}".format(name)) node1.query("SYSTEM STOP MOVES {}".format(name)) node1.query("SYSTEM STOP MERGES {}".format(name)) @@ -581,7 +581,7 @@ def test_start_stop_moves(start_cluster, name, engine): for i in range(5): data.append(get_random_string(1024 * 1024)) # 1MB row # jbod size is 40MB, so lets insert 5MB batch 7 times - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) used_disks = get_used_disks_for_table(node1, name) @@ -613,7 +613,7 @@ def test_start_stop_moves(start_cluster, name, engine): assert used_disks[0] == 'external' finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") def get_path_for_part_from_part_log(node, table, part_name): @@ -633,14 +633,14 @@ def get_paths_for_partition_from_part_log(node, table, partition_id): @pytest.mark.parametrize("name,engine", [ - ("altering_mt", "MergeTree()"), + pytest.param("altering_mt", "MergeTree()", id="mt"), # ("altering_replicated_mt","ReplicatedMergeTree('/clickhouse/altering_replicated_mt', '1')",), # SYSTEM STOP MERGES doesn't disable merges assignments ]) def test_alter_move(start_cluster, name, engine): try: node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( EventDate Date, number UInt64 ) ENGINE = {engine} @@ -713,7 +713,7 @@ def test_alter_move_half_of_partition(start_cluster, volume_or_disk): engine = "MergeTree()" try: node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( EventDate Date, number UInt64 ) ENGINE = {engine} @@ -762,7 +762,7 @@ def test_alter_double_move_partition(start_cluster, volume_or_disk): engine = "MergeTree()" try: node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( EventDate Date, number UInt64 ) ENGINE = {engine} @@ -827,14 +827,14 @@ def produce_alter_move(node, name): @pytest.mark.parametrize("name,engine", [ - ("concurrently_altering_mt", "MergeTree()"), - ("concurrently_altering_replicated_mt", - "ReplicatedMergeTree('/clickhouse/concurrently_altering_replicated_mt', '1')",), + pytest.param("concurrently_altering_mt", "MergeTree()", id="mt"), + pytest.param("concurrently_altering_replicated_mt", + "ReplicatedMergeTree('/clickhouse/concurrently_altering_replicated_mt', '1')", id="replicated"), ]) def test_concurrent_alter_move(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( EventDate Date, number UInt64 ) ENGINE = {engine} @@ -850,7 +850,7 @@ def test_concurrent_alter_move(start_cluster, name, engine): day = random.randint(11, 30) value = values.pop() month = '0' + str(random.choice([3, 4])) - node1.query("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value)) + node1.query_with_retry("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value)) def alter_move(num): for i in range(num): @@ -862,7 +862,7 @@ def test_concurrent_alter_move(start_cluster, name, engine): def optimize_table(num): for i in range(num): - node1.query("OPTIMIZE TABLE {} FINAL".format(name)) + node1.query_with_retry("OPTIMIZE TABLE {} FINAL".format(name)) p = Pool(15) tasks = [] @@ -882,14 +882,14 @@ def test_concurrent_alter_move(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("concurrently_dropping_mt", "MergeTree()"), - ("concurrently_dropping_replicated_mt", - "ReplicatedMergeTree('/clickhouse/concurrently_dropping_replicated_mt', '1')",), + pytest.param("concurrently_dropping_mt", "MergeTree()", id="mt"), + pytest.param("concurrently_dropping_replicated_mt", + "ReplicatedMergeTree('/clickhouse/concurrently_dropping_replicated_mt', '1')", id="replicated"), ]) def test_concurrent_alter_move_and_drop(start_cluster, name, engine): try: node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( EventDate Date, number UInt64 ) ENGINE = {engine} @@ -905,7 +905,7 @@ def test_concurrent_alter_move_and_drop(start_cluster, name, engine): day = random.randint(11, 30) value = values.pop() month = '0' + str(random.choice([3, 4])) - node1.query("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value)) + node1.query_with_retry("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value)) def alter_move(num): for i in range(num): @@ -926,22 +926,22 @@ def test_concurrent_alter_move_and_drop(start_cluster, name, engine): tasks.append(p.apply_async(alter_drop, (100,))) for task in tasks: - task.get(timeout=60) + task.get(timeout=120) assert node1.query("SELECT 1") == "1\n" finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ - ("detach_attach_mt", "MergeTree()"), - ("replicated_detach_attach_mt", "ReplicatedMergeTree('/clickhouse/replicated_detach_attach_mt', '1')",), + pytest.param("detach_attach_mt", "MergeTree()", id="mt"), + pytest.param("replicated_detach_attach_mt", "ReplicatedMergeTree('/clickhouse/replicated_detach_attach_mt', '1')", id="replicated"), ]) def test_detach_attach(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( s1 String ) ENGINE = {engine} ORDER BY tuple() @@ -951,7 +951,7 @@ def test_detach_attach(start_cluster, name, engine): data = [] # 5MB in total for i in range(5): data.append(get_random_string(1024 * 1024)) # 1MB row - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) node1.query("ALTER TABLE {} DETACH PARTITION tuple()".format(name)) assert node1.query("SELECT count() FROM {}".format(name)).strip() == "0" @@ -962,17 +962,17 @@ def test_detach_attach(start_cluster, name, engine): assert node1.query("SELECT count() FROM {}".format(name)).strip() == "5" finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ - ("mutating_mt", "MergeTree()"), - ("replicated_mutating_mt", "ReplicatedMergeTree('/clickhouse/replicated_mutating_mt', '1')",), + pytest.param("mutating_mt", "MergeTree()", id="mt"), + pytest.param("replicated_mutating_mt", "ReplicatedMergeTree('/clickhouse/replicated_mutating_mt', '1')", id="replicated"), ]) def test_mutate_to_another_disk(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( s1 String ) ENGINE = {engine} ORDER BY tuple() @@ -983,7 +983,7 @@ def test_mutate_to_another_disk(start_cluster, name, engine): data = [] # 5MB in total for i in range(5): data.append(get_random_string(1024 * 1024)) # 1MB row - node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + node1.query_with_retry("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) node1.query("ALTER TABLE {} UPDATE s1 = concat(s1, 'x') WHERE 1".format(name)) @@ -996,7 +996,7 @@ def test_mutate_to_another_disk(start_cluster, name, engine): assert node1.query("SELECT sum(endsWith(s1, 'x')) FROM {}".format(name)) == "25\n" else: # mutation failed, let's try on another disk print("Mutation failed") - node1.query("OPTIMIZE TABLE {} FINAL".format(name)) + node1.query_with_retry("OPTIMIZE TABLE {} FINAL".format(name)) node1.query("ALTER TABLE {} UPDATE s1 = concat(s1, 'x') WHERE 1".format(name)) retry = 20 while node1.query("SELECT * FROM system.mutations WHERE is_done = 0") != "" and retry > 0: @@ -1008,17 +1008,17 @@ def test_mutate_to_another_disk(start_cluster, name, engine): finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") @pytest.mark.parametrize("name,engine", [ - ("alter_modifying_mt", "MergeTree()"), - ("replicated_alter_modifying_mt", "ReplicatedMergeTree('/clickhouse/replicated_alter_modifying_mt', '1')",), + pytest.param("alter_modifying_mt", "MergeTree()", id="mt"), + pytest.param("replicated_alter_modifying_mt", "ReplicatedMergeTree('/clickhouse/replicated_alter_modifying_mt', '1')", id="replicated"), ]) def test_concurrent_alter_modify(start_cluster, name, engine): try: - node1.query(""" - CREATE TABLE {name} ( + node1.query_with_retry(""" + CREATE TABLE IF NOT EXISTS {name} ( EventDate Date, number UInt64 ) ENGINE = {engine} @@ -1034,7 +1034,7 @@ def test_concurrent_alter_modify(start_cluster, name, engine): day = random.randint(11, 30) value = values.pop() month = '0' + str(random.choice([3, 4])) - node1.query("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value)) + node1.query_with_retry("INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format(name, m=month, d=day, v=value)) def alter_move(num): for i in range(num): @@ -1066,14 +1066,14 @@ def test_concurrent_alter_modify(start_cluster, name, engine): assert node1.query("SELECT COUNT() FROM {}".format(name)) == "100\n" finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") + node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") def test_simple_replication_and_moves(start_cluster): try: for i, node in enumerate([node1, node2]): - node.query(""" - CREATE TABLE replicated_table_for_moves ( + node.query_with_retry(""" + CREATE TABLE IF NOT EXISTS replicated_table_for_moves ( s1 String ) ENGINE = ReplicatedMergeTree('/clickhouse/replicated_table_for_moves', '{}') ORDER BY tuple() @@ -1086,13 +1086,13 @@ def test_simple_replication_and_moves(start_cluster): data = [] # 1MB in total for i in range(2): data.append(get_random_string(512 * 1024)) # 500KB value - node.query("INSERT INTO replicated_table_for_moves VALUES {}".format( + node.query_with_retry("INSERT INTO replicated_table_for_moves VALUES {}".format( ','.join(["('" + x + "')" for x in data]))) def optimize(num): for i in range(num): node = random.choice([node1, node2]) - node.query("OPTIMIZE TABLE replicated_table_for_moves FINAL") + node.query_with_retry("OPTIMIZE TABLE replicated_table_for_moves FINAL") p = Pool(60) tasks = [] @@ -1102,8 +1102,8 @@ def test_simple_replication_and_moves(start_cluster): for task in tasks: task.get(timeout=60) - node1.query("SYSTEM SYNC REPLICA replicated_table_for_moves", timeout=5) - node2.query("SYSTEM SYNC REPLICA replicated_table_for_moves", timeout=5) + node1.query_with_retry("SYSTEM SYNC REPLICA replicated_table_for_moves", timeout=5) + node2.query_with_retry("SYSTEM SYNC REPLICA replicated_table_for_moves", timeout=5) node1.query("SELECT COUNT() FROM replicated_table_for_moves") == "40\n" node2.query("SELECT COUNT() FROM replicated_table_for_moves") == "40\n" @@ -1116,9 +1116,9 @@ def test_simple_replication_and_moves(start_cluster): node1.query("SYSTEM STOP MERGES") node2.query("SYSTEM STOP MERGES") - node1.query( + node1.query_with_retry( "INSERT INTO replicated_table_for_moves VALUES {}".format(','.join(["('" + x + "')" for x in data]))) - node2.query( + node2.query_with_retry( "INSERT INTO replicated_table_for_moves VALUES {}".format(','.join(["('" + x + "')" for x in data]))) time.sleep(3) # nothing was moved @@ -1139,8 +1139,8 @@ def test_simple_replication_and_moves(start_cluster): def test_download_appropriate_disk(start_cluster): try: for i, node in enumerate([node1, node2]): - node.query(""" - CREATE TABLE replicated_table_for_download ( + node.query_with_retry(""" + CREATE TABLE IF NOT EXISTS replicated_table_for_download ( s1 String ) ENGINE = ReplicatedMergeTree('/clickhouse/replicated_table_for_download', '{}') ORDER BY tuple() @@ -1150,13 +1150,13 @@ def test_download_appropriate_disk(start_cluster): data = [] for i in range(50): data.append(get_random_string(1024 * 1024)) # 1MB value - node1.query( + node1.query_with_retry( "INSERT INTO replicated_table_for_download VALUES {}".format(','.join(["('" + x + "')" for x in data]))) for _ in range(10): try: print("Syncing replica") - node2.query("SYSTEM SYNC REPLICA replicated_table_for_download") + node2.query_with_retry("SYSTEM SYNC REPLICA replicated_table_for_download") break except: time.sleep(0.5) @@ -1167,13 +1167,13 @@ def test_download_appropriate_disk(start_cluster): finally: for node in [node1, node2]: - node.query("DROP TABLE IF EXISTS replicated_table_for_download SYNC") + node.query_with_retry("DROP TABLE IF EXISTS replicated_table_for_download SYNC") def test_rename(start_cluster): try: node1.query(""" - CREATE TABLE default.renaming_table ( + CREATE TABLE IF NOT EXISTS default.renaming_table ( s String ) ENGINE = MergeTree ORDER BY tuple() @@ -1212,7 +1212,7 @@ def test_rename(start_cluster): def test_freeze(start_cluster): try: node1.query(""" - CREATE TABLE default.freezing_table ( + CREATE TABLE IF NOT EXISTS default.freezing_table ( d Date, s String ) ENGINE = MergeTree @@ -1249,7 +1249,7 @@ def test_kill_while_insert(start_cluster): name = "test_kill_while_insert" node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( s String ) ENGINE = MergeTree ORDER BY tuple() @@ -1294,7 +1294,7 @@ def test_move_while_merge(start_cluster): name = "test_move_while_merge" node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( n Int64 ) ENGINE = MergeTree ORDER BY sleep(2) @@ -1353,7 +1353,7 @@ def test_move_across_policies_does_not_work(start_cluster): name = "test_move_across_policies_does_not_work" node1.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( n Int64 ) ENGINE = MergeTree ORDER BY tuple() @@ -1361,7 +1361,7 @@ def test_move_across_policies_does_not_work(start_cluster): """.format(name=name)) node1.query(""" - CREATE TABLE {name}2 ( + CREATE TABLE IF NOT EXISTS {name}2 ( n Int64 ) ENGINE = MergeTree ORDER BY tuple() @@ -1393,7 +1393,7 @@ def test_move_across_policies_does_not_work(start_cluster): def _insert_merge_execute(node, name, policy, parts, cmds, parts_before_cmds, parts_after_cmds): try: node.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( n Int64 ) ENGINE = MergeTree ORDER BY tuple() @@ -1430,7 +1430,7 @@ def _check_merges_are_working(node, storage_policy, volume, shall_work): name = "_check_merges_are_working_{storage_policy}_{volume}".format(storage_policy=storage_policy, volume=volume) node.query(""" - CREATE TABLE {name} ( + CREATE TABLE IF NOT EXISTS {name} ( n Int64 ) ENGINE = MergeTree ORDER BY tuple() diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index f4b0bb1b2fc..22f790e39c3 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -21,7 +21,7 @@ def started_cluster(): class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=3308): + def __init__(self, user, password, hostname, port): self.user = user self.port = port self.hostname = hostname @@ -55,11 +55,12 @@ class MySQLNodeInstance: def test_mysql_ddl_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: + mysql_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + "CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse')") assert 'test_database' in clickhouse_node.query('SHOW DATABASES') mysql_node.query( @@ -88,13 +89,13 @@ def test_mysql_ddl_for_mysql_database(started_cluster): def test_clickhouse_ddl_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( 'CREATE TABLE `test_database`.`test_table` ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;') clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + "CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse')") assert 'test_table' in clickhouse_node.query('SHOW TABLES FROM test_database') clickhouse_node.query("DROP TABLE test_database.test_table") @@ -113,12 +114,12 @@ def test_clickhouse_ddl_for_mysql_database(started_cluster): def test_clickhouse_dml_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( 'CREATE TABLE `test_database`.`test_table` ( `i``d` int(11) NOT NULL, PRIMARY KEY (`i``d`)) ENGINE=InnoDB;') clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test_database, 'root', 'clickhouse')") + "CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', test_database, 'root', 'clickhouse')") assert clickhouse_node.query("SELECT count() FROM `test_database`.`test_table`").rstrip() == '0' clickhouse_node.query("INSERT INTO `test_database`.`test_table`(`i``d`) select number from numbers(10000)") @@ -131,7 +132,7 @@ def test_clickhouse_dml_for_mysql_database(started_cluster): def test_clickhouse_join_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test.t1_mysql_local (" "pays VARCHAR(55) DEFAULT 'FRA' NOT NULL," @@ -143,12 +144,12 @@ def test_clickhouse_join_for_mysql_database(started_cluster): "opco VARCHAR(5) DEFAULT ''" ")") clickhouse_node.query( - "CREATE TABLE default.t1_remote_mysql AS mysql('mysql1:3306','test','t1_mysql_local','root','clickhouse')") + "CREATE TABLE default.t1_remote_mysql AS mysql('mysql57:3306','test','t1_mysql_local','root','clickhouse')") clickhouse_node.query( - "CREATE TABLE default.t2_remote_mysql AS mysql('mysql1:3306','test','t2_mysql_local','root','clickhouse')") + "CREATE TABLE default.t2_remote_mysql AS mysql('mysql57:3306','test','t2_mysql_local','root','clickhouse')") clickhouse_node.query("INSERT INTO `default`.`t1_remote_mysql` VALUES ('EN','A',''),('RU','B','AAA')") clickhouse_node.query("INSERT INTO `default`.`t2_remote_mysql` VALUES ('A','AAA'),('Z','')") - + assert clickhouse_node.query("SELECT s.pays " "FROM default.t1_remote_mysql AS s " "LEFT JOIN default.t1_remote_mysql AS s_ref " @@ -158,19 +159,19 @@ def test_clickhouse_join_for_mysql_database(started_cluster): def test_bad_arguments_for_mysql_database_engine(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, port=started_cluster.mysql_port)) as mysql_node: with pytest.raises(QueryRuntimeException) as exception: mysql_node.query("CREATE DATABASE IF NOT EXISTS test_bad_arguments DEFAULT CHARACTER SET 'utf8'") clickhouse_node.query( - "CREATE DATABASE test_database_bad_arguments ENGINE = MySQL('mysql1:3306', test_bad_arguments, root, 'clickhouse')") + "CREATE DATABASE test_database_bad_arguments ENGINE = MySQL('mysql57:3306', test_bad_arguments, root, 'clickhouse')") assert 'Database engine MySQL requested literal argument.' in str(exception.value) mysql_node.query("DROP DATABASE test_bad_arguments") def test_data_types_support_level_for_mysql_database_engine(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'") - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test, 'root', 'clickhouse')", + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', test, 'root', 'clickhouse')", settings={"mysql_datatypes_support_level": "decimal,datetime64"}) assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") @@ -181,7 +182,7 @@ def test_data_types_support_level_for_mysql_database_engine(started_cluster): assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") clickhouse_node.query( - "CREATE DATABASE test_database_1 ENGINE = MySQL('mysql1:3306', test, 'root', 'clickhouse') SETTINGS mysql_datatypes_support_level = 'decimal,datetime64'", + "CREATE DATABASE test_database_1 ENGINE = MySQL('mysql57:3306', test, 'root', 'clickhouse') SETTINGS mysql_datatypes_support_level = 'decimal,datetime64'", settings={"mysql_datatypes_support_level": "decimal"}) assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database_1 FORMAT TSV") @@ -223,60 +224,60 @@ timestamp_values_no_subsecond = ["'2015-05-18 07:40:01'", "'2019-09-16 19:20:11' # ("common_types", "FLOAT", "Nullable(Float32)", float_values, ""), # ("common_types", "FLOAT UNSIGNED", "Nullable(Float32)", float_values, ""), - ("common_types", "INT", "Nullable(Int32)", int32_values, ""), - ("common_types", "INT NOT NULL", "Int32", int32_values, ""), - ("common_types", "INT UNSIGNED NOT NULL", "UInt32", uint32_values, ""), - ("common_types", "INT UNSIGNED", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT UNSIGNED DEFAULT NULL", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT UNSIGNED DEFAULT '1'", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT(10)", "Nullable(Int32)", int32_values, ""), - ("common_types", "INT(10) NOT NULL", "Int32", int32_values, ""), - ("common_types", "INT(10) UNSIGNED NOT NULL", "UInt32", uint32_values, ""), - ("common_types", "INT(10) UNSIGNED", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT(10) UNSIGNED DEFAULT NULL", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT(10) UNSIGNED DEFAULT '1'", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INTEGER", "Nullable(Int32)", int32_values, ""), - ("common_types", "INTEGER UNSIGNED", "Nullable(UInt32)", uint32_values, ""), + pytest.param("common_types", "INT", "Nullable(Int32)", int32_values, "", id="common_types_1"), + pytest.param("common_types", "INT NOT NULL", "Int32", int32_values, "", id="common_types_2"), + pytest.param("common_types", "INT UNSIGNED NOT NULL", "UInt32", uint32_values, "", id="common_types_3"), + pytest.param("common_types", "INT UNSIGNED", "Nullable(UInt32)", uint32_values, "", id="common_types_4"), + pytest.param("common_types", "INT UNSIGNED DEFAULT NULL", "Nullable(UInt32)", uint32_values, "", id="common_types_5"), + pytest.param("common_types", "INT UNSIGNED DEFAULT '1'", "Nullable(UInt32)", uint32_values, "", id="common_types_6"), + pytest.param("common_types", "INT(10)", "Nullable(Int32)", int32_values, "", id="common_types_7"), + pytest.param("common_types", "INT(10) NOT NULL", "Int32", int32_values, "", id="common_types_8"), + pytest.param("common_types", "INT(10) UNSIGNED NOT NULL", "UInt32", uint32_values, "", id="common_types_8"), + pytest.param("common_types", "INT(10) UNSIGNED", "Nullable(UInt32)", uint32_values, "", id="common_types_9"), + pytest.param("common_types", "INT(10) UNSIGNED DEFAULT NULL", "Nullable(UInt32)", uint32_values, "", id="common_types_10"), + pytest.param("common_types", "INT(10) UNSIGNED DEFAULT '1'", "Nullable(UInt32)", uint32_values, "", id="common_types_11"), + pytest.param("common_types", "INTEGER", "Nullable(Int32)", int32_values, "", id="common_types_12"), + pytest.param("common_types", "INTEGER UNSIGNED", "Nullable(UInt32)", uint32_values, "", id="common_types_13"), - ("common_types", "MEDIUMINT", "Nullable(Int32)", mint_values, ""), - ("common_types", "MEDIUMINT UNSIGNED", "Nullable(UInt32)", umint_values, ""), + pytest.param("common_types", "MEDIUMINT", "Nullable(Int32)", mint_values, "", id="common_types_14"), + pytest.param("common_types", "MEDIUMINT UNSIGNED", "Nullable(UInt32)", umint_values, "", id="common_types_15"), - ("common_types", "SMALLINT", "Nullable(Int16)", int16_values, ""), - ("common_types", "SMALLINT UNSIGNED", "Nullable(UInt16)", uint16_values, ""), + pytest.param("common_types", "SMALLINT", "Nullable(Int16)", int16_values, "", id="common_types_16"), + pytest.param("common_types", "SMALLINT UNSIGNED", "Nullable(UInt16)", uint16_values, "", id="common_types_17"), - ("common_types", "TINYINT", "Nullable(Int8)", int8_values, ""), - ("common_types", "TINYINT UNSIGNED", "Nullable(UInt8)", uint8_values, ""), + pytest.param("common_types", "TINYINT", "Nullable(Int8)", int8_values, "", id="common_types_18"), + pytest.param("common_types", "TINYINT UNSIGNED", "Nullable(UInt8)", uint8_values, "", id="common_types_19"), - ("common_types", "VARCHAR(10)", "Nullable(String)", string_values, ""), + pytest.param("common_types", "VARCHAR(10)", "Nullable(String)", string_values, "", id="common_types_20"), - ("decimal_default", "decimal NOT NULL", "Decimal(10, 0)", decimal_values, - "decimal,datetime64"), - ("decimal_default_nullable", "decimal", "Nullable(Decimal(10, 0))", decimal_values, - "decimal,datetime64"), - ("decimal_18_6", "decimal(18, 6) NOT NULL", "Decimal(18, 6)", decimal_values, - "decimal,datetime64"), - ("decimal_38_6", "decimal(38, 6) NOT NULL", "Decimal(38, 6)", decimal_values, - "decimal,datetime64"), + pytest.param("decimal_default", "decimal NOT NULL", "Decimal(10, 0)", decimal_values, + "decimal,datetime64", id="decimal_1"), + pytest.param("decimal_default_nullable", "decimal", "Nullable(Decimal(10, 0))", decimal_values, + "decimal,datetime64", id="decimal_2"), + pytest.param("decimal_18_6", "decimal(18, 6) NOT NULL", "Decimal(18, 6)", decimal_values, + "decimal,datetime64", id="decimal_3"), + pytest.param("decimal_38_6", "decimal(38, 6) NOT NULL", "Decimal(38, 6)", decimal_values, + "decimal,datetime64", id="decimal_4"), # Due to python DB driver roundtrip MySQL timestamp and datetime values # are printed with 6 digits after decimal point, so to simplify tests a bit, # we only validate precision of 0 and 6. - ("timestamp_default", "timestamp", "DateTime", timestamp_values, "decimal,datetime64"), - ("timestamp_6", "timestamp(6)", "DateTime64(6)", timestamp_values, "decimal,datetime64"), - ("datetime_default", "DATETIME NOT NULL", "DateTime64(0)", timestamp_values, - "decimal,datetime64"), - ("datetime_6", "DATETIME(6) NOT NULL", "DateTime64(6)", timestamp_values, - "decimal,datetime64"), + pytest.param("timestamp_default", "timestamp", "DateTime", timestamp_values, "decimal,datetime64", id="timestamp_default"), + pytest.param("timestamp_6", "timestamp(6)", "DateTime64(6)", timestamp_values, "decimal,datetime64", id="timestamp_6"), + pytest.param("datetime_default", "DATETIME NOT NULL", "DateTime64(0)", timestamp_values, + "decimal,datetime64", id="datetime_default"), + pytest.param("datetime_6", "DATETIME(6) NOT NULL", "DateTime64(6)", timestamp_values, + "decimal,datetime64", id="datetime_6_1"), # right now precision bigger than 39 is not supported by ClickHouse's Decimal, hence fall back to String - ("decimal_40_6", "decimal(40, 6) NOT NULL", "String", decimal_values, - "decimal,datetime64"), - ("decimal_18_6", "decimal(18, 6) NOT NULL", "String", decimal_values, "datetime64"), - ("decimal_18_6", "decimal(18, 6) NOT NULL", "String", decimal_values, ""), - ("datetime_6", "DATETIME(6) NOT NULL", "DateTime", timestamp_values_no_subsecond, - "decimal"), - ("datetime_6", "DATETIME(6) NOT NULL", "DateTime", timestamp_values_no_subsecond, ""), + pytest.param("decimal_40_6", "decimal(40, 6) NOT NULL", "String", decimal_values, + "decimal,datetime64", id="decimal_40_6"), + pytest.param("decimal_18_6", "decimal(18, 6) NOT NULL", "String", decimal_values, "datetime64", id="decimal_18_6_1"), + pytest.param("decimal_18_6", "decimal(18, 6) NOT NULL", "String", decimal_values, "", id="decimal_18_6_2"), + pytest.param("datetime_6", "DATETIME(6) NOT NULL", "DateTime", timestamp_values_no_subsecond, + "decimal", id="datetime_6_2"), + pytest.param("datetime_6", "DATETIME(6) NOT NULL", "DateTime", timestamp_values_no_subsecond, "", id="datetime_6_3"), ]) def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, mysql_values, setting_mysql_datatypes_support_level): @@ -310,7 +311,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m else: return [do_execute(q) for q in query] - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, port=started_cluster.mysql_port)) as mysql_node: execute_query(mysql_node, [ "DROP DATABASE IF EXISTS ${mysql_db}", "CREATE DATABASE ${mysql_db} DEFAULT CHARACTER SET 'utf8'", @@ -327,7 +328,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m # MySQL TABLE ENGINE execute_query(clickhouse_node, [ "DROP TABLE IF EXISTS ${ch_mysql_table};", - "CREATE TABLE ${ch_mysql_table} (value ${expected_ch_type}) ENGINE = MySQL('mysql1:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", + "CREATE TABLE ${ch_mysql_table} (value ${expected_ch_type}) ENGINE = MySQL('mysql57:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", ], settings=clickhouse_query_settings) # Validate type @@ -347,7 +348,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m # MySQL DATABASE ENGINE execute_query(clickhouse_node, [ "DROP DATABASE IF EXISTS ${ch_mysql_db}", - "CREATE DATABASE ${ch_mysql_db} ENGINE = MySQL('mysql1:3306', '${mysql_db}', 'root', 'clickhouse')" + "CREATE DATABASE ${ch_mysql_db} ENGINE = MySQL('mysql57:3306', '${mysql_db}', 'root', 'clickhouse')" ], settings=clickhouse_query_settings) # Validate type @@ -368,7 +369,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m # Validate type assert \ execute_query(clickhouse_node, - "SELECT toTypeName(value) FROM mysql('mysql1:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse') LIMIT 1", + "SELECT toTypeName(value) FROM mysql('mysql57:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse') LIMIT 1", settings=clickhouse_query_settings) \ == \ expected_ch_type @@ -378,5 +379,5 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m execute_query(mysql_node, "SELECT value FROM ${mysql_db}.${table_name}") \ == \ execute_query(clickhouse_node, - "SELECT value FROM mysql('mysql1:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", + "SELECT value FROM mysql('mysql57:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", settings=clickhouse_query_settings) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 43daeebeaf5..f2d3f46b9bc 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -5,6 +5,7 @@ import math import os import time +import logging import docker import pymysql.connections import pytest @@ -17,115 +18,87 @@ DOCKER_COMPOSE_PATH = get_docker_compose_path() cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', main_configs=["configs/log_conf.xml", "configs/ssl_conf.xml", "configs/mysql.xml", "configs/dhparam.pem", "configs/server.crt", "configs/server.key"], - user_configs=["configs/users.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) + user_configs=["configs/users.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}, with_mysql_client=True) server_port = 9001 - @pytest.fixture(scope="module") -def server_address(): +def started_cluster(): cluster.start() try: - yield cluster.get_instance_ip('node') + yield cluster finally: cluster.shutdown() -@pytest.fixture(scope='module') -def mysql_client(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml') - run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_mysql1_1') - - -@pytest.fixture(scope='module') -def mysql_server(mysql_client): - """Return MySQL container when it is healthy. - - :type mysql_client: Container - :rtype: Container - """ - retries = 30 - for i in range(retries): - info = mysql_client.client.api.inspect_container(mysql_client.name) - if info['State']['Health']['Status'] == 'healthy': - break - time.sleep(1) - else: - raise Exception('Mysql server has not started in %d seconds.' % retries) - - return mysql_client - - @pytest.fixture(scope='module') def golang_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_golang1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_golang1_1') @pytest.fixture(scope='module') def php_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_php1_1') + ['docker-compose', '--env-file', cluster.instances["node"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_php1_1') @pytest.fixture(scope='module') def nodejs_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1') + ['docker-compose', '--env-file', cluster.instances["node"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_mysqljs1_1') @pytest.fixture(scope='module') def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_java1_1') + ['docker-compose', '--env-file', cluster.instances["node"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_java1_1') -def test_mysql_client(mysql_client, server_address): +def test_mysql_client(started_cluster): # type: (Container, str) -> None - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u user_with_double_sha1 --password=abacaba -e "SELECT 1;" - '''.format(host=server_address, port=server_port), demux=True) - + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) + logging.debug(f"test_mysql_client code:{code} stdout:{stdout}, stderr:{stderr}") assert stdout.decode() == '\n'.join(['1', '1', '']) - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "SELECT 1 as a;" -e "SELECT 'тест' as b;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert stdout.decode() == '\n'.join(['a', '1', 'b', 'тест', '']) - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=abc -e "select 1 as a;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert stderr.decode() == 'mysql: [Warning] Using a password on the command line interface can be insecure.\n' \ 'ERROR 516 (00000): default: Authentication failed: password is incorrect or there is no user with such name\n' - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "use system;" -e "select count(*) from (select name from tables limit 1);" -e "use system2;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert stdout.decode() == 'count()\n1\n' assert stderr[0:182].decode() == "mysql: [Warning] Using a password on the command line interface can be insecure.\n" \ "ERROR 81 (00000) at line 1: Code: 81, e.displayText() = DB::Exception: Database system2 doesn't exist" - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "CREATE DATABASE x;" -e "USE x;" @@ -137,127 +110,127 @@ def test_mysql_client(mysql_client, server_address): -e "CREATE TEMPORARY TABLE tmp (tmp_column UInt32);" -e "INSERT INTO tmp VALUES (0), (1);" -e "SELECT * FROM tmp ORDER BY tmp_column;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert stdout.decode() == '\n'.join(['column', '0', '0', '1', '1', '5', '5', 'tmp_column', '0', '1', '']) -def test_mysql_client_exception(mysql_client, server_address): +def test_mysql_client_exception(started_cluster): # Poco exception. - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "CREATE TABLE default.t1_remote_mysql AS mysql('127.0.0.1:10086','default','t1_local','default','');" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert stderr[0:258].decode() == "mysql: [Warning] Using a password on the command line interface can be insecure.\n" \ "ERROR 1000 (00000) at line 1: Poco::Exception. Code: 1000, e.code() = 0, e.displayText() = Exception: Connections to all replicas failed: default@127.0.0.1:10086 as user default" -def test_mysql_affected_rows(mysql_client, server_address): - code, (stdout, stderr) = mysql_client.exec_run(''' +def test_mysql_affected_rows(started_cluster): + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "CREATE TABLE IF NOT EXISTS default.t1 (n UInt64) ENGINE MergeTree() ORDER BY tuple();" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql -vvv --protocol tcp -h {host} -P {port} default -u default --password=123 -e "INSERT INTO default.t1(n) VALUES(1);" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert "1 row affected" in stdout.decode() - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql -vvv --protocol tcp -h {host} -P {port} default -u default --password=123 -e "INSERT INTO default.t1(n) SELECT * FROM numbers(1000)" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert "1000 rows affected" in stdout.decode() - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "DROP TABLE default.t1;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 -def test_mysql_replacement_query(mysql_client, server_address): +def test_mysql_replacement_query(started_cluster): # SHOW TABLE STATUS LIKE. - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "show table status like 'xx';" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 # SHOW VARIABLES. - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "show variables;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 # KILL QUERY. - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "kill query 0;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "kill query where query_id='mysql:0';" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 # SELECT DATABASE(). - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "select database();" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == 'DATABASE()\ndefault\n' - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "select DATABASE();" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == 'DATABASE()\ndefault\n' -def test_mysql_explain(mysql_client, server_address): +def test_mysql_explain(started_cluster): # EXPLAIN SELECT 1 - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "EXPLAIN SELECT 1;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 # EXPLAIN AST SELECT 1 - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "EXPLAIN AST SELECT 1;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 # EXPLAIN PLAN SELECT 1 - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "EXPLAIN PLAN SELECT 1;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 # EXPLAIN PIPELINE graph=1 SELECT 1 - code, (stdout, stderr) = mysql_client.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "EXPLAIN PIPELINE graph=1 SELECT 1;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 -def test_mysql_federated(mysql_server, server_address): +def test_mysql_federated(started_cluster): # For some reason it occasionally fails without retries. retries = 100 for try_num in range(retries): @@ -272,14 +245,14 @@ def test_mysql_federated(mysql_server, server_address): or "MySQL server has gone away" in stderr or "Server shutdown in progress" in stderr) - code, (stdout, stderr) = mysql_server.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql -e "DROP SERVER IF EXISTS clickhouse;" -e "CREATE SERVER clickhouse FOREIGN DATA WRAPPER mysql OPTIONS (USER 'default', PASSWORD '123', HOST '{host}', PORT {port}, DATABASE 'mysql_federated');" -e "DROP DATABASE IF EXISTS mysql_federated;" -e "CREATE DATABASE mysql_federated;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) if code != 0: print(("stdout", stdout)) @@ -289,11 +262,11 @@ def test_mysql_federated(mysql_server, server_address): continue assert code == 0 - code, (stdout, stderr) = mysql_server.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql -e "CREATE TABLE mysql_federated.test(`col` int UNSIGNED) ENGINE=FEDERATED CONNECTION='clickhouse';" -e "SELECT * FROM mysql_federated.test ORDER BY col;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) if code != 0: print(("stdout", stdout)) @@ -305,11 +278,11 @@ def test_mysql_federated(mysql_server, server_address): assert stdout.decode() == '\n'.join(['col', '0', '1', '5', '']) - code, (stdout, stderr) = mysql_server.exec_run(''' + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql -e "INSERT INTO mysql_federated.test VALUES (0), (1), (5);" -e "SELECT * FROM mysql_federated.test ORDER BY col;" - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) if code != 0: print(("stdout", stdout)) @@ -322,8 +295,8 @@ def test_mysql_federated(mysql_server, server_address): assert stdout.decode() == '\n'.join(['col', '0', '0', '1', '1', '5', '5', '']) -def test_mysql_set_variables(mysql_client, server_address): - code, (stdout, stderr) = mysql_client.exec_run(''' +def test_mysql_set_variables(started_cluster): + code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e " @@ -335,12 +308,12 @@ def test_mysql_set_variables(mysql_client, server_address): SET @@wait_timeout = 2147483; SET SESSION TRANSACTION ISOLATION LEVEL READ; " - '''.format(host=server_address, port=server_port), demux=True) + '''.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 -def test_python_client(server_address): - client = pymysql.connections.Connection(host=server_address, user='user_with_double_sha1', password='abacaba', +def test_python_client(started_cluster): + client = pymysql.connections.Connection(host=started_cluster.get_instance_ip('node'), user='user_with_double_sha1', password='abacaba', database='default', port=server_port) with pytest.raises(pymysql.InternalError) as exc_info: @@ -354,13 +327,13 @@ def test_python_client(server_address): assert cursor.fetchall() == [{'a': 1, 'b': 'тест'}] with pytest.raises(pymysql.InternalError) as exc_info: - pymysql.connections.Connection(host=server_address, user='default', password='abacab', database='default', + pymysql.connections.Connection(host=started_cluster.get_instance_ip('node'), user='default', password='abacab', database='default', port=server_port) assert exc_info.value.args == ( 516, 'default: Authentication failed: password is incorrect or there is no user with such name') - client = pymysql.connections.Connection(host=server_address, user='default', password='123', database='default', + client = pymysql.connections.Connection(host=started_cluster.get_instance_ip('node'), user='default', password='123', database='default', port=server_port) with pytest.raises(pymysql.InternalError) as exc_info: @@ -390,80 +363,80 @@ def test_python_client(server_address): assert cursor.fetchall() == [{'a': 1}, {'a': 1}, {'a': 3}, {'a': 4}] -def test_golang_client(server_address, golang_container): +def test_golang_client(started_cluster, golang_container): # type: (str, Container) -> None with open(os.path.join(SCRIPT_DIR, 'golang.reference'), 'rb') as fp: reference = fp.read() code, (stdout, stderr) = golang_container.exec_run( './main --host {host} --port {port} --user default --password 123 --database ' - 'abc'.format(host=server_address, port=server_port), demux=True) + 'abc'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 1 assert stderr.decode() == "Error 81: Database abc doesn't exist\n" code, (stdout, stderr) = golang_container.exec_run( './main --host {host} --port {port} --user default --password 123 --database ' - 'default'.format(host=server_address, port=server_port), demux=True) + 'default'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout == reference code, (stdout, stderr) = golang_container.exec_run( './main --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database ' - 'default'.format(host=server_address, port=server_port), demux=True) + 'default'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout == reference -def test_php_client(server_address, php_container): +def test_php_client(started_cluster, php_container): # type: (str, Container) -> None code, (stdout, stderr) = php_container.exec_run( - 'php -f test.php {host} {port} default 123'.format(host=server_address, port=server_port), demux=True) + 'php -f test.php {host} {port} default 123'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == 'tables\n' code, (stdout, stderr) = php_container.exec_run( - 'php -f test_ssl.php {host} {port} default 123'.format(host=server_address, port=server_port), demux=True) + 'php -f test_ssl.php {host} {port} default 123'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == 'tables\n' code, (stdout, stderr) = php_container.exec_run( - 'php -f test.php {host} {port} user_with_double_sha1 abacaba'.format(host=server_address, port=server_port), + 'php -f test.php {host} {port} user_with_double_sha1 abacaba'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == 'tables\n' code, (stdout, stderr) = php_container.exec_run( - 'php -f test_ssl.php {host} {port} user_with_double_sha1 abacaba'.format(host=server_address, port=server_port), + 'php -f test_ssl.php {host} {port} user_with_double_sha1 abacaba'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == 'tables\n' -def test_mysqljs_client(server_address, nodejs_container): +def test_mysqljs_client(started_cluster, nodejs_container): code, (_, stderr) = nodejs_container.exec_run( - 'node test.js {host} {port} user_with_sha256 abacaba'.format(host=server_address, port=server_port), demux=True) + 'node test.js {host} {port} user_with_sha256 abacaba'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 1 assert 'MySQL is requesting the sha256_password authentication method, which is not supported.' in stderr.decode() code, (_, stderr) = nodejs_container.exec_run( - 'node test.js {host} {port} user_with_empty_password ""'.format(host=server_address, port=server_port), + 'node test.js {host} {port} user_with_empty_password ""'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 code, (_, _) = nodejs_container.exec_run( - 'node test.js {host} {port} user_with_double_sha1 abacaba'.format(host=server_address, port=server_port), + 'node test.js {host} {port} user_with_double_sha1 abacaba'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 code, (_, _) = nodejs_container.exec_run( - 'node test.js {host} {port} user_with_empty_password 123'.format(host=server_address, port=server_port), + 'node test.js {host} {port} user_with_empty_password 123'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 1 -def test_java_client(server_address, java_container): +def test_java_client(started_cluster, java_container): # type: (str, Container) -> None with open(os.path.join(SCRIPT_DIR, 'java.reference')) as fp: reference = fp.read() @@ -471,33 +444,33 @@ def test_java_client(server_address, java_container): # database not exists exception. code, (stdout, stderr) = java_container.exec_run( 'java JavaConnectorTest --host {host} --port {port} --user user_with_empty_password --database ' - 'abc'.format(host=server_address, port=server_port), demux=True) + 'abc'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 1 # empty password passed. code, (stdout, stderr) = java_container.exec_run( 'java JavaConnectorTest --host {host} --port {port} --user user_with_empty_password --database ' - 'default'.format(host=server_address, port=server_port), demux=True) + 'default'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == reference # non-empty password passed. code, (stdout, stderr) = java_container.exec_run( 'java JavaConnectorTest --host {host} --port {port} --user default --password 123 --database ' - 'default'.format(host=server_address, port=server_port), demux=True) + 'default'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == reference # double-sha1 password passed. code, (stdout, stderr) = java_container.exec_run( 'java JavaConnectorTest --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database ' - 'default'.format(host=server_address, port=server_port), demux=True) + 'default'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) assert code == 0 assert stdout.decode() == reference -def test_types(server_address): - client = pymysql.connections.Connection(host=server_address, user='default', password='123', database='default', +def test_types(started_cluster): + client = pymysql.connections.Connection(host=started_cluster.get_instance_ip('node'), user='default', password='123', database='default', port=server_port) cursor = client.cursor(pymysql.cursors.DictCursor) diff --git a/tests/integration/test_non_default_compression/configs/allow_experimental_codecs.xml b/tests/integration/test_non_default_compression/configs/allow_experimental_codecs.xml new file mode 100644 index 00000000000..2c7de77cd37 --- /dev/null +++ b/tests/integration/test_non_default_compression/configs/allow_experimental_codecs.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_non_default_compression/test.py b/tests/integration/test_non_default_compression/test.py index 03210e47081..0cfffd28e12 100644 --- a/tests/integration/test_non_default_compression/test.py +++ b/tests/integration/test_non_default_compression/test.py @@ -17,7 +17,8 @@ node4 = cluster.add_instance('node4', user_configs=['configs/enable_uncompressed node5 = cluster.add_instance('node5', main_configs=['configs/zstd_compression_by_default.xml'], user_configs=['configs/enable_uncompressed_cache.xml', 'configs/allow_suspicious_codecs.xml']) - +node6 = cluster.add_instance('node6', main_configs=['configs/allow_experimental_codecs.xml'], + user_configs=['configs/allow_suspicious_codecs.xml']) @pytest.fixture(scope="module") def start_cluster(): diff --git a/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml b/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml index 19eed6ebd6a..6aad3ad9917 100644 --- a/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml +++ b/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml @@ -4,7 +4,7 @@ clickhouse.test_table
- DSN=postgresql_odbc; + DSN=postgresql_odbc postgres
diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 25668737885..ee1332f7460 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -3,19 +3,27 @@ import time import psycopg2 import pymysql.cursors import pytest +import logging +import os.path + from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT from multiprocessing.dummy import Pool cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, +node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, with_postgres=True, main_configs=['configs/openssl.xml', 'configs/odbc_logging.xml', 'configs/enable_dictionaries.xml', 'configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml', 'configs/dictionaries/sqlite3_odbc_cached_dictionary.xml', 'configs/dictionaries/postgres_odbc_hashed_dictionary.xml'], stay_alive=True) + +drop_table_sql_template = """ + DROP TABLE IF EXISTS `clickhouse`.`{}` + """ + create_table_sql_template = """ CREATE TABLE `clickhouse`.`{}` ( `id` int(11) NOT NULL, @@ -33,27 +41,50 @@ def skip_test_msan(instance): def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308) - return conn - + errors = [] + conn = None + for _ in range(15): + try: + if conn is None: + conn = pymysql.connect(user='root', password='clickhouse', host=cluster.mysql_ip, port=cluster.mysql_port) + else: + conn.ping(reconnect=True) + logging.debug(f"MySQL Connection establised: {cluster.mysql_ip}:{cluster.mysql_port}") + return conn + except Exception as e: + errors += [str(e)] + time.sleep(1) + + raise Exception("Connection not establised, {}".format(errors)) + def create_mysql_db(conn, name): with conn.cursor() as cursor: - cursor.execute( - "CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) + cursor.execute("CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) def create_mysql_table(conn, table_name): with conn.cursor() as cursor: + cursor.execute(drop_table_sql_template.format(table_name)) cursor.execute(create_table_sql_template.format(table_name)) -def get_postgres_conn(): - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" - conn = psycopg2.connect(conn_string) - conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) - conn.autocommit = True - return conn +def get_postgres_conn(started_cluster): + conn_string = "host={} port={} user='postgres' password='mysecretpassword'".format(started_cluster.postgres_ip, started_cluster.postgres_port) + errors = [] + for _ in range(15): + try: + conn = psycopg2.connect(conn_string) + logging.debug("Postgre Connection establised: {}".format(conn_string)) + conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) + conn.autocommit = True + return conn + except Exception as e: + errors += [str(e)] + time.sleep(1) + + raise Exception("Postgre connection not establised DSN={}, {}".format(conn_string, errors)) def create_postgres_db(conn, name): @@ -90,7 +121,7 @@ def started_cluster(): create_mysql_db(mysql_conn, 'clickhouse') print("mysql database created") - postgres_conn = get_postgres_conn() + postgres_conn = get_postgres_conn(cluster) print("postgres connection received") create_postgres_db(postgres_conn, 'clickhouse') @@ -130,7 +161,7 @@ def test_mysql_simple_select_works(started_cluster): settings={"external_table_functions_use_nulls": "0"}) == '0\n511\n' node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nullable(UInt32)) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nullable(UInt32)) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node1.query( @@ -303,7 +334,7 @@ def test_sqlite_odbc_cached_dictionary(started_cluster): def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) cursor = conn.cursor() cursor.execute("truncate table clickhouse.test_table") cursor.execute("insert into clickhouse.test_table values(1, 'hello'),(2, 'world')") @@ -315,13 +346,13 @@ def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) cursor = conn.cursor() cursor.execute("truncate table clickhouse.test_table") cursor.execute("insert into clickhouse.test_table values(3, 'xxx')") for i in range(100): try: - node1.query("system reload dictionary postgres_odbc_hashed", timeout=5) + node1.query("system reload dictionary postgres_odbc_hashed", timeout=15) except Exception as ex: assert False, "Exception occured -- odbc-bridge hangs: " + str(ex) @@ -331,7 +362,7 @@ def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): def test_postgres_insert(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) conn.cursor().execute("truncate table clickhouse.test_table") # Also test with Servername containing '.' and '-' symbols (defined in @@ -342,12 +373,13 @@ def test_postgres_insert(started_cluster): "create table pg_insert (column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')") node1.query("insert into pg_insert values (1, 'hello'), (2, 'world')") assert node1.query("select * from pg_insert") == '1\thello\n2\tworld\n' - node1.query("insert into table function odbc('DSN=postgresql_odbc;', 'clickhouse', 'test_table') format CSV 3,test") + node1.query("insert into table function odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table') format CSV 3,test") node1.query( - "insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table') select number, 's' || toString(number) from numbers (4, 7)") + "insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')" \ + " select number, 's' || toString(number) from numbers (4, 7)") assert node1.query("select sum(column1), count(column1) from pg_insert") == "55\t10\n" assert node1.query( - "select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc;', 'clickhouse', 'test_table')))") == "55\t10\n" + "select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table')))") == "55\t10\n" def test_bridge_dies_with_parent(started_cluster): @@ -394,7 +426,7 @@ def test_bridge_dies_with_parent(started_cluster): def test_odbc_postgres_date_data_type(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn(); + conn = get_postgres_conn(started_cluster); cursor = conn.cursor() cursor.execute("CREATE TABLE IF NOT EXISTS clickhouse.test_date (column1 integer, column2 date)") @@ -418,7 +450,7 @@ def test_odbc_postgres_date_data_type(started_cluster): def test_odbc_postgres_conversions(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) cursor = conn.cursor() cursor.execute( @@ -460,7 +492,7 @@ def test_odbc_postgres_conversions(started_cluster): def test_odbc_cyrillic_with_varchar(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) cursor = conn.cursor() cursor.execute("DROP TABLE IF EXISTS clickhouse.test_cyrillic") @@ -482,7 +514,7 @@ def test_odbc_cyrillic_with_varchar(started_cluster): def test_many_connections(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) cursor = conn.cursor() cursor.execute('DROP TABLE IF EXISTS clickhouse.test_pg_table') @@ -506,7 +538,7 @@ def test_many_connections(started_cluster): def test_concurrent_queries(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) cursor = conn.cursor() node1.query(''' @@ -547,7 +579,7 @@ def test_concurrent_queries(started_cluster): def test_odbc_long_column_names(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn(); + conn = get_postgres_conn(started_cluster); cursor = conn.cursor() column_name = "column" * 8 @@ -581,7 +613,7 @@ def test_odbc_long_column_names(started_cluster): def test_odbc_long_text(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) cursor = conn.cursor() cursor.execute("drop table if exists clickhouse.test_long_text") cursor.execute("create table clickhouse.test_long_text(flen int, field1 text)"); diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index b5facb5f4b2..f8051771d09 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -42,7 +42,7 @@ def test_partition_simple(partition_table_simple): def exec_bash(cmd): - cmd = '/bin/bash -c "{}"'.format(cmd.replace('"', '\\"')) + cmd = ["/bin/bash", "-c", cmd.replace('"', '\\"')] return instance.exec_in_container(cmd) diff --git a/tests/integration/test_postgresql_database_engine/test.py b/tests/integration/test_postgresql_database_engine/test.py index b3f7c0fa9af..e89f1109c3a 100644 --- a/tests/integration/test_postgresql_database_engine/test.py +++ b/tests/integration/test_postgresql_database_engine/test.py @@ -14,11 +14,11 @@ postgres_table_template = """ id Integer NOT NULL, value Integer, PRIMARY KEY (id)) """ -def get_postgres_conn(database=False): +def get_postgres_conn(cluster, database=False): if database == True: - conn_string = "host='localhost' dbname='test_database' user='postgres' password='mysecretpassword'" + conn_string = f"host={cluster.postgres_ip} port={cluster.postgres_port} dbname='test_database' user='postgres' password='mysecretpassword'" else: - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn_string = f"host={cluster.postgres_ip} port={cluster.postgres_port} user='postgres' password='mysecretpassword'" conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) conn.autocommit = True @@ -36,7 +36,7 @@ def create_postgres_table(cursor, table_name): def started_cluster(): try: cluster.start() - conn = get_postgres_conn() + conn = get_postgres_conn(cluster) cursor = conn.cursor() create_postgres_db(cursor, 'test_database') yield cluster @@ -47,7 +47,7 @@ def started_cluster(): def test_postgres_database_engine_with_postgres_ddl(started_cluster): # connect to database as well - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( @@ -68,7 +68,7 @@ def test_postgres_database_engine_with_postgres_ddl(started_cluster): def test_postgresql_database_engine_with_clickhouse_ddl(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( @@ -94,7 +94,7 @@ def test_postgresql_database_engine_with_clickhouse_ddl(started_cluster): def test_postgresql_database_engine_queries(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( @@ -114,7 +114,7 @@ def test_postgresql_database_engine_queries(started_cluster): def test_get_create_table_query_with_multidim_arrays(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( @@ -147,7 +147,7 @@ def test_get_create_table_query_with_multidim_arrays(started_cluster): def test_postgresql_database_engine_table_cache(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index c63896eb196..7bea9569880 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -38,10 +38,10 @@ def server_address(): @pytest.fixture(scope='module') def psql_client(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.from_env().containers.get(cluster.project_name + '_psql_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_psql_1') @pytest.fixture(scope='module') @@ -62,10 +62,10 @@ def psql_server(psql_client): @pytest.fixture(scope='module') def java_container(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql_java_client.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql_java_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.from_env().containers.get(cluster.project_name + '_java_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_java_1') def test_psql_is_ready(psql_server): diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index 3d65a489610..52f14c99b4a 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -5,9 +5,6 @@ import pytest import requests from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): @@ -60,7 +57,7 @@ def get_minio_stat(cluster): "rx_bytes": 0, "tx_bytes": 0, } - stat = requests.get(url="http://{}:{}/minio/prometheus/metrics".format("localhost", cluster.minio_port)).text.split( + stat = requests.get(url="http://{}:{}/minio/prometheus/metrics".format(cluster.minio_ip, cluster.minio_port)).text.split( "\n") for line in stat: x = re.search("s3_requests_total(\{.*\})?\s(\d+)(\s.*)?", line) diff --git a/tests/integration/test_quota/test.py b/tests/integration/test_quota/test.py index 0f59ae27583..5f43d581178 100644 --- a/tests/integration/test_quota/test.py +++ b/tests/integration/test_quota/test.py @@ -61,7 +61,7 @@ def copy_quota_xml(local_file_name, reload_immediately=True): def started_cluster(): try: cluster.start() - + instance.query("DROP TABLE IF EXISTS test_table") instance.query("CREATE TABLE test_table(x UInt32) ENGINE = MergeTree ORDER BY tuple()") instance.query("INSERT INTO test_table SELECT number FROM numbers(50)") diff --git a/tests/integration/test_redirect_url_storage/test.py b/tests/integration/test_redirect_url_storage/test.py index d899d316abc..7b9adba32e7 100644 --- a/tests/integration/test_redirect_url_storage/test.py +++ b/tests/integration/test_redirect_url_storage/test.py @@ -17,8 +17,10 @@ def started_cluster(): def test_url_without_redirect(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") - assert started_cluster.hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" # access datanode port directly node1.query( @@ -27,12 +29,14 @@ def test_url_without_redirect(started_cluster): def test_url_with_globs(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage_1_1", "1\n") - started_cluster.hdfs_api.write_data("/simple_storage_1_2", "2\n") - started_cluster.hdfs_api.write_data("/simple_storage_1_3", "3\n") - started_cluster.hdfs_api.write_data("/simple_storage_2_1", "4\n") - started_cluster.hdfs_api.write_data("/simple_storage_2_2", "5\n") - started_cluster.hdfs_api.write_data("/simple_storage_2_3", "6\n") + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage_1_1", "1\n") + hdfs_api.write_data("/simple_storage_1_2", "2\n") + hdfs_api.write_data("/simple_storage_1_3", "3\n") + hdfs_api.write_data("/simple_storage_2_1", "4\n") + hdfs_api.write_data("/simple_storage_2_2", "5\n") + hdfs_api.write_data("/simple_storage_2_3", "6\n") result = node1.query( "select * from url('http://hdfs1:50075/webhdfs/v1/simple_storage_{1..2}_{1..3}?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'data String') as data order by data") @@ -40,12 +44,14 @@ def test_url_with_globs(started_cluster): def test_url_with_globs_and_failover(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage_1_1", "1\n") - started_cluster.hdfs_api.write_data("/simple_storage_1_2", "2\n") - started_cluster.hdfs_api.write_data("/simple_storage_1_3", "3\n") - started_cluster.hdfs_api.write_data("/simple_storage_3_1", "4\n") - started_cluster.hdfs_api.write_data("/simple_storage_3_2", "5\n") - started_cluster.hdfs_api.write_data("/simple_storage_3_3", "6\n") + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage_1_1", "1\n") + hdfs_api.write_data("/simple_storage_1_2", "2\n") + hdfs_api.write_data("/simple_storage_1_3", "3\n") + hdfs_api.write_data("/simple_storage_3_1", "4\n") + hdfs_api.write_data("/simple_storage_3_2", "5\n") + hdfs_api.write_data("/simple_storage_3_3", "6\n") result = node1.query( "select * from url('http://hdfs1:50075/webhdfs/v1/simple_storage_{0|1|2|3}_{1..3}?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'data String') as data order by data") @@ -53,8 +59,10 @@ def test_url_with_globs_and_failover(started_cluster): def test_url_with_redirect_not_allowed(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") - assert started_cluster.hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" # access proxy port without allowing redirects node1.query( @@ -64,8 +72,10 @@ def test_url_with_redirect_not_allowed(started_cluster): def test_url_with_redirect_allowed(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") - assert started_cluster.hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" # access proxy port with allowing redirects # http://localhost:50070/webhdfs/v1/b?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0 diff --git a/tests/integration/test_reload_auxiliary_zookeepers/configs/zookeeper.xml b/tests/integration/test_reload_auxiliary_zookeepers/configs/zookeeper.xml deleted file mode 100644 index 8157cc1d30b..00000000000 --- a/tests/integration/test_reload_auxiliary_zookeepers/configs/zookeeper.xml +++ /dev/null @@ -1,17 +0,0 @@ - - - - zoo1 - 2181 - - - zoo2 - 2181 - - - zoo3 - 2181 - - 2000 - - diff --git a/tests/integration/test_reload_auxiliary_zookeepers/test.py b/tests/integration/test_reload_auxiliary_zookeepers/test.py index 1b14408bc12..e01331269d0 100644 --- a/tests/integration/test_reload_auxiliary_zookeepers/test.py +++ b/tests/integration/test_reload_auxiliary_zookeepers/test.py @@ -6,7 +6,7 @@ from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException from helpers.test_tools import assert_eq_with_retry -cluster = ClickHouseCluster(__file__, zookeeper_config_path="configs/zookeeper.xml") +cluster = ClickHouseCluster(__file__) node = cluster.add_instance("node", with_zookeeper=True) @@ -60,10 +60,12 @@ def test_reload_auxiliary_zookeepers(start_cluster): """ - node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) + node.replace_config("/etc/clickhouse-server/conf.d/zookeeper_config.xml", new_config) node.query("SYSTEM RELOAD CONFIG") + time.sleep(5) + node.query( "ALTER TABLE simple2 FETCH PARTITION '2020-08-27' FROM 'zookeeper2:/clickhouse/tables/0/simple';" ) @@ -79,8 +81,10 @@ def test_reload_auxiliary_zookeepers(start_cluster): 2000 """ - node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) + node.replace_config("/etc/clickhouse-server/conf.d/zookeeper_config.xml", new_config) node.query("SYSTEM RELOAD CONFIG") + time.sleep(5) + with pytest.raises(QueryRuntimeException): node.query( "ALTER TABLE simple2 FETCH PARTITION '2020-08-27' FROM 'zookeeper2:/clickhouse/tables/0/simple';" diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 70d779ea737..4bcad7de16f 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -230,26 +230,27 @@ def test_recover_staled_replica(started_cluster): with PartitionManager() as pm: pm.drop_instance_zk_connections(dummy_node) dummy_node.query_and_get_error("RENAME TABLE recover.t1 TO recover.m1") - main_node.query("RENAME TABLE recover.t1 TO recover.m1", settings=settings) - main_node.query("ALTER TABLE recover.mt1 ADD COLUMN m int", settings=settings) - main_node.query("ALTER TABLE recover.rmt1 ADD COLUMN m int", settings=settings) - main_node.query("RENAME TABLE recover.rmt3 TO recover.rmt4", settings=settings) - main_node.query("DROP TABLE recover.rmt5", settings=settings) - main_node.query("DROP DICTIONARY recover.d2", settings=settings) - main_node.query("CREATE DICTIONARY recover.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " + + main_node.query_with_retry("RENAME TABLE recover.t1 TO recover.m1", settings=settings) + main_node.query_with_retry("ALTER TABLE recover.mt1 ADD COLUMN m int", settings=settings) + main_node.query_with_retry("ALTER TABLE recover.rmt1 ADD COLUMN m int", settings=settings) + main_node.query_with_retry("RENAME TABLE recover.rmt3 TO recover.rmt4", settings=settings) + main_node.query_with_retry("DROP TABLE recover.rmt5", settings=settings) + main_node.query_with_retry("DROP DICTIONARY recover.d2", settings=settings) + main_node.query_with_retry("CREATE DICTIONARY recover.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB 'recover')) " "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT());", settings=settings) - inner_table = ".inner_id." + dummy_node.query("SELECT uuid FROM system.tables WHERE database='recover' AND name='mv1'").strip() - main_node.query("ALTER TABLE recover.`{}` MODIFY COLUMN n int DEFAULT 42".format(inner_table), settings=settings) - main_node.query("ALTER TABLE recover.mv1 MODIFY QUERY SELECT m FROM recover.rmt1".format(inner_table), settings=settings) - main_node.query("RENAME TABLE recover.mv2 TO recover.mv3".format(inner_table), settings=settings) + inner_table = ".inner_id." + dummy_node.query_with_retry("SELECT uuid FROM system.tables WHERE database='recover' AND name='mv1'").strip() + main_node.query_with_retry("ALTER TABLE recover.`{}` MODIFY COLUMN n int DEFAULT 42".format(inner_table), settings=settings) + main_node.query_with_retry("ALTER TABLE recover.mv1 MODIFY QUERY SELECT m FROM recover.rmt1".format(inner_table), settings=settings) + main_node.query_with_retry("RENAME TABLE recover.mv2 TO recover.mv3".format(inner_table), settings=settings) - main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings) - main_node.query("DROP TABLE recover.tmp", settings=settings) - main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings) - main_node.query("DROP TABLE recover.tmp", settings=settings) - main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings) + main_node.query_with_retry("CREATE TABLE recover.tmp AS recover.m1", settings=settings) + main_node.query_with_retry("DROP TABLE recover.tmp", settings=settings) + main_node.query_with_retry("CREATE TABLE recover.tmp AS recover.m1", settings=settings) + main_node.query_with_retry("DROP TABLE recover.tmp", settings=settings) + main_node.query_with_retry("CREATE TABLE recover.tmp AS recover.m1", settings=settings) assert main_node.query("SELECT name FROM system.tables WHERE database='recover' AND name NOT LIKE '.inner_id.%' ORDER BY name") == \ "d1\nd2\nm1\nmt1\nmt2\nmv1\nmv3\nrmt1\nrmt2\nrmt4\nt2\ntmp\n" diff --git a/tests/integration/test_replicated_fetches_bandwidth/__init__.py b/tests/integration/test_replicated_fetches_bandwidth/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_replicated_fetches_bandwidth/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml b/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml new file mode 100644 index 00000000000..566c7c07f5c --- /dev/null +++ b/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml @@ -0,0 +1,9 @@ + + + + + 5242880 + 10485760 + + + diff --git a/tests/integration/test_replicated_fetches_bandwidth/test.py b/tests/integration/test_replicated_fetches_bandwidth/test.py new file mode 100644 index 00000000000..5b8332cda16 --- /dev/null +++ b/tests/integration/test_replicated_fetches_bandwidth/test.py @@ -0,0 +1,205 @@ +#!/usr/bin/env python3 +from helpers.cluster import ClickHouseCluster +import pytest +import random +import string +from helpers.network import NetThroughput +import subprocess +import time +import statistics + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_zookeeper=True) +node2 = cluster.add_instance('node2', with_zookeeper=True) +node3 = cluster.add_instance('node3', user_configs=['configs/limit_replication_config.xml'], with_zookeeper=True) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + +def get_random_string(length): + return ''.join(random.choice(string.ascii_uppercase + string.digits) for _ in range(length)) + +def test_limited_fetch_single_table(start_cluster): + print("Limited fetches single table") + try: + for i, node in enumerate([node1, node2]): + node.query(f"CREATE TABLE limited_fetch_table(key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/limited_fetch_table', '{i}') ORDER BY tuple() PARTITION BY key SETTINGS max_replicated_fetches_network_bandwidth=10485760") + + node2.query("SYSTEM STOP FETCHES limited_fetch_table") + + for i in range(5): + node1.query("INSERT INTO limited_fetch_table SELECT {}, '{}' FROM numbers(300)".format(i, get_random_string(104857))) + + n1_net = NetThroughput(node1) + n2_net = NetThroughput(node2) + + node2.query("SYSTEM START FETCHES limited_fetch_table") + n2_fetch_speed = [] + for i in range(10): + n1_in, n1_out = n1_net.measure_speed('megabytes') + n2_in, n2_out = n2_net.measure_speed('megabytes') + print("[N1] input:", n1_in, 'MB/s', "output:", n1_out, "MB/s") + print("[N2] input:", n2_in, 'MB/s', "output:", n2_out, "MB/s") + n2_fetch_speed.append(n2_in) + time.sleep(0.5) + + median_speed = statistics.median(n2_fetch_speed) + # approximate border. Without limit we will have more than 100 MB/s for very slow builds. + assert median_speed <= 15, "We exceeded max fetch speed for more than 10MB/s. Must be around 10 (+- 5), got " + str(median_speed) + + finally: + for node in [node1, node2]: + node.query("DROP TABLE IF EXISTS limited_fetch_table SYNC") + +def test_limited_send_single_table(start_cluster): + print("Limited sends single table") + try: + for i, node in enumerate([node1, node2]): + node.query(f"CREATE TABLE limited_send_table(key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/limited_fetch_table', '{i}') ORDER BY tuple() PARTITION BY key SETTINGS max_replicated_sends_network_bandwidth=5242880") + + node2.query("SYSTEM STOP FETCHES limited_send_table") + + for i in range(5): + node1.query("INSERT INTO limited_send_table SELECT {}, '{}' FROM numbers(150)".format(i, get_random_string(104857))) + + n1_net = NetThroughput(node1) + n2_net = NetThroughput(node2) + + node2.query("SYSTEM START FETCHES limited_send_table") + n1_sends_speed = [] + for i in range(10): + n1_in, n1_out = n1_net.measure_speed('megabytes') + n2_in, n2_out = n2_net.measure_speed('megabytes') + print("[N1] input:", n1_in, 'MB/s', "output:", n1_out, "MB/s") + print("[N2] input:", n2_in, 'MB/s', "output:", n2_out, "MB/s") + n1_sends_speed.append(n1_out) + time.sleep(0.5) + + median_speed = statistics.median(n1_sends_speed) + # approximate border. Without limit we will have more than 100 MB/s for very slow builds. + assert median_speed <= 10, "We exceeded max send speed for more than 5MB/s. Must be around 5 (+- 5), got " + str(median_speed) + + finally: + for node in [node1, node2]: + node.query("DROP TABLE IF EXISTS limited_send_table SYNC") + + +def test_limited_fetches_for_server(start_cluster): + print("Limited fetches for server") + try: + for i, node in enumerate([node1, node3]): + for j in range(5): + node.query(f"CREATE TABLE limited_fetches{j}(key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/limited_fetches{j}', '{i}') ORDER BY tuple() PARTITION BY key") + + for j in range(5): + node3.query(f"SYSTEM STOP FETCHES limited_fetches{j}") + for i in range(5): + node1.query("INSERT INTO limited_fetches{} SELECT {}, '{}' FROM numbers(50)".format(j, i, get_random_string(104857))) + + n1_net = NetThroughput(node1) + n3_net = NetThroughput(node3) + + for j in range(5): + node3.query(f"SYSTEM START FETCHES limited_fetches{j}") + + n3_fetches_speed = [] + for i in range(5): + n1_in, n1_out = n1_net.measure_speed('megabytes') + n3_in, n3_out = n3_net.measure_speed('megabytes') + print("[N1] input:", n1_in, 'MB/s', "output:", n1_out, "MB/s") + print("[N3] input:", n3_in, 'MB/s', "output:", n3_out, "MB/s") + n3_fetches_speed.append(n3_in) + time.sleep(0.5) + + median_speed = statistics.median(n3_fetches_speed) + # approximate border. Without limit we will have more than 100 MB/s for very slow builds. + assert median_speed <= 15, "We exceeded max fetch speed for more than 15MB/s. Must be around 5 (+- 10), got " + str(median_speed) + + finally: + for node in [node1, node3]: + for j in range(5): + node.query(f"DROP TABLE IF EXISTS limited_fetches{j} SYNC") + + +def test_limited_sends_for_server(start_cluster): + print("Limited sends for server") + try: + for i, node in enumerate([node1, node3]): + for j in range(5): + node.query(f"CREATE TABLE limited_sends{j}(key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/limited_sends{j}', '{i}') ORDER BY tuple() PARTITION BY key") + + for j in range(5): + node1.query(f"SYSTEM STOP FETCHES limited_sends{j}") + for i in range(5): + node3.query("INSERT INTO limited_sends{} SELECT {}, '{}' FROM numbers(50)".format(j, i, get_random_string(104857))) + + n1_net = NetThroughput(node1) + n3_net = NetThroughput(node3) + + for j in range(5): + node1.query(f"SYSTEM START FETCHES limited_sends{j}") + + n3_sends_speed = [] + for i in range(5): + n1_in, n1_out = n1_net.measure_speed('megabytes') + n3_in, n3_out = n3_net.measure_speed('megabytes') + print("[N1] input:", n1_in, 'MB/s', "output:", n1_out, "MB/s") + print("[N3] input:", n3_in, 'MB/s', "output:", n3_out, "MB/s") + n3_sends_speed.append(n3_out) + time.sleep(0.5) + + median_speed = statistics.median(n3_sends_speed) + # approximate border. Without limit we will have more than 100 MB/s for very slow builds. + assert median_speed <= 20, "We exceeded max send speed for more than 20MB/s. Must be around 5 (+- 10), got " + str(median_speed) + + finally: + for node in [node1, node3]: + for j in range(5): + node.query(f"DROP TABLE IF EXISTS limited_sends{j} SYNC") + + +def test_should_execute_fetch(start_cluster): + print("Should execute fetch") + try: + for i, node in enumerate([node1, node2]): + node.query(f"CREATE TABLE should_execute_table(key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/should_execute_table', '{i}') ORDER BY tuple() PARTITION BY key SETTINGS max_replicated_fetches_network_bandwidth=3505253") + + node2.query("SYSTEM STOP FETCHES should_execute_table") + + for i in range(3): + node1.query("INSERT INTO should_execute_table SELECT {}, '{}' FROM numbers(200)".format(i, get_random_string(104857))) + + n1_net = NetThroughput(node1) + n2_net = NetThroughput(node2) + + node2.query("SYSTEM START FETCHES should_execute_table") + + for i in range(10): + node1.query("INSERT INTO should_execute_table SELECT {}, '{}' FROM numbers(3)".format(i, get_random_string(104857))) + + n2_fetch_speed = [] + replication_queue_data = [] + for i in range(10): + n1_in, n1_out = n1_net.measure_speed('megabytes') + n2_in, n2_out = n2_net.measure_speed('megabytes') + fetches_count = node2.query("SELECT count() FROM system.replicated_fetches") + if fetches_count == "0\n": + break + + print("Fetches count", fetches_count) + replication_queue_data.append(node2.query("SELECT count() FROM system.replication_queue WHERE postpone_reason like '%fetches have already throttled%'")) + n2_fetch_speed.append(n2_in) + time.sleep(0.5) + + node2.query("SYSTEM SYNC REPLICA should_execute_table") + assert any(int(f.strip()) != 0 for f in replication_queue_data) + assert node2.query("SELECT COUNT() FROM should_execute_table") == "630\n" + finally: + for node in [node1, node2]: + node.query("DROP TABLE IF EXISTS should_execute_table SYNC") diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 3b3540ef1b8..d04bdae36e2 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -5,9 +5,6 @@ import string import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_s3_with_https/test.py b/tests/integration/test_s3_with_https/test.py index 58e3b3c2a3b..6c7b47ea0b1 100644 --- a/tests/integration/test_s3_with_https/test.py +++ b/tests/integration/test_s3_with_https/test.py @@ -3,9 +3,6 @@ import logging import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - def check_proxy_logs(cluster, proxy_instance): logs = cluster.get_container_logs(proxy_instance) @@ -20,7 +17,7 @@ def cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml", "configs/config.d/log_conf.xml", "configs/config.d/ssl.xml"], with_minio=True, - minio_certs_dir='minio_certs') + minio_certs_dir="minio_certs") logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 586895fffe5..7a1a2292eef 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -5,10 +5,6 @@ import time import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - - # Runs simple proxy resolver in python env container. def run_resolver(cluster): container_id = cluster.get_container_id('resolver') diff --git a/tests/integration/test_send_request_to_leader_replica/test.py b/tests/integration/test_send_request_to_leader_replica/test.py index 8c58d02d104..721e446ff82 100644 --- a/tests/integration/test_send_request_to_leader_replica/test.py +++ b/tests/integration/test_send_request_to_leader_replica/test.py @@ -39,12 +39,12 @@ def started_cluster(): @pytest.mark.parametrize("table,query,expected,n1,n2", [ - ("sometable", "ALTER TABLE sometable DROP PARTITION 201706", '1', node1, node2), - ("sometable", "TRUNCATE TABLE sometable", '0', node1, node2), - ("sometable", "OPTIMIZE TABLE sometable", '4', node1, node2), - ("someothertable", "ALTER TABLE someothertable DROP PARTITION 201706", '1', node3, node4), - ("someothertable", "TRUNCATE TABLE someothertable", '0', node3, node4), - ("someothertable", "OPTIMIZE TABLE someothertable", '4', node3, node4), + pytest.param("sometable", "ALTER TABLE sometable DROP PARTITION 201706", '1', node1, node2, id="case1"), + pytest.param("sometable", "TRUNCATE TABLE sometable", '0', node1, node2, id="case2"), + pytest.param("sometable", "OPTIMIZE TABLE sometable", '4', node1, node2, id="case3"), + pytest.param("someothertable", "ALTER TABLE someothertable DROP PARTITION 201706", '1', node3, node4, id="case4"), + pytest.param("someothertable", "TRUNCATE TABLE someothertable", '0', node3, node4, id="case5"), + pytest.param("someothertable", "OPTIMIZE TABLE someothertable", '4', node3, node4, id="case6"), ]) def test_alter_table_drop_partition(started_cluster, table, query, expected, n1, n2): to_insert = '''\ diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index a0dc342e910..b3b5f9e6ded 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -4,7 +4,6 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.hdfs_api import HDFSApi -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', with_hdfs=True, main_configs=['configs/log_conf.xml']) @@ -13,25 +12,22 @@ node1 = cluster.add_instance('node1', with_hdfs=True, main_configs=['configs/log def started_cluster(): try: cluster.start() - yield cluster - - except Exception as ex: - print(ex) - raise ex finally: cluster.shutdown() def test_read_write_storage(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() node1.query( "create table SimpleHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/simple_storage', 'TSV')") node1.query("insert into SimpleHDFSStorage values (1, 'Mark', 72.53)") - assert started_cluster.hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" assert node1.query("select * from SimpleHDFSStorage") == "1\tMark\t72.53\n" def test_read_write_storage_with_globs(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() node1.query( "create table HDFSStorageWithRange (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage{1..5}', 'TSV')") node1.query( @@ -42,8 +38,8 @@ def test_read_write_storage_with_globs(started_cluster): "create table HDFSStorageWithAsterisk (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage*', 'TSV')") for i in ["1", "2", "3"]: - started_cluster.hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n") - assert started_cluster.hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" + hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n") + assert hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" assert node1.query("select count(*) from HDFSStorageWithRange") == "3\n" assert node1.query("select count(*) from HDFSStorageWithEnum") == "3\n" @@ -73,23 +69,26 @@ def test_read_write_storage_with_globs(started_cluster): def test_read_write_table(started_cluster): - data = "1\tSerialize\t555.222\n2\tData\t777.333\n" - started_cluster.hdfs_api.write_data("/simple_table_function", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_data("/simple_table_function") == data + data = "1\tSerialize\t555.222\n2\tData\t777.333\n" + hdfs_api.write_data("/simple_table_function", data) + + assert hdfs_api.read_data("/simple_table_function") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") == data def test_write_table(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() node1.query( "create table OtherHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/other_storage', 'TSV')") node1.query("insert into OtherHDFSStorage values (10, 'tomas', 55.55), (11, 'jack', 32.54)") result = "10\ttomas\t55.55\n11\tjack\t32.54\n" - assert started_cluster.hdfs_api.read_data("/other_storage") == result + assert hdfs_api.read_data("/other_storage") == result assert node1.query("select * from OtherHDFSStorage order by id") == result @@ -116,12 +115,14 @@ def test_bad_hdfs_uri(started_cluster): @pytest.mark.timeout(800) def test_globs_in_read_table(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + some_data = "1\tSerialize\t555.222\n2\tData\t777.333\n" globs_dir = "/dir_for_test_with_globs/" files = ["dir1/dir_dir/file1", "dir2/file2", "simple_table_function", "dir/file", "some_dir/dir1/file", "some_dir/dir2/file", "some_dir/file", "table1_function", "table2_function", "table3_function"] for filename in files: - started_cluster.hdfs_api.write_data(globs_dir + filename, some_data) + hdfs_api.write_data(globs_dir + filename, some_data) test_requests = [("dir{1..5}/dir_dir/file1", 1, 1), ("*_table_functio?", 1, 1), @@ -147,74 +148,88 @@ def test_globs_in_read_table(started_cluster): def test_read_write_gzip_table(started_cluster): - data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" - started_cluster.hdfs_api.write_gzip_data("/simple_table_function.gz", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_gzip_data("/simple_table_function.gz") == data + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_gzip_data("/simple_table_function.gz", data) + + assert hdfs_api.read_gzip_data("/simple_table_function.gz") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64')") == data def test_read_write_gzip_table_with_parameter_gzip(started_cluster): - data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" - started_cluster.hdfs_api.write_gzip_data("/simple_table_function", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_gzip_data("/simple_table_function") == data + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_gzip_data("/simple_table_function", data) + + assert hdfs_api.read_gzip_data("/simple_table_function") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64', 'gzip')") == data def test_read_write_table_with_parameter_none(started_cluster): - data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" - started_cluster.hdfs_api.write_data("/simple_table_function.gz", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_data("/simple_table_function.gz") == data + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_data("/simple_table_function.gz", data) + + assert hdfs_api.read_data("/simple_table_function.gz") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'none')") == data def test_read_write_gzip_table_with_parameter_auto_gz(started_cluster): - data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" - started_cluster.hdfs_api.write_gzip_data("/simple_table_function.gz", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_gzip_data("/simple_table_function.gz") == data + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_gzip_data("/simple_table_function.gz", data) + + assert hdfs_api.read_gzip_data("/simple_table_function.gz") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'auto')") == data def test_write_gz_storage(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + node1.query( "create table GZHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage.gz', 'TSV')") node1.query("insert into GZHDFSStorage values (1, 'Mark', 72.53)") - assert started_cluster.hdfs_api.read_gzip_data("/storage.gz") == "1\tMark\t72.53\n" + assert hdfs_api.read_gzip_data("/storage.gz") == "1\tMark\t72.53\n" assert node1.query("select * from GZHDFSStorage") == "1\tMark\t72.53\n" def test_write_gzip_storage(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + node1.query( "create table GZIPHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/gzip_storage', 'TSV', 'gzip')") node1.query("insert into GZIPHDFSStorage values (1, 'Mark', 72.53)") - assert started_cluster.hdfs_api.read_gzip_data("/gzip_storage") == "1\tMark\t72.53\n" + assert hdfs_api.read_gzip_data("/gzip_storage") == "1\tMark\t72.53\n" assert node1.query("select * from GZIPHDFSStorage") == "1\tMark\t72.53\n" def test_virtual_columns(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() node1.query("create table virtual_cols (id UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/file*', 'TSV')") - started_cluster.hdfs_api.write_data("/file1", "1\n") - started_cluster.hdfs_api.write_data("/file2", "2\n") - started_cluster.hdfs_api.write_data("/file3", "3\n") + hdfs_api.write_data("/file1", "1\n") + hdfs_api.write_data("/file2", "2\n") + hdfs_api.write_data("/file3", "3\n") expected = "1\tfile1\thdfs://hdfs1:9000//file1\n2\tfile2\thdfs://hdfs1:9000//file2\n3\tfile3\thdfs://hdfs1:9000//file3\n" assert node1.query("select id, _file as file_name, _path as file_path from virtual_cols order by id") == expected def test_read_files_with_spaces(started_cluster): - started_cluster.hdfs_api.write_data("/test test test 1.txt", "1\n") - started_cluster.hdfs_api.write_data("/test test test 2.txt", "2\n") - started_cluster.hdfs_api.write_data("/test test test 3.txt", "3\n") + hdfs_api = started_cluster.make_hdfs_api() + hdfs_api.write_data("/test test test 1.txt", "1\n") + hdfs_api.write_data("/test test test 2.txt", "2\n") + hdfs_api.write_data("/test test test 3.txt", "3\n") node1.query("create table test (id UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/test*', 'TSV')") assert node1.query("select * from test order by id") == "1\n2\n3\n" diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index eca0f254d8e..def78b824f9 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5,6 +5,7 @@ import socket import subprocess import threading import time +import logging import io import string @@ -22,16 +23,17 @@ from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager from helpers.test_tools import TSV -from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer +from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnection +from kafka.protocol.admin import DescribeGroupsRequest_v1 +from kafka.protocol.group import MemberAssignment from kafka.admin import NewTopic -""" -protoc --version -libprotoc 3.0.0 -# to create kafka_pb2.py -protoc --python_out=. kafka.proto -""" +# protoc --version +# libprotoc 3.0.0 +# # to create kafka_pb2.py +# protoc --python_out=. kafka.proto + from . import kafka_pb2 from . import social_pb2 @@ -43,7 +45,7 @@ cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', main_configs=['configs/kafka.xml', 'configs/log_conf.xml'], with_kafka=True, - with_zookeeper=True, + with_zookeeper=True, # For Replicated Table macros={"kafka_broker":"kafka1", "kafka_topic_old":"old", "kafka_group_name_old":"old", @@ -52,51 +54,37 @@ instance = cluster.add_instance('instance', "kafka_client_id":"instance", "kafka_format_json_each_row":"JSONEachRow"}, clickhouse_path_dir='clickhouse_path') -kafka_id = '' -# Helpers - -def check_kafka_is_available(): - p = subprocess.Popen(('docker', - 'exec', - '-i', - kafka_id, - '/usr/bin/kafka-broker-api-versions', - '--bootstrap-server', - 'INSIDE://localhost:9092'), - stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def wait_kafka_is_available(max_retries=50): - retries = 0 - while True: - if check_kafka_is_available(): - break - else: - retries += 1 - if retries > max_retries: - raise "Kafka is not available" - print("Waiting for Kafka to start up") +def get_kafka_producer(port, serializer, retries): + errors = [] + for _ in range(retries): + try: + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(port), value_serializer=serializer) + logging.debug("Kafka Connection establised: localhost:{}".format(port)) + return producer + except Exception as e: + errors += [str(e)] time.sleep(1) + + raise Exception("Connection not establised, {}".format(errors)) def producer_serializer(x): return x.encode() if isinstance(x, str) else x -def kafka_produce(topic, messages, timestamp=None, retries=2): - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer, retries=retries, max_in_flight_requests_per_connection=1) +def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15): + logging.debug("kafka_produce server:{}:{} topic:{}".format("localhost", kafka_cluster.kafka_port, topic)) + producer = get_kafka_producer(kafka_cluster.kafka_port, producer_serializer, retries) for message in messages: producer.send(topic=topic, value=message, timestamp_ms=timestamp) producer.flush() ## just to ensure the python client / producer is working properly def kafka_producer_send_heartbeat_msg(max_retries=50): - kafka_produce('test_heartbeat_topic', ['test'], retries=max_retries) + kafka_produce(kafka_cluster, 'test_heartbeat_topic', ['test'], retries=max_retries) -def kafka_consume(topic): - consumer = KafkaConsumer(bootstrap_servers="localhost:9092", auto_offset_reset="earliest") +def kafka_consume(kafka_cluster, topic): + consumer = KafkaConsumer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), auto_offset_reset="earliest") consumer.subscribe(topics=(topic)) for toppar, messages in list(consumer.poll(5000).items()): if toppar.topic == topic: @@ -106,7 +94,7 @@ def kafka_consume(topic): consumer.close() -def kafka_produce_protobuf_messages(topic, start_index, num_messages): +def kafka_produce_protobuf_messages(kafka_cluster, topic, start_index, num_messages): data = b'' for i in range(start_index, start_index + num_messages): msg = kafka_pb2.KeyValuePair() @@ -114,14 +102,14 @@ def kafka_produce_protobuf_messages(topic, start_index, num_messages): msg.value = str(i) serialized_msg = msg.SerializeToString() data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer) + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), value_serializer=producer_serializer) producer.send(topic=topic, value=data) producer.flush() - print(("Produced {} messages for topic {}".format(num_messages, topic))) + logging.debug(("Produced {} messages for topic {}".format(num_messages, topic))) -def kafka_produce_protobuf_messages_no_delimeters(topic, start_index, num_messages): +def kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, topic, start_index, num_messages): data = '' - producer = KafkaProducer(bootstrap_servers="localhost:9092") + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) for i in range(start_index, start_index + num_messages): msg = kafka_pb2.KeyValuePair() msg.key = i @@ -129,9 +117,9 @@ def kafka_produce_protobuf_messages_no_delimeters(topic, start_index, num_messag serialized_msg = msg.SerializeToString() producer.send(topic=topic, value=serialized_msg) producer.flush() - print("Produced {} messages for topic {}".format(num_messages, topic)) + logging.debug("Produced {} messages for topic {}".format(num_messages, topic)) -def kafka_produce_protobuf_social(topic, start_index, num_messages): +def kafka_produce_protobuf_social(kafka_cluster,topic, start_index, num_messages): data = b'' for i in range(start_index, start_index + num_messages): msg = social_pb2.User() @@ -139,10 +127,10 @@ def kafka_produce_protobuf_social(topic, start_index, num_messages): msg.timestamp=1000000+i serialized_msg = msg.SerializeToString() data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer) + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), value_serializer=producer_serializer) producer.send(topic=topic, value=data) producer.flush() - print(("Produced {} messages for topic {}".format(num_messages, topic))) + logging.debug(("Produced {} messages for topic {}".format(num_messages, topic))) def avro_message(value): schema = avro.schema.make_avsc_object({ @@ -203,8 +191,8 @@ def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference' else: return TSV(result) == TSV(reference) -def describe_consumer_group(name): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") +def describe_consumer_group(kafka_cluster, name): + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) consumer_groups = admin_client.describe_consumer_groups([name]) res = [] for member in consumer_groups[0].members: @@ -265,7 +253,7 @@ kafka_topic_old old messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('old', messages) + kafka_produce(kafka_cluster, 'old', messages) result = '' while True: @@ -297,16 +285,16 @@ def test_kafka_settings_new_syntax(kafka_cluster): messages = [] for i in range(25): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('new', messages) + kafka_produce(kafka_cluster, 'new', messages) # Insert couple of malformed messages. - kafka_produce('new', ['}{very_broken_message,']) - kafka_produce('new', ['}another{very_broken_message,']) + kafka_produce(kafka_cluster, 'new', ['}{very_broken_message,']) + kafka_produce(kafka_cluster, 'new', ['}another{very_broken_message,']) messages = [] for i in range(25, 50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('new', messages) + kafka_produce(kafka_cluster, 'new', messages) result = '' while True: @@ -322,7 +310,7 @@ def test_kafka_settings_new_syntax(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_json_as_string(kafka_cluster): - kafka_produce('kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', + kafka_produce(kafka_cluster, 'kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) instance.query(''' @@ -343,11 +331,13 @@ def test_kafka_json_as_string(kafka_cluster): ''' assert TSV(result) == TSV(expected) assert instance.contains_in_log( - "Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") + "Parsing of message (topic: kafka_json_as_string, partition: 0, offset: [0-9]*) return no rows") @pytest.mark.timeout(120) def test_kafka_formats(kafka_cluster): + schema_registry_client = CachedSchemaRegistryClient('http://localhost:{}'.format(kafka_cluster.schema_registry_port)) + # data was dumped from clickhouse itself in a following manner # clickhouse-client --format=Native --query='SELECT toInt64(number) as id, toUInt16( intDiv( id, 65536 ) ) as blockNo, reinterpretAsString(19777) as val1, toFloat32(0.5) as val2, toUInt8(1) as val3 from numbers(100) ORDER BY id' | xxd -ps | tr -d '\n' | sed 's/\(..\)/\\x\1/g' @@ -583,19 +573,19 @@ def test_kafka_formats(kafka_cluster): }, 'AvroConfluent': { 'data_sample': [ - avro_confluent_message(cluster.schema_registry_client, + avro_confluent_message(schema_registry_client, {'id': 0, 'blockNo': 0, 'val1': str('AM'), 'val2': 0.5, "val3": 1}), - b''.join([avro_confluent_message(cluster.schema_registry_client, + b''.join([avro_confluent_message(schema_registry_client, {'id': id, 'blockNo': 0, 'val1': str('AM'), 'val2': 0.5, "val3": 1}) for id in range(1, 16)]), - avro_confluent_message(cluster.schema_registry_client, + avro_confluent_message(schema_registry_client, {'id': 0, 'blockNo': 0, 'val1': str('AM'), 'val2': 0.5, "val3": 1}), ], 'extra_settings': ", format_avro_schema_registry_url='http://{}:{}'".format( - cluster.schema_registry_host, - cluster.schema_registry_port + kafka_cluster.schema_registry_host, + 8081 ), 'supports_empty_value': True, }, @@ -629,14 +619,14 @@ def test_kafka_formats(kafka_cluster): } for format_name, format_opts in list(all_formats.items()): - print(('Set up {}'.format(format_name))) + logging.debug(('Set up {}'.format(format_name))) topic_name = 'format_tests_{}'.format(format_name) data_sample = format_opts['data_sample'] data_prefix = [] # prepend empty value when supported if format_opts.get('supports_empty_value', False): data_prefix = data_prefix + [''] - kafka_produce(topic_name, data_prefix + data_sample) + kafka_produce(kafka_cluster, topic_name, data_prefix + data_sample) instance.query(''' DROP TABLE IF EXISTS test.kafka_{format_name}; @@ -663,7 +653,7 @@ def test_kafka_formats(kafka_cluster): instance.wait_for_log_line('kafka.*Committed offset [0-9]+.*format_tests_', repetitions=len(all_formats.keys()), look_behind_lines=12000) for format_name, format_opts in list(all_formats.items()): - print(('Checking {}'.format(format_name))) + logging.debug(('Checking {}'.format(format_name))) topic_name = 'format_tests_{}'.format(format_name) # shift offsets by 1 if format supports empty value offsets = [1, 2, 3] if format_opts.get('supports_empty_value', False) else [0, 1, 2] @@ -689,10 +679,147 @@ def test_kafka_formats(kafka_cluster): '''.format(topic_name=topic_name, offset_0=offsets[0], offset_1=offsets[1], offset_2=offsets[2]) assert TSV(result) == TSV(expected), 'Proper result for format: {}'.format(format_name) + +# Since everything is async and shaky when receiving messages from Kafka, +# we may want to try and check results multiple times in a loop. +def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference'): + fpath = p.join(p.dirname(__file__), ref_file) + with open(fpath) as reference: + if check: + assert TSV(result) == TSV(reference) + else: + return TSV(result) == TSV(reference) + + +# https://stackoverflow.com/a/57692111/1555175 +def describe_consumer_group(kafka_cluster, name): + client = BrokerConnection('localhost', kafka_cluster.kafka_port, socket.AF_INET) + client.connect_blocking() + + list_members_in_groups = DescribeGroupsRequest_v1(groups=[name]) + future = client.send(list_members_in_groups) + while not future.is_done: + for resp, f in client.recv(): + f.success(resp) + + (error_code, group_id, state, protocol_type, protocol, members) = future.value.groups[0] + + res = [] + for member in members: + (member_id, client_id, client_host, member_metadata, member_assignment) = member + member_info = {} + member_info['member_id'] = member_id + member_info['client_id'] = client_id + member_info['client_host'] = client_host + member_topics_assignment = [] + for (topic, partitions) in MemberAssignment.decode(member_assignment).assignment: + member_topics_assignment.append({'topic': topic, 'partitions': partitions}) + member_info['assignment'] = member_topics_assignment + res.append(member_info) + return res + + +# Fixtures + +@pytest.fixture(scope="module") +def kafka_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def kafka_setup_teardown(): + instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') + # logging.debug("kafka is available - running test") + yield # run test + + +# Tests + +@pytest.mark.timeout(180) +def test_kafka_settings_old_syntax(kafka_cluster): + assert TSV(instance.query("SELECT * FROM system.macros WHERE macro like 'kafka%' ORDER BY macro", + ignore_error=True)) == TSV('''kafka_broker kafka1 +kafka_client_id instance +kafka_format_json_each_row JSONEachRow +kafka_group_name_new new +kafka_group_name_old old +kafka_topic_new new +kafka_topic_old old +''') + + instance.query(''' + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_old}', '{kafka_group_name_old}', '{kafka_format_json_each_row}', '\\n'); + ''') + + # Don't insert malformed messages since old settings syntax + # doesn't support skipping of broken messages. + messages = [] + for i in range(50): + messages.append(json.dumps({'key': i, 'value': i})) + kafka_produce(kafka_cluster, 'old', messages) + + result = '' + while True: + result += instance.query('SELECT * FROM test.kafka', ignore_error=True) + if kafka_check_result(result): + break + + kafka_check_result(result, True) + + members = describe_consumer_group(kafka_cluster, 'old') + assert members[0]['client_id'] == 'ClickHouse-instance-test-kafka' + # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:{} --describe --members --group old --verbose".format(cluster.kafka_port))) + + +@pytest.mark.timeout(180) +def test_kafka_settings_new_syntax(kafka_cluster): + instance.query(''' + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = '{kafka_broker}:19092', + kafka_topic_list = '{kafka_topic_new}', + kafka_group_name = '{kafka_group_name_new}', + kafka_format = '{kafka_format_json_each_row}', + kafka_row_delimiter = '\\n', + kafka_client_id = '{kafka_client_id} test 1234', + kafka_skip_broken_messages = 1; + ''') + + messages = [] + for i in range(25): + messages.append(json.dumps({'key': i, 'value': i})) + kafka_produce(kafka_cluster, 'new', messages) + + # Insert couple of malformed messages. + kafka_produce(kafka_cluster, 'new', ['}{very_broken_message,']) + kafka_produce(kafka_cluster, 'new', ['}another{very_broken_message,']) + + messages = [] + for i in range(25, 50): + messages.append(json.dumps({'key': i, 'value': i})) + kafka_produce(kafka_cluster, 'new', messages) + + result = '' + while True: + result += instance.query('SELECT * FROM test.kafka', ignore_error=True) + if kafka_check_result(result): + break + + kafka_check_result(result, True) + + members = describe_consumer_group(kafka_cluster, 'new') + assert members[0]['client_id'] == 'instance test 1234' + + @pytest.mark.timeout(180) def test_kafka_issue11308(kafka_cluster): # Check that matview does respect Kafka SETTINGS - kafka_produce('issue11308', ['{"t": 123, "e": {"x": "woof"} }', '{"t": 123, "e": {"x": "woof"} }', + kafka_produce(kafka_cluster, 'issue11308', ['{"t": 123, "e": {"x": "woof"} }', '{"t": 123, "e": {"x": "woof"} }', '{"t": 124, "e": {"x": "test"} }']) instance.query(''' @@ -741,7 +868,7 @@ def test_kafka_issue11308(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_issue4116(kafka_cluster): # Check that format_csv_delimiter parameter works now - as part of all available format settings. - kafka_produce('issue4116', ['1|foo', '2|bar', '42|answer', '100|multi\n101|row\n103|message']) + kafka_produce(kafka_cluster, 'issue4116', ['1|foo', '2|bar', '42|answer', '100|multi\n101|row\n103|message']) instance.query(''' CREATE TABLE test.kafka (a UInt64, b String) @@ -769,7 +896,7 @@ def test_kafka_issue4116(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_consumer_hang(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="consumer_hang", num_partitions=8, replication_factor=1)) @@ -800,7 +927,7 @@ def test_kafka_consumer_hang(kafka_cluster): instance.wait_for_log_line('heartbeat error') kafka_cluster.unpause_container('kafka1') - # print("Attempt to drop") + # logging.debug("Attempt to drop") instance.query('DROP TABLE test.kafka') # kafka_cluster.open_bash_shell('instance') @@ -821,7 +948,7 @@ def test_kafka_consumer_hang(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_consumer_hang2(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="consumer_hang2", num_partitions=1, replication_factor=1)) @@ -872,7 +999,7 @@ def test_kafka_csv_with_delimiter(kafka_cluster): messages = [] for i in range(50): messages.append('{i}, {i}'.format(i=i)) - kafka_produce('csv', messages) + kafka_produce(kafka_cluster, 'csv', messages) instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) @@ -897,7 +1024,7 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): messages = [] for i in range(50): messages.append('{i}\t{i}'.format(i=i)) - kafka_produce('tsv', messages) + kafka_produce(kafka_cluster, 'tsv', messages) instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) @@ -919,7 +1046,7 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): @pytest.mark.timeout(120) def test_kafka_select_empty(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="empty", num_partitions=1, replication_factor=1)) admin_client.create_topics(new_topics=topic_list, validate_only=False) @@ -942,12 +1069,12 @@ def test_kafka_json_without_delimiter(kafka_cluster): messages = '' for i in range(25): messages += json.dumps({'key': i, 'value': i}) + '\n' - kafka_produce('json', [messages]) + kafka_produce(kafka_cluster, 'json', [messages]) messages = '' for i in range(25, 50): messages += json.dumps({'key': i, 'value': i}) + '\n' - kafka_produce('json', [messages]) + kafka_produce(kafka_cluster, 'json', [messages]) instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) @@ -969,9 +1096,9 @@ def test_kafka_json_without_delimiter(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_protobuf(kafka_cluster): - kafka_produce_protobuf_messages('pb', 0, 20) - kafka_produce_protobuf_messages('pb', 20, 1) - kafka_produce_protobuf_messages('pb', 21, 29) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 0, 20) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 20, 1) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 21, 29) instance.query(''' CREATE TABLE test.kafka (key UInt64, value String) @@ -995,9 +1122,9 @@ def test_kafka_protobuf(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_string_field_on_first_position_in_protobuf(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/12615 - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 0, 20) - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 20, 1) - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 21, 29) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 0, 20) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 20, 1) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 21, 29) instance.query(''' CREATE TABLE test.kafka ( @@ -1079,9 +1206,9 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): kafka_schema = 'kafka.proto:KeyValuePair'; ''') - kafka_produce_protobuf_messages_no_delimeters('pb_no_delimiter', 0, 20) - kafka_produce_protobuf_messages_no_delimeters('pb_no_delimiter', 20, 1) - kafka_produce_protobuf_messages_no_delimeters('pb_no_delimiter', 21, 29) + kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, 'pb_no_delimiter', 0, 20) + kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, 'pb_no_delimiter', 20, 1) + kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, 'pb_no_delimiter', 21, 29) result = '' while True: @@ -1138,7 +1265,7 @@ def test_kafka_materialized_view(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('mv', messages) + kafka_produce(kafka_cluster, 'mv', messages) while True: result = instance.query('SELECT * FROM test.view') @@ -1190,10 +1317,11 @@ def test_librdkafka_compression(kafka_cluster): expected = '\n'.join(expected) for compression_type in supported_compression_types: - print(('Check compression {}'.format(compression_type))) + logging.debug(('Check compression {}'.format(compression_type))) topic_name = 'test_librdkafka_compression_{}'.format(compression_type) - admin_client = admin.AdminClient({'bootstrap.servers': 'localhost:9092'}) + admin_client = admin.AdminClient({'bootstrap.servers': "localhost:{}".format(kafka_cluster.kafka_port)}) + topic = admin.NewTopic(topic=topic_name, num_partitions=1, replication_factor=1, config={ 'compression.type': compression_type, }) @@ -1211,7 +1339,7 @@ def test_librdkafka_compression(kafka_cluster): SELECT * FROM test.kafka; '''.format(topic_name=topic_name) ) - kafka_produce(topic_name, messages) + kafka_produce(kafka_cluster, topic_name, messages) instance.wait_for_log_line("Committed offset {}".format(number_of_messages)) @@ -1243,7 +1371,7 @@ def test_kafka_materialized_view_with_subquery(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('mvsq', messages) + kafka_produce(kafka_cluster, 'mvsq', messages) while True: result = instance.query('SELECT * FROM test.view') @@ -1287,7 +1415,7 @@ def test_kafka_many_materialized_views(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('mmv', messages) + kafka_produce(kafka_cluster, 'mmv', messages) while True: result1 = instance.query('SELECT * FROM test.view1') @@ -1312,7 +1440,7 @@ def test_kafka_flush_on_big_message(kafka_cluster): kafka_messages = 1000 batch_messages = 1000 messages = [json.dumps({'key': i, 'value': 'x' * 100}) * batch_messages for i in range(kafka_messages)] - kafka_produce('flush', messages) + kafka_produce(kafka_cluster, 'flush', messages) instance.query(''' DROP TABLE IF EXISTS test.view; @@ -1331,7 +1459,7 @@ def test_kafka_flush_on_big_message(kafka_cluster): SELECT * FROM test.kafka; ''') - client = KafkaAdminClient(bootstrap_servers="localhost:9092") + client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) received = False while not received: try: @@ -1370,12 +1498,12 @@ def test_kafka_virtual_columns(kafka_cluster): messages = '' for i in range(25): messages += json.dumps({'key': i, 'value': i}) + '\n' - kafka_produce('virt1', [messages], 0) + kafka_produce(kafka_cluster, 'virt1', [messages], 0) messages = '' for i in range(25, 50): messages += json.dumps({'key': i, 'value': i}) + '\n' - kafka_produce('virt1', [messages], 0) + kafka_produce(kafka_cluster, 'virt1', [messages], 0) result = '' while True: @@ -1410,7 +1538,7 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('virt2', messages, 0) + kafka_produce(kafka_cluster, 'virt2', messages, 0) while True: result = instance.query('SELECT kafka_key, key, topic, value, offset, partition, timestamp FROM test.view') @@ -1454,7 +1582,7 @@ def test_kafka_insert(kafka_cluster): messages = [] while True: - messages.extend(kafka_consume('insert1')) + messages.extend(kafka_consume(kafka_cluster, 'insert1')) if len(messages) == 50: break @@ -1554,7 +1682,7 @@ def test_kafka_commit_on_block_write(kafka_cluster): for _ in range(101): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 - kafka_produce('block', messages) + kafka_produce(kafka_cluster, 'block', messages) kafka_thread = threading.Thread(target=produce) kafka_thread.start() @@ -1596,7 +1724,7 @@ def test_kafka_commit_on_block_write(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_virtual_columns2(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="virt2_0", num_partitions=2, replication_factor=1)) topic_list.append(NewTopic(name="virt2_1", num_partitions=2, replication_factor=1)) @@ -1616,7 +1744,7 @@ def test_kafka_virtual_columns2(kafka_cluster): SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms), _headers.name, _headers.value FROM test.kafka; ''') - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer, key_serializer=producer_serializer) + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(cluster.kafka_port), value_serializer=producer_serializer, key_serializer=producer_serializer) producer.send(topic='virt2_0', value=json.dumps({'value': 1}), partition=0, key='k1', timestamp_ms=1577836801001, headers=[('content-encoding', b'base64')]) @@ -1640,7 +1768,7 @@ def test_kafka_virtual_columns2(kafka_cluster): instance.wait_for_log_line('kafka.*Committed offset 2.*virt2_[01]', repetitions=4, look_behind_lines=6000) - members = describe_consumer_group('virt2') + members = describe_consumer_group(kafka_cluster, 'virt2') # pprint.pprint(members) members[0]['client_id'] = 'ClickHouse-instance-test-kafka-0' members[1]['client_id'] = 'ClickHouse-instance-test-kafka-1' @@ -1664,7 +1792,7 @@ def test_kafka_virtual_columns2(kafka_cluster): @pytest.mark.timeout(120) def test_kafka_produce_key_timestamp(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="insert3", num_partitions=1, replication_factor=1)) @@ -1708,7 +1836,7 @@ def test_kafka_produce_key_timestamp(kafka_cluster): result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) - # print(result) + # logging.debug(result) expected = '''\ 1 1 k1 1577836801 k1 insert3 0 0 1577836801 @@ -1723,7 +1851,7 @@ def test_kafka_produce_key_timestamp(kafka_cluster): @pytest.mark.timeout(600) def test_kafka_flush_by_time(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="flush_by_time", num_partitions=1, replication_factor=1)) admin_client.create_topics(new_topics=topic_list, validate_only=False) @@ -1754,7 +1882,7 @@ def test_kafka_flush_by_time(kafka_cluster): while not cancel.is_set(): messages = [] messages.append(json.dumps({'key': 0, 'value': 0})) - kafka_produce('flush_by_time', messages) + kafka_produce(kafka_cluster, 'flush_by_time', messages) time.sleep(0.8) kafka_thread = threading.Thread(target=produce) @@ -1790,7 +1918,7 @@ def test_kafka_flush_by_block_size(kafka_cluster): while not cancel.is_set(): messages = [] messages.append(json.dumps({'key': 0, 'value': 0})) - kafka_produce('flush_by_block_size', messages) + kafka_produce(kafka_cluster, 'flush_by_block_size', messages) kafka_thread = threading.Thread(target=produce) kafka_thread.start() @@ -1828,7 +1956,7 @@ def test_kafka_flush_by_block_size(kafka_cluster): # more flushes can happens during test, we need to check only result of first flush (part named all_1_1_0). result = instance.query("SELECT count() FROM test.view WHERE _part='all_1_1_0'") - # print(result) + # logging.debug(result) instance.query(''' DROP TABLE test.consumer; @@ -1843,7 +1971,7 @@ def test_kafka_flush_by_block_size(kafka_cluster): @pytest.mark.timeout(600) def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="topic_with_multiple_partitions2", num_partitions=10, replication_factor=1)) @@ -1875,12 +2003,12 @@ def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster): count = count + 1 rows.append(json.dumps({'key': count, 'value': count})) messages.append("\n".join(rows)) - kafka_produce('topic_with_multiple_partitions2', messages) + kafka_produce(kafka_cluster, 'topic_with_multiple_partitions2', messages) instance.wait_for_log_line('kafka.*Stalled', repetitions=5) result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') - print(result) + logging.debug(result) assert TSV(result) == TSV('{0}\t{0}\t{0}'.format(count)) instance.query(''' @@ -1913,7 +2041,7 @@ def test_kafka_rebalance(kafka_cluster): # time.sleep(2) - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="topic_with_multiple_partitions", num_partitions=11, replication_factor=1)) admin_client.create_topics(new_topics=topic_list, validate_only=False) @@ -1928,14 +2056,14 @@ def test_kafka_rebalance(kafka_cluster): for _ in range(59): messages.append(json.dumps({'key': msg_index[0], 'value': msg_index[0]})) msg_index[0] += 1 - kafka_produce('topic_with_multiple_partitions', messages) + kafka_produce(kafka_cluster, 'topic_with_multiple_partitions', messages) kafka_thread = threading.Thread(target=produce) kafka_thread.start() for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): table_name = 'kafka_consumer{}'.format(consumer_index) - print(("Setting up {}".format(table_name))) + logging.debug(("Setting up {}".format(table_name))) instance.query(''' DROP TABLE IF EXISTS test.{0}; @@ -1968,10 +2096,10 @@ def test_kafka_rebalance(kafka_cluster): # I leave last one working by intent (to finish consuming after all rebalances) for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS - 1): - print(("Dropping test.kafka_consumer{}".format(consumer_index))) + logging.debug(("Dropping test.kafka_consumer{}".format(consumer_index))) instance.query('DROP TABLE IF EXISTS test.kafka_consumer{} SYNC'.format(consumer_index)) - # print(instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination')) + # logging.debug(instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination')) # kafka_cluster.open_bash_shell('instance') while 1: @@ -1979,9 +2107,9 @@ def test_kafka_rebalance(kafka_cluster): if messages_consumed >= msg_index[0]: break time.sleep(1) - print(("Waiting for finishing consuming (have {}, should be {})".format(messages_consumed, msg_index[0]))) + logging.debug(("Waiting for finishing consuming (have {}, should be {})".format(messages_consumed, msg_index[0]))) - print((instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination'))) + logging.debug((instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination'))) # Some queries to debug... # SELECT * FROM test.destination where key in (SELECT key FROM test.destination group by key having count() <> 1) @@ -2006,7 +2134,7 @@ def test_kafka_rebalance(kafka_cluster): result = int(instance.query('SELECT count() == uniqExact(key) FROM test.destination')) for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): - print(("kafka_consumer{}".format(consumer_index))) + logging.debug(("kafka_consumer{}".format(consumer_index))) table_name = 'kafka_consumer{}'.format(consumer_index) instance.query(''' DROP TABLE IF EXISTS test.{0}; @@ -2025,7 +2153,7 @@ def test_kafka_rebalance(kafka_cluster): @pytest.mark.timeout(120) def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(22)] - kafka_produce('no_holes_when_write_suffix_failed', messages) + kafka_produce(kafka_cluster, 'no_holes_when_write_suffix_failed', messages) instance.query(''' DROP TABLE IF EXISTS test.view; @@ -2064,7 +2192,7 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): instance.wait_for_log_line("Committed offset 22") result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') - print(result) + logging.debug(result) # kafka_cluster.open_bash_shell('instance') @@ -2112,7 +2240,7 @@ def test_exception_from_destructor(kafka_cluster): @pytest.mark.timeout(120) def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(1)] - kafka_produce('commits_of_unprocessed_messages_on_drop', messages) + kafka_produce(kafka_cluster, 'commits_of_unprocessed_messages_on_drop', messages) instance.query(''' DROP TABLE IF EXISTS test.destination SYNC; @@ -2163,7 +2291,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): for _ in range(113): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 - kafka_produce('commits_of_unprocessed_messages_on_drop', messages) + kafka_produce(kafka_cluster, 'commits_of_unprocessed_messages_on_drop', messages) time.sleep(0.5) kafka_thread = threading.Thread(target=produce) @@ -2192,7 +2320,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): # SELECT key, _timestamp, _offset FROM test.destination where runningDifference(key) <> 1 ORDER BY key; result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.destination') - print(result) + logging.debug(result) instance.query(''' DROP TABLE test.kafka_consumer SYNC; @@ -2203,10 +2331,10 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): assert TSV(result) == TSV('{0}\t{0}\t{0}'.format(i[0] - 1)), 'Missing data!' -@pytest.mark.timeout(120) +@pytest.mark.timeout(300) def test_bad_reschedule(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(20000)] - kafka_produce('test_bad_reschedule', messages) + kafka_produce(kafka_cluster, 'test_bad_reschedule', messages) instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) @@ -2239,7 +2367,7 @@ def test_bad_reschedule(kafka_cluster): @pytest.mark.timeout(300) def test_kafka_duplicates_when_commit_failed(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(22)] - kafka_produce('duplicates_when_commit_failed', messages) + kafka_produce(kafka_cluster, 'duplicates_when_commit_failed', messages) instance.query(''' DROP TABLE IF EXISTS test.view SYNC; @@ -2285,7 +2413,7 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): instance.wait_for_log_line("Committed offset 22") result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') - print(result) + logging.debug(result) instance.query(''' DROP TABLE test.consumer SYNC; @@ -2334,7 +2462,7 @@ def test_premature_flush_on_eof(kafka_cluster): # next poll can return more data, and we should respect kafka_flush_interval_ms # and try to form bigger block messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(1)] - kafka_produce('premature_flush_on_eof', messages) + kafka_produce(kafka_cluster, 'premature_flush_on_eof', messages) instance.query(''' CREATE MATERIALIZED VIEW test.kafka_consumer TO test.destination AS @@ -2356,7 +2484,7 @@ def test_premature_flush_on_eof(kafka_cluster): instance.wait_for_log_line("Stalled") # produce more messages after delay - kafka_produce('premature_flush_on_eof', messages) + kafka_produce(kafka_cluster, 'premature_flush_on_eof', messages) # data was not flushed yet (it will be flushed 7.5 sec after creating MV) assert int(instance.query("SELECT count() FROM test.destination")) == 0 @@ -2373,24 +2501,23 @@ def test_premature_flush_on_eof(kafka_cluster): ''') -@pytest.mark.timeout(120) +@pytest.mark.timeout(180) def test_kafka_unavailable(kafka_cluster): - messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(2000)] - kafka_produce('test_kafka_unavailable', messages) + messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(20000)] + kafka_produce(kafka_cluster, 'test_bad_reschedule', messages) kafka_cluster.pause_container('kafka1') instance.query(''' - CREATE TABLE test.kafka (key UInt64, value UInt64) + CREATE TABLE test.test_bad_reschedule (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'test_kafka_unavailable', - kafka_group_name = 'test_kafka_unavailable', + kafka_topic_list = 'test_bad_reschedule', + kafka_group_name = 'test_bad_reschedule', kafka_format = 'JSONEachRow', - kafka_max_block_size = 1000, - kafka_flush_interval_ms = 1000; + kafka_max_block_size = 1000; - CREATE MATERIALIZED VIEW test.destination Engine=Log AS + CREATE MATERIALIZED VIEW test.destination_unavailable Engine=Log AS SELECT key, now() as consume_ts, @@ -2400,19 +2527,20 @@ def test_kafka_unavailable(kafka_cluster): _offset, _partition, _timestamp - FROM test.kafka; + FROM test.test_bad_reschedule; ''') - instance.query("SELECT * FROM test.kafka") - - instance.wait_for_log_line('brokers are down') - instance.wait_for_log_line('stalled. Reschedule', repetitions=2) + instance.query("SELECT * FROM test.test_bad_reschedule") + instance.query("SELECT count() FROM test.destination_unavailable") + # enough to trigger issue + time.sleep(30) kafka_cluster.unpause_container('kafka1') - instance.wait_for_log_line("Committed offset 2000") - assert int(instance.query("SELECT count() FROM test.destination")) == 2000 - time.sleep(5) # needed to give time for kafka client in python test to recovery + while int(instance.query("SELECT count() FROM test.destination_unavailable")) < 20000: + print("Waiting for consume") + time.sleep(1) + @pytest.mark.timeout(180) def test_kafka_issue14202(kafka_cluster): @@ -2464,7 +2592,7 @@ def test_kafka_csv_with_thread_per_consumer(kafka_cluster): messages = [] for i in range(50): messages.append('{i}, {i}'.format(i=i)) - kafka_produce('csv_with_thread_per_consumer', messages) + kafka_produce(kafka_cluster, 'csv_with_thread_per_consumer', messages) result = '' while True: @@ -2515,7 +2643,7 @@ def test_kafka_engine_put_errors_to_stream(kafka_cluster): # Unexpected json content for table test.kafka. messages.append(json.dumps({'i': 'n_' + random_string(4), 's': random_string(8)})) - kafka_produce('kafka_engine_put_errors_to_stream', messages) + kafka_produce(kafka_cluster, 'kafka_engine_put_errors_to_stream', messages) instance.wait_for_log_line("Committed offset 128") assert TSV(instance.query('SELECT count() FROM test.kafka_data')) == TSV('64') @@ -2588,7 +2716,7 @@ def test_kafka_engine_put_errors_to_stream_with_random_malformed_json(kafka_clus else: messages.append(gen_message_with_jsons(10, 0)) - kafka_produce('kafka_engine_put_errors_to_stream_with_random_malformed_json', messages) + kafka_produce(kafka_cluster, 'kafka_engine_put_errors_to_stream_with_random_malformed_json', messages) instance.wait_for_log_line("Committed offset 128") # 64 good messages, each containing 10 rows @@ -2785,7 +2913,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): data_prefix = data_prefix + [''] if format_opts.get('printable', False) == False: raw_message = 'hex(_raw_message)' - kafka_produce(topic_name, data_prefix + data_sample) + kafka_produce(kafka_cluster, topic_name, data_prefix + data_sample) instance.query(''' DROP TABLE IF EXISTS test.kafka_{format_name}; diff --git a/tests/integration/test_storage_kerberized_hdfs/secrets/krb.conf b/tests/integration/test_storage_kerberized_hdfs/secrets/krb.conf index 2c1e6f15f77..90ad6b61c93 100644 --- a/tests/integration/test_storage_kerberized_hdfs/secrets/krb.conf +++ b/tests/integration/test_storage_kerberized_hdfs/secrets/krb.conf @@ -7,7 +7,7 @@ default_realm = TEST.CLICKHOUSE.TECH dns_lookup_realm = false dns_lookup_kdc = false - ticket_lifetime = 15s + ticket_lifetime = 5s forwardable = true default_tgs_enctypes = des3-hmac-sha1 default_tkt_enctypes = des3-hmac-sha1 diff --git a/tests/integration/test_storage_kerberized_hdfs/test.py b/tests/integration/test_storage_kerberized_hdfs/test.py index 1fffd7a8c12..4733defd5c2 100644 --- a/tests/integration/test_storage_kerberized_hdfs/test.py +++ b/tests/integration/test_storage_kerberized_hdfs/test.py @@ -22,42 +22,55 @@ def started_cluster(): finally: cluster.shutdown() +# TODO Remove it and enable test +@pytest.mark.skip(reason="Don't work in parallel mode for some reason") def test_read_table(started_cluster): - data = "1\tSerialize\t555.222\n2\tData\t777.333\n" - started_cluster.hdfs_api.write_data("/simple_table_function", data) + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) - api_read = started_cluster.hdfs_api.read_data("/simple_table_function") + data = "1\tSerialize\t555.222\n2\tData\t777.333\n" + hdfs_api.write_data("/simple_table_function", data) + + api_read = hdfs_api.read_data("/simple_table_function") assert api_read == data select_read = node1.query("select * from hdfs('hdfs://kerberizedhdfs1:9010/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") assert select_read == data - +# TODO Remove it and enable test +@pytest.mark.skip(reason="Don't work in parallel mode for some reason") def test_read_write_storage(started_cluster): + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) + node1.query("create table SimpleHDFSStorage2 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage1', 'TSV')") node1.query("insert into SimpleHDFSStorage2 values (1, 'Mark', 72.53)") - api_read = started_cluster.hdfs_api.read_data("/simple_storage1") + api_read = hdfs_api.read_data("/simple_storage1") assert api_read == "1\tMark\t72.53\n" select_read = node1.query("select * from SimpleHDFSStorage2") assert select_read == "1\tMark\t72.53\n" - +# TODO Remove it and enable test +@pytest.mark.skip(reason="Don't work in parallel mode for some reason") def test_write_storage_not_expired(started_cluster): + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) + node1.query("create table SimpleHDFSStorageNotExpired (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage_not_expired', 'TSV')") - time.sleep(45) # wait for ticket expiration + time.sleep(15) # wait for ticket expiration node1.query("insert into SimpleHDFSStorageNotExpired values (1, 'Mark', 72.53)") - api_read = started_cluster.hdfs_api.read_data("/simple_storage_not_expired") + api_read = hdfs_api.read_data("/simple_storage_not_expired") assert api_read == "1\tMark\t72.53\n" select_read = node1.query("select * from SimpleHDFSStorageNotExpired") assert select_read == "1\tMark\t72.53\n" - +# TODO Remove it and enable test +@pytest.mark.skip(reason="Don't work in parallel mode for some reason") def test_two_users(started_cluster): + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) + node1.query("create table HDFSStorOne (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/storage_user_one', 'TSV')") node1.query("insert into HDFSStorOne values (1, 'Real', 86.00)") @@ -68,12 +81,16 @@ def test_two_users(started_cluster): select_read_2 = node1.query("select * from hdfs('hdfs://suser@kerberizedhdfs1:9010/storage_user_one', 'TSV', 'id UInt64, text String, number Float64')") +# TODO Remove it and enable test +@pytest.mark.skip(reason="Don't work in parallel mode for some reason") def test_read_table_expired(started_cluster): + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) + data = "1\tSerialize\t555.222\n2\tData\t777.333\n" - started_cluster.hdfs_api.write_data("/simple_table_function_relogin", data) + hdfs_api.write_data("/simple_table_function_relogin", data) started_cluster.pause_container('hdfskerberos') - time.sleep(45) + time.sleep(15) try: select_read = node1.query("select * from hdfs('hdfs://reloginuser&kerberizedhdfs1:9010/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") @@ -83,7 +100,8 @@ def test_read_table_expired(started_cluster): started_cluster.unpause_container('hdfskerberos') - +# TODO Remove it and enable test +@pytest.mark.skip(reason="Don't work in parallel mode for some reason") def test_prohibited(started_cluster): node1.query("create table HDFSStorTwoProhibited (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://suser@kerberizedhdfs1:9010/storage_user_two_prohibited', 'TSV')") try: @@ -92,7 +110,8 @@ def test_prohibited(started_cluster): except Exception as ex: assert "Unable to open HDFS file: /storage_user_two_prohibited error: Permission denied: user=specuser, access=WRITE" in str(ex) - +# TODO Remove it and enable test +@pytest.mark.skip(reason="Don't work in parallel mode for some reason") def test_cache_path(started_cluster): node1.query("create table HDFSStorCachePath (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://dedicatedcachepath@kerberizedhdfs1:9010/storage_dedicated_cache_path', 'TSV')") try: diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index 865afc8b162..94d2156c2ff 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -3,6 +3,7 @@ import random import threading import time import pytest +import logging from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV @@ -24,47 +25,29 @@ instance = cluster.add_instance('instance', with_kerberized_kafka=True, clickhouse_path_dir="clickhouse_path" ) -kafka_id = '' # instance.cluster.kafka_docker_id - -# Helpers - -def check_kafka_is_available(): - - # plaintext - p = subprocess.Popen(('docker', - 'exec', - '-i', - kafka_id, - '/usr/bin/kafka-broker-api-versions', - '--bootstrap-server', - 'localhost:9093'), - stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def wait_kafka_is_available(max_retries=50): - retries = 0 - while True: - if check_kafka_is_available(): - break - else: - retries += 1 - if retries > max_retries: - raise "Kafka is not available" - print("Waiting for Kafka to start up") - time.sleep(1) - def producer_serializer(x): return x.encode() if isinstance(x, str) else x -def kafka_produce(topic, messages, timestamp=None): - producer = KafkaProducer(bootstrap_servers="localhost:9093", value_serializer=producer_serializer) + +def get_kafka_producer(port, serializer): + errors = [] + for _ in range(15): + try: + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(port), value_serializer=serializer) + logging.debug("Kafka Connection establised: localhost:{}".format(port)) + return producer + except Exception as e: + errors += [str(e)] + time.sleep(1) + + raise Exception("Connection not establised, {}".format(errors)) + +def kafka_produce(kafka_cluster, topic, messages, timestamp=None): + logging.debug("kafka_produce server:{}:{} topic:{}".format("localhost", kafka_cluster.kerberized_kafka_port, topic)) + producer = get_kafka_producer(kafka_cluster.kerberized_kafka_port, producer_serializer) for message in messages: producer.send(topic=topic, value=message, timestamp_ms=timestamp) producer.flush() - print ("Produced {} messages for topic {}".format(len(messages), topic)) - # Fixtures @@ -72,12 +55,8 @@ def kafka_produce(topic, messages, timestamp=None): @pytest.fixture(scope="module") def kafka_cluster(): try: - global kafka_id cluster.start() - kafka_id = instance.cluster.kerberized_kafka_docker_id - print("kafka_id is {}".format(kafka_id)) yield cluster - finally: cluster.shutdown() @@ -85,15 +64,13 @@ def kafka_cluster(): @pytest.fixture(autouse=True) def kafka_setup_teardown(): instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') - wait_kafka_is_available() - print("kafka is available - running test") yield # run test # Tests @pytest.mark.timeout(180) # wait to build containers def test_kafka_json_as_string(kafka_cluster): - kafka_produce('kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) + kafka_produce(kafka_cluster, 'kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) instance.query(''' CREATE TABLE test.kafka (field String) @@ -117,7 +94,7 @@ def test_kafka_json_as_string(kafka_cluster): assert instance.contains_in_log("Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") def test_kafka_json_as_string_no_kdc(kafka_cluster): - kafka_produce('kafka_json_as_string_no_kdc', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) + kafka_produce(kafka_cluster, 'kafka_json_as_string_no_kdc', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) kafka_cluster.pause_container('kafka_kerberos') time.sleep(45) # wait for ticket expiration diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index f75a9aac237..75af909faec 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -13,20 +13,18 @@ node = cluster.add_instance('node', with_mongo=True) def started_cluster(): try: cluster.start() - yield cluster - finally: cluster.shutdown() -def get_mongo_connection(): - connection_str = 'mongodb://root:clickhouse@localhost:27018' +def get_mongo_connection(started_cluster): + connection_str = 'mongodb://root:clickhouse@localhost:{}'.format(started_cluster.mongo_port) return pymongo.MongoClient(connection_str) def test_simple_select(started_cluster): - mongo_connection = get_mongo_connection() + mongo_connection = get_mongo_connection(started_cluster) db = mongo_connection['test'] db.add_user('root', 'clickhouse') simple_mongo_table = db['simple_table'] @@ -45,7 +43,7 @@ def test_simple_select(started_cluster): def test_complex_data_type(started_cluster): - mongo_connection = get_mongo_connection() + mongo_connection = get_mongo_connection(started_cluster) db = mongo_connection['test'] db.add_user('root', 'clickhouse') incomplete_mongo_table = db['complex_table'] @@ -64,7 +62,7 @@ def test_complex_data_type(started_cluster): def test_incorrect_data_type(started_cluster): - mongo_connection = get_mongo_connection() + mongo_connection = get_mongo_connection(started_cluster) db = mongo_connection['test'] db.add_user('root', 'clickhouse') strange_mongo_table = db['strange_table'] diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index 9c3abd799af..2c993e3d696 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -21,11 +21,6 @@ create_table_sql_template = """ PRIMARY KEY (`id`)) ENGINE=InnoDB; """ -def get_mysql_conn(port=3308): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=port) - return conn - - def create_mysql_db(conn, name): with conn.cursor() as cursor: cursor.execute( @@ -42,8 +37,11 @@ def started_cluster(): try: cluster.start() + conn = get_mysql_conn(cluster, cluster.mysql_ip) + create_mysql_db(conn, 'clickhouse') + ## create mysql db and table - conn1 = get_mysql_conn(port=3308) + conn1 = get_mysql_conn(cluster, cluster.mysql2_ip) create_mysql_db(conn1, 'clickhouse') yield cluster @@ -53,11 +51,11 @@ def started_cluster(): def test_many_connections(started_cluster): table_name = 'test_many_connections' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node1.query("INSERT INTO {} (id, name) SELECT number, concat('name_', toString(number)) from numbers(10) ".format(table_name)) @@ -73,11 +71,11 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_insert_select(started_cluster): table_name = 'test_insert_select' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node1.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000) ".format( @@ -89,11 +87,11 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_replace_select(started_cluster): table_name = 'test_replace_select' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); '''.format(table_name, table_name)) node1.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000) ".format( @@ -108,11 +106,11 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_insert_on_duplicate_select(started_cluster): table_name = 'test_insert_on_duplicate_select' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse', 0, 'update money = money + values(money)'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 0, 'update money = money + values(money)'); '''.format(table_name, table_name)) node1.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000) ".format( @@ -127,10 +125,10 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_where(started_cluster): table_name = 'test_where' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node1.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000) ".format( @@ -147,9 +145,9 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_table_function(started_cluster): - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, 'table_function') - table_function = "mysql('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('table_function') + table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('table_function') assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '0' node1.query( "INSERT INTO {} (id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000)".format( @@ -169,37 +167,51 @@ def test_table_function(started_cluster): def test_binary_type(started_cluster): - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) with conn.cursor() as cursor: cursor.execute("CREATE TABLE clickhouse.binary_type (id INT PRIMARY KEY, data BINARY(16) NOT NULL)") - table_function = "mysql('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('binary_type') + table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('binary_type') node1.query("INSERT INTO {} VALUES (42, 'clickhouse')".format('TABLE FUNCTION ' + table_function)) assert node1.query("SELECT * FROM {}".format(table_function)) == '42\tclickhouse\\0\\0\\0\\0\\0\\0\n' def test_enum_type(started_cluster): table_name = 'test_enum_type' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('IP' = 1, 'URL' = 2)) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('IP' = 1, 'URL' = 2)) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); '''.format(table_name, table_name)) node1.query("INSERT INTO {} (id, name, age, money, source) VALUES (1, 'name', 0, 0, 'URL')".format(table_name)) assert node1.query("SELECT source FROM {} LIMIT 1".format(table_name)).rstrip() == 'URL' conn.close() +def get_mysql_conn(started_cluster, host): + conn = pymysql.connect(user='root', password='clickhouse', host=host, port=started_cluster.mysql_port) + return conn + + +def create_mysql_db(conn, name): + with conn.cursor() as cursor: + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) + cursor.execute("CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) + +def create_mysql_table(conn, tableName): + with conn.cursor() as cursor: + cursor.execute(create_table_sql_template.format(tableName)) def test_mysql_distributed(started_cluster): table_name = 'test_replicas' - conn1 = get_mysql_conn(port=3348) - conn2 = get_mysql_conn(port=3388) - conn3 = get_mysql_conn(port=3368) - conn4 = get_mysql_conn(port=3308) + conn1 = get_mysql_conn(started_cluster, started_cluster.mysql_ip) + conn2 = get_mysql_conn(started_cluster, started_cluster.mysql2_ip) + conn3 = get_mysql_conn(started_cluster, started_cluster.mysql3_ip) + conn4 = get_mysql_conn(started_cluster, started_cluster.mysql4_ip) create_mysql_db(conn1, 'clickhouse') create_mysql_db(conn2, 'clickhouse') create_mysql_db(conn3, 'clickhouse') + create_mysql_db(conn4, 'clickhouse') create_mysql_table(conn1, table_name) create_mysql_table(conn2, table_name) @@ -218,13 +230,13 @@ def test_mysql_distributed(started_cluster): nodes[i-1].query(''' CREATE TABLE test_replica{} (id UInt32, name String, age UInt32, money UInt32) - ENGINE = MySQL(`mysql{}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse');'''.format(i, i)) + ENGINE = MySQL(`mysql{}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse');'''.format(i, 57 if i==1 else i)) nodes[i-1].query("INSERT INTO test_replica{} (id, name) SELECT number, 'host{}' from numbers(10) ".format(i, i)) # test multiple ports parsing - result = node2.query('''SELECT DISTINCT(name) FROM mysql(`mysql{1|2|3}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') + result = node2.query('''SELECT DISTINCT(name) FROM mysql(`mysql{57|2|3}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') assert(result == 'host1\n' or result == 'host2\n' or result == 'host3\n') - result = node2.query('''SELECT DISTINCT(name) FROM mysql(`mysql1:3306|mysql2:3306|mysql3:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') + result = node2.query('''SELECT DISTINCT(name) FROM mysql(`mysql57:3306|mysql2:3306|mysql3:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') assert(result == 'host1\n' or result == 'host2\n' or result == 'host3\n') # check all replicas are traversed @@ -240,7 +252,7 @@ def test_mysql_distributed(started_cluster): node2.query(''' CREATE TABLE test_shards (id UInt32, name String, age UInt32, money UInt32) - ENGINE = ExternalDistributed('MySQL', `mysql{1|2}:3306,mysql{3|4}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') + ENGINE = ExternalDistributed('MySQL', `mysql{57|2}:3306,mysql{3|4}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') # Check only one replica in each shard is used result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name") @@ -254,20 +266,20 @@ def test_mysql_distributed(started_cluster): result = node2.query(query) assert(result == 'host1\nhost2\nhost3\nhost4\n') - # disconnect mysql1 - started_cluster.pause_container('mysql1') + # disconnect mysql57 + started_cluster.pause_container('mysql57') result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name") - started_cluster.unpause_container('mysql1') + started_cluster.unpause_container('mysql57') assert(result == 'host2\nhost4\n' or result == 'host3\nhost4\n') def test_external_settings(started_cluster): table_name = 'test_external_settings' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster, started_cluster.mysql_ip) create_mysql_table(conn, table_name) node3.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node3.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(100) ".format( diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 4daf46360d3..abcfb03c750 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -12,11 +12,11 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', main_configs=["configs/log_conf.xml"], with_postgres=True) node2 = cluster.add_instance('node2', main_configs=['configs/log_conf.xml'], with_postgres_cluster=True) -def get_postgres_conn(database=False, port=5432): +def get_postgres_conn(cluster, ip, database=False): if database == True: - conn_string = "host='localhost' port={} dbname='clickhouse' user='postgres' password='mysecretpassword'".format(port) + conn_string = f"host={ip} port='{cluster.postgres_port}' dbname='clickhouse' user='postgres' password='mysecretpassword'" else: - conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(port) + conn_string = f"host={ip} port='{cluster.postgres_port}' user='postgres' password='mysecretpassword'" conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) @@ -25,6 +25,7 @@ def get_postgres_conn(database=False, port=5432): def create_postgres_db(conn, name): cursor = conn.cursor() + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) cursor.execute("CREATE DATABASE {}".format(name)) @@ -32,17 +33,20 @@ def create_postgres_db(conn, name): def started_cluster(): try: cluster.start() - - postgres_conn = get_postgres_conn(port=5432) + postgres_conn = get_postgres_conn(cluster, ip=cluster.postgres_ip) + print("postgres connected") create_postgres_db(postgres_conn, 'clickhouse') - postgres_conn = get_postgres_conn(port=5421) + postgres_conn = get_postgres_conn(cluster, ip=cluster.postgres2_ip) + print("postgres2 connected") create_postgres_db(postgres_conn, 'clickhouse') - postgres_conn = get_postgres_conn(port=5441) + postgres_conn = get_postgres_conn(cluster, ip=cluster.postgres3_ip) + print("postgres2 connected") create_postgres_db(postgres_conn, 'clickhouse') - postgres_conn = get_postgres_conn(port=5461) + postgres_conn = get_postgres_conn(cluster, ip=cluster.postgres4_ip) + print("postgres2 connected") create_postgres_db(postgres_conn, 'clickhouse') print("postgres connected") @@ -53,10 +57,10 @@ def started_cluster(): def test_postgres_select_insert(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, started_cluster.postgres_ip, True) cursor = conn.cursor() table_name = 'test_many' - table = '''postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name) + table = f'''postgresql('{started_cluster.postgres_ip}:{started_cluster.postgres_port}', 'clickhouse', '{table_name}', 'postgres', 'mysecretpassword')''' cursor.execute('CREATE TABLE IF NOT EXISTS {} (a integer, b text, c integer)'.format(table_name)) result = node1.query(''' @@ -73,7 +77,7 @@ def test_postgres_select_insert(started_cluster): def test_postgres_conversions(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, started_cluster.postgres_ip, True) cursor = conn.cursor() cursor.execute( '''CREATE TABLE IF NOT EXISTS test_types ( @@ -154,7 +158,7 @@ def test_postgres_conversions(started_cluster): def test_non_default_scema(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, started_cluster.postgres_ip, True) cursor = conn.cursor() cursor.execute('CREATE SCHEMA test_schema') cursor.execute('CREATE TABLE test_schema.test_table (a integer)') @@ -186,7 +190,7 @@ def test_non_default_scema(started_cluster): def test_concurrent_queries(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, started_cluster.postgres_ip, True) cursor = conn.cursor() node1.query(''' @@ -239,10 +243,10 @@ def test_concurrent_queries(started_cluster): def test_postgres_distributed(started_cluster): - conn0 = get_postgres_conn(port=5432, database=True) - conn1 = get_postgres_conn(port=5421, database=True) - conn2 = get_postgres_conn(port=5441, database=True) - conn3 = get_postgres_conn(port=5461, database=True) + conn0 = get_postgres_conn(started_cluster, started_cluster.postgres_ip, database=True) + conn1 = get_postgres_conn(started_cluster, started_cluster.postgres2_ip, database=True) + conn2 = get_postgres_conn(started_cluster, started_cluster.postgres3_ip, database=True) + conn3 = get_postgres_conn(started_cluster, started_cluster.postgres4_ip, database=True) cursor0 = conn0.cursor() cursor1 = conn1.cursor() diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 1f14886e50f..02df75f0f04 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -3,6 +3,7 @@ import os.path as p import random import subprocess import threading +import logging import time from random import randrange @@ -19,49 +20,10 @@ cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', main_configs=['configs/rabbitmq.xml', 'configs/log_conf.xml'], with_rabbitmq=True) -# clickhouse_path_dir='clickhouse_path') -rabbitmq_id = '' # Helpers -def check_rabbitmq_is_available(): - p = subprocess.Popen(('docker', 'exec', '-i', rabbitmq_id, 'rabbitmqctl', 'await_startup'), stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def enable_consistent_hash_plugin(): - p = subprocess.Popen(('docker', 'exec', '-i', rabbitmq_id, "rabbitmq-plugins", "enable", "rabbitmq_consistent_hash_exchange"), stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def wait_rabbitmq_is_available(max_retries=50): - retries = 0 - while True: - if check_rabbitmq_is_available(): - break - else: - retries += 1 - if retries > max_retries: - raise "RabbitMQ is not available" - print("Waiting for RabbitMQ to start up") - time.sleep(1) - - -def wait_rabbitmq_plugin_enabled(max_retries=50): - retries = 0 - while True: - if enable_consistent_hash_plugin(): - break - else: - retries += 1 - if retries > max_retries: - raise "RabbitMQ plugin is not available" - print("Waiting for plugin") - time.sleep(1) - def rabbitmq_check_result(result, check=False, ref_file='test_rabbitmq_json.reference'): fpath = p.join(p.dirname(__file__), ref_file) @@ -72,13 +34,13 @@ def rabbitmq_check_result(result, check=False, ref_file='test_rabbitmq_json.refe return TSV(result) == TSV(reference) -def kill_rabbitmq(): +def kill_rabbitmq(rabbitmq_id): p = subprocess.Popen(('docker', 'stop', rabbitmq_id), stdout=subprocess.PIPE) p.communicate() return p.returncode == 0 -def revive_rabbitmq(): +def revive_rabbitmq(rabbitmq_id): p = subprocess.Popen(('docker', 'start', rabbitmq_id), stdout=subprocess.PIPE) p.communicate() return p.returncode == 0 @@ -89,10 +51,8 @@ def revive_rabbitmq(): @pytest.fixture(scope="module") def rabbitmq_cluster(): try: - global rabbitmq_id cluster.start() - rabbitmq_id = instance.cluster.rabbitmq_docker_id - print(("rabbitmq_id is {}".format(rabbitmq_id))) + logging.debug("rabbitmq_id is {}".format(instance.cluster.rabbitmq_docker_id)) instance.query('CREATE DATABASE test') yield cluster @@ -103,8 +63,6 @@ def rabbitmq_cluster(): @pytest.fixture(autouse=True) def rabbitmq_setup_teardown(): - wait_rabbitmq_is_available() - wait_rabbitmq_plugin_enabled() print("RabbitMQ is available - running test") yield # run test instance.query('DROP TABLE IF EXISTS test.rabbitmq') @@ -117,14 +75,14 @@ def test_rabbitmq_select(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_exchange_name = 'select', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; - ''') + '''.format(rabbitmq_cluster.rabbitmq_host)) credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -153,11 +111,11 @@ def test_rabbitmq_select_empty(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_exchange_name = 'empty', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; - ''') + '''.format(rabbitmq_cluster.rabbitmq_host)) assert int(instance.query('SELECT count() FROM test.rabbitmq')) == 0 @@ -167,13 +125,13 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_exchange_name = 'json', rabbitmq_format = 'JSONEachRow' - ''') + '''.format(rabbitmq_cluster.rabbitmq_host)) credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -216,7 +174,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -259,7 +217,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -299,7 +257,7 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -342,7 +300,7 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -392,7 +350,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -440,7 +398,7 @@ def test_rabbitmq_protobuf(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -493,7 +451,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): messages = [json.dumps({'key': i, 'value': 'x' * 100}) * batch_messages for i in range(rabbitmq_messages)] credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -557,7 +515,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): messages_num = 10000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): connection = pika.BlockingConnection(parameters) @@ -634,7 +592,7 @@ def test_rabbitmq_mv_combo(rabbitmq_cluster): messages_num = 10000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): connection = pika.BlockingConnection(parameters) @@ -692,7 +650,7 @@ def test_rabbitmq_insert(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() @@ -745,7 +703,7 @@ def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() @@ -974,7 +932,7 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1047,7 +1005,7 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1133,7 +1091,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1211,7 +1169,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): messages_num = 500 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): # init connection here because otherwise python rabbitmq client might fail @@ -1290,7 +1248,7 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): messages_num = 500 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): # init connection here because otherwise python rabbitmq client might fail @@ -1387,7 +1345,7 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1441,7 +1399,7 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1508,7 +1466,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1586,7 +1544,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): connection = pika.BlockingConnection(parameters) @@ -1666,7 +1624,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1689,9 +1647,9 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): while int(instance.query('SELECT count() FROM test.view')) == 0: time.sleep(0.1) - kill_rabbitmq() + kill_rabbitmq(rabbitmq_cluster.rabbitmq_docker_id) time.sleep(4) - revive_rabbitmq() + revive_rabbitmq(rabbitmq_cluster.rabbitmq_docker_id) while True: result = instance.query('SELECT count(DISTINCT key) FROM test.view') @@ -1726,7 +1684,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): messages_num = 150000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1752,9 +1710,9 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): while int(instance.query('SELECT count() FROM test.view')) == 0: time.sleep(0.1) - kill_rabbitmq() + kill_rabbitmq(rabbitmq_cluster.rabbitmq_docker_id) time.sleep(8) - revive_rabbitmq() + revive_rabbitmq(rabbitmq_cluster.rabbitmq_docker_id) # while int(instance.query('SELECT count() FROM test.view')) == 0: # time.sleep(0.1) @@ -1798,7 +1756,7 @@ def test_rabbitmq_commit_on_block_write(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1873,7 +1831,7 @@ def test_rabbitmq_no_connection_at_startup(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() for i in range(messages_num): @@ -1909,7 +1867,7 @@ def test_rabbitmq_format_factory_settings(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1961,7 +1919,7 @@ def test_rabbitmq_vhost(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() channel.basic_publish(exchange='vhost', routing_key='', body=json.dumps({'key': 1, 'value': 2})) @@ -1984,7 +1942,7 @@ def test_rabbitmq_drop_table_properly(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -2021,7 +1979,7 @@ def test_rabbitmq_queue_settings(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -2054,7 +2012,7 @@ def test_rabbitmq_queue_settings(rabbitmq_cluster): @pytest.mark.timeout(120) def test_rabbitmq_queue_consume(rabbitmq_cluster): credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() channel.queue_declare(queue='rabbit_queue', durable=True) diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py index 4a27845ff9f..03e0d8fa288 100644 --- a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -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"] diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 667b8219b00..69b0bfe205a 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -11,15 +11,14 @@ import helpers.client import pytest from helpers.cluster import ClickHouseCluster, ClickHouseInstance -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) +MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/dummy/configs/config.d/defaultS3.xml') # Creates S3 bucket for tests and allows anonymous read-write access to it. -def prepare_s3_bucket(cluster): +def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. bucket_read_write_policy = {"Version": "2012-10-17", "Statement": [ @@ -53,26 +52,26 @@ def prepare_s3_bucket(cluster): } ]} - minio_client = cluster.minio_client - minio_client.set_bucket_policy(cluster.minio_bucket, json.dumps(bucket_read_write_policy)) + minio_client = started_cluster.minio_client + minio_client.set_bucket_policy(started_cluster.minio_bucket, json.dumps(bucket_read_write_policy)) - cluster.minio_restricted_bucket = "{}-with-auth".format(cluster.minio_bucket) - if minio_client.bucket_exists(cluster.minio_restricted_bucket): - minio_client.remove_bucket(cluster.minio_restricted_bucket) + started_cluster.minio_restricted_bucket = "{}-with-auth".format(started_cluster.minio_bucket) + if minio_client.bucket_exists(started_cluster.minio_restricted_bucket): + minio_client.remove_bucket(started_cluster.minio_restricted_bucket) - minio_client.make_bucket(cluster.minio_restricted_bucket) + minio_client.make_bucket(started_cluster.minio_restricted_bucket) -def put_s3_file_content(cluster, bucket, filename, data): +def put_s3_file_content(started_cluster, bucket, filename, data): buf = io.BytesIO(data) - cluster.minio_client.put_object(bucket, filename, buf, len(data)) + started_cluster.minio_client.put_object(bucket, filename, buf, len(data)) # Returns content of given S3 file as string. -def get_s3_file_content(cluster, bucket, filename, decode=True): - # type: (ClickHouseCluster, str) -> str +def get_s3_file_content(started_cluster, bucket, filename, decode=True): + # type: (ClickHouseCluster, str, str, bool) -> str - data = cluster.minio_client.get_object(bucket, filename) + data = started_cluster.minio_client.get_object(bucket, filename) data_str = b"" for chunk in data.stream(): data_str += chunk @@ -82,7 +81,7 @@ def get_s3_file_content(cluster, bucket, filename, decode=True): @pytest.fixture(scope="module") -def cluster(): +def started_cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance("restricted_dummy", main_configs=["configs/config_for_test_remote_host_filter.xml"], @@ -115,25 +114,25 @@ def run_query(instance, query, stdin=None, settings=None): # Test simple put. Also checks that wrong credentials produce an error with every compression method. @pytest.mark.parametrize("maybe_auth,positive,compression", [ - ("", True, 'auto'), - ("'minio','minio123',", True, 'auto'), - ("'wrongid','wrongkey',", False, 'auto'), - ("'wrongid','wrongkey',", False, 'gzip'), - ("'wrongid','wrongkey',", False, 'deflate'), - ("'wrongid','wrongkey',", False, 'brotli'), - ("'wrongid','wrongkey',", False, 'xz'), - ("'wrongid','wrongkey',", False, 'zstd') + pytest.param("", True, 'auto', id="positive"), + pytest.param("'minio','minio123',", True, 'auto', id="auth_positive"), + pytest.param("'wrongid','wrongkey',", False, 'auto', id="auto"), + pytest.param("'wrongid','wrongkey',", False, 'gzip', id="gzip"), + pytest.param("'wrongid','wrongkey',", False, 'deflate', id="deflate"), + pytest.param("'wrongid','wrongkey',", False, 'brotli', id="brotli"), + pytest.param("'wrongid','wrongkey',", False, 'xz', id="xz"), + pytest.param("'wrongid','wrongkey',", False, 'zstd', id="zstd") ]) -def test_put(cluster, maybe_auth, positive, compression): +def test_put(started_cluster, maybe_auth, positive, compression): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket if not maybe_auth else cluster.minio_restricted_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket if not maybe_auth else started_cluster.minio_restricted_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" values_csv = "1,2,3\n3,2,1\n78,43,45\n" filename = "test.csv" - put_query = f"""insert into table function s3('http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/{filename}', + put_query = f"""insert into table function s3('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{filename}', {maybe_auth}'CSV', '{table_format}', {compression}) values {values}""" try: @@ -143,32 +142,32 @@ def test_put(cluster, maybe_auth, positive, compression): raise else: assert positive - assert values_csv == get_s3_file_content(cluster, bucket, filename) + assert values_csv == get_s3_file_content(started_cluster, bucket, filename) @pytest.mark.parametrize("special", [ "space", "plus" ]) -def test_get_file_with_special(cluster, special): +def test_get_file_with_special(started_cluster, special): symbol = {"space": " ", "plus": "+"}[special] urlsafe_symbol = {"space": "%20", "plus": "%2B"}[special] auth = "'minio','minio123'," - bucket = cluster.minio_restricted_bucket - instance = cluster.instances["dummy"] + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["dummy"] table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = [[12549, 2463, 19893], [64021, 38652, 66703], [81611, 39650, 83516], [11079, 59507, 61546], [51764, 69952, 6876], [41165, 90293, 29095], [40167, 78432, 48309], [81629, 81327, 11855], [55852, 21643, 98507], [6738, 54643, 41155]] values_csv = ('\n'.join((','.join(map(str, row)) for row in values)) + '\n').encode() filename = f"get_file_with_{special}_{symbol}two.csv" - put_s3_file_content(cluster, bucket, filename, values_csv) + put_s3_file_content(started_cluster, bucket, filename, values_csv) - get_query = f"SELECT * FROM s3('http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/get_file_with_{special}_{urlsafe_symbol}two.csv', {auth}'CSV', '{table_format}') FORMAT TSV" + get_query = f"SELECT * FROM s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/get_file_with_{special}_{urlsafe_symbol}two.csv', {auth}'CSV', '{table_format}') FORMAT TSV" assert [list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()] == values - get_query = f"SELECT * FROM s3('http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/get_file_with_{special}*.csv', {auth}'CSV', '{table_format}') FORMAT TSV" + get_query = f"SELECT * FROM s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/get_file_with_{special}*.csv', {auth}'CSV', '{table_format}') FORMAT TSV" assert [list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()] == values - get_query = f"SELECT * FROM s3('http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/get_file_with_{special}_{urlsafe_symbol}*.csv', {auth}'CSV', '{table_format}') FORMAT TSV" + get_query = f"SELECT * FROM s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/get_file_with_{special}_{urlsafe_symbol}*.csv', {auth}'CSV', '{table_format}') FORMAT TSV" assert [list(map(int, l.split())) for l in run_query(instance, get_query).splitlines()] == values @@ -177,25 +176,25 @@ def test_get_file_with_special(cluster, special): "plus", "plus2" ]) -def test_get_path_with_special(cluster, special): +def test_get_path_with_special(started_cluster, special): symbol = {"space": "%20", "plus": "%2B", "plus2": "%2B"}[special] safe_symbol = {"space": "%20", "plus": "+", "plus2": "%2B"}[special] auth = "'minio','minio123'," table_format = "column1 String" - instance = cluster.instances["dummy"] + instance = started_cluster.instances["dummy"] get_query = f"SELECT * FROM s3('http://resolver:8082/get-my-path/{safe_symbol}.csv', {auth}'CSV', '{table_format}') FORMAT TSV" assert run_query(instance, get_query).splitlines() == [f"/{symbol}.csv"] # Test put no data to S3. @pytest.mark.parametrize("auth", [ - "'minio','minio123'," + pytest.param("'minio','minio123',", id="minio") ]) -def test_empty_put(cluster, auth): - # type: (ClickHouseCluster) -> None +def test_empty_put(started_cluster, auth): + # type: (ClickHouseCluster, str) -> None - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" create_empty_table_query = """ @@ -208,13 +207,13 @@ def test_empty_put(cluster, auth): filename = "empty_put_test.csv" put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') select * from empty_table".format( - cluster.minio_host, cluster.minio_port, bucket, filename, auth, table_format) + started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, filename, auth, table_format) run_query(instance, put_query) try: run_query(instance, "select count(*) from s3('http://{}:{}/{}/{}', {}'CSV', '{}')".format( - cluster.minio_host, cluster.minio_port, bucket, filename, auth, table_format)) + started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, filename, auth, table_format)) assert False, "Query should be failed." except helpers.client.QueryRuntimeException as e: @@ -223,19 +222,19 @@ def test_empty_put(cluster, auth): # Test put values in CSV format. @pytest.mark.parametrize("maybe_auth,positive", [ - ("", True), - ("'minio','minio123',", True), - ("'wrongid','wrongkey',", False) + pytest.param("", True, id="positive"), + pytest.param("'minio','minio123',", True, id="auth_positive"), + pytest.param("'wrongid','wrongkey',", False, id="negative"), ]) -def test_put_csv(cluster, maybe_auth, positive): - # type: (ClickHouseCluster) -> None +def test_put_csv(started_cluster, maybe_auth, positive): + # type: (ClickHouseCluster, bool, str) -> None - bucket = cluster.minio_bucket if not maybe_auth else cluster.minio_restricted_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket if not maybe_auth else started_cluster.minio_restricted_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') format CSV".format( - cluster.minio_host, cluster.minio_port, bucket, filename, maybe_auth, table_format) + started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, filename, maybe_auth, table_format) csv_data = "8,9,16\n11,18,13\n22,14,2\n" try: @@ -245,27 +244,27 @@ def test_put_csv(cluster, maybe_auth, positive): raise else: assert positive - assert csv_data == get_s3_file_content(cluster, bucket, filename) + assert csv_data == get_s3_file_content(started_cluster, bucket, filename) # Test put and get with S3 server redirect. -def test_put_get_with_redirect(cluster): +def test_put_get_with_redirect(started_cluster): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" values_csv = "1,1,1\n1,1,1\n11,11,11\n" filename = "test.csv" query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, filename, table_format, values) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, table_format, values) run_query(instance, query) - assert values_csv == get_s3_file_content(cluster, bucket, filename) + assert values_csv == get_s3_file_content(started_cluster, bucket, filename) query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/{}', 'CSV', '{}')".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, filename, table_format) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, table_format) stdout = run_query(instance, query) assert list(map(str.split, stdout.splitlines())) == [ @@ -276,23 +275,23 @@ def test_put_get_with_redirect(cluster): # Test put with restricted S3 server redirect. -def test_put_with_zero_redirect(cluster): +def test_put_with_zero_redirect(started_cluster): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket - instance = cluster.instances["s3_max_redirects"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["s3_max_redirects"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" filename = "test.csv" # Should work without redirect query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_host, cluster.minio_port, bucket, filename, table_format, values) + started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, filename, table_format, values) run_query(instance, query) # Should not work with redirect query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, filename, table_format, values) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, table_format, values) exception_raised = False try: run_query(instance, query) @@ -303,11 +302,11 @@ def test_put_with_zero_redirect(cluster): assert exception_raised -def test_put_get_with_globs(cluster): +def test_put_get_with_globs(started_cluster): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" for i in range(10): @@ -316,26 +315,26 @@ def test_put_get_with_globs(cluster): max_path = max(path, max_path) values = "({},{},{})".format(i, j, i + j) query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_host, cluster.minio_port, bucket, path, table_format, values) + started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, path, table_format, values) run_query(instance, query) query = "select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from s3('http://{}:{}/{}/*_{{a,b,c,d}}/%3f.csv', 'CSV', '{}')".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, table_format) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, table_format) assert run_query(instance, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format(bucket=bucket, max_path=max_path)] # Test multipart put. @pytest.mark.parametrize("maybe_auth,positive", [ - ("", True), + pytest.param("", True, id="positive"), + pytest.param("'wrongid','wrongkey'", False, id="negative"), # ("'minio','minio123',",True), Redirect with credentials not working with nginx. - ("'wrongid','wrongkey',", False), ]) -def test_multipart_put(cluster, maybe_auth, positive): +def test_multipart_put(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket if not maybe_auth else cluster.minio_restricted_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket if not maybe_auth else started_cluster.minio_restricted_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" # Minimum size of part is 5 Mb for Minio. @@ -353,7 +352,7 @@ def test_multipart_put(cluster, maybe_auth, positive): filename = "test_multipart.csv" put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') format CSV".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, filename, maybe_auth, table_format) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, maybe_auth, table_format) try: run_query(instance, put_query, stdin=csv_data, settings={'s3_min_upload_part_size': min_part_size_bytes, @@ -365,32 +364,32 @@ def test_multipart_put(cluster, maybe_auth, positive): assert positive # Use proxy access logs to count number of parts uploaded to Minio. - proxy_logs = cluster.get_container_logs("proxy1") # type: str + proxy_logs = started_cluster.get_container_logs("proxy1") # type: str assert proxy_logs.count("PUT /{}/{}".format(bucket, filename)) >= 2 - assert csv_data == get_s3_file_content(cluster, bucket, filename) + assert csv_data == get_s3_file_content(started_cluster, bucket, filename) -def test_remote_host_filter(cluster): - instance = cluster.instances["restricted_dummy"] +def test_remote_host_filter(started_cluster): + instance = started_cluster.instances["restricted_dummy"] format = "column1 UInt32, column2 UInt32, column3 UInt32" query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format( - "invalid_host", cluster.minio_port, cluster.minio_bucket, format) + "invalid_host", MINIO_INTERNAL_PORT, started_cluster.minio_bucket, format) assert "not allowed in config.xml" in instance.query_and_get_error(query) other_values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format( - "invalid_host", cluster.minio_port, cluster.minio_bucket, format, other_values) + "invalid_host", MINIO_INTERNAL_PORT, started_cluster.minio_bucket, format, other_values) assert "not allowed in config.xml" in instance.query_and_get_error(query) @pytest.mark.parametrize("s3_storage_args", [ - "''", # 1 arguments - "'','','','','',''" # 6 arguments + pytest.param("''", id="1_argument"), + pytest.param("'','','','','',''", id="6_arguments"), ]) -def test_wrong_s3_syntax(cluster, s3_storage_args): - instance = cluster.instances["dummy"] # type: ClickHouseInstance +def test_wrong_s3_syntax(started_cluster, s3_storage_args): + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance expected_err_msg = "Code: 42" # NUMBER_OF_ARGUMENTS_DOESNT_MATCH query = "create table test_table_s3_syntax (id UInt32) ENGINE = S3({})".format(s3_storage_args) @@ -398,9 +397,9 @@ def test_wrong_s3_syntax(cluster, s3_storage_args): # https://en.wikipedia.org/wiki/One_Thousand_and_One_Nights -def test_s3_glob_scheherazade(cluster): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance +def test_s3_glob_scheherazade(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" values = "(1, 1, 1)" @@ -411,7 +410,7 @@ def test_s3_glob_scheherazade(cluster): for i in range(start, end): path = "night_{}/tale.csv".format(i) query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_host, cluster.minio_port, bucket, path, table_format, values) + started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, path, table_format, values) run_query(instance, query) jobs.append(threading.Thread(target=add_tales, args=(night, min(night + nights_per_job, 1001)))) @@ -421,11 +420,12 @@ def test_s3_glob_scheherazade(cluster): job.join() query = "select count(), sum(column1), sum(column2), sum(column3) from s3('http://{}:{}/{}/night_*/tale.csv', 'CSV', '{}')".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, table_format) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, table_format) assert run_query(instance, query).splitlines() == ["1001\t1001\t1001\t1001"] -def run_s3_mocks(cluster): + +def run_s3_mocks(started_cluster): logging.info("Starting s3 mocks") mocks = ( ("mock_s3.py", "resolver", "8080"), @@ -433,22 +433,24 @@ def run_s3_mocks(cluster): ("echo.py", "resolver", "8082"), ) for mock_filename, container, port in mocks: - container_id = cluster.get_container_id(container) + container_id = started_cluster.get_container_id(container) current_dir = os.path.dirname(__file__) - cluster.copy_file_to_container(container_id, os.path.join(current_dir, "s3_mocks", mock_filename), mock_filename) - cluster.exec_in_container(container_id, ["python", mock_filename, port], detach=True) + started_cluster.copy_file_to_container(container_id, os.path.join(current_dir, "s3_mocks", mock_filename), mock_filename) + started_cluster.exec_in_container(container_id, ["python", mock_filename, port], detach=True) # Wait for S3 mocks to start for mock_filename, container, port in mocks: - for attempt in range(10): - ping_response = cluster.exec_in_container(cluster.get_container_id(container), - ["curl", "-s", f"http://{container}:{port}/"], nothrow=True) + num_attempts = 100 + for attempt in range(num_attempts): + ping_response = started_cluster.exec_in_container(started_cluster.get_container_id(container), + ["curl", "-s", f"http://localhost:{port}/"], nothrow=True) if ping_response != 'OK': - if attempt == 9: + if attempt == num_attempts - 1: assert ping_response == 'OK', 'Expected "OK", but got "{}"'.format(ping_response) else: time.sleep(1) else: + logging.debug(f"mock {mock_filename} ({port}) answered {ping_response} on attempt {attempt}") break logging.info("S3 mocks started") @@ -464,21 +466,21 @@ def replace_config(old, new): config.close() -def test_custom_auth_headers(cluster): +def test_custom_auth_headers(started_cluster): table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" get_query = "select * from s3('http://resolver:8080/{bucket}/{file}', 'CSV', '{table_format}')".format( - bucket=cluster.minio_restricted_bucket, + bucket=started_cluster.minio_restricted_bucket, file=filename, table_format=table_format) - instance = cluster.instances["dummy"] # type: ClickHouseInstance + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance result = run_query(instance, get_query) assert result == '1\t2\t3\n' instance.query( "CREATE TABLE test ({table_format}) ENGINE = S3('http://resolver:8080/{bucket}/{file}', 'CSV')".format( - bucket=cluster.minio_restricted_bucket, + bucket=started_cluster.minio_restricted_bucket, file=filename, table_format=table_format )) @@ -493,12 +495,12 @@ def test_custom_auth_headers(cluster): assert run_query(instance, "SELECT * FROM test") == '1\t2\t3\n' -def test_custom_auth_headers_exclusion(cluster): +def test_custom_auth_headers_exclusion(started_cluster): table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" - get_query = f"SELECT * FROM s3('http://resolver:8080/{cluster.minio_restricted_bucket}/restricteddirectory/{filename}', 'CSV', '{table_format}')" + get_query = f"SELECT * FROM s3('http://resolver:8080/{started_cluster.minio_restricted_bucket}/restricteddirectory/{filename}', 'CSV', '{table_format}')" - instance = cluster.instances["dummy"] # type: ClickHouseInstance + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance with pytest.raises(helpers.client.QueryRuntimeException) as ei: result = run_query(instance, get_query) print(result) @@ -507,15 +509,12 @@ def test_custom_auth_headers_exclusion(cluster): assert 'Forbidden Error' in ei.value.stderr -def test_infinite_redirect(cluster): +def test_infinite_redirect(started_cluster): bucket = "redirected" table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" - get_query = "select * from s3('http://resolver:8080/{bucket}/{file}', 'CSV', '{table_format}')".format( - bucket=bucket, - file=filename, - table_format=table_format) - instance = cluster.instances["dummy"] # type: ClickHouseInstance + get_query = f"select * from s3('http://resolver:{started_cluster.minio_redirect_port}/{bucket}/{filename}', 'CSV', '{table_format}')" + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance exception_raised = False try: run_query(instance, get_query) @@ -524,17 +523,15 @@ def test_infinite_redirect(cluster): exception_raised = True finally: assert exception_raised - - @pytest.mark.parametrize("extension,method", [ - ("bin", "gzip"), - ("gz", "auto") + pytest.param("bin", "gzip", id="bin"), + pytest.param("gz", "auto", id="gz"), ]) -def test_storage_s3_get_gzip(cluster, extension, method): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] +def test_storage_s3_get_gzip(started_cluster, extension, method): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] filename = f"test_get_gzip.{extension}" - name = "test_get_gzip" + name = f"test_get_gzip_{extension}" data = [ "Sophia Intrieri,55", "Jack Taylor,71", @@ -557,32 +554,28 @@ def test_storage_s3_get_gzip(cluster, extension, method): compressed = gzip.GzipFile(fileobj=buf, mode="wb") compressed.write(("\n".join(data)).encode()) compressed.close() - put_s3_file_content(cluster, bucket, filename, buf.getvalue()) + put_s3_file_content(started_cluster, bucket, filename, buf.getvalue()) - try: - run_query(instance, f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = S3( - 'http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/{filename}', - 'CSV', - '{method}')""") + run_query(instance, f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = S3( + 'http://{started_cluster.minio_ip}:{MINIO_INTERNAL_PORT}/{bucket}/{filename}', + 'CSV', + '{method}')""") - run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["565"] - - finally: - run_query(instance, f"DROP TABLE {name}") + run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["565"] -def test_storage_s3_get_unstable(cluster): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] +def test_storage_s3_get_unstable(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] table_format = "column1 Int64, column2 Int64, column3 Int64, column4 Int64" - get_query = f"SELECT count(), sum(column3) FROM s3('http://resolver:8081/{cluster.minio_bucket}/test.csv', 'CSV', '{table_format}') FORMAT CSV" + get_query = f"SELECT count(), sum(column3) FROM s3('http://resolver:8081/{started_cluster.minio_bucket}/test.csv', 'CSV', '{table_format}') FORMAT CSV" result = run_query(instance, get_query) assert result.splitlines() == ["500000,500000"] -def test_storage_s3_put_uncompressed(cluster): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] +def test_storage_s3_put_uncompressed(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] filename = "test_put_uncompressed.bin" name = "test_put_uncompressed" data = [ @@ -602,29 +595,26 @@ def test_storage_s3_put_uncompressed(cluster): "'Kathie Dawson',100", "'Gregg Mcquistion',11", ] - try: - run_query(instance, "CREATE TABLE {} (name String, id UInt32) ENGINE = S3('http://{}:{}/{}/{}', 'CSV')".format( - name, cluster.minio_host, cluster.minio_port, bucket, filename)) + run_query(instance, "CREATE TABLE {} (name String, id UInt32) ENGINE = S3('http://{}:{}/{}/{}', 'CSV')".format( + name, started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, filename)) - run_query(instance, "INSERT INTO {} VALUES ({})".format(name, "),(".join(data))) + run_query(instance, "INSERT INTO {} VALUES ({})".format(name, "),(".join(data))) - run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["753"] + run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["753"] - uncompressed_content = get_s3_file_content(cluster, bucket, filename) - assert sum([ int(i.split(',')[1]) for i in uncompressed_content.splitlines() ]) == 753 - finally: - run_query(instance, f"DROP TABLE {name}") + uncompressed_content = get_s3_file_content(started_cluster, bucket, filename) + assert sum([ int(i.split(',')[1]) for i in uncompressed_content.splitlines() ]) == 753 @pytest.mark.parametrize("extension,method", [ - ("bin", "gzip"), - ("gz", "auto") + pytest.param("bin", "gzip", id="bin"), + pytest.param("gz", "auto", id="gz") ]) -def test_storage_s3_put_gzip(cluster, extension, method): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] +def test_storage_s3_put_gzip(started_cluster, extension, method): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] filename = f"test_put_gzip.{extension}" - name = "test_put_gzip" + name = f"test_put_gzip_{extension}" data = [ "'Joseph Tomlinson',5", "'Earnest Essary',44", @@ -642,19 +632,16 @@ def test_storage_s3_put_gzip(cluster, extension, method): "'Amanda Cave',83", "'Yolanda Joseph',89" ] - try: - run_query(instance, f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = S3( - 'http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/{filename}', - 'CSV', - '{method}')""") + run_query(instance, f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = S3( + 'http://{started_cluster.minio_ip}:{MINIO_INTERNAL_PORT}/{bucket}/{filename}', + 'CSV', + '{method}')""") - run_query(instance, "INSERT INTO {} VALUES ({})".format(name, "),(".join(data))) + run_query(instance, f"INSERT INTO {name} VALUES ({'),('.join(data)})") - run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["708"] + run_query(instance, f"SELECT sum(id) FROM {name}").splitlines() == ["708"] - buf = io.BytesIO(get_s3_file_content(cluster, bucket, filename, decode=False)) - f = gzip.GzipFile(fileobj=buf, mode="rb") - uncompressed_content = f.read().decode() - assert sum([ int(i.split(',')[1]) for i in uncompressed_content.splitlines() ]) == 708 - finally: - run_query(instance, f"DROP TABLE {name}") + buf = io.BytesIO(get_s3_file_content(started_cluster, bucket, filename, decode=False)) + f = gzip.GzipFile(fileobj=buf, mode="rb") + uncompressed_content = f.read().decode() + assert sum([ int(i.split(',')[1]) for i in uncompressed_content.splitlines() ]) == 708 diff --git a/tests/integration/test_system_clusters_actual_information/test.py b/tests/integration/test_system_clusters_actual_information/test.py index afbaf53089f..48f654dc30a 100644 --- a/tests/integration/test_system_clusters_actual_information/test.py +++ b/tests/integration/test_system_clusters_actual_information/test.py @@ -17,11 +17,12 @@ node_1 = cluster.add_instance('node_1', with_zookeeper=True) def started_cluster(): try: cluster.start() + node_1.query_with_retry('DROP TABLE IF EXISTS replicated') - node_1.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = + node_1.query_with_retry('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated', 'node_1') ORDER BY id PARTITION BY toYYYYMM(date)''') - node.query("CREATE TABLE distributed (id UInt32, date Date) ENGINE = Distributed('test_cluster', 'default', 'replicated')") + node.query_with_retry("CREATE TABLE distributed (id UInt32, date Date) ENGINE = Distributed('test_cluster', 'default', 'replicated')") yield cluster diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 7e01b919589..254447478f9 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -73,12 +73,12 @@ def wait_parts_mover(node, table, *args, **kwargs): @pytest.mark.parametrize("name,engine,alter", [ - ("mt_test_rule_with_invalid_destination", "MergeTree()", 0), - ("replicated_mt_test_rule_with_invalid_destination", - "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 0), - ("mt_test_rule_with_invalid_destination", "MergeTree()", 1), - ("replicated_mt_test_rule_with_invalid_destination", - "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 1), + pytest.param("mt_test_rule_with_invalid_destination", "MergeTree()", 0, id="case0"), + pytest.param("replicated_mt_test_rule_with_invalid_destination", + "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 0, id="case1"), + pytest.param("mt_test_rule_with_invalid_destination", "MergeTree()", 1, id="case2"), + pytest.param("replicated_mt_test_rule_with_invalid_destination", + "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 1, id="case3"), ]) def test_rule_with_invalid_destination(started_cluster, name, engine, alter): name = unique_table_name(name) @@ -136,12 +136,12 @@ def test_rule_with_invalid_destination(started_cluster, name, engine, alter): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_inserts_to_disk_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_inserts_to_disk_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_do_not_work', '1')", 0), - ("mt_test_inserts_to_disk_work", "MergeTree()", 1), - ("replicated_mt_test_inserts_to_disk_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_work', '1')", 1), + pytest.param("mt_test_inserts_to_disk_do_not_work", "MergeTree()", 0, id="mt_test_inserts_to_disk_do_not_work"), + pytest.param("replicated_mt_test_inserts_to_disk_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_do_not_work', '1')", 0, id="replicated_mt_test_inserts_to_disk_do_not_work"), + pytest.param("mt_test_inserts_to_disk_work", "MergeTree()", 1, id="mt_test_inserts_to_disk_work_1"), + pytest.param("replicated_mt_test_inserts_to_disk_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_work', '1')", 1, id="replicated_mt_test_inserts_to_disk_work_1"), ]) def test_inserts_to_disk_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -176,9 +176,9 @@ def test_inserts_to_disk_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine", [ - ("mt_test_moves_work_after_storage_policy_change", "MergeTree()"), - ("replicated_mt_test_moves_work_after_storage_policy_change", - "ReplicatedMergeTree('/clickhouse/test_moves_work_after_storage_policy_change', '1')"), + pytest.param("mt_test_moves_work_after_storage_policy_change", "MergeTree()", id="mt_test_moves_work_after_storage_policy_change"), + pytest.param("replicated_mt_test_moves_work_after_storage_policy_change", + "ReplicatedMergeTree('/clickhouse/test_moves_work_after_storage_policy_change', '1')", id="replicated_mt_test_moves_work_after_storage_policy_change"), ]) def test_moves_work_after_storage_policy_change(started_cluster, name, engine): name = unique_table_name(name) @@ -224,12 +224,12 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_moves_to_disk_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_moves_to_disk_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_do_not_work', '1')", 0), - ("mt_test_moves_to_disk_work", "MergeTree()", 1), - ("replicated_mt_test_moves_to_disk_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_work', '1')", 1), + pytest.param("mt_test_moves_to_disk_do_not_work", "MergeTree()", 0, id="mt_test_moves_to_disk_do_not_work"), + pytest.param("replicated_mt_test_moves_to_disk_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_do_not_work', '1')", 0, id="replicated_mt_test_moves_to_disk_do_not_work"), + pytest.param("mt_test_moves_to_disk_work", "MergeTree()", 1, id="mt_test_moves_to_disk_work"), + pytest.param("replicated_mt_test_moves_to_disk_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_work', '1')", 1, id="replicated_mt_test_moves_to_disk_work"), ]) def test_moves_to_disk_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -275,9 +275,9 @@ def test_moves_to_disk_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine", [ - ("mt_test_moves_to_volume_work", "MergeTree()"), - ("replicated_mt_test_moves_to_volume_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_volume_work', '1')"), + pytest.param("mt_test_moves_to_volume_work", "MergeTree()", id="mt_test_moves_to_volume_work"), + pytest.param("replicated_mt_test_moves_to_volume_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_volume_work', '1')", id="replicated_mt_test_moves_to_volume_work"), ]) def test_moves_to_volume_work(started_cluster, name, engine): name = unique_table_name(name) @@ -322,12 +322,12 @@ def test_moves_to_volume_work(started_cluster, name, engine): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_inserts_to_volume_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_inserts_to_volume_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_do_not_work', '1')", 0), - ("mt_test_inserts_to_volume_work", "MergeTree()", 1), - ("replicated_mt_test_inserts_to_volume_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_work', '1')", 1), + pytest.param("mt_test_inserts_to_volume_do_not_work", "MergeTree()", 0, id="mt_test_inserts_to_volume_do_not_work"), + pytest.param("replicated_mt_test_inserts_to_volume_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_do_not_work', '1')", 0, id="replicated_mt_test_inserts_to_volume_do_not_work"), + pytest.param("mt_test_inserts_to_volume_work", "MergeTree()", 1, id="mt_test_inserts_to_volume_work"), + pytest.param("replicated_mt_test_inserts_to_volume_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_work', '1')", 1, id="replicated_mt_test_inserts_to_volume_work"), ]) def test_inserts_to_volume_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -366,9 +366,9 @@ def test_inserts_to_volume_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine", [ - ("mt_test_moves_to_disk_eventually_work", "MergeTree()"), - ("replicated_mt_test_moves_to_disk_eventually_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_eventually_work', '1')"), + pytest.param("mt_test_moves_to_disk_eventually_work", "MergeTree()", id="mt_test_moves_to_disk_eventually_work"), + pytest.param("replicated_mt_test_moves_to_disk_eventually_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_eventually_work', '1')", id="replicated_mt_test_moves_to_disk_eventually_work"), ]) def test_moves_to_disk_eventually_work(started_cluster, name, engine): name = unique_table_name(name) @@ -460,12 +460,12 @@ def test_replicated_download_ttl_info(started_cluster): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_merges_to_disk_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_merges_to_disk_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_do_not_work', '1')", 0), - ("mt_test_merges_to_disk_work", "MergeTree()", 1), - ("replicated_mt_test_merges_to_disk_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_work', '1')", 1), + pytest.param("mt_test_merges_to_disk_do_not_work", "MergeTree()", 0, id="mt_test_merges_to_disk_do_not_work"), + pytest.param("replicated_mt_test_merges_to_disk_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_do_not_work', '1')", 0, id="mt_test_merges_to_disk_do_not_work"), + pytest.param("mt_test_merges_to_disk_work", "MergeTree()", 1, id="mt_test_merges_to_disk_work"), + pytest.param("replicated_mt_test_merges_to_disk_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_work', '1')", 1, id="replicated_mt_test_merges_to_disk_work"), ]) def test_merges_to_disk_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -524,9 +524,9 @@ def test_merges_to_disk_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine", [ - ("mt_test_merges_with_full_disk_work", "MergeTree()"), - ("replicated_mt_test_merges_with_full_disk_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_merges_with_full_disk_work', '1')"), + pytest.param("mt_test_merges_with_full_disk_work", "MergeTree()", id="mt_test_merges_with_full_disk_work"), + pytest.param("replicated_mt_test_merges_with_full_disk_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_merges_with_full_disk_work', '1')", id="replicated_mt_test_merges_with_full_disk_work"), ]) def test_merges_with_full_disk_work(started_cluster, name, engine): name = unique_table_name(name) @@ -596,12 +596,12 @@ def test_merges_with_full_disk_work(started_cluster, name, engine): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_moves_after_merges_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_moves_after_merges_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_do_not_work', '1')", 0), - ("mt_test_moves_after_merges_work", "MergeTree()", 1), - ("replicated_mt_test_moves_after_merges_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_work', '1')", 1), + pytest.param("mt_test_moves_after_merges_do_not_work", "MergeTree()", 0, id="mt_test_moves_after_merges_do_not_work"), + pytest.param("replicated_mt_test_moves_after_merges_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_do_not_work', '1')", 0, id="replicated_mt_test_moves_after_merges_do_not_work"), + pytest.param("mt_test_moves_after_merges_work", "MergeTree()", 1, id="mt_test_moves_after_merges_work"), + pytest.param("replicated_mt_test_moves_after_merges_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_work', '1')", 1, id="replicated_mt_test_moves_after_merges_work"), ]) def test_moves_after_merges_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -654,18 +654,18 @@ def test_moves_after_merges_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine,positive,bar", [ - ("mt_test_moves_after_alter_do_not_work", "MergeTree()", 0, "DELETE"), - ("replicated_mt_test_moves_after_alter_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')", 0, "DELETE"), - ("mt_test_moves_after_alter_work", "MergeTree()", 1, "DELETE"), - ("replicated_mt_test_moves_after_alter_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "DELETE"), - ("mt_test_moves_after_alter_do_not_work", "MergeTree()", 0, "TO DISK 'external'"), - ("replicated_mt_test_moves_after_alter_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')", 0, "TO DISK 'external'"), - ("mt_test_moves_after_alter_work", "MergeTree()", 1, "TO DISK 'external'"), - ("replicated_mt_test_moves_after_alter_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "TO DISK 'external'"), + pytest.param("mt_test_moves_after_alter_do_not_work", "MergeTree()", 0, "DELETE", id="mt_negative"), + pytest.param("replicated_mt_test_moves_after_alter_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')", 0, "DELETE", id="repicated_negative"), + pytest.param("mt_test_moves_after_alter_work", "MergeTree()", 1, "DELETE", id="mt_positive"), + pytest.param("replicated_mt_test_moves_after_alter_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "DELETE", id="repicated_positive"), + pytest.param("mt_test_moves_after_alter_do_not_work", "MergeTree()", 0, "TO DISK 'external'", id="mt_external_negative"), + pytest.param("replicated_mt_test_moves_after_alter_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')", 0, "TO DISK 'external'", id="replicated_external_negative"), + pytest.param("mt_test_moves_after_alter_work", "MergeTree()", 1, "TO DISK 'external'", id="mt_external_positive"), + pytest.param("replicated_mt_test_moves_after_alter_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "TO DISK 'external'", id="replicated_external_positive"), ]) def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive, bar): name = unique_table_name(name) @@ -704,9 +704,9 @@ def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive, b @pytest.mark.parametrize("name,engine", [ - ("mt_test_materialize_ttl_in_partition", "MergeTree()"), - ("replicated_mt_test_materialize_ttl_in_partition", - "ReplicatedMergeTree('/clickhouse/test_materialize_ttl_in_partition', '1')"), + pytest.param("mt_test_materialize_ttl_in_partition", "MergeTree()", id="mt"), + pytest.param("replicated_mt_test_materialize_ttl_in_partition", + "ReplicatedMergeTree('/clickhouse/test_materialize_ttl_in_partition', '1')", id="replicated"), ]) def test_materialize_ttl_in_partition(started_cluster, name, engine): name = unique_table_name(name) @@ -769,12 +769,12 @@ def test_materialize_ttl_in_partition(started_cluster, name, engine): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_alter_multiple_ttls_positive", "MergeTree()", True), - ("mt_replicated_test_alter_multiple_ttls_positive", - "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_positive', '1')", True), - ("mt_test_alter_multiple_ttls_negative", "MergeTree()", False), - ("mt_replicated_test_alter_multiple_ttls_negative", - "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_negative', '1')", False), + pytest.param("mt_test_alter_multiple_ttls_positive", "MergeTree()", True, id="positive"), + pytest.param("mt_replicated_test_alter_multiple_ttls_positive", + "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_positive', '1')", True, id="replicated_positive"), + pytest.param("mt_test_alter_multiple_ttls_negative", "MergeTree()", False, id="negative"), + pytest.param("mt_replicated_test_alter_multiple_ttls_negative", + "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_negative', '1')", False, id="replicated_negative"), ]) def test_alter_multiple_ttls(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -868,9 +868,9 @@ limitations under the License.""" @pytest.mark.parametrize("name,engine", [ - ("concurrently_altering_ttl_mt", "MergeTree()"), - ("concurrently_altering_ttl_replicated_mt", - "ReplicatedMergeTree('/clickhouse/concurrently_altering_ttl_replicated_mt', '1')",), + pytest.param("concurrently_altering_ttl_mt", "MergeTree()", id="mt"), + pytest.param("concurrently_altering_ttl_replicated_mt", + "ReplicatedMergeTree('/clickhouse/concurrently_altering_ttl_replicated_mt', '1')", id="replicated_mt"), ]) def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): name = unique_table_name(name) @@ -977,8 +977,8 @@ def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): @pytest.mark.skip(reason="Flacky test") @pytest.mark.parametrize("name,positive", [ - ("test_double_move_while_select_negative", 0), - ("test_double_move_while_select_positive", 1), + pytest.param("test_double_move_while_select_negative", 0, id="negative"), + pytest.param("test_double_move_while_select_positive", 1, id="positive"), ]) def test_double_move_while_select(started_cluster, name, positive): name = unique_table_name(name) @@ -1038,12 +1038,12 @@ def test_double_move_while_select(started_cluster, name, positive): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_alter_with_merge_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_alter_with_merge_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_do_not_work', '1')", 0), - ("mt_test_alter_with_merge_work", "MergeTree()", 1), - ("replicated_mt_test_alter_with_merge_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_work', '1')", 1), + pytest.param("mt_test_alter_with_merge_do_not_work", "MergeTree()", 0, id="mt"), + pytest.param("replicated_mt_test_alter_with_merge_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_do_not_work', '1')", 0, id="replicated"), + pytest.param("mt_test_alter_with_merge_work", "MergeTree()", 1, id="mt_work"), + pytest.param("replicated_mt_test_alter_with_merge_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_work', '1')", 1, id="replicated_work"), ]) def test_alter_with_merge_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -1131,10 +1131,10 @@ limitations under the License.""" @pytest.mark.parametrize("name,dest_type,engine", [ - ("mt_test_disabled_ttl_move_on_insert_work", "DISK", "MergeTree()"), - ("mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "MergeTree()"), - ("replicated_mt_test_disabled_ttl_move_on_insert_work", "DISK", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')"), - ("replicated_mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')"), + pytest.param("mt_test_disabled_ttl_move_on_insert_work", "DISK", "MergeTree()", id="disk"), + pytest.param("mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "MergeTree()", id="volume"), + pytest.param("replicated_mt_test_disabled_ttl_move_on_insert_work", "DISK", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')", id="replicated_disk"), + pytest.param("replicated_mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')", id="replicated_volume"), ]) def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): name = unique_table_name(name) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index f32edc36a71..4d8d36e538f 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -227,8 +227,8 @@ def optimize_with_retry(node, table_name, retry=20): time.sleep(0.5) @pytest.mark.parametrize("name,engine", [ - ("test_ttl_alter_delete", "MergeTree()"), - ("test_replicated_ttl_alter_delete", "ReplicatedMergeTree('/clickhouse/test_replicated_ttl_alter_delete', '1')"), + pytest.param("test_ttl_alter_delete", "MergeTree()", id="test_ttl_alter_delete"), + pytest.param("test_replicated_ttl_alter_delete", "ReplicatedMergeTree('/clickhouse/test_replicated_ttl_alter_delete', '1')", id="test_ttl_alter_delete_replicated"), ]) def test_ttl_alter_delete(started_cluster, name, engine): """Copyright 2019, Altinity LTD diff --git a/tests/integration/test_user_directories/test.py b/tests/integration/test_user_directories/test.py index 84547293f0d..1ce4e377f2b 100644 --- a/tests/integration/test_user_directories/test.py +++ b/tests/integration/test_user_directories/test.py @@ -15,7 +15,7 @@ def started_cluster(): cluster.start() for i in range(2, 8): - node.exec_in_container("cp /etc/clickhouse-server/users.xml /etc/clickhouse-server/users{}.xml".format(i)) + node.exec_in_container(["cp", "/etc/clickhouse-server/users.xml", "/etc/clickhouse-server/users{}.xml".format(i)]) yield cluster diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index 1ef65512959..dd8e1bc7a9e 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -33,7 +33,7 @@ def test_mutate_and_upgrade(start_cluster): node1.query("INSERT INTO mt VALUES ('2020-02-13', 1), ('2020-02-13', 2);") node1.query("ALTER TABLE mt DELETE WHERE id = 2", settings={"mutations_sync": "2"}) - node2.query("SYSTEM SYNC REPLICA mt", timeout=5) + node2.query("SYSTEM SYNC REPLICA mt", timeout=15) node1.restart_with_latest_version(signal=9) node2.restart_with_latest_version(signal=9) @@ -47,14 +47,14 @@ def test_mutate_and_upgrade(start_cluster): node1.query("INSERT INTO mt VALUES ('2020-02-13', 4);") - node2.query("SYSTEM SYNC REPLICA mt", timeout=5) + node2.query("SYSTEM SYNC REPLICA mt", timeout=15) assert node1.query("SELECT COUNT() FROM mt") == "3\n" assert node2.query("SELECT COUNT() FROM mt") == "3\n" node2.query("ALTER TABLE mt DELETE WHERE id = 3", settings={"mutations_sync": "2"}) - node1.query("SYSTEM SYNC REPLICA mt", timeout=5) + node1.query("SYSTEM SYNC REPLICA mt", timeout=15) assert node1.query("SELECT COUNT() FROM mt") == "2\n" assert node2.query("SELECT COUNT() FROM mt") == "2\n" diff --git a/tests/integration/test_zookeeper_config/configs/remote_servers.xml b/tests/integration/test_zookeeper_config/configs/remote_servers.xml index 01865e33a85..8e972b31017 100644 --- a/tests/integration/test_zookeeper_config/configs/remote_servers.xml +++ b/tests/integration/test_zookeeper_config/configs/remote_servers.xml @@ -11,6 +11,12 @@ node2 9000 + + + node3 + 9000 + + diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 432765e6dd2..732816ea224 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -1,195 +1,55 @@ - - import time -import threading -from os import path as p, unlink -from tempfile import NamedTemporaryFile - -import helpers import pytest +import logging from helpers.cluster import ClickHouseCluster +cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') -def test_chroot_with_same_root(): - cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') - cluster_2 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') +node1 = cluster.add_instance('node1', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"]) +node2 = cluster.add_instance('node2', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"]) +node3 = cluster.add_instance('node3', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_b.xml"]) - node1 = cluster_1.add_instance('node1', - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"], - with_zookeeper=True, zookeeper_use_tmpfs=False) - node2 = cluster_2.add_instance('node2', - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"], - with_zookeeper=True, zookeeper_use_tmpfs=False) - nodes = [node1, node2] - - def create_zk_root(zk): - zk.ensure_path('/root_a') - print(zk.get_children('/')) - - cluster_1.add_zookeeper_startup_command(create_zk_root) - - try: - cluster_1.start() - - try: - cluster_2.start(destroy_dirs=False) - for i, node in enumerate(nodes): - node.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); - '''.format(replica=node.name)) - for j in range(2): # Second insert to test deduplication - node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) - - time.sleep(1) - - assert node1.query('select count() from simple').strip() == '2' - assert node2.query('select count() from simple').strip() == '2' - - finally: - cluster_2.shutdown() - - finally: - cluster_1.shutdown() - - -def test_chroot_with_different_root(): - cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml') - cluster_2 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_b.xml') - - node1 = cluster_1.add_instance('node1', - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"], - with_zookeeper=True, zookeeper_use_tmpfs=False) - node2 = cluster_2.add_instance('node2', - main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_b.xml"], - with_zookeeper=True, zookeeper_use_tmpfs=False) - nodes = [node1, node2] - - def create_zk_roots(zk): - zk.ensure_path('/root_a') - zk.ensure_path('/root_b') - print(zk.get_children('/')) - - cluster_1.add_zookeeper_startup_command(create_zk_roots) - - try: - cluster_1.start() - - try: - cluster_2.start(destroy_dirs=False) - - for i, node in enumerate(nodes): - node.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); - '''.format(replica=node.name)) - for j in range(2): # Second insert to test deduplication - node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) - - assert node1.query('select count() from simple').strip() == '1' - assert node2.query('select count() from simple').strip() == '1' - - finally: - cluster_2.shutdown() - - finally: - cluster_1.shutdown() - - -def test_identity(): - cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_with_password.xml') - cluster_2 = ClickHouseCluster(__file__) - - node1 = cluster_1.add_instance('node1', main_configs=["configs/remote_servers.xml", - "configs/zookeeper_config_with_password.xml"], - with_zookeeper=True, zookeeper_use_tmpfs=False) - node2 = cluster_2.add_instance('node2', main_configs=["configs/remote_servers.xml"], with_zookeeper=True, - zookeeper_use_tmpfs=False) - - try: - cluster_1.start() - - node1.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); - '''.format(replica=node1.name)) - - with pytest.raises(Exception): - cluster_2.start(destroy_dirs=False) - node2.query(''' - CREATE TABLE simple (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '1', date, id, 8192); - ''') - - finally: - cluster_1.shutdown() - cluster_2.shutdown() - - -# NOTE this test have to be ported to Keeper -def test_secure_connection(): - # We need absolute path in zookeeper volumes. Generate it dynamically. - TEMPLATE = ''' - zoo{zoo_id}: - image: zookeeper:3.6.2 - restart: always - environment: - ZOO_TICK_TIME: 500 - ZOO_MY_ID: {zoo_id} - ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181 - ZOO_SECURE_CLIENT_PORT: 2281 - volumes: - - {helpers_dir}/zookeeper-ssl-entrypoint.sh:/zookeeper-ssl-entrypoint.sh - - {configs_dir}:/clickhouse-config - command: ["zkServer.sh", "start-foreground"] - entrypoint: /zookeeper-ssl-entrypoint.sh - ''' - configs_dir = p.abspath(p.join(p.dirname(__file__), 'configs_secure')) - helpers_dir = p.abspath(p.dirname(helpers.__file__)) - - cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_with_ssl.xml') - - docker_compose = NamedTemporaryFile(mode='w+', delete=False) - - docker_compose.write( - "version: '2.3'\nservices:\n" + - TEMPLATE.format(zoo_id=1, configs_dir=configs_dir, helpers_dir=helpers_dir) + - TEMPLATE.format(zoo_id=2, configs_dir=configs_dir, helpers_dir=helpers_dir) + - TEMPLATE.format(zoo_id=3, configs_dir=configs_dir, helpers_dir=helpers_dir) - ) - docker_compose.close() - - node1 = cluster.add_instance('node1', main_configs=["configs_secure/client.crt", "configs_secure/client.key", - "configs_secure/conf.d/remote_servers.xml", - "configs_secure/conf.d/ssl_conf.xml"], - with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name, - zookeeper_use_tmpfs=False, use_keeper=False) - node2 = cluster.add_instance('node2', main_configs=["configs_secure/client.crt", "configs_secure/client.key", - "configs_secure/conf.d/remote_servers.xml", - "configs_secure/conf.d/ssl_conf.xml"], - with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name, - zookeeper_use_tmpfs=False, use_keeper=False) +def create_zk_roots(zk): + zk.ensure_path('/root_a') + zk.ensure_path('/root_b') + logging.debug(f"Create ZK roots:{zk.get_children('/')}") +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): try: + cluster.add_zookeeper_startup_command(create_zk_roots) cluster.start() - assert node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") == '2\n' - assert node2.query("SELECT count() FROM system.zookeeper WHERE path = '/'") == '2\n' - - - kThreadsNumber = 16 - kIterations = 100 - threads = [] - for _ in range(kThreadsNumber): - threads.append(threading.Thread(target=(lambda: - [node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") for _ in range(kIterations)]))) - - for thread in threads: - thread.start() - - for thread in threads: - thread.join() + yield cluster finally: cluster.shutdown() - unlink(docker_compose.name) + +def test_chroot_with_same_root(started_cluster): + for i, node in enumerate([node1, node2]): + node.query(''' + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); + '''.format(replica=node.name)) + for j in range(2): # Second insert to test deduplication + node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) + + time.sleep(1) + + assert node1.query('select count() from simple').strip() == '2' + assert node2.query('select count() from simple').strip() == '2' + +def test_chroot_with_different_root(started_cluster): + for i, node in [(1, node1), (3, node3)]: + node.query(''' + CREATE TABLE simple_different (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple_different', '{replica}', date, id, 8192); + '''.format(replica=node.name)) + for j in range(2): # Second insert to test deduplication + node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) + + assert node1.query('select count() from simple_different').strip() == '1' + assert node3.query('select count() from simple_different').strip() == '1' diff --git a/tests/integration/test_zookeeper_config/test_password.py b/tests/integration/test_zookeeper_config/test_password.py new file mode 100644 index 00000000000..c0ed4375978 --- /dev/null +++ b/tests/integration/test_zookeeper_config/test_password.py @@ -0,0 +1,34 @@ + + +import time +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__, name="password") + +# TODO ACL not implemented in Keeper. +node1 = cluster.add_instance('node1', with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_with_password.xml"]) + +node2 = cluster.add_instance('node2', with_zookeeper=True, main_configs=["configs/remote_servers.xml"]) + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + +def test_identity(started_cluster): + node1.query(''' + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192); + '''.format(replica=node1.name)) + + with pytest.raises(Exception): + node2.query(''' + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '1', date, id, 8192); + ''') diff --git a/tests/integration/test_zookeeper_config/test_secure.py b/tests/integration/test_zookeeper_config/test_secure.py new file mode 100644 index 00000000000..f57b8d8a80a --- /dev/null +++ b/tests/integration/test_zookeeper_config/test_secure.py @@ -0,0 +1,52 @@ + + +import threading +import os +from tempfile import NamedTemporaryFile + +import pytest +from helpers.cluster import ClickHouseCluster + +TEST_DIR = os.path.dirname(__file__) + +cluster = ClickHouseCluster(__file__, name="secure", + zookeeper_certfile=os.path.join(TEST_DIR, "configs_secure", "client.crt"), + zookeeper_keyfile=os.path.join(TEST_DIR, "configs_secure", "client.key")) + +node1 = cluster.add_instance('node1', main_configs=["configs_secure/client.crt", "configs_secure/client.key", + "configs_secure/conf.d/remote_servers.xml", + "configs_secure/conf.d/ssl_conf.xml", + "configs/zookeeper_config_with_ssl.xml"], + with_zookeeper_secure=True) +node2 = cluster.add_instance('node2', main_configs=["configs_secure/client.crt", "configs_secure/client.key", + "configs_secure/conf.d/remote_servers.xml", + "configs_secure/conf.d/ssl_conf.xml", + "configs/zookeeper_config_with_ssl.xml"], + with_zookeeper_secure=True) + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + +# NOTE this test have to be ported to Keeper +def test_secure_connection(started_cluster): + assert node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") == '2\n' + assert node2.query("SELECT count() FROM system.zookeeper WHERE path = '/'") == '2\n' + + kThreadsNumber = 16 + kIterations = 100 + threads = [] + for _ in range(kThreadsNumber): + threads.append(threading.Thread(target=(lambda: + [node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") for _ in range(kIterations)]))) + + for thread in threads: + thread.start() + + for thread in threads: + thread.join() diff --git a/tests/queries/.gitignore b/tests/queries/.gitignore deleted file mode 100644 index df2c6d60133..00000000000 --- a/tests/queries/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -*.stderr -*.stdout diff --git a/tests/queries/0_stateless/00632_aggregation_window_funnel.sql b/tests/queries/0_stateless/00632_aggregation_window_funnel.sql index bddb856f5b7..aa0dc804238 100644 --- a/tests/queries/0_stateless/00632_aggregation_window_funnel.sql +++ b/tests/queries/0_stateless/00632_aggregation_window_funnel.sql @@ -88,7 +88,4 @@ select 2 = windowFunnel(10000, 'strict_increase')(timestamp, event = 1000, event select 3 = windowFunnel(10000)(timestamp, event = 1004, event = 1004, event = 1004) from funnel_test_strict_increase; select 1 = windowFunnel(10000, 'strict_increase')(timestamp, event = 1004, event = 1004, event = 1004) from funnel_test_strict_increase; -set allow_experimental_funnel_functions = 0; -select 1 = windowFunnel(10000, 'strict_increase')(timestamp, event = 1004, event = 1004, event = 1004) from funnel_test_strict_increase; -- { serverError 63 } - drop table funnel_test_strict_increase; diff --git a/tests/queries/0_stateless/01289_min_execution_speed_not_too_early.sql b/tests/queries/0_stateless/01289_min_execution_speed_not_too_early.sql index e3a18d0c515..222a85094d0 100644 --- a/tests/queries/0_stateless/01289_min_execution_speed_not_too_early.sql +++ b/tests/queries/0_stateless/01289_min_execution_speed_not_too_early.sql @@ -3,9 +3,11 @@ DROP TABLE IF EXISTS ES; create table ES(A String) Engine=MergeTree order by tuple(); insert into ES select toString(number) from numbers(10000000); -SET max_execution_time = 100, max_execution_speed = 1000000; -SET max_threads = 1; -SET max_block_size = 1000000; +SET max_execution_time = 100, + timeout_before_checking_execution_speed = 100, + max_execution_speed = 1000000, + max_threads = 1, + max_block_size = 1000000; -- Exception about execution speed is not thrown from these queries. SELECT * FROM ES LIMIT 1 format Null; diff --git a/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh b/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh index 21fc88d7c2d..37ee5bf7ad7 100755 --- a/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh +++ b/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh @@ -12,7 +12,7 @@ DATA_SIZE=200 SEQ=$(seq 0 $(($NUM_REPLICAS - 1))) for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "DROP TABLE IF EXISTS r$REPLICA"; done -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE r$REPLICA (x UInt64) ENGINE = ReplicatedMergeTree('/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r$REPLICA') ORDER BY x SETTINGS min_bytes_for_wide_part = '10M';"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE r$REPLICA (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r$REPLICA') ORDER BY x SETTINGS min_bytes_for_wide_part = '10M';"; done function thread() { diff --git a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh index 97c200c651f..678a3aa12fe 100755 --- a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh +++ b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01320 ENGINE=Ordinary" # Diff function thread1() { - while true; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_01320.r (x UInt64) ENGINE = ReplicatedMergeTree('/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r') ORDER BY x; DROP TABLE test_01320.r;"; done + while true; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_01320.r (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r') ORDER BY x; DROP TABLE test_01320.r;"; done } function thread2() diff --git a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh index bb935a950ff..16b39db6e84 100755 --- a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh +++ b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh @@ -11,7 +11,7 @@ FREEZE_OUT_STRUCTURE='backup_name String, backup_path String , part_backup_path # setup ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated;" -${CLICKHOUSE_CLIENT} --query "CREATE TABLE table_for_freeze_replicated (key UInt64, value String) ENGINE = ReplicatedMergeTree('/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table_for_freeze_replicated', '1') ORDER BY key PARTITION BY key % 10;" +${CLICKHOUSE_CLIENT} --query "CREATE TABLE table_for_freeze_replicated (key UInt64, value String) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table_for_freeze_replicated', '1') ORDER BY key PARTITION BY key % 10;" ${CLICKHOUSE_CLIENT} --query "INSERT INTO table_for_freeze_replicated SELECT number, toString(number) from numbers(10);" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE WITH NAME 'test_01417' FORMAT TSVWithNames SETTINGS alter_partition_verbose_result = 1;" \ diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index 0e1917a95f2..9e30263c6b3 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -1,47 +1,46 @@ -DROP TABLE IF EXISTS test; +DROP TABLE IF EXISTS 01504_test; -CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB; -- { serverError 36 } -CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key2); -- { serverError 47 } -CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key, value); -- { serverError 36 } -CREATE TABLE test (key Tuple(String, UInt32), value UInt64) Engine=EmbeddedRocksDB PRIMARY KEY(key); +CREATE TABLE 01504_test (key String, value UInt32) Engine=EmbeddedRocksDB; -- { serverError 36 } +CREATE TABLE 01504_test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key2); -- { serverError 47 } +CREATE TABLE 01504_test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key, value); -- { serverError 36 } +CREATE TABLE 01504_test (key Tuple(String, UInt32), value UInt64) Engine=EmbeddedRocksDB PRIMARY KEY(key); -DROP TABLE IF EXISTS test; -CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key); +DROP TABLE IF EXISTS 01504_test; +CREATE TABLE 01504_test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key); -INSERT INTO test SELECT '1_1', number FROM numbers(10000); -SELECT COUNT(1) == 1 FROM test; +INSERT INTO 01504_test SELECT '1_1', number FROM numbers(10000); +SELECT COUNT(1) == 1 FROM 01504_test; -INSERT INTO test SELECT concat(toString(number), '_1'), number FROM numbers(10000); -SELECT COUNT(1) == 10000 FROM test; -SELECT uniqExact(key) == 32 FROM (SELECT * FROM test LIMIT 32 SETTINGS max_block_size = 1); -SELECT SUM(value) == 1 + 99 + 900 FROM test WHERE key IN ('1_1', '99_1', '900_1'); +INSERT INTO 01504_test SELECT concat(toString(number), '_1'), number FROM numbers(10000); +SELECT COUNT(1) == 10000 FROM 01504_test; +SELECT uniqExact(key) == 32 FROM (SELECT * FROM 01504_test LIMIT 32 SETTINGS max_block_size = 1); +SELECT SUM(value) == 1 + 99 + 900 FROM 01504_test WHERE key IN ('1_1', '99_1', '900_1'); -DROP TABLE IF EXISTS test; -DROP TABLE IF EXISTS test_memory; +DROP TABLE IF EXISTS 01504_test; +DROP TABLE IF EXISTS 01504_test_memory; -CREATE TABLE test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksDB PRIMARY KEY(k); -CREATE TABLE test_memory AS test Engine = Memory; +CREATE TABLE 01504_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksDB PRIMARY KEY(k); +CREATE TABLE 01504_test_memory AS 01504_test Engine = Memory; -INSERT INTO test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; +INSERT INTO 01504_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; -INSERT INTO test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; +INSERT INTO 01504_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; -SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test_memory) B USING a ORDER BY a; +SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 01504_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 01504_test_memory) B USING a ORDER BY a; CREATE TEMPORARY TABLE keys AS SELECT * FROM numbers(1000); SET max_rows_to_read = 2; -SELECT dummy == (1,1.2) FROM test WHERE k IN (1, 3) OR k IN (1) OR k IN (3, 1) OR k IN [1] OR k IN [1, 3] ; -SELECT k == 4 FROM test WHERE k = 4 OR k IN [4] OR k in (4, 10000001, 10000002) AND value > 0; -SELECT k == 4 FROM test WHERE k IN (SELECT toUInt32(number) FROM keys WHERE number = 4); -SELECT k, value FROM test WHERE k = 0 OR value > 0; -- { serverError 158 } -SELECT k, value FROM test WHERE k = 0 AND k IN (1, 3) OR k > 8; -- { serverError 158 } +SELECT dummy == (1,1.2) FROM 01504_test WHERE k IN (1, 3) OR k IN (1) OR k IN (3, 1) OR k IN [1] OR k IN [1, 3] ; +SELECT k == 4 FROM 01504_test WHERE k = 4 OR k IN [4] OR k in (4, 10000001, 10000002) AND value > 0; +SELECT k == 4 FROM 01504_test WHERE k IN (SELECT toUInt32(number) FROM keys WHERE number = 4); +SELECT k, value FROM 01504_test WHERE k = 0 OR value > 0; -- { serverError 158 } +SELECT k, value FROM 01504_test WHERE k = 0 AND k IN (1, 3) OR k > 8; -- { serverError 158 } -TRUNCATE TABLE test; -SELECT 0 == COUNT(1) FROM test; - -DROP TABLE IF EXISTS test; -DROP TABLE IF EXISTS test_memory; +TRUNCATE TABLE 01504_test; +SELECT 0 == COUNT(1) FROM 01504_test; +DROP TABLE IF EXISTS 01504_test; +DROP TABLE IF EXISTS 01504_test_memory; diff --git a/tests/queries/0_stateless/01594_too_low_memory_limits.config.xml b/tests/queries/0_stateless/01594_too_low_memory_limits.config.xml new file mode 100644 index 00000000000..2d0a480a375 --- /dev/null +++ b/tests/queries/0_stateless/01594_too_low_memory_limits.config.xml @@ -0,0 +1,35 @@ + + + + trace + true + + + 9000 + + ./ + + 0 + + + + + + + ::/0 + + + default + default + 1 + + + + + + + + + + + diff --git a/tests/queries/0_stateless/01594_too_low_memory_limits.sh b/tests/queries/0_stateless/01594_too_low_memory_limits.sh index 0d4dbbcabe1..6a33bd81dd1 100755 --- a/tests/queries/0_stateless/01594_too_low_memory_limits.sh +++ b/tests/queries/0_stateless/01594_too_low_memory_limits.sh @@ -1,16 +1,103 @@ #!/usr/bin/env bash +# +# Regression for INSERT SELECT, that abnormally terminates the server +# in case of too small memory limits. +# +# NOTE: After #24483 had been merged the only place where the allocation may +# fail is the insert into PODArray in DB::OwnSplitChannel::log, but after +# #24069 those errors will be ignored, so to check new behaviour separate +# server is required. +# + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +server_opts=( + "--config-file=$CURDIR/$(basename "${BASH_SOURCE[0]}" .sh).config.xml" + "--" + # to avoid multiple listen sockets (complexity for port discovering) + "--listen_host=127.1" + # we will discover the real port later. + "--tcp_port=0" + "--shutdown_wait_unfinished=0" +) +CLICKHOUSE_WATCHDOG_ENABLE=0 $CLICKHOUSE_SERVER_BINARY "${server_opts[@]}" >clickhouse-server.log 2>clickhouse-server.stderr & +server_pid=$! + +trap cleanup EXIT +function cleanup() +{ + kill -9 $server_pid + + echo "Test failed. Server log:" + cat clickhouse-server.log + cat clickhouse-server.stderr + rm -f clickhouse-server.log + rm -f clickhouse-server.stderr + + exit 1 +} + +server_port= +i=0 retries=300 +# wait until server will start to listen (max 30 seconds) +while [[ -z $server_port ]] && [[ $i -lt $retries ]]; do + server_port=$(lsof -n -a -P -i tcp -s tcp:LISTEN -p $server_pid 2>/dev/null | awk -F'[ :]' '/LISTEN/ { print $(NF-1) }') + ((++i)) + sleep 0.1 + if ! kill -0 $server_pid >& /dev/null; then + echo "No server (pid $server_pid)" + break + fi +done +if [[ -z $server_port ]]; then + echo "Cannot wait for LISTEN socket" >&2 + exit 1 +fi + +# wait for the server to start accepting tcp connections (max 30 seconds) +i=0 retries=300 +while ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'select 1' 2>/dev/null && [[ $i -lt $retries ]]; do + sleep 0.1 + if ! kill -0 $server_pid >& /dev/null; then + echo "No server (pid $server_pid)" + break + fi +done +if ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'select 1'; then + echo "Cannot wait until server will start accepting connections on " >&2 + exit 1 +fi + # it is not mandatory to use existing table since it fails earlier, hence just a placeholder. # this is format of INSERT SELECT, that pass these settings exactly for INSERT query not the SELECT -${CLICKHOUSE_CLIENT} --format Null -q 'insert into placeholder_table_name select * from numbers_mt(65535) format Null settings max_memory_usage=1, max_untracked_memory=1' >& /dev/null -exit_code=$? +if $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null --send_logs_level=warning --max_memory_usage=1 --max_untracked_memory=1 -q 'insert into placeholder_table_name select * from numbers_mt(65535)' >& /dev/null; then + echo "INSERT SELECT should fail" >&2 + exit 1 +fi -# expecting ATTEMPT_TO_READ_AFTER_EOF, 32 -test $exit_code -eq 32 || exit 1 +# no sleep, since flushing to stderr should not be buffered. +if ! grep -E -q 'Cannot add message to the log: Code: 60.*placeholder_table_name' clickhouse-server.stderr; then + echo "Adding message to the log should fail" >&2 + exit 1 +fi # check that server is still alive -${CLICKHOUSE_CLIENT} --format Null -q 'SELECT 1' +$CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'SELECT 1' + +# send TERM and save the error code to ensure that it is 0 (EXIT_SUCCESS) +kill $server_pid +wait $server_pid +return_code=$? + +trap '' EXIT +if [ $return_code != 0 ]; then + cat clickhouse-server.log + cat clickhouse-server.stderr +fi +rm -f clickhouse-server.log +rm -f clickhouse-server.stderr + +exit $return_code diff --git a/tests/queries/0_stateless/01656_sequence_next_node_long.reference b/tests/queries/0_stateless/01656_sequence_next_node_long.reference new file mode 100644 index 00000000000..739f19818a6 --- /dev/null +++ b/tests/queries/0_stateless/01656_sequence_next_node_long.reference @@ -0,0 +1,491 @@ +(forward, head, A) 1 B +(forward, head, A) 2 \N +(forward, head, A) 3 B +(forward, head, A) 4 A +(forward, head, A) 5 B +(forward, head, A) 6 B +(forward, head, B) 1 \N +(forward, head, B) 2 B +(forward, head, B) 3 \N +(forward, head, B) 4 \N +(forward, head, B) 5 \N +(forward, head, B) 6 \N +(forward, head, C) 1 \N +(forward, head, C) 2 \N +(forward, head, C) 3 \N +(forward, head, C) 4 \N +(forward, head, C) 5 \N +(forward, head, C) 6 \N +(forward, head, D) 1 \N +(forward, head, D) 2 \N +(forward, head, D) 3 \N +(forward, head, D) 4 \N +(forward, head, D) 5 \N +(forward, head, D) 6 \N +(forward, head, E) 1 \N +(forward, head, E) 2 \N +(forward, head, E) 3 \N +(forward, head, E) 4 \N +(forward, head, E) 5 \N +(forward, head, E) 6 \N +(backward, tail, A) 1 \N +(backward, tail, A) 2 \N +(backward, tail, A) 3 \N +(backward, tail, A) 4 \N +(backward, tail, A) 5 \N +(backward, tail, A) 6 \N +(backward, tail, B) 1 \N +(backward, tail, B) 2 \N +(backward, tail, B) 3 A +(backward, tail, B) 4 \N +(backward, tail, B) 5 \N +(backward, tail, B) 6 \N +(backward, tail, C) 1 \N +(backward, tail, C) 2 D +(backward, tail, C) 3 \N +(backward, tail, C) 4 B +(backward, tail, C) 5 A +(backward, tail, C) 6 B +(backward, tail, D) 1 C +(backward, tail, D) 2 \N +(backward, tail, D) 3 \N +(backward, tail, D) 4 \N +(backward, tail, D) 5 \N +(backward, tail, D) 6 \N +(backward, tail, E) 1 \N +(backward, tail, E) 2 \N +(backward, tail, E) 3 \N +(backward, tail, E) 4 \N +(backward, tail, E) 5 \N +(backward, tail, E) 6 \N +(forward, head, A->B) 1 C +(forward, head, A->B) 2 \N +(forward, head, A->B) 3 \N +(forward, head, A->B) 4 \N +(forward, head, A->B) 5 A +(forward, head, A->B) 6 A +(forward, head, A->C) 1 \N +(forward, head, A->C) 2 \N +(forward, head, A->C) 3 \N +(forward, head, A->C) 4 \N +(forward, head, A->C) 5 \N +(forward, head, A->C) 6 \N +(forward, head, B->A) 1 \N +(forward, head, B->A) 2 \N +(forward, head, B->A) 3 \N +(forward, head, B->A) 4 \N +(forward, head, B->A) 5 \N +(forward, head, B->A) 6 \N +(backward, tail, A->B) 1 \N +(backward, tail, A->B) 2 \N +(backward, tail, A->B) 3 \N +(backward, tail, A->B) 4 \N +(backward, tail, A->B) 5 \N +(backward, tail, A->B) 6 \N +(backward, tail, A->C) 1 \N +(backward, tail, A->C) 2 \N +(backward, tail, A->C) 3 \N +(backward, tail, A->C) 4 \N +(backward, tail, A->C) 5 \N +(backward, tail, A->C) 6 \N +(backward, tail, B->A) 1 \N +(backward, tail, B->A) 2 \N +(backward, tail, B->A) 3 \N +(backward, tail, B->A) 4 \N +(backward, tail, B->A) 5 \N +(backward, tail, B->A) 6 \N +(forward, head, A->A->B) 1 \N +(forward, head, A->A->B) 2 \N +(forward, head, A->A->B) 3 \N +(forward, head, A->A->B) 4 \N +(forward, head, A->A->B) 5 \N +(forward, head, A->A->B) 6 \N +(forward, head, B->A->A) 1 \N +(forward, head, B->A->A) 2 \N +(forward, head, B->A->A) 3 \N +(forward, head, B->A->A) 4 \N +(forward, head, B->A->A) 5 \N +(forward, head, B->A->A) 6 \N +(backward, tail, A->A->B) 1 \N +(backward, tail, A->A->B) 2 \N +(backward, tail, A->A->B) 3 \N +(backward, tail, A->A->B) 4 \N +(backward, tail, A->A->B) 5 \N +(backward, tail, A->A->B) 6 \N +(backward, tail, B->A->A) 1 \N +(backward, tail, B->A->A) 2 \N +(backward, tail, B->A->A) 3 \N +(backward, tail, B->A->A) 4 \N +(backward, tail, B->A->A) 5 \N +(backward, tail, B->A->A) 6 \N +(forward, head, A) id >= 10 10 B +(forward, head, A) id >= 10 10 C +(forward, head, A) id >= 10 10 \N +(forward, head, A) id >= 10 10 \N +(backward, tail, A) id >= 10 10 B +(backward, tail, A) id >= 10 10 A +(0, A) id = 11 1 +(0, C) id = 11 0 +(0, B->C) id = 11 0 +(0, A->B->C) id = 11 1 +(0, A) id = 11 1 +(0, C) id = 11 0 +(0, C->B) id = 11 0 +(0, C->B->A) id = 11 1 +(forward, head) id < 10 1 A +(forward, head) id < 10 2 B +(forward, head) id < 10 3 A +(forward, head) id < 10 4 A +(forward, head) id < 10 5 A +(forward, head) id < 10 6 A +(backward, tail) id < 10 1 D +(backward, tail) id < 10 2 C +(backward, tail) id < 10 3 B +(backward, tail) id < 10 4 C +(backward, tail) id < 10 5 C +(backward, tail) id < 10 6 C +(forward, first_match, A) 1 B +(forward, first_match, A) 2 \N +(forward, first_match, A) 3 B +(forward, first_match, A) 4 A +(forward, first_match, A) 5 B +(forward, first_match, A) 6 B +(forward, first_match, A) 10 B +(forward, first_match, A) 11 B +(forward, first_match, A->B) 1 C +(forward, first_match, A->B) 2 \N +(forward, first_match, A->B) 3 \N +(forward, first_match, A->B) 4 \N +(forward, first_match, A->B) 5 A +(forward, first_match, A->B) 6 A +(forward, first_match, A->B) 10 C +(forward, first_match, A->B) 11 C +(forward, first_match, A->B->C) 1 D +(forward, first_match, A->B->C) 2 \N +(forward, first_match, A->B->C) 3 \N +(forward, first_match, A->B->C) 4 \N +(forward, first_match, A->B->C) 5 \N +(forward, first_match, A->B->C) 6 \N +(forward, first_match, A->B->C) 10 D +(forward, first_match, A->B->C) 11 D +(forward, first_match, B) 1 C +(forward, first_match, B) 2 B +(forward, first_match, B) 3 \N +(forward, first_match, B) 4 C +(forward, first_match, B) 5 A +(forward, first_match, B) 6 A +(forward, first_match, B) 10 C +(forward, first_match, B) 11 C +(forward, first_match, B->B) 1 \N +(forward, first_match, B->B) 2 D +(forward, first_match, B->B) 3 \N +(forward, first_match, B->B) 4 \N +(forward, first_match, B->B) 5 \N +(forward, first_match, B->B) 6 \N +(forward, first_match, B->B) 10 \N +(forward, first_match, B->B) 11 \N +(forward, first_match, B->A) 1 \N +(forward, first_match, B->A) 2 \N +(forward, first_match, B->A) 3 \N +(forward, first_match, B->A) 4 \N +(forward, first_match, B->A) 5 C +(forward, first_match, B->A) 6 B +(forward, first_match, B->A) 10 \N +(forward, first_match, B->A) 11 \N +(backward, first_match, A) 1 \N +(backward, first_match, A) 2 \N +(backward, first_match, A) 3 \N +(backward, first_match, A) 4 \N +(backward, first_match, A) 5 \N +(backward, first_match, A) 6 \N +(backward, first_match, A) 10 \N +(backward, first_match, A) 11 \N +(backward, first_match, B) 1 A +(backward, first_match, B) 2 \N +(backward, first_match, B) 3 A +(backward, first_match, B) 4 A +(backward, first_match, B) 5 A +(backward, first_match, B) 6 A +(backward, first_match, B) 10 A +(backward, first_match, B) 11 A +(backward, first_match, B->A) 1 \N +(backward, first_match, B->A) 2 \N +(backward, first_match, B->A) 3 \N +(backward, first_match, B->A) 4 A +(backward, first_match, B->A) 5 \N +(backward, first_match, B->A) 6 \N +(backward, first_match, B->A) 10 \N +(backward, first_match, B->A) 11 \N +(backward, first_match, B->B) 1 \N +(backward, first_match, B->B) 2 \N +(backward, first_match, B->B) 3 \N +(backward, first_match, B->B) 4 \N +(backward, first_match, B->B) 5 \N +(backward, first_match, B->B) 6 \N +(backward, first_match, B->B) 10 \N +(backward, first_match, B->B) 11 \N +(forward, head, A) 1 B +(forward, head, A) 2 \N +(forward, head, A) 3 B +(forward, head, A) 4 A +(forward, head, A) 5 B +(forward, head, A) 6 B +(forward, head, B) 1 \N +(forward, head, B) 2 B +(forward, head, B) 3 \N +(forward, head, B) 4 \N +(forward, head, B) 5 \N +(forward, head, B) 6 \N +(forward, head, C) 1 \N +(forward, head, C) 2 \N +(forward, head, C) 3 \N +(forward, head, C) 4 \N +(forward, head, C) 5 \N +(forward, head, C) 6 \N +(forward, head, D) 1 \N +(forward, head, D) 2 \N +(forward, head, D) 3 \N +(forward, head, D) 4 \N +(forward, head, D) 5 \N +(forward, head, D) 6 \N +(forward, head, E) 1 \N +(forward, head, E) 2 \N +(forward, head, E) 3 \N +(forward, head, E) 4 \N +(forward, head, E) 5 \N +(forward, head, E) 6 \N +(backward, tail, A) 1 \N +(backward, tail, A) 2 \N +(backward, tail, A) 3 \N +(backward, tail, A) 4 \N +(backward, tail, A) 5 \N +(backward, tail, A) 6 \N +(backward, tail, B) 1 \N +(backward, tail, B) 2 \N +(backward, tail, B) 3 A +(backward, tail, B) 4 \N +(backward, tail, B) 5 \N +(backward, tail, B) 6 \N +(backward, tail, C) 1 \N +(backward, tail, C) 2 D +(backward, tail, C) 3 \N +(backward, tail, C) 4 B +(backward, tail, C) 5 A +(backward, tail, C) 6 B +(backward, tail, D) 1 C +(backward, tail, D) 2 \N +(backward, tail, D) 3 \N +(backward, tail, D) 4 \N +(backward, tail, D) 5 \N +(backward, tail, D) 6 \N +(backward, tail, E) 1 \N +(backward, tail, E) 2 \N +(backward, tail, E) 3 \N +(backward, tail, E) 4 \N +(backward, tail, E) 5 \N +(backward, tail, E) 6 \N +(forward, head, A->B) 1 C +(forward, head, A->B) 2 \N +(forward, head, A->B) 3 \N +(forward, head, A->B) 4 \N +(forward, head, A->B) 5 A +(forward, head, A->B) 6 A +(forward, head, A->C) 1 \N +(forward, head, A->C) 2 \N +(forward, head, A->C) 3 \N +(forward, head, A->C) 4 \N +(forward, head, A->C) 5 \N +(forward, head, A->C) 6 \N +(forward, head, B->A) 1 \N +(forward, head, B->A) 2 \N +(forward, head, B->A) 3 \N +(forward, head, B->A) 4 \N +(forward, head, B->A) 5 \N +(forward, head, B->A) 6 \N +(backward, tail, A->B) 1 \N +(backward, tail, A->B) 2 \N +(backward, tail, A->B) 3 \N +(backward, tail, A->B) 4 \N +(backward, tail, A->B) 5 \N +(backward, tail, A->B) 6 \N +(backward, tail, A->C) 1 \N +(backward, tail, A->C) 2 \N +(backward, tail, A->C) 3 \N +(backward, tail, A->C) 4 \N +(backward, tail, A->C) 5 \N +(backward, tail, A->C) 6 \N +(backward, tail, B->A) 1 \N +(backward, tail, B->A) 2 \N +(backward, tail, B->A) 3 \N +(backward, tail, B->A) 4 \N +(backward, tail, B->A) 5 \N +(backward, tail, B->A) 6 \N +(forward, head, A->A->B) 1 \N +(forward, head, A->A->B) 2 \N +(forward, head, A->A->B) 3 \N +(forward, head, A->A->B) 4 \N +(forward, head, A->A->B) 5 \N +(forward, head, A->A->B) 6 \N +(forward, head, B->A->A) 1 \N +(forward, head, B->A->A) 2 \N +(forward, head, B->A->A) 3 \N +(forward, head, B->A->A) 4 \N +(forward, head, B->A->A) 5 \N +(forward, head, B->A->A) 6 \N +(backward, tail, A->A->B) 1 \N +(backward, tail, A->A->B) 2 \N +(backward, tail, A->A->B) 3 \N +(backward, tail, A->A->B) 4 \N +(backward, tail, A->A->B) 5 \N +(backward, tail, A->A->B) 6 \N +(backward, tail, B->A->A) 1 \N +(backward, tail, B->A->A) 2 \N +(backward, tail, B->A->A) 3 \N +(backward, tail, B->A->A) 4 \N +(backward, tail, B->A->A) 5 \N +(backward, tail, B->A->A) 6 \N +(forward, head, A) id >= 10 10 B +(forward, head, A) id >= 10 10 C +(forward, head, A) id >= 10 10 \N +(forward, head, A) id >= 10 10 \N +(backward, tail, A) id >= 10 10 B +(backward, tail, A) id >= 10 10 A +(0, A) id = 11 1 +(0, C) id = 11 0 +(0, B->C) id = 11 0 +(0, A->B->C) id = 11 1 +(0, A) id = 11 1 +(0, C) id = 11 0 +(0, C->B) id = 11 0 +(0, C->B->A) id = 11 1 +(forward, head) id < 10 1 A +(forward, head) id < 10 2 B +(forward, head) id < 10 3 A +(forward, head) id < 10 4 A +(forward, head) id < 10 5 A +(forward, head) id < 10 6 A +(backward, tail) id < 10 1 D +(backward, tail) id < 10 2 C +(backward, tail) id < 10 3 B +(backward, tail) id < 10 4 C +(backward, tail) id < 10 5 C +(backward, tail) id < 10 6 C +(forward, first_match, A) 1 B +(forward, first_match, A) 2 \N +(forward, first_match, A) 3 B +(forward, first_match, A) 4 A +(forward, first_match, A) 5 B +(forward, first_match, A) 6 B +(forward, first_match, A) 10 B +(forward, first_match, A) 11 B +(forward, first_match, A->B) 1 C +(forward, first_match, A->B) 2 \N +(forward, first_match, A->B) 3 \N +(forward, first_match, A->B) 4 \N +(forward, first_match, A->B) 5 A +(forward, first_match, A->B) 6 A +(forward, first_match, A->B) 10 C +(forward, first_match, A->B) 11 C +(forward, first_match, A->B->C) 1 D +(forward, first_match, A->B->C) 2 \N +(forward, first_match, A->B->C) 3 \N +(forward, first_match, A->B->C) 4 \N +(forward, first_match, A->B->C) 5 \N +(forward, first_match, A->B->C) 6 \N +(forward, first_match, A->B->C) 10 D +(forward, first_match, A->B->C) 11 D +(forward, first_match, B) 1 C +(forward, first_match, B) 2 B +(forward, first_match, B) 3 \N +(forward, first_match, B) 4 C +(forward, first_match, B) 5 A +(forward, first_match, B) 6 A +(forward, first_match, B) 10 C +(forward, first_match, B) 11 C +(forward, first_match, B->B) 1 \N +(forward, first_match, B->B) 2 D +(forward, first_match, B->B) 3 \N +(forward, first_match, B->B) 4 \N +(forward, first_match, B->B) 5 \N +(forward, first_match, B->B) 6 \N +(forward, first_match, B->B) 10 \N +(forward, first_match, B->B) 11 \N +(forward, first_match, B->A) 1 \N +(forward, first_match, B->A) 2 \N +(forward, first_match, B->A) 3 \N +(forward, first_match, B->A) 4 \N +(forward, first_match, B->A) 5 C +(forward, first_match, B->A) 6 B +(forward, first_match, B->A) 10 \N +(forward, first_match, B->A) 11 \N +(backward, first_match, A) 1 \N +(backward, first_match, A) 2 \N +(backward, first_match, A) 3 \N +(backward, first_match, A) 4 \N +(backward, first_match, A) 5 \N +(backward, first_match, A) 6 \N +(backward, first_match, A) 10 \N +(backward, first_match, A) 11 \N +(backward, first_match, B) 1 A +(backward, first_match, B) 2 \N +(backward, first_match, B) 3 A +(backward, first_match, B) 4 A +(backward, first_match, B) 5 A +(backward, first_match, B) 6 A +(backward, first_match, B) 10 A +(backward, first_match, B) 11 A +(backward, first_match, B->A) 1 \N +(backward, first_match, B->A) 2 \N +(backward, first_match, B->A) 3 \N +(backward, first_match, B->A) 4 A +(backward, first_match, B->A) 5 \N +(backward, first_match, B->A) 6 \N +(backward, first_match, B->A) 10 \N +(backward, first_match, B->A) 11 \N +(backward, first_match, B->B) 1 \N +(backward, first_match, B->B) 2 \N +(backward, first_match, B->B) 3 \N +(backward, first_match, B->B) 4 \N +(backward, first_match, B->B) 5 \N +(backward, first_match, B->B) 6 \N +(backward, first_match, B->B) 10 \N +(backward, first_match, B->B) 11 \N +(max_args) 1 \N +(max_args) 2 \N +(max_args) 3 \N +(max_args) 4 \N +(max_args) 5 \N +(max_args) 6 \N +(max_args) 10 \N +(max_args) 11 \N +(forward, head, A) id = 12 A +(forward, head, 1) 1 A +(forward, head, 1) 2 \N +(forward, head, 1) 3 \N +(forward, head, 1, A) 1 B +(forward, head, 1, A) 2 \N +(forward, head, 1, A) 3 \N +(forward, head, 1, A->B) 1 C +(forward, head, 1, A->B) 2 \N +(forward, head, 1, A->B) 3 \N +(backward, tail, 1) 1 \N +(backward, tail, 1) 2 A +(backward, tail, 1) 3 \N +(backward, tail, 1, A) 1 \N +(backward, tail, 1, A) 2 B +(backward, tail, 1, A) 3 \N +(backward, tail, 1, A->B) 1 \N +(backward, tail, 1, A->B) 2 C +(backward, tail, 1, A->B) 3 \N +(forward, first_match, 1, B) 1 C +(forward, first_match, 1, B) 2 A +(forward, first_match, 1, B) 3 D +(forward, first_match, 1, B->C) 1 D +(forward, first_match, 1, B->C) 2 \N +(forward, first_match, 1, B->C) 3 \N +(backward, first_match, 1, B) 1 A +(backward, first_match, 1, B) 2 C +(backward, first_match, 1, B) 3 B +(backward, first_match, 1, B->C) 1 \N +(backward, first_match, 1, B->C) 2 \N +(backward, first_match, 1, B->C) 3 \N diff --git a/tests/queries/0_stateless/01656_sequence_next_node_long.sql b/tests/queries/0_stateless/01656_sequence_next_node_long.sql new file mode 100644 index 00000000000..d0d01e989b8 --- /dev/null +++ b/tests/queries/0_stateless/01656_sequence_next_node_long.sql @@ -0,0 +1,233 @@ +SET allow_experimental_funnel_functions = 1; + +DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; + +CREATE TABLE IF NOT EXISTS test_sequenceNextNode_Nullable (dt DateTime, id int, action Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; + +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',1,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',1,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',1,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',1,'D'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',2,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',2,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',2,'D'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',2,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',3,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',3,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',4,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',4,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',4,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',4,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',4,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',5,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',5,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',5,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',5,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',6,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',6,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',6,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',6,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',6,'C'); + +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; + +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',10,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,NULL); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',10,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',10,'D'); + +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; + +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'D'); + +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); + +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action, 1) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; + +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; + +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; + +DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; + +-- The same testcases for a non-null type. + +DROP TABLE IF EXISTS test_sequenceNextNode; + +CREATE TABLE IF NOT EXISTS test_sequenceNextNode (dt DateTime, id int, action String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; + +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',1,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',1,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',1,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',1,'D'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',2,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',2,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',2,'D'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',2,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',3,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',3,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',4,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',4,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',4,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',4,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',4,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',5,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',5,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',5,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',5,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',6,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',6,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C'); + +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; + +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',10,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D'); + +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; + +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D'); + +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); + +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action, 1) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; + +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; + +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; + +SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id; + +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); + +SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12; + +DROP TABLE IF EXISTS test_sequenceNextNode; + +DROP TABLE IF EXISTS test_base_condition; + +CREATE TABLE IF NOT EXISTS test_base_condition (dt DateTime, id int, action String, referrer String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; + +INSERT INTO test_base_condition values ('1970-01-01 09:00:01',1,'A','1'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:02',1,'B','2'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:03',1,'C','3'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:04',1,'D','4'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:01',2,'D','4'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:02',2,'C','3'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:03',2,'B','2'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:04',2,'A','1'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:01',3,'B','10'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:02',3,'B','2'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:03',3,'D','3'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:04',3,'C','4'); + +SELECT '(forward, head, 1)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, head, 1, A)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, head, 1, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1', action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; + +SELECT '(backward, tail, 1)', id, sequenceNextNode('backward', 'tail')(dt, action, referrer = '1') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, tail, 1, A)', id, sequenceNextNode('backward', 'tail')(dt, action, referrer = '1', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, tail, 1, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, referrer = '1', action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; + +SELECT '(forward, first_match, 1, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, referrer = '2', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, first_match, 1, B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, referrer = '2', action = 'B', action = 'C') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; + +SELECT '(backward, first_match, 1, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, referrer = '2', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, first_match, 1, B->C)', id, sequenceNextNode('backward', 'first_match')(dt, action, referrer = '2', action = 'B', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; + +SET allow_experimental_funnel_functions = 0; +SELECT '(backward, first_match, 1, B->C)', id, sequenceNextNode('backward', 'first_match')(dt, action, referrer = '2', action = 'B', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -- { serverError 63 } + +DROP TABLE IF EXISTS test_base_condition; diff --git a/tests/queries/0_stateless/01686_rocksdb.sql b/tests/queries/0_stateless/01686_rocksdb.sql index c9b133acff3..9a8662453c1 100644 --- a/tests/queries/0_stateless/01686_rocksdb.sql +++ b/tests/queries/0_stateless/01686_rocksdb.sql @@ -1,27 +1,26 @@ -DROP TABLE IF EXISTS test; +DROP TABLE IF EXISTS 01686_test; -CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); +CREATE TABLE 01686_test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); -INSERT INTO test SELECT number, format('Hello, world ({})', toString(number)) FROM numbers(10000); +INSERT INTO 01686_test SELECT number, format('Hello, world ({})', toString(number)) FROM numbers(10000); -SELECT * FROM test WHERE key = 123; +SELECT * FROM 01686_test WHERE key = 123; SELECT '--'; -SELECT * FROM test WHERE key = -123; +SELECT * FROM 01686_test WHERE key = -123; SELECT '--'; -SELECT * FROM test WHERE key = 123 OR key = 4567 ORDER BY key; +SELECT * FROM 01686_test WHERE key = 123 OR key = 4567 ORDER BY key; SELECT '--'; -SELECT * FROM test WHERE key = NULL; +SELECT * FROM 01686_test WHERE key = NULL; SELECT '--'; -SELECT * FROM test WHERE key = NULL OR key = 0; +SELECT * FROM 01686_test WHERE key = NULL OR key = 0; SELECT '--'; -SELECT * FROM test WHERE key IN (123, 456, -123) ORDER BY key; +SELECT * FROM 01686_test WHERE key IN (123, 456, -123) ORDER BY key; SELECT '--'; -SELECT * FROM test WHERE key = 'Hello'; -- { serverError 53 } +SELECT * FROM 01686_test WHERE key = 'Hello'; -- { serverError 53 } -DETACH TABLE test NO DELAY; -ATTACH TABLE test; +DETACH TABLE 01686_test NO DELAY; +ATTACH TABLE 01686_test; -SELECT * FROM test WHERE key IN (99, 999, 9999, -123) ORDER BY key; - -DROP TABLE IF EXISTS test; +SELECT * FROM 01686_test WHERE key IN (99, 999, 9999, -123) ORDER BY key; +DROP TABLE IF EXISTS 01686_test; diff --git a/tests/queries/0_stateless/01710_aggregate_projections.sh b/tests/queries/0_stateless/01710_aggregate_projections.sh index 1b09a3f1383..a8b3e6bf99d 100755 --- a/tests/queries/0_stateless/01710_aggregate_projections.sh +++ b/tests/queries/0_stateless/01710_aggregate_projections.sh @@ -25,3 +25,5 @@ $CLICKHOUSE_CLIENT -q "select x + y + 1, argMax(y, x), sum(x - y) as s from test $CLICKHOUSE_CLIENT -q "select x + y, sum(x - y) as s from test_agg_proj prewhere (x + y) % 2 = 1 group by x + y order by s desc limit 5 settings allow_experimental_projection_optimization=1" $CLICKHOUSE_CLIENT -q "select x + y, sum(x - y) as s from test_agg_proj prewhere (x + y) % 2 = 1 group by x + y order by s desc limit 5 settings allow_experimental_projection_optimization=1 format JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT -q "drop table test_agg_proj" diff --git a/tests/queries/0_stateless/01710_normal_projections.sh b/tests/queries/0_stateless/01710_normal_projections.sh index 89795d12bc9..6b16021de17 100755 --- a/tests/queries/0_stateless/01710_normal_projections.sh +++ b/tests/queries/0_stateless/01710_normal_projections.sh @@ -77,3 +77,4 @@ echo "optimize_move_to_prewhere = 1, allow_experimental_projection_optimization $CLICKHOUSE_CLIENT -q "SELECT * FROM test_sort_proj WHERE y > 4294967286 order by x FORMAT JSON SETTINGS optimize_move_to_prewhere = 1, allow_experimental_projection_optimization = 1" | grep rows_read +$CLICKHOUSE_CLIENT -q "DROP TABLE test_sort_proj" diff --git a/tests/queries/0_stateless/01710_projections_group_by_no_key.reference b/tests/queries/0_stateless/01710_projections_group_by_no_key.reference new file mode 100644 index 00000000000..caacaa85780 --- /dev/null +++ b/tests/queries/0_stateless/01710_projections_group_by_no_key.reference @@ -0,0 +1,2 @@ +499500 +499500 diff --git a/tests/queries/0_stateless/01710_projections_group_by_no_key.sql b/tests/queries/0_stateless/01710_projections_group_by_no_key.sql new file mode 100644 index 00000000000..eefc03afb7a --- /dev/null +++ b/tests/queries/0_stateless/01710_projections_group_by_no_key.sql @@ -0,0 +1,8 @@ +drop table if exists projection_without_key; + +create table projection_without_key (key UInt32, PROJECTION x (SELECT sum(key) group by key % 3)) engine MergeTree order by key; +insert into projection_without_key select number from numbers(1000); +select sum(key) from projection_without_key settings allow_experimental_projection_optimization = 1; +select sum(key) from projection_without_key settings allow_experimental_projection_optimization = 0; + +drop table projection_without_key; diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference index a1bfcf043da..3bc986f4d2b 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference @@ -12,6 +12,8 @@ WITH CAST(\'default\', \'String\') AS id_2 SELECT one.dummy, ignore(id_2) FROM s optimize_skip_unused_shards_rewrite_in(0,) 0 0 WITH CAST(\'default\', \'String\') AS id_0 SELECT one.dummy, ignore(id_0) FROM system.one WHERE dummy IN tuple(0) +0 +0 errors others 0 diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql index dc481ccca72..dbe76f146b0 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql @@ -81,18 +81,18 @@ select query from system.query_log where type = 'QueryFinish' order by query; +-- not tuple +select * from dist_01756 where dummy in (0); +select * from dist_01756 where dummy in ('0'); + -- -- errors -- select 'errors'; --- not tuple -select * from dist_01756 where dummy in (0); -- { serverError 507 } -- optimize_skip_unused_shards does not support non-constants select * from dist_01756 where dummy in (select * from system.one); -- { serverError 507 } select * from dist_01756 where dummy in (toUInt8(0)); -- { serverError 507 } --- wrong type (tuple) -select * from dist_01756 where dummy in ('0'); -- { serverError 507 } -- intHash64 does not accept string select * from dist_01756 where dummy in ('0', '2'); -- { serverError 43 } -- NOT IN does not supported diff --git a/tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.reference b/tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.sh b/tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.sh new file mode 100755 index 00000000000..9181cc94564 --- /dev/null +++ b/tests/queries/0_stateless/01762_deltasumtimestamp_datetime64.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="SELECT deltaSumTimestamp(1, now64());" 2>&1 | grep -q "Code: 43.*Illegal type DateTime64" && echo 'OK' || echo 'FAIL'; + diff --git a/tests/queries/0_stateless/01785_pmj_lc_bug.sql b/tests/queries/0_stateless/01785_pmj_lc_bug.sql index 722faa9b40d..3020692c80a 100644 --- a/tests/queries/0_stateless/01785_pmj_lc_bug.sql +++ b/tests/queries/0_stateless/01785_pmj_lc_bug.sql @@ -12,3 +12,6 @@ SELECT 1025 == count(n) FROM foo AS t1 ANY LEFT JOIN foo_lc AS t2 ON t1.n == t2. SELECT 1025 == count(n) FROM foo_lc AS t1 ANY LEFT JOIN foo AS t2 ON t1.n == t2.n; SELECT 1025 == count(n) FROM foo_lc AS t1 ALL LEFT JOIN foo_lc AS t2 ON t1.n == t2.n; + +DROP TABLE foo; +DROP TABLE foo_lc; diff --git a/tests/queries/0_stateless/01802_test_postgresql_protocol_with_row_policy.sh b/tests/queries/0_stateless/01802_test_postgresql_protocol_with_row_policy.sh index edd73131020..4b3e35047bc 100755 --- a/tests/queries/0_stateless/01802_test_postgresql_protocol_with_row_policy.sh +++ b/tests/queries/0_stateless/01802_test_postgresql_protocol_with_row_policy.sh @@ -41,3 +41,5 @@ SELECT 'after row policy with plaintext_password'; psql "postgresql://postgresql_user:qwerty@localhost:${CLICKHOUSE_PORT_POSTGRESQL}/db01802" -c "SELECT * FROM postgresql;" +$CLICKHOUSE_CLIENT -q "DROP TABLE db01802.postgresql" +$CLICKHOUSE_CLIENT -q "DROP DATABASE db01802" diff --git a/tests/queries/0_stateless/01832_memory_write_suffix.sql b/tests/queries/0_stateless/01832_memory_write_suffix.sql index 718a4e4ac9d..274736c5c06 100644 --- a/tests/queries/0_stateless/01832_memory_write_suffix.sql +++ b/tests/queries/0_stateless/01832_memory_write_suffix.sql @@ -5,3 +5,5 @@ drop table if exists data_01832; create table data_01832 (key Int) Engine=Memory; insert into data_01832 values (1); select * from data_01832; + +drop table data_01832; diff --git a/tests/queries/0_stateless/01838_system_dictionaries_virtual_key_column.sql b/tests/queries/0_stateless/01838_system_dictionaries_virtual_key_column.sql index 97d96f643cf..bcc26491ef2 100644 --- a/tests/queries/0_stateless/01838_system_dictionaries_virtual_key_column.sql +++ b/tests/queries/0_stateless/01838_system_dictionaries_virtual_key_column.sql @@ -24,3 +24,6 @@ LAYOUT(COMPLEX_KEY_DIRECT()); SELECT 'complex key'; SELECT name, key FROM system.dictionaries WHERE name='example_complex_key_dictionary' AND database=currentDatabase(); + +DROP DICTIONARY example_complex_key_dictionary; +DROP DICTIONARY example_simple_key_dictionary; diff --git a/tests/queries/0_stateless/01846_null_as_default_for_insert_select.sql b/tests/queries/0_stateless/01846_null_as_default_for_insert_select.sql index d6f3b8a2136..44a1537f720 100644 --- a/tests/queries/0_stateless/01846_null_as_default_for_insert_select.sql +++ b/tests/queries/0_stateless/01846_null_as_default_for_insert_select.sql @@ -28,3 +28,5 @@ CREATE TABLE test_null_as_default (a Int8, b Int64 DEFAULT c - 500, c Int32 DEFA INSERT INTO test_null_as_default(a, c) SELECT 1, NULL UNION ALL SELECT 2, NULL; SELECT * FROM test_null_as_default ORDER BY a; + +DROP TABLE test_null_as_default; diff --git a/tests/queries/0_stateless/01848_http_insert_segfault.sh b/tests/queries/0_stateless/01848_http_insert_segfault.sh index a263ded44eb..c766e9794ae 100755 --- a/tests/queries/0_stateless/01848_http_insert_segfault.sh +++ b/tests/queries/0_stateless/01848_http_insert_segfault.sh @@ -4,5 +4,4 @@ # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh - ${CLICKHOUSE_LOCAL} -q "select col1, initializeAggregation('argMaxState', col2, insertTime) as col2, now() as insertTime FROM generateRandom('col1 String, col2 Array(Float64)') LIMIT 1000000 FORMAT CSV" | curl -s 'http://localhost:8123/?query=INSERT%20INTO%20non_existing_table%20SELECT%20col1%2C%20initializeAggregation(%27argMaxState%27%2C%20col2%2C%20insertTime)%20as%20col2%2C%20now()%20as%20insertTime%20FROM%20input(%27col1%20String%2C%20col2%20Array(Float64)%27)%20FORMAT%20CSV' --data-binary @- | grep -q "Table default.non_existing_table doesn't exist" && echo 'Ok.' || echo 'FAIL' ||: - + ${CLICKHOUSE_LOCAL} -q "select col1, initializeAggregation('argMaxState', col2, insertTime) as col2, now() as insertTime FROM generateRandom('col1 String, col2 Array(Float64)') LIMIT 1000000 FORMAT CSV" | ${CLICKHOUSE_CURL} -s 'http://localhost:8123/?query=INSERT%20INTO%20non_existing_table%20SELECT%20col1%2C%20initializeAggregation(%27argMaxState%27%2C%20col2%2C%20insertTime)%20as%20col2%2C%20now()%20as%20insertTime%20FROM%20input(%27col1%20String%2C%20col2%20Array(Float64)%27)%20FORMAT%20CSV' --data-binary @- | grep -q "Table default.non_existing_table doesn't exist" && echo 'Ok.' || echo 'FAIL' ||: diff --git a/tests/queries/0_stateless/01850_dist_INSERT_preserve_error.sql b/tests/queries/0_stateless/01850_dist_INSERT_preserve_error.sql index 6ed4eafcc8f..ab4c63ccdf2 100644 --- a/tests/queries/0_stateless/01850_dist_INSERT_preserve_error.sql +++ b/tests/queries/0_stateless/01850_dist_INSERT_preserve_error.sql @@ -13,3 +13,6 @@ insert into dist_01850 values (1); -- { serverError 60 } drop table if exists dist_01850; drop table shard_0.data_01850; + +drop database shard_0; +drop database shard_1; diff --git a/tests/queries/0_stateless/01852_dictionary_found_rate_long.sql b/tests/queries/0_stateless/01852_dictionary_found_rate_long.sql index 2501e21b95e..a51309b3ffd 100644 --- a/tests/queries/0_stateless/01852_dictionary_found_rate_long.sql +++ b/tests/queries/0_stateless/01852_dictionary_found_rate_long.sql @@ -307,3 +307,4 @@ SELECT name, found_rate FROM system.dictionaries WHERE database = currentDatabas DROP TABLE polygons_01862; DROP TABLE points_01862; +DROP DICTIONARY polygon_dictionary_01862; diff --git a/tests/queries/0_stateless/01852_jit_if.sql b/tests/queries/0_stateless/01852_jit_if.sql index cdfeba5b895..9a9d74274c2 100644 --- a/tests/queries/0_stateless/01852_jit_if.sql +++ b/tests/queries/0_stateless/01852_jit_if.sql @@ -14,3 +14,6 @@ INSERT INTO test_jit_nullable VALUES (0), (1), (NULL); SELECT 'test_jit_nullable'; SELECT value, multiIf(value = 1, 2, value, 1, 0), if (value, 1, 0) FROM test_jit_nullable; + +DROP TABLE test_jit_nonnull; +DROP TABLE test_jit_nullable; diff --git a/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh b/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh index a585785d710..409f8d5ecf5 100755 --- a/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh +++ b/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh @@ -36,6 +36,7 @@ function run_test_once() $CLICKHOUSE_CLIENT -nm -q " DROP DICTIONARY simple_key_cache_dictionary_01863; + DROP TABLE simple_key_source_table_01863; " if [ "$prev" == "$curr" ]; then diff --git a/tests/queries/0_stateless/01854_HTTP_dict_decompression.sh b/tests/queries/0_stateless/01854_HTTP_dict_decompression.sh index cca710e85cf..75c0e4de29f 100755 --- a/tests/queries/0_stateless/01854_HTTP_dict_decompression.sh +++ b/tests/queries/0_stateless/01854_HTTP_dict_decompression.sh @@ -5,3 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh python3 "$CURDIR"/01854_HTTP_dict_decompression.python + +$CLICKHOUSE_CLIENT -q "DROP DICTIONARY test_table_select" diff --git a/tests/queries/0_stateless/01866_aggregate_function_segment_length_sum.reference b/tests/queries/0_stateless/01866_aggregate_function_interval_length_sum.reference similarity index 100% rename from tests/queries/0_stateless/01866_aggregate_function_segment_length_sum.reference rename to tests/queries/0_stateless/01866_aggregate_function_interval_length_sum.reference diff --git a/tests/queries/0_stateless/01866_aggregate_function_interval_length_sum.sql b/tests/queries/0_stateless/01866_aggregate_function_interval_length_sum.sql new file mode 100644 index 00000000000..4da308453c0 --- /dev/null +++ b/tests/queries/0_stateless/01866_aggregate_function_interval_length_sum.sql @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS interval; +DROP TABLE IF EXISTS fl_interval; +DROP TABLE IF EXISTS dt_interval; +DROP TABLE IF EXISTS date_interval; + +CREATE TABLE interval ( `id` String, `start` Int64, `end` Int64 ) ENGINE = MergeTree ORDER BY start; +INSERT INTO interval VALUES ('a', 1, 3), ('a', 1, 3), ('a', 2, 4), ('a', 1, 1), ('a', 5, 6), ('a', 5, 7), ('b', 10, 12), ('b', 13, 19), ('b', 14, 16), ('c', -1, 1), ('c', -2, -1); + +CREATE TABLE fl_interval ( `id` String, `start` Float, `end` Float ) ENGINE = MergeTree ORDER BY start; +INSERT INTO fl_interval VALUES ('a', 1.1, 3.2), ('a', 1.5, 3.6), ('a', 4.0, 5.0); + +CREATE TABLE dt_interval ( `id` String, `start` DateTime, `end` DateTime ) ENGINE = MergeTree ORDER BY start; +INSERT INTO dt_interval VALUES ('a', '2020-01-01 02:11:22', '2020-01-01 03:12:31'), ('a', '2020-01-01 01:12:30', '2020-01-01 02:50:11'); + +CREATE TABLE date_interval ( `id` String, `start` Date, `end` Date ) ENGINE = MergeTree ORDER BY start; +INSERT INTO date_interval VALUES ('a', '2020-01-01', '2020-01-04'), ('a', '2020-01-03', '2020-01-08 02:50:11'); + +SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM interval GROUP BY id ORDER BY id; +SELECT id, 3.4 < intervalLengthSum(start, end) AND intervalLengthSum(start, end) < 3.6, toTypeName(intervalLengthSum(start, end)) FROM fl_interval GROUP BY id ORDER BY id; +SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM dt_interval GROUP BY id ORDER BY id; +SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM date_interval GROUP BY id ORDER BY id; + +DROP TABLE interval; +DROP TABLE fl_interval; +DROP TABLE dt_interval; +DROP TABLE date_interval; diff --git a/tests/queries/0_stateless/01866_aggregate_function_segment_length_sum.sql b/tests/queries/0_stateless/01866_aggregate_function_segment_length_sum.sql deleted file mode 100644 index c4e4f8afa85..00000000000 --- a/tests/queries/0_stateless/01866_aggregate_function_segment_length_sum.sql +++ /dev/null @@ -1,26 +0,0 @@ -DROP TABLE IF EXISTS segment; -DROP TABLE IF EXISTS fl_segment; -DROP TABLE IF EXISTS dt_segment; -DROP TABLE IF EXISTS date_segment; - -CREATE TABLE segment ( `id` String, `start` Int64, `end` Int64 ) ENGINE = MergeTree ORDER BY start; -INSERT INTO segment VALUES ('a', 1, 3), ('a', 1, 3), ('a', 2, 4), ('a', 1, 1), ('a', 5, 6), ('a', 5, 7), ('b', 10, 12), ('b', 13, 19), ('b', 14, 16), ('c', -1, 1), ('c', -2, -1); - -CREATE TABLE fl_segment ( `id` String, `start` Float, `end` Float ) ENGINE = MergeTree ORDER BY start; -INSERT INTO fl_segment VALUES ('a', 1.1, 3.2), ('a', 1.5, 3.6), ('a', 4.0, 5.0); - -CREATE TABLE dt_segment ( `id` String, `start` DateTime, `end` DateTime ) ENGINE = MergeTree ORDER BY start; -INSERT INTO dt_segment VALUES ('a', '2020-01-01 02:11:22', '2020-01-01 03:12:31'), ('a', '2020-01-01 01:12:30', '2020-01-01 02:50:11'); - -CREATE TABLE date_segment ( `id` String, `start` Date, `end` Date ) ENGINE = MergeTree ORDER BY start; -INSERT INTO date_segment VALUES ('a', '2020-01-01', '2020-01-04'), ('a', '2020-01-03', '2020-01-08 02:50:11'); - -SELECT id, segmentLengthSum(start, end), toTypeName(segmentLengthSum(start, end)) FROM segment GROUP BY id ORDER BY id; -SELECT id, 3.4 < segmentLengthSum(start, end) AND segmentLengthSum(start, end) < 3.6, toTypeName(segmentLengthSum(start, end)) FROM fl_segment GROUP BY id ORDER BY id; -SELECT id, segmentLengthSum(start, end), toTypeName(segmentLengthSum(start, end)) FROM dt_segment GROUP BY id ORDER BY id; -SELECT id, segmentLengthSum(start, end), toTypeName(segmentLengthSum(start, end)) FROM date_segment GROUP BY id ORDER BY id; - -DROP TABLE segment; -DROP TABLE fl_segment; -DROP TABLE dt_segment; -DROP TABLE date_segment; diff --git a/tests/queries/0_stateless/01866_datetime64_cmp_with_constant.sql b/tests/queries/0_stateless/01866_datetime64_cmp_with_constant.sql index e6782656887..15edf8b2f9d 100644 --- a/tests/queries/0_stateless/01866_datetime64_cmp_with_constant.sql +++ b/tests/queries/0_stateless/01866_datetime64_cmp_with_constant.sql @@ -38,3 +38,5 @@ SELECT 'dt64 != const dt' FROM dt64test WHERE dt64_column != toDateTime('2020-01 SELECT 'dt64 != dt' FROM dt64test WHERE dt64_column != materialize(toDateTime('2020-01-13 13:37:00')); SELECT 'dt != const dt64' FROM dt64test WHERE dt_column != toDateTime64('2020-01-13 13:37:00', 3); SELECT 'dt != dt64' FROM dt64test WHERE dt_column != materialize(toDateTime64('2020-01-13 13:37:00', 3)); + +DROP TABLE dt64test; diff --git a/tests/queries/0_stateless/01870_modulo_partition_key.sql b/tests/queries/0_stateless/01870_modulo_partition_key.sql index 344b32659a8..0226bcc0a3c 100644 --- a/tests/queries/0_stateless/01870_modulo_partition_key.sql +++ b/tests/queries/0_stateless/01870_modulo_partition_key.sql @@ -48,3 +48,7 @@ SELECT count() FROM table4 WHERE id % 10 = 7; SELECT 'comparison:'; SELECT v, v-205 as vv, modulo(vv, 200), moduloLegacy(vv, 200) FROM table1 ORDER BY v; +DROP TABLE table1; +DROP TABLE table2; +DROP TABLE table3; +DROP TABLE table4; diff --git a/tests/queries/0_stateless/01881_total_bytes_storage_buffer.reference b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.reference new file mode 100644 index 00000000000..eaa9e9259df --- /dev/null +++ b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.reference @@ -0,0 +1,5 @@ +0 +8192 +32 +8192 +32 diff --git a/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql new file mode 100644 index 00000000000..1fd9ea56e8f --- /dev/null +++ b/tests/queries/0_stateless/01881_total_bytes_storage_buffer.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS test_buffer_table; + +CREATE TABLE test_buffer_table +( + `a` Int64 +) +ENGINE = Buffer('', '', 1, 1000000, 1000000, 1000000, 1000000, 1000000, 1000000); + +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); + +INSERT INTO test_buffer_table SELECT number FROM numbers(1000); +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); + +OPTIMIZE TABLE test_buffer_table; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); + +INSERT INTO test_buffer_table SELECT number FROM numbers(1000); +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); + +OPTIMIZE TABLE test_buffer_table; +SELECT total_bytes FROM system.tables WHERE name = 'test_buffer_table' and database = currentDatabase(); + +DROP TABLE test_buffer_table; diff --git a/tests/queries/0_stateless/01882_scalar_subquery_exception.sql b/tests/queries/0_stateless/01882_scalar_subquery_exception.sql index c7a4555d876..0fb50846502 100644 --- a/tests/queries/0_stateless/01882_scalar_subquery_exception.sql +++ b/tests/queries/0_stateless/01882_scalar_subquery_exception.sql @@ -14,3 +14,6 @@ select / (select count() from nums_in_mem_dist where rand() > 0) from system.one; -- { serverError 158 } + +drop table nums_in_mem; +drop table nums_in_mem_dist; diff --git a/tests/queries/0_stateless/01882_total_rows_approx.sh b/tests/queries/0_stateless/01882_total_rows_approx.sh index 2c03298b324..f51e95b15c0 100755 --- a/tests/queries/0_stateless/01882_total_rows_approx.sh +++ b/tests/queries/0_stateless/01882_total_rows_approx.sh @@ -51,3 +51,5 @@ check_background_query & $CLICKHOUSE_CLIENT -q "select *, sleepEachRow(1) from data_01882" --max_threads=1 --format Null --query_id="$QUERY_ID" --max_block_size=1 wait + +$CLICKHOUSE_CLIENT -q "drop table data_01882" diff --git a/tests/integration/test_adaptive_granularity/__init__.py b/tests/queries/0_stateless/01882_uniqueState_over_uniqueState.reference similarity index 100% rename from tests/integration/test_adaptive_granularity/__init__.py rename to tests/queries/0_stateless/01882_uniqueState_over_uniqueState.reference diff --git a/tests/queries/0_stateless/01882_uniqueState_over_uniqueState.sh b/tests/queries/0_stateless/01882_uniqueState_over_uniqueState.sh new file mode 100755 index 00000000000..6be05a128d7 --- /dev/null +++ b/tests/queries/0_stateless/01882_uniqueState_over_uniqueState.sh @@ -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 diff --git a/tests/queries/0_stateless/01889_postgresql_protocol_null_fields.reference b/tests/queries/0_stateless/01889_postgresql_protocol_null_fields.reference new file mode 100644 index 00000000000..17ee5ae55a2 --- /dev/null +++ b/tests/queries/0_stateless/01889_postgresql_protocol_null_fields.reference @@ -0,0 +1,5 @@ + NULL +------ + +(1 row) + diff --git a/tests/queries/0_stateless/01889_postgresql_protocol_null_fields.sh b/tests/queries/0_stateless/01889_postgresql_protocol_null_fields.sh new file mode 100755 index 00000000000..9d61f7034cf --- /dev/null +++ b/tests/queries/0_stateless/01889_postgresql_protocol_null_fields.sh @@ -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;" diff --git a/tests/queries/0_stateless/01891_not_in_partition_prune.reference b/tests/queries/0_stateless/01891_not_in_partition_prune.reference new file mode 100644 index 00000000000..628053cd4f8 --- /dev/null +++ b/tests/queries/0_stateless/01891_not_in_partition_prune.reference @@ -0,0 +1,6 @@ +7 +0 100 +6 106 +7 107 +8 108 +9 109 diff --git a/tests/queries/0_stateless/01891_not_in_partition_prune.sql b/tests/queries/0_stateless/01891_not_in_partition_prune.sql new file mode 100644 index 00000000000..edbfad93e5d --- /dev/null +++ b/tests/queries/0_stateless/01891_not_in_partition_prune.sql @@ -0,0 +1,10 @@ +drop table if exists test1; + +create table test1 (i int, j int) engine MergeTree partition by i order by tuple() settings index_granularity = 1; + +insert into test1 select number, number + 100 from numbers(10); +select count() from test1 where i not in (1,2,3); +set max_rows_to_read = 5; +select * from test1 where i not in (1,2,3,4,5) order by i; + +drop table test1; diff --git a/tests/queries/0_stateless/01892_setting_limit_offset_distributed.reference b/tests/queries/0_stateless/01892_setting_limit_offset_distributed.reference new file mode 100644 index 00000000000..466e80931e5 --- /dev/null +++ b/tests/queries/0_stateless/01892_setting_limit_offset_distributed.reference @@ -0,0 +1,14 @@ +limit 0 +limit 1 +limit 2 +limit 3 +limit 4 +offset 5 +offset 6 +offset 7 +offset 8 +offset 9 +limit w/ GROUP BY 4 4 +limit w/ GROUP BY 4 3 +limit/offset w/ GROUP BY 4 2 +limit/offset w/ GROUP BY 4 1 diff --git a/tests/queries/0_stateless/01892_setting_limit_offset_distributed.sql b/tests/queries/0_stateless/01892_setting_limit_offset_distributed.sql new file mode 100644 index 00000000000..bcd6e47d4c7 --- /dev/null +++ b/tests/queries/0_stateless/01892_setting_limit_offset_distributed.sql @@ -0,0 +1,30 @@ +SELECT 'limit', * FROM remote('127.1', view(SELECT * FROM numbers(10))) SETTINGS limit=5; +SELECT 'offset', * FROM remote('127.1', view(SELECT * FROM numbers(10))) SETTINGS offset=5; + +SELECT + 'limit w/ GROUP BY', + count(), + number +FROM remote('127.{1,2}', view( + SELECT intDiv(number, 2) AS number + FROM numbers(10) +)) +GROUP BY number +ORDER BY + count() ASC, + number DESC +SETTINGS limit=2; + +SELECT + 'limit/offset w/ GROUP BY', + count(), + number +FROM remote('127.{1,2}', view( + SELECT intDiv(number, 2) AS number + FROM numbers(10) +)) +GROUP BY number +ORDER BY + count() ASC, + number DESC +SETTINGS limit=2, offset=2; diff --git a/tests/queries/0_stateless/01901_in_literal_shard_prune.reference b/tests/queries/0_stateless/01901_in_literal_shard_prune.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01901_in_literal_shard_prune.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01901_in_literal_shard_prune.sql b/tests/queries/0_stateless/01901_in_literal_shard_prune.sql new file mode 100644 index 00000000000..cc587c36613 --- /dev/null +++ b/tests/queries/0_stateless/01901_in_literal_shard_prune.sql @@ -0,0 +1,15 @@ +set optimize_skip_unused_shards=1; +set force_optimize_skip_unused_shards=1; + +drop table if exists d; +drop table if exists dp; + +create table d (i UInt8) Engine=Memory; +create table dp as d Engine=Distributed(test_cluster_two_shards, currentDatabase(), d, i); + +insert into d values (1), (2); + +select * from dp where i in (1); + +drop table if exists d; +drop table if exists dp; diff --git a/tests/queries/0_stateless/01901_test_attach_partition_from.reference b/tests/queries/0_stateless/01901_test_attach_partition_from.reference new file mode 100644 index 00000000000..be589c9ceb0 --- /dev/null +++ b/tests/queries/0_stateless/01901_test_attach_partition_from.reference @@ -0,0 +1,4 @@ +1 +1 +2 +2 diff --git a/tests/queries/0_stateless/01901_test_attach_partition_from.sql b/tests/queries/0_stateless/01901_test_attach_partition_from.sql new file mode 100644 index 00000000000..0ffa4cdecf2 --- /dev/null +++ b/tests/queries/0_stateless/01901_test_attach_partition_from.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS test_alter_attach_01901S; +DROP TABLE IF EXISTS test_alter_attach_01901D; + +CREATE TABLE test_alter_attach_01901S (A Int64, D date) ENGINE = MergeTree PARTITION BY D ORDER BY A; +INSERT INTO test_alter_attach_01901S VALUES (1, '2020-01-01'); + +CREATE TABLE test_alter_attach_01901D (A Int64, D date) +Engine=ReplicatedMergeTree('/clickhouse/tables/test_alter_attach_01901D', 'r1') +PARTITION BY D ORDER BY A; + +ALTER TABLE test_alter_attach_01901D ATTACH PARTITION '2020-01-01' FROM test_alter_attach_01901S; + +SELECT count() FROM test_alter_attach_01901D; +SELECT count() FROM test_alter_attach_01901S; + +INSERT INTO test_alter_attach_01901S VALUES (1, '2020-01-01'); +ALTER TABLE test_alter_attach_01901D REPLACE PARTITION '2020-01-01' FROM test_alter_attach_01901S; + +SELECT count() FROM test_alter_attach_01901D; +SELECT count() FROM test_alter_attach_01901S; + +DROP TABLE test_alter_attach_01901S; +DROP TABLE test_alter_attach_01901D; diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index a59c1a72a52..dac43ff5d4b 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -240,3 +240,4 @@ 01880_remote_ipv6 01882_scalar_subquery_exception 01882_check_max_parts_to_merge_at_once +01892_setting_limit_offset_distributed diff --git a/tests/queries/conftest.py b/tests/queries/conftest.py index 40a9a6b3a2e..a18ba846066 100644 --- a/tests/queries/conftest.py +++ b/tests/queries/conftest.py @@ -1,18 +1,35 @@ -import pytest - import os import sys -import tempfile +import re + +import pytest from .server import ServerThread +# Command-line arguments + def pytest_addoption(parser): parser.addoption( "--builddir", action="store", default=None, help="Path to build directory to use binaries from", ) +# HTML report hooks + +def pytest_html_report_title(report): + report.title = "ClickHouse Functional Stateless Tests (PyTest)" + + +RE_TEST_NAME = re.compile(r"\[(.*)\]") +def pytest_itemcollected(item): + match = RE_TEST_NAME.search(item.name) + if match: + item._nodeid = match.group(1) + + +# Fixtures + @pytest.fixture(scope='module') def cmdopts(request): return { diff --git a/tests/queries/pytest.ini b/tests/queries/pytest.ini new file mode 100644 index 00000000000..13a2ebbaf83 --- /dev/null +++ b/tests/queries/pytest.ini @@ -0,0 +1,2 @@ +[pytest] +render_collapsed = True diff --git a/tests/queries/query_test.py b/tests/queries/query_test.py index 6ebeccbeeac..d344abe0a1e 100644 --- a/tests/queries/query_test.py +++ b/tests/queries/query_test.py @@ -15,6 +15,7 @@ SKIP_LIST = [ # just fail "00133_long_shard_memory_tracker_and_exception_safety", + "00463_long_sessions_in_http_interface", "00505_secure", "00505_shard_secure", "00646_url_engine", @@ -56,8 +57,10 @@ SKIP_LIST = [ "01098_temporary_and_external_tables", "01099_parallel_distributed_insert_select", "01103_check_cpu_instructions_at_startup", + "01107_atomic_db_detach_attach", "01114_database_atomic", "01148_zookeeper_path_macros_unfolding", + "01152_cross_replication", # tcp port in reference "01175_distributed_ddl_output_mode_long", "01181_db_atomic_drop_on_cluster", # tcp port in reference "01280_ssd_complex_key_dictionary", @@ -99,19 +102,30 @@ SKIP_LIST = [ "01601_custom_tld", "01606_git_import", "01610_client_spawn_editor", # expect-test + "01654_test_writer_block_sequence", # No module named 'pandas' "01658_read_file_to_stringcolumn", "01666_merge_tree_max_query_limit", "01674_unicode_asan", "01676_clickhouse_client_autocomplete", # expect-test (partially) + "01676_long_clickhouse_client_autocomplete", "01683_text_log_deadlock", # secure tcp "01684_ssd_cache_dictionary_simple_key", "01685_ssd_cache_dictionary_complex_key", + "01737_clickhouse_server_wait_server_pool_long", "01746_executable_pool_dictionary", "01747_executable_pool_dictionary_implicit_key", "01747_join_view_filter_dictionary", "01748_dictionary_table_dot", "01754_cluster_all_replicas_shard_num", "01759_optimize_skip_unused_shards_zero_shards", + "01763_max_distributed_depth", # BROKEN + "01780_clickhouse_dictionary_source_loop", + "01801_s3_cluster", + "01802_test_postgresql_protocol_with_row_policy", + "01804_dictionary_decimal256_type", # hardcoded path + "01848_http_insert_segfault", + "01875_ssd_cache_dictionary_decimal256_type", + "01880_remote_ipv6", ] @@ -154,6 +168,7 @@ def run_shell(bin_prefix, server, database, path, reference, replace_map=None): 'CLICKHOUSE_PORT_TCP_WITH_PROXY': str(server.proxy_port), 'CLICKHOUSE_PORT_HTTP': str(server.http_port), 'CLICKHOUSE_PORT_INTERSERVER': str(server.inter_port), + 'CLICKHOUSE_PORT_POSTGRESQL': str(server.postgresql_port), 'CLICKHOUSE_TMP': server.tmp_dir, 'CLICKHOUSE_CONFIG_CLIENT': server.client_config } diff --git a/tests/queries/server.py b/tests/queries/server.py index ed12931e658..50110bc41b9 100644 --- a/tests/queries/server.py +++ b/tests/queries/server.py @@ -38,6 +38,7 @@ class ServerThread(threading.Thread): self.https_port = port_base + 5 self.odbc_port = port_base + 6 self.proxy_port = port_base + 7 + self.postgresql_port = port_base + 8 self._args = [ '--config-file={config_path}'.format(config_path=self.server_config), @@ -46,6 +47,7 @@ class ServerThread(threading.Thread): '--http_port={http_port}'.format(http_port=self.http_port), '--interserver_http_port={inter_port}'.format(inter_port=self.inter_port), '--tcp_with_proxy_port={proxy_port}'.format(proxy_port=self.proxy_port), + '--postgresql_port={psql_port}'.format(psql_port=self.postgresql_port), # TODO: SSL certificate is not specified '--tcp_port_secure={tcps_port}'.format(tcps_port=self.tcps_port), ] @@ -75,7 +77,7 @@ class ServerThread(threading.Thread): time.sleep(ServerThread.DEFAULT_SERVER_DELAY) s = socket.create_connection(('localhost', self.tcp_port), ServerThread.DEFAULT_CONNECTION_TIMEOUT) s.sendall(b'G') # trigger expected "bad" HELLO response - print('Successful server response:', s.recv(1024)) # FIXME: read whole buffered response + s.recv(1024) # FIXME: read whole buffered response s.shutdown(socket.SHUT_RDWR) s.close() except Exception: @@ -116,7 +118,7 @@ class ServerThread(threading.Thread): if self._proc.returncode is None: self._proc.terminate() self.join() - print('Stop clickhouse-server') + print('Stopped clickhouse-server') ServerThread.DEFAULT_SERVER_CONFIG = \ @@ -289,6 +291,21 @@ ServerThread.DEFAULT_SERVER_CONFIG = \ + + + + + shard_0 + localhost + {tcp_port} + + + shard_1 + localhost + {tcp_port} + + + diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index d3fec057c2a..010dce27da7 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -13,6 +13,7 @@ "01193_metadata_loading", "01473_event_time_microseconds", "01526_max_untracked_memory", /// requires TraceCollector, does not available under sanitizers + "01594_too_low_memory_limits", /// requires jemalloc to track small allocations "01474_executable_dictionary", /// informational stderr from sanitizer at start "functions_bad_arguments", /// Too long for TSan "01603_read_with_backoff_bug", /// Too long for TSan @@ -28,6 +29,7 @@ "01103_check_cpu_instructions_at_startup", "01473_event_time_microseconds", "01526_max_untracked_memory", /// requires TraceCollector, does not available under sanitizers + "01594_too_low_memory_limits", /// requires jemalloc to track small allocations "01193_metadata_loading", "01017_uniqCombined_memory_usage" /// Fine thresholds on memory usage ], @@ -39,6 +41,7 @@ "00900_orc_load", "01473_event_time_microseconds", "01526_max_untracked_memory", /// requires TraceCollector, does not available under sanitizers + "01594_too_low_memory_limits", /// requires jemalloc to track small allocations "01193_metadata_loading" ], "memory-sanitizer": [ @@ -51,6 +54,7 @@ "00877_memory_limit_for_new_delete", /// memory limits don't work correctly under msan because it replaces malloc/free "01473_event_time_microseconds", "01526_max_untracked_memory", /// requires TraceCollector, does not available under sanitizers + "01594_too_low_memory_limits", /// requires jemalloc to track small allocations "01193_metadata_loading", "01017_uniqCombined_memory_usage" /// Fine thresholds on memory usage ], @@ -406,7 +410,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": [ @@ -717,6 +722,7 @@ "01684_ssd_cache_dictionary_simple_key", "01685_ssd_cache_dictionary_complex_key", "01737_clickhouse_server_wait_server_pool_long", // This test is fully compatible to run in parallel, however under ASAN processes are pretty heavy and may fail under flaky adress check. + "01594_too_low_memory_limits", // This test is fully compatible to run in parallel, however under ASAN processes are pretty heavy and may fail under flaky adress check. "01760_system_dictionaries", "01760_polygon_dictionaries", "01778_hierarchical_dictionaries", @@ -728,6 +734,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" ] } diff --git a/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot b/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot new file mode 100644 index 00000000000..6e3848b9e68 --- /dev/null +++ b/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot @@ -0,0 +1,6282 @@ +I_check_plus_with_Int128_max_and_min_value = r""" +plus(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) plus(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +-170141183460469231731687303715884105728 -170141183460469231731687303715884105727 +""" + +I_check_plus_with_Int256_max_and_min_value = r""" +plus(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) plus(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 -57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_plus_with_UInt128_max_and_min_value = r""" +plus(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) plus(toUInt128(\'0\'), toUInt128(1)) +0 1 +""" + +I_check_plus_with_UInt256_max_and_min_value = r""" +plus(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) plus(toUInt256(\'0\'), toUInt256(1)) +0 1 +""" + +I_check_minus_with_Int128_max_and_min_value = r""" +minus(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) minus(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +170141183460469231731687303715884105726 170141183460469231731687303715884105727 +""" + +I_check_minus_with_Int256_max_and_min_value = r""" +minus(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) minus(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +57896044618658097711785492504343953926634992332820282019728792003956564819966 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_minus_with_UInt128_max_and_min_value = r""" +minus(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) minus(toUInt128(\'0\'), toUInt128(1)) +-2 -1 +""" + +I_check_minus_with_UInt256_max_and_min_value = r""" +minus(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) minus(toUInt256(\'0\'), toUInt256(1)) +-2 -1 +""" + +I_check_multiply_with_Int128_max_and_min_value = r""" +multiply(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) multiply(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +170141183460469231731687303715884105727 -170141183460469231731687303715884105728 +""" + +I_check_multiply_with_Int256_max_and_min_value = r""" +multiply(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) multiply(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +57896044618658097711785492504343953926634992332820282019728792003956564819967 -57896044618658097711785492504343953926634992332820282019728792003956564819968 +""" + +I_check_multiply_with_UInt128_max_and_min_value = r""" +multiply(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) multiply(toUInt128(\'0\'), toUInt128(1)) +340282366920938463463374607431768211455 0 +""" + +I_check_multiply_with_UInt256_max_and_min_value = r""" +multiply(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) multiply(toUInt256(\'0\'), toUInt256(1)) +115792089237316195423570985008687907853269984665640564039457584007913129639935 0 +""" + +I_check_divide_with_Int128_max_and_min_value = r""" +divide(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) divide(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +1.7014118346046923e38 -1.7014118346046923e38 +""" + +I_check_divide_with_Int256_max_and_min_value = r""" +divide(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) divide(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +5.78960446186581e76 -5.78960446186581e76 +""" + +I_check_divide_with_UInt128_max_and_min_value = r""" +divide(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) divide(toUInt128(\'0\'), toUInt128(1)) +3.402823669209385e38 0 +""" + +I_check_divide_with_UInt256_max_and_min_value = r""" +divide(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) divide(toUInt256(\'0\'), toUInt256(1)) +1.157920892373162e77 0 +""" + +I_check_intDiv_with_Int128_max_and_min_value = r""" +intDiv(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) intDiv(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +170141183460469231731687303715884105727 -170141183460469231731687303715884105728 +""" + +I_check_intDiv_with_Int256_max_and_min_value = r""" +intDiv(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) intDiv(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +57896044618658097711785492504343953926634992332820282019728792003956564819967 -57896044618658097711785492504343953926634992332820282019728792003956564819968 +""" + +I_check_intDiv_with_UInt128_max_and_min_value = r""" +intDiv(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) intDiv(toUInt128(\'0\'), toUInt128(1)) +340282366920938463463374607431768211455 0 +""" + +I_check_intDiv_with_UInt256_max_and_min_value = r""" +intDiv(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) intDiv(toUInt256(\'0\'), toUInt256(1)) +115792089237316195423570985008687907853269984665640564039457584007913129639935 0 +""" + +I_check_intDivOrZero_with_Int128_max_and_min_value = r""" +intDivOrZero(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) intDivOrZero(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +170141183460469231731687303715884105727 -170141183460469231731687303715884105728 +""" + +I_check_intDivOrZero_with_Int256_max_and_min_value = r""" +intDivOrZero(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) intDivOrZero(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +57896044618658097711785492504343953926634992332820282019728792003956564819967 -57896044618658097711785492504343953926634992332820282019728792003956564819968 +""" + +I_check_intDivOrZero_with_UInt128_max_and_min_value = r""" +intDivOrZero(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) intDivOrZero(toUInt128(\'0\'), toUInt128(1)) +340282366920938463463374607431768211455 0 +""" + +I_check_intDivOrZero_with_UInt256_max_and_min_value = r""" +intDivOrZero(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) intDivOrZero(toUInt256(\'0\'), toUInt256(1)) +115792089237316195423570985008687907853269984665640564039457584007913129639935 0 +""" + +I_check_modulo_with_Int128_max_and_min_value = r""" +modulo(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) modulo(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +0 0 +""" + +I_check_modulo_with_Int256_max_and_min_value = r""" +modulo(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) modulo(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +0 0 +""" + +I_check_modulo_with_UInt128_max_and_min_value = r""" +modulo(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) modulo(toUInt128(\'0\'), toUInt128(1)) +0 0 +""" + +I_check_modulo_with_UInt256_max_and_min_value = r""" +modulo(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) modulo(toUInt256(\'0\'), toUInt256(1)) +0 0 +""" + +I_check_moduloOrZero_with_Int128_max_and_min_value = r""" +moduloOrZero(toInt128(\'170141183460469231731687303715884105727\'), toInt128(1)) moduloOrZero(toInt128(\'-170141183460469231731687303715884105728\'), toInt128(1)) +0 0 +""" + +I_check_moduloOrZero_with_Int256_max_and_min_value = r""" +moduloOrZero(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(1)) moduloOrZero(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), toInt256(1)) +0 0 +""" + +I_check_moduloOrZero_with_UInt128_max_and_min_value = r""" +moduloOrZero(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(1)) moduloOrZero(toUInt128(\'0\'), toUInt128(1)) +0 0 +""" + +I_check_moduloOrZero_with_UInt256_max_and_min_value = r""" +moduloOrZero(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(1)) moduloOrZero(toUInt256(\'0\'), toUInt256(1)) +0 0 +""" + +I_check_negate_with_Int128_max_and_min_value = r""" +negate(toInt128(\'170141183460469231731687303715884105727\')) negate(toInt128(\'-170141183460469231731687303715884105728\')) +-170141183460469231731687303715884105727 -170141183460469231731687303715884105728 +""" + +I_check_negate_with_Int256_max_and_min_value = r""" +negate(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) negate(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819967 -57896044618658097711785492504343953926634992332820282019728792003956564819968 +""" + +I_check_negate_with_UInt128_max_and_min_value = r""" +negate(toUInt128(\'340282366920938463463374607431768211455\')) negate(toUInt128(\'0\')) +1 0 +""" + +I_check_negate_with_UInt256_max_and_min_value = r""" +negate(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) negate(toUInt256(\'0\')) +1 0 +""" + +I_check_abs_with_Int128_max_and_min_value = r""" +abs(toInt128(\'170141183460469231731687303715884105727\')) abs(toInt128(\'-170141183460469231731687303715884105728\')) +170141183460469231731687303715884105727 170141183460469231731687303715884105728 +""" + +I_check_abs_with_Int256_max_and_min_value = r""" +abs(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) abs(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +57896044618658097711785492504343953926634992332820282019728792003956564819967 57896044618658097711785492504343953926634992332820282019728792003956564819968 +""" + +I_check_abs_with_UInt128_max_and_min_value = r""" +abs(toUInt128(\'340282366920938463463374607431768211455\')) abs(toUInt128(\'0\')) +340282366920938463463374607431768211455 0 +""" + +I_check_abs_with_UInt256_max_and_min_value = r""" +abs(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) abs(toUInt256(\'0\')) +115792089237316195423570985008687907853269984665640564039457584007913129639935 0 +""" + +I_check_the_table_output_of_plus_with_Int128 = r""" +a +-170141183460469231731687303715884105728 +-170141183460469231731687303715884105727 +2 +""" + +I_check_the_table_output_of_plus_with_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +-57896044618658097711785492504343953926634992332820282019728792003956564819967 +2 +""" + +I_check_the_table_output_of_plus_with_UInt128 = r""" +a +0 +1 +2 +""" + +I_check_the_table_output_of_plus_with_UInt256 = r""" +a +0 +1 +2 +""" + +I_check_the_table_output_of_minus_with_Int128 = r""" +a +0 +170141183460469231731687303715884105726 +170141183460469231731687303715884105727 +""" + +I_check_the_table_output_of_minus_with_Int256 = r""" +a +0 +57896044618658097711785492504343953926634992332820282019728792003956564819966 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_output_of_minus_with_UInt128 = r""" +a +0 +340282366920938463463374607431768211454 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_minus_with_UInt256 = r""" +a +0 +115792089237316195423570985008687907853269984665640564039457584007913129639934 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_output_of_multiply_with_Int128 = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_the_table_output_of_multiply_with_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_output_of_multiply_with_UInt128 = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_multiply_with_UInt256 = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_output_of_divide_with_Int128 = r""" +a +-170141183460469231722463931679029329921 +1 +170141183460469231722463931679029329921 +""" + +I_check_the_table_output_of_divide_with_Int256 = r""" +a +-57896044618658097702369839901263932781391731748390190090761097376371310592000 +1 +57896044618658097702369839901263932781391731748390190090761097376371310592000 +""" + +I_check_the_table_output_of_divide_with_UInt128 = r""" +a +0 +1 +340282366920938463426481119284349108225 +""" + +I_check_the_table_output_of_divide_with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_output_of_intDiv_with_Int128 = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_the_table_output_of_intDiv_with_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_output_of_intDiv_with_UInt128 = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_intDiv_with_UInt256 = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_output_of_intDivOrZero_with_Int128 = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_the_table_output_of_intDivOrZero_with_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_output_of_intDivOrZero_with_UInt128 = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_intDivOrZero_with_UInt256 = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_output_of_modulo_with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_modulo_with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_modulo_with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_modulo_with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_moduloOrZero_with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_moduloOrZero_with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_moduloOrZero_with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_moduloOrZero_with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_table_output_of_negate_with_Int128 = r""" +a +-170141183460469231731687303715884105728 +-170141183460469231731687303715884105727 +-1 +""" + +I_check_the_table_output_of_negate_with_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +-57896044618658097711785492504343953926634992332820282019728792003956564819967 +-1 +""" + +I_check_the_table_output_of_negate_with_UInt128 = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_negate_with_UInt256 = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_output_of_abs_with_Int128 = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_the_table_output_of_abs_with_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_output_of_abs_with_UInt128 = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_abs_with_UInt256 = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_output_of_gcd_with_Int128 = r""" +a +1 +""" + +I_check_the_table_output_of_gcd_with_Int256 = r""" +a +1 +""" + +I_check_the_table_output_of_gcd_with_UInt128 = r""" +a +1 +1 +""" + +I_check_the_table_output_of_gcd_with_UInt256 = r""" +a +1 +1 +""" + +I_check_the_table_output_of_lcm_with_Int128 = r""" +a +1 +""" + +I_check_the_table_output_of_lcm_with_Int256 = r""" +a +1 +""" + +I_check_the_table_output_of_lcm_with_UInt128 = r""" +a +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_output_of_lcm_with_UInt256 = r""" +a +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_for_output_of_negate_with_Decimal256 = r""" +a +-1 +""" + +I_check_the_table_for_output_of_abs_with_Decimal256 = r""" +a +1 +""" + +Inline___Int128___arrayPopBack_ = r""" +arrayPopBack(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,2] +""" + +Table___Int128___arrayPopBack_ = r""" +a +[3,2] +""" + +Inline___Int128___arrayPopFront_ = r""" +arrayPopFront(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[2,1] +""" + +Table___Int128___arrayPopFront_ = r""" +a +[2,1] +""" + +Inline___Int128___arraySort_ = r""" +arraySort(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1,2,3] +""" + +Table___Int128___arraySort_ = r""" +a +[1,2,3] +""" + +Inline___Int128___arrayReverseSort_ = r""" +arrayReverseSort(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,2,1] +""" + +Table___Int128___arrayReverseSort_ = r""" +a +[3,2,1] +""" + +Inline___Int128___arrayDistinct_ = r""" +arrayDistinct(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,2,1] +""" + +Table___Int128___arrayDistinct_ = r""" +a +[3,2,1] +""" + +Inline___Int128___arrayEnumerate_ = r""" +arrayEnumerate(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1,2,3] +""" + +Table___Int128___arrayEnumerate_ = r""" +a +[1,2,3] +""" + +Inline___Int128___arrayEnumerateDense_ = r""" +arrayEnumerateDense(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1,2,3] +""" + +Table___Int128___arrayEnumerateDense_ = r""" +a +[1,2,3] +""" + +Inline___Int128___arrayEnumerateUniq_ = r""" +arrayEnumerateUniq(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1,1,1] +""" + +Table___Int128___arrayEnumerateUniq_ = r""" +a +[1,1,1] +""" + +Inline___Int128___arrayReverse_ = r""" +arrayReverse(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1,2,3] +""" + +Table___Int128___arrayReverse_ = r""" +a +[1,2,3] +""" + +Inline___Int128___reverse_ = r""" +reverse(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1,2,3] +""" + +Table___Int128___reverse_ = r""" +a +[1,2,3] +""" + +Inline___Int128___arrayFlatten_ = r""" +arrayFlatten(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,2,1] +""" + +Table___Int128___arrayFlatten_ = r""" +a +[3,2,1] +""" + +Inline___Int128___arrayCompact_ = r""" +arrayCompact(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,2,1] +""" + +Table___Int128___arrayCompact_ = r""" +a +[3,2,1] +""" + +Inline___Int128___arrayReduceInRanges__sum_____1__5___ = r""" +arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[6] +""" + +Table___Int128___arrayReduceInRanges__sum_____1__5___ = r""" +a +[6] +""" + +Inline___Int128___arrayMap_x_____x___2__ = r""" +arrayMap(lambda(tuple(x), plus(x, 2)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[5,4,3] +""" + +Table___Int128___arrayMap_x_____x___2__ = r""" +a +[5,4,3] +""" + +Inline___Int128___arrayFill_x____x_3_ = r""" +arrayFill(lambda(tuple(x), equals(x, 3)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,3,3] +""" + +Table___Int128___arrayFill_x____x_3_ = r""" +a +[3,3,3] +""" + +Inline___Int128___arrayReverseFill_x____x_3_ = r""" +arrayReverseFill(lambda(tuple(x), equals(x, 3)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,1,1] +""" + +Table___Int128___arrayReverseFill_x____x_3_ = r""" +a +[3,1,1] +""" + +Inline___Int128___arrayConcat__toInt128__3____toInt128__2____toInt128__1____ = r""" +arrayConcat(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[3,2,1,3,2,1] +""" + +Table___Int128___arrayConcat__toInt128__3____toInt128__2____toInt128__1____ = r""" +a +[3,2,1,3,2,1] +""" + +Inline___Int128___arrayFilter_x____x____1__ = r""" +arrayFilter(lambda(tuple(x), equals(x, 1)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[1] +""" + +Table___Int128___arrayFilter_x____x____1__ = r""" +a +[1] +""" + +Inline___Int128___arraySplit__x__y_____x_y___0__0__0__ = r""" +arraySplit(lambda(tuple(x, y), equals(x, y)), [0, 0, 0], array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +[[0,0,0]] +""" + +Table___Int128___arraySplit__x__y_____x_y___0__0__0__ = r""" +a +[[0,0,0]] +""" + +Inline___Int128___arrayZip__toInt128__1____ = r""" +arrayZip(array(toInt128(\'1\')), array(toInt128(\'3\'))) +[(1,3)] +""" + +Table___Int128___arrayZip__toInt128__1____ = r""" +a +[(1,1)] +""" + +Inline___Int128___empty_ = r""" +empty(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +0 +""" + +Table___Int128___empty_ = r""" +a +0 +""" + +Inline___Int128___notEmpty_ = r""" +notEmpty(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___notEmpty_ = r""" +a +1 +""" + +Inline___Int128___length_ = r""" +length(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +3 +""" + +Table___Int128___length_ = r""" +a +3 +""" + +Inline___Int128___arrayCount_x____x____1__ = r""" +arrayCount(lambda(tuple(x), equals(x, 1)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___arrayCount_x____x____1__ = r""" +a +1 +""" + +Inline___Int128___arrayUniq_ = r""" +arrayUniq(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +3 +""" + +Table___Int128___arrayUniq_ = r""" +a +3 +""" + +Inline___Int128___arrayJoin_ = r""" +arrayJoin(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +3 +2 +1 +""" + +Table___Int128___arrayJoin_ = r""" +a +1 +2 +3 +""" + +Inline___Int128___arrayExists_x____x__1_ = r""" +arrayExists(lambda(tuple(x), equals(x, 1)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___arrayExists_x____x__1_ = r""" +a +1 +""" + +Inline___Int128___arrayAll_x____x__1_ = r""" +arrayAll(lambda(tuple(x), equals(x, 1)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +0 +""" + +Table___Int128___arrayAll_x____x__1_ = r""" +a +0 +""" + +Inline___Int128___arrayMin_ = r""" +arrayMin(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___arrayMin_ = r""" +a +1 +""" + +Inline___Int128___arrayMax_ = r""" +arrayMax(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +3 +""" + +Table___Int128___arrayMax_ = r""" +a +3 +""" + +Inline___Int128___arraySum_ = r""" +arraySum(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +6 +""" + +Table___Int128___arraySum_ = r""" +a +6 +""" + +Inline___Int128___arrayAvg_ = r""" +arrayAvg(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +2 +""" + +Table___Int128___arrayAvg_ = r""" +a +2 +""" + +Inline___Int128___arrayReduce__max___ = r""" +arrayReduce(\'max\', array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +3 +""" + +Table___Int128___arrayReduce__max___ = r""" +a +3 +""" + +Inline___Int128___arrayFirst_x____x__3_ = r""" +arrayFirst(lambda(tuple(x), equals(x, 3)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +3 +""" + +Table___Int128___arrayFirst_x____x__3_ = r""" +a +3 +""" + +Inline___Int128___arrayFirstIndex_x____x__3_ = r""" +arrayFirstIndex(lambda(tuple(x), equals(x, 3)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___arrayFirstIndex_x____x__3_ = r""" +a +1 +""" + +Inline___Int128___hasAll__toInt128__3____toInt128__2____toInt128__1_____ = r""" +hasAll(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___hasAll__toInt128__3____toInt128__2____toInt128__1_____ = r""" +a +1 +""" + +Inline___Int128___hasAny__toInt128__2____toInt128__1_____ = r""" +hasAny(array(toInt128(\'2\'), toInt128(\'1\')), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +1 +""" + +Table___Int128___hasAny__toInt128__2____toInt128__1_____ = r""" +a +1 +""" + +Inline___Int128___hasSubstr__toInt128__2____toInt128__1_____ = r""" +hasSubstr(array(toInt128(\'2\'), toInt128(\'1\')), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\'))) +0 +""" + +Table___Int128___hasSubstr__toInt128__2____toInt128__1_____ = r""" +a +0 +""" + +Table___Int128___arrayDifference_ = r""" +a +""" + +Table___Int128___arrayCumSum_ = r""" +a +""" + +Table___Int128___arrayCumSumNonNegative_ = r""" +a +""" + +Inline___Int128___arrayElement = r""" +arrayElement(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), 1) +3 +""" + +Table___Int128___arrayElement = r""" +a +3 +""" + +Inline___Int128___arrayPushBack = r""" +arrayPushBack(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), toInt128(\'1\')) +[3,2,1,1] +""" + +Table___Int128___arrayPushBack = r""" +a +[3,2,1,1] +""" + +Inline___Int128___arrayPushFront = r""" +arrayPushFront(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), toInt128(\'1\')) +[1,3,2,1] +""" + +Table___Int128___arrayPushFront = r""" +a +[1,3,2,1] +""" + +Inline___Int128___arrayResize = r""" +arrayResize(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), 1) +[3] +""" + +Table___Int128___arrayResize = r""" +a +[3] +""" + +Inline___Int128___arraySlice = r""" +arraySlice(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), 1) +[3,2,1] +""" + +Table___Int128___arraySlice = r""" +a +[3,2,1] +""" + +Inline___Int128___has = r""" +has(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), NULL) +0 +""" + +Table___Int128___has = r""" +a +0 +""" + +Inline___Int128___indexOf = r""" +indexOf(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), NULL) +0 +""" + +Table___Int128___indexOf = r""" +a +0 +""" + +Inline___Int128___countEqual = r""" +countEqual(array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')), NULL) +0 +""" + +Table___Int128___countEqual = r""" +a +0 +""" + +Creating_a_tuple_with_Int128_on_a_table = r""" +a +(1,1,1) +""" + +tupleElement_with_Int128_on_a_table = r""" +a +1 +""" + +untuple_with_Int128_on_a_table = r""" +a +1 +""" + +tupleHammingDistance_with_Int128_on_a_table = r""" +a +2 +""" + +Creating_a_map_with_Int128_on_a_table = r""" +a +{'key1':1,'key2':2} +""" + +mapAdd_with_Int128_on_a_table = r""" +a +""" + +mapSubtract_with_Int128_on_a_table = r""" +a +""" + +mapPopulateSeries_with_Int128_on_a_table = r""" +a +""" + +mapContains_with_Int128_on_a_table = r""" +a +1 +""" + +mapKeys_with_Int128_on_a_table = r""" +a +['key1','key2'] +""" + +mapValues_with_Int128_on_a_table = r""" +a +[1,2] +""" + +Inline___Int256___arrayPopBack_ = r""" +arrayPopBack(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,2] +""" + +Table___Int256___arrayPopBack_ = r""" +a +[3,2] +""" + +Inline___Int256___arrayPopFront_ = r""" +arrayPopFront(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[2,1] +""" + +Table___Int256___arrayPopFront_ = r""" +a +[2,1] +""" + +Inline___Int256___arraySort_ = r""" +arraySort(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1,2,3] +""" + +Table___Int256___arraySort_ = r""" +a +[1,2,3] +""" + +Inline___Int256___arrayReverseSort_ = r""" +arrayReverseSort(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,2,1] +""" + +Table___Int256___arrayReverseSort_ = r""" +a +[3,2,1] +""" + +Inline___Int256___arrayDistinct_ = r""" +arrayDistinct(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,2,1] +""" + +Table___Int256___arrayDistinct_ = r""" +a +[3,2,1] +""" + +Inline___Int256___arrayEnumerate_ = r""" +arrayEnumerate(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1,2,3] +""" + +Table___Int256___arrayEnumerate_ = r""" +a +[1,2,3] +""" + +Inline___Int256___arrayEnumerateDense_ = r""" +arrayEnumerateDense(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1,2,3] +""" + +Table___Int256___arrayEnumerateDense_ = r""" +a +[1,2,3] +""" + +Inline___Int256___arrayEnumerateUniq_ = r""" +arrayEnumerateUniq(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1,1,1] +""" + +Table___Int256___arrayEnumerateUniq_ = r""" +a +[1,1,1] +""" + +Inline___Int256___arrayReverse_ = r""" +arrayReverse(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1,2,3] +""" + +Table___Int256___arrayReverse_ = r""" +a +[1,2,3] +""" + +Inline___Int256___reverse_ = r""" +reverse(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1,2,3] +""" + +Table___Int256___reverse_ = r""" +a +[1,2,3] +""" + +Inline___Int256___arrayFlatten_ = r""" +arrayFlatten(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,2,1] +""" + +Table___Int256___arrayFlatten_ = r""" +a +[3,2,1] +""" + +Inline___Int256___arrayCompact_ = r""" +arrayCompact(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,2,1] +""" + +Table___Int256___arrayCompact_ = r""" +a +[3,2,1] +""" + +Inline___Int256___arrayReduceInRanges__sum_____1__5___ = r""" +arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[6] +""" + +Table___Int256___arrayReduceInRanges__sum_____1__5___ = r""" +a +[6] +""" + +Inline___Int256___arrayMap_x_____x___2__ = r""" +arrayMap(lambda(tuple(x), plus(x, 2)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[5,4,3] +""" + +Table___Int256___arrayMap_x_____x___2__ = r""" +a +[5,4,3] +""" + +Inline___Int256___arrayFill_x____x_3_ = r""" +arrayFill(lambda(tuple(x), equals(x, 3)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,3,3] +""" + +Table___Int256___arrayFill_x____x_3_ = r""" +a +[3,3,3] +""" + +Inline___Int256___arrayReverseFill_x____x_3_ = r""" +arrayReverseFill(lambda(tuple(x), equals(x, 3)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,1,1] +""" + +Table___Int256___arrayReverseFill_x____x_3_ = r""" +a +[3,1,1] +""" + +Inline___Int256___arrayConcat__toInt256__3____toInt256__2____toInt256__1____ = r""" +arrayConcat(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[3,2,1,3,2,1] +""" + +Table___Int256___arrayConcat__toInt256__3____toInt256__2____toInt256__1____ = r""" +a +[3,2,1,3,2,1] +""" + +Inline___Int256___arrayFilter_x____x____1__ = r""" +arrayFilter(lambda(tuple(x), equals(x, 1)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[1] +""" + +Table___Int256___arrayFilter_x____x____1__ = r""" +a +[1] +""" + +Inline___Int256___arraySplit__x__y_____x_y___0__0__0__ = r""" +arraySplit(lambda(tuple(x, y), equals(x, y)), [0, 0, 0], array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +[[0,0,0]] +""" + +Table___Int256___arraySplit__x__y_____x_y___0__0__0__ = r""" +a +[[0,0,0]] +""" + +Inline___Int256___arrayZip__toInt256__1____ = r""" +arrayZip(array(toInt256(\'1\')), array(toInt256(\'3\'))) +[(1,3)] +""" + +Table___Int256___arrayZip__toInt256__1____ = r""" +a +[(1,1)] +""" + +Inline___Int256___empty_ = r""" +empty(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +0 +""" + +Table___Int256___empty_ = r""" +a +0 +""" + +Inline___Int256___notEmpty_ = r""" +notEmpty(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___notEmpty_ = r""" +a +1 +""" + +Inline___Int256___length_ = r""" +length(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +3 +""" + +Table___Int256___length_ = r""" +a +3 +""" + +Inline___Int256___arrayCount_x____x____1__ = r""" +arrayCount(lambda(tuple(x), equals(x, 1)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___arrayCount_x____x____1__ = r""" +a +1 +""" + +Inline___Int256___arrayUniq_ = r""" +arrayUniq(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +3 +""" + +Table___Int256___arrayUniq_ = r""" +a +3 +""" + +Inline___Int256___arrayJoin_ = r""" +arrayJoin(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +3 +2 +1 +""" + +Table___Int256___arrayJoin_ = r""" +a +1 +2 +3 +""" + +Inline___Int256___arrayExists_x____x__1_ = r""" +arrayExists(lambda(tuple(x), equals(x, 1)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___arrayExists_x____x__1_ = r""" +a +1 +""" + +Inline___Int256___arrayAll_x____x__1_ = r""" +arrayAll(lambda(tuple(x), equals(x, 1)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +0 +""" + +Table___Int256___arrayAll_x____x__1_ = r""" +a +0 +""" + +Inline___Int256___arrayMin_ = r""" +arrayMin(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___arrayMin_ = r""" +a +1 +""" + +Inline___Int256___arrayMax_ = r""" +arrayMax(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +3 +""" + +Table___Int256___arrayMax_ = r""" +a +3 +""" + +Inline___Int256___arraySum_ = r""" +arraySum(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +6 +""" + +Table___Int256___arraySum_ = r""" +a +6 +""" + +Inline___Int256___arrayAvg_ = r""" +arrayAvg(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +2 +""" + +Table___Int256___arrayAvg_ = r""" +a +2 +""" + +Inline___Int256___arrayReduce__max___ = r""" +arrayReduce(\'max\', array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +3 +""" + +Table___Int256___arrayReduce__max___ = r""" +a +3 +""" + +Inline___Int256___arrayFirst_x____x__3_ = r""" +arrayFirst(lambda(tuple(x), equals(x, 3)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +3 +""" + +Table___Int256___arrayFirst_x____x__3_ = r""" +a +3 +""" + +Inline___Int256___arrayFirstIndex_x____x__3_ = r""" +arrayFirstIndex(lambda(tuple(x), equals(x, 3)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___arrayFirstIndex_x____x__3_ = r""" +a +1 +""" + +Inline___Int256___hasAll__toInt256__3____toInt256__2____toInt256__1_____ = r""" +hasAll(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___hasAll__toInt256__3____toInt256__2____toInt256__1_____ = r""" +a +1 +""" + +Inline___Int256___hasAny__toInt256__2____toInt256__1_____ = r""" +hasAny(array(toInt256(\'2\'), toInt256(\'1\')), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +1 +""" + +Table___Int256___hasAny__toInt256__2____toInt256__1_____ = r""" +a +1 +""" + +Inline___Int256___hasSubstr__toInt256__2____toInt256__1_____ = r""" +hasSubstr(array(toInt256(\'2\'), toInt256(\'1\')), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\'))) +0 +""" + +Table___Int256___hasSubstr__toInt256__2____toInt256__1_____ = r""" +a +0 +""" + +Table___Int256___arrayDifference_ = r""" +a +""" + +Table___Int256___arrayCumSum_ = r""" +a +""" + +Table___Int256___arrayCumSumNonNegative_ = r""" +a +""" + +Inline___Int256___arrayElement = r""" +arrayElement(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), 1) +3 +""" + +Table___Int256___arrayElement = r""" +a +3 +""" + +Inline___Int256___arrayPushBack = r""" +arrayPushBack(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), toInt256(\'1\')) +[3,2,1,1] +""" + +Table___Int256___arrayPushBack = r""" +a +[3,2,1,1] +""" + +Inline___Int256___arrayPushFront = r""" +arrayPushFront(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), toInt256(\'1\')) +[1,3,2,1] +""" + +Table___Int256___arrayPushFront = r""" +a +[1,3,2,1] +""" + +Inline___Int256___arrayResize = r""" +arrayResize(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), 1) +[3] +""" + +Table___Int256___arrayResize = r""" +a +[3] +""" + +Inline___Int256___arraySlice = r""" +arraySlice(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), 1) +[3,2,1] +""" + +Table___Int256___arraySlice = r""" +a +[3,2,1] +""" + +Inline___Int256___has = r""" +has(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), NULL) +0 +""" + +Table___Int256___has = r""" +a +0 +""" + +Inline___Int256___indexOf = r""" +indexOf(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), NULL) +0 +""" + +Table___Int256___indexOf = r""" +a +0 +""" + +Inline___Int256___countEqual = r""" +countEqual(array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')), NULL) +0 +""" + +Table___Int256___countEqual = r""" +a +0 +""" + +Creating_a_tuple_with_Int256_on_a_table = r""" +a +(1,1,1) +""" + +tupleElement_with_Int256_on_a_table = r""" +a +1 +""" + +untuple_with_Int256_on_a_table = r""" +a +1 +""" + +tupleHammingDistance_with_Int256_on_a_table = r""" +a +2 +""" + +Creating_a_map_with_Int256_on_a_table = r""" +a +{'key1':1,'key2':2} +""" + +mapAdd_with_Int256_on_a_table = r""" +a +""" + +mapSubtract_with_Int256_on_a_table = r""" +a +""" + +mapPopulateSeries_with_Int256_on_a_table = r""" +a +""" + +mapContains_with_Int256_on_a_table = r""" +a +1 +""" + +mapKeys_with_Int256_on_a_table = r""" +a +['key1','key2'] +""" + +mapValues_with_Int256_on_a_table = r""" +a +[1,2] +""" + +Inline___UInt128___arrayPopBack_ = r""" +arrayPopBack(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,2] +""" + +Table___UInt128___arrayPopBack_ = r""" +a +[3,2] +""" + +Inline___UInt128___arrayPopFront_ = r""" +arrayPopFront(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[2,1] +""" + +Table___UInt128___arrayPopFront_ = r""" +a +[2,1] +""" + +Inline___UInt128___arraySort_ = r""" +arraySort(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1,2,3] +""" + +Table___UInt128___arraySort_ = r""" +a +[1,2,3] +""" + +Inline___UInt128___arrayReverseSort_ = r""" +arrayReverseSort(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,2,1] +""" + +Table___UInt128___arrayReverseSort_ = r""" +a +[3,2,1] +""" + +Inline___UInt128___arrayDistinct_ = r""" +arrayDistinct(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,2,1] +""" + +Table___UInt128___arrayDistinct_ = r""" +a +[3,2,1] +""" + +Inline___UInt128___arrayEnumerate_ = r""" +arrayEnumerate(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1,2,3] +""" + +Table___UInt128___arrayEnumerate_ = r""" +a +[1,2,3] +""" + +Inline___UInt128___arrayEnumerateDense_ = r""" +arrayEnumerateDense(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1,2,3] +""" + +Table___UInt128___arrayEnumerateDense_ = r""" +a +[1,2,3] +""" + +Inline___UInt128___arrayEnumerateUniq_ = r""" +arrayEnumerateUniq(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1,1,1] +""" + +Table___UInt128___arrayEnumerateUniq_ = r""" +a +[1,1,1] +""" + +Inline___UInt128___arrayReverse_ = r""" +arrayReverse(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1,2,3] +""" + +Table___UInt128___arrayReverse_ = r""" +a +[1,2,3] +""" + +Inline___UInt128___reverse_ = r""" +reverse(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1,2,3] +""" + +Table___UInt128___reverse_ = r""" +a +[1,2,3] +""" + +Inline___UInt128___arrayFlatten_ = r""" +arrayFlatten(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,2,1] +""" + +Table___UInt128___arrayFlatten_ = r""" +a +[3,2,1] +""" + +Inline___UInt128___arrayCompact_ = r""" +arrayCompact(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,2,1] +""" + +Table___UInt128___arrayCompact_ = r""" +a +[3,2,1] +""" + +Inline___UInt128___arrayReduceInRanges__sum_____1__5___ = r""" +arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[6] +""" + +Table___UInt128___arrayReduceInRanges__sum_____1__5___ = r""" +a +[6] +""" + +Inline___UInt128___arrayMap_x_____x___2__ = r""" +arrayMap(lambda(tuple(x), plus(x, 2)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[5,4,3] +""" + +Table___UInt128___arrayMap_x_____x___2__ = r""" +a +[5,4,3] +""" + +Inline___UInt128___arrayFill_x____x_3_ = r""" +arrayFill(lambda(tuple(x), equals(x, 3)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,3,3] +""" + +Table___UInt128___arrayFill_x____x_3_ = r""" +a +[3,3,3] +""" + +Inline___UInt128___arrayReverseFill_x____x_3_ = r""" +arrayReverseFill(lambda(tuple(x), equals(x, 3)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,1,1] +""" + +Table___UInt128___arrayReverseFill_x____x_3_ = r""" +a +[3,1,1] +""" + +Inline___UInt128___arrayConcat__toUInt128__3____toUInt128__2____toUInt128__1____ = r""" +arrayConcat(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[3,2,1,3,2,1] +""" + +Table___UInt128___arrayConcat__toUInt128__3____toUInt128__2____toUInt128__1____ = r""" +a +[3,2,1,3,2,1] +""" + +Inline___UInt128___arrayFilter_x____x____1__ = r""" +arrayFilter(lambda(tuple(x), equals(x, 1)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[1] +""" + +Table___UInt128___arrayFilter_x____x____1__ = r""" +a +[1] +""" + +Inline___UInt128___arraySplit__x__y_____x_y___0__0__0__ = r""" +arraySplit(lambda(tuple(x, y), equals(x, y)), [0, 0, 0], array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +[[0,0,0]] +""" + +Table___UInt128___arraySplit__x__y_____x_y___0__0__0__ = r""" +a +[[0,0,0]] +""" + +Inline___UInt128___arrayZip__toUInt128__1____ = r""" +arrayZip(array(toUInt128(\'1\')), array(toUInt128(\'3\'))) +[(1,3)] +""" + +Table___UInt128___arrayZip__toUInt128__1____ = r""" +a +[(1,1)] +""" + +Inline___UInt128___empty_ = r""" +empty(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +0 +""" + +Table___UInt128___empty_ = r""" +a +0 +""" + +Inline___UInt128___notEmpty_ = r""" +notEmpty(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___notEmpty_ = r""" +a +1 +""" + +Inline___UInt128___length_ = r""" +length(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +3 +""" + +Table___UInt128___length_ = r""" +a +3 +""" + +Inline___UInt128___arrayCount_x____x____1__ = r""" +arrayCount(lambda(tuple(x), equals(x, 1)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___arrayCount_x____x____1__ = r""" +a +1 +""" + +Inline___UInt128___arrayUniq_ = r""" +arrayUniq(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +3 +""" + +Table___UInt128___arrayUniq_ = r""" +a +3 +""" + +Inline___UInt128___arrayJoin_ = r""" +arrayJoin(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +3 +2 +1 +""" + +Table___UInt128___arrayJoin_ = r""" +a +1 +2 +3 +""" + +Inline___UInt128___arrayExists_x____x__1_ = r""" +arrayExists(lambda(tuple(x), equals(x, 1)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___arrayExists_x____x__1_ = r""" +a +1 +""" + +Inline___UInt128___arrayAll_x____x__1_ = r""" +arrayAll(lambda(tuple(x), equals(x, 1)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +0 +""" + +Table___UInt128___arrayAll_x____x__1_ = r""" +a +0 +""" + +Inline___UInt128___arrayMin_ = r""" +arrayMin(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___arrayMin_ = r""" +a +1 +""" + +Inline___UInt128___arrayMax_ = r""" +arrayMax(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +3 +""" + +Table___UInt128___arrayMax_ = r""" +a +3 +""" + +Inline___UInt128___arraySum_ = r""" +arraySum(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +6 +""" + +Table___UInt128___arraySum_ = r""" +a +6 +""" + +Inline___UInt128___arrayAvg_ = r""" +arrayAvg(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +2 +""" + +Table___UInt128___arrayAvg_ = r""" +a +2 +""" + +Inline___UInt128___arrayReduce__max___ = r""" +arrayReduce(\'max\', array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +3 +""" + +Table___UInt128___arrayReduce__max___ = r""" +a +3 +""" + +Inline___UInt128___arrayFirst_x____x__3_ = r""" +arrayFirst(lambda(tuple(x), equals(x, 3)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +3 +""" + +Table___UInt128___arrayFirst_x____x__3_ = r""" +a +3 +""" + +Inline___UInt128___arrayFirstIndex_x____x__3_ = r""" +arrayFirstIndex(lambda(tuple(x), equals(x, 3)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___arrayFirstIndex_x____x__3_ = r""" +a +1 +""" + +Inline___UInt128___hasAll__toUInt128__3____toUInt128__2____toUInt128__1_____ = r""" +hasAll(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___hasAll__toUInt128__3____toUInt128__2____toUInt128__1_____ = r""" +a +1 +""" + +Inline___UInt128___hasAny__toUInt128__2____toUInt128__1_____ = r""" +hasAny(array(toUInt128(\'2\'), toUInt128(\'1\')), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +1 +""" + +Table___UInt128___hasAny__toUInt128__2____toUInt128__1_____ = r""" +a +1 +""" + +Inline___UInt128___hasSubstr__toUInt128__2____toUInt128__1_____ = r""" +hasSubstr(array(toUInt128(\'2\'), toUInt128(\'1\')), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\'))) +0 +""" + +Table___UInt128___hasSubstr__toUInt128__2____toUInt128__1_____ = r""" +a +0 +""" + +Table___UInt128___arrayDifference_ = r""" +a +""" + +Table___UInt128___arrayCumSum_ = r""" +a +""" + +Table___UInt128___arrayCumSumNonNegative_ = r""" +a +""" + +Inline___UInt128___arrayElement = r""" +arrayElement(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), 1) +3 +""" + +Table___UInt128___arrayElement = r""" +a +3 +""" + +Inline___UInt128___arrayPushBack = r""" +arrayPushBack(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), toUInt128(\'1\')) +[3,2,1,1] +""" + +Table___UInt128___arrayPushBack = r""" +a +[3,2,1,1] +""" + +Inline___UInt128___arrayPushFront = r""" +arrayPushFront(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), toUInt128(\'1\')) +[1,3,2,1] +""" + +Table___UInt128___arrayPushFront = r""" +a +[1,3,2,1] +""" + +Inline___UInt128___arrayResize = r""" +arrayResize(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), 1) +[3] +""" + +Table___UInt128___arrayResize = r""" +a +[3] +""" + +Inline___UInt128___arraySlice = r""" +arraySlice(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), 1) +[3,2,1] +""" + +Table___UInt128___arraySlice = r""" +a +[3,2,1] +""" + +Inline___UInt128___has = r""" +has(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), NULL) +0 +""" + +Table___UInt128___has = r""" +a +0 +""" + +Inline___UInt128___indexOf = r""" +indexOf(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), NULL) +0 +""" + +Table___UInt128___indexOf = r""" +a +0 +""" + +Inline___UInt128___countEqual = r""" +countEqual(array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')), NULL) +0 +""" + +Table___UInt128___countEqual = r""" +a +0 +""" + +Creating_a_tuple_with_UInt128_on_a_table = r""" +a +(1,1,1) +""" + +tupleElement_with_UInt128_on_a_table = r""" +a +1 +""" + +untuple_with_UInt128_on_a_table = r""" +a +1 +""" + +tupleHammingDistance_with_UInt128_on_a_table = r""" +a +2 +""" + +Creating_a_map_with_UInt128_on_a_table = r""" +a +{'key1':1,'key2':2} +""" + +mapAdd_with_UInt128_on_a_table = r""" +a +""" + +mapSubtract_with_UInt128_on_a_table = r""" +a +""" + +mapPopulateSeries_with_UInt128_on_a_table = r""" +a +""" + +mapContains_with_UInt128_on_a_table = r""" +a +1 +""" + +mapKeys_with_UInt128_on_a_table = r""" +a +['key1','key2'] +""" + +mapValues_with_UInt128_on_a_table = r""" +a +[1,2] +""" + +Inline___UInt256___arrayPopBack_ = r""" +arrayPopBack(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,2] +""" + +Table___UInt256___arrayPopBack_ = r""" +a +[3,2] +""" + +Inline___UInt256___arrayPopFront_ = r""" +arrayPopFront(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[2,1] +""" + +Table___UInt256___arrayPopFront_ = r""" +a +[2,1] +""" + +Inline___UInt256___arraySort_ = r""" +arraySort(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1,2,3] +""" + +Table___UInt256___arraySort_ = r""" +a +[1,2,3] +""" + +Inline___UInt256___arrayReverseSort_ = r""" +arrayReverseSort(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,2,1] +""" + +Table___UInt256___arrayReverseSort_ = r""" +a +[3,2,1] +""" + +Inline___UInt256___arrayDistinct_ = r""" +arrayDistinct(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,2,1] +""" + +Table___UInt256___arrayDistinct_ = r""" +a +[3,2,1] +""" + +Inline___UInt256___arrayEnumerate_ = r""" +arrayEnumerate(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1,2,3] +""" + +Table___UInt256___arrayEnumerate_ = r""" +a +[1,2,3] +""" + +Inline___UInt256___arrayEnumerateDense_ = r""" +arrayEnumerateDense(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1,2,3] +""" + +Table___UInt256___arrayEnumerateDense_ = r""" +a +[1,2,3] +""" + +Inline___UInt256___arrayEnumerateUniq_ = r""" +arrayEnumerateUniq(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1,1,1] +""" + +Table___UInt256___arrayEnumerateUniq_ = r""" +a +[1,1,1] +""" + +Inline___UInt256___arrayReverse_ = r""" +arrayReverse(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1,2,3] +""" + +Table___UInt256___arrayReverse_ = r""" +a +[1,2,3] +""" + +Inline___UInt256___reverse_ = r""" +reverse(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1,2,3] +""" + +Table___UInt256___reverse_ = r""" +a +[1,2,3] +""" + +Inline___UInt256___arrayFlatten_ = r""" +arrayFlatten(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,2,1] +""" + +Table___UInt256___arrayFlatten_ = r""" +a +[3,2,1] +""" + +Inline___UInt256___arrayCompact_ = r""" +arrayCompact(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,2,1] +""" + +Table___UInt256___arrayCompact_ = r""" +a +[3,2,1] +""" + +Inline___UInt256___arrayReduceInRanges__sum_____1__5___ = r""" +arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[6] +""" + +Table___UInt256___arrayReduceInRanges__sum_____1__5___ = r""" +a +[6] +""" + +Inline___UInt256___arrayMap_x_____x___2__ = r""" +arrayMap(lambda(tuple(x), plus(x, 2)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[5,4,3] +""" + +Table___UInt256___arrayMap_x_____x___2__ = r""" +a +[5,4,3] +""" + +Inline___UInt256___arrayFill_x____x_3_ = r""" +arrayFill(lambda(tuple(x), equals(x, 3)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,3,3] +""" + +Table___UInt256___arrayFill_x____x_3_ = r""" +a +[3,3,3] +""" + +Inline___UInt256___arrayReverseFill_x____x_3_ = r""" +arrayReverseFill(lambda(tuple(x), equals(x, 3)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,1,1] +""" + +Table___UInt256___arrayReverseFill_x____x_3_ = r""" +a +[3,1,1] +""" + +Inline___UInt256___arrayConcat__toUInt256__3____toUInt256__2____toUInt256__1____ = r""" +arrayConcat(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[3,2,1,3,2,1] +""" + +Table___UInt256___arrayConcat__toUInt256__3____toUInt256__2____toUInt256__1____ = r""" +a +[3,2,1,3,2,1] +""" + +Inline___UInt256___arrayFilter_x____x____1__ = r""" +arrayFilter(lambda(tuple(x), equals(x, 1)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[1] +""" + +Table___UInt256___arrayFilter_x____x____1__ = r""" +a +[1] +""" + +Inline___UInt256___arraySplit__x__y_____x_y___0__0__0__ = r""" +arraySplit(lambda(tuple(x, y), equals(x, y)), [0, 0, 0], array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +[[0,0,0]] +""" + +Table___UInt256___arraySplit__x__y_____x_y___0__0__0__ = r""" +a +[[0,0,0]] +""" + +Inline___UInt256___arrayZip__toUInt256__1____ = r""" +arrayZip(array(toUInt256(\'1\')), array(toUInt256(\'3\'))) +[(1,3)] +""" + +Table___UInt256___arrayZip__toUInt256__1____ = r""" +a +[(1,1)] +""" + +Inline___UInt256___empty_ = r""" +empty(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +0 +""" + +Table___UInt256___empty_ = r""" +a +0 +""" + +Inline___UInt256___notEmpty_ = r""" +notEmpty(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___notEmpty_ = r""" +a +1 +""" + +Inline___UInt256___length_ = r""" +length(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +3 +""" + +Table___UInt256___length_ = r""" +a +3 +""" + +Inline___UInt256___arrayCount_x____x____1__ = r""" +arrayCount(lambda(tuple(x), equals(x, 1)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___arrayCount_x____x____1__ = r""" +a +1 +""" + +Inline___UInt256___arrayUniq_ = r""" +arrayUniq(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +3 +""" + +Table___UInt256___arrayUniq_ = r""" +a +3 +""" + +Inline___UInt256___arrayJoin_ = r""" +arrayJoin(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +3 +2 +1 +""" + +Table___UInt256___arrayJoin_ = r""" +a +1 +2 +3 +""" + +Inline___UInt256___arrayExists_x____x__1_ = r""" +arrayExists(lambda(tuple(x), equals(x, 1)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___arrayExists_x____x__1_ = r""" +a +1 +""" + +Inline___UInt256___arrayAll_x____x__1_ = r""" +arrayAll(lambda(tuple(x), equals(x, 1)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +0 +""" + +Table___UInt256___arrayAll_x____x__1_ = r""" +a +0 +""" + +Inline___UInt256___arrayMin_ = r""" +arrayMin(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___arrayMin_ = r""" +a +1 +""" + +Inline___UInt256___arrayMax_ = r""" +arrayMax(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +3 +""" + +Table___UInt256___arrayMax_ = r""" +a +3 +""" + +Inline___UInt256___arraySum_ = r""" +arraySum(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +6 +""" + +Table___UInt256___arraySum_ = r""" +a +6 +""" + +Inline___UInt256___arrayAvg_ = r""" +arrayAvg(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +2 +""" + +Table___UInt256___arrayAvg_ = r""" +a +2 +""" + +Inline___UInt256___arrayReduce__max___ = r""" +arrayReduce(\'max\', array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +3 +""" + +Table___UInt256___arrayReduce__max___ = r""" +a +3 +""" + +Inline___UInt256___arrayFirst_x____x__3_ = r""" +arrayFirst(lambda(tuple(x), equals(x, 3)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +3 +""" + +Table___UInt256___arrayFirst_x____x__3_ = r""" +a +3 +""" + +Inline___UInt256___arrayFirstIndex_x____x__3_ = r""" +arrayFirstIndex(lambda(tuple(x), equals(x, 3)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___arrayFirstIndex_x____x__3_ = r""" +a +1 +""" + +Inline___UInt256___hasAll__toUInt256__3____toUInt256__2____toUInt256__1_____ = r""" +hasAll(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___hasAll__toUInt256__3____toUInt256__2____toUInt256__1_____ = r""" +a +1 +""" + +Inline___UInt256___hasAny__toUInt256__2____toUInt256__1_____ = r""" +hasAny(array(toUInt256(\'2\'), toUInt256(\'1\')), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +1 +""" + +Table___UInt256___hasAny__toUInt256__2____toUInt256__1_____ = r""" +a +1 +""" + +Inline___UInt256___hasSubstr__toUInt256__2____toUInt256__1_____ = r""" +hasSubstr(array(toUInt256(\'2\'), toUInt256(\'1\')), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\'))) +0 +""" + +Table___UInt256___hasSubstr__toUInt256__2____toUInt256__1_____ = r""" +a +0 +""" + +Table___UInt256___arrayDifference_ = r""" +a +""" + +Table___UInt256___arrayCumSum_ = r""" +a +""" + +Table___UInt256___arrayCumSumNonNegative_ = r""" +a +""" + +Inline___UInt256___arrayElement = r""" +arrayElement(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), 1) +3 +""" + +Table___UInt256___arrayElement = r""" +a +3 +""" + +Inline___UInt256___arrayPushBack = r""" +arrayPushBack(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), toUInt256(\'1\')) +[3,2,1,1] +""" + +Table___UInt256___arrayPushBack = r""" +a +[3,2,1,1] +""" + +Inline___UInt256___arrayPushFront = r""" +arrayPushFront(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), toUInt256(\'1\')) +[1,3,2,1] +""" + +Table___UInt256___arrayPushFront = r""" +a +[1,3,2,1] +""" + +Inline___UInt256___arrayResize = r""" +arrayResize(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), 1) +[3] +""" + +Table___UInt256___arrayResize = r""" +a +[3] +""" + +Inline___UInt256___arraySlice = r""" +arraySlice(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), 1) +[3,2,1] +""" + +Table___UInt256___arraySlice = r""" +a +[3,2,1] +""" + +Inline___UInt256___has = r""" +has(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), NULL) +0 +""" + +Table___UInt256___has = r""" +a +0 +""" + +Inline___UInt256___indexOf = r""" +indexOf(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), NULL) +0 +""" + +Table___UInt256___indexOf = r""" +a +0 +""" + +Inline___UInt256___countEqual = r""" +countEqual(array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')), NULL) +0 +""" + +Table___UInt256___countEqual = r""" +a +0 +""" + +Creating_a_tuple_with_UInt256_on_a_table = r""" +a +(1,1,1) +""" + +tupleElement_with_UInt256_on_a_table = r""" +a +1 +""" + +untuple_with_UInt256_on_a_table = r""" +a +1 +""" + +tupleHammingDistance_with_UInt256_on_a_table = r""" +a +2 +""" + +Creating_a_map_with_UInt256_on_a_table = r""" +a +{'key1':1,'key2':2} +""" + +mapAdd_with_UInt256_on_a_table = r""" +a +""" + +mapSubtract_with_UInt256_on_a_table = r""" +a +""" + +mapPopulateSeries_with_UInt256_on_a_table = r""" +a +""" + +mapContains_with_UInt256_on_a_table = r""" +a +1 +""" + +mapKeys_with_UInt256_on_a_table = r""" +a +['key1','key2'] +""" + +mapValues_with_UInt256_on_a_table = r""" +a +[1,2] +""" + +Inline___Decimal256_0____arrayPopBack_ = r""" +arrayPopBack(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,2] +""" + +Table___Decimal256_0____arrayPopBack_ = r""" +a +[3,2] +""" + +Inline___Decimal256_0____arrayPopFront_ = r""" +arrayPopFront(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[2,1] +""" + +Table___Decimal256_0____arrayPopFront_ = r""" +a +[2,1] +""" + +Inline___Decimal256_0____arraySort_ = r""" +arraySort(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1,2,3] +""" + +Table___Decimal256_0____arraySort_ = r""" +a +[1,2,3] +""" + +Inline___Decimal256_0____arrayReverseSort_ = r""" +arrayReverseSort(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,2,1] +""" + +Table___Decimal256_0____arrayReverseSort_ = r""" +a +[3,2,1] +""" + +Inline___Decimal256_0____arrayDistinct_ = r""" +arrayDistinct(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,2,1] +""" + +Table___Decimal256_0____arrayDistinct_ = r""" +a +[3,2,1] +""" + +Inline___Decimal256_0____arrayEnumerate_ = r""" +arrayEnumerate(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1,2,3] +""" + +Table___Decimal256_0____arrayEnumerate_ = r""" +a +[1,2,3] +""" + +Inline___Decimal256_0____arrayEnumerateDense_ = r""" +arrayEnumerateDense(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1,2,3] +""" + +Table___Decimal256_0____arrayEnumerateDense_ = r""" +a +[1,2,3] +""" + +Inline___Decimal256_0____arrayEnumerateUniq_ = r""" +arrayEnumerateUniq(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1,1,1] +""" + +Table___Decimal256_0____arrayEnumerateUniq_ = r""" +a +[1,1,1] +""" + +Inline___Decimal256_0____arrayReverse_ = r""" +arrayReverse(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1,2,3] +""" + +Table___Decimal256_0____arrayReverse_ = r""" +a +[1,2,3] +""" + +Inline___Decimal256_0____reverse_ = r""" +reverse(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1,2,3] +""" + +Table___Decimal256_0____reverse_ = r""" +a +[1,2,3] +""" + +Inline___Decimal256_0____arrayFlatten_ = r""" +arrayFlatten(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,2,1] +""" + +Table___Decimal256_0____arrayFlatten_ = r""" +a +[3,2,1] +""" + +Inline___Decimal256_0____arrayCompact_ = r""" +arrayCompact(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,2,1] +""" + +Table___Decimal256_0____arrayCompact_ = r""" +a +[3,2,1] +""" + +Inline___Decimal256_0____arrayReduceInRanges__sum_____1__5___ = r""" +arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[6] +""" + +Table___Decimal256_0____arrayReduceInRanges__sum_____1__5___ = r""" +a +[6] +""" + +Inline___Decimal256_0____arrayMap_x_____x___2__ = r""" +arrayMap(lambda(tuple(x), plus(x, 2)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[5,4,3] +""" + +Table___Decimal256_0____arrayMap_x_____x___2__ = r""" +a +[5,4,3] +""" + +Inline___Decimal256_0____arrayFill_x____x_3_ = r""" +arrayFill(lambda(tuple(x), equals(x, 3)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,3,3] +""" + +Table___Decimal256_0____arrayFill_x____x_3_ = r""" +a +[3,3,3] +""" + +Inline___Decimal256_0____arrayReverseFill_x____x_3_ = r""" +arrayReverseFill(lambda(tuple(x), equals(x, 3)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,1,1] +""" + +Table___Decimal256_0____arrayReverseFill_x____x_3_ = r""" +a +[3,1,1] +""" + +Inline___Decimal256_0____arrayConcat__toDecimal256__3__0___toDecimal256__2__0___toDecimal256__1__0___ = r""" +arrayConcat(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[3,2,1,3,2,1] +""" + +Table___Decimal256_0____arrayConcat__toDecimal256__3__0___toDecimal256__2__0___toDecimal256__1__0___ = r""" +a +[3,2,1,3,2,1] +""" + +Inline___Decimal256_0____arrayFilter_x____x____1__ = r""" +arrayFilter(lambda(tuple(x), equals(x, 1)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[1] +""" + +Table___Decimal256_0____arrayFilter_x____x____1__ = r""" +a +[1] +""" + +Inline___Decimal256_0____arraySplit__x__y_____x_y___0__0__0__ = r""" +arraySplit(lambda(tuple(x, y), equals(x, y)), [0, 0, 0], array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +[[0,0,0]] +""" + +Table___Decimal256_0____arraySplit__x__y_____x_y___0__0__0__ = r""" +a +[[0,0,0]] +""" + +Inline___Decimal256_0____arrayZip__toDecimal256__1__0___ = r""" +arrayZip(array(toDecimal256(\'1\', 0)), array(toDecimal256(\'3\', 0))) +[(1,3)] +""" + +Table___Decimal256_0____arrayZip__toDecimal256__1__0___ = r""" +a +[(1,1)] +""" + +Inline___Decimal256_0____empty_ = r""" +empty(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +0 +""" + +Table___Decimal256_0____empty_ = r""" +a +0 +""" + +Inline___Decimal256_0____notEmpty_ = r""" +notEmpty(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +1 +""" + +Table___Decimal256_0____notEmpty_ = r""" +a +1 +""" + +Inline___Decimal256_0____length_ = r""" +length(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +3 +""" + +Table___Decimal256_0____length_ = r""" +a +3 +""" + +Inline___Decimal256_0____arrayCount_x____x____1__ = r""" +arrayCount(lambda(tuple(x), equals(x, 1)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +1 +""" + +Table___Decimal256_0____arrayCount_x____x____1__ = r""" +a +1 +""" + +Inline___Decimal256_0____arrayUniq_ = r""" +arrayUniq(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +3 +""" + +Table___Decimal256_0____arrayUniq_ = r""" +a +3 +""" + +Inline___Decimal256_0____arrayJoin_ = r""" +arrayJoin(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +3 +2 +1 +""" + +Table___Decimal256_0____arrayJoin_ = r""" +a +1 +2 +3 +""" + +Inline___Decimal256_0____arrayExists_x____x__1_ = r""" +arrayExists(lambda(tuple(x), equals(x, 1)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +1 +""" + +Table___Decimal256_0____arrayExists_x____x__1_ = r""" +a +1 +""" + +Inline___Decimal256_0____arrayAll_x____x__1_ = r""" +arrayAll(lambda(tuple(x), equals(x, 1)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +0 +""" + +Table___Decimal256_0____arrayAll_x____x__1_ = r""" +a +0 +""" + +Table___Decimal256_0____arrayMin_ = r""" +a +""" + +Table___Decimal256_0____arrayMax_ = r""" +a +""" + +Table___Decimal256_0____arraySum_ = r""" +a +""" + +Table___Decimal256_0____arrayAvg_ = r""" +a +""" + +Inline___Decimal256_0____arrayReduce__max___ = r""" +arrayReduce(\'max\', array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +3 +""" + +Table___Decimal256_0____arrayReduce__max___ = r""" +a +3 +""" + +Inline___Decimal256_0____arrayFirst_x____x__3_ = r""" +arrayFirst(lambda(tuple(x), equals(x, 3)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +3 +""" + +Table___Decimal256_0____arrayFirst_x____x__3_ = r""" +a +3 +""" + +Inline___Decimal256_0____arrayFirstIndex_x____x__3_ = r""" +arrayFirstIndex(lambda(tuple(x), equals(x, 3)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +1 +""" + +Table___Decimal256_0____arrayFirstIndex_x____x__3_ = r""" +a +1 +""" + +Inline___Decimal256_0____hasAll__toDecimal256__3__0___toDecimal256__2__0___toDecimal256__1__0____ = r""" +hasAll(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +1 +""" + +Table___Decimal256_0____hasAll__toDecimal256__3__0___toDecimal256__2__0___toDecimal256__1__0____ = r""" +a +1 +""" + +Inline___Decimal256_0____hasAny__toDecimal256__2__0___toDecimal256__1__0____ = r""" +hasAny(array(toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +1 +""" + +Table___Decimal256_0____hasAny__toDecimal256__2__0___toDecimal256__1__0____ = r""" +a +1 +""" + +Inline___Decimal256_0____hasSubstr__toDecimal256__2__0___toDecimal256__1__0____ = r""" +hasSubstr(array(toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0))) +0 +""" + +Table___Decimal256_0____hasSubstr__toDecimal256__2__0___toDecimal256__1__0____ = r""" +a +0 +""" + +Table___Decimal256_0____arrayDifference_ = r""" +a +""" + +Table___Decimal256_0____arrayCumSum_ = r""" +a +""" + +Table___Decimal256_0____arrayCumSumNonNegative_ = r""" +a +""" + +Inline___Decimal256_0____arrayElement = r""" +arrayElement(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), 1) +3 +""" + +Table___Decimal256_0____arrayElement = r""" +a +3 +""" + +Inline___Decimal256_0____arrayPushBack = r""" +arrayPushBack(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), toDecimal256(\'1\', 0)) +[3,2,1,1] +""" + +Table___Decimal256_0____arrayPushBack = r""" +a +[3,2,1,1] +""" + +Inline___Decimal256_0____arrayPushFront = r""" +arrayPushFront(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), toDecimal256(\'1\', 0)) +[1,3,2,1] +""" + +Table___Decimal256_0____arrayPushFront = r""" +a +[1,3,2,1] +""" + +Inline___Decimal256_0____arrayResize = r""" +arrayResize(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), 1) +[3] +""" + +Table___Decimal256_0____arrayResize = r""" +a +[3] +""" + +Inline___Decimal256_0____arraySlice = r""" +arraySlice(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), 1) +[3,2,1] +""" + +Table___Decimal256_0____arraySlice = r""" +a +[3,2,1] +""" + +Inline___Decimal256_0____has = r""" +has(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), NULL) +0 +""" + +Table___Decimal256_0____has = r""" +a +0 +""" + +Inline___Decimal256_0____indexOf = r""" +indexOf(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), NULL) +0 +""" + +Table___Decimal256_0____indexOf = r""" +a +0 +""" + +Inline___Decimal256_0____countEqual = r""" +countEqual(array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)), NULL) +0 +""" + +Table___Decimal256_0____countEqual = r""" +a +0 +""" + +Creating_a_tuple_with_Decimal256_0__on_a_table = r""" +a +(1,1,1) +""" + +tupleElement_with_Decimal256_0__on_a_table = r""" +a +1 +""" + +untuple_with_Decimal256_0__on_a_table = r""" +a +1 +""" + +tupleHammingDistance_with_Decimal256_0__on_a_table = r""" +a +2 +""" + +Creating_a_map_with_Decimal256_0__on_a_table = r""" +a +{'key1':1,'key2':2} +""" + +mapAdd_with_Decimal256_0__on_a_table = r""" +a +""" + +mapSubtract_with_Decimal256_0__on_a_table = r""" +a +""" + +mapPopulateSeries_with_Decimal256_0__on_a_table = r""" +a +""" + +mapContains_with_Decimal256_0__on_a_table = r""" +a +1 +""" + +mapKeys_with_Decimal256_0__on_a_table = r""" +a +['key1','key2'] +""" + +mapValues_with_Decimal256_0__on_a_table = r""" +a +[1,2] +""" + +I_check_equals_with_Int128 = r""" +equals(toInt128(1), toInt128(1)) equals(toInt128(\'170141183460469231731687303715884105727\'), toInt128(\'-170141183460469231731687303715884105728\')) +1 0 +""" + +I_check_equals_with_Int256 = r""" +equals(toInt256(1), toInt256(1)) equals(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 0 +""" + +I_check_equals_with_UInt128 = r""" +equals(toUInt128(1), toUInt128(1)) equals(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(\'0\')) +1 0 +""" + +I_check_equals_with_UInt256 = r""" +equals(toUInt256(1), toUInt256(1)) equals(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(\'0\')) +1 0 +""" + +I_check_notEquals_with_Int128 = r""" +notEquals(toInt128(1), toInt128(1)) notEquals(toInt128(\'170141183460469231731687303715884105727\'), toInt128(\'-170141183460469231731687303715884105728\')) +0 1 +""" + +I_check_notEquals_with_Int256 = r""" +notEquals(toInt256(1), toInt256(1)) notEquals(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +0 1 +""" + +I_check_notEquals_with_UInt128 = r""" +notEquals(toUInt128(1), toUInt128(1)) notEquals(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(\'0\')) +0 1 +""" + +I_check_notEquals_with_UInt256 = r""" +notEquals(toUInt256(1), toUInt256(1)) notEquals(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(\'0\')) +0 1 +""" + +I_check_less_with_Int128 = r""" +less(toInt128(1), toInt128(1)) less(toInt128(\'170141183460469231731687303715884105727\'), toInt128(\'-170141183460469231731687303715884105728\')) +0 0 +""" + +I_check_less_with_Int256 = r""" +less(toInt256(1), toInt256(1)) less(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +0 0 +""" + +I_check_less_with_UInt128 = r""" +less(toUInt128(1), toUInt128(1)) less(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(\'0\')) +0 0 +""" + +I_check_less_with_UInt256 = r""" +less(toUInt256(1), toUInt256(1)) less(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(\'0\')) +0 0 +""" + +I_check_greater_with_Int128 = r""" +greater(toInt128(1), toInt128(1)) greater(toInt128(\'170141183460469231731687303715884105727\'), toInt128(\'-170141183460469231731687303715884105728\')) +0 1 +""" + +I_check_greater_with_Int256 = r""" +greater(toInt256(1), toInt256(1)) greater(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +0 1 +""" + +I_check_greater_with_UInt128 = r""" +greater(toUInt128(1), toUInt128(1)) greater(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(\'0\')) +0 1 +""" + +I_check_greater_with_UInt256 = r""" +greater(toUInt256(1), toUInt256(1)) greater(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(\'0\')) +0 1 +""" + +I_check_lessOrEquals_with_Int128 = r""" +lessOrEquals(toInt128(1), toInt128(1)) lessOrEquals(toInt128(\'170141183460469231731687303715884105727\'), toInt128(\'-170141183460469231731687303715884105728\')) +1 0 +""" + +I_check_lessOrEquals_with_Int256 = r""" +lessOrEquals(toInt256(1), toInt256(1)) lessOrEquals(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 0 +""" + +I_check_lessOrEquals_with_UInt128 = r""" +lessOrEquals(toUInt128(1), toUInt128(1)) lessOrEquals(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(\'0\')) +1 0 +""" + +I_check_lessOrEquals_with_UInt256 = r""" +lessOrEquals(toUInt256(1), toUInt256(1)) lessOrEquals(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(\'0\')) +1 0 +""" + +I_check_greaterOrEquals_with_Int128 = r""" +greaterOrEquals(toInt128(1), toInt128(1)) greaterOrEquals(toInt128(\'170141183460469231731687303715884105727\'), toInt128(\'-170141183460469231731687303715884105728\')) +1 1 +""" + +I_check_greaterOrEquals_with_Int256 = r""" +greaterOrEquals(toInt256(1), toInt256(1)) greaterOrEquals(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 1 +""" + +I_check_greaterOrEquals_with_UInt128 = r""" +greaterOrEquals(toUInt128(1), toUInt128(1)) greaterOrEquals(toUInt128(\'340282366920938463463374607431768211455\'), toUInt128(\'0\')) +1 1 +""" + +I_check_greaterOrEquals_with_UInt256 = r""" +greaterOrEquals(toUInt256(1), toUInt256(1)) greaterOrEquals(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), toUInt256(\'0\')) +1 1 +""" + +I_check_the_table_for_the_output_of_equals_with_Int128 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_equals_with_Int256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_equals_with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_equals_with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_notEquals_with_Int128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_notEquals_with_Int256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_notEquals_with_UInt128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_notEquals_with_UInt256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_less_with_Int128 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_less_with_Int256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_less_with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_less_with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_greater_with_Int128 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_greater_with_Int256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_greater_with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_greater_with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_lessOrEquals_with_Int128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_lessOrEquals_with_Int256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_lessOrEquals_with_UInt128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_lessOrEquals_with_UInt256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_greaterOrEquals_with_Int128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_greaterOrEquals_with_Int256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_greaterOrEquals_with_UInt128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_greaterOrEquals_with_UInt256 = r""" +a +0 +1 +1 +""" + +I_check_equals_with_Decimal256 = r""" +equals(toDecimal256(1, 0), toDecimal256(1, 0)) equals(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0), toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 0 +""" + +I_check_notEquals_with_Decimal256 = r""" +notEquals(toDecimal256(1, 0), toDecimal256(1, 0)) notEquals(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0), toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +0 1 +""" + +I_check_less_with_Decimal256 = r""" +less(toDecimal256(1, 0), toDecimal256(1, 0)) less(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0), toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +0 0 +""" + +I_check_greater_with_Decimal256 = r""" +greater(toDecimal256(1, 0), toDecimal256(1, 0)) greater(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0), toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +0 1 +""" + +I_check_lessOrEquals_with_Decimal256 = r""" +lessOrEquals(toDecimal256(1, 0), toDecimal256(1, 0)) lessOrEquals(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0), toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 0 +""" + +I_check_greaterOrEquals_with_Decimal256 = r""" +greaterOrEquals(toDecimal256(1, 0), toDecimal256(1, 0)) greaterOrEquals(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0), toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 1 +""" + +I_check_the_table_for_the_output_of_equals_with_Decimal256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_notEquals_with_Decimal256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_less_with_Decimal256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_greater_with_Decimal256 = r""" +a +0 +0 +1 +""" + +I_check_the_table_for_the_output_of_lessOrEquals_with_Decimal256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_for_the_output_of_greaterOrEquals_with_Decimal256 = r""" +a +0 +1 +1 +""" + +I_check_exp__with_Int128_using_max_and_min = r""" +exp(toInt128(\'170141183460469231731687303715884105727\')) exp(toInt128(\'-170141183460469231731687303715884105728\')) +inf 0 +""" + +I_check_exp__with_Int256_using_max_and_min = r""" +exp(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) exp(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +inf 0 +""" + +I_check_exp__with_UInt128_using_max_and_min = r""" +exp(toUInt128(\'340282366920938463463374607431768211455\')) exp(toUInt128(\'0\')) +inf 1 +""" + +I_check_exp__with_UInt256_using_max_and_min = r""" +exp(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) exp(toUInt256(\'0\')) +inf 1 +""" + +I_check_log__with_Int128_using_max_and_min = r""" +log(toInt128(\'170141183460469231731687303715884105727\')) log(toInt128(\'-170141183460469231731687303715884105728\')) +88.02969193111305 nan +""" + +I_check_log__with_Int256_using_max_and_min = r""" +log(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) log(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +176.75253104278605 nan +""" + +I_check_log__with_UInt128_using_max_and_min = r""" +log(toUInt128(\'340282366920938463463374607431768211455\')) log(toUInt128(\'0\')) +88.722839111673 -inf +""" + +I_check_log__with_UInt256_using_max_and_min = r""" +log(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) log(toUInt256(\'0\')) +177.445678223346 -inf +""" + +I_check_ln__with_Int128_using_max_and_min = r""" +log(toInt128(\'170141183460469231731687303715884105727\')) log(toInt128(\'-170141183460469231731687303715884105728\')) +88.02969193111305 nan +""" + +I_check_ln__with_Int256_using_max_and_min = r""" +log(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) log(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +176.75253104278605 nan +""" + +I_check_ln__with_UInt128_using_max_and_min = r""" +log(toUInt128(\'340282366920938463463374607431768211455\')) log(toUInt128(\'0\')) +88.722839111673 -inf +""" + +I_check_ln__with_UInt256_using_max_and_min = r""" +log(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) log(toUInt256(\'0\')) +177.445678223346 -inf +""" + +I_check_exp2__with_Int128_using_max_and_min = r""" +exp2(toInt128(\'170141183460469231731687303715884105727\')) exp2(toInt128(\'-170141183460469231731687303715884105728\')) +inf 0 +""" + +I_check_exp2__with_Int256_using_max_and_min = r""" +exp2(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) exp2(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +inf 0 +""" + +I_check_exp2__with_UInt128_using_max_and_min = r""" +exp2(toUInt128(\'340282366920938463463374607431768211455\')) exp2(toUInt128(\'0\')) +inf 1 +""" + +I_check_exp2__with_UInt256_using_max_and_min = r""" +exp2(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) exp2(toUInt256(\'0\')) +inf 1 +""" + +I_check_log2__with_Int128_using_max_and_min = r""" +log2(toInt128(\'170141183460469231731687303715884105727\')) log2(toInt128(\'-170141183460469231731687303715884105728\')) +127 nan +""" + +I_check_log2__with_Int256_using_max_and_min = r""" +log2(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) log2(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +255 nan +""" + +I_check_log2__with_UInt128_using_max_and_min = r""" +log2(toUInt128(\'340282366920938463463374607431768211455\')) log2(toUInt128(\'0\')) +128 -inf +""" + +I_check_log2__with_UInt256_using_max_and_min = r""" +log2(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) log2(toUInt256(\'0\')) +256 -inf +""" + +I_check_exp10__with_Int128_using_max_and_min = r""" +exp10(toInt128(\'170141183460469231731687303715884105727\')) exp10(toInt128(\'-170141183460469231731687303715884105728\')) +inf 0 +""" + +I_check_exp10__with_Int256_using_max_and_min = r""" +exp10(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) exp10(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +inf 0 +""" + +I_check_exp10__with_UInt128_using_max_and_min = r""" +exp10(toUInt128(\'340282366920938463463374607431768211455\')) exp10(toUInt128(\'0\')) +inf 1 +""" + +I_check_exp10__with_UInt256_using_max_and_min = r""" +exp10(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) exp10(toUInt256(\'0\')) +inf 1 +""" + +I_check_log10__with_Int128_using_max_and_min = r""" +log10(toInt128(\'170141183460469231731687303715884105727\')) log10(toInt128(\'-170141183460469231731687303715884105728\')) +38.23080944932561 nan +""" + +I_check_log10__with_Int256_using_max_and_min = r""" +log10(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) log10(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +76.7626488943152 nan +""" + +I_check_log10__with_UInt128_using_max_and_min = r""" +log10(toUInt128(\'340282366920938463463374607431768211455\')) log10(toUInt128(\'0\')) +38.53183944498959 -inf +""" + +I_check_log10__with_UInt256_using_max_and_min = r""" +log10(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) log10(toUInt256(\'0\')) +77.06367888997919 -inf +""" + +I_check_sqrt__with_Int128_using_max_and_min = r""" +sqrt(toInt128(\'170141183460469231731687303715884105727\')) sqrt(toInt128(\'-170141183460469231731687303715884105728\')) +13043817825332783000 nan +""" + +I_check_sqrt__with_Int256_using_max_and_min = r""" +sqrt(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) sqrt(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +2.4061596916800453e38 nan +""" + +I_check_sqrt__with_UInt128_using_max_and_min = r""" +sqrt(toUInt128(\'340282366920938463463374607431768211455\')) sqrt(toUInt128(\'0\')) +18446744073709552000 0 +""" + +I_check_sqrt__with_UInt256_using_max_and_min = r""" +sqrt(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) sqrt(toUInt256(\'0\')) +3.402823669209385e38 0 +""" + +I_check_cbrt__with_Int128_using_max_and_min = r""" +cbrt(toInt128(\'170141183460469231731687303715884105727\')) cbrt(toInt128(\'-170141183460469231731687303715884105728\')) +5541191377756.637 -5541191377756.637 +""" + +I_check_cbrt__with_Int256_using_max_and_min = r""" +cbrt(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) cbrt(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +3.8685626227668134e25 -3.8685626227668134e25 +""" + +I_check_cbrt__with_UInt128_using_max_and_min = r""" +cbrt(toUInt128(\'340282366920938463463374607431768211455\')) cbrt(toUInt128(\'0\')) +6981463658331.56 0 +""" + +I_check_cbrt__with_UInt256_using_max_and_min = r""" +cbrt(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) cbrt(toUInt256(\'0\')) +4.874083481260429e25 0 +""" + +I_check_erf__with_Int128_using_max_and_min = r""" +erf(toInt128(\'170141183460469231731687303715884105727\')) erf(toInt128(\'-170141183460469231731687303715884105728\')) +1 -1 +""" + +I_check_erf__with_Int256_using_max_and_min = r""" +erf(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) erf(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 -1 +""" + +I_check_erf__with_UInt128_using_max_and_min = r""" +erf(toUInt128(\'340282366920938463463374607431768211455\')) erf(toUInt128(\'0\')) +1 0 +""" + +I_check_erf__with_UInt256_using_max_and_min = r""" +erf(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) erf(toUInt256(\'0\')) +1 0 +""" + +I_check_erfc__with_Int128_using_max_and_min = r""" +erfc(toInt128(\'170141183460469231731687303715884105727\')) erfc(toInt128(\'-170141183460469231731687303715884105728\')) +0 2 +""" + +I_check_erfc__with_Int256_using_max_and_min = r""" +erfc(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) erfc(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +0 2 +""" + +I_check_erfc__with_UInt128_using_max_and_min = r""" +erfc(toUInt128(\'340282366920938463463374607431768211455\')) erfc(toUInt128(\'0\')) +0 1 +""" + +I_check_erfc__with_UInt256_using_max_and_min = r""" +erfc(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) erfc(toUInt256(\'0\')) +0 1 +""" + +I_check_lgamma__with_Int128_using_max_and_min = r""" +lgamma(toInt128(\'170141183460469231731687303715884105727\')) lgamma(toInt128(\'-170141183460469231731687303715884105728\')) +1.4807334781359624e40 -1.4807334781359624e40 +""" + +I_check_lgamma__with_Int256_using_max_and_min = r""" +lgamma(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) lgamma(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1.0175376379095233e79 -1.0175376379095233e79 +""" + +I_check_lgamma__with_UInt128_using_max_and_min = r""" +lgamma(toUInt128(\'340282366920938463463374607431768211455\')) lgamma(toUInt128(\'0\')) +2.985053532594476e40 inf +""" + +I_check_lgamma__with_UInt256_using_max_and_min = r""" +lgamma(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) lgamma(toUInt256(\'0\')) +2.0431013718376458e79 inf +""" + +I_check_tgamma__with_Int128_using_max_and_min = r""" +tgamma(toInt128(\'170141183460469231731687303715884105727\')) tgamma(toInt128(\'-170141183460469231731687303715884105728\')) +inf nan +""" + +I_check_tgamma__with_Int256_using_max_and_min = r""" +tgamma(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) tgamma(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +inf nan +""" + +I_check_tgamma__with_UInt128_using_max_and_min = r""" +tgamma(toUInt128(\'340282366920938463463374607431768211455\')) tgamma(toUInt128(\'0\')) +inf inf +""" + +I_check_tgamma__with_UInt256_using_max_and_min = r""" +tgamma(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) tgamma(toUInt256(\'0\')) +inf inf +""" + +I_check_sin__with_Int128_using_max_and_min = r""" +sin(toInt128(\'170141183460469231731687303715884105727\')) sin(toInt128(\'-170141183460469231731687303715884105728\')) +0.6233855129558702 -0.6233855129558702 +""" + +I_check_sin__with_Int256_using_max_and_min = r""" +sin(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) sin(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +0.9751222164851924 -0.9751222164851924 +""" + +I_check_sin__with_UInt128_using_max_and_min = r""" +sin(toUInt128(\'340282366920938463463374607431768211455\')) sin(toUInt128(\'0\')) +0.9748685162860586 0 +""" + +I_check_sin__with_UInt256_using_max_and_min = r""" +sin(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) sin(toUInt256(\'0\')) +0.4323066100553458 0 +""" + +I_check_cos__with_Int128_using_max_and_min = r""" +cos(toInt128(\'170141183460469231731687303715884105727\')) cos(toInt128(\'-170141183460469231731687303715884105728\')) +0.78191463871496 0.78191463871496 +""" + +I_check_cos__with_Int256_using_max_and_min = r""" +cos(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) cos(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +0.22166791133812228 0.22166791133812228 +""" + +I_check_cos__with_UInt128_using_max_and_min = r""" +cos(toUInt128(\'340282366920938463463374607431768211455\')) cos(toUInt128(\'0\')) +0.22278100447349308 1 +""" + +I_check_cos__with_UInt256_using_max_and_min = r""" +cos(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) cos(toUInt256(\'0\')) +-0.9017266741659887 1 +""" + +I_check_tan__with_Int128_using_max_and_min = r""" +tan(toInt128(\'170141183460469231731687303715884105727\')) tan(toInt128(\'-170141183460469231731687303715884105728\')) +0.7972552016424389 -0.7972552016424389 +""" + +I_check_tan__with_Int256_using_max_and_min = r""" +tan(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) tan(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +4.399022892392326 -4.399022892392326 +""" + +I_check_tan__with_UInt128_using_max_and_min = r""" +tan(toUInt128(\'340282366920938463463374607431768211455\')) tan(toUInt128(\'0\')) +4.375905022019283 0 +""" + +I_check_tan__with_UInt256_using_max_and_min = r""" +tan(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) tan(toUInt256(\'0\')) +-0.4794208959773628 0 +""" + +I_check_asin__with_Int128_using_max_and_min = r""" +asin(toInt128(\'170141183460469231731687303715884105727\')) asin(toInt128(\'-170141183460469231731687303715884105728\')) +nan nan +""" + +I_check_asin__with_Int256_using_max_and_min = r""" +asin(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) asin(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +nan nan +""" + +I_check_asin__with_UInt128_using_max_and_min = r""" +asin(toUInt128(\'340282366920938463463374607431768211455\')) asin(toUInt128(\'0\')) +nan 0 +""" + +I_check_asin__with_UInt256_using_max_and_min = r""" +asin(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) asin(toUInt256(\'0\')) +nan 0 +""" + +I_check_acos__with_Int128_using_max_and_min = r""" +acos(toInt128(\'170141183460469231731687303715884105727\')) acos(toInt128(\'-170141183460469231731687303715884105728\')) +nan nan +""" + +I_check_acos__with_Int256_using_max_and_min = r""" +acos(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) acos(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +nan nan +""" + +I_check_acos__with_UInt128_using_max_and_min = r""" +acos(toUInt128(\'340282366920938463463374607431768211455\')) acos(toUInt128(\'0\')) +nan 1.5707963267948966 +""" + +I_check_acos__with_UInt256_using_max_and_min = r""" +acos(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) acos(toUInt256(\'0\')) +nan 1.5707963267948966 +""" + +I_check_atan__with_Int128_using_max_and_min = r""" +atan(toInt128(\'170141183460469231731687303715884105727\')) atan(toInt128(\'-170141183460469231731687303715884105728\')) +1.5707963267948966 -1.5707963267948966 +""" + +I_check_atan__with_Int256_using_max_and_min = r""" +atan(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) atan(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1.5707963267948966 -1.5707963267948966 +""" + +I_check_atan__with_UInt128_using_max_and_min = r""" +atan(toUInt128(\'340282366920938463463374607431768211455\')) atan(toUInt128(\'0\')) +1.5707963267948966 0 +""" + +I_check_atan__with_UInt256_using_max_and_min = r""" +atan(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) atan(toUInt256(\'0\')) +1.5707963267948966 0 +""" + +I_check_cosh__with_Int128_using_max_and_min = r""" +cosh(toInt128(\'170141183460469231731687303715884105727\')) cosh(toInt128(\'-170141183460469231731687303715884105728\')) +inf inf +""" + +I_check_cosh__with_Int256_using_max_and_min = r""" +cosh(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) cosh(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +inf inf +""" + +I_check_cosh__with_UInt128_using_max_and_min = r""" +cosh(toUInt128(\'340282366920938463463374607431768211455\')) cosh(toUInt128(\'0\')) +inf 1 +""" + +I_check_cosh__with_UInt256_using_max_and_min = r""" +cosh(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) cosh(toUInt256(\'0\')) +inf 1 +""" + +I_check_acosh__with_Int128_using_max_and_min = r""" +acosh(toInt128(\'170141183460469231731687303715884105727\')) acosh(toInt128(\'-170141183460469231731687303715884105728\')) +88.722839111673 nan +""" + +I_check_acosh__with_Int256_using_max_and_min = r""" +acosh(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) acosh(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +177.445678223346 nan +""" + +I_check_acosh__with_UInt128_using_max_and_min = r""" +acosh(toUInt128(\'340282366920938463463374607431768211455\')) acosh(toUInt128(\'0\')) +89.41598629223294 nan +""" + +I_check_acosh__with_UInt256_using_max_and_min = r""" +acosh(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) acosh(toUInt256(\'0\')) +178.13882540390594 nan +""" + +I_check_sinh__with_Int128_using_max_and_min = r""" +sinh(toInt128(\'170141183460469231731687303715884105727\')) sinh(toInt128(\'-170141183460469231731687303715884105728\')) +inf -inf +""" + +I_check_sinh__with_Int256_using_max_and_min = r""" +sinh(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) sinh(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +inf -inf +""" + +I_check_sinh__with_UInt128_using_max_and_min = r""" +sinh(toUInt128(\'340282366920938463463374607431768211455\')) sinh(toUInt128(\'0\')) +inf 0 +""" + +I_check_sinh__with_UInt256_using_max_and_min = r""" +sinh(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) sinh(toUInt256(\'0\')) +inf 0 +""" + +I_check_asinh__with_Int128_using_max_and_min = r""" +asinh(toInt128(\'170141183460469231731687303715884105727\')) asinh(toInt128(\'-170141183460469231731687303715884105728\')) +88.722839111673 -88.722839111673 +""" + +I_check_asinh__with_Int256_using_max_and_min = r""" +asinh(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) asinh(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +177.445678223346 -177.445678223346 +""" + +I_check_asinh__with_UInt128_using_max_and_min = r""" +asinh(toUInt128(\'340282366920938463463374607431768211455\')) asinh(toUInt128(\'0\')) +89.41598629223294 0 +""" + +I_check_asinh__with_UInt256_using_max_and_min = r""" +asinh(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) asinh(toUInt256(\'0\')) +178.13882540390594 0 +""" + +I_check_tanh__with_Int128_using_max_and_min = r""" +tanh(toInt128(\'170141183460469231731687303715884105727\')) tanh(toInt128(\'-170141183460469231731687303715884105728\')) +1 -1 +""" + +I_check_tanh__with_Int256_using_max_and_min = r""" +tanh(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) tanh(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 -1 +""" + +I_check_tanh__with_UInt128_using_max_and_min = r""" +tanh(toUInt128(\'340282366920938463463374607431768211455\')) tanh(toUInt128(\'0\')) +1 0 +""" + +I_check_tanh__with_UInt256_using_max_and_min = r""" +tanh(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) tanh(toUInt256(\'0\')) +1 0 +""" + +I_check_atanh__with_Int128_using_max_and_min = r""" +atanh(toInt128(\'170141183460469231731687303715884105727\')) atanh(toInt128(\'-170141183460469231731687303715884105728\')) +nan nan +""" + +I_check_atanh__with_Int256_using_max_and_min = r""" +atanh(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) atanh(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +nan nan +""" + +I_check_atanh__with_UInt128_using_max_and_min = r""" +atanh(toUInt128(\'340282366920938463463374607431768211455\')) atanh(toUInt128(\'0\')) +nan 0 +""" + +I_check_atanh__with_UInt256_using_max_and_min = r""" +atanh(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) atanh(toUInt256(\'0\')) +nan 0 +""" + +I_check_log1p__with_Int128_using_max_and_min = r""" +log1p(toInt128(\'170141183460469231731687303715884105727\')) log1p(toInt128(\'-170141183460469231731687303715884105728\')) +88.02969193111305 nan +""" + +I_check_log1p__with_Int256_using_max_and_min = r""" +log1p(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) log1p(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +176.75253104278605 nan +""" + +I_check_log1p__with_UInt128_using_max_and_min = r""" +log1p(toUInt128(\'340282366920938463463374607431768211455\')) log1p(toUInt128(\'0\')) +88.722839111673 0 +""" + +I_check_log1p__with_UInt256_using_max_and_min = r""" +log1p(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) log1p(toUInt256(\'0\')) +177.445678223346 0 +""" + +I_check_sign__with_Int128_using_max_and_min = r""" +sign(toInt128(\'170141183460469231731687303715884105727\')) sign(toInt128(\'-170141183460469231731687303715884105728\')) +1 -1 +""" + +I_check_sign__with_Int256_using_max_and_min = r""" +sign(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) sign(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 -1 +""" + +I_check_sign__with_UInt128_using_max_and_min = r""" +sign(toUInt128(\'340282366920938463463374607431768211455\')) sign(toUInt128(\'0\')) +1 0 +""" + +I_check_sign__with_UInt256_using_max_and_min = r""" +sign(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) sign(toUInt256(\'0\')) +1 0 +""" + +I_check_the_outputs_of_exp__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_exp__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_exp__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_exp__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_log__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_ln__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_ln__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_ln__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_ln__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_exp2__with_Int128 = r""" +a +0 +0 +2 +""" + +I_check_the_outputs_of_exp2__with_Int256 = r""" +a +0 +0 +2 +""" + +I_check_the_outputs_of_exp2__with_UInt128 = r""" +a +0 +1 +2 +""" + +I_check_the_outputs_of_exp2__with_UInt256 = r""" +a +0 +1 +2 +""" + +I_check_the_outputs_of_log2__with_Int128 = r""" +a +0 +0 +127 +""" + +I_check_the_outputs_of_log2__with_Int256 = r""" +a +0 +0 +255 +""" + +I_check_the_outputs_of_log2__with_UInt128 = r""" +a +0 +0 +128 +""" + +I_check_the_outputs_of_log2__with_UInt256 = r""" +a +0 +0 +256 +""" + +I_check_the_outputs_of_exp10__with_Int128 = r""" +a +0 +0 +10 +""" + +I_check_the_outputs_of_exp10__with_Int256 = r""" +a +0 +0 +10 +""" + +I_check_the_outputs_of_exp10__with_UInt128 = r""" +a +0 +1 +10 +""" + +I_check_the_outputs_of_exp10__with_UInt256 = r""" +a +0 +1 +10 +""" + +I_check_the_outputs_of_log10__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log10__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log10__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log10__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sqrt__with_Int128 = r""" +a +0 +1 +13043817825332783000 +""" + +I_check_the_outputs_of_sqrt__with_Int256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_sqrt__with_UInt128 = r""" +a +0 +1 +18446744073709552000 +""" + +I_check_the_outputs_of_sqrt__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_cbrt__with_Int128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_cbrt__with_Int256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_cbrt__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_cbrt__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_erf__with_Int128 = r""" +a +-1 +0 +1 +""" + +I_check_the_outputs_of_erf__with_Int256 = r""" +a +-1 +0 +1 +""" + +I_check_the_outputs_of_erf__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_erf__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_erfc__with_Int128 = r""" +a +0 +0 +2 +""" + +I_check_the_outputs_of_erfc__with_Int256 = r""" +a +0 +0 +2 +""" + +I_check_the_outputs_of_erfc__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_erfc__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_lgamma__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_lgamma__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_lgamma__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_lgamma__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tgamma__with_Int128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_tgamma__with_Int256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_tgamma__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_tgamma__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_sin__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sin__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sin__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sin__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cos__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cos__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cos__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_cos__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_tan__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tan__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tan__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tan__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asin__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asin__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asin__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asin__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acos__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acos__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acos__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acos__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atan__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atan__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atan__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atan__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cosh__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cosh__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cosh__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_cosh__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_acosh__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acosh__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acosh__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acosh__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sinh__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sinh__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sinh__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sinh__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asinh__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asinh__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asinh__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asinh__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tanh__with_Int128 = r""" +a +-1 +0 +1 +""" + +I_check_the_outputs_of_tanh__with_Int256 = r""" +a +-1 +0 +1 +""" + +I_check_the_outputs_of_tanh__with_UInt128 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_tanh__with_UInt256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_atanh__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atanh__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atanh__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atanh__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log1p__with_Int128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log1p__with_Int256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log1p__with_UInt128 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log1p__with_UInt256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sign__with_Int128 = r""" +a +-1 +1 +1 +""" + +I_check_the_outputs_of_sign__with_Int256 = r""" +a +-1 +1 +1 +""" + +I_check_the_outputs_of_sign__with_UInt128 = r""" +a +0 +1 +1 +""" + +I_check_the_outputs_of_sign__with_UInt256 = r""" +a +0 +1 +1 +""" + +I_check_exp__with_Decimal256_using_max_and_min = r""" +exp(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) exp(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +inf 0 +""" + +I_check_log__with_Decimal256_using_max_and_min = r""" +log(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) log(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +172.69388197463743 nan +""" + +I_check_ln__with_Decimal256_using_max_and_min = r""" +log(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) log(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +172.69388197463743 nan +""" + +I_check_exp2__with_Decimal256_using_max_and_min = r""" +exp2(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) exp2(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +inf 0 +""" + +I_check_log2__with_Decimal256_using_max_and_min = r""" +log2(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) log2(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +249.14460711655218 nan +""" + +I_check_exp10__with_Decimal256_using_max_and_min = r""" +exp10(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) exp10(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +inf 0 +""" + +I_check_log10__with_Decimal256_using_max_and_min = r""" +log10(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) log10(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +75 nan +""" + +I_check_sqrt__with_Decimal256_using_max_and_min = r""" +sqrt(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) sqrt(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +3.1622776601683794e37 nan +""" + +I_check_cbrt__with_Decimal256_using_max_and_min = r""" +cbrt(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) cbrt(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1e25 -1e25 +""" + +I_check_erf__with_Decimal256_using_max_and_min = r""" +erf(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) erf(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 -1 +""" + +I_check_erfc__with_Decimal256_using_max_and_min = r""" +erfc(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) erfc(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +0 2 +""" + +I_check_lgamma__with_Decimal256_using_max_and_min = r""" +lgamma(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) lgamma(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1.7169388197455342e77 -1.7169388197455342e77 +""" + +I_check_tgamma__with_Decimal256_using_max_and_min = r""" +tgamma(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) tgamma(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +inf nan +""" + +I_check_sin__with_Decimal256_using_max_and_min = r""" +sin(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) sin(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +0.66339975236386 -0.66339975236386 +""" + +I_check_cos__with_Decimal256_using_max_and_min = r""" +cos(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) cos(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-0.7482651726250322 -0.7482651726250322 +""" + +I_check_tan__with_Decimal256_using_max_and_min = r""" +tan(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) tan(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-0.8865837628611647 0.8865837628611647 +""" + +I_check_asin__with_Decimal256_using_max_and_min = r""" +asin(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) asin(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +nan nan +""" + +I_check_acos__with_Decimal256_using_max_and_min = r""" +acos(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) acos(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +nan nan +""" + +I_check_atan__with_Decimal256_using_max_and_min = r""" +atan(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) atan(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1.5707963267948966 -1.5707963267948966 +""" + +I_check_cosh__with_Decimal256_using_max_and_min = r""" +cosh(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) cosh(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +inf inf +""" + +I_check_acosh__with_Decimal256_using_max_and_min = r""" +acosh(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) acosh(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +173.38702915511337 nan +""" + +I_check_sinh__with_Decimal256_using_max_and_min = r""" +sinh(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) sinh(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +inf -inf +""" + +I_check_asinh__with_Decimal256_using_max_and_min = r""" +asinh(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) asinh(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +173.38702915511337 -173.38702915511337 +""" + +I_check_tanh__with_Decimal256_using_max_and_min = r""" +tanh(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) tanh(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 -1 +""" + +I_check_atanh__with_Decimal256_using_max_and_min = r""" +atanh(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) atanh(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +nan nan +""" + +I_check_log1p__with_Decimal256_using_max_and_min = r""" +log1p(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) log1p(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +172.69388197455342 nan +""" + +I_check_sign__with_Decimal256_using_max_and_min = r""" +sign(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) sign(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 -1 +""" + +I_check_the_outputs_of_exp__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_ln__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_exp2__with_Decimal256 = r""" +a +0 +0 +2 +""" + +I_check_the_outputs_of_log2__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_exp10__with_Decimal256 = r""" +a +0 +0 +10 +""" + +I_check_the_outputs_of_log10__with_Decimal256 = r""" +a +0 +0 +75 +""" + +I_check_the_outputs_of_sqrt__with_Decimal256 = r""" +a +0 +1 +31622776601683794000000000000000000000 +""" + +I_check_the_outputs_of_cbrt__with_Decimal256 = r""" +a +-10000000000000000000000000 +1 +10000000000000000000000000 +""" + +I_check_the_outputs_of_erf__with_Decimal256 = r""" +a +-1 +0 +1 +""" + +I_check_the_outputs_of_erfc__with_Decimal256 = r""" +a +0 +0 +2 +""" + +I_check_the_outputs_of_lgamma__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tgamma__with_Decimal256 = r""" +a +0 +0 +1 +""" + +I_check_the_outputs_of_sin__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cos__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tan__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asin__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acos__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_atan__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_cosh__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_acosh__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sinh__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_asinh__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_tanh__with_Decimal256 = r""" +a +-1 +0 +1 +""" + +I_check_the_outputs_of_atanh__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_log1p__with_Decimal256 = r""" +a +0 +0 +0 +""" + +I_check_the_outputs_of_sign__with_Decimal256 = r""" +a +-1 +1 +1 +""" + +I_check_ceil_with_Int128_using_min_and_max_values = r""" +ceil(toInt128(\'-170141183460469231731687303715884105728\')) ceil(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_ceil_with_Int256_using_min_and_max_values = r""" +ceil(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) ceil(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_ceil_with_UInt128_using_min_and_max_values = r""" +ceil(toUInt128(\'0\')) ceil(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_ceil_with_UInt256_using_min_and_max_values = r""" +ceil(toUInt256(\'0\')) ceil(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_floor_with_Int128_using_min_and_max_values = r""" +floor(toInt128(\'-170141183460469231731687303715884105728\')) floor(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_floor_with_Int256_using_min_and_max_values = r""" +floor(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) floor(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_floor_with_UInt128_using_min_and_max_values = r""" +floor(toUInt128(\'0\')) floor(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_floor_with_UInt256_using_min_and_max_values = r""" +floor(toUInt256(\'0\')) floor(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_trunc_with_Int128_using_min_and_max_values = r""" +trunc(toInt128(\'-170141183460469231731687303715884105728\')) trunc(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_trunc_with_Int256_using_min_and_max_values = r""" +trunc(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) trunc(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_trunc_with_UInt128_using_min_and_max_values = r""" +trunc(toUInt128(\'0\')) trunc(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_trunc_with_UInt256_using_min_and_max_values = r""" +trunc(toUInt256(\'0\')) trunc(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_round_with_Int128_using_min_and_max_values = r""" +round(toInt128(\'-170141183460469231731687303715884105728\')) round(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_round_with_Int256_using_min_and_max_values = r""" +round(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) round(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_round_with_UInt128_using_min_and_max_values = r""" +round(toUInt128(\'0\')) round(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_round_with_UInt256_using_min_and_max_values = r""" +round(toUInt256(\'0\')) round(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_roundBankers_with_Int128_using_min_and_max_values = r""" +roundBankers(toInt128(\'-170141183460469231731687303715884105728\')) roundBankers(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_roundBankers_with_Int256_using_min_and_max_values = r""" +roundBankers(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) roundBankers(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_roundBankers_with_UInt128_using_min_and_max_values = r""" +roundBankers(toUInt128(\'0\')) roundBankers(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_roundBankers_with_UInt256_using_min_and_max_values = r""" +roundBankers(toUInt256(\'0\')) roundBankers(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_roundDuration_with_Int128_using_min_and_max_values = r""" +roundDuration(toInt128(\'-170141183460469231731687303715884105728\')) roundDuration(toInt128(\'170141183460469231731687303715884105727\')) +0 36000 +""" + +I_check_roundDuration_with_Int256_using_min_and_max_values = r""" +roundDuration(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) roundDuration(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +0 36000 +""" + +I_check_roundDuration_with_UInt128_using_min_and_max_values = r""" +roundDuration(toUInt128(\'0\')) roundDuration(toUInt128(\'340282366920938463463374607431768211455\')) +0 36000 +""" + +I_check_roundDuration_with_UInt256_using_min_and_max_values = r""" +roundDuration(toUInt256(\'0\')) roundDuration(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 36000 +""" + +I_check_roundAge_with_Int128_using_min_and_max_values = r""" +roundAge(toInt128(\'-170141183460469231731687303715884105728\')) roundAge(toInt128(\'170141183460469231731687303715884105727\')) +0 55 +""" + +I_check_roundAge_with_Int256_using_min_and_max_values = r""" +roundAge(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) roundAge(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +0 55 +""" + +I_check_roundAge_with_UInt128_using_min_and_max_values = r""" +roundAge(toUInt128(\'0\')) roundAge(toUInt128(\'340282366920938463463374607431768211455\')) +0 55 +""" + +I_check_roundAge_with_UInt256_using_min_and_max_values = r""" +roundAge(toUInt256(\'0\')) roundAge(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 55 +""" + +I_select_the_output_of_ceil_with_Int128_from_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_select_the_output_of_ceil_with_Int256_from_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_select_the_output_of_ceil_with_UInt128_from_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_select_the_output_of_ceil_with_UInt256_from_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_select_the_output_of_floor_with_Int128_from_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_select_the_output_of_floor_with_Int256_from_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_select_the_output_of_floor_with_UInt128_from_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_select_the_output_of_floor_with_UInt256_from_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_select_the_output_of_trunc_with_Int128_from_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_select_the_output_of_trunc_with_Int256_from_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_select_the_output_of_trunc_with_UInt128_from_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_select_the_output_of_trunc_with_UInt256_from_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_select_the_output_of_round_with_Int128_from_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_select_the_output_of_round_with_Int256_from_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_select_the_output_of_round_with_UInt128_from_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_select_the_output_of_round_with_UInt256_from_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_select_the_output_of_roundBankers_with_Int128_from_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_select_the_output_of_roundBankers_with_Int256_from_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_select_the_output_of_roundBankers_with_UInt128_from_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_select_the_output_of_roundBankers_with_UInt256_from_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_select_the_output_of_roundDuration_with_Int128_from_the_table = r""" +a +0 +1 +36000 +""" + +I_select_the_output_of_roundDuration_with_Int256_from_the_table = r""" +a +0 +1 +36000 +""" + +I_select_the_output_of_roundDuration_with_UInt128_from_the_table = r""" +a +0 +1 +36000 +""" + +I_select_the_output_of_roundDuration_with_UInt256_from_the_table = r""" +a +0 +1 +36000 +""" + +I_select_the_output_of_roundAge_with_Int128_from_the_table = r""" +a +0 +17 +55 +""" + +I_select_the_output_of_roundAge_with_Int256_from_the_table = r""" +a +0 +17 +55 +""" + +I_select_the_output_of_roundAge_with_UInt128_from_the_table = r""" +a +0 +17 +55 +""" + +I_select_the_output_of_roundAge_with_UInt256_from_the_table = r""" +a +0 +17 +55 +""" + +I_check_ceil_with_Decimal256_using_min_and_max_values = r""" +ceil(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) ceil(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_floor_with_Decimal256_using_min_and_max_values = r""" +floor(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) floor(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_trunc_with_Decimal256_using_min_and_max_values = r""" +trunc(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) trunc(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_round_with_Decimal256_using_min_and_max_values = r""" +round(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) round(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_roundBankers_with_Decimal256_using_min_and_max_values = r""" +roundBankers(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) roundBankers(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_select_the_output_of_ceil_with_Decimal256_from_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_select_the_output_of_floor_with_Decimal256_from_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_select_the_output_of_trunc_with_Decimal256_from_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_select_the_output_of_round_with_Decimal256_from_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_select_the_output_of_roundBankers_with_Decimal256_from_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_bitAnd_with_Int128 = r""" +bitAnd(toInt128(1), 1) bitAnd(toInt128(\'170141183460469231731687303715884105727\'), 1) bitAnd(toInt128(\'-170141183460469231731687303715884105728\'), 1) +1 1 0 +""" + +I_check_bitAnd_with_Int256 = r""" +bitAnd(toInt256(1), 1) bitAnd(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), 1) bitAnd(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), 1) +1 1 0 +""" + +I_check_bitAnd_with_UInt128 = r""" +bitAnd(toUInt128(1), 1) bitAnd(toUInt128(\'340282366920938463463374607431768211455\'), 1) bitAnd(toUInt128(\'0\'), 1) +1 1 0 +""" + +I_check_bitAnd_with_UInt256 = r""" +bitAnd(toUInt256(1), 1) bitAnd(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), 1) bitAnd(toUInt256(\'0\'), 1) +1 1 0 +""" + +I_check_bitOr_with_Int128 = r""" +bitOr(toInt128(1), 1) bitOr(toInt128(\'170141183460469231731687303715884105727\'), 1) bitOr(toInt128(\'-170141183460469231731687303715884105728\'), 1) +1 170141183460469231731687303715884105727 -170141183460469231731687303715884105727 +""" + +I_check_bitOr_with_Int256 = r""" +bitOr(toInt256(1), 1) bitOr(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), 1) bitOr(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), 1) +1 57896044618658097711785492504343953926634992332820282019728792003956564819967 -57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_bitOr_with_UInt128 = r""" +bitOr(toUInt128(1), 1) bitOr(toUInt128(\'340282366920938463463374607431768211455\'), 1) bitOr(toUInt128(\'0\'), 1) +1 340282366920938463463374607431768211455 1 +""" + +I_check_bitOr_with_UInt256 = r""" +bitOr(toUInt256(1), 1) bitOr(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), 1) bitOr(toUInt256(\'0\'), 1) +1 115792089237316195423570985008687907853269984665640564039457584007913129639935 1 +""" + +I_check_bitXor_with_Int128 = r""" +bitXor(toInt128(1), 1) bitXor(toInt128(\'170141183460469231731687303715884105727\'), 1) bitXor(toInt128(\'-170141183460469231731687303715884105728\'), 1) +0 170141183460469231731687303715884105726 -170141183460469231731687303715884105727 +""" + +I_check_bitXor_with_Int256 = r""" +bitXor(toInt256(1), 1) bitXor(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), 1) bitXor(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), 1) +0 57896044618658097711785492504343953926634992332820282019728792003956564819966 -57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_bitXor_with_UInt128 = r""" +bitXor(toUInt128(1), 1) bitXor(toUInt128(\'340282366920938463463374607431768211455\'), 1) bitXor(toUInt128(\'0\'), 1) +0 340282366920938463463374607431768211454 1 +""" + +I_check_bitXor_with_UInt256 = r""" +bitXor(toUInt256(1), 1) bitXor(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), 1) bitXor(toUInt256(\'0\'), 1) +0 115792089237316195423570985008687907853269984665640564039457584007913129639934 1 +""" + +I_check_bitShiftLeft_with_Int128 = r""" +bitShiftLeft(toInt128(1), 1) bitShiftLeft(toInt128(\'170141183460469231731687303715884105727\'), 1) bitShiftLeft(toInt128(\'-170141183460469231731687303715884105728\'), 1) +2 -2 0 +""" + +I_check_bitShiftLeft_with_Int256 = r""" +bitShiftLeft(toInt256(1), 1) bitShiftLeft(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), 1) bitShiftLeft(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), 1) +2 -2 0 +""" + +I_check_bitShiftLeft_with_UInt128 = r""" +bitShiftLeft(toUInt128(1), 1) bitShiftLeft(toUInt128(\'340282366920938463463374607431768211455\'), 1) bitShiftLeft(toUInt128(\'0\'), 1) +2 340282366920938463463374607431768211454 0 +""" + +I_check_bitShiftLeft_with_UInt256 = r""" +bitShiftLeft(toUInt256(1), 1) bitShiftLeft(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), 1) bitShiftLeft(toUInt256(\'0\'), 1) +2 115792089237316195423570985008687907853269984665640564039457584007913129639934 0 +""" + +I_check_bitShiftRight_with_Int128 = r""" +bitShiftRight(toInt128(1), 1) bitShiftRight(toInt128(\'170141183460469231731687303715884105727\'), 1) bitShiftRight(toInt128(\'-170141183460469231731687303715884105728\'), 1) +0 85070591730234615865843651857942052863 -85070591730234615865843651857942052864 +""" + +I_check_bitShiftRight_with_Int256 = r""" +bitShiftRight(toInt256(1), 1) bitShiftRight(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\'), 1) bitShiftRight(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\'), 1) +0 28948022309329048855892746252171976963317496166410141009864396001978282409983 -28948022309329048855892746252171976963317496166410141009864396001978282409984 +""" + +I_check_bitShiftRight_with_UInt128 = r""" +bitShiftRight(toUInt128(1), 1) bitShiftRight(toUInt128(\'340282366920938463463374607431768211455\'), 1) bitShiftRight(toUInt128(\'0\'), 1) +0 170141183460469231731687303715884105727 0 +""" + +I_check_bitShiftRight_with_UInt256 = r""" +bitShiftRight(toUInt256(1), 1) bitShiftRight(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\'), 1) bitShiftRight(toUInt256(\'0\'), 1) +0 57896044618658097711785492504343953926634992332820282019728792003956564819967 0 +""" + +Check_bitNot_with_Int128 = r""" +bitNot(toInt128(1)) bitNot(toInt128(\'170141183460469231731687303715884105727\')) bitNot(toInt128(\'-170141183460469231731687303715884105728\')) +-2 -170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +Check_bitNot_with_Int256 = r""" +bitNot(toInt256(1)) bitNot(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) bitNot(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +-2 -57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +Check_bitNot_with_UInt128 = r""" +bitNot(toUInt128(1)) bitNot(toUInt128(\'340282366920938463463374607431768211455\')) bitNot(toUInt128(\'0\')) +340282366920938463463374607431768211454 0 340282366920938463463374607431768211455 +""" + +Check_bitNot_with_UInt256 = r""" +bitNot(toUInt256(1)) bitNot(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) bitNot(toUInt256(\'0\')) +115792089237316195423570985008687907853269984665640564039457584007913129639934 0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +Check_bitCount_with_Int128 = r""" +bitCount(toInt128(1)) bitCount(toInt128(\'170141183460469231731687303715884105727\')) bitCount(toInt128(\'-170141183460469231731687303715884105728\')) +1 64 0 +""" + +Check_bitCount_with_Int256 = r""" +bitCount(toInt256(1)) bitCount(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) bitCount(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) +1 64 0 +""" + +Check_bitCount_with_UInt128 = r""" +bitCount(toUInt128(1)) bitCount(toUInt128(\'340282366920938463463374607431768211455\')) bitCount(toUInt128(\'0\')) +1 64 0 +""" + +Check_bitCount_with_UInt256 = r""" +bitCount(toUInt256(1)) bitCount(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) bitCount(toUInt256(\'0\')) +1 64 0 +""" + +I_check_the_table_with_values_of_bitAnd_and_Int128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_with_values_of_bitAnd_and_Int256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_with_values_of_bitAnd_and_UInt128 = r""" +a +0 +1 +1 +""" + +I_check_the_table_with_values_of_bitAnd_and_UInt256 = r""" +a +0 +1 +1 +""" + +I_check_the_table_with_values_of_bitOr_and_Int128 = r""" +a +-170141183460469231731687303715884105727 +1 +170141183460469231731687303715884105727 +""" + +I_check_the_table_with_values_of_bitOr_and_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819967 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_with_values_of_bitOr_and_UInt128 = r""" +a +1 +1 +340282366920938463463374607431768211455 +""" + +I_check_the_table_with_values_of_bitOr_and_UInt256 = r""" +a +1 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_with_values_of_bitXor_and_Int128 = r""" +a +-170141183460469231731687303715884105727 +0 +170141183460469231731687303715884105726 +""" + +I_check_the_table_with_values_of_bitXor_and_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819967 +0 +57896044618658097711785492504343953926634992332820282019728792003956564819966 +""" + +I_check_the_table_with_values_of_bitXor_and_UInt128 = r""" +a +0 +1 +340282366920938463463374607431768211454 +""" + +I_check_the_table_with_values_of_bitXor_and_UInt256 = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639934 +""" + +I_check_the_table_with_values_of_bitShiftLeft_and_Int128 = r""" +a +-2 +0 +2 +""" + +I_check_the_table_with_values_of_bitShiftLeft_and_Int256 = r""" +a +-2 +0 +2 +""" + +I_check_the_table_with_values_of_bitShiftLeft_and_UInt128 = r""" +a +0 +2 +340282366920938463463374607431768211454 +""" + +I_check_the_table_with_values_of_bitShiftLeft_and_UInt256 = r""" +a +0 +2 +115792089237316195423570985008687907853269984665640564039457584007913129639934 +""" + +I_check_the_table_with_values_of_bitShiftRight_and_Int128 = r""" +a +-85070591730234615865843651857942052864 +0 +85070591730234615865843651857942052863 +""" + +I_check_the_table_with_values_of_bitShiftRight_and_Int256 = r""" +a +-28948022309329048855892746252171976963317496166410141009864396001978282409984 +0 +28948022309329048855892746252171976963317496166410141009864396001978282409983 +""" + +I_check_the_table_with_values_of_bitShiftRight_and_UInt128 = r""" +a +0 +0 +170141183460469231731687303715884105727 +""" + +I_check_the_table_with_values_of_bitShiftRight_and_UInt256 = r""" +a +0 +0 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_with_values_of_bitNot_and_Int128 = r""" +a +-170141183460469231731687303715884105728 +-2 +170141183460469231731687303715884105727 +""" + +I_check_the_table_with_values_of_bitNot_and_Int256 = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +-2 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_the_table_with_values_of_bitNot_and_UInt128 = r""" +a +0 +340282366920938463463374607431768211454 +340282366920938463463374607431768211455 +""" + +I_check_the_table_with_values_of_bitNot_and_UInt256 = r""" +a +0 +115792089237316195423570985008687907853269984665640564039457584007913129639934 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_the_table_with_values_of_bitCount_and_Int128 = r""" +a +0 +1 +64 +""" + +I_check_the_table_with_values_of_bitCount_and_Int256 = r""" +a +0 +1 +64 +""" + +I_check_the_table_with_values_of_bitCount_and_UInt128 = r""" +a +0 +1 +64 +""" + +I_check_the_table_with_values_of_bitCount_and_UInt256 = r""" +a +0 +1 +64 +""" + +I_check_isNull__with_Int128_using_min_and_max = r""" +isNull(toInt128(\'-170141183460469231731687303715884105728\')) isNull(toInt128(\'170141183460469231731687303715884105727\')) +0 0 +""" + +I_check_isNull__with_Int256_using_min_and_max = r""" +isNull(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) isNull(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +0 0 +""" + +I_check_isNull__with_UInt128_using_min_and_max = r""" +isNull(toUInt128(\'0\')) isNull(toUInt128(\'340282366920938463463374607431768211455\')) +0 0 +""" + +I_check_isNull__with_UInt256_using_min_and_max = r""" +isNull(toUInt256(\'0\')) isNull(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 0 +""" + +I_check_isNotNull__with_Int128_using_min_and_max = r""" +isNotNull(toInt128(\'-170141183460469231731687303715884105728\')) isNotNull(toInt128(\'170141183460469231731687303715884105727\')) +1 1 +""" + +I_check_isNotNull__with_Int256_using_min_and_max = r""" +isNotNull(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) isNotNull(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +1 1 +""" + +I_check_isNotNull__with_UInt128_using_min_and_max = r""" +isNotNull(toUInt128(\'0\')) isNotNull(toUInt128(\'340282366920938463463374607431768211455\')) +1 1 +""" + +I_check_isNotNull__with_UInt256_using_min_and_max = r""" +isNotNull(toUInt256(\'0\')) isNotNull(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +1 1 +""" + +I_check_coalesce__with_Int128_using_min_and_max = r""" +coalesce(toInt128(\'-170141183460469231731687303715884105728\')) coalesce(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_coalesce__with_Int256_using_min_and_max = r""" +coalesce(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) coalesce(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_coalesce__with_UInt128_using_min_and_max = r""" +coalesce(toUInt128(\'0\')) coalesce(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_coalesce__with_UInt256_using_min_and_max = r""" +coalesce(toUInt256(\'0\')) coalesce(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_assumeNotNull__with_Int128_using_min_and_max = r""" +assumeNotNull(toInt128(\'-170141183460469231731687303715884105728\')) assumeNotNull(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_assumeNotNull__with_Int256_using_min_and_max = r""" +assumeNotNull(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) assumeNotNull(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_assumeNotNull__with_UInt128_using_min_and_max = r""" +assumeNotNull(toUInt128(\'0\')) assumeNotNull(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_assumeNotNull__with_UInt256_using_min_and_max = r""" +assumeNotNull(toUInt256(\'0\')) assumeNotNull(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_toNullable__with_Int128_using_min_and_max = r""" +toNullable(toInt128(\'-170141183460469231731687303715884105728\')) toNullable(toInt128(\'170141183460469231731687303715884105727\')) +-170141183460469231731687303715884105728 170141183460469231731687303715884105727 +""" + +I_check_toNullable__with_Int256_using_min_and_max = r""" +toNullable(toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) toNullable(toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +-57896044618658097711785492504343953926634992332820282019728792003956564819968 57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_toNullable__with_UInt128_using_min_and_max = r""" +toNullable(toUInt128(\'0\')) toNullable(toUInt128(\'340282366920938463463374607431768211455\')) +0 340282366920938463463374607431768211455 +""" + +I_check_toNullable__with_UInt256_using_min_and_max = r""" +toNullable(toUInt256(\'0\')) toNullable(toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +0 115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_ifNull_1__with_Int128_using_min_and_max = r""" +ifNull(1, toInt128(\'-170141183460469231731687303715884105728\')) ifNull(1, toInt128(\'170141183460469231731687303715884105727\')) +1 1 +""" + +I_check_ifNull_1__with_Int256_using_min_and_max = r""" +ifNull(1, toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) ifNull(1, toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +1 1 +""" + +I_check_ifNull_1__with_UInt128_using_min_and_max = r""" +ifNull(1, toUInt128(\'0\')) ifNull(1, toUInt128(\'340282366920938463463374607431768211455\')) +1 1 +""" + +I_check_ifNull_1__with_UInt256_using_min_and_max = r""" +ifNull(1, toUInt256(\'0\')) ifNull(1, toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +1 1 +""" + +I_check_nullIf_1__with_Int128_using_min_and_max = r""" +nullIf(1, toInt128(\'-170141183460469231731687303715884105728\')) nullIf(1, toInt128(\'170141183460469231731687303715884105727\')) +1 1 +""" + +I_check_nullIf_1__with_Int256_using_min_and_max = r""" +nullIf(1, toInt256(\'-57896044618658097711785492504343953926634992332820282019728792003956564819968\')) nullIf(1, toInt256(\'57896044618658097711785492504343953926634992332820282019728792003956564819967\')) +1 1 +""" + +I_check_nullIf_1__with_UInt128_using_min_and_max = r""" +nullIf(1, toUInt128(\'0\')) nullIf(1, toUInt128(\'340282366920938463463374607431768211455\')) +1 1 +""" + +I_check_nullIf_1__with_UInt256_using_min_and_max = r""" +nullIf(1, toUInt256(\'0\')) nullIf(1, toUInt256(\'115792089237316195423570985008687907853269984665640564039457584007913129639935\')) +1 1 +""" + +I_check_isNull__with_Int128_on_the_table = r""" +a +0 +0 +0 +""" + +I_check_isNull__with_Int256_on_the_table = r""" +a +0 +0 +0 +""" + +I_check_isNull__with_UInt128_on_the_table = r""" +a +0 +0 +0 +""" + +I_check_isNull__with_UInt256_on_the_table = r""" +a +0 +0 +0 +""" + +I_check_isNotNull__with_Int128_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_isNotNull__with_Int256_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_isNotNull__with_UInt128_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_isNotNull__with_UInt256_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_coalesce__with_Int128_on_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_coalesce__with_Int256_on_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_coalesce__with_UInt128_on_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_coalesce__with_UInt256_on_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_assumeNotNull__with_Int128_on_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_assumeNotNull__with_Int256_on_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_assumeNotNull__with_UInt128_on_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_assumeNotNull__with_UInt256_on_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_toNullable__with_Int128_on_the_table = r""" +a +-170141183460469231731687303715884105728 +1 +170141183460469231731687303715884105727 +""" + +I_check_toNullable__with_Int256_on_the_table = r""" +a +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +1 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +""" + +I_check_toNullable__with_UInt128_on_the_table = r""" +a +0 +1 +340282366920938463463374607431768211455 +""" + +I_check_toNullable__with_UInt256_on_the_table = r""" +a +0 +1 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +""" + +I_check_ifNull_1__with_Int128_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_ifNull_1__with_Int256_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_ifNull_1__with_UInt128_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_ifNull_1__with_UInt256_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_nullIf_1__with_Int128_on_the_table = r""" +a +1 +1 +\N +""" + +I_check_nullIf_1__with_Int256_on_the_table = r""" +a +1 +1 +\N +""" + +I_check_nullIf_1__with_UInt128_on_the_table = r""" +a +1 +1 +\N +""" + +I_check_nullIf_1__with_UInt256_on_the_table = r""" +a +1 +1 +\N +""" + +I_check_isNull__with_Decimal256_using_min_and_max = r""" +isNull(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) isNull(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +0 0 +""" + +I_check_isNotNull__with_Decimal256_using_min_and_max = r""" +isNotNull(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) isNotNull(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 1 +""" + +I_check_coalesce__with_Decimal256_using_min_and_max = r""" +coalesce(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) coalesce(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_assumeNotNull__with_Decimal256_using_min_and_max = r""" +assumeNotNull(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) assumeNotNull(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_toNullable__with_Decimal256_using_min_and_max = r""" +toNullable(toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) toNullable(toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +-1000000000000000000000000000000000000000000000000000000000000000000000000000 1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_ifNull_1__with_Decimal256_using_min_and_max = r""" +ifNull(1, toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) ifNull(1, toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 1 +""" + +I_check_nullIf_1__with_Decimal256_using_min_and_max = r""" +nullIf(1, toDecimal256(\'-1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) nullIf(1, toDecimal256(\'1000000000000000000000000000000000000000000000000000000000000000000000000000\', 0)) +1 1 +""" + +I_check_isNull__with_Decimal256_on_the_table = r""" +a +0 +0 +0 +""" + +I_check_isNotNull__with_Decimal256_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_coalesce__with_Decimal256_on_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_assumeNotNull__with_Decimal256_on_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_toNullable__with_Decimal256_on_the_table = r""" +a +-1000000000000000000000000000000000000000000000000000000000000000000000000000 +1 +1000000000000000000000000000000000000000000000000000000000000000000000000000 +""" + +I_check_ifNull_1__with_Decimal256_on_the_table = r""" +a +1 +1 +1 +""" + +I_check_nullIf_1__with_Decimal256_on_the_table = r""" +a +1 +1 +\N +""" + diff --git a/utils/changelog/.gitignore b/utils/changelog/.gitignore new file mode 100644 index 00000000000..9ab24b6c8b8 --- /dev/null +++ b/utils/changelog/.gitignore @@ -0,0 +1,3 @@ +*.md +*.txt +*.json diff --git a/utils/changelog/README.md b/utils/changelog/README.md new file mode 100644 index 00000000000..ff3ac39f632 --- /dev/null +++ b/utils/changelog/README.md @@ -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 +``` diff --git a/utils/simple-backport/changelog.sh b/utils/changelog/changelog.sh similarity index 97% rename from utils/simple-backport/changelog.sh rename to utils/changelog/changelog.sh index ca2dcfffff0..52817acfae4 100755 --- a/utils/simple-backport/changelog.sh +++ b/utils/changelog/changelog.sh @@ -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" diff --git a/utils/simple-backport/format-changelog.py b/utils/changelog/format-changelog.py similarity index 100% rename from utils/simple-backport/format-changelog.py rename to utils/changelog/format-changelog.py diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index e08a6ad0ffe..4b46d4783de 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -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 diff --git a/utils/simple-backport/.gitignore b/utils/simple-backport/.gitignore deleted file mode 100644 index 72e8ffc0db8..00000000000 --- a/utils/simple-backport/.gitignore +++ /dev/null @@ -1 +0,0 @@ -* diff --git a/utils/simple-backport/README.md b/utils/simple-backport/README.md deleted file mode 100644 index c5a625ca0d1..00000000000 --- a/utils/simple-backport/README.md +++ /dev/null @@ -1,107 +0,0 @@ -# Упрощённый скрипт для бекпортирования - -Это упрощённый скрипт для бекпортирования. Он определяет, какие пулреквесты ещё не бекпортировали из мастера в указанную ветку. Запускать скрипт нужно из папки, где он лежит, указав ему название ветки. Он предполагает, что ваш апстримный remote называется origin. -``` -cd my-clickhouse-repo/utils/simple-backport -git fetch origin -time GITHUB_TOKEN= ./backport.sh 20.1 -``` - -Скрипт выведет примитивный отчёт: -``` -$ time 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 ". - -### Я поправил пулреквест, почему скрипт не видит? -В процессе работы скрипт кеширует данные о пулреквестах в текущей папке, чтобы -экономить квоту гитхаба. Удалите закешированные файлы, например, для всех -реквестов, которые не помечены как пропущенные: -``` -$ 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. diff --git a/utils/simple-backport/backport.sh b/utils/simple-backport/backport.sh deleted file mode 100755 index 64f8e6004bf..00000000000 --- a/utils/simple-backport/backport.sh +++ /dev/null @@ -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."