diff --git a/CHANGELOG.md b/CHANGELOG.md index ca14134753a..6a4666f08bb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,48 @@ ## ClickHouse release v20.4 +### ClickHouse release v20.4.3.16-stable 2020-05-23 + +#### Bug Fix + +* Removed logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). +* Fixed memory leak in registerDiskS3. [#11074](https://github.com/ClickHouse/ClickHouse/pull/11074) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fixed the potential missed data during termination of Kafka engine table. [#11048](https://github.com/ClickHouse/ClickHouse/pull/11048) ([filimonov](https://github.com/filimonov)). +* Fixed `parseDateTime64BestEffort` argument resolution bugs. [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). +* Fixed very rare potential use-after-free error in `MergeTree` if table was not created successfully. [#10986](https://github.com/ClickHouse/ClickHouse/pull/10986), [#10970](https://github.com/ClickHouse/ClickHouse/pull/10970) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed metadata (relative path for rename) and data (relative path for symlink) handling for Atomic database. [#10980](https://github.com/ClickHouse/ClickHouse/pull/10980) ([Azat Khuzhin](https://github.com/azat)). +* Fixed server crash on concurrent `ALTER` and `DROP DATABASE` queries with `Atomic` database engine. [#10968](https://github.com/ClickHouse/ClickHouse/pull/10968) ([tavplubix](https://github.com/tavplubix)). +* Fixed incorrect raw data size in `getRawData()` method. [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). +* Fixed incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed sending partially written files by the `DistributedBlockOutputStream`. [#10940](https://github.com/ClickHouse/ClickHouse/pull/10940) ([Azat Khuzhin](https://github.com/azat)). +* Fixed crash in `SELECT count(notNullIn(NULL, []))`. [#10920](https://github.com/ClickHouse/ClickHouse/pull/10920) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed the hang which was happening sometimes during `DROP` of `Kafka` table engine. (or during server restarts). [#10910](https://github.com/ClickHouse/ClickHouse/pull/10910) ([filimonov](https://github.com/filimonov)). +* Fixed the impossibility of executing multiple `ALTER RENAME` like `a TO b, c TO a`. [#10895](https://github.com/ClickHouse/ClickHouse/pull/10895) ([alesapin](https://github.com/alesapin)). +* Fixed possible race which could happen when you get result from aggregate function state from multiple thread for the same column. The only way it can happen is when you use `finalizeAggregation` function while reading from table with `Memory` engine which stores `AggregateFunction` state for `quantile*` function. [#10890](https://github.com/ClickHouse/ClickHouse/pull/10890) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed `SIGSEGV` in `StringHashTable` if such a key does not exist. [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `WATCH` hangs after `LiveView` table was dropped from database with `Atomic` engine. [#10859](https://github.com/ClickHouse/ClickHouse/pull/10859) ([tavplubix](https://github.com/tavplubix)). +* Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). +* Now constraints are updated if the column participating in `CONSTRAINT` expression was renamed. Fixes [#10844](https://github.com/ClickHouse/ClickHouse/issues/10844). [#10847](https://github.com/ClickHouse/ClickHouse/pull/10847) ([alesapin](https://github.com/alesapin)). +* Fixed potential read of uninitialized memory in cache-dictionary. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed columns order after `Block::sortColumns()`. [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the issue with `ODBC` bridge when no quoting of identifiers is requested. Fixes [#7984] (https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `UBSan` and `MSan` report in `DateLUT`. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incorrect type conversion in key conditions. Fixes [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). +* Fixed `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed combinator `-OrNull` and `-OrDefault` when combined with `-State`. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). +* Fixed possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). +* Fixed `nullptr` dereference in `StorageBuffer` if server was shutdown before table startup. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `optimize_skip_unused_shards` with `LowCardinality`. [#10611](https://github.com/ClickHouse/ClickHouse/pull/10611) ([Azat Khuzhin](https://github.com/azat)). +* Fixed handling condition variable for synchronous mutations. In some cases signals to that condition variable could be lost. [#10588](https://github.com/ClickHouse/ClickHouse/pull/10588) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fixed possible crash when `createDictionary()` is called before `loadStoredObject()` has finished. [#10587](https://github.com/ClickHouse/ClickHouse/pull/10587) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed `SELECT` of column `ALIAS` which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). +* Implemented comparison between DateTime64 and String values. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). +* Disable `GROUP BY` sharding_key optimization by default (`optimize_distributed_group_by_sharding_key` had been introduced and turned of by default, due to trickery of sharding_key analyzing, simple example is `if` in sharding key) and fix it for `WITH ROLLUP/CUBE/TOTALS`. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). +* Fixed [#10263](https://github.com/ClickHouse/ClickHouse/issues/10263). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). +* Added tests about `max_rows_to_sort` setting. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added backward compatibility for create bloom filter index. [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). + ### ClickHouse release v20.4.2.9, 2020-05-12 #### Backward Incompatible Change @@ -280,6 +323,57 @@ ## ClickHouse release v20.3 +### ClickHouse release v20.3.10.75-lts 2020-05-23 + +#### Bug Fix + +* Removed logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). +* Fixed `parseDateTime64BestEffort` argument resolution bugs. [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). +* Fixed incorrect raw data size in method `getRawData()`. [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). +* Fixed incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of `GROUP BY` result is large and aggregation is performed by a single `String` field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed backward compatibility with tuples in `Distributed` tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed `SIGSEGV` in `StringHashTable` if such a key does not exist. [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). +* Fixed columns order after `Block::sortColumns()`. [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the issue with `ODBC` bridge when no quoting of identifiers is requested. Fixes [#7984] (https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `UBSan` and `MSan` report in `DateLUT`. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incorrect type conversion in key conditions. Fixes [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)) +* Fixed `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed combinator -`OrNull` and `-OrDefault` when combined with `-State`. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). +* Fixed crash in `generateRandom` with nested types. Fixes [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed disappearing totals. Totals could have being filtered if query had had join or subquery with external where condition. Fixes [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed multiple usages of `IN` operator with the identical set in one query. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed bug, which causes http requests stuck on client close when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. Fixes [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). +* Fixed order of parameters in `AggregateTransform` constructor. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). +* Fixed the lack of parallel execution of remote queries with `distributed_aggregation_memory_efficient` enabled. Fixes [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible incorrect number of rows for queries with `LIMIT`. Fixes [#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709). [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed a bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). +* Fixed a bug when on `SYSTEM DROP DNS CACHE` query also drop caches, which are used to check if user is allowed to connect from some IP addresses. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). +* Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed `SELECT` of column `ALIAS` which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). +* Implemented comparison between DateTime64 and String values. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). +* Fixed index corruption, which may accur in some cases after merge compact parts into another compact part. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed the situation, when mutation finished all parts, but hung up in `is_done=0`. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)). +* Fixed overflow at beginning of unix epoch for timezones with fractional offset from `UTC`. This fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed improper shutdown of `Distributed` storage. [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). +* Fixed numeric overflow in `simpleLinearRegression` over large integers. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). + + +#### Build/Testing/Packaging Improvement + +* Fix UBSan report in LZ4 library. [#10631](https://github.com/ClickHouse/ClickHouse/pull/10631) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix clang-10 build. https://github.com/ClickHouse/ClickHouse/issues/10238. [#10370](https://github.com/ClickHouse/ClickHouse/pull/10370) ([Amos Bird](https://github.com/amosbird)). +* Added failing tests about `max_rows_to_sort` setting. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added some improvements in printing diagnostic info in input formats. Fixes [#10204](https://github.com/ClickHouse/ClickHouse/issues/10204). [#10418](https://github.com/ClickHouse/ClickHouse/pull/10418) ([tavplubix](https://github.com/tavplubix)). +* Added CA certificates to clickhouse-server docker image. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)). + +#### Bug fix + +* #10551. [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). + + ### ClickHouse release v20.3.8.53, 2020-04-23 #### Bug Fix @@ -630,6 +724,35 @@ ## ClickHouse release v20.1 +### ClickHouse release v20.1.12.86, 2020-05-26 + +#### Bug Fix + +* Fixed incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug, which causes http requests stuck on client close when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. Fixes [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). +* Fixed a bug when on `SYSTEM DROP DNS CACHE` query also drop caches, which are used to check if user is allowed to connect from some IP addresses. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). +* Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed the situation when mutation finished all parts, but hung up in `is_done=0`. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)). +* Fixed overflow at beginning of unix epoch for timezones with fractional offset from UTC. This fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed improper shutdown of Distributed storage. [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). +* Fixed numeric overflow in `simpleLinearRegression` over large integers. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). +* Fixed removing metadata directory when attach database fails. [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)). +* Added a check of number and type of arguments when creating `BloomFilter` index [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623). [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed the issue when a query with `ARRAY JOIN`, `ORDER BY` and `LIMIT` may return incomplete result. This fixes [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Prefer `fallback_to_stale_replicas` over `skip_unavailable_shards`. [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)). +* Fixed wrong flattening of `Array(Tuple(...))` data types. This fixes [#10259](https://github.com/ClickHouse/ClickHouse/issues/10259). [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed wrong behavior in `HashTable` that caused compilation error when trying to read HashMap from buffer. [#10386](https://github.com/ClickHouse/ClickHouse/pull/10386) ([palasonic1](https://github.com/palasonic1)). +* Fixed possible `Pipeline stuck` error in `ConcatProcessor` which could have happened in remote query. [#10381](https://github.com/ClickHouse/ClickHouse/pull/10381) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Pipeline stuck` with `max_rows_to_group_by` and `group_by_overflow_mode = 'break'`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed several bugs when some data was inserted with quorum, then deleted somehow (DROP PARTITION, TTL) and this leaded to the stuck of INSERTs or false-positive exceptions in SELECTs. This fixes [#9946](https://github.com/ClickHouse/ClickHouse/issues/9946). [#10188](https://github.com/ClickHouse/ClickHouse/pull/10188) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed incompatibility when versions prior to 18.12.17 are used on remote servers and newer is used on initiating server, and GROUP BY both fixed and non-fixed keys, and when two-level group by method is activated. [#3254](https://github.com/ClickHouse/ClickHouse/pull/3254) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement + +* Added CA certificates to clickhouse-server docker image. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)). + + ### ClickHouse release v20.1.10.70, 2020-04-17 #### Bug Fix diff --git a/base/common/ErrorHandlers.h b/base/common/ErrorHandlers.h index b35cdcab93d..67194ee043b 100644 --- a/base/common/ErrorHandlers.h +++ b/base/common/ErrorHandlers.h @@ -28,7 +28,7 @@ public: void exception() override { logException(); } private: - Logger * log = &Logger::get("ServerErrorHandler"); + Poco::Logger * log = &Poco::Logger::get("ServerErrorHandler"); void logException() { diff --git a/base/common/logger_useful.h b/base/common/logger_useful.h index c89ec63ef22..f760d59de45 100644 --- a/base/common/logger_useful.h +++ b/base/common/logger_useful.h @@ -9,28 +9,33 @@ #include -/// TODO Remove this. -using Poco::Logger; -using Poco::Message; -using DB::LogsLevel; -using DB::CurrentThread; +namespace +{ + template constexpr size_t numArgs(Ts &&...) { return sizeof...(Ts); } + template constexpr auto firstArg(T && x, Ts &&...) { return std::forward(x); } +} + /// Logs a message to a specified logger with that level. +/// If more than one argument is provided, +/// the first argument is interpreted as template with {}-substitutions +/// and the latter arguments treat as values to substitute. +/// If only one argument is provided, it is threat as message without substitutions. -#define LOG_IMPL(logger, priority, PRIORITY, ...) do \ +#define LOG_IMPL(logger, priority, PRIORITY, ...) do \ { \ - const bool is_clients_log = (CurrentThread::getGroup() != nullptr) && \ - (CurrentThread::getGroup()->client_logs_level >= (priority)); \ + const bool is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \ + (DB::CurrentThread::getGroup()->client_logs_level >= (priority)); \ if ((logger)->is((PRIORITY)) || is_clients_log) \ { \ - std::string formatted_message = fmt::format(__VA_ARGS__); \ + std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \ if (auto channel = (logger)->getChannel()) \ { \ std::string file_function; \ file_function += __FILE__; \ file_function += "; "; \ file_function += __PRETTY_FUNCTION__; \ - Message poco_message((logger)->name(), formatted_message, \ + Poco::Message poco_message((logger)->name(), formatted_message, \ (PRIORITY), file_function.c_str(), __LINE__); \ channel->log(poco_message); \ } \ @@ -38,9 +43,18 @@ using DB::CurrentThread; } while (false) -#define LOG_TRACE(logger, ...) LOG_IMPL(logger, LogsLevel::trace, Message::PRIO_TRACE, __VA_ARGS__) -#define LOG_DEBUG(logger, ...) LOG_IMPL(logger, LogsLevel::debug, Message::PRIO_DEBUG, __VA_ARGS__) -#define LOG_INFO(logger, ...) LOG_IMPL(logger, LogsLevel::information, Message::PRIO_INFORMATION, __VA_ARGS__) -#define LOG_WARNING(logger, ...) LOG_IMPL(logger, LogsLevel::warning, Message::PRIO_WARNING, __VA_ARGS__) -#define LOG_ERROR(logger, ...) LOG_IMPL(logger, LogsLevel::error, Message::PRIO_ERROR, __VA_ARGS__) -#define LOG_FATAL(logger, ...) LOG_IMPL(logger, LogsLevel::error, Message::PRIO_FATAL, __VA_ARGS__) +#define LOG_TRACE(logger, ...) LOG_IMPL(logger, DB::LogsLevel::trace, Poco::Message::PRIO_TRACE, __VA_ARGS__) +#define LOG_DEBUG(logger, ...) LOG_IMPL(logger, DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG, __VA_ARGS__) +#define LOG_INFO(logger, ...) LOG_IMPL(logger, DB::LogsLevel::information, Poco::Message::PRIO_INFORMATION, __VA_ARGS__) +#define LOG_WARNING(logger, ...) LOG_IMPL(logger, DB::LogsLevel::warning, Poco::Message::PRIO_WARNING, __VA_ARGS__) +#define LOG_ERROR(logger, ...) LOG_IMPL(logger, DB::LogsLevel::error, Poco::Message::PRIO_ERROR, __VA_ARGS__) +#define LOG_FATAL(logger, ...) LOG_IMPL(logger, DB::LogsLevel::error, Poco::Message::PRIO_FATAL, __VA_ARGS__) + + +/// Compatibility for external projects. +#if defined(ARCADIA_BUILD) + using Poco::Logger; + using Poco::Message; + using DB::LogsLevel; + using DB::CurrentThread; +#endif diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 10c7173d5b1..2a36777218e 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -124,7 +124,7 @@ static void signalHandler(int sig, siginfo_t * info, void * context) const ucontext_t signal_context = *reinterpret_cast(context); const StackTrace stack_trace(signal_context); - StringRef query_id = CurrentThread::getQueryId(); /// This is signal safe. + StringRef query_id = DB::CurrentThread::getQueryId(); /// This is signal safe. query_id.size = std::min(query_id.size, max_query_id_size); DB::writeBinary(sig, out); @@ -162,7 +162,7 @@ public: }; explicit SignalListener(BaseDaemon & daemon_) - : log(&Logger::get("BaseDaemon")) + : log(&Poco::Logger::get("BaseDaemon")) , daemon(daemon_) { } @@ -231,7 +231,7 @@ public: } private: - Logger * log; + Poco::Logger * log; BaseDaemon & daemon; void onTerminate(const std::string & message, UInt32 thread_num) const @@ -288,9 +288,9 @@ extern "C" void __sanitizer_set_death_callback(void (*)()); static void sanitizerDeathCallback() { - Logger * log = &Logger::get("BaseDaemon"); + Poco::Logger * log = &Poco::Logger::get("BaseDaemon"); - StringRef query_id = CurrentThread::getQueryId(); /// This is signal safe. + StringRef query_id = DB::CurrentThread::getQueryId(); /// This is signal safe. { std::stringstream message; @@ -498,10 +498,10 @@ void debugIncreaseOOMScore() } catch (const Poco::Exception & e) { - LOG_WARNING(&Logger::root(), "Failed to adjust OOM score: '{}'.", e.displayText()); + LOG_WARNING(&Poco::Logger::root(), "Failed to adjust OOM score: '{}'.", e.displayText()); return; } - LOG_INFO(&Logger::root(), "Set OOM score adjustment to {}", new_score); + LOG_INFO(&Poco::Logger::root(), "Set OOM score adjustment to {}", new_score); } #else void debugIncreaseOOMScore() {} @@ -715,7 +715,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() void BaseDaemon::logRevision() const { - Logger::root().information("Starting " + std::string{VERSION_FULL} + Poco::Logger::root().information("Starting " + std::string{VERSION_FULL} + " with revision " + std::to_string(ClickHouseRevision::get()) + ", PID " + std::to_string(getpid())); } @@ -732,7 +732,7 @@ void BaseDaemon::handleNotification(Poco::TaskFailedNotification *_tfn) { task_failed = true; Poco::AutoPtr fn(_tfn); - Logger *lg = &(logger()); + Poco::Logger * lg = &(logger()); LOG_ERROR(lg, "Task '{}' failed. Daemon is shutting down. Reason - {}", fn->task()->name(), fn->reason().displayText()); ServerApplication::terminate(); } diff --git a/base/loggers/OwnFormattingChannel.cpp b/base/loggers/OwnFormattingChannel.cpp index 9fccc929364..f03d155bde7 100644 --- a/base/loggers/OwnFormattingChannel.cpp +++ b/base/loggers/OwnFormattingChannel.cpp @@ -4,6 +4,7 @@ namespace DB { + void OwnFormattingChannel::logExtended(const ExtendedLogMessage & msg) { if (pChannel && priority >= msg.base.getPriority()) @@ -28,5 +29,4 @@ void OwnFormattingChannel::log(const Poco::Message & msg) OwnFormattingChannel::~OwnFormattingChannel() = default; - } diff --git a/base/loggers/OwnSplitChannel.cpp b/base/loggers/OwnSplitChannel.cpp index 3f77b594fbb..22951d1b509 100644 --- a/base/loggers/OwnSplitChannel.cpp +++ b/base/loggers/OwnSplitChannel.cpp @@ -69,7 +69,6 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) logs_queue->emplace(std::move(columns)); } - /// Also log to system.text_log table, if message is not too noisy auto text_log_max_priority_loaded = text_log_max_priority.load(std::memory_order_relaxed); if (text_log_max_priority_loaded && msg.getPriority() <= text_log_max_priority_loaded) diff --git a/cmake/protobuf_generate_cpp.cmake b/cmake/protobuf_generate_cpp.cmake index 7ee9e8d7c81..cc2502e5eeb 100644 --- a/cmake/protobuf_generate_cpp.cmake +++ b/cmake/protobuf_generate_cpp.cmake @@ -55,6 +55,7 @@ function(protobuf_generate_cpp_impl SRCS HDRS MODES OUTPUT_FILE_EXTS PLUGIN) endif() set (intermediate_dir ${CMAKE_CURRENT_BINARY_DIR}/intermediate) + file (MAKE_DIRECTORY ${intermediate_dir}) set (protoc_args) foreach (mode ${MODES}) @@ -112,7 +113,7 @@ if (PROTOBUF_GENERATE_CPP_SCRIPT_MODE) set (intermediate_dir ${DIR}/intermediate) set (intermediate_output "${intermediate_dir}/${FILENAME}") - if (COMPILER_ID STREQUAL "Clang") + if (COMPILER_ID MATCHES "Clang") set (pragma_push "#pragma clang diagnostic push\n") set (pragma_pop "#pragma clang diagnostic pop\n") set (pragma_disable_warnings "#pragma clang diagnostic ignored \"-Weverything\"\n") diff --git a/contrib/librdkafka b/contrib/librdkafka index 4ffe54b4f59..b0d91bd74ab 160000 --- a/contrib/librdkafka +++ b/contrib/librdkafka @@ -1 +1 @@ -Subproject commit 4ffe54b4f59ee5ae3767f9f25dc14651a3384d62 +Subproject commit b0d91bd74abb5f0e1ee972d326a317ad610f6300 diff --git a/contrib/librdkafka-cmake/CMakeLists.txt b/contrib/librdkafka-cmake/CMakeLists.txt index 0f7b8e349ff..93ef9d2357b 100644 --- a/contrib/librdkafka-cmake/CMakeLists.txt +++ b/contrib/librdkafka-cmake/CMakeLists.txt @@ -3,20 +3,30 @@ set(RDKAFKA_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/librdkafka/src) set(SRCS ${RDKAFKA_SOURCE_DIR}/crc32c.c ${RDKAFKA_SOURCE_DIR}/rdkafka_zstd.c +# ${RDKAFKA_SOURCE_DIR}/lz4.c +# ${RDKAFKA_SOURCE_DIR}/lz4frame.c +# ${RDKAFKA_SOURCE_DIR}/lz4hc.c +# ${RDKAFKA_SOURCE_DIR}/rdxxhash.c +# ${RDKAFKA_SOURCE_DIR}/regexp.c ${RDKAFKA_SOURCE_DIR}/rdaddr.c ${RDKAFKA_SOURCE_DIR}/rdavl.c ${RDKAFKA_SOURCE_DIR}/rdbuf.c ${RDKAFKA_SOURCE_DIR}/rdcrc32.c ${RDKAFKA_SOURCE_DIR}/rddl.c + ${RDKAFKA_SOURCE_DIR}/rdfnv1a.c ${RDKAFKA_SOURCE_DIR}/rdhdrhistogram.c ${RDKAFKA_SOURCE_DIR}/rdkafka.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_admin.c # looks optional ${RDKAFKA_SOURCE_DIR}/rdkafka_assignor.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_aux.c # looks optional ${RDKAFKA_SOURCE_DIR}/rdkafka_background.c ${RDKAFKA_SOURCE_DIR}/rdkafka_broker.c ${RDKAFKA_SOURCE_DIR}/rdkafka_buf.c ${RDKAFKA_SOURCE_DIR}/rdkafka_cert.c ${RDKAFKA_SOURCE_DIR}/rdkafka_cgrp.c ${RDKAFKA_SOURCE_DIR}/rdkafka_conf.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_coord.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_error.c ${RDKAFKA_SOURCE_DIR}/rdkafka_event.c ${RDKAFKA_SOURCE_DIR}/rdkafka_feature.c ${RDKAFKA_SOURCE_DIR}/rdkafka_idempotence.c @@ -24,6 +34,7 @@ set(SRCS ${RDKAFKA_SOURCE_DIR}/rdkafka_metadata.c ${RDKAFKA_SOURCE_DIR}/rdkafka_metadata_cache.c ${RDKAFKA_SOURCE_DIR}/rdkafka_mock.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_mock_cgrp.c ${RDKAFKA_SOURCE_DIR}/rdkafka_mock_handlers.c ${RDKAFKA_SOURCE_DIR}/rdkafka_msg.c ${RDKAFKA_SOURCE_DIR}/rdkafka_msgset_reader.c @@ -38,9 +49,11 @@ set(SRCS ${RDKAFKA_SOURCE_DIR}/rdkafka_request.c ${RDKAFKA_SOURCE_DIR}/rdkafka_roundrobin_assignor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl.c +# ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_cyrus.c # needed to support Kerberos, requires cyrus-sasl ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_oauthbearer.c ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_plain.c ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_scram.c +# ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_win32.c ${RDKAFKA_SOURCE_DIR}/rdkafka_ssl.c ${RDKAFKA_SOURCE_DIR}/rdkafka_subscription.c ${RDKAFKA_SOURCE_DIR}/rdkafka_timer.c @@ -48,6 +61,7 @@ set(SRCS ${RDKAFKA_SOURCE_DIR}/rdkafka_transport.c ${RDKAFKA_SOURCE_DIR}/rdkafka_interceptor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_header.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_txnmgr.c ${RDKAFKA_SOURCE_DIR}/rdlist.c ${RDKAFKA_SOURCE_DIR}/rdlog.c ${RDKAFKA_SOURCE_DIR}/rdmurmur2.c diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index ae70bc8c594..786e6620eac 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -80,7 +80,9 @@ RUN apt-get --allow-unauthenticated update -y \ pigz \ moreutils \ libcctz-dev \ - libldap2-dev + libldap2-dev \ + libsasl2-dev \ + heimdal-multidev diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index e51efadf653..3aff49bf5a1 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -32,8 +32,8 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/lib/llvm-9/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; \ - if [ -n $USE_DATABASE_ATOMIC ] && [ $USE_DATABASE_ATOMIC -eq 1 ]; then ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/; fi; \ - if [ -n $USE_DATABASE_ATOMIC ] && [ $USE_DATABASE_ATOMIC -eq 1 ]; then ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/; fi; \ + if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/; fi; \ + if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/; fi; \ echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ echo "TSAN_SYMBOLIZER_PATH=/usr/lib/llvm-8/bin/llvm-symbolizer" >> /etc/environment; \ echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 35a8a5a9d3d..41a53f8a3f5 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -78,9 +78,9 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/; \ - if [ -n $USE_POLYMORPHIC_PARTS ] && [ $USE_POLYMORPHIC_PARTS -eq 1 ]; then ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/; fi; \ - if [ -n $USE_DATABASE_ATOMIC ] && [ $USE_DATABASE_ATOMIC -eq 1 ]; then ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/; fi; \ - if [ -n $USE_DATABASE_ATOMIC ] && [ $USE_DATABASE_ATOMIC -eq 1 ]; then ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/; fi; \ + if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/; fi; \ + if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/; fi; \ + if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/; fi; \ ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml; \ service zookeeper start; sleep 5; \ service clickhouse-server start && sleep 5 && clickhouse-test --testname --shard --zookeeper $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 93acf3cae7a..5961c701283 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -586,11 +586,11 @@ If the table doesn’t exist, ClickHouse will create it. If the structure of the ``` -## query\_thread\_log {#server_configuration_parameters-query-thread-log} +## query\_thread\_log {#server_configuration_parameters-query_thread_log} Setting for logging threads of queries received with the [log\_query\_threads=1](../settings/settings.md#settings-log-query-threads) setting. -Queries are logged in the [system.query\_thread\_log](../../operations/system-tables.md#system_tables-query-thread-log) table, not in a separate file. You can change the name of the table in the `table` parameter (see below). +Queries are logged in the [system.query\_thread\_log](../../operations/system-tables.md#system_tables-query_thread_log) table, not in a separate file. You can change the name of the table in the `table` parameter (see below). Use the following parameters to configure logging: diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7103819e499..880f0ffedb1 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -598,7 +598,7 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' Setting up query threads logging. -Queries’ threads runned by ClickHouse with this setup are logged according to the rules in the [query\_thread\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) server configuration parameter. +Queries’ threads runned by ClickHouse with this setup are logged according to the rules in the [query\_thread\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter. Example: diff --git a/docs/en/operations/system-tables.md b/docs/en/operations/system-tables.md index cc5456f5324..f364d4e3068 100644 --- a/docs/en/operations/system-tables.md +++ b/docs/en/operations/system-tables.md @@ -5,7 +5,7 @@ toc_title: System Tables # System Tables {#system-tables} -## Introduction +## Introduction {#system-tables-introduction} System tables provide information about: @@ -18,9 +18,12 @@ System tables: - Available only for reading data. - Can't be dropped or altered, but can be detached. -The `metric_log`, `query_log`, `query_thread_log`, `trace_log` system tables store data in a storage filesystem. Other system tables store their data in RAM. ClickHouse server creates such system tables at the start. +Most of system tables store their data in RAM. ClickHouse server creates such system tables at the start. -### Sources of System Metrics +The [metric_log](#system_tables-metric_log), [query_log](#system_tables-query_log), [query_thread_log](#system_tables-query_thread_log), [trace_log](#system_tables-trace_log) system tables store data in a storage filesystem. You can alter them or remove from a disk manually. If you remove one of that tables from a disk, the ClickHouse server creates the table again at the time of the next recording. A storage period for these tables is not limited, and ClickHouse server doesn't delete their data automatically. You need to organize removing of outdated logs by yourself. For example, you can use [TTL](../sql-reference/statements/alter.md#manipulations-with-table-ttl) settings for removing outdated log records. + + +### Sources of System Metrics {#system-tables-sources-of-system-metrics} For collecting system metrics ClickHouse server uses: @@ -587,97 +590,150 @@ Columns: - `source_file` (LowCardinality(String)) — Source file from which the logging was done. - `source_line` (UInt64) — Source line from which the logging was done. -## system.query\_log {#system_tables-query_log} +## system.query_log {#system_tables-query_log} -Contains information about execution of queries. For each query, you can see processing start time, duration of processing, error messages and other information. +Contains information about executed queries, for example, start time, duration of processing, error messages. !!! note "Note" The table doesn’t contain input data for `INSERT` queries. -ClickHouse creates this table only if the [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server parameter is specified. This parameter sets the logging rules, such as the logging interval or the name of the table the queries will be logged in. +You can change settings of queries logging in the [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) section of the server configuration. -To enable query logging, set the [log\_queries](settings/settings.md#settings-log-queries) parameter to 1. For details, see the [Settings](settings/settings.md) section. +You can disable queries logging by setting [log_queries = 0](settings/settings.md#settings-log-queries). We don't recommend to turn off logging because information in this table is important for solving issues. + +The flushing period of logs is set in `flush_interval_milliseconds` parameter of the [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server settings section. To force flushing logs, use the [SYSTEM FLUSH LOGS](../sql-reference/statements/system.md#query_language-system-flush_logs) query. + +ClickHouse doesn't delete logs from the table automatically. See [Introduction](#system-tables-introduction) for more details. The `system.query_log` table registers two kinds of queries: 1. Initial queries that were run directly by the client. 2. Child queries that were initiated by other queries (for distributed query execution). For these types of queries, information about the parent queries is shown in the `initial_*` columns. +Each query creates one or two rows in the `query_log` table, depending on the status (see the `type` column) of the query: + +1. If the query execution was successful, two rows with the `QueryStart` and `QueryFinish` types are created . +2. If an error occurred during query processing, two events with the `QueryStart` and `ExceptionWhileProcessing` types are created . +3. If an error occurred before launching the query, a single event with the `ExceptionBeforeStart` type is created. + Columns: -- `type` (`Enum8`) — Type of event that occurred when executing the query. Values: +- `type` ([Enum8](../sql-reference/data-types/enum.md)) — Type of an event that occurred when executing the query. Values: - `'QueryStart' = 1` — Successful start of query execution. - `'QueryFinish' = 2` — Successful end of query execution. - `'ExceptionBeforeStart' = 3` — Exception before the start of query execution. - `'ExceptionWhileProcessing' = 4` — Exception during the query execution. -- `event_date` (Date) — Query starting date. -- `event_time` (DateTime) — Query starting time. -- `query_start_time` (DateTime) — Start time of query execution. -- `query_duration_ms` (UInt64) — Duration of query execution. -- `read_rows` (UInt64) — Number of read rows. -- `read_bytes` (UInt64) — Number of read bytes. -- `written_rows` (UInt64) — For `INSERT` queries, the number of written rows. For other queries, the column value is 0. -- `written_bytes` (UInt64) — For `INSERT` queries, the number of written bytes. For other queries, the column value is 0. -- `result_rows` (UInt64) — Number of rows in the result. -- `result_bytes` (UInt64) — Number of bytes in the result. -- `memory_usage` (UInt64) — Memory consumption by the query. -- `query` (String) — Query string. -- `exception` (String) — Exception message. -- `stack_trace` (String) — Stack trace (a list of methods called before the error occurred). An empty string, if the query is completed successfully. -- `is_initial_query` (UInt8) — Query type. Possible values: +- `event_date` ([Date](../sql-reference/data-types/date.md)) — Query starting date. +- `event_time` ([DateTime](../sql-reference/data-types/datetime.md)) — Query starting time. +- `query_start_time` ([DateTime](../sql-reference/data-types/datetime.md)) — Start time of query execution. +- `query_duration_ms` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution in milliseconds. +- `read_rows` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Total number or rows read from all tables and table functions participated in query. It includes usual subqueries, subqueries for `IN` and `JOIN`. For distributed queries `read_rows` includes the total number of rows read at all replicas. Each replica sends it's `read_rows` value, and the server-initiator of the query summarize all received and local values. The cache volumes doesn't affect this value. +- `read_bytes` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Total number or bytes read from all tables and table functions participated in query. It includes usual subqueries, subqueries for `IN` and `JOIN`. For distributed queries `read_bytes` includes the total number of rows read at all replicas. Each replica sends it's `read_bytes` value, and the server-initiator of the query summarize all received and local values. The cache volumes doesn't affect this value. +- `written_rows` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — For `INSERT` queries, the number of written rows. For other queries, the column value is 0. +- `written_bytes` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — For `INSERT` queries, the number of written bytes. For other queries, the column value is 0. +- `result_rows` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of rows in a result of the `SELECT` query, or a number of rows in the `INSERT` query. +- `result_bytes` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — RAM volume in bytes used to store a query result. +- `memory_usage` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Memory consumption by the query. +- `query` ([String](../sql-reference/data-types/string.md)) — Query string. +- `exception` ([String](../sql-reference/data-types/string.md)) — Exception message. +- `exception_code` ([Int32](../sql-reference/data-types/int-uint.md)) — Code of an exception. +- `stack_trace` ([String](../sql-reference/data-types/string.md)) — [Stack trace](https://en.wikipedia.org/wiki/Stack_trace). An empty string, if the query was completed successfully. +- `is_initial_query` ([UInt8](../sql-reference/data-types/int-uint.md)) — Query type. Possible values: - 1 — Query was initiated by the client. - - 0 — Query was initiated by another query for distributed query execution. -- `user` (String) — Name of the user who initiated the current query. -- `query_id` (String) — ID of the query. -- `address` (IPv6) — IP address that was used to make the query. -- `port` (UInt16) — The client port that was used to make the query. -- `initial_user` (String) — Name of the user who ran the initial query (for distributed query execution). -- `initial_query_id` (String) — ID of the initial query (for distributed query execution). -- `initial_address` (IPv6) — IP address that the parent query was launched from. -- `initial_port` (UInt16) — The client port that was used to make the parent query. -- `interface` (UInt8) — Interface that the query was initiated from. Possible values: + - 0 — Query was initiated by another query as part of distributed query execution. +- `user` ([String](../sql-reference/data-types/string.md)) — Name of the user who initiated the current query. +- `query_id` ([String](../sql-reference/data-types/string.md)) — ID of the query. +- `address` ([IPv6](../sql-reference/data-types/domains/ipv6.md)) — IP address that was used to make the query. +- `port` ([UInt16](../sql-reference/data-types/int-uint.md)) — The client port that was used to make the query. +- `initial_user` ([String](../sql-reference/data-types/string.md)) — Name of the user who ran the initial query (for distributed query execution). +- `initial_query_id` ([String](../sql-reference/data-types/string.md)) — ID of the initial query (for distributed query execution). +- `initial_address` ([IPv6](../sql-reference/data-types/domains/ipv6.md)) — IP address that the parent query was launched from. +- `initial_port` ([UInt16](../sql-reference/data-types/int-uint.md)) — The client port that was used to make the parent query. +- `interface` ([UInt8](../sql-reference/data-types/int-uint.md)) — Interface that the query was initiated from. Possible values: - 1 — TCP. - 2 — HTTP. -- `os_user` (String) — OS’s username who runs [clickhouse-client](../interfaces/cli.md). -- `client_hostname` (String) — Hostname of the client machine where the [clickhouse-client](../interfaces/cli.md) or another TCP client is run. -- `client_name` (String) — The [clickhouse-client](../interfaces/cli.md) or another TCP client name. -- `client_revision` (UInt32) — Revision of the [clickhouse-client](../interfaces/cli.md) or another TCP client. -- `client_version_major` (UInt32) — Major version of the [clickhouse-client](../interfaces/cli.md) or another TCP client. -- `client_version_minor` (UInt32) — Minor version of the [clickhouse-client](../interfaces/cli.md) or another TCP client. -- `client_version_patch` (UInt32) — Patch component of the [clickhouse-client](../interfaces/cli.md) or another TCP client version. +- `os_user` ([String](../sql-reference/data-types/string.md)) — Operating system username who runs [clickhouse-client](../interfaces/cli.md). +- `client_hostname` ([String](../sql-reference/data-types/string.md)) — Hostname of the client machine where the [clickhouse-client](../interfaces/cli.md) or another TCP client is run. +- `client_name` ([String](../sql-reference/data-types/string.md)) — The [clickhouse-client](../interfaces/cli.md) or another TCP client name. +- `client_revision` ([UInt32](../sql-reference/data-types/int-uint.md)) — Revision of the [clickhouse-client](../interfaces/cli.md) or another TCP client. +- `client_version_major` ([UInt32](../sql-reference/data-types/int-uint.md)) — Major version of the [clickhouse-client](../interfaces/cli.md) or another TCP client. +- `client_version_minor` ([UInt32](../sql-reference/data-types/int-uint.md)) — Minor version of the [clickhouse-client](../interfaces/cli.md) or another TCP client. +- `client_version_patch` ([UInt32](../sql-reference/data-types/int-uint.md)) — Patch component of the [clickhouse-client](../interfaces/cli.md) or another TCP client version. - `http_method` (UInt8) — HTTP method that initiated the query. Possible values: - 0 — The query was launched from the TCP interface. - 1 — `GET` method was used. - 2 — `POST` method was used. -- `http_user_agent` (String) — The `UserAgent` header passed in the HTTP request. -- `quota_key` (String) — The “quota key” specified in the [quotas](quotas.md) setting (see `keyed`). -- `revision` (UInt32) — ClickHouse revision. -- `thread_numbers` (Array(UInt32)) — Number of threads that are participating in query execution. -- `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics. The description of them could be found in the table [system.events](#system_tables-events) -- `ProfileEvents.Values` (Array(UInt64)) — Values of metrics that are listed in the `ProfileEvents.Names` column. -- `Settings.Names` (Array(String)) — Names of settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parameter to 1. -- `Settings.Values` (Array(String)) — Values of settings that are listed in the `Settings.Names` column. +- `http_user_agent` ([String](../sql-reference/data-types/string.md)) — The `UserAgent` header passed in the HTTP request. +- `quota_key` ([String](../sql-reference/data-types/string.md)) — The “quota key” specified in the [quotas](quotas.md) setting (see `keyed`). +- `revision` ([UInt32](../sql-reference/data-types/int-uint.md)) — ClickHouse revision. +- `thread_numbers` ([Array(UInt32)](../sql-reference/data-types/array.md)) — Number of threads that are participating in query execution. +- `ProfileEvents.Names` ([Array(String)](../sql-reference/data-types/array.md)) — Counters that measure different metrics. The description of them could be found in the table [system.events](#system_tables-events) +- `ProfileEvents.Values` ([Array(UInt64)](../sql-reference/data-types/array.md)) — Values of metrics that are listed in the `ProfileEvents.Names` column. +- `Settings.Names` ([Array(String)](../sql-reference/data-types/array.md)) — Names of settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parameter to 1. +- `Settings.Values` ([Array(String)](../sql-reference/data-types/array.md)) — Values of settings that are listed in the `Settings.Names` column. -Each query creates one or two rows in the `query_log` table, depending on the status of the query: +**Example** -1. If the query execution is successful, two events with types 1 and 2 are created (see the `type` column). -2. If an error occurred during query processing, two events with types 1 and 4 are created. -3. If an error occurred before launching the query, a single event with type 3 is created. +``` sql +SELECT * FROM system.query_log LIMIT 1 FORMAT Vertical; +``` -By default, logs are added to the table at intervals of 7.5 seconds. You can set this interval in the [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server setting (see the `flush_interval_milliseconds` parameter). To flush the logs forcibly from the memory buffer into the table, use the `SYSTEM FLUSH LOGS` query. +``` text +Row 1: +────── +type: QueryStart +event_date: 2020-05-13 +event_time: 2020-05-13 14:02:28 +query_start_time: 2020-05-13 14:02:28 +query_duration_ms: 0 +read_rows: 0 +read_bytes: 0 +written_rows: 0 +written_bytes: 0 +result_rows: 0 +result_bytes: 0 +memory_usage: 0 +query: SELECT 1 +exception_code: 0 +exception: +stack_trace: +is_initial_query: 1 +user: default +query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a +address: ::ffff:127.0.0.1 +port: 57720 +initial_user: default +initial_query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a +initial_address: ::ffff:127.0.0.1 +initial_port: 57720 +interface: 1 +os_user: bayonet +client_hostname: clickhouse.ru-central1.internal +client_name: ClickHouse client +client_revision: 54434 +client_version_major: 20 +client_version_minor: 4 +client_version_patch: 1 +http_method: 0 +http_user_agent: +quota_key: +revision: 54434 +thread_ids: [] +ProfileEvents.Names: [] +ProfileEvents.Values: [] +Settings.Names: ['use_uncompressed_cache','load_balancing','log_queries','max_memory_usage'] +Settings.Values: ['0','random','1','10000000000'] -When the table is deleted manually, it will be automatically created on the fly. Note that all the previous logs will be deleted. +``` +**See Also** -!!! note "Note" - The storage period for logs is unlimited. Logs aren’t automatically deleted from the table. You need to organize the removal of outdated logs yourself. +- [system.query_thread_log](#system_tables-query_thread_log) — This table contains information about each query execution thread. -You can specify an arbitrary partitioning key for the `system.query_log` table in the [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server setting (see the `partition_by` parameter). - -## system.query\_thread\_log {#system_tables-query-thread-log} +## system.query_thread_log {#system_tables-query_thread_log} The table contains information about each query execution thread. -ClickHouse creates this table only if the [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) server parameter is specified. This parameter sets the logging rules, such as the logging interval or the name of the table the queries will be logged in. +ClickHouse creates this table only if the [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server parameter is specified. This parameter sets the logging rules, such as the logging interval or the name of the table the queries will be logged in. To enable query logging, set the [log\_query\_threads](settings/settings.md#settings-log-query-threads) parameter to 1. For details, see the [Settings](settings/settings.md) section. @@ -729,14 +785,14 @@ Columns: - `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics for this thread. The description of them could be found in the table [system.events](#system_tables-events) - `ProfileEvents.Values` (Array(UInt64)) — Values of metrics for this thread that are listed in the `ProfileEvents.Names` column. -By default, logs are added to the table at intervals of 7.5 seconds. You can set this interval in the [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) server setting (see the `flush_interval_milliseconds` parameter). To flush the logs forcibly from the memory buffer into the table, use the `SYSTEM FLUSH LOGS` query. +By default, logs are added to the table at intervals of 7.5 seconds. You can set this interval in the [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server setting (see the `flush_interval_milliseconds` parameter). To flush the logs forcibly from the memory buffer into the table, use the `SYSTEM FLUSH LOGS` query. When the table is deleted manually, it will be automatically created on the fly. Note that all the previous logs will be deleted. !!! note "Note" The storage period for logs is unlimited. Logs aren’t automatically deleted from the table. You need to organize the removal of outdated logs yourself. -You can specify an arbitrary partitioning key for the `system.query_thread_log` table in the [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) server setting (see the `partition_by` parameter). +You can specify an arbitrary partitioning key for the `system.query_thread_log` table in the [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server setting (see the `partition_by` parameter). ## system.trace\_log {#system_tables-trace_log} diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index c664580c659..ccde7a945ac 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -574,11 +574,11 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat ``` -## query\_thread\_log {#server_configuration_parameters-query-thread-log} +## query\_thread\_log {#server_configuration_parameters-query_thread_log} Настройка логирования потоков выполнения запросов, принятых с настройкой [log\_query\_threads=1](../settings/settings.md#settings-log-query-threads). -Запросы логируются не в отдельный файл, а в системную таблицу [system.query\_thread\_log](../../operations/server-configuration-parameters/settings.md#system_tables-query-thread-log). Вы можете изменить название этой таблицы в параметре `table` (см. ниже). +Запросы логируются не в отдельный файл, а в системную таблицу [system.query\_thread\_log](../../operations/server-configuration-parameters/settings.md#system_tables-query_thread_log). Вы можете изменить название этой таблицы в параметре `table` (см. ниже). При настройке логирования используются следующие параметры: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 56c3042bfa3..4dd43e9607b 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -536,7 +536,7 @@ log_queries=1 Установка логирования информации о потоках выполнения запроса. -Лог информации о потоках выполнения запросов, переданных в ClickHouse с этой установкой, записывается согласно правилам конфигурационного параметра сервера [query\_thread\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log). +Лог информации о потоках выполнения запросов, переданных в ClickHouse с этой установкой, записывается согласно правилам конфигурационного параметра сервера [query\_thread\_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log). Пример: diff --git a/docs/ru/operations/system-tables.md b/docs/ru/operations/system-tables.md index ae0e67a4515..8954a484560 100644 --- a/docs/ru/operations/system-tables.md +++ b/docs/ru/operations/system-tables.md @@ -1,4 +1,7 @@ -# Системные таблицы {#sistemnye-tablitsy} +# Системные таблицы {#system-tables} + + +## Введение {#system-tables-introduction} Системные таблицы используются для реализации части функциональности системы, а также предоставляют доступ к информации о работе системы. Вы не можете удалить системную таблицу (хотя можете сделать DETACH). @@ -544,182 +547,156 @@ CurrentMetric_ReplicatedChecks: 0 - `source_file` (LowCardinality(String)) — Исходный файл, из которого была сделана запись. - `source_line` (UInt64) — Исходная строка, из которой была сделана запись. -## system.query\_log {#system_tables-query_log} +## system.query_log {#system_tables-query_log} -Содержит информацию о выполнении запросов. Для каждого запроса вы можете увидеть время начала обработки, продолжительность обработки, сообщения об ошибках и другую информацию. +Содержит информацию о выполняемых запросах, например, время начала обработки, продолжительность обработки, сообщения об ошибках. !!! note "Внимание" Таблица не содержит входных данных для запросов `INSERT`. -ClickHouse создаёт таблицу только в том случае, когда установлен конфигурационный параметр сервера [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log). Параметр задаёт правила ведения лога, такие как интервал логирования или имя таблицы, в которую будут логгироваться запросы. +Настойки логгирования можно изменить в секции серверной конфигурации [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log). -Чтобы включить логирование, задайте значение параметра [log\_queries](settings/settings.md#settings-log-queries) равным 1. Подробности смотрите в разделе [Настройки](settings/settings.md#settings). +Можно отключить логгирование настройкой [log_queries = 0](settings/settings.md#settings-log-queries). По-возможности, не отключайте логгирование, поскольку информация из таблицы важна при решении проблем. + +Период сброса логов в таблицу задаётся параметром `flush_interval_milliseconds` в конфигурационной секции [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log). Чтобы принудительно записать логи из буффера памяти в таблицу, используйте запрос [SYSTEM FLUSH LOGS](../sql-reference/statements/system.md#query_language-system-flush_logs). + +ClickHouse не удаляет логи из таблица автоматически. Смотрите [Введение](#system-tables-introduction). + +Можно указать произвольный ключ партиционирования для таблицы `system.query_log` в конфигурации [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) (параметр `partition_by`). + + + +Если таблицу удалить вручную, она создается заново автоматически «на лету». При этом все логи на момент удаления таблицы будут убраны. Таблица `system.query_log` содержит информацию о двух видах запросов: 1. Первоначальные запросы, которые были выполнены непосредственно клиентом. 2. Дочерние запросы, инициированные другими запросами (для выполнения распределенных запросов). Для дочерних запросов информация о первоначальном запросе содержится в столбцах `initial_*`. +В зависимости от статуса (столбец `type`) каждый запрос создаёт одну или две строки в таблице `query_log`: + +1. Если запрос выполнен успешно, создаются два события типа `QueryStart` и `QueryFinish`. +2. Если во время обработки запроса возникла ошибка, создаются два события с типами `QueryStart` и `ExceptionWhileProcessing`. +3. Если ошибка произошла ещё до запуска запроса, создается одно событие с типом `ExceptionBeforeStart`. + Столбцы: -- `type` (`Enum8`) — тип события, произошедшего при выполнении запроса. Значения: +- `type` ([Enum8](../sql-reference/data-types/enum.md)) — тип события, произошедшего при выполнении запроса. Значения: - `'QueryStart' = 1` — успешное начало выполнения запроса. - `'QueryFinish' = 2` — успешное завершение выполнения запроса. - `'ExceptionBeforeStart' = 3` — исключение перед началом обработки запроса. - `'ExceptionWhileProcessing' = 4` — исключение во время обработки запроса. -- `event_date` (Date) — дата начала запроса. -- `event_time` (DateTime) — время начала запроса. -- `query_start_time` (DateTime) — время начала обработки запроса. -- `query_duration_ms` (UInt64) — длительность обработки запроса. -- `read_rows` (UInt64) — количество прочитанных строк. -- `read_bytes` (UInt64) — количество прочитанных байтов. -- `written_rows` (UInt64) — количество записанных строк для запросов `INSERT`. Для других запросов, значение столбца 0. -- `written_bytes` (UInt64) — объём записанных данных в байтах для запросов `INSERT`. Для других запросов, значение столбца 0. -- `result_rows` (UInt64) — количество строк в результате. -- `result_bytes` (UInt64) — объём результата в байтах. -- `memory_usage` (UInt64) — потребление RAM запросом. -- `query` (String) — текст запроса. -- `exception` (String) — сообщение исключения, если запрос завершился по исключению. -- `stack_trace` (String) — трассировка (список функций, последовательно вызванных перед ошибкой). Пустая строка, если запрос успешно завершен. -- `is_initial_query` (UInt8) — вид запроса. Возможные значения: +- `event_date` ([Date](../sql-reference/data-types/date.md)) — дата начала запроса. +- `event_time` ([DateTime](../sql-reference/data-types/datetime.md)) — время начала запроса. +- `query_start_time` ([DateTime](../sql-reference/data-types/datetime.md)) — время начала обработки запроса. +- `query_duration_ms` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — длительность выполнения запроса в миллисекундах. +- `read_rows` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Общее количество строк, считанных из всех таблиц и табличных функций, участвующих в запросе. Включает в себя обычные подзапросы, подзапросы для `IN` и `JOIN`. Для распределенных запросов `read_rows` включает в себя общее количество строк, прочитанных на всех репликах. Каждая реплика передает собственное значение `read_rows`, а сервер-инициатор запроса суммирует все полученные и локальные значения. Объемы кэша не учитываюся. +- `read_bytes` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Общее количество байтов, считанных из всех таблиц и табличных функций, участвующих в запросе. Включает в себя обычные подзапросы, подзапросы для `IN` и `JOIN`. Для распределенных запросов `read_bytes` включает в себя общее количество байтов, прочитанных на всех репликах. Каждая реплика передает собственное значение `read_bytes`, а сервер-инициатор запроса суммирует все полученные и локальные значения. Объемы кэша не учитываюся. +- `written_rows` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — количество записанных строк для запросов `INSERT`. Для других запросов, значение столбца 0. +- `written_bytes` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — объём записанных данных в байтах для запросов `INSERT`. Для других запросов, значение столбца 0. +- `result_rows` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — количество строк в результате запроса `SELECT` или количество строк в запросе `INSERT`. +- `result_bytes` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — объём RAM в байтах, использованный для хранения результата запроса. +- `memory_usage` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — потребление RAM запросом. +- `query` ([String](../sql-reference/data-types/string.md)) — текст запроса. +- `exception` ([String](../sql-reference/data-types/string.md)) — сообщение исключения, если запрос завершился по исключению. +- `exception_code` ([Int32](../sql-reference/data-types/int-uint.md)) — код исключения. +- `stack_trace` ([String](../sql-reference/data-types/string.md)) — [stack trace](https://en.wikipedia.org/wiki/Stack_trace). Пустая строка, если запрос успешно завершен. +- `is_initial_query` ([UInt8](../sql-reference/data-types/int-uint.md)) — вид запроса. Возможные значения: - 1 — запрос был инициирован клиентом. - - 0 — запрос был инициирован другим запросом при распределенном запросе. -- `user` (String) — пользователь, запустивший текущий запрос. -- `query_id` (String) — ID запроса. -- `address` (IPv6) — IP адрес, с которого пришел запрос. -- `port` (UInt16) — порт, с которого клиент сделал запрос -- `initial_user` (String) — пользователь, запустивший первоначальный запрос (для распределенных запросов). -- `initial_query_id` (String) — ID родительского запроса. -- `initial_address` (IPv6) — IP адрес, с которого пришел родительский запрос. -- `initial_port` (UInt16) — порт, с которого клиент сделал родительский запрос. -- `interface` (UInt8) — интерфейс, с которого ушёл запрос. Возможные значения: + - 0 — запрос был инициирован другим запросом при выполнении распределенного запроса. +- `user` ([String](../sql-reference/data-types/string.md)) — пользователь, запустивший текущий запрос. +- `query_id` ([String](../sql-reference/data-types/string.md)) — ID запроса. +- `address` ([IPv6](../sql-reference/data-types/domains/ipv6.md)) — IP адрес, с которого пришел запрос. +- `port` ([UInt16](../sql-reference/data-types/int-uint.md)) — порт, с которого клиент сделал запрос +- `initial_user` ([String](../sql-reference/data-types/string.md)) — пользователь, запустивший первоначальный запрос (для распределенных запросов). +- `initial_query_id` ([String](../sql-reference/data-types/string.md)) — ID родительского запроса. +- `initial_address` ([IPv6](../sql-reference/data-types/domains/ipv6.md)) — IP адрес, с которого пришел родительский запрос. +- `initial_port` ([UInt16](../sql-reference/data-types/int-uint.md)) — порт, с которого клиент сделал родительский запрос. +- `interface` ([UInt8](../sql-reference/data-types/int-uint.md)) — интерфейс, с которого ушёл запрос. Возможные значения: - 1 — TCP. - 2 — HTTP. -- `os_user` (String) — имя пользователя в OS, который запустил [clickhouse-client](../interfaces/cli.md). -- `client_hostname` (String) — имя сервера, с которого присоединился [clickhouse-client](../interfaces/cli.md) или другой TCP клиент. -- `client_name` (String) — [clickhouse-client](../interfaces/cli.md) или другой TCP клиент. -- `client_revision` (UInt32) — ревизия [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. -- `client_version_major` (UInt32) — старшая версия [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. -- `client_version_minor` (UInt32) — младшая версия [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. -- `client_version_patch` (UInt32) — патч [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. -- `http_method` (UInt8) — HTTP метод, инициировавший запрос. Возможные значения: +- `os_user` ([String](../sql-reference/data-types/string.md)) — имя пользователя операционной системы, который запустил [clickhouse-client](../interfaces/cli.md). +- `client_hostname` ([String](../sql-reference/data-types/string.md)) — имя сервера, с которого присоединился [clickhouse-client](../interfaces/cli.md) или другой TCP клиент. +- `client_name` ([String](../sql-reference/data-types/string.md)) — [clickhouse-client](../interfaces/cli.md) или другой TCP клиент. +- `client_revision` ([UInt32](../sql-reference/data-types/int-uint.md)) — ревизия [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. +- `client_version_major` ([UInt32](../sql-reference/data-types/int-uint.md)) — старшая версия [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. +- `client_version_minor` ([UInt32](../sql-reference/data-types/int-uint.md)) — младшая версия [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. +- `client_version_patch` ([UInt32](../sql-reference/data-types/int-uint.md)) — патч [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. +- `http_method` ([UInt8](../sql-reference/data-types/int-uint.md)) — HTTP метод, инициировавший запрос. Возможные значения: - 0 — запрос запущен с интерфейса TCP. - 1 — `GET`. - 2 — `POST`. -- `http_user_agent` (String) — HTTP заголовок `UserAgent`. -- `quota_key` (String) — «ключ квоты» из настроек [квот](quotas.md) (см. `keyed`). -- `revision` (UInt32) — ревизия ClickHouse. -- `thread_numbers` (Array(UInt32)) — количество потоков, участвующих в обработке запросов. -- `ProfileEvents.Names` (Array(String)) — Счетчики для изменения различных метрик. Описание метрик можно получить из таблицы [system.events](#system_tables-events)(\#system\_tables-events -- `ProfileEvents.Values` (Array(UInt64)) — метрики, перечисленные в столбце `ProfileEvents.Names`. -- `Settings.Names` (Array(String)) — имена настроек, которые меняются, когда клиент выполняет запрос. Чтобы разрешить логирование изменений настроек, установите параметр `log_query_settings` равным 1. -- `Settings.Values` (Array(String)) — Значения настроек, которые перечислены в столбце `Settings.Names`. +- `http_user_agent` ([String](../sql-reference/data-types/string.md)) — HTTP заголовок `UserAgent`. +- `quota_key` ([String](../sql-reference/data-types/string.md)) — «ключ квоты» из настроек [квот](quotas.md) (см. `keyed`). +- `revision` ([UInt32](../sql-reference/data-types/int-uint.md)) — ревизия ClickHouse. +- `thread_numbers` ([Array(UInt32)](../sql-reference/data-types/array.md)) — количество потоков, участвующих в обработке запросов. +- `ProfileEvents.Names` ([Array(String)](../sql-reference/data-types/array.md)) — Счетчики для изменения различных метрик. Описание метрик можно получить из таблицы [system.events](#system_tables-events)(\#system\_tables-events +- `ProfileEvents.Values` ([Array(UInt64)](../sql-reference/data-types/array.md)) — метрики, перечисленные в столбце `ProfileEvents.Names`. +- `Settings.Names` ([Array(String)](../sql-reference/data-types/array.md)) — имена настроек, которые меняются, когда клиент выполняет запрос. Чтобы разрешить логирование изменений настроек, установите параметр `log_query_settings` равным 1. +- `Settings.Values` ([Array(String)](../sql-reference/data-types/array.md)) — Значения настроек, которые перечислены в столбце `Settings.Names`. -Каждый запрос создаёт одну или две строки в таблице `query_log`, в зависимости от статуса запроса: +**Пример** -1. Если запрос выполнен успешно, создаются два события типа 1 и 2 (смотрите столбец `type`). -2. Если во время обработки запроса произошла ошибка, создаются два события с типами 1 и 4. -3. Если ошибка произошла до запуска запроса, создается одно событие с типом 3. +``` sql +SELECT * FROM system.query_log LIMIT 1 FORMAT Vertical; +``` -По умолчанию, строки добавляются в таблицу логирования с интервалом в 7,5 секунд. Можно задать интервал в конфигурационном параметре сервера [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) (смотрите параметр `flush_interval_milliseconds`). Чтобы принудительно записать логи из буффера памяти в таблицу, используйте запрос `SYSTEM FLUSH LOGS`. +``` text +Row 1: +────── +type: QueryStart +event_date: 2020-05-13 +event_time: 2020-05-13 14:02:28 +query_start_time: 2020-05-13 14:02:28 +query_duration_ms: 0 +read_rows: 0 +read_bytes: 0 +written_rows: 0 +written_bytes: 0 +result_rows: 0 +result_bytes: 0 +memory_usage: 0 +query: SELECT 1 +exception_code: 0 +exception: +stack_trace: +is_initial_query: 1 +user: default +query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a +address: ::ffff:127.0.0.1 +port: 57720 +initial_user: default +initial_query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a +initial_address: ::ffff:127.0.0.1 +initial_port: 57720 +interface: 1 +os_user: bayonet +client_hostname: clickhouse.ru-central1.internal +client_name: ClickHouse client +client_revision: 54434 +client_version_major: 20 +client_version_minor: 4 +client_version_patch: 1 +http_method: 0 +http_user_agent: +quota_key: +revision: 54434 +thread_ids: [] +ProfileEvents.Names: [] +ProfileEvents.Values: [] +Settings.Names: ['use_uncompressed_cache','load_balancing','log_queries','max_memory_usage'] +Settings.Values: ['0','random','1','10000000000'] -Если таблицу удалить вручную, она пересоздастся автоматически «на лету». При этом все логи на момент удаления таблицы будут удалены. +``` +**Смотрите также** -!!! note "Примечание" - Срок хранения логов не ограничен. Логи не удаляются из таблицы автоматически. Вам необходимо самостоятельно организовать удаление устаревших логов. +- [system.query_thread_log](#system_tables-query_thread_log) — в этой таблице содержится информация о цепочке каждого выполненного запроса. -Можно указать произвольный ключ партиционирования для таблицы `system.query_log` в конфигурации [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) (параметр `partition_by`). - -## system.query\_log {#system_tables-query_log} - -Contains information about execution of queries. For each query, you can see processing start time, duration of processing, error messages and other information. - -!!! note "Note" - The table doesn’t contain input data for `INSERT` queries. - -ClickHouse creates this table only if the [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server parameter is specified. This parameter sets the logging rules, such as the logging interval or the name of the table the queries will be logged in. - -To enable query logging, set the [log\_queries](settings/settings.md#settings-log-queries) parameter to 1. For details, see the [Settings](settings/settings.md) section. - -The `system.query_log` table registers two kinds of queries: - -1. Initial queries that were run directly by the client. -2. Child queries that were initiated by other queries (for distributed query execution). For these types of queries, information about the parent queries is shown in the `initial_*` columns. - -Columns: - -- `type` (`Enum8`) — Type of event that occurred when executing the query. Values: - - `'QueryStart' = 1` — Successful start of query execution. - - `'QueryFinish' = 2` — Successful end of query execution. - - `'ExceptionBeforeStart' = 3` — Exception before the start of query execution. - - `'ExceptionWhileProcessing' = 4` — Exception during the query execution. -- `event_date` (Date) — Query starting date. -- `event_time` (DateTime) — Query starting time. -- `query_start_time` (DateTime) — Start time of query execution. -- `query_duration_ms` (UInt64) — Duration of query execution. -- `read_rows` (UInt64) — Number of read rows. -- `read_bytes` (UInt64) — Number of read bytes. -- `written_rows` (UInt64) — For `INSERT` queries, the number of written rows. For other queries, the column value is 0. -- `written_bytes` (UInt64) — For `INSERT` queries, the number of written bytes. For other queries, the column value is 0. -- `result_rows` (UInt64) — Number of rows in the result. -- `result_bytes` (UInt64) — Number of bytes in the result. -- `memory_usage` (UInt64) — Memory consumption by the query. -- `query` (String) — Query string. -- `exception` (String) — Exception message. -- `stack_trace` (String) — Stack trace (a list of methods called before the error occurred). An empty string, if the query is completed successfully. -- `is_initial_query` (UInt8) — Query type. Possible values: - - 1 — Query was initiated by the client. - - 0 — Query was initiated by another query for distributed query execution. -- `user` (String) — Name of the user who initiated the current query. -- `query_id` (String) — ID of the query. -- `address` (IPv6) — IP address that was used to make the query. -- `port` (UInt16) — The client port that was used to make the query. -- `initial_user` (String) — Name of the user who ran the initial query (for distributed query execution). -- `initial_query_id` (String) — ID of the initial query (for distributed query execution). -- `initial_address` (IPv6) — IP address that the parent query was launched from. -- `initial_port` (UInt16) — The client port that was used to make the parent query. -- `interface` (UInt8) — Interface that the query was initiated from. Possible values: - - 1 — TCP. - - 2 — HTTP. -- `os_user` (String) — OS’s username who runs [clickhouse-client](../interfaces/cli.md). -- `client_hostname` (String) — Hostname of the client machine where the [clickhouse-client](../interfaces/cli.md) or another TCP client is run. -- `client_name` (String) — The [clickhouse-client](../interfaces/cli.md) or another TCP client name. -- `client_revision` (UInt32) — Revision of the [clickhouse-client](../interfaces/cli.md) or another TCP client. -- `client_version_major` (UInt32) — Major version of the [clickhouse-client](../interfaces/cli.md) or another TCP client. -- `client_version_minor` (UInt32) — Minor version of the [clickhouse-client](../interfaces/cli.md) or another TCP client. -- `client_version_patch` (UInt32) — Patch component of the [clickhouse-client](../interfaces/cli.md) or another TCP client version. -- `http_method` (UInt8) — HTTP method that initiated the query. Possible values: - - 0 — The query was launched from the TCP interface. - - 1 — `GET` method was used. - - 2 — `POST` method was used. -- `http_user_agent` (String) — The `UserAgent` header passed in the HTTP request. -- `quota_key` (String) — The «quota key» specified in the [quotas](quotas.md) setting (see `keyed`). -- `revision` (UInt32) — ClickHouse revision. -- `thread_numbers` (Array(UInt32)) — Number of threads that are participating in query execution. -- `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics. The description of them could be found in the table [system.events](#system_tables-events) -- `ProfileEvents.Values` (Array(UInt64)) — Values of metrics that are listed in the `ProfileEvents.Names` column. -- `Settings.Names` (Array(String)) — Names of settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parameter to 1. -- `Settings.Values` (Array(String)) — Values of settings that are listed in the `Settings.Names` column. - -Each query creates one or two rows in the `query_log` table, depending on the status of the query: - -1. If the query execution is successful, two events with types 1 and 2 are created (see the `type` column). -2. If an error occurred during query processing, two events with types 1 and 4 are created. -3. If an error occurred before launching the query, a single event with type 3 is created. - -By default, logs are added to the table at intervals of 7.5 seconds. You can set this interval in the [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server setting (see the `flush_interval_milliseconds` parameter). To flush the logs forcibly from the memory buffer into the table, use the `SYSTEM FLUSH LOGS` query. - -When the table is deleted manually, it will be automatically created on the fly. Note that all the previous logs will be deleted. - -!!! note "Note" - The storage period for logs is unlimited. Logs aren’t automatically deleted from the table. You need to organize the removal of outdated logs yourself. - -You can specify an arbitrary partitioning key for the `system.query_log` table in the [query\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server setting (see the `partition_by` parameter). -\#\# system.query\_thread\_log {\#system\_tables-query-thread-log} +## system.query_thread_log {#system_tables-query_thread_log} Содержит информацию о каждом потоке выполняемых запросов. -ClickHouse создаёт таблицу только в том случае, когда установлен конфигурационный параметр сервера [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log). Параметр задаёт правила ведения лога, такие как интервал логирования или имя таблицы, в которую будут логгироваться запросы. +ClickHouse создаёт таблицу только в том случае, когда установлен конфигурационный параметр сервера [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log). Параметр задаёт правила ведения лога, такие как интервал логирования или имя таблицы, в которую будут логгироваться запросы. Чтобы включить логирование, задайте значение параметра [log\_query\_threads](settings/settings.md#settings-log-query-threads) равным 1. Подробности смотрите в разделе [Настройки](settings/settings.md#settings). @@ -770,16 +747,16 @@ ClickHouse создаёт таблицу только в том случае, к - `ProfileEvents.Names` (Array(String)) — Счетчики для изменения различных метрик для данного потока. Описание метрик можно получить из таблицы [system.events](#system_tables-events)(\#system\_tables-events - `ProfileEvents.Values` (Array(UInt64)) — метрики для данного потока, перечисленные в столбце `ProfileEvents.Names`. -По умолчанию, строки добавляются в таблицу логирования с интервалом в 7,5 секунд. Можно задать интервал в конфигурационном параметре сервера [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) (смотрите параметр `flush_interval_milliseconds`). Чтобы принудительно записать логи из буффера памяти в таблицу, используйте запрос `SYSTEM FLUSH LOGS`. +По умолчанию, строки добавляются в таблицу логирования с интервалом в 7,5 секунд. Можно задать интервал в конфигурационном параметре сервера [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) (смотрите параметр `flush_interval_milliseconds`). Чтобы принудительно записать логи из буффера памяти в таблицу, используйте запрос `SYSTEM FLUSH LOGS`. Если таблицу удалить вручную, она пересоздастся автоматически «на лету». При этом все логи на момент удаления таблицы будут удалены. !!! note "Примечание" Срок хранения логов не ограничен. Логи не удаляются из таблицы автоматически. Вам необходимо самостоятельно организовать удаление устаревших логов. -Можно указать произвольный ключ партиционирования для таблицы `system.query_log` в конфигурации [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) (параметр `partition_by`). +Можно указать произвольный ключ партиционирования для таблицы `system.query_log` в конфигурации [query\_thread\_log](server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) (параметр `partition_by`). -## system.query_thread_log {#system_tables-query-thread-log} +## system.query_thread_log {#system_tables-query_thread_log} Содержит информацию о каждом потоке исполнения запроса. diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index afc8f9a72b1..930e55a06f8 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1175,7 +1175,7 @@ private: /// Poll for changes after a cancellation check, otherwise it never reached /// because of progress updates from server. if (connection->poll(poll_interval)) - break; + break; } if (!receiveAndProcessPacket()) @@ -1583,6 +1583,11 @@ private: if (std::string::npos != embedded_stack_trace_pos && !config().getBool("stacktrace", false)) text.resize(embedded_stack_trace_pos); + /// If we probably have progress bar, we should add additional newline, + /// otherwise exception may display concatenated with the progress bar. + if (need_render_progress) + std::cerr << '\n'; + std::cerr << "Received exception from server (version " << server_version << "):" << std::endl << "Code: " << e.code() << ". " << text << std::endl; } diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index d50b89738aa..5254d2a97ac 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -4,6 +4,8 @@ #include #include +#include + namespace DB { @@ -177,7 +179,11 @@ void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts, ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); for (const TaskShardPtr & task_shard : task_table.all_shards) - thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); }); + thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() + { + setThreadName("DiscoverPartns"); + discoverShardPartitions(timeouts, task_shard); + }); LOG_DEBUG(log, "Waiting for {} setup jobs", thread_pool.active()); thread_pool.wait(); @@ -609,8 +615,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t size_t num_nodes = executeQueryOnCluster( task_table.cluster_push, query_alter_ast_string, - nullptr, - &settings_push, + settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); @@ -638,8 +643,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t UInt64 num_nodes = executeQueryOnCluster( task_table.cluster_push, query_deduplicate_ast_string, - nullptr, - &task_cluster->settings_push, + task_cluster->settings_push, PoolMode::GET_MANY); LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : {}", toString(num_nodes)); @@ -818,8 +822,7 @@ bool ClusterCopier::tryDropPartitionPiece( /// We have to drop partition_piece on each replica size_t num_shards = executeQueryOnCluster( cluster_push, query, - nullptr, - &settings_push, + settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); @@ -1293,7 +1296,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( local_context.setSettings(task_cluster->settings_pull); local_context.setSetting("skip_unavailable_shards", true); - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in); + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().getInputStream()); count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; } @@ -1356,9 +1359,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( String query = queryToString(create_query_push_ast); LOG_DEBUG(log, "Create destination tables. Query: {}", query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, - create_query_push_ast, &task_cluster->settings_push, - PoolMode::GET_MANY); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, PoolMode::GET_MANY); LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount()); } @@ -1403,7 +1404,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); - input = io_select.in; + input = io_select.getInputStream(); output = io_insert.out; } @@ -1479,9 +1480,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( String query = queryToString(create_query_push_ast); LOG_DEBUG(log, "Create destination tables. Query: {}", query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, - create_query_push_ast, &task_cluster->settings_push, - PoolMode::GET_MANY); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, PoolMode::GET_MANY); LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount()); } catch (...) @@ -1548,8 +1547,7 @@ void ClusterCopier::dropHelpingTables(const TaskTable & task_table) /// We have to drop partition_piece on each replica UInt64 num_nodes = executeQueryOnCluster( cluster_push, query, - nullptr, - &settings_push, + settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); @@ -1575,8 +1573,7 @@ void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskT /// We have to drop partition_piece on each replica UInt64 num_nodes = executeQueryOnCluster( cluster_push, query, - nullptr, - &settings_push, + settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); @@ -1690,7 +1687,7 @@ std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti Context local_context = context; local_context.setSettings(task_cluster->settings_pull); - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().in); + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()); std::set res; if (block) @@ -1735,7 +1732,7 @@ const auto & settings = context.getSettingsRef(); Context local_context = context; local_context.setSettings(task_cluster->settings_pull); - return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; + return InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows() != 0; } bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, @@ -1774,7 +1771,7 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi Context local_context = context; local_context.setSettings(task_cluster->settings_pull); - auto result = InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows(); + auto result = InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows(); if (result != 0) LOG_DEBUG(log, "Partition {} piece number {} is PRESENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription()); else @@ -1788,25 +1785,16 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi UInt64 ClusterCopier::executeQueryOnCluster( const ClusterPtr & cluster, const String & query, - const ASTPtr & query_ast_, - const Settings * settings, + const Settings & current_settings, PoolMode pool_mode, ClusterExecutionMode execution_mode, UInt64 max_successful_executions_per_shard) const { - Settings current_settings = settings ? *settings : task_cluster->settings_common; - auto num_shards = cluster->getShardsInfo().size(); std::vector per_shard_num_successful_replicas(num_shards, 0); - ASTPtr query_ast; - if (query_ast_ == nullptr) - { - ParserQuery p_query(query.data() + query.size()); - query_ast = parseQuery(p_query, query, current_settings.max_query_size, current_settings.max_parser_depth); - } - else - query_ast = query_ast_; + ParserQuery p_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(p_query, query, current_settings.max_query_size, current_settings.max_parser_depth); /// We will have to execute query on each replica of a shard. if (execution_mode == ClusterExecutionMode::ON_EACH_NODE) @@ -1815,8 +1803,10 @@ UInt64 ClusterCopier::executeQueryOnCluster( std::atomic origin_replicas_number; /// We need to execute query on one replica at least - auto do_for_shard = [&] (UInt64 shard_index) + auto do_for_shard = [&] (UInt64 shard_index, Settings shard_settings) { + setThreadName("QueryForShard"); + const Cluster::ShardInfo & shard = cluster->getShardsInfo().at(shard_index); UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); num_successful_executions = 0; @@ -1846,10 +1836,10 @@ UInt64 ClusterCopier::executeQueryOnCluster( /// Will try to make as many as possible queries if (shard.hasRemoteConnections()) { - current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; + shard_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings).getSaturated(current_settings.max_execution_time); - auto connections = shard.pool->getMany(timeouts, ¤t_settings, pool_mode); + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(shard_settings).getSaturated(shard_settings.max_execution_time); + auto connections = shard.pool->getMany(timeouts, &shard_settings, pool_mode); for (auto & connection : connections) { @@ -1859,7 +1849,7 @@ UInt64 ClusterCopier::executeQueryOnCluster( try { /// CREATE TABLE and DROP PARTITION queries return empty block - RemoteBlockInputStream stream{*connection, query, Block{}, context, ¤t_settings}; + RemoteBlockInputStream stream{*connection, query, Block{}, context, &shard_settings}; NullBlockOutputStream output{Block{}}; copyData(stream, output); @@ -1878,7 +1868,7 @@ UInt64 ClusterCopier::executeQueryOnCluster( ThreadPool thread_pool(std::min(num_shards, getNumberOfPhysicalCPUCores())); for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index) - thread_pool.scheduleOrThrowOnError([=] { do_for_shard(shard_index); }); + thread_pool.scheduleOrThrowOnError([=, shard_settings = current_settings] { do_for_shard(shard_index, std::move(shard_settings)); }); thread_pool.wait(); } @@ -1898,7 +1888,7 @@ UInt64 ClusterCopier::executeQueryOnCluster( LOG_INFO(log, "There was an error while executing ALTER on each node. Query was executed on {} nodes. But had to be executed on {}", toString(successful_nodes), toString(origin_replicas_number.load())); } - return successful_nodes; } + } diff --git a/programs/copier/ClusterCopier.h b/programs/copier/ClusterCopier.h index 3d6400f51d4..beaf247dfc8 100644 --- a/programs/copier/ClusterCopier.h +++ b/programs/copier/ClusterCopier.h @@ -15,7 +15,6 @@ namespace DB class ClusterCopier { public: - ClusterCopier(const String & task_path_, const String & host_id_, const String & proxy_database_name_, @@ -187,8 +186,7 @@ protected: UInt64 executeQueryOnCluster( const ClusterPtr & cluster, const String & query, - const ASTPtr & query_ast_ = nullptr, - const Settings * settings = nullptr, + const Settings & current_settings, PoolMode pool_mode = PoolMode::GET_ALL, ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_SHARD, UInt64 max_successful_executions_per_shard = 0) const; diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 9b1393204d4..ce4bf94589e 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -114,7 +114,7 @@ void ClusterCopierApp::mainImpl() registerDisks(); static const std::string default_database = "_local"; - DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database)); + DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, *context)); context->setCurrentDatabase(default_database); /// Initialize query scope just in case. diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 01acf250b1b..8ce03ac0867 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -118,13 +118,13 @@ void LocalServer::tryInitPath() } -static void attachSystemTables() +static void attachSystemTables(const Context & context) { DatabasePtr system_database = DatabaseCatalog::instance().tryGetDatabase(DatabaseCatalog::SYSTEM_DATABASE); if (!system_database) { /// TODO: add attachTableDelayed into DatabaseMemory to speedup loading - system_database = std::make_shared(DatabaseCatalog::SYSTEM_DATABASE); + system_database = std::make_shared(DatabaseCatalog::SYSTEM_DATABASE, context); DatabaseCatalog::instance().attachDatabase(DatabaseCatalog::SYSTEM_DATABASE, system_database); } @@ -135,7 +135,7 @@ static void attachSystemTables() int LocalServer::main(const std::vector & /*args*/) try { - Logger * log = &logger(); + Poco::Logger * log = &logger(); ThreadStatus thread_status; UseSSL use_ssl; @@ -202,7 +202,7 @@ try * if such tables will not be dropped, clickhouse-server will not be able to load them due to security reasons. */ std::string default_database = config().getString("default_database", "_local"); - DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database)); + DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, *context)); context->setCurrentDatabase(default_database); applyCmdOptions(); @@ -213,14 +213,14 @@ try LOG_DEBUG(log, "Loading metadata from {}", context->getPath()); loadMetadataSystem(*context); - attachSystemTables(); + attachSystemTables(*context); loadMetadata(*context); DatabaseCatalog::instance().loadDatabases(); LOG_DEBUG(log, "Loaded metadata."); } else { - attachSystemTables(); + attachSystemTables(*context); } processQueries(); diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 6ada6dd3a2d..1316ff8f4c6 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -25,7 +25,7 @@ ODBCBlockInputStream::ODBCBlockInputStream( , result{statement} , iterator{result.begin()} , max_block_size{max_block_size_} - , log(&Logger::get("ODBCBlockInputStream")) + , log(&Poco::Logger::get("ODBCBlockInputStream")) { if (sample_block.columns() != result.columnCount()) throw Exception{"RecordSet contains " + toString(result.columnCount()) + " columns while " + toString(sample_block.columns()) diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.cpp b/programs/odbc-bridge/ODBCBlockOutputStream.cpp index ab24c008e40..b5bffc58c55 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockOutputStream.cpp @@ -94,7 +94,7 @@ ODBCBlockOutputStream::ODBCBlockOutputStream(Poco::Data::Session && session_, , table_name(remote_table_name_) , sample_block(sample_block_) , quoting(quoting_) - , log(&Logger::get("ODBCBlockOutputStream")) + , log(&Poco::Logger::get("ODBCBlockOutputStream")) { description.init(sample_block); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ce1d35e65d4..e587e134075 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -89,7 +89,7 @@ namespace CurrentMetrics namespace { -void setupTmpPath(Logger * log, const std::string & path) +void setupTmpPath(Poco::Logger * log, const std::string & path) { LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); @@ -212,7 +212,7 @@ void Server::defineOptions(Poco::Util::OptionSet & options) int Server::main(const std::vector & /*args*/) { - Logger * log = &logger(); + Poco::Logger * log = &logger(); UseSSL use_ssl; ThreadStatus thread_status; @@ -236,6 +236,14 @@ int Server::main(const std::vector & /*args*/) if (ThreadFuzzer::instance().isEffective()) LOG_WARNING(log, "ThreadFuzzer is enabled. Application will run slowly and unstable."); +#if !defined(NDEBUG) || !defined(__OPTIMIZE__) + LOG_WARNING(log, "Server was built in debug mode. It will work slowly."); +#endif + +#if defined(ADDRESS_SANITIZER) || defined(THREAD_SANITIZER) || defined(MEMORY_SANITIZER) + LOG_WARNING(log, "Server was built with sanitizer. It will work slowly."); +#endif + /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases... */ diff --git a/src/Access/AllowedClientHosts.cpp b/src/Access/AllowedClientHosts.cpp index 9c8a7cc12f5..82372fd8b14 100644 --- a/src/Access/AllowedClientHosts.cpp +++ b/src/Access/AllowedClientHosts.cpp @@ -309,7 +309,7 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const throw; /// Try to ignore DNS errors: if host cannot be resolved, skip it and try next. LOG_WARNING( - &Logger::get("AddressPatterns"), + &Poco::Logger::get("AddressPatterns"), "Failed to check if the allowed client hosts contain address {}. {}, code = {}", client_address.toString(), e.displayText(), e.code()); return false; @@ -342,7 +342,7 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const throw; /// Try to ignore DNS errors: if host cannot be resolved, skip it and try next. LOG_WARNING( - &Logger::get("AddressPatterns"), + &Poco::Logger::get("AddressPatterns"), "Failed to check if the allowed client hosts contain address {}. {}, code = {}", client_address.toString(), e.displayText(), e.code()); return false; diff --git a/src/Access/ExtendedRoleSet.cpp b/src/Access/ExtendedRoleSet.cpp index a29ee40380c..a8e674b3722 100644 --- a/src/Access/ExtendedRoleSet.cpp +++ b/src/Access/ExtendedRoleSet.cpp @@ -68,15 +68,27 @@ void ExtendedRoleSet::init(const ASTExtendedRoleSet & ast, const AccessControlMa { all = ast.all; - auto name_to_id = [id_mode{ast.id_mode}, manager](const String & name) -> UUID + auto name_to_id = [&ast, manager](const String & name) -> UUID { - if (id_mode) + if (ast.id_mode) return parse(name); assert(manager); - auto id = manager->find(name); - if (id) - return *id; - return manager->getID(name); + if (ast.can_contain_users && ast.can_contain_roles) + { + auto id = manager->find(name); + if (id) + return *id; + return manager->getID(name); + } + else if (ast.can_contain_users) + { + return manager->getID(name); + } + else + { + assert(ast.can_contain_roles); + return manager->getID(name); + } }; if (!ast.names.empty() && !all) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index baa0fbcb883..13e8aac6906 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -74,6 +74,7 @@ if(USE_RDKAFKA) endif() if (USE_AWS_S3) + add_headers_and_sources(dbms Common/S3) add_headers_and_sources(dbms Disks/S3) endif() diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 6c1f7f8e953..367d4bab1dc 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -508,18 +508,18 @@ void Connection::sendScalarsData(Scalars & data) "Sent data for {} scalars, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), compressed {} times to {} ({}/sec.)", data.size(), rows, elapsed, static_cast(rows / watch.elapsedSeconds()), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds()), + ReadableSize(maybe_compressed_out_bytes), + ReadableSize(maybe_compressed_out_bytes / watch.elapsedSeconds()), static_cast(maybe_compressed_out_bytes) / out_bytes, - formatReadableSizeWithBinarySuffix(out_bytes), - formatReadableSizeWithBinarySuffix(out_bytes / watch.elapsedSeconds())); + ReadableSize(out_bytes), + ReadableSize(out_bytes / watch.elapsedSeconds())); else LOG_DEBUG(log_wrapper.get(), "Sent data for {} scalars, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), no compression.", data.size(), rows, elapsed, static_cast(rows / watch.elapsedSeconds()), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds())); + ReadableSize(maybe_compressed_out_bytes), + ReadableSize(maybe_compressed_out_bytes / watch.elapsedSeconds())); } namespace @@ -612,18 +612,18 @@ void Connection::sendExternalTablesData(ExternalTablesData & data) "Sent data for {} external tables, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), compressed {} times to {} ({}/sec.)", data.size(), rows, elapsed, static_cast(rows / watch.elapsedSeconds()), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds()), + ReadableSize(maybe_compressed_out_bytes), + ReadableSize(maybe_compressed_out_bytes / watch.elapsedSeconds()), static_cast(maybe_compressed_out_bytes) / out_bytes, - formatReadableSizeWithBinarySuffix(out_bytes), - formatReadableSizeWithBinarySuffix(out_bytes / watch.elapsedSeconds())); + ReadableSize(out_bytes), + ReadableSize(out_bytes / watch.elapsedSeconds())); else LOG_DEBUG(log_wrapper.get(), "Sent data for {} external tables, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), no compression.", data.size(), rows, elapsed, static_cast(rows / watch.elapsedSeconds()), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds())); + ReadableSize(maybe_compressed_out_bytes), + ReadableSize(maybe_compressed_out_bytes / watch.elapsedSeconds())); } std::optional Connection::getResolvedAddress() const diff --git a/src/Client/Connection.h b/src/Client/Connection.h index e056a4323df..f93fa7871e5 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -249,16 +249,16 @@ private: { } - Logger * get() + Poco::Logger * get() { if (!log) - log = &Logger::get("Connection (" + parent.getDescription() + ")"); + log = &Poco::Logger::get("Connection (" + parent.getDescription() + ")"); return log; } private: - std::atomic log; + std::atomic log; Connection & parent; }; diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index be52234b904..bbdcae894e7 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -56,7 +56,7 @@ public: Protocol::Compression compression_ = Protocol::Compression::Enable, Protocol::Secure secure_ = Protocol::Secure::Disable) : Base(max_connections_, - &Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")), + &Poco::Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")), host(host_), port(port_), default_database(default_database_), diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 9933cc6b555..713bb33342f 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -35,7 +35,7 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover( LoadBalancing load_balancing, time_t decrease_error_period_, size_t max_error_cap_) - : Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Logger::get("ConnectionPoolWithFailover")) + : Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Poco::Logger::get("ConnectionPoolWithFailover")) , default_load_balancing(load_balancing) { const std::string & local_hostname = getFQDNOrHostName(); diff --git a/src/Client/TimeoutSetter.cpp b/src/Client/TimeoutSetter.cpp index 10cc9fbffef..a512485407e 100644 --- a/src/Client/TimeoutSetter.cpp +++ b/src/Client/TimeoutSetter.cpp @@ -35,7 +35,7 @@ TimeoutSetter::~TimeoutSetter() catch (std::exception & e) { // Sometimes catched on macos - LOG_ERROR(&Logger::get("Client"), "TimeoutSetter: Can't reset timeouts: {}", e.what()); + LOG_ERROR(&Poco::Logger::get("Client"), "TimeoutSetter: Can't reset timeouts: {}", e.what()); } } } diff --git a/src/Common/AlignedBuffer.cpp b/src/Common/AlignedBuffer.cpp index 6d4775ebadf..f1d3f98ff3a 100644 --- a/src/Common/AlignedBuffer.cpp +++ b/src/Common/AlignedBuffer.cpp @@ -18,8 +18,8 @@ void AlignedBuffer::alloc(size_t size, size_t alignment) void * new_buf; int res = ::posix_memalign(&new_buf, std::max(alignment, sizeof(void*)), size); if (0 != res) - throwFromErrno("Cannot allocate memory (posix_memalign), size: " - + formatReadableSizeWithBinarySuffix(size) + ", alignment: " + formatReadableSizeWithBinarySuffix(alignment) + ".", + throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign), size: {}, alignment: {}.", + ReadableSize(size), ReadableSize(alignment)), ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); buf = new_buf; } diff --git a/src/Common/Allocator.h b/src/Common/Allocator.h index 43d7e67c4bb..ead456f935e 100644 --- a/src/Common/Allocator.h +++ b/src/Common/Allocator.h @@ -129,7 +129,7 @@ public: void * new_buf = ::realloc(buf, new_size); if (nullptr == new_buf) - DB::throwFromErrno("Allocator: Cannot realloc from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + DB::throwFromErrno(fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); buf = new_buf; if constexpr (clear_memory) @@ -145,7 +145,8 @@ public: buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE, PROT_READ | PROT_WRITE, mmap_flags, -1, 0); if (MAP_FAILED == buf) - DB::throwFromErrno("Allocator: Cannot mremap memory chunk from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_MREMAP); + DB::throwFromErrno(fmt::format("Allocator: Cannot mremap memory chunk from {} to {}.", + ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_MREMAP); /// No need for zero-fill, because mmap guarantees it. } @@ -201,13 +202,13 @@ private: if (size >= MMAP_THRESHOLD) { if (alignment > MMAP_MIN_ALIGNMENT) - throw DB::Exception("Too large alignment " + formatReadableSizeWithBinarySuffix(alignment) + ": more than page size when allocating " - + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::BAD_ARGUMENTS); + throw DB::Exception(fmt::format("Too large alignment {}: more than page size when allocating {}.", + ReadableSize(alignment), ReadableSize(size)), DB::ErrorCodes::BAD_ARGUMENTS); buf = mmap(getMmapHint(), size, PROT_READ | PROT_WRITE, mmap_flags, -1, 0); if (MAP_FAILED == buf) - DB::throwFromErrno("Allocator: Cannot mmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + DB::throwFromErrno(fmt::format("Allocator: Cannot mmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); /// No need for zero-fill, because mmap guarantees it. } @@ -221,7 +222,7 @@ private: buf = ::malloc(size); if (nullptr == buf) - DB::throwFromErrno("Allocator: Cannot malloc " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + DB::throwFromErrno(fmt::format("Allocator: Cannot malloc {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); } else { @@ -229,7 +230,8 @@ private: int res = posix_memalign(&buf, alignment, size); if (0 != res) - DB::throwFromErrno("Cannot allocate memory (posix_memalign) " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); + DB::throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)), + DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); if constexpr (clear_memory) memset(buf, 0, size); @@ -243,7 +245,7 @@ private: if (size >= MMAP_THRESHOLD) { if (0 != munmap(buf, size)) - DB::throwFromErrno("Allocator: Cannot munmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_MUNMAP); + DB::throwFromErrno(fmt::format("Allocator: Cannot munmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_MUNMAP); } else { diff --git a/src/Common/ArrayCache.h b/src/Common/ArrayCache.h index 304ab8e03c3..5e3af09a2fd 100644 --- a/src/Common/ArrayCache.h +++ b/src/Common/ArrayCache.h @@ -177,13 +177,13 @@ private: { ptr = mmap(address_hint, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); if (MAP_FAILED == ptr) - DB::throwFromErrno("Allocator: Cannot mmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + DB::throwFromErrno(fmt::format("Allocator: Cannot mmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); } ~Chunk() { if (ptr && 0 != munmap(ptr, size)) - DB::throwFromErrno("Allocator: Cannot munmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_MUNMAP); + DB::throwFromErrno(fmt::format("Allocator: Cannot munmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_MUNMAP); } Chunk(Chunk && other) : ptr(other.ptr), size(other.size) diff --git a/src/Common/AutoArray.h b/src/Common/AutoArray.h index 92e8d45b07b..a39706baa3d 100644 --- a/src/Common/AutoArray.h +++ b/src/Common/AutoArray.h @@ -278,7 +278,7 @@ private: void * new_data = nullptr; int res = posix_memalign(&new_data, alignment, prefix_size + new_size * sizeof(T)); if (0 != res) - throwFromErrno("Cannot allocate memory (posix_memalign) " + formatReadableSizeWithBinarySuffix(new_size) + ".", + throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(new_size)), ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); data_ptr = static_cast(new_data); diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 7eb7e13a93e..bf65b7028cc 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -66,21 +66,21 @@ ConfigProcessor::ConfigProcessor( , name_pool(new Poco::XML::NamePool(65521)) , dom_parser(name_pool) { - if (log_to_console && !Logger::has("ConfigProcessor")) + if (log_to_console && !Poco::Logger::has("ConfigProcessor")) { channel_ptr = new Poco::ConsoleChannel; - log = &Logger::create("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE); + log = &Poco::Logger::create("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE); } else { - log = &Logger::get("ConfigProcessor"); + log = &Poco::Logger::get("ConfigProcessor"); } } ConfigProcessor::~ConfigProcessor() { if (channel_ptr) /// This means we have created a new console logger in the constructor. - Logger::destroy("ConfigProcessor"); + Poco::Logger::destroy("ConfigProcessor"); } diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index b6f772f8c16..5a942e73bdb 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -116,7 +116,7 @@ private: bool throw_on_bad_incl; - Logger * log; + Poco::Logger * log; Poco::AutoPtr channel_ptr; Substitutions substitutions; diff --git a/src/Common/Config/ConfigReloader.h b/src/Common/Config/ConfigReloader.h index c0904422b39..553589fbd37 100644 --- a/src/Common/Config/ConfigReloader.h +++ b/src/Common/Config/ConfigReloader.h @@ -69,7 +69,7 @@ private: static constexpr auto reload_interval = std::chrono::seconds(2); - Poco::Logger * log = &Logger::get("ConfigReloader"); + Poco::Logger * log = &Poco::Logger::get("ConfigReloader"); std::string path; std::string include_from_path; diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 1d2d89f477c..c5b4fd0c585 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -202,7 +202,7 @@ bool DNSResolver::updateCache() } if (!lost_hosts.empty()) - LOG_INFO(&Logger::get("DNSResolver"), "Cached hosts not found: {}", lost_hosts); + LOG_INFO(&Poco::Logger::get("DNSResolver"), "Cached hosts not found: {}", lost_hosts); return updated; } diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 23e5e0a762e..b0c897127c6 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -5,8 +5,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -25,6 +27,8 @@ namespace ErrorCodes extern const int STD_EXCEPTION; extern const int UNKNOWN_EXCEPTION; extern const int LOGICAL_ERROR; + extern const int CANNOT_ALLOCATE_MEMORY; + extern const int CANNOT_MREMAP; } @@ -118,7 +122,7 @@ void throwFromErrnoWithPath(const std::string & s, const std::string & path, int void tryLogCurrentException(const char * log_name, const std::string & start_of_message) { - tryLogCurrentException(&Logger::get(log_name), start_of_message); + tryLogCurrentException(&Poco::Logger::get(log_name), start_of_message); } void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message) @@ -144,18 +148,79 @@ static void getNoSpaceLeftInfoMessage(std::filesystem::path path, std::string & path = path.parent_path(); auto fs = getStatVFS(path); - msg += "\nTotal space: " + formatReadableSizeWithBinarySuffix(fs.f_blocks * fs.f_bsize) - + "\nAvailable space: " + formatReadableSizeWithBinarySuffix(fs.f_bavail * fs.f_bsize) - + "\nTotal inodes: " + formatReadableQuantity(fs.f_files) - + "\nAvailable inodes: " + formatReadableQuantity(fs.f_favail); - auto mount_point = getMountPoint(path).string(); - msg += "\nMount point: " + mount_point; + + fmt::format_to(std::back_inserter(msg), + "\nTotal space: {}\nAvailable space: {}\nTotal inodes: {}\nAvailable inodes: {}\nMount point: {}", + ReadableSize(fs.f_blocks * fs.f_bsize), + ReadableSize(fs.f_bavail * fs.f_bsize), + formatReadableQuantity(fs.f_files), + formatReadableQuantity(fs.f_favail), + mount_point); + #if defined(__linux__) msg += "\nFilesystem: " + getFilesystemName(mount_point); #endif } + +/** It is possible that the system has enough memory, + * but we have shortage of the number of available memory mappings. + * Provide good diagnostic to user in that case. + */ +static void getNotEnoughMemoryMessage(std::string & msg) +{ +#if defined(__linux__) + try + { + static constexpr size_t buf_size = 4096; + char buf[buf_size]; + + UInt64 max_map_count = 0; + { + ReadBufferFromFile file("/proc/sys/vm/max_map_count", buf_size, -1, buf); + readText(max_map_count, file); + } + + UInt64 num_maps = 0; + { + ReadBufferFromFile file("/proc/self/maps", buf_size, -1, buf); + while (!file.eof()) + { + char * next_pos = find_first_symbols<'\n'>(file.position(), file.buffer().end()); + file.position() = next_pos; + + if (!file.hasPendingData()) + continue; + + if (*file.position() == '\n') + { + ++num_maps; + ++file.position(); + } + } + } + + if (num_maps > max_map_count * 0.99) + { + msg += fmt::format( + "\nIt looks like that the process is near the limit on number of virtual memory mappings." + "\nCurrent number of mappings (/proc/self/maps): {}." + "\nLimit on number of mappings (/proc/sys/vm/max_map_count): {}." + "\nYou should increase the limit for vm.max_map_count in /etc/sysctl.conf" + "\n", + num_maps, max_map_count); + } + } + catch (...) + { + msg += "\nCannot obtain additional info about memory usage."; + } +#else + (void)msg; +#endif +} + static std::string getExtraExceptionInfo(const std::exception & e) { String msg; @@ -170,6 +235,13 @@ static std::string getExtraExceptionInfo(const std::exception & e) { if (errno_exception->getErrno() == ENOSPC && errno_exception->getPath()) getNoSpaceLeftInfoMessage(errno_exception->getPath().value(), msg); + else if (errno_exception->code() == ErrorCodes::CANNOT_ALLOCATE_MEMORY + || errno_exception->code() == ErrorCodes::CANNOT_MREMAP) + getNotEnoughMemoryMessage(msg); + } + else if (dynamic_cast(&e)) + { + getNotEnoughMemoryMessage(msg); } } catch (...) diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index c4a919e9ec1..83db397e78c 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -37,7 +37,7 @@ private: Map map; bool initialized = false; - Logger * log = &Logger::get("FileChecker"); + Poco::Logger * log = &Poco::Logger::get("FileChecker"); }; } diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 5bcfc8fc2db..76aa9705a91 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -306,7 +306,7 @@ private: auto it = cells.find(key); if (it == cells.end()) { - LOG_ERROR(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); + LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); abort(); } @@ -324,7 +324,7 @@ private: if (current_size > (1ull << 63)) { - LOG_ERROR(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); + LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); abort(); } } diff --git a/src/Common/MemoryStatisticsOS.cpp b/src/Common/MemoryStatisticsOS.cpp index adc5bf5d904..6082d23cbd0 100644 --- a/src/Common/MemoryStatisticsOS.cpp +++ b/src/Common/MemoryStatisticsOS.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB @@ -19,6 +20,7 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; extern const int CANNOT_OPEN_FILE; extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; + extern const int CANNOT_CLOSE_FILE; } static constexpr auto filename = "/proc/self/statm"; @@ -35,7 +37,18 @@ MemoryStatisticsOS::MemoryStatisticsOS() MemoryStatisticsOS::~MemoryStatisticsOS() { if (0 != ::close(fd)) - tryLogCurrentException(__PRETTY_FUNCTION__); + { + try + { + throwFromErrno( + "File descriptor for \"" + std::string(filename) + "\" could not be closed. " + "Something seems to have gone wrong. Inspect errno.", ErrorCodes::CANNOT_CLOSE_FILE); + } + catch (const ErrnoException &) + { + DB::tryLogCurrentException(__PRETTY_FUNCTION__); + } + } } MemoryStatisticsOS::Data MemoryStatisticsOS::get() const diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 082b78682a2..03bd8be94f3 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -49,12 +49,14 @@ MemoryTracker::~MemoryTracker() void MemoryTracker::logPeakMemoryUsage() const { - LOG_DEBUG(&Logger::get("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), formatReadableSizeWithBinarySuffix(peak)); + const auto * description = description_ptr.load(std::memory_order_relaxed); + LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak)); } void MemoryTracker::logMemoryUsage(Int64 current) const { - LOG_DEBUG(&Logger::get("MemoryTracker"), "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), formatReadableSizeWithBinarySuffix(current)); + const auto * description = description_ptr.load(std::memory_order_relaxed); + LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current)); } @@ -85,7 +87,7 @@ void MemoryTracker::alloc(Int64 size) std::stringstream message; message << "Memory tracker"; - if (description) + if (const auto * description = description_ptr.load(std::memory_order_relaxed)) message << " " << description; message << ": fault injected. Would use " << formatReadableSizeWithBinarySuffix(will_be) << " (attempt to allocate chunk of " << size << " bytes)" @@ -117,7 +119,7 @@ void MemoryTracker::alloc(Int64 size) std::stringstream message; message << "Memory limit"; - if (description) + if (const auto * description = description_ptr.load(std::memory_order_relaxed)) message << " " << description; message << " exceeded: would use " << formatReadableSizeWithBinarySuffix(will_be) << " (attempt to allocate chunk of " << size << " bytes)" diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 23f32b051b2..8af683ae790 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -35,7 +35,7 @@ private: CurrentMetrics::Metric metric = CurrentMetrics::end(); /// This description will be used as prefix into log messages (if isn't nullptr) - const char * description = nullptr; + std::atomic description_ptr = nullptr; void updatePeak(Int64 will_be); void logMemoryUsage(Int64 current) const; @@ -114,9 +114,9 @@ public: metric = metric_; } - void setDescription(const char * description_) + void setDescription(const char * description) { - description = description_; + description_ptr.store(description, std::memory_order_relaxed); } /// Reset the accumulated data diff --git a/src/Common/PipeFDs.cpp b/src/Common/PipeFDs.cpp index cec8cb1cf29..1f57234534f 100644 --- a/src/Common/PipeFDs.cpp +++ b/src/Common/PipeFDs.cpp @@ -102,7 +102,7 @@ void LazyPipeFDs::tryIncreaseSize(int desired_size) if (-1 == fcntl(fds_rw[1], F_SETPIPE_SZ, pipe_size * 2) && errno != EPERM) throwFromErrno("Cannot increase pipe capacity to " + std::to_string(pipe_size * 2), ErrorCodes::CANNOT_FCNTL); - LOG_TRACE(log, "Pipe capacity is {}", formatReadableSizeWithBinarySuffix(std::min(pipe_size, desired_size))); + LOG_TRACE(log, "Pipe capacity is {}", ReadableSize(std::min(pipe_size, desired_size))); } #else (void)desired_size; diff --git a/src/Common/PoolBase.h b/src/Common/PoolBase.h index 201f9ec1d00..43f4fbff9fe 100644 --- a/src/Common/PoolBase.h +++ b/src/Common/PoolBase.h @@ -152,9 +152,9 @@ private: protected: - Logger * log; + Poco::Logger * log; - PoolBase(unsigned max_items_, Logger * log_) + PoolBase(unsigned max_items_, Poco::Logger * log_) : max_items(max_items_), log(log_) { items.reserve(max_items); diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index e9f3d7d5d23..4d9e52364b9 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -57,7 +57,7 @@ public: NestedPools nested_pools_, time_t decrease_error_period_, size_t max_error_cap_, - Logger * log_) + Poco::Logger * log_) : nested_pools(std::move(nested_pools_)) , decrease_error_period(decrease_error_period_) , max_error_cap(max_error_cap_) @@ -134,7 +134,7 @@ protected: /// The time when error counts were last decreased. time_t last_error_decrease_time = 0; - Logger * log; + Poco::Logger * log; }; template diff --git a/src/Common/ProcfsMetricsProvider.cpp b/src/Common/ProcfsMetricsProvider.cpp index 633558a7b67..fcc4124aa85 100644 --- a/src/Common/ProcfsMetricsProvider.cpp +++ b/src/Common/ProcfsMetricsProvider.cpp @@ -7,6 +7,7 @@ #include #include +#include #include #include @@ -22,6 +23,7 @@ namespace ErrorCodes { extern const int FILE_DOESNT_EXIST; extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_CLOSE_FILE; extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; } @@ -39,6 +41,20 @@ namespace errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); } +inline void emitErrorMsgWithFailedToCloseFile(const std::string & filename) +{ + try + { + throwFromErrno( + "File descriptor for \"" + filename + "\" could not be closed. " + "Something seems to have gone wrong. Inspect errno.", ErrorCodes::CANNOT_CLOSE_FILE); + } + catch (const ErrnoException &) + { + DB::tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + ssize_t readFromFD(const int fd, const char * filename, char * buf, size_t buf_size) { ssize_t res = 0; @@ -100,11 +116,11 @@ ProcfsMetricsProvider::ProcfsMetricsProvider(const pid_t /*tid*/) ProcfsMetricsProvider::~ProcfsMetricsProvider() { if (stats_version >= 3 && 0 != ::close(thread_io_fd)) - tryLogCurrentException(__PRETTY_FUNCTION__); + emitErrorMsgWithFailedToCloseFile(thread_io); if (0 != ::close(thread_stat_fd)) - tryLogCurrentException(__PRETTY_FUNCTION__); + emitErrorMsgWithFailedToCloseFile(thread_stat); if (0 != ::close(thread_schedstat_fd)) - tryLogCurrentException(__PRETTY_FUNCTION__); + emitErrorMsgWithFailedToCloseFile(thread_schedstat); } diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index cd68edd3ba1..a8b7d51a260 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -79,7 +79,7 @@ namespace ErrorCodes template QueryProfilerBase::QueryProfilerBase(const UInt64 thread_id, const int clock_type, UInt32 period, const int pause_signal_) - : log(&Logger::get("QueryProfiler")) + : log(&Poco::Logger::get("QueryProfiler")) , pause_signal(pause_signal_) { #if USE_UNWIND diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index 0d8e155d44a..d3d9991bc90 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -102,7 +102,7 @@ SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); - Logger * logger = &Logger::get("SensitiveDataMaskerConfigRead"); + Poco::Logger * logger = &Poco::Logger::get("SensitiveDataMaskerConfigRead"); std::set used_names; diff --git a/src/Common/StatusFile.cpp b/src/Common/StatusFile.cpp index 79e34d244bc..758f500e9d2 100644 --- a/src/Common/StatusFile.cpp +++ b/src/Common/StatusFile.cpp @@ -43,9 +43,9 @@ StatusFile::StatusFile(const std::string & path_) } if (!contents.empty()) - LOG_INFO(&Logger::get("StatusFile"), "Status file {} already exists - unclean restart. Contents:\n{}", path, contents); + LOG_INFO(&Poco::Logger::get("StatusFile"), "Status file {} already exists - unclean restart. Contents:\n{}", path, contents); else - LOG_INFO(&Logger::get("StatusFile"), "Status file {} already exists and is empty - probably unclean hardware restart.", path); + LOG_INFO(&Poco::Logger::get("StatusFile"), "Status file {} already exists and is empty - probably unclean hardware restart.", path); } fd = ::open(path.c_str(), O_WRONLY | O_CREAT | O_CLOEXEC, 0666); @@ -90,10 +90,10 @@ StatusFile::StatusFile(const std::string & path_) StatusFile::~StatusFile() { if (0 != close(fd)) - LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE)); + LOG_ERROR(&Poco::Logger::get("StatusFile"), "Cannot close file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE)); if (0 != unlink(path.c_str())) - LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE)); + LOG_ERROR(&Poco::Logger::get("StatusFile"), "Cannot unlink file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE)); } } diff --git a/src/Common/StringUtils/StringUtils.h b/src/Common/StringUtils/StringUtils.h index 390015f3d2b..bac33bdb508 100644 --- a/src/Common/StringUtils/StringUtils.h +++ b/src/Common/StringUtils/StringUtils.h @@ -116,6 +116,12 @@ inline bool isControlASCII(char c) return static_cast(c) <= 31; } +inline bool isPrintableASCII(char c) +{ + uint8_t uc = c; + return uc >= 32 && uc <= 126; /// 127 is ASCII DEL. +} + /// Works assuming isAlphaASCII. inline char toLowerIfAlphaASCII(char c) { diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 7d6935c0383..3e6e31ed3fc 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -234,14 +234,6 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ --scheduled_jobs; } - DB::tryLogCurrentException("ThreadPool", - std::string("Exception in ThreadPool(") + - "max_threads: " + std::to_string(max_threads) - + ", max_free_threads: " + std::to_string(max_free_threads) - + ", queue_size: " + std::to_string(queue_size) - + ", shutdown_on_exception: " + std::to_string(shutdown_on_exception) - + ")."); - job_finished.notify_all(); new_job_or_shutdown.notify_all(); return; diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index d393ee2328b..3cc29371b64 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -1,7 +1,9 @@ #include +#include #include + namespace DB { namespace UTF8 @@ -94,6 +96,42 @@ size_t computeWidth(const UInt8 * data, size_t size, size_t prefix) noexcept size_t rollback = 0; for (size_t i = 0; i < size; ++i) { + /// Quickly skip regular ASCII + +#if defined(__SSE2__) + const auto lower_bound = _mm_set1_epi8(32); + const auto upper_bound = _mm_set1_epi8(126); + + while (i + 15 < size) + { + __m128i bytes = _mm_loadu_si128(reinterpret_cast(&data[i])); + + const uint16_t non_regular_width_mask = _mm_movemask_epi8( + _mm_or_si128( + _mm_cmplt_epi8(bytes, lower_bound), + _mm_cmpgt_epi8(bytes, upper_bound))); + + if (non_regular_width_mask) + { + auto num_regular_chars = __builtin_ctz(non_regular_width_mask); + width += num_regular_chars; + i += num_regular_chars; + break; + } + else + { + i += 16; + width += 16; + } + } +#endif + + while (i < size && isPrintableASCII(data[i])) + { + ++width; + ++i; + } + switch (decoder.decode(data[i])) { case UTF8Decoder::REJECT: diff --git a/src/Common/ZooKeeper/Increment.h b/src/Common/ZooKeeper/Increment.h index 26128480635..fa5f550ca9b 100644 --- a/src/Common/ZooKeeper/Increment.h +++ b/src/Common/ZooKeeper/Increment.h @@ -43,7 +43,7 @@ public: private: zkutil::ZooKeeperHolderPtr zookeeper_holder; std::string path; - Logger * log = &Logger::get("zkutil::Increment"); + Poco::Logger * log = &Poco::Logger::get("zkutil::Increment"); }; } diff --git a/src/Common/ZooKeeper/LeaderElection.h b/src/Common/ZooKeeper/LeaderElection.h index fe10a3bf00e..e3b97e7f8ca 100644 --- a/src/Common/ZooKeeper/LeaderElection.h +++ b/src/Common/ZooKeeper/LeaderElection.h @@ -39,7 +39,7 @@ public: LeaderElection(DB::BackgroundSchedulePool & pool_, const std::string & path_, ZooKeeper & zookeeper_, LeadershipHandler handler_, const std::string & identifier_ = "") : pool(pool_), path(path_), zookeeper(zookeeper_), handler(handler_), identifier(identifier_) , log_name("LeaderElection (" + path + ")") - , log(&Logger::get(log_name)) + , log(&Poco::Logger::get(log_name)) { task = pool.createTask(log_name, [this] { threadFunction(); }); createNode(); @@ -67,7 +67,7 @@ private: LeadershipHandler handler; std::string identifier; std::string log_name; - Logger * log; + Poco::Logger * log; EphemeralNodeHolderPtr node; std::string node_name; diff --git a/src/Common/ZooKeeper/Lock.h b/src/Common/ZooKeeper/Lock.h index 683470cf5a5..67116124b4a 100644 --- a/src/Common/ZooKeeper/Lock.h +++ b/src/Common/ZooKeeper/Lock.h @@ -21,7 +21,7 @@ namespace zkutil zookeeper_holder(zookeeper_holder_), lock_path(lock_prefix_ + "/" + lock_name_), lock_message(lock_message_), - log(&Logger::get("zkutil::Lock")) + log(&Poco::Logger::get("zkutil::Lock")) { auto zookeeper = zookeeper_holder->getZooKeeper(); if (create_parent_path_) @@ -72,7 +72,7 @@ namespace zkutil std::string lock_path; std::string lock_message; - Logger * log; + Poco::Logger * log; }; } diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 6e0cc22a952..115518e2bf9 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -48,7 +48,7 @@ static void check(int32_t code, const std::string & path) void ZooKeeper::init(const std::string & implementation_, const std::string & hosts_, const std::string & identity_, int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_) { - log = &Logger::get("ZooKeeper"); + log = &Poco::Logger::get("ZooKeeper"); hosts = hosts_; identity = identity_; session_timeout_ms = session_timeout_ms_; diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index e8ab06c2182..3bf9ad3c100 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -269,7 +269,7 @@ private: std::mutex mutex; - Logger * log = nullptr; + Poco::Logger * log = nullptr; }; diff --git a/src/Common/ZooKeeper/ZooKeeperHolder.h b/src/Common/ZooKeeper/ZooKeeperHolder.h index d5792b8fde6..74a0a7fbc34 100644 --- a/src/Common/ZooKeeper/ZooKeeperHolder.h +++ b/src/Common/ZooKeeper/ZooKeeperHolder.h @@ -70,7 +70,7 @@ private: mutable std::mutex mutex; ZooKeeper::Ptr ptr; - Logger * log = &Logger::get("ZooKeeperHolder"); + Poco::Logger * log = &Poco::Logger::get("ZooKeeperHolder"); static std::string nullptr_exception_message; }; diff --git a/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp b/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp index 1c74985d5f9..d245428db8e 100644 --- a/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp +++ b/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp @@ -20,8 +20,8 @@ int main(int argc, char ** argv) } Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); zkutil::ZooKeeper zk(argv[1]); std::string unused; diff --git a/src/Common/formatReadable.h b/src/Common/formatReadable.h index 87d24b75c20..1c46bd8af09 100644 --- a/src/Common/formatReadable.h +++ b/src/Common/formatReadable.h @@ -1,6 +1,8 @@ #pragma once #include +#include + namespace DB { @@ -20,3 +22,35 @@ std::string formatReadableSizeWithDecimalSuffix(double value, int precision = 2) /// Prints the number as 123.45 billion. void formatReadableQuantity(double value, DB::WriteBuffer & out, int precision = 2); std::string formatReadableQuantity(double value, int precision = 2); + + +/// Wrapper around value. If used with fmt library (e.g. for log messages), +/// value is automatically formatted as size with binary suffix. +struct ReadableSize +{ + double value; + explicit ReadableSize(double value_) : value(value_) {} +}; + +/// See https://fmt.dev/latest/api.html#formatting-user-defined-types +template <> +struct fmt::formatter +{ + constexpr auto parse(format_parse_context & ctx) + { + auto it = ctx.begin(); + auto end = ctx.end(); + + /// Only support {}. + if (it != end && *it != '}') + throw format_error("invalid format"); + + return it; + } + + template + auto format(const ReadableSize & size, FormatContext & ctx) + { + return format_to(ctx.out(), "{}", formatReadableSizeWithBinarySuffix(size.value)); + } +}; diff --git a/src/Common/tests/gtest_global_context.h b/src/Common/tests/gtest_global_context.h index 05f60e01774..b6529f09b46 100644 --- a/src/Common/tests/gtest_global_context.h +++ b/src/Common/tests/gtest_global_context.h @@ -11,15 +11,15 @@ struct ContextHolder : shared_context(DB::Context::createShared()) , context(DB::Context::createGlobal(shared_context.get())) { + context.makeGlobalContext(); + context.setPath("./"); } ContextHolder(ContextHolder &&) = default; }; -inline ContextHolder getContext() +inline const ContextHolder & getContext() { - ContextHolder holder; - holder.context.makeGlobalContext(); - holder.context.setPath("./"); + static ContextHolder holder; return holder; } diff --git a/src/Common/tests/gtest_log.cpp b/src/Common/tests/gtest_log.cpp new file mode 100644 index 00000000000..9f4ef41f642 --- /dev/null +++ b/src/Common/tests/gtest_log.cpp @@ -0,0 +1,19 @@ +#include +#include +#include +#include + +#include +#include +#include + + +TEST(Logger, Log) +{ + Poco::Logger::root().setLevel("none"); + Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::NullChannel())); + Poco::Logger * log = &Poco::Logger::get("Log"); + + /// This test checks that we don't pass this string to fmtlib, because it is the only argument. + EXPECT_NO_THROW(LOG_INFO(log, "Hello {} World")); +} diff --git a/src/Core/BackgroundSchedulePool.cpp b/src/Core/BackgroundSchedulePool.cpp index cc9ee23c9b8..a72838c92e9 100644 --- a/src/Core/BackgroundSchedulePool.cpp +++ b/src/Core/BackgroundSchedulePool.cpp @@ -111,7 +111,7 @@ void BackgroundSchedulePoolTaskInfo::execute() static const int32_t slow_execution_threshold_ms = 200; if (milliseconds >= slow_execution_threshold_ms) - LOG_TRACE(&Logger::get(log_name), "Execution took {} ms.", milliseconds); + LOG_TRACE(&Poco::Logger::get(log_name), "Execution took {} ms.", milliseconds); { std::lock_guard lock_schedule(schedule_mutex); @@ -156,7 +156,7 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size_, CurrentMetrics::Met , memory_metric(memory_metric_) , thread_name(thread_name_) { - LOG_INFO(&Logger::get("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with {} threads", size); + LOG_INFO(&Poco::Logger::get("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with {} threads", size); threads.resize(size); for (auto & thread : threads) @@ -179,7 +179,7 @@ BackgroundSchedulePool::~BackgroundSchedulePool() queue.wakeUpAll(); delayed_thread.join(); - LOG_TRACE(&Logger::get("BackgroundSchedulePool/" + thread_name), "Waiting for threads to finish."); + LOG_TRACE(&Poco::Logger::get("BackgroundSchedulePool/" + thread_name), "Waiting for threads to finish."); for (auto & thread : threads) thread.join(); } diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 62a99cea97e..5ec6980dbfa 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -164,7 +164,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, /// Create table NamesAndTypesList columns = sample_block.getNamesAndTypesList(); - auto temporary_table = TemporaryTableHolder(context, ColumnsDescription{columns}); + auto temporary_table = TemporaryTableHolder(context, ColumnsDescription{columns}, {}); auto storage = temporary_table.getTable(); context.addExternalTable(data->table_name, std::move(temporary_table)); BlockOutputStreamPtr output = storage->write(ASTPtr(), context); diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 2fd15f78c05..ee53276ee1b 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -994,7 +994,7 @@ private: class Sha256Password : public IPlugin { public: - Sha256Password(RSA & public_key_, RSA & private_key_, Logger * log_) + Sha256Password(RSA & public_key_, RSA & private_key_, Poco::Logger * log_) : public_key(public_key_) , private_key(private_key_) , log(log_) @@ -1130,7 +1130,7 @@ public: private: RSA & public_key; RSA & private_key; - Logger * log; + Poco::Logger * log; String scramble; }; #endif diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 142e0872d72..c3c8eedbb12 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -126,7 +126,7 @@ struct Settings : public SettingsCollection M(SettingBool, force_optimize_skip_unused_shards_no_nested, false, "Do not apply force_optimize_skip_unused_shards for nested Distributed tables.", 0) \ \ M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \ - M(SettingUInt64, min_chunk_bytes_for_parallel_parsing, (1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \ + M(SettingUInt64, min_chunk_bytes_for_parallel_parsing, (10 * 1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \ \ M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \ M(SettingUInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.", 0) \ @@ -310,7 +310,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, max_execution_speed, 0, "Maximum number of execution rows per second.", 0) \ M(SettingUInt64, min_execution_speed_bytes, 0, "Minimum number of execution bytes per second.", 0) \ M(SettingUInt64, max_execution_speed_bytes, 0, "Maximum number of execution bytes per second.", 0) \ - M(SettingSeconds, timeout_before_checking_execution_speed, 0, "Check that the speed is not too low after the specified time has elapsed.", 0) \ + M(SettingSeconds, timeout_before_checking_execution_speed, 10, "Check that the speed is not too low after the specified time has elapsed.", 0) \ \ M(SettingUInt64, max_columns_to_read, 0, "", 0) \ M(SettingUInt64, max_temporary_columns, 0, "", 0) \ @@ -425,6 +425,8 @@ struct Settings : public SettingsCollection M(SettingSeconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \ M(SettingBool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \ \ + M(SettingBool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ + \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13", 0) \ @@ -437,6 +439,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, mark_cache_min_lifetime, 0, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ M(SettingBool, partial_merge_join, false, "Obsolete. Use join_algorithm='prefer_partial_merge' instead.", 0) \ M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \ + M(SettingBool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) \ DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) diff --git a/src/Core/SettingsCollection.cpp b/src/Core/SettingsCollection.cpp index 3b291406df9..324ad889a65 100644 --- a/src/Core/SettingsCollection.cpp +++ b/src/Core/SettingsCollection.cpp @@ -598,7 +598,7 @@ namespace details void SettingsCollectionUtils::warningNameNotFound(const StringRef & name) { - static auto * log = &Logger::get("Settings"); + static auto * log = &Poco::Logger::get("Settings"); LOG_WARNING(log, "Unknown setting {}, skipping", name); } diff --git a/src/DataStreams/AggregatingBlockInputStream.cpp b/src/DataStreams/AggregatingBlockInputStream.cpp index 7c891271468..150f794ca59 100644 --- a/src/DataStreams/AggregatingBlockInputStream.cpp +++ b/src/DataStreams/AggregatingBlockInputStream.cpp @@ -60,7 +60,7 @@ Block AggregatingBlockInputStream::readImpl() input_streams.emplace_back(temporary_inputs.back()->block_in); } - LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), formatReadableSizeWithBinarySuffix(files.sum_size_compressed), formatReadableSizeWithBinarySuffix(files.sum_size_uncompressed)); + LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), ReadableSize(files.sum_size_compressed), ReadableSize(files.sum_size_uncompressed)); impl = std::make_unique(input_streams, params, final, 1, 1); } diff --git a/src/DataStreams/AggregatingBlockInputStream.h b/src/DataStreams/AggregatingBlockInputStream.h index 5e993949b63..009a9704e4e 100644 --- a/src/DataStreams/AggregatingBlockInputStream.h +++ b/src/DataStreams/AggregatingBlockInputStream.h @@ -47,7 +47,7 @@ protected: /** From here we will get the completed blocks after the aggregation. */ std::unique_ptr impl; - Logger * log = &Logger::get("AggregatingBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("AggregatingBlockInputStream"); }; } diff --git a/src/DataStreams/BlocksBlockInputStream.h b/src/DataStreams/BlocksBlockInputStream.h index 6301a92b6a4..3062a24a44c 100644 --- a/src/DataStreams/BlocksBlockInputStream.h +++ b/src/DataStreams/BlocksBlockInputStream.h @@ -13,6 +13,7 @@ limitations under the License. */ #include #include +#include namespace DB @@ -38,7 +39,12 @@ protected: Block res = *it; ++it; - return Chunk(res.getColumns(), res.rows()); + + auto info = std::make_shared(); + info->bucket_num = res.info.bucket_num; + info->is_overflows = res.info.is_overflows; + + return Chunk(res.getColumns(), res.rows(), std::move(info)); } private: diff --git a/src/DataStreams/CollapsingFinalBlockInputStream.h b/src/DataStreams/CollapsingFinalBlockInputStream.h index c237d5465c3..d090c53ddf9 100644 --- a/src/DataStreams/CollapsingFinalBlockInputStream.h +++ b/src/DataStreams/CollapsingFinalBlockInputStream.h @@ -168,7 +168,7 @@ private: const SortDescription description; String sign_column_name; - Logger * log = &Logger::get("CollapsingFinalBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("CollapsingFinalBlockInputStream"); bool first = true; diff --git a/src/DataStreams/ColumnGathererStream.cpp b/src/DataStreams/ColumnGathererStream.cpp index b98ee96d26a..7a6dc7d88f0 100644 --- a/src/DataStreams/ColumnGathererStream.cpp +++ b/src/DataStreams/ColumnGathererStream.cpp @@ -21,7 +21,7 @@ ColumnGathererStream::ColumnGathererStream( const String & column_name_, const BlockInputStreams & source_streams, ReadBuffer & row_sources_buf_, size_t block_preferred_size_) : column_name(column_name_), sources(source_streams.size()), row_sources_buf(row_sources_buf_) - , block_preferred_size(block_preferred_size_), log(&Logger::get("ColumnGathererStream")) + , block_preferred_size(block_preferred_size_), log(&Poco::Logger::get("ColumnGathererStream")) { if (source_streams.empty()) throw Exception("There are no streams to gather", ErrorCodes::EMPTY_DATA_PASSED); @@ -105,7 +105,7 @@ void ColumnGathererStream::readSuffixImpl() else LOG_DEBUG(log, "Gathered column {} ({} bytes/elem.) in {} sec., {} rows/sec., {}/sec.", column_name, static_cast(profile_info.bytes) / profile_info.rows, seconds, - profile_info.rows / seconds, formatReadableSizeWithBinarySuffix(profile_info.bytes / seconds)); + profile_info.rows / seconds, ReadableSize(profile_info.bytes / seconds)); } } diff --git a/src/DataStreams/CreatingSetsBlockInputStream.h b/src/DataStreams/CreatingSetsBlockInputStream.h index 4a4ffef29d6..178fa3f289f 100644 --- a/src/DataStreams/CreatingSetsBlockInputStream.h +++ b/src/DataStreams/CreatingSetsBlockInputStream.h @@ -44,7 +44,7 @@ private: size_t bytes_to_transfer = 0; using Logger = Poco::Logger; - Logger * log = &Logger::get("CreatingSetsBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("CreatingSetsBlockInputStream"); void createAll(); void createOne(SubqueryForSet & subquery); diff --git a/src/DataStreams/ExecutionSpeedLimits.cpp b/src/DataStreams/ExecutionSpeedLimits.cpp index 28a8cd94994..6cc1b9006bf 100644 --- a/src/DataStreams/ExecutionSpeedLimits.cpp +++ b/src/DataStreams/ExecutionSpeedLimits.cpp @@ -30,7 +30,8 @@ static void limitProgressingSpeed(size_t total_progress_size, size_t max_speed_i { UInt64 sleep_microseconds = desired_microseconds - total_elapsed_microseconds; - /// Never sleep more than one second (it should be enough to limit speed for a reasonable amount, and otherwise it's too easy to make query hang). + /// Never sleep more than one second (it should be enough to limit speed for a reasonable amount, + /// and otherwise it's too easy to make query hang). sleep_microseconds = std::min(UInt64(1000000), sleep_microseconds); sleepForMicroseconds(sleep_microseconds); @@ -45,14 +46,14 @@ void ExecutionSpeedLimits::throttle( { if ((min_execution_rps != 0 || max_execution_rps != 0 || min_execution_bps != 0 || max_execution_bps != 0 - || (total_rows_to_read != 0 && timeout_before_checking_execution_speed != 0)) && - (static_cast(total_elapsed_microseconds) > timeout_before_checking_execution_speed.totalMicroseconds())) + || (total_rows_to_read != 0 && timeout_before_checking_execution_speed != 0)) + && (static_cast(total_elapsed_microseconds) > timeout_before_checking_execution_speed.totalMicroseconds())) { /// Do not count sleeps in throttlers UInt64 throttler_sleep_microseconds = CurrentThread::getProfileEvents()[ProfileEvents::ThrottlerSleepMicroseconds]; double elapsed_seconds = 0; - if (throttler_sleep_microseconds > total_elapsed_microseconds) + if (total_elapsed_microseconds > throttler_sleep_microseconds) elapsed_seconds = static_cast(total_elapsed_microseconds - throttler_sleep_microseconds) / 1000000.0; if (elapsed_seconds > 0) diff --git a/src/DataStreams/InputStreamFromASTInsertQuery.cpp b/src/DataStreams/InputStreamFromASTInsertQuery.cpp index 3b4946e4bc9..47b61294da3 100644 --- a/src/DataStreams/InputStreamFromASTInsertQuery.cpp +++ b/src/DataStreams/InputStreamFromASTInsertQuery.cpp @@ -58,7 +58,7 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery( if (context.getSettingsRef().input_format_defaults_for_omitted_fields && ast_insert_query->table_id && !input_function) { - StoragePtr storage = DatabaseCatalog::instance().getTable(ast_insert_query->table_id); + StoragePtr storage = DatabaseCatalog::instance().getTable(ast_insert_query->table_id, context); auto column_defaults = storage->getColumns().getDefaults(); if (!column_defaults.empty()) res_stream = std::make_shared(res_stream, column_defaults, context); diff --git a/src/DataStreams/MergeSortingBlockInputStream.cpp b/src/DataStreams/MergeSortingBlockInputStream.cpp index 587f21d2467..5e1fbe599b7 100644 --- a/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -264,7 +264,7 @@ void MergeSortingBlockInputStream::remerge() } merger.readSuffix(); - LOG_DEBUG(log, "Memory usage is lowered from {} to {}", formatReadableSizeWithBinarySuffix(sum_bytes_in_blocks), formatReadableSizeWithBinarySuffix(new_sum_bytes_in_blocks)); + LOG_DEBUG(log, "Memory usage is lowered from {} to {}", ReadableSize(sum_bytes_in_blocks), ReadableSize(new_sum_bytes_in_blocks)); /// If the memory consumption was not lowered enough - we will not perform remerge anymore. 2 is a guess. if (new_sum_bytes_in_blocks * 2 > sum_bytes_in_blocks) diff --git a/src/DataStreams/MergeSortingBlockInputStream.h b/src/DataStreams/MergeSortingBlockInputStream.h index ee03f202be0..c21c548ff24 100644 --- a/src/DataStreams/MergeSortingBlockInputStream.h +++ b/src/DataStreams/MergeSortingBlockInputStream.h @@ -104,7 +104,7 @@ private: String codec; size_t min_free_disk_space; - Logger * log = &Logger::get("MergeSortingBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("MergeSortingBlockInputStream"); Blocks blocks; size_t sum_rows_in_blocks = 0; diff --git a/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp b/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp index 9f514b6701d..56dcbda0487 100644 --- a/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp +++ b/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp @@ -555,7 +555,7 @@ MergingAggregatedMemoryEfficientBlockInputStream::BlocksToMerge MergingAggregate /// Not yet partitioned (splitted to buckets) block. Will partition it and place result to 'splitted_blocks'. if (input.block.info.bucket_num == -1 && input.block && input.splitted_blocks.empty()) { - LOG_TRACE(&Logger::get("MergingAggregatedMemoryEfficient"), "Having block without bucket: will split."); + LOG_TRACE(&Poco::Logger::get("MergingAggregatedMemoryEfficient"), "Having block without bucket: will split."); input.splitted_blocks = aggregator.convertBlockToTwoLevel(input.block); input.block = Block(); diff --git a/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h b/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h index 4a6eccfd2a4..f071ac42ee5 100644 --- a/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h +++ b/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h @@ -96,7 +96,7 @@ private: std::atomic has_overflows {false}; int current_bucket_num = -1; - Logger * log = &Logger::get("MergingAggregatedMemoryEfficientBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("MergingAggregatedMemoryEfficientBlockInputStream"); struct Input diff --git a/src/DataStreams/MergingSortedBlockInputStream.cpp b/src/DataStreams/MergingSortedBlockInputStream.cpp index 877213294ec..434c89803b8 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -23,7 +23,7 @@ MergingSortedBlockInputStream::MergingSortedBlockInputStream( : description(std::move(description_)), max_block_size(max_block_size_), limit(limit_), quiet(quiet_) , source_blocks(inputs_.size()) , cursors(inputs_.size()), out_row_sources_buf(out_row_sources_buf_) - , log(&Logger::get("MergingSortedBlockInputStream")) + , log(&Poco::Logger::get("MergingSortedBlockInputStream")) { children.insert(children.end(), inputs_.begin(), inputs_.end()); header = children.at(0)->getHeader(); @@ -269,7 +269,7 @@ void MergingSortedBlockInputStream::readSuffixImpl() LOG_DEBUG(log, "Merge sorted {} blocks, {} rows in {} sec., {} rows/sec., {}/sec", profile_info.blocks, profile_info.rows, seconds, profile_info.rows / seconds, - formatReadableSizeWithBinarySuffix(profile_info.bytes / seconds)); + ReadableSize(profile_info.bytes / seconds)); } } diff --git a/src/DataStreams/ParallelAggregatingBlockInputStream.cpp b/src/DataStreams/ParallelAggregatingBlockInputStream.cpp index aeda8ea5be9..611059c1443 100644 --- a/src/DataStreams/ParallelAggregatingBlockInputStream.cpp +++ b/src/DataStreams/ParallelAggregatingBlockInputStream.cpp @@ -82,7 +82,7 @@ Block ParallelAggregatingBlockInputStream::readImpl() input_streams.emplace_back(temporary_inputs.back()->block_in); } - LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), formatReadableSizeWithBinarySuffix(files.sum_size_compressed), formatReadableSizeWithBinarySuffix(files.sum_size_uncompressed)); + LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), ReadableSize(files.sum_size_compressed), ReadableSize(files.sum_size_uncompressed)); impl = std::make_unique( input_streams, params, final, temporary_data_merge_threads, temporary_data_merge_threads); @@ -178,16 +178,16 @@ void ParallelAggregatingBlockInputStream::execute() { size_t rows = many_data[i]->size(); LOG_TRACE(log, "Aggregated. {} to {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)", - threads_data[i].src_rows, rows, formatReadableSizeWithBinarySuffix(threads_data[i].src_bytes), + threads_data[i].src_rows, rows, ReadableSize(threads_data[i].src_bytes), elapsed_seconds, threads_data[i].src_rows / elapsed_seconds, - formatReadableSizeWithBinarySuffix(threads_data[i].src_bytes / elapsed_seconds)); + ReadableSize(threads_data[i].src_bytes / elapsed_seconds)); total_src_rows += threads_data[i].src_rows; total_src_bytes += threads_data[i].src_bytes; } LOG_TRACE(log, "Total aggregated. {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)", - total_src_rows, formatReadableSizeWithBinarySuffix(total_src_bytes), elapsed_seconds, - total_src_rows / elapsed_seconds, formatReadableSizeWithBinarySuffix(total_src_bytes / elapsed_seconds)); + total_src_rows, ReadableSize(total_src_bytes), elapsed_seconds, + total_src_rows / elapsed_seconds, ReadableSize(total_src_bytes / elapsed_seconds)); /// If there was no data, and we aggregate without keys, we must return single row with the result of empty aggregation. /// To do this, we pass a block with zero rows to aggregate. diff --git a/src/DataStreams/ParallelAggregatingBlockInputStream.h b/src/DataStreams/ParallelAggregatingBlockInputStream.h index 942c906b872..4b0a2e806fa 100644 --- a/src/DataStreams/ParallelAggregatingBlockInputStream.h +++ b/src/DataStreams/ParallelAggregatingBlockInputStream.h @@ -60,7 +60,7 @@ private: std::atomic executed {false}; std::vector> temporary_inputs; - Logger * log = &Logger::get("ParallelAggregatingBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("ParallelAggregatingBlockInputStream"); ManyAggregatedDataVariants many_data; diff --git a/src/DataStreams/ParallelInputsProcessor.h b/src/DataStreams/ParallelInputsProcessor.h index 714174e6ac1..326506d28ca 100644 --- a/src/DataStreams/ParallelInputsProcessor.h +++ b/src/DataStreams/ParallelInputsProcessor.h @@ -359,7 +359,7 @@ private: /// Wait for the completion of all threads. std::atomic joined_threads { false }; - Logger * log = &Logger::get("ParallelInputsProcessor"); + Poco::Logger * log = &Poco::Logger::get("ParallelInputsProcessor"); }; diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 7f730b5fd3f..a4f0b58adeb 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -59,7 +59,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( for (const auto & database_table : dependencies) { - auto dependent_table = DatabaseCatalog::instance().getTable(database_table); + auto dependent_table = DatabaseCatalog::instance().getTable(database_table, context); ASTPtr query; BlockOutputStreamPtr out; @@ -274,7 +274,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n StorageValues::create( storage->getStorageID(), storage->getColumns(), block, storage->getVirtuals())); select.emplace(view.query, local_context, SelectQueryOptions()); - in = std::make_shared(select->execute().in); + in = std::make_shared(select->execute().getInputStream()); /// Squashing is needed here because the materialized view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY diff --git a/src/DataStreams/RemoteBlockInputStream.h b/src/DataStreams/RemoteBlockInputStream.h index 66b1ebbb6c3..f6bac4155da 100644 --- a/src/DataStreams/RemoteBlockInputStream.h +++ b/src/DataStreams/RemoteBlockInputStream.h @@ -151,7 +151,7 @@ private: PoolMode pool_mode = PoolMode::GET_MANY; StorageID main_table = StorageID::createEmpty(); - Logger * log = &Logger::get("RemoteBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("RemoteBlockInputStream"); }; } diff --git a/src/DataStreams/SizeLimits.cpp b/src/DataStreams/SizeLimits.cpp index ed57f66d21a..06dde923e55 100644 --- a/src/DataStreams/SizeLimits.cpp +++ b/src/DataStreams/SizeLimits.cpp @@ -16,8 +16,8 @@ bool SizeLimits::check(UInt64 rows, UInt64 bytes, const char * what, int too_man + ", current rows: " + formatReadableQuantity(rows), too_many_rows_exception_code); if (max_bytes && bytes > max_bytes) - throw Exception("Limit for " + std::string(what) + " exceeded, max bytes: " + formatReadableSizeWithBinarySuffix(max_bytes) - + ", current bytes: " + formatReadableSizeWithBinarySuffix(bytes), too_many_bytes_exception_code); + throw Exception(fmt::format("Limit for {} exceeded, max bytes: {}, current bytes: {}", + std::string(what), ReadableSize(max_bytes), ReadableSize(bytes)), too_many_bytes_exception_code); return true; } diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index c6542763533..ca65ae520c6 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -28,7 +28,7 @@ TTLBlockInputStream::TTLBlockInputStream( , current_time(current_time_) , force(force_) , old_ttl_infos(data_part->ttl_infos) - , log(&Logger::get(storage.getLogName() + " (TTLBlockInputStream)")) + , log(&Poco::Logger::get(storage.getLogName() + " (TTLBlockInputStream)")) , date_lut(DateLUT::instance()) { children.push_back(input_); diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index c6ffa95cd75..060306f7d2d 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -52,7 +52,7 @@ private: NameSet empty_columns; size_t rows_removed = 0; - Logger * log; + Poco::Logger * log; const DateLUTImpl & date_lut; /// TODO rewrite defaults logic to evaluteMissingDefaults diff --git a/src/DataStreams/UnionBlockInputStream.h b/src/DataStreams/UnionBlockInputStream.h index c4e84e85845..ab667ac2e36 100644 --- a/src/DataStreams/UnionBlockInputStream.h +++ b/src/DataStreams/UnionBlockInputStream.h @@ -253,7 +253,7 @@ private: bool started = false; bool all_read = false; - Logger * log = &Logger::get("UnionBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("UnionBlockInputStream"); }; } diff --git a/src/DataStreams/tests/collapsing_sorted_stream.cpp b/src/DataStreams/tests/collapsing_sorted_stream.cpp index fd7dc11add6..e6d2167578b 100644 --- a/src/DataStreams/tests/collapsing_sorted_stream.cpp +++ b/src/DataStreams/tests/collapsing_sorted_stream.cpp @@ -20,8 +20,8 @@ try using namespace DB; Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); Block block1; diff --git a/src/DataStreams/tests/union_stream2.cpp b/src/DataStreams/tests/union_stream2.cpp index 6e0997e7f72..5b84d89a435 100644 --- a/src/DataStreams/tests/union_stream2.cpp +++ b/src/DataStreams/tests/union_stream2.cpp @@ -35,7 +35,7 @@ try Names column_names; column_names.push_back("WatchID"); - StoragePtr table = DatabaseCatalog::instance().getTable({"default", "hits6"}); + StoragePtr table = DatabaseCatalog::instance().getTable({"default", "hits6"}, context); QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); auto pipes = table->read(column_names, {}, context, stage, settings.max_block_size, settings.max_threads); diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp new file mode 100644 index 00000000000..baffde700c8 --- /dev/null +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -0,0 +1,131 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +class DataTypeCustomPointSerialization : public DataTypeCustomSimpleTextSerialization +{ +public: + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override + { + nestedDataType()->serializeAsText(column, row_num, ostr, settings); + } + + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + nestedDataType()->deserializeAsWholeText(column, istr, settings); + } + + static DataTypePtr nestedDataType() + { + static auto data_type = DataTypePtr(std::make_unique( + DataTypes({std::make_unique(), std::make_unique()}))); + return data_type; + } +}; + +class DataTypeCustomRingSerialization : public DataTypeCustomSimpleTextSerialization +{ +public: + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override + { + nestedDataType()->serializeAsText(column, row_num, ostr, settings); + } + + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + nestedDataType()->deserializeAsWholeText(column, istr, settings); + } + + static DataTypePtr nestedDataType() + { + static auto data_type = DataTypePtr(std::make_unique(DataTypeCustomPointSerialization::nestedDataType())); + return data_type; + } +}; + +class DataTypeCustomPolygonSerialization : public DataTypeCustomSimpleTextSerialization +{ +public: + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override + { + nestedDataType()->serializeAsText(column, row_num, ostr, settings); + } + + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + nestedDataType()->deserializeAsWholeText(column, istr, settings); + } + + static DataTypePtr nestedDataType() + { + static auto data_type = DataTypePtr(std::make_unique(DataTypeCustomRingSerialization::nestedDataType())); + return data_type; + } +}; + +class DataTypeCustomMultiPolygonSerialization : public DataTypeCustomSimpleTextSerialization +{ +public: + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override + { + nestedDataType()->serializeAsText(column, row_num, ostr, settings); + } + + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + nestedDataType()->deserializeAsWholeText(column, istr, settings); + } + + static DataTypePtr nestedDataType() + { + static auto data_type = DataTypePtr(std::make_unique(DataTypeCustomPolygonSerialization::nestedDataType())); + return data_type; + } +}; + +} + +void registerDataTypeDomainGeo(DataTypeFactory & factory) +{ + // Custom type for point represented as its coordinates stored as Tuple(Float64, Float64) + factory.registerSimpleDataTypeCustom("Point", [] + { + return std::make_pair(DataTypeFactory::instance().get("Tuple(Float64, Float64)"), + std::make_unique(std::make_unique("Point"), std::make_unique())); + }); + + // Custom type for simple polygon without holes stored as Array(Point) + factory.registerSimpleDataTypeCustom("Ring", [] + { + return std::make_pair(DataTypeFactory::instance().get("Array(Point)"), + std::make_unique(std::make_unique("Ring"), std::make_unique())); + }); + + // Custom type for polygon with holes stored as Array(Ring) + // First element of outer array is outer shape of polygon and all the following are holes + factory.registerSimpleDataTypeCustom("Polygon", [] + { + return std::make_pair(DataTypeFactory::instance().get("Array(Ring)"), + std::make_unique(std::make_unique("Polygon"), std::make_unique())); + }); + + // Custom type for multiple polygons with holes stored as Array(Polygon) + factory.registerSimpleDataTypeCustom("MultiPolygon", [] + { + return std::make_pair(DataTypeFactory::instance().get("Array(Polygon)"), + std::make_unique(std::make_unique("MultiPolygon"), std::make_unique())); + }); +} + +} diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index f81adfe347c..880f25d009d 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -180,6 +180,7 @@ DataTypeFactory::DataTypeFactory() registerDataTypeLowCardinality(*this); registerDataTypeDomainIPv4AndIPv6(*this); registerDataTypeDomainSimpleAggregateFunction(*this); + registerDataTypeDomainGeo(*this); } DataTypeFactory & DataTypeFactory::instance() diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index 8844e95d1bc..6bf09d31727 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -83,5 +83,6 @@ void registerDataTypeLowCardinality(DataTypeFactory & factory); void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory); void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory); void registerDataTypeDateTime64(DataTypeFactory & factory); +void registerDataTypeDomainGeo(DataTypeFactory & factory); } diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make index c3a1e452d0d..82e9baf76f2 100644 --- a/src/DataTypes/ya.make +++ b/src/DataTypes/ya.make @@ -1,3 +1,4 @@ +# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. LIBRARY() PEERDIR( @@ -9,12 +10,13 @@ SRCS( convertMySQLDataType.cpp DataTypeAggregateFunction.cpp DataTypeArray.cpp + DataTypeCustomGeo.cpp DataTypeCustomIPv4AndIPv6.cpp DataTypeCustomSimpleAggregateFunction.cpp DataTypeCustomSimpleTextSerialization.cpp DataTypeDate.cpp - DataTypeDateTime.cpp DataTypeDateTime64.cpp + DataTypeDateTime.cpp DataTypeDecimalBase.cpp DataTypeEnum.cpp DataTypeFactory.cpp @@ -36,6 +38,7 @@ SRCS( getMostSubtype.cpp IDataType.cpp NestedUtils.cpp + ) END() diff --git a/src/DataTypes/ya.make.in b/src/DataTypes/ya.make.in new file mode 100644 index 00000000000..39cbdefe361 --- /dev/null +++ b/src/DataTypes/ya.make.in @@ -0,0 +1,12 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + clickhouse/src/Formats +) + +SRCS( + +) + +END() diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 27a2441cec6..ed875f5fff4 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -288,15 +288,15 @@ void DatabaseAtomic::assertCanBeDetached(bool cleenup) "because some tables are still in use. Retry later.", ErrorCodes::DATABASE_NOT_EMPTY); } -DatabaseTablesIteratorPtr DatabaseAtomic::getTablesIterator(const IDatabase::FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseAtomic::getTablesIterator(const Context & context, const IDatabase::FilterByNameFunction & filter_by_table_name) { - auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(filter_by_table_name); + auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(context, filter_by_table_name); return std::make_unique(std::move(typeid_cast(*base_iter))); } UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const { - if (auto table = tryGetTable(table_name)) + if (auto table = tryGetTable(table_name, global_context)) return table->getStorageID().uuid; return UUIDHelpers::Nil; } diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 84297a42843..71428fdb420 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -42,7 +42,7 @@ public: void drop(const Context & /*context*/) override; - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index 81bcbf7bc6b..67d33d7bfd7 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -32,7 +32,7 @@ namespace DatabaseDictionary::DatabaseDictionary(const String & name_, const Context & global_context_) : IDatabase(name_) - , log(&Logger::get("DatabaseDictionary(" + database_name + ")")) + , log(&Poco::Logger::get("DatabaseDictionary(" + database_name + ")")) , global_context(global_context_.getGlobalContext()) { } @@ -50,18 +50,18 @@ Tables DatabaseDictionary::listTables(const FilterByNameFunction & filter_by_nam return tables; } -bool DatabaseDictionary::isTableExist(const String & table_name) const +bool DatabaseDictionary::isTableExist(const String & table_name, const Context &) const { return global_context.getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST; } -StoragePtr DatabaseDictionary::tryGetTable(const String & table_name) const +StoragePtr DatabaseDictionary::tryGetTable(const String & table_name, const Context &) const { auto load_result = global_context.getExternalDictionariesLoader().getLoadResult(table_name); return createStorageDictionary(getDatabaseName(), load_result); } -DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) { return std::make_unique(listTables(filter_by_table_name)); } @@ -71,7 +71,7 @@ bool DatabaseDictionary::empty() const return !global_context.getExternalDictionariesLoader().hasObjects(); } -ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const +ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const { String query; { diff --git a/src/Databases/DatabaseDictionary.h b/src/Databases/DatabaseDictionary.h index bf10dc134bd..b61c85033e8 100644 --- a/src/Databases/DatabaseDictionary.h +++ b/src/Databases/DatabaseDictionary.h @@ -29,11 +29,11 @@ public: return "Dictionary"; } - bool isTableExist(const String & table_name) const override; + bool isTableExist(const String & table_name, const Context & context) const override; - StoragePtr tryGetTable(const String & table_name) const override; + StoragePtr tryGetTable(const String & table_name, const Context & context) const override; - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; bool empty() const override; @@ -44,7 +44,7 @@ public: void shutdown() override; protected: - ASTPtr getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const override; + ASTPtr getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const override; private: mutable std::mutex mutex; diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 21feb122da1..f27bc509ebe 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -82,7 +82,7 @@ DatabasePtr DatabaseFactory::getImpl( else if (engine_name == "Atomic") return std::make_shared(database_name, metadata_path, context); else if (engine_name == "Memory") - return std::make_shared(database_name); + return std::make_shared(database_name, context); else if (engine_name == "Dictionary") return std::make_shared(database_name, context); diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 51ee664a42a..11e5272110e 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -132,7 +132,7 @@ StoragePtr DatabaseLazy::tryGetTable(const String & table_name) const return loadTable(table_name); } -DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) { std::lock_guard lock(mutex); Strings filtered_tables; diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 4306e61b37b..2e24b687be5 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -51,13 +51,15 @@ public: time_t getObjectMetadataModificationTime(const String & table_name) const override; - bool isTableExist(const String & table_name) const override; + bool isTableExist(const String & table_name, const Context &) const override { return isTableExist(table_name); } + bool isTableExist(const String & table_name) const; - StoragePtr tryGetTable(const String & table_name) const override; + StoragePtr tryGetTable(const String & table_name, const Context &) const override { return tryGetTable(table_name); } + StoragePtr tryGetTable(const String & table_name) const; bool empty() const override; - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) override; diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 84fec6bcc22..52b1f889943 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -16,8 +16,8 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } -DatabaseMemory::DatabaseMemory(const String & name_) - : DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")") +DatabaseMemory::DatabaseMemory(const String & name_, const Context & context) + : DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")", context) , data_path("data/" + escapeForFileName(database_name) + "/") {} @@ -64,7 +64,7 @@ ASTPtr DatabaseMemory::getCreateDatabaseQuery() const return create_query; } -ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const +ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const { std::lock_guard lock{mutex}; auto it = create_queries.find(table_name); @@ -80,7 +80,7 @@ ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, bool t UUID DatabaseMemory::tryGetTableUUID(const String & table_name) const { - if (auto table = tryGetTable(table_name)) + if (auto table = tryGetTable(table_name, global_context)) return table->getStorageID().uuid; return UUIDHelpers::Nil; } diff --git a/src/Databases/DatabaseMemory.h b/src/Databases/DatabaseMemory.h index ad34c4d9097..40cc808e42b 100644 --- a/src/Databases/DatabaseMemory.h +++ b/src/Databases/DatabaseMemory.h @@ -19,7 +19,7 @@ namespace DB class DatabaseMemory final : public DatabaseWithOwnTablesBase { public: - DatabaseMemory(const String & name_); + DatabaseMemory(const String & name_, const Context & context); String getEngineName() const override { return "Memory"; } @@ -34,7 +34,7 @@ public: const String & table_name, bool no_delay) override; - ASTPtr getCreateTableQueryImpl(const String & name, bool throw_on_error) const override; + ASTPtr getCreateTableQueryImpl(const String & name, const Context & context, bool throw_on_error) const override; ASTPtr getCreateDatabaseQuery() const override; /// DatabaseMemory allows to create tables, which store data on disk. diff --git a/src/Databases/DatabaseMySQL.cpp b/src/Databases/DatabaseMySQL.cpp index cbd3261975a..5d4b81014f9 100644 --- a/src/Databases/DatabaseMySQL.cpp +++ b/src/Databases/DatabaseMySQL.cpp @@ -89,7 +89,7 @@ bool DatabaseMySQL::empty() const return true; } -DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) { Tables tables; std::lock_guard lock(mutex); @@ -103,12 +103,12 @@ DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const FilterByNameFun return std::make_unique(tables); } -bool DatabaseMySQL::isTableExist(const String & name) const +bool DatabaseMySQL::isTableExist(const String & name, const Context &) const { - return bool(tryGetTable(name)); + return bool(tryGetTable(name, global_context)); } -StoragePtr DatabaseMySQL::tryGetTable(const String & mysql_table_name) const +StoragePtr DatabaseMySQL::tryGetTable(const String & mysql_table_name, const Context &) const { std::lock_guard lock(mutex); @@ -155,7 +155,7 @@ static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr return create_table_query; } -ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const +ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const { std::lock_guard lock(mutex); @@ -501,7 +501,7 @@ void DatabaseMySQL::createTable(const Context &, const String & table_name, cons /// XXX: hack /// In order to prevent users from broken the table structure by executing attach table database_name.table_name (...) /// we should compare the old and new create_query to make them completely consistent - const auto & origin_create_query = getCreateTableQuery(table_name); + const auto & origin_create_query = getCreateTableQuery(table_name, global_context); origin_create_query->as()->attach = true; if (queryToString(origin_create_query) != queryToString(create_query)) diff --git a/src/Databases/DatabaseMySQL.h b/src/Databases/DatabaseMySQL.h index a43da5d1762..70d8dc81e0f 100644 --- a/src/Databases/DatabaseMySQL.h +++ b/src/Databases/DatabaseMySQL.h @@ -32,13 +32,13 @@ public: bool empty() const override; - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; ASTPtr getCreateDatabaseQuery() const override; - bool isTableExist(const String & name) const override; + bool isTableExist(const String & name, const Context & context) const override; - StoragePtr tryGetTable(const String & name) const override; + StoragePtr tryGetTable(const String & name, const Context & context) const override; time_t getObjectMetadataModificationTime(const String & name) const override; @@ -59,7 +59,7 @@ public: void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; protected: - ASTPtr getCreateTableQueryImpl(const String & name, bool throw_on_error) const override; + ASTPtr getCreateTableQueryImpl(const String & name, const Context & context, bool throw_on_error) const override; private: const Context & global_context; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 70f86b19f1f..1b542c7a1ff 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -123,10 +123,9 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) } DatabaseOnDisk::DatabaseOnDisk(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context) - : DatabaseWithOwnTablesBase(name, logger) + : DatabaseWithOwnTablesBase(name, logger, context) , metadata_path(metadata_path_) , data_path(data_path_) - , global_context(context.getGlobalContext()) { Poco::File(context.getPath() + data_path).createDirectories(); Poco::File(metadata_path).createDirectories(); @@ -160,7 +159,7 @@ void DatabaseOnDisk::createTable( throw Exception("Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); - if (isTableExist(table_name)) + if (isTableExist(table_name, global_context)) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); if (create.attach_short_syntax) @@ -267,7 +266,7 @@ void DatabaseOnDisk::renameTable( String table_metadata_path; ASTPtr attach_query; /// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case. - StoragePtr table = tryGetTable(table_name); + StoragePtr table = tryGetTable(table_name, global_context); detachTable(table_name); try { @@ -304,10 +303,10 @@ void DatabaseOnDisk::renameTable( Poco::File(table_metadata_path).remove(); } -ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const +ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const { ASTPtr ast; - bool has_table = tryGetTable(table_name) != nullptr; + bool has_table = tryGetTable(table_name, global_context) != nullptr; auto table_metadata_path = getObjectMetadataPath(table_name); try { diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index 75609e231af..d4fb9b2aa17 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -76,6 +76,7 @@ protected: ASTPtr getCreateTableQueryImpl( const String & table_name, + const Context & context, bool throw_on_error) const override; ASTPtr getCreateQueryFromMetadata(const String & metadata_path, bool throw_on_error) const; @@ -85,7 +86,6 @@ protected: const String metadata_path; const String data_path; - const Context & global_context; }; } diff --git a/src/Databases/DatabaseWithDictionaries.cpp b/src/Databases/DatabaseWithDictionaries.cpp index 0d49078bd08..e0f2aa9286b 100644 --- a/src/Databases/DatabaseWithDictionaries.cpp +++ b/src/Databases/DatabaseWithDictionaries.cpp @@ -127,7 +127,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S "Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); - if (isTableExist(dictionary_name)) + if (isTableExist(dictionary_name, global_context)) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 5ad4fa20690..47c54fae800 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -18,18 +19,18 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } -DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger) - : IDatabase(name_), log(&Logger::get(logger)) +DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, const Context & context) + : IDatabase(name_), log(&Poco::Logger::get(logger)), global_context(context.getGlobalContext()) { } -bool DatabaseWithOwnTablesBase::isTableExist(const String & table_name) const +bool DatabaseWithOwnTablesBase::isTableExist(const String & table_name, const Context &) const { std::lock_guard lock(mutex); return tables.find(table_name) != tables.end(); } -StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name) const +StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, const Context &) const { std::lock_guard lock(mutex); auto it = tables.find(table_name); @@ -38,7 +39,7 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name) con return {}; } -DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) { std::lock_guard lock(mutex); if (!filter_by_table_name) diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index e195fc59ce9..4238fd30137 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -19,9 +19,9 @@ class Context; class DatabaseWithOwnTablesBase : public IDatabase { public: - bool isTableExist(const String & table_name) const override; + bool isTableExist(const String & table_name, const Context & context) const override; - StoragePtr tryGetTable(const String & table_name) const override; + StoragePtr tryGetTable(const String & table_name, const Context & context) const override; bool empty() const override; @@ -29,18 +29,19 @@ public: StoragePtr detachTable(const String & table_name) override; - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; void shutdown() override; - virtual ~DatabaseWithOwnTablesBase() override; + ~DatabaseWithOwnTablesBase() override; protected: mutable std::mutex mutex; Tables tables; Poco::Logger * log; + const Context & global_context; - DatabaseWithOwnTablesBase(const String & name_, const String & logger); + DatabaseWithOwnTablesBase(const String & name_, const String & logger, const Context & context); void attachTableUnlocked(const String & table_name, const StoragePtr & table, std::unique_lock & lock); StoragePtr detachTableUnlocked(const String & table_name, std::unique_lock & lock); diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 24147a0e7c5..3d108cfd91c 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -130,7 +130,7 @@ public: virtual void loadStoredObjects(Context & /*context*/, bool /*has_force_restore_data_flag*/) {} /// Check the existence of the table. - virtual bool isTableExist(const String & name) const = 0; + virtual bool isTableExist(const String & name, const Context & context) const = 0; /// Check the existence of the dictionary virtual bool isDictionaryExist(const String & /*name*/) const @@ -139,7 +139,7 @@ public: } /// Get the table for work. Return nullptr if there is no table. - virtual StoragePtr tryGetTable(const String & name) const = 0; + virtual StoragePtr tryGetTable(const String & name, const Context & context) const = 0; virtual UUID tryGetTableUUID(const String & /*table_name*/) const { return UUIDHelpers::Nil; } @@ -147,7 +147,7 @@ public: /// Get an iterator that allows you to pass through all the tables. /// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above. - virtual DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name = {}) = 0; + virtual DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0; /// Get an iterator to pass through all the dictionaries. virtual DatabaseDictionariesIteratorPtr getDictionariesIterator([[maybe_unused]] const FilterByNameFunction & filter_by_dictionary_name = {}) @@ -249,14 +249,14 @@ public: } /// Get the CREATE TABLE query for the table. It can also provide information for detached tables for which there is metadata. - ASTPtr tryGetCreateTableQuery(const String & name) const noexcept + ASTPtr tryGetCreateTableQuery(const String & name, const Context & context) const noexcept { - return getCreateTableQueryImpl(name, false); + return getCreateTableQueryImpl(name, context, false); } - ASTPtr getCreateTableQuery(const String & name) const + ASTPtr getCreateTableQuery(const String & name, const Context & context) const { - return getCreateTableQueryImpl(name, true); + return getCreateTableQueryImpl(name, context, true); } /// Get the CREATE DICTIONARY query for the dictionary. Returns nullptr if dictionary doesn't exists. @@ -304,7 +304,7 @@ public: virtual ~IDatabase() {} protected: - virtual ASTPtr getCreateTableQueryImpl(const String & /*name*/, bool throw_on_error) const + virtual ASTPtr getCreateTableQueryImpl(const String & /*name*/, const Context & /*context*/, bool throw_on_error) const { if (throw_on_error) throw Exception("There is no SHOW CREATE TABLE query for Database" + getEngineName(), ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY); diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index fffeb796c56..99d2445807b 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -85,7 +85,7 @@ CacheDictionary::CacheDictionary( , update_queue_push_timeout_milliseconds(update_queue_push_timeout_milliseconds_) , query_wait_timeout_milliseconds(query_wait_timeout_milliseconds_) , max_threads_for_updates(max_threads_for_updates_) - , log(&Logger::get("ExternalDictionaries")) + , log(&Poco::Logger::get("ExternalDictionaries")) , size{roundUpToPowerOfTwoOrZero(std::max(size_, size_t(max_collision_length)))} , size_overlap_mask{this->size - 1} , cells{this->size} diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index bb103c61107..8109a6d706b 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -314,7 +314,7 @@ private: const size_t query_wait_timeout_milliseconds; const size_t max_threads_for_updates; - Logger * const log; + Poco::Logger * log; mutable std::shared_mutex rw_lock; @@ -356,7 +356,7 @@ private: * How the update goes: we basically have a method like get(keys)->values. Values are cached, so sometimes we * can return them from the cache. For values not in cache, we query them from the dictionary, and add to the * cache. The cache is lossy, so we can't expect it to store all the keys, and we store them separately. Normally, - * they would be passed as a return value of get(), but for Unknown Reasons the dictionaries use a baroque + * they would be passed as a return value of get(), but for Unknown Reasons the dictionaries use a baroque * interface where get() accepts two callback, one that it calls for found values, and one for not found. * * Now we make it even uglier by doing this from multiple threads. The missing values are retreived from the diff --git a/src/Dictionaries/CacheDictionary.inc.h b/src/Dictionaries/CacheDictionary.inc.h index 71f3f1857ce..27064d113e6 100644 --- a/src/Dictionaries/CacheDictionary.inc.h +++ b/src/Dictionaries/CacheDictionary.inc.h @@ -302,37 +302,33 @@ void CacheDictionary::getItemsString( /// Request new values sync. /// We have request both cache_not_found_ids and cache_expired_ids. - if (!cache_not_found_ids.empty()) + std::vector required_ids; + required_ids.reserve(cache_not_found_ids.size() + cache_expired_ids.size()); + std::transform( + std::begin(cache_not_found_ids), std::end(cache_not_found_ids), + std::back_inserter(required_ids), [](auto & pair) { return pair.first; }); + std::transform( + std::begin(cache_expired_ids), std::end(cache_expired_ids), + std::back_inserter(required_ids), [](auto & pair) { return pair.first; }); + + auto on_cell_updated = [&] (const auto id, const auto cell_idx) { - std::vector required_ids; - required_ids.reserve(cache_not_found_ids.size() + cache_expired_ids.size()); - std::transform( - std::begin(cache_not_found_ids), std::end(cache_not_found_ids), - std::back_inserter(required_ids), [](auto & pair) { return pair.first; }); - std::transform( - std::begin(cache_expired_ids), std::end(cache_expired_ids), - std::back_inserter(required_ids), [](auto & pair) { return pair.first; }); + const auto attribute_value = attribute_array[cell_idx]; - auto on_cell_updated = [&] (const auto id, const auto cell_idx) - { - const auto attribute_value = attribute_array[cell_idx]; + map[id] = String{attribute_value}; + total_length += (attribute_value.size + 1) * cache_not_found_ids[id].size(); + }; - map[id] = String{attribute_value}; - total_length += (attribute_value.size + 1) * cache_not_found_ids[id].size(); - }; + auto on_id_not_found = [&] (const auto id, const auto) + { + for (const auto row : cache_not_found_ids[id]) + total_length += get_default(row).size + 1; + }; - auto on_id_not_found = [&] (const auto id, const auto) - { - for (const auto row : cache_not_found_ids[id]) - total_length += get_default(row).size + 1; - }; - - auto update_unit_ptr = std::make_shared(required_ids, on_cell_updated, on_id_not_found); - - tryPushToUpdateQueueOrThrow(update_unit_ptr); - waitForCurrentUpdateFinish(update_unit_ptr); - } + auto update_unit_ptr = std::make_shared(required_ids, on_cell_updated, on_id_not_found); + tryPushToUpdateQueueOrThrow(update_unit_ptr); + waitForCurrentUpdateFinish(update_unit_ptr); out->getChars().reserve(total_length); for (const auto row : ext::range(0, ext::size(ids))) diff --git a/src/Dictionaries/Embedded/RegionsHierarchies.cpp b/src/Dictionaries/Embedded/RegionsHierarchies.cpp index 9fb51d72457..15e14db4664 100644 --- a/src/Dictionaries/Embedded/RegionsHierarchies.cpp +++ b/src/Dictionaries/Embedded/RegionsHierarchies.cpp @@ -6,7 +6,7 @@ RegionsHierarchies::RegionsHierarchies(IRegionsHierarchiesDataProviderPtr data_provider) { - Logger * log = &Logger::get("RegionsHierarchies"); + Poco::Logger * log = &Poco::Logger::get("RegionsHierarchies"); LOG_DEBUG(log, "Adding default regions hierarchy"); data.emplace("", data_provider->getDefaultHierarchySource()); diff --git a/src/Dictionaries/Embedded/RegionsHierarchy.cpp b/src/Dictionaries/Embedded/RegionsHierarchy.cpp index dac1e948e0a..115ae30d93e 100644 --- a/src/Dictionaries/Embedded/RegionsHierarchy.cpp +++ b/src/Dictionaries/Embedded/RegionsHierarchy.cpp @@ -23,7 +23,7 @@ RegionsHierarchy::RegionsHierarchy(IRegionsHierarchyDataSourcePtr data_source_) void RegionsHierarchy::reload() { - Logger * log = &Logger::get("RegionsHierarchy"); + Poco::Logger * log = &Poco::Logger::get("RegionsHierarchy"); if (!data_source->isModified()) return; diff --git a/src/Dictionaries/Embedded/RegionsNames.cpp b/src/Dictionaries/Embedded/RegionsNames.cpp index b31debdbc26..30ba8259b3e 100644 --- a/src/Dictionaries/Embedded/RegionsNames.cpp +++ b/src/Dictionaries/Embedded/RegionsNames.cpp @@ -42,7 +42,7 @@ std::string RegionsNames::dumpSupportedLanguagesNames() void RegionsNames::reload() { - Logger * log = &Logger::get("RegionsNames"); + Poco::Logger * log = &Poco::Logger::get("RegionsNames"); LOG_DEBUG(log, "Reloading regions names"); RegionID max_region_id = 0; diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 0bb8c8543e8..38965e00a84 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -53,7 +53,7 @@ ExecutableDictionarySource::ExecutableDictionarySource( const std::string & config_prefix, Block & sample_block_, const Context & context_) - : log(&Logger::get("ExecutableDictionarySource")) + : log(&Poco::Logger::get("ExecutableDictionarySource")) , dict_struct{dict_struct_} , command{config.getString(config_prefix + ".command")} , update_field{config.getString(config_prefix + ".update_field", "")} @@ -64,7 +64,7 @@ ExecutableDictionarySource::ExecutableDictionarySource( } ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionarySource & other) - : log(&Logger::get("ExecutableDictionarySource")) + : log(&Poco::Logger::get("ExecutableDictionarySource")) , update_time{other.update_time} , dict_struct{other.dict_struct} , command{other.command} diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 7600253d5b8..95aab78ba2b 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -31,7 +31,7 @@ HTTPDictionarySource::HTTPDictionarySource( Block & sample_block_, const Context & context_, bool check_config) - : log(&Logger::get("HTTPDictionarySource")) + : log(&Poco::Logger::get("HTTPDictionarySource")) , update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , url{config.getString(config_prefix + ".url", "")} @@ -71,7 +71,7 @@ HTTPDictionarySource::HTTPDictionarySource( } HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) - : log(&Logger::get("HTTPDictionarySource")) + : log(&Poco::Logger::get("HTTPDictionarySource")) , update_time{other.update_time} , dict_struct{other.dict_struct} , url{other.url} diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 83d081707bd..ba538201910 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -125,7 +125,7 @@ LibraryDictionarySource::LibraryDictionarySource( Block & sample_block_, const Context & context, bool check_config) - : log(&Logger::get("LibraryDictionarySource")) + : log(&Poco::Logger::get("LibraryDictionarySource")) , dict_struct{dict_struct_} , config_prefix{config_prefix_} , path{config.getString(config_prefix + ".path", "")} @@ -157,7 +157,7 @@ LibraryDictionarySource::LibraryDictionarySource( } LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other) - : log(&Logger::get("LibraryDictionarySource")) + : log(&Poco::Logger::get("LibraryDictionarySource")) , dict_struct{other.dict_struct} , config_prefix{other.config_prefix} , path{other.path} diff --git a/src/Dictionaries/LibraryDictionarySourceExternal.cpp b/src/Dictionaries/LibraryDictionarySourceExternal.cpp index 03447df339c..2e944056283 100644 --- a/src/Dictionaries/LibraryDictionarySourceExternal.cpp +++ b/src/Dictionaries/LibraryDictionarySourceExternal.cpp @@ -10,7 +10,7 @@ void ClickHouseLibrary::log(ClickHouseLibrary::LogLevel level, ClickHouseLibrary { using ClickHouseLibrary::LogLevel; - auto & logger = Logger::get(DICT_LOGGER_NAME); + auto & logger = Poco::Logger::get(DICT_LOGGER_NAME); switch (level) { case LogLevel::TRACE: diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 00f592460a7..505ce7b0c12 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -58,7 +58,7 @@ MySQLDictionarySource::MySQLDictionarySource( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const Block & sample_block_) - : log(&Logger::get("MySQLDictionarySource")) + : log(&Poco::Logger::get("MySQLDictionarySource")) , update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , db{config.getString(config_prefix + ".db", "")} @@ -77,7 +77,7 @@ MySQLDictionarySource::MySQLDictionarySource( /// copy-constructor is provided in order to support cloneability MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other) - : log(&Logger::get("MySQLDictionarySource")) + : log(&Poco::Logger::get("MySQLDictionarySource")) , update_time{other.update_time} , dict_struct{other.dict_struct} , db{other.db} diff --git a/src/Dictionaries/TrieDictionary.h b/src/Dictionaries/TrieDictionary.h index 59f946ebe71..5f8b5df89bf 100644 --- a/src/Dictionaries/TrieDictionary.h +++ b/src/Dictionaries/TrieDictionary.h @@ -253,7 +253,7 @@ private: std::exception_ptr creation_exception; - Logger * logger; + Poco::Logger * logger; }; } diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 03d31fa001f..92af20e646b 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -72,7 +72,7 @@ XDBCDictionarySource::XDBCDictionarySource( const Block & sample_block_, const Context & context_, const BridgeHelperPtr bridge_) - : log(&Logger::get(bridge_->getName() + "DictionarySource")) + : log(&Poco::Logger::get(bridge_->getName() + "DictionarySource")) , update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , db{config_.getString(config_prefix_ + ".db", "")} @@ -96,7 +96,7 @@ XDBCDictionarySource::XDBCDictionarySource( /// copy-constructor is provided in order to support cloneability XDBCDictionarySource::XDBCDictionarySource(const XDBCDictionarySource & other) - : log(&Logger::get(other.bridge_helper->getName() + "DictionarySource")) + : log(&Poco::Logger::get(other.bridge_helper->getName() + "DictionarySource")) , update_time{other.update_time} , dict_struct{other.dict_struct} , db{other.db} diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index e47b55d5254..12983b9527a 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -1,3 +1,4 @@ +# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. LIBRARY() PEERDIR( @@ -24,8 +25,8 @@ SRCS( ComplexKeyCacheDictionary_generate3.cpp ComplexKeyCacheDictionary_setAttributeValue.cpp ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp - ComplexKeyHashedDictionary.cpp ComplexKeyDirectDictionary.cpp + ComplexKeyHashedDictionary.cpp DictionaryBlockInputStreamBase.cpp DictionaryFactory.cpp DictionarySourceFactory.cpp diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index cea900d5bad..68f5ee99a7a 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -91,7 +91,7 @@ bool DiskLocal::tryReserve(UInt64 bytes) std::lock_guard lock(DiskLocal::reservation_mutex); if (bytes == 0) { - LOG_DEBUG(&Logger::get("DiskLocal"), "Reserving 0 bytes on disk {}", backQuote(name)); + LOG_DEBUG(&Poco::Logger::get("DiskLocal"), "Reserving 0 bytes on disk {}", backQuote(name)); ++reservation_count; return true; } @@ -100,8 +100,8 @@ bool DiskLocal::tryReserve(UInt64 bytes) UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes); if (unreserved_space >= bytes) { - LOG_DEBUG(&Logger::get("DiskLocal"), "Reserving {} on disk {}, having unreserved {}.", - formatReadableSizeWithBinarySuffix(bytes), backQuote(name), formatReadableSizeWithBinarySuffix(unreserved_space)); + LOG_DEBUG(&Poco::Logger::get("DiskLocal"), "Reserving {} on disk {}, having unreserved {}.", + ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space)); ++reservation_count; reserved_bytes += bytes; return true; @@ -310,7 +310,7 @@ DiskLocalReservation::~DiskLocalReservation() if (disk->reserved_bytes < size) { disk->reserved_bytes = 0; - LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName()); + LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName()); } else { @@ -318,7 +318,7 @@ DiskLocalReservation::~DiskLocalReservation() } if (disk->reservation_count == 0) - LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName()); + LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName()); else --disk->reservation_count; } diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index 69549e4520d..0fb728a4f02 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -99,7 +99,7 @@ DiskSelectorPtr DiskSelector::updateFromConfig( } writeString(" disappeared from configuration, this change will be applied after restart of ClickHouse", warning); - LOG_WARNING(&Logger::get("DiskSelector"), warning.str()); + LOG_WARNING(&Poco::Logger::get("DiskSelector"), warning.str()); } return result; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 4b36deb7e98..71b5991f770 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -515,7 +515,7 @@ std::unique_ptr DiskS3::readFile(const String & path, si { Metadata metadata(metadata_path, path); - LOG_DEBUG(&Logger::get("DiskS3"), "Read from file by path: {}. Existing S3 objects: {}", + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Read from file by path: {}. Existing S3 objects: {}", backQuote(metadata_path + path), metadata.s3_objects.size()); return std::make_unique(client, bucket, metadata, buf_size); @@ -536,7 +536,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, /// Save empty metadata to disk to have ability to get file size while buffer is not finalized. metadata.save(); - LOG_DEBUG(&Logger::get("DiskS3"), "Write to file by path: {} New S3 path: {}", backQuote(metadata_path + path), s3_path); + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Write to file by path: {} New S3 path: {}", backQuote(metadata_path + path), s3_path); return std::make_unique(client, bucket, metadata, s3_path, min_upload_part_size, buf_size); } @@ -544,7 +544,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, { Metadata metadata(metadata_path, path); - LOG_DEBUG(&Logger::get("DiskS3"), "Append to file by path: {}. New S3 path: {}. Existing S3 objects: {}.", + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Append to file by path: {}. New S3 path: {}. Existing S3 objects: {}.", backQuote(metadata_path + path), s3_path, metadata.s3_objects.size()); return std::make_unique(client, bucket, metadata, s3_path, min_upload_part_size, buf_size); @@ -553,7 +553,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, void DiskS3::remove(const String & path) { - LOG_DEBUG(&Logger::get("DiskS3"), "Remove file by path: {}", backQuote(metadata_path + path)); + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Remove file by path: {}", backQuote(metadata_path + path)); Poco::File file(metadata_path + path); if (file.isFile()) @@ -607,7 +607,7 @@ bool DiskS3::tryReserve(UInt64 bytes) std::lock_guard lock(reservation_mutex); if (bytes == 0) { - LOG_DEBUG(&Logger::get("DiskS3"), "Reserving 0 bytes on s3 disk {}", backQuote(name)); + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Reserving 0 bytes on s3 disk {}", backQuote(name)); ++reservation_count; return true; } @@ -616,8 +616,8 @@ bool DiskS3::tryReserve(UInt64 bytes) UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes); if (unreserved_space >= bytes) { - LOG_DEBUG(&Logger::get("DiskS3"), "Reserving {} on disk {}, having unreserved {}.", - formatReadableSizeWithBinarySuffix(bytes), backQuote(name), formatReadableSizeWithBinarySuffix(unreserved_space)); + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Reserving {} on disk {}, having unreserved {}.", + ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space)); ++reservation_count; reserved_bytes += bytes; return true; @@ -672,7 +672,7 @@ DiskS3Reservation::~DiskS3Reservation() if (disk->reserved_bytes < size) { disk->reserved_bytes = 0; - LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName()); + LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName()); } else { @@ -680,7 +680,7 @@ DiskS3Reservation::~DiskS3Reservation() } if (disk->reservation_count == 0) - LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName()); + LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName()); else --disk->reservation_count; } diff --git a/src/Disks/S3/ProxyListConfiguration.cpp b/src/Disks/S3/ProxyListConfiguration.cpp index 430b6412408..318b43a22dc 100644 --- a/src/Disks/S3/ProxyListConfiguration.cpp +++ b/src/Disks/S3/ProxyListConfiguration.cpp @@ -20,7 +20,7 @@ Aws::Client::ClientConfigurationPerRequest ProxyListConfiguration::getConfigurat cfg.proxyHost = proxies[index].getHost(); cfg.proxyPort = proxies[index].getPort(); - LOG_DEBUG(&Logger::get("AWSClient"), "Use proxy: {}", proxies[index].toString()); + LOG_DEBUG(&Poco::Logger::get("AWSClient"), "Use proxy: {}", proxies[index].toString()); return cfg; } diff --git a/src/Disks/S3/ProxyResolverConfiguration.cpp b/src/Disks/S3/ProxyResolverConfiguration.cpp index c0cbe4ac5bf..c11697fda0b 100644 --- a/src/Disks/S3/ProxyResolverConfiguration.cpp +++ b/src/Disks/S3/ProxyResolverConfiguration.cpp @@ -21,7 +21,7 @@ ProxyResolverConfiguration::ProxyResolverConfiguration(const Poco::URI & endpoin Aws::Client::ClientConfigurationPerRequest ProxyResolverConfiguration::getConfiguration(const Aws::Http::HttpRequest &) { - LOG_DEBUG(&Logger::get("AWSClient"), "Obtain proxy using resolver: {}", endpoint.toString()); + LOG_DEBUG(&Poco::Logger::get("AWSClient"), "Obtain proxy using resolver: {}", endpoint.toString()); /// 1 second is enough for now. /// TODO: Make timeouts configurable. @@ -49,7 +49,7 @@ Aws::Client::ClientConfigurationPerRequest ProxyResolverConfiguration::getConfig /// Read proxy host as string from response body. Poco::StreamCopier::copyToString(response_body_stream, proxy_host); - LOG_DEBUG(&Logger::get("AWSClient"), "Use proxy: {}://{}:{}", proxy_scheme, proxy_host, proxy_port); + LOG_DEBUG(&Poco::Logger::get("AWSClient"), "Use proxy: {}://{}:{}", proxy_scheme, proxy_host, proxy_port); cfg.proxyScheme = Aws::Http::SchemeMapper::FromString(proxy_scheme.c_str()); cfg.proxyHost = proxy_host; diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 53292bb30bc..999a81bd413 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -46,7 +46,7 @@ namespace throw Exception("Only HTTP/HTTPS schemas allowed in proxy resolver config: " + proxy_scheme, ErrorCodes::BAD_ARGUMENTS); auto proxy_port = proxy_resolver_config.getUInt(prefix + ".proxy_port"); - LOG_DEBUG(&Logger::get("DiskS3"), "Configured proxy resolver: {}, Scheme: {}, Port: {}", endpoint.toString(), proxy_scheme, proxy_port); + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Configured proxy resolver: {}, Scheme: {}, Port: {}", endpoint.toString(), proxy_scheme, proxy_port); return std::make_shared(endpoint, proxy_scheme, proxy_port); } @@ -70,7 +70,7 @@ namespace proxies.push_back(proxy_uri); - LOG_DEBUG(&Logger::get("DiskS3"), "Configured proxy: {}", proxy_uri.toString()); + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Configured proxy: {}", proxy_uri.toString()); } if (!proxies.empty()) diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index 3f01ed38e1c..f684dce0496 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -245,7 +245,7 @@ StoragePolicySelector::StoragePolicySelector( "StoragePolicy name can contain only alphanumeric and '_' (" + name + ")", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); policies.emplace(name, std::make_shared(name, config, config_prefix + "." + name, disks)); - LOG_INFO(&Logger::get("StoragePolicySelector"), "Storage policy {} loaded", backQuote(name)); + LOG_INFO(&Poco::Logger::get("StoragePolicySelector"), "Storage policy {} loaded", backQuote(name)); } constexpr auto default_storage_policy_name = "default"; diff --git a/src/Disks/VolumeJBOD.cpp b/src/Disks/VolumeJBOD.cpp index b9eb8038cac..7312f3d2365 100644 --- a/src/Disks/VolumeJBOD.cpp +++ b/src/Disks/VolumeJBOD.cpp @@ -20,7 +20,7 @@ VolumeJBOD::VolumeJBOD( DiskSelectorPtr disk_selector ) : IVolume(name_, config, config_prefix, disk_selector) { - Logger * logger = &Logger::get("StorageConfiguration"); + Poco::Logger * logger = &Poco::Logger::get("StorageConfiguration"); auto has_max_bytes = config.has(config_prefix + ".max_data_part_size_bytes"); auto has_max_ratio = config.has(config_prefix + ".max_data_part_size_ratio"); @@ -48,11 +48,11 @@ VolumeJBOD::VolumeJBOD( max_data_part_size = static_cast(sum_size * ratio / disks.size()); for (size_t i = 0; i < disks.size(); ++i) if (sizes[i] < max_data_part_size) - LOG_WARNING(logger, "Disk {} on volume {} have not enough space ({}) for containing part the size of max_data_part_size ({})", backQuote(disks[i]->getName()), backQuote(config_prefix), formatReadableSizeWithBinarySuffix(sizes[i]), formatReadableSizeWithBinarySuffix(max_data_part_size)); + LOG_WARNING(logger, "Disk {} on volume {} have not enough space ({}) for containing part the size of max_data_part_size ({})", backQuote(disks[i]->getName()), backQuote(config_prefix), ReadableSize(sizes[i]), ReadableSize(max_data_part_size)); } static constexpr UInt64 MIN_PART_SIZE = 8u * 1024u * 1024u; if (max_data_part_size != 0 && max_data_part_size < MIN_PART_SIZE) - LOG_WARNING(logger, "Volume {} max_data_part_size is too low ({} < {})", backQuote(name), formatReadableSizeWithBinarySuffix(max_data_part_size), formatReadableSizeWithBinarySuffix(MIN_PART_SIZE)); + LOG_WARNING(logger, "Volume {} max_data_part_size is too low ({} < {})", backQuote(name), ReadableSize(max_data_part_size), ReadableSize(MIN_PART_SIZE)); } DiskPtr VolumeJBOD::getNextDisk() diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index d6be2469759..a4569684e7f 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -43,7 +43,7 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co ++dot; } String table_name = join_name.substr(dot); - auto table = DatabaseCatalog::instance().getTable({database_name, table_name}); + auto table = DatabaseCatalog::instance().getTable({database_name, table_name}, context); auto storage_join = std::dynamic_pointer_cast(table); if (!storage_join) throw Exception{"Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 9d0b764e84b..879b885cf66 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -145,6 +145,8 @@ struct ConvertImpl vec_to[i] = convertFromDecimal(vec_from[i], vec_from.getScale()); else if constexpr (IsDataTypeNumber && IsDataTypeDecimal) vec_to[i] = convertToDecimal(vec_from[i], vec_to.getScale()); + else + throw Exception("Unsupported data type in conversion function", ErrorCodes::CANNOT_CONVERT_TYPE); } else vec_to[i] = static_cast(vec_from[i]); diff --git a/src/Functions/hasColumnInTable.cpp b/src/Functions/hasColumnInTable.cpp index 03e5a4513db..b9ec2b84837 100644 --- a/src/Functions/hasColumnInTable.cpp +++ b/src/Functions/hasColumnInTable.cpp @@ -113,7 +113,7 @@ void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & bool has_column; if (host_name.empty()) { - const StoragePtr & table = DatabaseCatalog::instance().getTable({database_name, table_name}); + const StoragePtr & table = DatabaseCatalog::instance().getTable({database_name, table_name}, global_context); has_column = table->getColumns().hasPhysical(column_name); } else diff --git a/src/Functions/trap.cpp b/src/Functions/trap.cpp index a7f8d81576d..94aa56919b4 100644 --- a/src/Functions/trap.cpp +++ b/src/Functions/trap.cpp @@ -7,11 +7,13 @@ #include #include #include +#include #include #include #include #include +#include namespace DB @@ -22,6 +24,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; + extern const int CANNOT_ALLOCATE_MEMORY; } @@ -132,6 +135,25 @@ public: { (void)context.getCurrentQueryId(); } + else if (mode == "mmap many") + { + std::vector maps; + SCOPE_EXIT( + { + //for (void * map : maps) + // munmap(map, 4096); + }); + + while (true) + { + void * hint = reinterpret_cast( + std::uniform_int_distribution(0x100000000000UL, 0x700000000000UL)(thread_local_rng)); + void * map = mmap(hint, 4096, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); + if (MAP_FAILED == map) + throwFromErrno("Allocator: Cannot mmap", ErrorCodes::CANNOT_ALLOCATE_MEMORY); + maps.push_back(map); + } + } else throw Exception("Unknown trap mode", ErrorCodes::BAD_ARGUMENTS); } diff --git a/src/Functions/visitParamExtractRaw.cpp b/src/Functions/visitParamExtractRaw.cpp index 7a02d29e446..add882f003f 100644 --- a/src/Functions/visitParamExtractRaw.cpp +++ b/src/Functions/visitParamExtractRaw.cpp @@ -22,6 +22,12 @@ struct ExtractRaw expects_end.pop_back(); current_expect_end = expects_end.empty() ? 0 : expects_end.back(); } + else if (current_expect_end == '"') + { + /// skip backslash + if (*pos == '\\' && pos + 1 < end && pos[1] == '"') + ++pos; + } else { switch (*pos) @@ -38,11 +44,6 @@ struct ExtractRaw current_expect_end = '"'; expects_end.push_back(current_expect_end); break; - case '\\': - /// skip backslash - if (pos + 1 < end && pos[1] == '"') - pos++; - break; default: if (!current_expect_end && (*pos == ',' || *pos == '}')) { diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 0b4776853e9..70c42dd5af7 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -1,3 +1,4 @@ +# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. LIBRARY() CFLAGS( @@ -171,7 +172,6 @@ SRCS( FunctionsRound.cpp FunctionsStringArray.cpp FunctionsStringSimilarity.cpp - FunctionUnixTimestamp64.h GatherUtils/concat.cpp GatherUtils/createArraySink.cpp GatherUtils/createArraySource.cpp @@ -285,10 +285,10 @@ SRCS( rand64.cpp randConstant.cpp rand.cpp + randomFixedString.cpp randomPrintableASCII.cpp randomString.cpp randomStringUTF8.cpp - randomFixedString.cpp regexpQuoteMeta.cpp registerFunctionsArithmetic.cpp registerFunctionsComparison.cpp @@ -308,8 +308,8 @@ SRCS( registerFunctionsStringRegexp.cpp registerFunctionsStringSearch.cpp registerFunctionsTuple.cpp - registerFunctionsVisitParam.cpp registerFunctionsUnixTimestamp64.cpp + registerFunctionsVisitParam.cpp reinterpretAsFixedString.cpp reinterpretAsString.cpp reinterpretStringAs.cpp @@ -390,10 +390,10 @@ SRCS( toTime.cpp toTimeZone.cpp toTypeName.cpp - toValidUTF8.cpp toUnixTimestamp64Micro.cpp toUnixTimestamp64Milli.cpp toUnixTimestamp64Nano.cpp + toValidUTF8.cpp toYear.cpp toYYYYMM.cpp toYYYYMMDD.cpp @@ -424,8 +424,8 @@ SRCS( URL/fragment.cpp URL/path.cpp URL/pathFull.cpp - URL/protocol.cpp URL/port.cpp + URL/protocol.cpp URL/queryStringAndFragment.cpp URL/queryString.cpp URL/registerFunctionsURL.cpp diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 14c97ee56f4..6b7f30cd9b6 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -68,7 +68,7 @@ namespace throw Exception("Unsupported scheme in URI '" + uri.toString() + "'", ErrorCodes::UNSUPPORTED_URI_SCHEME); } - HTTPSessionPtr makeHTTPSessionImpl(const std::string & host, UInt16 port, bool https, bool keep_alive) + HTTPSessionPtr makeHTTPSessionImpl(const std::string & host, UInt16 port, bool https, bool keep_alive, bool resolve_host=true) { HTTPSessionPtr session; @@ -83,7 +83,10 @@ namespace ProfileEvents::increment(ProfileEvents::CreatedHTTPConnections); - session->setHost(DNSResolver::instance().resolveHost(host).toString()); + if (resolve_host) + session->setHost(DNSResolver::instance().resolveHost(host).toString()); + else + session->setHost(host); session->setPort(port); /// doesn't work properly without patch @@ -173,7 +176,7 @@ namespace auto msg = Poco::AnyCast(session_data); if (!msg.empty()) { - LOG_TRACE((&Logger::get("HTTPCommon")), "Failed communicating with {} with error '{}' will try to reconnect session", host, msg); + LOG_TRACE((&Poco::Logger::get("HTTPCommon")), "Failed communicating with {} with error '{}' will try to reconnect session", host, msg); /// Host can change IP const auto ip = DNSResolver::instance().resolveHost(host).toString(); if (ip != session->getHost()) @@ -202,13 +205,13 @@ void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response, unsigne response.set("Keep-Alive", "timeout=" + std::to_string(timeout.totalSeconds())); } -HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts) +HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, bool resolve_host) { const std::string & host = uri.getHost(); UInt16 port = uri.getPort(); bool https = isHTTPS(uri); - auto session = makeHTTPSessionImpl(host, port, https, false); + auto session = makeHTTPSessionImpl(host, port, https, false, resolve_host); setTimeouts(*session, timeouts); return session; } diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index 7592c1c31b3..66764b1c805 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -45,7 +45,7 @@ using HTTPSessionPtr = std::shared_ptr; void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response, unsigned keep_alive_timeout); /// Create session object to perform requests and set required parameters. -HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts); +HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, bool resolve_host = true); /// As previous method creates session, but tooks it from pool PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size); diff --git a/src/IO/MMapReadBufferFromFileDescriptor.cpp b/src/IO/MMapReadBufferFromFileDescriptor.cpp index 027b95bc022..bbeec8959b4 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -34,7 +34,7 @@ void MMapReadBufferFromFileDescriptor::init(int fd_, size_t offset, size_t lengt { void * buf = mmap(nullptr, length, PROT_READ, MAP_PRIVATE, fd, offset); if (MAP_FAILED == buf) - throwFromErrno("MMapReadBufferFromFileDescriptor: Cannot mmap " + formatReadableSizeWithBinarySuffix(length) + ".", + throwFromErrno(fmt::format("MMapReadBufferFromFileDescriptor: Cannot mmap {}.", ReadableSize(length)), ErrorCodes::CANNOT_ALLOCATE_MEMORY); BufferBase::set(static_cast(buf), length, 0); @@ -84,7 +84,7 @@ MMapReadBufferFromFileDescriptor::~MMapReadBufferFromFileDescriptor() void MMapReadBufferFromFileDescriptor::finish() { if (0 != munmap(internalBuffer().begin(), length)) - throwFromErrno("MMapReadBufferFromFileDescriptor: Cannot munmap " + formatReadableSizeWithBinarySuffix(length) + ".", + throwFromErrno(fmt::format("MMapReadBufferFromFileDescriptor: Cannot munmap {}.", ReadableSize(length)), ErrorCodes::CANNOT_MUNMAP); length = 0; diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 15d3b4f0beb..829b73d0af6 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -33,7 +33,7 @@ private: Aws::S3::Model::GetObjectResult read_result; std::unique_ptr impl; - Logger * log = &Logger::get("ReadBufferFromS3"); + Poco::Logger * log = &Poco::Logger::get("ReadBufferFromS3"); public: explicit ReadBufferFromS3( diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index dccb413af2c..9d1d27611b5 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -283,7 +283,9 @@ static void parseComplexEscapeSequence(Vector & s, ReadBuffer & buf) if (buf.eof()) throw Exception("Cannot parse escape sequence", ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE); - if (*buf.position() == 'x') + char char_after_backslash = *buf.position(); + + if (char_after_backslash == 'x') { ++buf.position(); /// escape sequence of the form \xAA @@ -291,7 +293,7 @@ static void parseComplexEscapeSequence(Vector & s, ReadBuffer & buf) readPODBinary(hex_code, buf); s.push_back(unhex2(hex_code)); } - else if (*buf.position() == 'N') + else if (char_after_backslash == 'N') { /// Support for NULLs: \N sequence must be parsed as empty string. ++buf.position(); @@ -299,7 +301,22 @@ static void parseComplexEscapeSequence(Vector & s, ReadBuffer & buf) else { /// The usual escape sequence of a single character. - s.push_back(parseEscapeSequence(*buf.position())); + char decoded_char = parseEscapeSequence(char_after_backslash); + + /// For convenience using LIKE and regular expressions, + /// we leave backslash when user write something like 'Hello 100\%': + /// it is parsed like Hello 100\% instead of Hello 100% + if (decoded_char != '\\' + && decoded_char != '\'' + && decoded_char != '"' + && decoded_char != '`' /// MySQL style identifiers + && decoded_char != '/' /// JavaScript in HTML + && !isControlASCII(decoded_char)) + { + s.push_back('\\'); + } + + s.push_back(decoded_char); ++buf.position(); } } @@ -633,7 +650,6 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & ++next_pos; }(); - appendToStringOrVector(s, buf, next_pos); buf.position() = next_pos; diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 962c663a687..edd0b7f1579 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -127,7 +127,7 @@ namespace detail if (!credentials.getUsername().empty()) credentials.authenticate(request); - LOG_TRACE((&Logger::get("ReadWriteBufferFromHTTP")), "Sending request to {}", uri.toString()); + LOG_TRACE((&Poco::Logger::get("ReadWriteBufferFromHTTP")), "Sending request to {}", uri.toString()); auto sess = session->getSession(); diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp new file mode 100644 index 00000000000..b8de483a5a8 --- /dev/null +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -0,0 +1,165 @@ +#include "PocoHTTPClient.h" + +#include +#include +#include +#include +#include +#include +#include +#include "Poco/StreamCopier.h" +#include +#include +#include + +namespace DB::S3 +{ +PocoHTTPClient::PocoHTTPClient(const Aws::Client::ClientConfiguration & clientConfiguration) + : per_request_configuration(clientConfiguration.perRequestConfiguration) + , timeouts(ConnectionTimeouts( + Poco::Timespan(clientConfiguration.connectTimeoutMs * 1000), /// connection timeout. + Poco::Timespan(clientConfiguration.httpRequestTimeoutMs * 1000), /// send timeout. + Poco::Timespan(clientConfiguration.httpRequestTimeoutMs * 1000) /// receive timeout. + )) +{ +} + +std::shared_ptr PocoHTTPClient::MakeRequest( + Aws::Http::HttpRequest & request, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const +{ + auto response = Aws::MakeShared("PocoHTTPClient", request); + MakeRequestInternal(request, response, readLimiter, writeLimiter); + return response; +} + +std::shared_ptr PocoHTTPClient::MakeRequest( + const std::shared_ptr & request, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const +{ + auto response = Aws::MakeShared("PocoHTTPClient", request); + MakeRequestInternal(*request, response, readLimiter, writeLimiter); + return response; +} + +void PocoHTTPClient::MakeRequestInternal( + Aws::Http::HttpRequest & request, + std::shared_ptr & response, + Aws::Utils::RateLimits::RateLimiterInterface *, + Aws::Utils::RateLimits::RateLimiterInterface *) const +{ + Poco::Logger * log = &Poco::Logger::get("AWSClient"); + + auto uri = request.GetUri().GetURIString(); + LOG_DEBUG(log, "Make request to: {}", uri); + + const int MAX_REDIRECT_ATTEMPTS = 10; + try + { + for (int attempt = 0; attempt < MAX_REDIRECT_ATTEMPTS; ++attempt) + { + Poco::URI poco_uri(uri); + + /// Reverse proxy can replace host header with resolved ip address instead of host name. + /// This can lead to request signature difference on S3 side. + auto session = makeHTTPSession(poco_uri, timeouts, false); + + auto request_configuration = per_request_configuration(request); + if (!request_configuration.proxyHost.empty()) + session->setProxy(request_configuration.proxyHost, request_configuration.proxyPort); + + Poco::Net::HTTPRequest poco_request(Poco::Net::HTTPRequest::HTTP_1_1); + + poco_request.setURI(poco_uri.getPathAndQuery()); + + switch (request.GetMethod()) + { + case Aws::Http::HttpMethod::HTTP_GET: + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_GET); + break; + case Aws::Http::HttpMethod::HTTP_POST: + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_POST); + break; + case Aws::Http::HttpMethod::HTTP_DELETE: + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_DELETE); + break; + case Aws::Http::HttpMethod::HTTP_PUT: + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_PUT); + break; + case Aws::Http::HttpMethod::HTTP_HEAD: + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_HEAD); + break; + case Aws::Http::HttpMethod::HTTP_PATCH: + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_PATCH); + break; + } + + for (const auto & [header_name, header_value] : request.GetHeaders()) + poco_request.set(header_name, header_value); + + Poco::Net::HTTPResponse poco_response; + auto & request_body_stream = session->sendRequest(poco_request); + + if (request.GetContentBody()) + { + LOG_DEBUG(log, "Writing request body."); + if (attempt > 0) /// rewind content body buffer. + { + request.GetContentBody()->clear(); + request.GetContentBody()->seekg(0); + } + auto size = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream); + LOG_DEBUG(log, "Written {} bytes to request body", size); + } + + LOG_DEBUG(log, "Receiving response..."); + auto & response_body_stream = session->receiveResponse(poco_response); + + int status_code = static_cast(poco_response.getStatus()); + LOG_DEBUG(log, "Response status: {}, {}", status_code, poco_response.getReason()); + + if (poco_response.getStatus() == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT) + { + auto location = poco_response.get("location"); + uri = location; + LOG_DEBUG(log, "Redirecting request to new location: {}", location); + + continue; + } + + response->SetResponseCode(static_cast(status_code)); + response->SetContentType(poco_response.getContentType()); + + std::stringstream headers_ss; + for (const auto & [header_name, header_value] : poco_response) + { + response->AddHeader(header_name, header_value); + headers_ss << header_name << ": " << header_value << "; "; + } + LOG_DEBUG(log, "Received headers: {}", headers_ss.str()); + + if (status_code >= 300) + { + String error_message; + Poco::StreamCopier::copyToString(response_body_stream, error_message); + + response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); + response->SetClientErrorMessage(error_message); + } + else + /// TODO: Do not copy whole stream. + Poco::StreamCopier::copyStream(response_body_stream, response->GetResponseBody()); + + break; + } + } + catch (...) + { + tryLogCurrentException(log, fmt::format("Failed to make request to: {}", uri)); + response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); + response->SetClientErrorMessage(getCurrentExceptionMessage(false)); + } +} +} diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h new file mode 100644 index 00000000000..203ad94e86b --- /dev/null +++ b/src/IO/S3/PocoHTTPClient.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include + +namespace Aws::Http::Standard +{ +class StandardHttpResponse; +} + +namespace DB::S3 +{ + +class PocoHTTPClient : public Aws::Http::HttpClient +{ +public: + explicit PocoHTTPClient(const Aws::Client::ClientConfiguration & clientConfiguration); + ~PocoHTTPClient() override = default; + std::shared_ptr MakeRequest( + Aws::Http::HttpRequest & request, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const override; + + std::shared_ptr MakeRequest( + const std::shared_ptr & request, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const override; + +private: + void MakeRequestInternal( + Aws::Http::HttpRequest & request, + std::shared_ptr & response, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; + + std::function per_request_configuration; + ConnectionTimeouts timeouts; +}; + +} diff --git a/src/IO/S3/PocoHTTPClientFactory.cpp b/src/IO/S3/PocoHTTPClientFactory.cpp new file mode 100644 index 00000000000..033ad4af37c --- /dev/null +++ b/src/IO/S3/PocoHTTPClientFactory.cpp @@ -0,0 +1,32 @@ +#include "PocoHTTPClientFactory.h" + +#include +#include +#include +#include +#include + +namespace DB::S3 +{ +std::shared_ptr +PocoHTTPClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration & clientConfiguration) const +{ + return std::make_shared(clientConfiguration); +} + +std::shared_ptr PocoHTTPClientFactory::CreateHttpRequest( + const Aws::String & uri, Aws::Http::HttpMethod method, const Aws::IOStreamFactory & streamFactory) const +{ + return CreateHttpRequest(Aws::Http::URI(uri), method, streamFactory); +} + +std::shared_ptr PocoHTTPClientFactory::CreateHttpRequest( + const Aws::Http::URI & uri, Aws::Http::HttpMethod method, const Aws::IOStreamFactory & streamFactory) const +{ + auto request = Aws::MakeShared("PocoHTTPClientFactory", uri, method); + request->SetResponseStreamFactory(streamFactory); + + return request; +} + +} diff --git a/src/IO/S3/PocoHTTPClientFactory.h b/src/IO/S3/PocoHTTPClientFactory.h new file mode 100644 index 00000000000..ac586289113 --- /dev/null +++ b/src/IO/S3/PocoHTTPClientFactory.h @@ -0,0 +1,25 @@ +#pragma once + +#include + +namespace Aws::Http +{ + class HttpClient; + class HttpRequest; +} + +namespace DB::S3 +{ + +class PocoHTTPClientFactory : public Aws::Http::HttpClientFactory +{ +public: + ~PocoHTTPClientFactory() override = default; + [[nodiscard]] std::shared_ptr CreateHttpClient(const Aws::Client::ClientConfiguration & clientConfiguration) const override; + [[nodiscard]] std::shared_ptr + CreateHttpRequest(const Aws::String & uri, Aws::Http::HttpMethod method, const Aws::IOStreamFactory & streamFactory) const override; + [[nodiscard]] std::shared_ptr + CreateHttpRequest(const Aws::Http::URI & uri, Aws::Http::HttpMethod method, const Aws::IOStreamFactory & streamFactory) const override; +}; + +} diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 3c2c5835078..1539b3c7025 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -9,6 +9,11 @@ # include # include # include +# include +# include +# include +# include +# include # include # include # include @@ -16,16 +21,17 @@ namespace { -const std::pair & convertLogLevel(Aws::Utils::Logging::LogLevel log_level) +const std::pair & convertLogLevel(Aws::Utils::Logging::LogLevel log_level) { - static const std::unordered_map> mapping = { - {Aws::Utils::Logging::LogLevel::Off, {LogsLevel::none, Message::PRIO_FATAL}}, - {Aws::Utils::Logging::LogLevel::Fatal, {LogsLevel::error, Message::PRIO_FATAL}}, - {Aws::Utils::Logging::LogLevel::Error, {LogsLevel::error, Message::PRIO_ERROR}}, - {Aws::Utils::Logging::LogLevel::Warn, {LogsLevel::warning, Message::PRIO_WARNING}}, - {Aws::Utils::Logging::LogLevel::Info, {LogsLevel::information, Message::PRIO_INFORMATION}}, - {Aws::Utils::Logging::LogLevel::Debug, {LogsLevel::debug, Message::PRIO_DEBUG}}, - {Aws::Utils::Logging::LogLevel::Trace, {LogsLevel::trace, Message::PRIO_TRACE}}, + static const std::unordered_map> mapping = + { + {Aws::Utils::Logging::LogLevel::Off, {DB::LogsLevel::none, Poco::Message::PRIO_FATAL}}, + {Aws::Utils::Logging::LogLevel::Fatal, {DB::LogsLevel::error, Poco::Message::PRIO_FATAL}}, + {Aws::Utils::Logging::LogLevel::Error, {DB::LogsLevel::error, Poco::Message::PRIO_ERROR}}, + {Aws::Utils::Logging::LogLevel::Warn, {DB::LogsLevel::warning, Poco::Message::PRIO_WARNING}}, + {Aws::Utils::Logging::LogLevel::Info, {DB::LogsLevel::information, Poco::Message::PRIO_INFORMATION}}, + {Aws::Utils::Logging::LogLevel::Debug, {DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG}}, + {Aws::Utils::Logging::LogLevel::Trace, {DB::LogsLevel::trace, Poco::Message::PRIO_TRACE}}, }; return mapping.at(log_level); } @@ -70,6 +76,7 @@ namespace S3 aws_options = Aws::SDKOptions{}; Aws::InitAPI(aws_options); Aws::Utils::Logging::InitializeAWSLogging(std::make_shared()); + Aws::Http::SetHttpClientFactory(std::make_shared()); } ClientFactory::~ClientFactory() diff --git a/src/IO/WriteBufferFromHTTP.cpp b/src/IO/WriteBufferFromHTTP.cpp index 573ac82a1d3..70cbd7484b4 100644 --- a/src/IO/WriteBufferFromHTTP.cpp +++ b/src/IO/WriteBufferFromHTTP.cpp @@ -15,7 +15,7 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( request.setHost(uri.getHost()); request.setChunkedTransferEncoding(true); - LOG_TRACE((&Logger::get("WriteBufferToHTTP")), "Sending request to {}", uri.toString()); + LOG_TRACE((&Poco::Logger::get("WriteBufferToHTTP")), "Sending request to {}", uri.toString()); ostr = &session->sendRequest(request); } diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index f928941e482..71c176749af 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -36,7 +36,7 @@ private: String upload_id; std::vector part_tags; - Logger * log = &Logger::get("WriteBufferFromS3"); + Poco::Logger * log = &Poco::Logger::get("WriteBufferFromS3"); public: explicit WriteBufferFromS3( diff --git a/src/IO/tests/CMakeLists.txt b/src/IO/tests/CMakeLists.txt index 2767ce6e271..dfbbfa77853 100644 --- a/src/IO/tests/CMakeLists.txt +++ b/src/IO/tests/CMakeLists.txt @@ -37,9 +37,6 @@ target_link_libraries (parse_int_perf2 PRIVATE clickhouse_common_io) add_executable (read_write_int read_write_int.cpp) target_link_libraries (read_write_int PRIVATE clickhouse_common_io) -add_executable (mempbrk mempbrk.cpp) -target_link_libraries (mempbrk PRIVATE clickhouse_common_io) - add_executable (o_direct_and_dirty_pages o_direct_and_dirty_pages.cpp) target_link_libraries (o_direct_and_dirty_pages PRIVATE clickhouse_common_io) diff --git a/src/IO/tests/mempbrk.cpp b/src/IO/tests/mempbrk.cpp deleted file mode 100644 index 55097d989af..00000000000 --- a/src/IO/tests/mempbrk.cpp +++ /dev/null @@ -1,90 +0,0 @@ -#include -#include -#include - -#include - -#include -#include -#include -#include -#include - -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; -} -} - - -namespace test -{ -static void readEscapedString(DB::String & s, DB::ReadBuffer & buf) - { - s = ""; - while (!buf.eof()) - { - const char * next_pos = find_first_symbols<'\b', '\f', '\n', '\r', '\t', '\0', '\\'>(buf.position(), buf.buffer().end()); - - s.append(buf.position(), next_pos - buf.position()); - buf.position() += next_pos - buf.position(); - - if (!buf.hasPendingData()) - continue; - - if (*buf.position() == '\t' || *buf.position() == '\n') - return; - - if (*buf.position() == '\\') - { - ++buf.position(); - if (buf.eof()) - throw DB::Exception("Cannot parse escape sequence", DB::ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE); - s += DB::parseEscapeSequence(*buf.position()); - ++buf.position(); - } - } - } -} - - -int main(int, char **) -{ - try - { - DB::ReadBufferFromFileDescriptor in(STDIN_FILENO); -// DB::WriteBufferFromFileDescriptor out(STDOUT_FILENO); - std::string s; - size_t rows = 0; - - Stopwatch watch; - - while (!in.eof()) - { - test::readEscapedString(s, in); - in.ignore(); - - ++rows; - -/* DB::writeEscapedString(s, out); - DB::writeChar('\n', out);*/ - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "Read " << rows << " rows (" << in.count() / 1000000.0 << " MB) in " << watch.elapsedSeconds() << " sec., " - << rows / watch.elapsedSeconds() << " rows/sec. (" << in.count() / watch.elapsedSeconds() / 1000000 << " MB/s.)" - << std::endl; - } - catch (const DB::Exception & e) - { - std::cerr << e.what() << ", " << e.displayText() << std::endl; - return 1; - } - - return 0; -} diff --git a/src/IO/tests/read_float_perf.cpp b/src/IO/tests/read_float_perf.cpp index 8a870d4a960..aa0fa8ad15d 100644 --- a/src/IO/tests/read_float_perf.cpp +++ b/src/IO/tests/read_float_perf.cpp @@ -56,7 +56,8 @@ void NO_INLINE loop(ReadBuffer & in, WriteBuffer & out) } watch.stop(); - out << "Read in " << watch.elapsedSeconds() << " sec, " << formatReadableSizeWithBinarySuffix(in.count() / watch.elapsedSeconds()) << "/sec, result = " << sum << "\n"; + out << "Read in " << watch.elapsedSeconds() << " sec, " + << formatReadableSizeWithBinarySuffix(in.count() / watch.elapsedSeconds()) << "/sec, result = " << sum << "\n"; } diff --git a/src/Interpreters/ActionLocksManager.cpp b/src/Interpreters/ActionLocksManager.cpp index fe5b3def658..e8887f44a45 100644 --- a/src/Interpreters/ActionLocksManager.cpp +++ b/src/Interpreters/ActionLocksManager.cpp @@ -19,23 +19,28 @@ namespace ActionLocks } +ActionLocksManager::ActionLocksManager(const Context & context) + : global_context(context.getGlobalContext()) +{ +} + template -inline void forEachTable(F && f) +inline void forEachTable(F && f, const Context & context) { for (auto & elem : DatabaseCatalog::instance().getDatabases()) - for (auto iterator = elem.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next()) f(iterator->table()); } -void ActionLocksManager::add(StorageActionBlockType action_type) +void ActionLocksManager::add(StorageActionBlockType action_type, const Context & context) { - forEachTable([&](const StoragePtr & table) { add(table, action_type); }); + forEachTable([&](const StoragePtr & table) { add(table, action_type); }, context); } void ActionLocksManager::add(const StorageID & table_id, StorageActionBlockType action_type) { - if (auto table = DatabaseCatalog::instance().tryGetTable(table_id)) + if (auto table = DatabaseCatalog::instance().tryGetTable(table_id, global_context)) add(table, action_type); } @@ -60,7 +65,7 @@ void ActionLocksManager::remove(StorageActionBlockType action_type) void ActionLocksManager::remove(const StorageID & table_id, StorageActionBlockType action_type) { - if (auto table = DatabaseCatalog::instance().tryGetTable(table_id)) + if (auto table = DatabaseCatalog::instance().tryGetTable(table_id, global_context)) remove(table, action_type); } diff --git a/src/Interpreters/ActionLocksManager.h b/src/Interpreters/ActionLocksManager.h index 039a95ce218..ea5d96ad8ce 100644 --- a/src/Interpreters/ActionLocksManager.h +++ b/src/Interpreters/ActionLocksManager.h @@ -19,8 +19,10 @@ class Context; class ActionLocksManager { public: + ActionLocksManager(const Context & context); + /// Adds new locks for each table - void add(StorageActionBlockType action_type); + void add(StorageActionBlockType action_type, const Context & context); /// Add new lock for a table if it has not been already added void add(const StorageID & table_id, StorageActionBlockType action_type); void add(const StoragePtr & table, StorageActionBlockType action_type); @@ -41,6 +43,7 @@ private: mutable std::mutex mutex; StorageLocks storage_locks; + const Context & global_context; }; } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 7c2133e629f..b5d9f30573e 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -670,7 +670,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su if (identifier) { auto table_id = data.context.resolveStorageID(right_in_operand); - StoragePtr table = DatabaseCatalog::instance().tryGetTable(table_id); + StoragePtr table = DatabaseCatalog::instance().tryGetTable(table_id, data.context); if (table) { @@ -706,7 +706,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su { auto interpreter = interpretSubquery(right_in_operand, data.context, data.subquery_depth, {}); subquery_for_set.source = std::make_shared( - interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().in; }); + interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().getInputStream(); }); /** Why is LazyBlockInputStream used? * diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 98cb1018178..96a9b1fc1df 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -128,11 +128,11 @@ Block Aggregator::getHeader(bool final) const if (final) { - for (size_t i = 0; i < params.aggregates_size; ++i) + for (const auto & aggregate : params.aggregates) { - auto & elem = res.getByPosition(params.keys_size + i); + auto & elem = res.getByName(aggregate.column_name); - elem.type = params.aggregates[i].function->getReturnType(); + elem.type = aggregate.function->getReturnType(); elem.column = elem.type->createColumn(); } } @@ -768,14 +768,14 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, co " ({} rows/sec., {}/sec. uncompressed, {}/sec. compressed)", elapsed_seconds, rows, - formatReadableSizeWithBinarySuffix(uncompressed_bytes), - formatReadableSizeWithBinarySuffix(compressed_bytes), + ReadableSize(uncompressed_bytes), + ReadableSize(compressed_bytes), uncompressed_bytes / rows, compressed_bytes / rows, uncompressed_bytes / compressed_bytes, rows / elapsed_seconds, - formatReadableSizeWithBinarySuffix(uncompressed_bytes / elapsed_seconds), - formatReadableSizeWithBinarySuffix(compressed_bytes / elapsed_seconds)); + ReadableSize(uncompressed_bytes / elapsed_seconds), + ReadableSize(compressed_bytes / elapsed_seconds)); } void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants) { @@ -871,7 +871,7 @@ void Aggregator::writeToTemporaryFileImpl( /// `data_variants` will not destroy them in the destructor, they are now owned by ColumnAggregateFunction objects. data_variants.aggregator = nullptr; - LOG_TRACE(log, "Max size of temporary block: {} rows, {}.", max_temporary_block_size_rows, formatReadableSizeWithBinarySuffix(max_temporary_block_size_bytes)); + LOG_TRACE(log, "Max size of temporary block: {} rows, {}.", max_temporary_block_size_rows, ReadableSize(max_temporary_block_size_bytes)); } @@ -943,9 +943,9 @@ void Aggregator::execute(const BlockInputStreamPtr & stream, AggregatedDataVaria size_t rows = result.sizeWithoutOverflowRow(); LOG_TRACE(log, "Aggregated. {} to {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)", - src_rows, rows, formatReadableSizeWithBinarySuffix(src_bytes), + src_rows, rows, ReadableSize(src_bytes), elapsed_seconds, src_rows / elapsed_seconds, - formatReadableSizeWithBinarySuffix(src_bytes / elapsed_seconds)); + ReadableSize(src_bytes / elapsed_seconds)); } @@ -1060,7 +1060,8 @@ Block Aggregator::prepareBlockAndFill( { if (!final) { - aggregate_columns[i] = header.safeGetByPosition(i + params.keys_size).type->createColumn(); + const auto & aggregate_column_name = params.aggregates[i].column_name; + aggregate_columns[i] = header.getByName(aggregate_column_name).type->createColumn(); /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); @@ -1096,10 +1097,11 @@ Block Aggregator::prepareBlockAndFill( for (size_t i = 0; i < params.aggregates_size; ++i) { + const auto & aggregate_column_name = params.aggregates[i].column_name; if (final) - res.getByPosition(i + params.keys_size).column = std::move(final_aggregate_columns[i]); + res.getByName(aggregate_column_name).column = std::move(final_aggregate_columns[i]); else - res.getByPosition(i + params.keys_size).column = std::move(aggregate_columns[i]); + res.getByName(aggregate_column_name).column = std::move(aggregate_columns[i]); } /// Change the size of the columns-constants in the block. @@ -1313,9 +1315,9 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b double elapsed_seconds = watch.elapsedSeconds(); LOG_TRACE(log, "Converted aggregated data to blocks. {} rows, {} in {} sec. ({} rows/sec., {}/sec.)", - rows, formatReadableSizeWithBinarySuffix(bytes), + rows, ReadableSize(bytes), elapsed_seconds, rows / elapsed_seconds, - formatReadableSizeWithBinarySuffix(bytes / elapsed_seconds)); + ReadableSize(bytes / elapsed_seconds)); return blocks; } @@ -1565,7 +1567,7 @@ public: ~MergingAndConvertingBlockInputStream() override { - LOG_TRACE(&Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish"); + LOG_TRACE(&Poco::Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish"); /// We need to wait for threads to finish before destructor of 'parallel_merge_data', /// because the threads access 'parallel_merge_data'. @@ -1824,7 +1826,10 @@ void NO_INLINE Aggregator::mergeStreamsImplCase( key_columns[i] = block.safeGetByPosition(i).column.get(); for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns[i] = &typeid_cast(*block.safeGetByPosition(params.keys_size + i).column).getData(); + { + const auto & aggregate_column_name = params.aggregates[i].column_name; + aggregate_columns[i] = &typeid_cast(*block.getByName(aggregate_column_name).column).getData(); + } typename Method::State state(key_columns, key_sizes, aggregation_state_cache); @@ -1900,7 +1905,10 @@ void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl( /// Remember the columns we will work with for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns[i] = &typeid_cast(*block.safeGetByPosition(params.keys_size + i).column).getData(); + { + const auto & aggregate_column_name = params.aggregates[i].column_name; + aggregate_columns[i] = &typeid_cast(*block.getByName(aggregate_column_name).column).getData(); + } AggregatedDataWithoutKey & res = result.without_key; if (!res) @@ -2178,9 +2186,9 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final) size_t bytes = block.bytes(); double elapsed_seconds = watch.elapsedSeconds(); LOG_TRACE(log, "Merged partially aggregated blocks. {} rows, {}. in {} sec. ({} rows/sec., {}/sec.)", - rows, formatReadableSizeWithBinarySuffix(bytes), + rows, ReadableSize(bytes), elapsed_seconds, rows / elapsed_seconds, - formatReadableSizeWithBinarySuffix(bytes / elapsed_seconds)); + ReadableSize(bytes / elapsed_seconds)); if (isCancelled()) return {}; diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 1fa0ff282d6..abc1356787e 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1048,7 +1048,7 @@ protected: std::mutex mutex; - Logger * log = &Logger::get("Aggregator"); + Poco::Logger * log = &Poco::Logger::get("Aggregator"); /// Returns true if you can abort the current task. CancellationHook isCancelled; diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 59ff01bf972..a4e03d35d42 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -181,7 +181,7 @@ void AsynchronousMetrics::update() /// Lazy database can not contain MergeTree tables if (db.second->getEngineName() == "Lazy") continue; - for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { ++total_number_of_tables; const auto & table = iterator->table(); diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 53c61bea60e..5d41b0e87ce 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -15,6 +15,7 @@ #include #include #include +#include namespace ProfileEvents { @@ -70,35 +71,14 @@ SelectStreamFactory::SelectStreamFactory( namespace { -Pipe createLocalStream( - const ASTPtr & query_ast, const Block & header, const Context & context, QueryProcessingStage::Enum processed_stage, - bool add_totals_port, bool add_extremes_port, bool force_tree_shaped_pipeline) +QueryPipeline createLocalStream( + const ASTPtr & query_ast, const Block & header, const Context & context, QueryProcessingStage::Enum processed_stage) { checkStackSize(); InterpreterSelectQuery interpreter{query_ast, context, SelectQueryOptions(processed_stage)}; - if (force_tree_shaped_pipeline) - { - /// This flag means that pipeline must be tree-shaped, - /// so we can't enable processors for InterpreterSelectQuery here. - auto stream = interpreter.execute().in; - auto source = std::make_shared(std::move(stream)); - - if (add_totals_port) - source->addTotalsPort(); - if (add_extremes_port) - source->addExtremesPort(); - - Pipe pipe(std::move(source)); - - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), header, ConvertingTransform::MatchColumnsMode::Name)); - - return pipe; - } - - auto pipeline = interpreter.executeWithProcessors(); + auto pipeline = interpreter.execute().pipeline; pipeline.addSimpleTransform([&](const Block & source_header) { @@ -116,7 +96,8 @@ Pipe createLocalStream( */ /// return std::make_shared(stream); - return std::move(pipeline).getPipe(); + pipeline.setMaxThreads(1); + return pipeline; } String formattedAST(const ASTPtr & ast) @@ -134,7 +115,7 @@ void SelectStreamFactory::createForShard( const Cluster::ShardInfo & shard_info, const String &, const ASTPtr & query_ast, const Context & context, const ThrottlerPtr & throttler, - const SelectQueryInfo & query_info, + const SelectQueryInfo &, Pipes & res) { bool force_add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; @@ -152,8 +133,7 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { - res.emplace_back(createLocalStream(modified_query_ast, header, context, processed_stage, - add_totals_port, add_extremes_port, query_info.force_tree_shaped_pipeline)); + res.emplace_back(createLocalStream(modified_query_ast, header, context, processed_stage).getPipe()); }; String modified_query = formattedAST(modified_query_ast); @@ -191,7 +171,7 @@ void SelectStreamFactory::createForShard( else { auto resolved_id = context.resolveStorageID(main_table); - main_table_storage = DatabaseCatalog::instance().tryGetTable(resolved_id); + main_table_storage = DatabaseCatalog::instance().tryGetTable(resolved_id, context); } @@ -200,7 +180,7 @@ void SelectStreamFactory::createForShard( ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); if (shard_info.hasRemoteConnections()) { - LOG_WARNING(&Logger::get("ClusterProxy::SelectStreamFactory"), "There is no table {} on local replica of shard {}, will try remote replicas.", main_table.getNameForLogs(), shard_info.shard_num); + LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), "There is no table {} on local replica of shard {}, will try remote replicas.", main_table.getNameForLogs(), shard_info.shard_num); emplace_remote_stream(); } else @@ -236,7 +216,7 @@ void SelectStreamFactory::createForShard( /// If we reached this point, local replica is stale. ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); - LOG_WARNING(&Logger::get("ClusterProxy::SelectStreamFactory"), "Local replica of shard {} is stale (delay: {}s.)", shard_info.shard_num, local_delay); + LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), "Local replica of shard {} is stale (delay: {}s.)", shard_info.shard_num, local_delay); if (!settings.fallback_to_stale_replicas_for_distributed_queries) { @@ -266,7 +246,7 @@ void SelectStreamFactory::createForShard( auto lazily_create_stream = [ pool = shard_info.pool, shard_num = shard_info.shard_num, modified_query, header = header, modified_query_ast, context, throttler, main_table = main_table, table_func_ptr = table_func_ptr, scalars = scalars, external_tables = external_tables, - stage = processed_stage, local_delay, add_totals_port, add_extremes_port]() + stage = processed_stage, local_delay]() -> BlockInputStreamPtr { auto current_settings = context.getSettingsRef(); @@ -284,7 +264,7 @@ void SelectStreamFactory::createForShard( catch (const Exception & ex) { if (ex.code() == ErrorCodes::ALL_CONNECTION_TRIES_FAILED) - LOG_WARNING(&Logger::get("ClusterProxy::SelectStreamFactory"), "Connections to remote replicas of local shard {} failed, will use stale local replica", shard_num); + LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), "Connections to remote replicas of local shard {} failed, will use stale local replica", shard_num); else throw; } @@ -297,8 +277,8 @@ void SelectStreamFactory::createForShard( } if (try_results.empty() || local_delay < max_remote_delay) - return std::make_shared( - createLocalStream(modified_query_ast, header, context, stage, add_totals_port, add_extremes_port, true)); + return std::make_shared( + createLocalStream(modified_query_ast, header, context, stage)); else { std::vector connections; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4605c606594..5e2f4ecadab 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -287,7 +287,7 @@ void NamedSession::release() */ struct ContextShared { - Logger * log = &Logger::get("Context"); + Poco::Logger * log = &Poco::Logger::get("Context"); /// For access of most of shared objects. Recursive mutex. mutable std::recursive_mutex mutex; @@ -2017,7 +2017,7 @@ std::shared_ptr Context::getActionLocksManager() auto lock = getLock(); if (!shared->action_locks_manager) - shared->action_locks_manager = std::make_shared(); + shared->action_locks_manager = std::make_shared(*this); return shared->action_locks_manager; } diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index fefeeeed988..dac51b21081 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -220,7 +220,7 @@ static bool isSupportedAlterType(int type) DDLWorker::DDLWorker(const std::string & zk_root_dir, Context & context_, const Poco::Util::AbstractConfiguration * config, const String & prefix) - : context(context_), log(&Logger::get("DDLWorker")) + : context(context_), log(&Poco::Logger::get("DDLWorker")) { queue_dir = zk_root_dir; if (queue_dir.back() == '/') @@ -634,7 +634,7 @@ void DDLWorker::processTask(DDLTask & task, const ZooKeeperPtr & zookeeper) { /// It's not CREATE DATABASE auto table_id = context.tryResolveStorageID(*query_with_table, Context::ResolveOrdinary); - storage = DatabaseCatalog::instance().tryGetTable(table_id); + storage = DatabaseCatalog::instance().tryGetTable(table_id, context); } /// For some reason we check consistency of cluster definition only @@ -1073,7 +1073,7 @@ class DDLQueryStatusInputStream : public IBlockInputStream public: DDLQueryStatusInputStream(const String & zk_node_path, const DDLLogEntry & entry, const Context & context_) - : node_path(zk_node_path), context(context_), watch(CLOCK_MONOTONIC_COARSE), log(&Logger::get("DDLQueryStatusInputStream")) + : node_path(zk_node_path), context(context_), watch(CLOCK_MONOTONIC_COARSE), log(&Poco::Logger::get("DDLQueryStatusInputStream")) { sample = Block{ {std::make_shared(), "host"}, @@ -1235,7 +1235,7 @@ private: String node_path; const Context & context; Stopwatch watch; - Logger * log; + Poco::Logger * log; Block sample; diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 2f63d9dadee..c30f888b5a1 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -101,7 +101,7 @@ private: private: Context & context; - Logger * log; + Poco::Logger * log; std::unique_ptr current_context; std::string host_fqdn; /// current host domain name diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 7606fdb255a..4871d8d37aa 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -58,13 +58,17 @@ TemporaryTableHolder::TemporaryTableHolder(const Context & context_, } -TemporaryTableHolder::TemporaryTableHolder(const Context & context_, const ColumnsDescription & columns, const ASTPtr & query) +TemporaryTableHolder::TemporaryTableHolder( + const Context & context_, + const ColumnsDescription & columns, + const ConstraintsDescription & constraints, + const ASTPtr & query) : TemporaryTableHolder ( context_, [&](const StorageID & table_id) { - return StorageMemory::create(table_id, ColumnsDescription{columns}, ConstraintsDescription{}); + return StorageMemory::create(table_id, ColumnsDescription{columns}, ConstraintsDescription{constraints}); }, query ) @@ -97,7 +101,7 @@ StorageID TemporaryTableHolder::getGlobalTableID() const StoragePtr TemporaryTableHolder::getTable() const { - auto table = temporary_tables->tryGetTable("_tmp_" + toString(id)); + auto table = temporary_tables->tryGetTable("_tmp_" + toString(id), *global_context); if (!table) throw Exception("Temporary table " + getGlobalTableID().getNameForLogs() + " not found", ErrorCodes::LOGICAL_ERROR); return table; @@ -108,7 +112,7 @@ void DatabaseCatalog::loadDatabases() { drop_delay_sec = global_context->getConfigRef().getInt("database_atomic_delay_before_drop_table_sec", default_drop_delay_sec); - auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE); + auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE, *global_context); attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables); loadMarkedAsDroppedTables(); @@ -159,6 +163,7 @@ DatabaseAndTable DatabaseCatalog::tryGetByUUID(const UUID & uuid) const DatabaseAndTable DatabaseCatalog::getTableImpl( const StorageID & table_id, + const Context & context, std::optional * exception) const { if (!table_id) @@ -206,7 +211,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( database = it->second; } - auto table = database->tryGetTable(table_id.table_name); + auto table = database->tryGetTable(table_id.table_name, context); if (!table && exception) exception->emplace("Table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); @@ -319,7 +324,7 @@ Databases DatabaseCatalog::getDatabases() const return databases; } -bool DatabaseCatalog::isTableExist(const DB::StorageID & table_id) const +bool DatabaseCatalog::isTableExist(const DB::StorageID & table_id, const Context & context) const { if (table_id.hasUUID()) return tryGetByUUID(table_id.uuid).second != nullptr; @@ -331,12 +336,12 @@ bool DatabaseCatalog::isTableExist(const DB::StorageID & table_id) const if (iter != databases.end()) db = iter->second; } - return db && db->isTableExist(table_id.table_name); + return db && db->isTableExist(table_id.table_name, context); } -void DatabaseCatalog::assertTableDoesntExist(const StorageID & table_id) const +void DatabaseCatalog::assertTableDoesntExist(const StorageID & table_id, const Context & context) const { - if (isTableExist(table_id)) + if (isTableExist(table_id, context)) throw Exception("Table " + table_id.getNameForLogs() + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); } @@ -468,32 +473,32 @@ bool DatabaseCatalog::isDictionaryExist(const StorageID & table_id) const return db && db->isDictionaryExist(table_id.getTableName()); } -StoragePtr DatabaseCatalog::getTable(const StorageID & table_id) const +StoragePtr DatabaseCatalog::getTable(const StorageID & table_id, const Context & context) const { std::optional exc; - auto res = getTableImpl(table_id, &exc); + auto res = getTableImpl(table_id, context, &exc); if (!res.second) throw Exception(*exc); return res.second; } -StoragePtr DatabaseCatalog::tryGetTable(const StorageID & table_id) const +StoragePtr DatabaseCatalog::tryGetTable(const StorageID & table_id, const Context & context) const { - return getTableImpl(table_id, nullptr).second; + return getTableImpl(table_id, context, nullptr).second; } -DatabaseAndTable DatabaseCatalog::getDatabaseAndTable(const StorageID & table_id) const +DatabaseAndTable DatabaseCatalog::getDatabaseAndTable(const StorageID & table_id, const Context & context) const { std::optional exc; - auto res = getTableImpl(table_id, &exc); + auto res = getTableImpl(table_id, context, &exc); if (!res.second) throw Exception(*exc); return res; } -DatabaseAndTable DatabaseCatalog::tryGetDatabaseAndTable(const StorageID & table_id) const +DatabaseAndTable DatabaseCatalog::tryGetDatabaseAndTable(const StorageID & table_id, const Context & context) const { - return getTableImpl(table_id, nullptr); + return getTableImpl(table_id, context, nullptr); } void DatabaseCatalog::loadMarkedAsDroppedTables() diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index a481e3d7e5e..aefed0f372d 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -21,6 +21,7 @@ class Context; class IDatabase; class Exception; class ColumnsDescription; +struct ConstraintsDescription; using DatabasePtr = std::shared_ptr; using DatabaseAndTable = std::pair; @@ -71,7 +72,11 @@ struct TemporaryTableHolder : boost::noncopyable TemporaryTableHolder(const Context & context, const Creator & creator, const ASTPtr & query = {}); /// Creates temporary table with Engine=Memory - TemporaryTableHolder(const Context & context, const ColumnsDescription & columns, const ASTPtr & query = {}); + TemporaryTableHolder( + const Context & context, + const ColumnsDescription & columns, + const ConstraintsDescription & constraints, + const ASTPtr & query = {}); TemporaryTableHolder(TemporaryTableHolder && rhs); TemporaryTableHolder & operator = (TemporaryTableHolder && rhs); @@ -129,15 +134,17 @@ public: DatabasePtr getDatabase(const String & database_name, const Context & local_context) const; /// For all of the following methods database_name in table_id must be not empty (even for temporary tables). - void assertTableDoesntExist(const StorageID & table_id) const; - bool isTableExist(const StorageID & table_id) const; + void assertTableDoesntExist(const StorageID & table_id, const Context & context) const; + bool isTableExist(const StorageID & table_id, const Context & context) const; bool isDictionaryExist(const StorageID & table_id) const; - StoragePtr getTable(const StorageID & table_id) const; - StoragePtr tryGetTable(const StorageID & table_id) const; - DatabaseAndTable getDatabaseAndTable(const StorageID & table_id) const; - DatabaseAndTable tryGetDatabaseAndTable(const StorageID & table_id) const; - DatabaseAndTable getTableImpl(const StorageID & table_id, std::optional * exception = nullptr) const; + StoragePtr getTable(const StorageID & table_id, const Context & context) const; + StoragePtr tryGetTable(const StorageID & table_id, const Context & context) const; + DatabaseAndTable getDatabaseAndTable(const StorageID & table_id, const Context & context) const; + DatabaseAndTable tryGetDatabaseAndTable(const StorageID & table_id, const Context & context) const; + DatabaseAndTable getTableImpl(const StorageID & table_id, + const Context & context, + std::optional * exception = nullptr) const; void addDependency(const StorageID & from, const StorageID & where); void removeDependency(const StorageID & from, const StorageID & where); diff --git a/src/Interpreters/EmbeddedDictionaries.cpp b/src/Interpreters/EmbeddedDictionaries.cpp index 9ab3cf2dcbe..58b21fdda52 100644 --- a/src/Interpreters/EmbeddedDictionaries.cpp +++ b/src/Interpreters/EmbeddedDictionaries.cpp @@ -124,7 +124,7 @@ EmbeddedDictionaries::EmbeddedDictionaries( std::unique_ptr geo_dictionaries_loader_, Context & context_, const bool throw_on_error) - : log(&Logger::get("EmbeddedDictionaries")) + : log(&Poco::Logger::get("EmbeddedDictionaries")) , context(context_) , geo_dictionaries_loader(std::move(geo_dictionaries_loader_)) , reload_period(context_.getConfigRef().getInt("builtin_dictionaries_reload_interval", 3600)) diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 0e9be3c7a96..e2884d99516 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -111,11 +111,11 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr } else { - BlockIO res = interpreter.execute(); + auto stream = interpreter.execute().getInputStream(); try { - block = res.in->read(); + block = stream->read(); if (!block) { @@ -126,7 +126,7 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr return; } - if (block.rows() != 1 || res.in->read()) + if (block.rows() != 1 || stream->read()) throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); } catch (const Exception & e) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 363d4765019..c86239b71ae 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -342,11 +342,7 @@ void ExpressionAction::execute(Block & block, bool dry_run, ExtraBlockPtr & not_ { ColumnNumbers arguments(argument_names.size()); for (size_t i = 0; i < argument_names.size(); ++i) - { - if (!block.has(argument_names[i])) - throw Exception("Not found column: '" + argument_names[i] + "'", ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); arguments[i] = block.getPositionByName(argument_names[i]); - } size_t num_columns_without_result = block.columns(); block.insert({ nullptr, result_type, result_name}); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b38af6feef9..b4988f00699 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -297,13 +297,13 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr } auto interpreter_subquery = interpretSubquery(subquery_or_table_name, context, {}, query_options); - BlockIO res = interpreter_subquery->execute(); + auto stream = interpreter_subquery->execute().getInputStream(); SetPtr set = std::make_shared(settings.size_limits_for_set, true, context.getSettingsRef().transform_null_in); - set->setHeader(res.in->getHeader()); + set->setHeader(stream->getHeader()); - res.in->readPrefix(); - while (Block block = res.in->read()) + stream->readPrefix(); + while (Block block = stream->read()) { /// If the limits have been exceeded, give up and let the default subquery processing actions take place. if (!set->insertFromBlock(block)) @@ -311,7 +311,7 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr } set->finishInsert(); - res.in->readSuffix(); + stream->readSuffix(); prepared_sets[set_key] = std::move(set); } @@ -322,7 +322,7 @@ SetPtr SelectQueryExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & if (!table) return nullptr; auto table_id = context.resolveStorageID(subquery_or_table_name); - const auto storage = DatabaseCatalog::instance().getTable(table_id); + const auto storage = DatabaseCatalog::instance().getTable(table_id, context); if (storage->getName() != "Set") return nullptr; const auto storage_set = std::dynamic_pointer_cast(storage); diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 4e958a8c12b..ebbac4c5471 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -15,7 +15,7 @@ namespace DB /// Must not acquire Context lock in constructor to avoid possibility of deadlocks. ExternalDictionariesLoader::ExternalDictionariesLoader(Context & context_) - : ExternalLoader("external dictionary", &Logger::get("ExternalDictionariesLoader")) + : ExternalLoader("external dictionary", &Poco::Logger::get("ExternalDictionariesLoader")) , context(context_) { setConfigSettings({"dictionary", "name", "database"}); diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index fc6505e8446..cd46845e2ed 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -100,7 +100,7 @@ namespace class ExternalLoader::LoadablesConfigReader : private boost::noncopyable { public: - LoadablesConfigReader(const String & type_name_, Logger * log_) + LoadablesConfigReader(const String & type_name_, Poco::Logger * log_) : type_name(type_name_), log(log_) { } @@ -366,7 +366,7 @@ private: } const String type_name; - Logger * log; + Poco::Logger * log; std::mutex mutex; ExternalLoaderConfigSettings settings; @@ -389,7 +389,7 @@ public: LoadingDispatcher( const CreateObjectFunction & create_object_function_, const String & type_name_, - Logger * log_) + Poco::Logger * log_) : create_object(create_object_function_) , type_name(type_name_) , log(log_) @@ -1140,7 +1140,7 @@ private: const CreateObjectFunction create_object; const String type_name; - Logger * log; + Poco::Logger * log; mutable std::mutex mutex; std::condition_variable event; @@ -1220,7 +1220,7 @@ private: }; -ExternalLoader::ExternalLoader(const String & type_name_, Logger * log_) +ExternalLoader::ExternalLoader(const String & type_name_, Poco::Logger * log_) : config_files_reader(std::make_unique(type_name_, log_)) , loading_dispatcher(std::make_unique( [this](auto && a, auto && b, auto && c) { return createObject(a, b, c); }, diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index 9f9fa97b156..542a40e6cb2 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -82,7 +82,7 @@ public: template static constexpr bool is_vector_load_result_type = std::is_same_v || std::is_same_v; - ExternalLoader(const String & type_name_, Logger * log); + ExternalLoader(const String & type_name_, Poco::Logger * log); virtual ~ExternalLoader(); /// Adds a repository which will be used to read configurations from. diff --git a/src/Interpreters/ExternalModelsLoader.cpp b/src/Interpreters/ExternalModelsLoader.cpp index 31356e34855..0300bd44949 100644 --- a/src/Interpreters/ExternalModelsLoader.cpp +++ b/src/Interpreters/ExternalModelsLoader.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes ExternalModelsLoader::ExternalModelsLoader(Context & context_) - : ExternalLoader("external model", &Logger::get("ExternalModelsLoader")) + : ExternalLoader("external model", &Poco::Logger::get("ExternalModelsLoader")) , context(context_) { setConfigSettings({"model", "name", {}}); diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 37a358c3d28..041d26a3d58 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -103,7 +103,7 @@ public: Block sample = interpreter->getSampleBlock(); NamesAndTypesList columns = sample.getNamesAndTypesList(); - auto external_storage_holder = std::make_shared(context, ColumnsDescription{columns}); + auto external_storage_holder = std::make_shared(context, ColumnsDescription{columns}, ConstraintsDescription{}); StoragePtr external_storage = external_storage_holder->getTable(); /** We replace the subquery with the name of the temporary table. @@ -134,7 +134,7 @@ public: ast = database_and_table_name; external_tables[external_table_name] = external_storage_holder; - subqueries_for_sets[external_table_name].source = interpreter->execute().in; + subqueries_for_sets[external_table_name].source = interpreter->execute().getInputStream(); subqueries_for_sets[external_table_name].table = external_storage; /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index cc2d62a68fe..5fd2239d104 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -203,7 +203,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , any_take_last_row(any_take_last_row_) , asof_inequality(table_join->getAsofInequality()) , data(std::make_shared()) - , log(&Logger::get("HashJoin")) + , log(&Poco::Logger::get("HashJoin")) { setSampleBlock(right_sample_block); } diff --git a/src/Interpreters/IInterpreter.h b/src/Interpreters/IInterpreter.h index 2425d015e56..41613d30e87 100644 --- a/src/Interpreters/IInterpreter.h +++ b/src/Interpreters/IInterpreter.h @@ -2,14 +2,8 @@ #include -#include - namespace DB { -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} /** Interpreters interface for different queries. */ @@ -22,14 +16,10 @@ public: */ virtual BlockIO execute() = 0; - virtual QueryPipeline executeWithProcessors() { throw Exception("executeWithProcessors not implemented", ErrorCodes::NOT_IMPLEMENTED); } - - virtual bool canExecuteWithProcessors() const { return false; } - virtual bool ignoreQuota() const { return false; } virtual bool ignoreLimits() const { return false; } - virtual ~IInterpreter() {} + virtual ~IInterpreter() = default; }; } diff --git a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp index f40e91e7dcd..432a7f40b12 100644 --- a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp +++ b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp @@ -27,7 +27,7 @@ namespace StoragePtr tryGetTable(const ASTPtr & database_and_table, const Context & context) { auto table_id = context.resolveStorageID(database_and_table); - return DatabaseCatalog::instance().tryGetTable(table_id); + return DatabaseCatalog::instance().tryGetTable(table_id, context); } using CheckShardsAndTables = InJoinSubqueriesPreprocessor::CheckShardsAndTables; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index c3ee740a486..bd20d78279d 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -42,7 +42,7 @@ BlockIO InterpreterAlterQuery::execute() context.checkAccess(getRequiredAccess()); auto table_id = context.resolveStorageID(alter, Context::ResolveOrdinary); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); /// Add default database to table identifiers that we can encounter in e.g. default expressions, /// mutation expression, etc. diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index b8f7203e607..b3cd807abe5 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -41,7 +41,7 @@ BlockIO InterpreterCheckQuery::execute() auto table_id = context.resolveStorageID(check, Context::ResolveOrdinary); context.checkAccess(AccessType::SHOW_TABLES, table_id); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); auto check_results = table->checkData(query_ptr, context); Block block; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 2c0072c13c2..fc493c8dde7 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -71,6 +71,7 @@ namespace ErrorCodes extern const int BAD_DATABASE_FOR_TEMPORARY_TABLE; extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY; extern const int DICTIONARY_ALREADY_EXISTS; + extern const int ILLEGAL_COLUMN; } @@ -407,7 +408,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS else if (!create.as_table.empty()) { String as_database_name = context.resolveDatabase(create.as_database); - StoragePtr as_storage = DatabaseCatalog::instance().getTable({as_database_name, create.as_table}); + StoragePtr as_storage = DatabaseCatalog::instance().getTable({as_database_name, create.as_table}, context); /// as_storage->getColumns() and setEngine(...) must be called under structure lock of other_table for CREATE ... AS other_table. as_storage_lock = as_storage->lockStructureForShare( @@ -476,6 +477,21 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat } } } + + if (!create.attach && !context.getSettingsRef().allow_experimental_geo_types) + { + for (const auto & name_and_type_pair : properties.columns.getAllPhysical()) + { + const auto& type = name_and_type_pair.type->getName(); + if (type == "MultiPolygon" || type == "Polygon" || type == "Ring" || type == "Point") + { + String message = "Cannot create table with column '" + name_and_type_pair.name + "' which type is '" + + type + "' because experimental geo types are not allowed. " + + "Set setting allow_experimental_geo_types = 1 in order to allow it."; + throw Exception(message, ErrorCodes::ILLEGAL_COLUMN); + } + } + } } void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const @@ -505,7 +521,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const String as_database_name = context.resolveDatabase(create.as_database); String as_table_name = create.as_table; - ASTPtr as_create_ptr = DatabaseCatalog::instance().getDatabase(as_database_name)->getCreateTableQuery(as_table_name); + ASTPtr as_create_ptr = DatabaseCatalog::instance().getDatabase(as_database_name)->getCreateTableQuery(as_table_name, context); const auto & as_create = as_create_ptr->as(); const String qualified_name = backQuoteIfNeed(as_database_name) + "." + backQuoteIfNeed(as_table_name); @@ -547,7 +563,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) bool if_not_exists = create.if_not_exists; // Table SQL definition is available even if the table is detached - auto query = database->getCreateTableQuery(create.table); + auto query = database->getCreateTableQuery(create.table, context); create = query->as(); // Copy the saved create query, but use ATTACH instead of CREATE create.attach = true; create.attach_short_syntax = true; @@ -609,7 +625,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, guard = DatabaseCatalog::instance().getDDLGuard(create.database, table_name); /// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard. - if (database->isTableExist(table_name)) + if (database->isTableExist(table_name, context)) { /// TODO Check structure of table if (create.if_not_exists) @@ -638,7 +654,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, if (create.if_not_exists && context.tryResolveStorageID({"", table_name}, Context::ResolveExternal)) return false; - auto temporary_table = TemporaryTableHolder(context, properties.columns, query_ptr); + auto temporary_table = TemporaryTableHolder(context, properties.columns, properties.constraints, query_ptr); context.getSessionContext().addExternalTable(table_name, std::move(temporary_table)); return true; } diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 1a6dbbcfc4d..d457fefed6a 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -86,7 +86,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() { auto table_id = context.resolveStorageID(table_expression.database_and_table_name); context.checkAccess(AccessType::SHOW_COLUMNS, table_id); - table = DatabaseCatalog::instance().getTable(table_id); + table = DatabaseCatalog::instance().getTable(table_id, context); } auto table_lock = table->lockStructureForShare( diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index ecdad0168f0..5ffce2fc3ec 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -81,8 +81,8 @@ BlockIO InterpreterDropQuery::executeToTable( auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); /// If table was already dropped by anyone, an exception will be thrown - auto [database, table] = query.if_exists ? DatabaseCatalog::instance().tryGetDatabaseAndTable(table_id) - : DatabaseCatalog::instance().getDatabaseAndTable(table_id); + auto [database, table] = query.if_exists ? DatabaseCatalog::instance().tryGetDatabaseAndTable(table_id, context) + : DatabaseCatalog::instance().getDatabaseAndTable(table_id, context); if (database && table) { @@ -182,7 +182,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name, auto resolved_id = context_handle.tryResolveStorageID(StorageID("", table_name), Context::ResolveExternal); if (resolved_id) { - StoragePtr table = DatabaseCatalog::instance().getTable(resolved_id); + StoragePtr table = DatabaseCatalog::instance().getTable(resolved_id, context); if (kind == ASTDropQuery::Kind::Truncate) { auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); @@ -234,7 +234,7 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS ASTDropQuery query; query.kind = kind; query.database = database_name; - for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) { query.table = iterator->name(); executeToTable({query.database, query.table}, query); diff --git a/src/Interpreters/InterpreterExistsQuery.cpp b/src/Interpreters/InterpreterExistsQuery.cpp index 993b3631e06..8539cb6816e 100644 --- a/src/Interpreters/InterpreterExistsQuery.cpp +++ b/src/Interpreters/InterpreterExistsQuery.cpp @@ -50,7 +50,7 @@ BlockInputStreamPtr InterpreterExistsQuery::executeImpl() { String database = context.resolveDatabase(exists_query->database); context.checkAccess(AccessType::SHOW_TABLES, database, exists_query->table); - result = DatabaseCatalog::instance().isTableExist({database, exists_query->table}); + result = DatabaseCatalog::instance().isTableExist({database, exists_query->table}, context); } } else if ((exists_query = query_ptr->as())) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 1c1e21fc32c..dacd7ca5f20 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -76,7 +76,7 @@ namespace if (const auto * identifier = expression.database_and_table_name->as()) { auto table_id = data.context.resolveStorageID(*identifier); - const auto & storage = DatabaseCatalog::instance().getTable(table_id); + const auto & storage = DatabaseCatalog::instance().getTable(table_id, data.context); if (auto * storage_view = dynamic_cast(storage.get())) storage_view->getRuntimeViewQuery(&select_query, data.context, true); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b6efa5d6d46..c4a8e3041ac 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -70,7 +70,7 @@ StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query) } query.table_id = context.resolveStorageID(query.table_id); - return DatabaseCatalog::instance().getTable(query.table_id); + return DatabaseCatalog::instance().getTable(query.table_id, context); } Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table) const @@ -204,7 +204,7 @@ BlockIO InterpreterInsertQuery::execute() { /// Passing 1 as subquery_depth will disable limiting size of intermediate result. InterpreterSelectWithUnionQuery interpreter_select{ query.select, context, SelectQueryOptions(QueryProcessingStage::Complete, 1)}; - res.pipeline = interpreter_select.executeWithProcessors(); + res = interpreter_select.execute(); if (table->supportsParallelInsert() && settings.max_insert_threads > 1) out_streams_size = std::min(size_t(settings.max_insert_threads), res.pipeline.getNumStreams()); @@ -233,6 +233,21 @@ BlockIO InterpreterInsertQuery::execute() else out = std::make_shared(table, context, query_ptr, no_destination); + /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. + + /// Checking constraints. It must be done after calculation of all defaults, so we can check them on calculated columns. + if (const auto & constraints = table->getConstraints(); !constraints.empty()) + out = std::make_shared( + query.table_id, out, out->getHeader(), table->getConstraints(), context); + + /// Actually we don't know structure of input blocks from query/table, + /// because some clients break insertion protocol (columns != header) + out = std::make_shared( + out, query_sample_block, out->getHeader(), table->getColumns().getDefaults(), context); + + /// It's important to squash blocks as early as possible (before other transforms), + /// because other transforms may work inefficient if block size is small. + /// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side. /// Client-side bufferization might cause excessive timeouts (especially in case of big blocks). if (!(context.getSettingsRef().insert_distributed_sync && table->isRemote()) && !no_squash && !query.watch) @@ -244,15 +259,6 @@ BlockIO InterpreterInsertQuery::execute() context.getSettingsRef().min_insert_block_size_bytes); } - /// Actually we don't know structure of input blocks from query/table, - /// because some clients break insertion protocol (columns != header) - out = std::make_shared( - out, query_sample_block, out->getHeader(), table->getColumns().getDefaults(), context); - - if (const auto & constraints = table->getConstraints(); !constraints.empty()) - out = std::make_shared( - query.table_id, out, query_sample_block, table->getConstraints(), context); - auto out_wrapper = std::make_shared(out); out_wrapper->setProcessListElement(context.getProcessListElement()); out = std::move(out_wrapper); diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 42afd0ef477..82c134aeba6 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -261,7 +261,7 @@ BlockIO InterpreterKillQueryQuery::execute() CancellationCode code = CancellationCode::Unknown; if (!query.test) { - auto storage = DatabaseCatalog::instance().tryGetTable(table_id); + auto storage = DatabaseCatalog::instance().tryGetTable(table_id, context); if (!storage) code = CancellationCode::NotFound; else diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index b6c50e59cc3..c47fe1160cf 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -25,7 +25,7 @@ BlockIO InterpreterOptimizeQuery::execute() context.checkAccess(getRequiredAccess()); auto table_id = context.resolveStorageID(ast, Context::ResolveOrdinary); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context); return {}; } diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index 78d1c7ee486..de2b6bb0c1c 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -78,7 +78,7 @@ BlockIO InterpreterRenameQuery::execute() for (auto & elem : descriptions) { if (!rename.exchange) - database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name)); + database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name), context); database_catalog.getDatabase(elem.from_database_name)->renameTable( context, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 4f717eda706..4728c214db0 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1,39 +1,15 @@ #include -#include -#include -#include -#include -#include -#include -#include #include #include -#include -#include -#include -#include -#include -#include -#include #include #include -#include -#include -#include -#include -#include #include -#include -#include -#include - #include #include #include #include #include #include -#include #include #include @@ -59,7 +35,6 @@ #include #include -#include #include #include @@ -68,7 +43,6 @@ #include #include #include -#include #include #include #include @@ -98,7 +72,8 @@ #include #include #include -#include +#include +#include namespace DB @@ -226,7 +201,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( , storage(storage_) , input(input_) , input_pipe(std::move(input_pipe_)) - , log(&Logger::get("InterpreterSelectQuery")) + , log(&Poco::Logger::get("InterpreterSelectQuery")) { checkStackSize(); @@ -465,38 +440,23 @@ Block InterpreterSelectQuery::getSampleBlock() BlockIO InterpreterSelectQuery::execute() { - Pipeline pipeline; BlockIO res; - executeImpl(pipeline, input, std::move(input_pipe), res.pipeline); - executeUnion(pipeline, getSampleBlock()); - - res.in = pipeline.firstStream(); + executeImpl(res.pipeline, input, std::move(input_pipe)); res.pipeline.addInterpreterContext(context); res.pipeline.addStorageHolder(storage); + + /// We must guarantee that result structure is the same as in getSampleBlock() + if (!blocksHaveEqualStructure(res.pipeline.getHeader(), result_header)) + { + res.pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, result_header, ConvertingTransform::MatchColumnsMode::Name); + }); + } + return res; } -BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams(QueryPipeline & parent_pipeline) -{ - ///FIXME pipeline must be alive until query is finished - Pipeline pipeline; - executeImpl(pipeline, input, std::move(input_pipe), parent_pipeline); - unifyStreams(pipeline, getSampleBlock()); - parent_pipeline.addInterpreterContext(context); - parent_pipeline.addStorageHolder(storage); - return pipeline.streams; -} - -QueryPipeline InterpreterSelectQuery::executeWithProcessors() -{ - QueryPipeline query_pipeline; - executeImpl(query_pipeline, input, std::move(input_pipe), query_pipeline); - query_pipeline.addInterpreterContext(context); - query_pipeline.addStorageHolder(storage); - return query_pipeline; -} - - Block InterpreterSelectQuery::getSampleBlockImpl() { if (storage && !options.only_analyze) @@ -687,9 +647,7 @@ static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & c return 0; } - -template -void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, std::optional prepared_pipe, QueryPipeline & save_context_and_storage) +void InterpreterSelectQuery::executeImpl(QueryPipeline & pipeline, const BlockInputStreamPtr & prepared_input, std::optional prepared_pipe) { /** Streams of data. When the query is executed in parallel, we have several data streams. * If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then @@ -701,8 +659,6 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS * then perform the remaining operations with one resulting stream. */ - constexpr bool pipeline_with_processors = std::is_same::value; - /// Now we will compose block streams that perform the necessary actions. auto & query = getSelectQuery(); const Settings & settings = context->getSettingsRef(); @@ -712,40 +668,27 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (options.only_analyze) { - if constexpr (pipeline_with_processors) - pipeline.init(Pipe(std::make_shared(source_header))); - else - pipeline.streams.emplace_back(std::make_shared(source_header)); + pipeline.init(Pipe(std::make_shared(source_header))); if (expressions.prewhere_info) { - if constexpr (pipeline_with_processors) - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, - expressions.prewhere_info->prewhere_actions, - expressions.prewhere_info->prewhere_column_name, - expressions.prewhere_info->remove_prewhere_column); - }); - else - pipeline.streams.back() = std::make_shared( - pipeline.streams.back(), expressions.prewhere_info->prewhere_actions, - expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column); + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + expressions.prewhere_info->prewhere_actions, + expressions.prewhere_info->prewhere_column_name, + expressions.prewhere_info->remove_prewhere_column); + }); // To remove additional columns in dry run // For example, sample column which can be removed in this stage if (expressions.prewhere_info->remove_columns_actions) { - if constexpr (pipeline_with_processors) + pipeline.addSimpleTransform([&](const Block & header) { - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, expressions.prewhere_info->remove_columns_actions); - }); - } - else - pipeline.streams.back() = std::make_shared(pipeline.streams.back(), expressions.prewhere_info->remove_columns_actions); + return std::make_shared(header, expressions.prewhere_info->remove_columns_actions); + }); } } } @@ -753,17 +696,11 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS { if (prepared_input) { - if constexpr (pipeline_with_processors) - pipeline.init(Pipe(std::make_shared(prepared_input))); - else - pipeline.streams.push_back(prepared_input); + pipeline.init(Pipe(std::make_shared(prepared_input))); } else if (prepared_pipe) { - if constexpr (pipeline_with_processors) - pipeline.init(std::move(*prepared_pipe)); - else - pipeline.streams.push_back(std::make_shared(std::move(*prepared_pipe))); + pipeline.init(std::move(*prepared_pipe)); } if (from_stage == QueryProcessingStage::WithMergeableState && @@ -774,7 +711,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS throw Exception("PREWHERE is not supported if the table is filtered by row-level security expression", ErrorCodes::ILLEGAL_PREWHERE); /** Read the data from Storage. from_stage - to what stage the request was completed in Storage. */ - executeFetchColumns(from_stage, pipeline, expressions.prewhere_info, expressions.columns_to_remove_after_prewhere, save_context_and_storage); + executeFetchColumns(from_stage, pipeline, expressions.prewhere_info, expressions.columns_to_remove_after_prewhere); LOG_TRACE(log, "{} -> {}", QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(options.to_stage)); } @@ -817,12 +754,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS } if (query.limitLength()) - { - if constexpr (pipeline_with_processors) - executePreLimit(pipeline, true); - else - executePreLimit(pipeline); - } + executePreLimit(pipeline, true); } }; @@ -840,31 +772,17 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS { if (expressions.hasFilter()) { - if constexpr (pipeline_with_processors) + pipeline.addSimpleTransform([&](const Block & block, QueryPipeline::StreamType stream_type) -> ProcessorPtr { - pipeline.addSimpleTransform([&](const Block & block, QueryPipeline::StreamType stream_type) -> ProcessorPtr - { - bool on_totals = stream_type == QueryPipeline::StreamType::Totals; + bool on_totals = stream_type == QueryPipeline::StreamType::Totals; - return std::make_shared( - block, - expressions.filter_info->actions, - expressions.filter_info->column_name, - expressions.filter_info->do_remove_column, - on_totals); - }); - } - else - { - pipeline.transform([&](auto & stream) - { - stream = std::make_shared( - stream, - expressions.filter_info->actions, - expressions.filter_info->column_name, - expressions.filter_info->do_remove_column); - }); - } + return std::make_shared( + block, + expressions.filter_info->actions, + expressions.filter_info->column_name, + expressions.filter_info->do_remove_column, + on_totals); + }); } if (expressions.hasJoin()) @@ -872,59 +790,43 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS Block join_result_sample; JoinPtr join = expressions.before_join->getTableJoinAlgo(); - if constexpr (pipeline_with_processors) + join_result_sample = ExpressionBlockInputStream( + std::make_shared(pipeline.getHeader()), expressions.before_join).getHeader(); + + /// In case joined subquery has totals, and we don't, add default chunk to totals. + bool default_totals = false; + if (!pipeline.hasTotals()) { - join_result_sample = ExpressionBlockInputStream( - std::make_shared(pipeline.getHeader()), expressions.before_join).getHeader(); - - /// In case joined subquery has totals, and we don't, add default chunk to totals. - bool default_totals = false; - if (!pipeline.hasTotals()) - { - pipeline.addDefaultTotals(); - default_totals = true; - } - - bool inflating_join = false; - if (join) - { - inflating_join = true; - if (auto * hash_join = typeid_cast(join.get())) - inflating_join = isCross(hash_join->getKind()); - } - - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType type) - { - bool on_totals = type == QueryPipeline::StreamType::Totals; - std::shared_ptr ret; - if (inflating_join) - ret = std::make_shared(header, expressions.before_join, on_totals, default_totals); - else - ret = std::make_shared(header, expressions.before_join, on_totals, default_totals); - - return ret; - }); + pipeline.addDefaultTotals(); + default_totals = true; } - else + + bool inflating_join = false; + if (join) { - /// Applies to all sources except stream_with_non_joined_data. - for (auto & stream : pipeline.streams) - stream = std::make_shared(stream, expressions.before_join); - - join_result_sample = pipeline.firstStream()->getHeader(); + inflating_join = true; + if (auto * hash_join = typeid_cast(join.get())) + inflating_join = isCross(hash_join->getKind()); } + pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType type) + { + bool on_totals = type == QueryPipeline::StreamType::Totals; + std::shared_ptr ret; + if (inflating_join) + ret = std::make_shared(header, expressions.before_join, on_totals, default_totals); + else + ret = std::make_shared(header, expressions.before_join, on_totals, default_totals); + + return ret; + }); + if (join) { if (auto stream = join->createStreamWithNonJoinedRows(join_result_sample, settings.max_block_size)) { - if constexpr (pipeline_with_processors) - { - auto source = std::make_shared(std::move(stream)); - pipeline.addDelayedStream(source); - } - else - pipeline.stream_with_non_joined_data = std::move(stream); + auto source = std::make_shared(std::move(stream)); + pipeline.addDelayedStream(source); } } } @@ -1009,28 +911,14 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (query.limitLength() && !query.limit_with_ties && pipeline.hasMoreThanOneStream() && !query.distinct && !expressions.hasLimitBy() && !settings.extremes) { - if constexpr (pipeline_with_processors) - executePreLimit(pipeline, false); - else - executePreLimit(pipeline); - + executePreLimit(pipeline, false); has_prelimit = true; } - bool need_merge_streams = need_second_distinct_pass || query.limitBy() - || (!pipeline_with_processors && query.limitLength()); /// Don't merge streams for pre-limit more. - - if constexpr (!pipeline_with_processors) - if (pipeline.hasDelayedStream()) - need_merge_streams = true; + bool need_merge_streams = need_second_distinct_pass || query.limitBy(); if (need_merge_streams) - { - if constexpr (pipeline_with_processors) - pipeline.resize(1); - else - executeUnion(pipeline, {}); - } + pipeline.resize(1); /** If there was more than one stream, * then DISTINCT needs to be performed once again after merging all streams. @@ -1054,7 +942,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS */ executeExtremes(pipeline); - if (!(pipeline_with_processors && has_prelimit)) /// Limit is no longer needed if there is prelimit. + if (!has_prelimit) /// Limit is no longer needed if there is prelimit. executeLimit(pipeline); executeOffset(pipeline); @@ -1065,14 +953,10 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS executeSubqueriesInSetsAndJoins(pipeline, subqueries_for_sets); } -template void InterpreterSelectQuery::executeFetchColumns( - QueryProcessingStage::Enum processing_stage, TPipeline & pipeline, - const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere, - QueryPipeline & save_context_and_storage) + QueryProcessingStage::Enum processing_stage, QueryPipeline & pipeline, + const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere) { - constexpr bool pipeline_with_processors = std::is_same::value; - auto & query = getSelectQuery(); const Settings & settings = context->getSettingsRef(); @@ -1126,10 +1010,7 @@ void InterpreterSelectQuery::executeFetchColumns( {std::move(column), std::make_shared(func, argument_types, desc->parameters), desc->column_name}}; auto istream = std::make_shared(block_with_count); - if constexpr (pipeline_with_processors) - pipeline.init(Pipe(std::make_shared(istream))); - else - pipeline.streams.emplace_back(istream); + pipeline.init(Pipe(std::make_shared(istream))); from_stage = QueryProcessingStage::WithMergeableState; analysis_result.first_stage = false; return; @@ -1266,7 +1147,7 @@ void InterpreterSelectQuery::executeFetchColumns( /// Remove columns which will be added by prewhere. required_columns.erase(std::remove_if(required_columns.begin(), required_columns.end(), [&](const String & name) { - return !!required_columns_after_prewhere_set.count(name); + return required_columns_after_prewhere_set.count(name) != 0; }), required_columns.end()); if (prewhere_info) @@ -1377,11 +1258,7 @@ void InterpreterSelectQuery::executeFetchColumns( interpreter_subquery->ignoreWithTotals(); } - if constexpr (pipeline_with_processors) - /// Just use pipeline from subquery. - pipeline = interpreter_subquery->executeWithProcessors(); - else - pipeline.streams = interpreter_subquery->executeWithMultipleStreams(save_context_and_storage); + pipeline = interpreter_subquery->execute().pipeline; } else if (storage) { @@ -1411,48 +1288,9 @@ void InterpreterSelectQuery::executeFetchColumns( query_info.input_sorting_info = query_info.order_by_optimizer->getInputOrder(storage); } + Pipes pipes = storage->read(required_columns, query_info, *context, processing_stage, max_block_size, max_streams); - BlockInputStreams streams; - Pipes pipes; - - if (pipeline_with_processors) - pipes = storage->read(required_columns, query_info, *context, processing_stage, max_block_size, max_streams); - else - streams = storage->readStreams(required_columns, query_info, *context, processing_stage, max_block_size, max_streams); - - if (streams.empty() && !pipeline_with_processors) - { - streams = {std::make_shared(storage->getSampleBlockForColumns(required_columns))}; - - if (query_info.prewhere_info) - { - if (query_info.prewhere_info->alias_actions) - { - streams.back() = std::make_shared( - streams.back(), - query_info.prewhere_info->alias_actions); - } - - streams.back() = std::make_shared( - streams.back(), - prewhere_info->prewhere_actions, - prewhere_info->prewhere_column_name, - prewhere_info->remove_prewhere_column); - - // To remove additional columns - // In some cases, we did not read any marks so that the pipeline.streams is empty - // Thus, some columns in prewhere are not removed as expected - // This leads to mismatched header in distributed table - if (query_info.prewhere_info->remove_columns_actions) - { - streams.back() = std::make_shared(streams.back(), query_info.prewhere_info->remove_columns_actions); - } - } - } - - /// Copy-paste from prev if. - /// Code is temporarily copy-pasted while moving to new pipeline. - if (pipes.empty() && pipeline_with_processors) + if (pipes.empty()) { Pipe pipe(std::make_shared(storage->getSampleBlockForColumns(required_columns))); @@ -1468,6 +1306,10 @@ void InterpreterSelectQuery::executeFetchColumns( prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)); + // To remove additional columns + // In some cases, we did not read any marks so that the pipeline.streams is empty + // Thus, some columns in prewhere are not removed as expected + // This leads to mismatched header in distributed table if (query_info.prewhere_info->remove_columns_actions) pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), query_info.prewhere_info->remove_columns_actions)); } @@ -1475,14 +1317,8 @@ void InterpreterSelectQuery::executeFetchColumns( pipes.emplace_back(std::move(pipe)); } - for (auto & stream : streams) - stream->addTableLock(table_lock); - - if constexpr (pipeline_with_processors) - { - /// Table lock is stored inside pipeline here. - pipeline.addTableLock(table_lock); - } + /// Table lock is stored inside pipeline here. + pipeline.addTableLock(table_lock); /// Set the limits and quota for reading data, the speed and time of the query. { @@ -1498,28 +1334,21 @@ void InterpreterSelectQuery::executeFetchColumns( * But limits on data size to read and maximum execution time are reasonable to check both on initiator and * additionally on each remote server, because these limits are checked per block of data processed, * and remote servers may process way more blocks of data than are received by initiator. + * + * The limits to throttle maximum execution speed is also checked on all servers. */ if (options.to_stage == QueryProcessingStage::Complete) { limits.speed_limits.min_execution_rps = settings.min_execution_speed; - limits.speed_limits.max_execution_rps = settings.max_execution_speed; limits.speed_limits.min_execution_bps = settings.min_execution_speed_bytes; - limits.speed_limits.max_execution_bps = settings.max_execution_speed_bytes; - limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; } + limits.speed_limits.max_execution_rps = settings.max_execution_speed; + limits.speed_limits.max_execution_bps = settings.max_execution_speed_bytes; + limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; + auto quota = context->getQuota(); - for (auto & stream : streams) - { - if (!options.ignore_limits) - stream->setLimits(limits); - - if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete)) - stream->setQuota(quota); - } - - /// Copy-paste for (auto & pipe : pipes) { if (!options.ignore_limits) @@ -1530,47 +1359,13 @@ void InterpreterSelectQuery::executeFetchColumns( } } - if constexpr (pipeline_with_processors) - { - if (streams.size() == 1 || pipes.size() == 1) - pipeline.setMaxThreads(1); + if (pipes.size() == 1) + pipeline.setMaxThreads(1); - /// Unify streams. They must have same headers. - if (streams.size() > 1) - { - /// Unify streams in case they have different headers. - auto first_header = streams.at(0)->getHeader(); + for (auto & pipe : pipes) + pipe.enableQuota(); - if (first_header.columns() > 1 && first_header.has("_dummy")) - first_header.erase("_dummy"); - - for (auto & stream : streams) - { - auto header = stream->getHeader(); - auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; - if (!blocksHaveEqualStructure(first_header, header)) - stream = std::make_shared(stream, first_header, mode); - } - } - - for (auto & stream : streams) - { - bool force_add_agg_info = processing_stage == QueryProcessingStage::WithMergeableState; - auto source = std::make_shared(stream, force_add_agg_info); - - if (processing_stage == QueryProcessingStage::Complete) - source->addTotalsPort(); - - pipes.emplace_back(std::move(source)); - } - - for (auto & pipe : pipes) - pipe.enableQuota(); - - pipeline.init(std::move(pipes)); - } - else - pipeline.streams = std::move(streams); + pipeline.init(std::move(pipes)); } else throw Exception("Logical error in InterpreterSelectQuery: nowhere to read", ErrorCodes::LOGICAL_ERROR); @@ -1578,32 +1373,14 @@ void InterpreterSelectQuery::executeFetchColumns( /// Aliases in table declaration. if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions) { - if constexpr (pipeline_with_processors) + pipeline.addSimpleTransform([&](const Block & header) { - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, alias_actions); - }); - } - else - { - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, alias_actions); - }); - } + return std::make_shared(header, alias_actions); + }); } } -void InterpreterSelectQuery::executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter) -{ - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, expression, getSelectQuery().where()->getColumnName(), remove_filter); - }); -} - void InterpreterSelectQuery::executeWhere(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter) { pipeline.addSimpleTransform([&](const Block & block, QueryPipeline::StreamType stream_type) @@ -1613,69 +1390,6 @@ void InterpreterSelectQuery::executeWhere(QueryPipeline & pipeline, const Expres }); } -void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final) -{ - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, expression); - }); - - Block header = pipeline.firstStream()->getHeader(); - ColumnNumbers keys; - for (const auto & key : query_analyzer->aggregationKeys()) - keys.push_back(header.getPositionByName(key.name)); - - AggregateDescriptions aggregates = query_analyzer->aggregates(); - for (auto & descr : aggregates) - if (descr.arguments.empty()) - for (const auto & name : descr.argument_names) - descr.arguments.push_back(header.getPositionByName(name)); - - const Settings & settings = context->getSettingsRef(); - - /** Two-level aggregation is useful in two cases: - * 1. Parallel aggregation is done, and the results should be merged in parallel. - * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. - */ - bool allow_to_use_two_level_group_by = pipeline.streams.size() > 1 || settings.max_bytes_before_external_group_by != 0; - - Aggregator::Params params(header, keys, aggregates, - overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode, - allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), - allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), - settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); - - /// If there are several sources, then we perform parallel aggregation - if (pipeline.streams.size() > 1) - { - pipeline.firstStream() = std::make_shared( - pipeline.streams, pipeline.stream_with_non_joined_data, params, final, - max_streams, - settings.aggregation_memory_efficient_merge_threads - ? static_cast(settings.aggregation_memory_efficient_merge_threads) - : static_cast(settings.max_threads)); - - pipeline.stream_with_non_joined_data = nullptr; - pipeline.streams.resize(1); - } - else - { - BlockInputStreams inputs; - if (!pipeline.streams.empty()) - inputs.push_back(pipeline.firstStream()); - else - pipeline.streams.resize(1); - - if (pipeline.stream_with_non_joined_data) - inputs.push_back(pipeline.stream_with_non_joined_data); - - pipeline.firstStream() = std::make_shared(std::make_shared(inputs), params, final); - - pipeline.stream_with_non_joined_data = nullptr; - } -} - void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final) { @@ -1749,53 +1463,6 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const } -void InterpreterSelectQuery::executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final) -{ - Block header = pipeline.firstStream()->getHeader(); - - ColumnNumbers keys; - for (const auto & key : query_analyzer->aggregationKeys()) - keys.push_back(header.getPositionByName(key.name)); - - /** There are two modes of distributed aggregation. - * - * 1. In different threads read from the remote servers blocks. - * Save all the blocks in the RAM. Merge blocks. - * If the aggregation is two-level - parallelize to the number of buckets. - * - * 2. In one thread, read blocks from different servers in order. - * RAM stores only one block from each server. - * If the aggregation is a two-level aggregation, we consistently merge the blocks of each next level. - * - * The second option consumes less memory (up to 256 times less) - * in the case of two-level aggregation, which is used for large results after GROUP BY, - * but it can work more slowly. - */ - - const Settings & settings = context->getSettingsRef(); - - Aggregator::Params params(header, keys, query_analyzer->aggregates(), overflow_row, settings.max_threads); - - if (!settings.distributed_aggregation_memory_efficient) - { - /// We union several sources into one, parallelizing the work. - executeUnion(pipeline, {}); - - /// Now merge the aggregated blocks - pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params, final, settings.max_threads); - } - else - { - pipeline.firstStream() = std::make_shared(pipeline.streams, params, final, - max_streams, - settings.aggregation_memory_efficient_merge_threads - ? static_cast(settings.aggregation_memory_efficient_merge_threads) - : static_cast(settings.max_threads)); - - pipeline.streams.resize(1); - } -} - void InterpreterSelectQuery::executeMergeAggregated(QueryPipeline & pipeline, bool overflow_row, bool final) { Block header_before_merge = pipeline.getHeader(); @@ -1856,14 +1523,6 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPipeline & pipeline, bo } -void InterpreterSelectQuery::executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression) -{ - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, expression, getSelectQuery().having()->getColumnName()); - }); -} - void InterpreterSelectQuery::executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression) { pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr @@ -1876,22 +1535,6 @@ void InterpreterSelectQuery::executeHaving(QueryPipeline & pipeline, const Expre } -void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final) -{ - executeUnion(pipeline, {}); - - const Settings & settings = context->getSettingsRef(); - - pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), - overflow_row, - expression, - has_having ? getSelectQuery().having()->getColumnName() : "", - settings.totals_mode, - settings.totals_auto_threshold, - final); -} - void InterpreterSelectQuery::executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final) { const Settings & settings = context->getSettingsRef(); @@ -1905,31 +1548,6 @@ void InterpreterSelectQuery::executeTotalsAndHaving(QueryPipeline & pipeline, bo } -void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificator modificator) -{ - executeUnion(pipeline, {}); - - Block header = pipeline.firstStream()->getHeader(); - - ColumnNumbers keys; - - for (const auto & key : query_analyzer->aggregationKeys()) - keys.push_back(header.getPositionByName(key.name)); - - const Settings & settings = context->getSettingsRef(); - - Aggregator::Params params(header, keys, query_analyzer->aggregates(), - false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, - SettingUInt64(0), SettingUInt64(0), - settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); - - if (modificator == Modificator::ROLLUP) - pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); - else - pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); -} - void InterpreterSelectQuery::executeRollupOrCube(QueryPipeline & pipeline, Modificator modificator) { pipeline.resize(1); @@ -1964,14 +1582,6 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPipeline & pipeline, Modif } -void InterpreterSelectQuery::executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression) -{ - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, expression); - }); -} - void InterpreterSelectQuery::executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression) { pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr @@ -1980,68 +1590,6 @@ void InterpreterSelectQuery::executeExpression(QueryPipeline & pipeline, const E }); } -void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, InputSortingInfoPtr input_sorting_info) -{ - auto & query = getSelectQuery(); - SortDescription output_order_descr = getSortDescription(query, *context); - const Settings & settings = context->getSettingsRef(); - UInt64 limit = getLimitForSorting(query, *context); - - if (input_sorting_info) - { - /* Case of sorting with optimization using sorting key. - * We have several threads, each of them reads batch of parts in direct - * or reverse order of sorting key using one input stream per part - * and then merge them into one sorted stream. - * At this stage we merge per-thread streams into one. - * If the input is sorted by some prefix of the sorting key required for output, - * we have to finish sorting after the merge. - */ - - bool need_finish_sorting = (input_sorting_info->order_key_prefix_descr.size() < output_order_descr.size()); - - UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); - executeMergeSorted(pipeline, input_sorting_info->order_key_prefix_descr, limit_for_merging); - - if (need_finish_sorting) - { - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, output_order_descr, limit); - }); - - pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), input_sorting_info->order_key_prefix_descr, - output_order_descr, settings.max_block_size, limit); - } - } - else - { - pipeline.transform([&](auto & stream) - { - auto sorting_stream = std::make_shared(stream, output_order_descr, limit); - - /// Limits on sorting - IBlockInputStream::LocalLimits limits; - limits.mode = IBlockInputStream::LIMITS_TOTAL; - limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); - sorting_stream->setLimits(limits); - - auto merging_stream = std::make_shared( - sorting_stream, output_order_descr, settings.max_block_size, limit, - settings.max_bytes_before_remerge_sort, - settings.max_bytes_before_external_sort / pipeline.streams.size(), - context->getTemporaryVolume(), - settings.temporary_files_codec, - settings.min_free_disk_space_for_temporary_data); - - stream = merging_stream; - }); - - /// If there are several streams, we merge them into one - executeMergeSorted(pipeline, output_order_descr, limit); - } -} void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSortingInfoPtr input_sorting_info) { @@ -2138,41 +1686,6 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting } -void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline) -{ - auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, *context); - UInt64 limit = getLimitForSorting(query, *context); - - /// If there are several streams, then we merge them into one - if (pipeline.hasMoreThanOneStream()) - { - unifyStreams(pipeline, pipeline.firstStream()->getHeader()); - executeMergeSorted(pipeline, order_descr, limit); - } -} - - -void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline, const SortDescription & sort_description, UInt64 limit) -{ - if (pipeline.hasMoreThanOneStream()) - { - const Settings & settings = context->getSettingsRef(); - - /** MergingSortedBlockInputStream reads the sources sequentially. - * To make the data on the remote servers prepared in parallel, we wrap it in AsynchronousBlockInputStream. - */ - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream); - }); - - pipeline.firstStream() = std::make_shared( - pipeline.streams, sort_description, settings.max_block_size, limit); - pipeline.streams.resize(1); - } -} - void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline) { auto & query = getSelectQuery(); @@ -2202,14 +1715,6 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline, const } -void InterpreterSelectQuery::executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression) -{ - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, expression); - }); -} - void InterpreterSelectQuery::executeProjection(QueryPipeline & pipeline, const ExpressionActionsPtr & expression) { pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr @@ -2219,28 +1724,6 @@ void InterpreterSelectQuery::executeProjection(QueryPipeline & pipeline, const E } -void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_order, Names columns) -{ - auto & query = getSelectQuery(); - if (query.distinct) - { - const Settings & settings = context->getSettingsRef(); - - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context); - UInt64 limit_for_distinct = 0; - - /// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows. - if ((!query.orderBy() || !before_order) && !query.limit_with_ties) - limit_for_distinct = limit_length + limit_offset; - - pipeline.transform([&](auto & stream) - { - SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); - stream = std::make_shared(stream, limits, limit_for_distinct, columns); - }); - } -} - void InterpreterSelectQuery::executeDistinct(QueryPipeline & pipeline, bool before_order, Names columns) { auto & query = getSelectQuery(); @@ -2268,51 +1751,6 @@ void InterpreterSelectQuery::executeDistinct(QueryPipeline & pipeline, bool befo } -void InterpreterSelectQuery::executeUnion(Pipeline & pipeline, Block header) -{ - /// If there are still several streams, then we combine them into one - if (pipeline.hasMoreThanOneStream()) - { - if (!header) - header = pipeline.firstStream()->getHeader(); - - unifyStreams(pipeline, std::move(header)); - - pipeline.firstStream() = std::make_shared(pipeline.streams, pipeline.stream_with_non_joined_data, max_streams); - pipeline.stream_with_non_joined_data = nullptr; - pipeline.streams.resize(1); - pipeline.union_stream = true; - } - else if (pipeline.stream_with_non_joined_data) - { - pipeline.streams.push_back(pipeline.stream_with_non_joined_data); - pipeline.stream_with_non_joined_data = nullptr; - } -} - - -/// Preliminary LIMIT - is used in every source, if there are several sources, before they are combined. -void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) -{ - auto & query = getSelectQuery(); - /// If there is LIMIT - if (query.limitLength()) - { - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context); - SortDescription sort_descr; - if (query.limit_with_ties) - { - if (!query.orderBy()) - throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - sort_descr = getSortDescription(query, *context); - } - pipeline.transform([&, limit = limit_length + limit_offset](auto & stream) - { - stream = std::make_shared(stream, limit, 0, false, false, query.limit_with_ties, sort_descr); - }); - } -} - /// Preliminary LIMIT - is used in every source, if there are several sources, before they are combined. void InterpreterSelectQuery::executePreLimit(QueryPipeline & pipeline, bool do_not_skip_offset) { @@ -2334,24 +1772,6 @@ void InterpreterSelectQuery::executePreLimit(QueryPipeline & pipeline, bool do_n } -void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline) -{ - auto & query = getSelectQuery(); - if (!query.limitByLength() || !query.limitBy()) - return; - - Names columns; - for (const auto & elem : query.limitBy()->children) - columns.emplace_back(elem->getColumnName()); - UInt64 length = getLimitUIntValue(query.limitByLength(), *context, "LIMIT"); - UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context, "OFFSET") : 0); - - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, length, offset, columns); - }); -} - void InterpreterSelectQuery::executeLimitBy(QueryPipeline & pipeline) { auto & query = getSelectQuery(); @@ -2400,73 +1820,6 @@ namespace } } -void InterpreterSelectQuery::executeLimit(Pipeline & pipeline) -{ - auto & query = getSelectQuery(); - /// If there is LIMIT - if (query.limitLength()) - { - /** Rare case: - * if there is no WITH TOTALS and there is a subquery in FROM, and there is WITH TOTALS on one of the levels, - * then when using LIMIT, you should read the data to the end, rather than cancel the query earlier, - * because if you cancel the query, we will not get `totals` data from the remote server. - * - * Another case: - * if there is WITH TOTALS and there is no ORDER BY, then read the data to the end, - * otherwise TOTALS is counted according to incomplete data. - */ - bool always_read_till_end = false; - - if (query.group_by_with_totals && !query.orderBy()) - always_read_till_end = true; - - if (!query.group_by_with_totals && hasWithTotalsInAnySubqueryInFromClause(query)) - always_read_till_end = true; - - SortDescription order_descr; - if (query.limit_with_ties) - { - if (!query.orderBy()) - throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - order_descr = getSortDescription(query, *context); - } - - UInt64 limit_length; - UInt64 limit_offset; - std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context); - - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, limit_length, limit_offset, always_read_till_end, false, query.limit_with_ties, order_descr); - }); - } -} - -void InterpreterSelectQuery::executeOffset(Pipeline & /* pipeline */) {} - -void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline) -{ - auto & query = getSelectQuery(); - if (query.orderBy()) - { - SortDescription order_descr = getSortDescription(query, *context); - SortDescription fill_descr; - for (auto & desc : order_descr) - { - if (desc.with_fill) - fill_descr.push_back(desc); - } - - if (fill_descr.empty()) - return; - - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, fill_descr); - }); - } -} - void InterpreterSelectQuery::executeWithFill(QueryPipeline & pipeline) { auto & query = getSelectQuery(); @@ -2557,18 +1910,6 @@ void InterpreterSelectQuery::executeOffset(QueryPipeline & pipeline) } } - -void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline) -{ - if (!context->getSettingsRef().extremes) - return; - - pipeline.transform([&](auto & stream) - { - stream->enableExtremes(); - }); -} - void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline) { if (!context->getSettingsRef().extremes) @@ -2577,23 +1918,6 @@ void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline) pipeline.addExtremesTransform(); } - -void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline, const SubqueriesForSets & subqueries_for_sets) -{ - /// Merge streams to one. Use MergeSorting if data was read in sorted order, Union otherwise. - if (query_info.input_sorting_info) - { - if (pipeline.stream_with_non_joined_data) - throw Exception("Using read in order optimization, but has stream with non-joined data in pipeline", ErrorCodes::LOGICAL_ERROR); - executeMergeSorted(pipeline, query_info.input_sorting_info->order_key_prefix_descr, 0); - } - else - executeUnion(pipeline, {}); - - pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), subqueries_for_sets, *context); -} - void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, const SubqueriesForSets & subqueries_for_sets) { if (query_info.input_sorting_info) @@ -2610,25 +1934,6 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pip } -void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline, Block header) -{ - /// Unify streams in case they have different headers. - - /// TODO: remove previous addition of _dummy column. - if (header.columns() > 1 && header.has("_dummy")) - header.erase("_dummy"); - - for (auto & stream : pipeline.streams) - { - auto stream_header = stream->getHeader(); - auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; - - if (!blocksHaveEqualStructure(header, stream_header)) - stream = std::make_shared(stream, header, mode); - } -} - - void InterpreterSelectQuery::ignoreWithTotals() { getSelectQuery().group_by_with_totals = false; diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index b97ff65e988..ca7fb4c72ba 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -77,12 +77,6 @@ public: /// Execute a query. Get the stream of blocks to read. BlockIO execute() override; - /// Execute the query and return multuple streams for parallel processing. - BlockInputStreams executeWithMultipleStreams(QueryPipeline & parent_pipeline); - - QueryPipeline executeWithProcessors() override; - bool canExecuteWithProcessors() const override { return true; } - bool ignoreLimits() const override { return options.ignore_limits; } bool ignoreQuota() const override { return options.ignore_quota; } @@ -108,89 +102,15 @@ private: Block getSampleBlockImpl(); - struct Pipeline - { - /** Streams of data. - * The source data streams are produced in the executeFetchColumns function. - * Then they are converted (wrapped in other streams) using the `execute*` functions, - * to get the whole pipeline running the query. - */ - BlockInputStreams streams; - - /** When executing FULL or RIGHT JOIN, there will be a data stream from which you can read "not joined" rows. - * It has a special meaning, since reading from it should be done after reading from the main streams. - * It is appended to the main streams in UnionBlockInputStream or ParallelAggregatingBlockInputStream. - */ - BlockInputStreamPtr stream_with_non_joined_data; - bool union_stream = false; - - /// Cache value of InterpreterSelectQuery::max_streams - size_t max_threads = 1; - - BlockInputStreamPtr & firstStream() { return streams.at(0); } - - template - void transform(Transform && transformation) - { - for (auto & stream : streams) - transformation(stream); - - if (stream_with_non_joined_data) - transformation(stream_with_non_joined_data); - } - - bool hasMoreThanOneStream() const - { - return streams.size() + (stream_with_non_joined_data ? 1 : 0) > 1; - } - - /// Resulting stream is mix of other streams data. Distinct and/or order guaranties are broken. - bool hasMixedStreams() const - { - return hasMoreThanOneStream() || union_stream; - } - - bool hasDelayedStream() const { return stream_with_non_joined_data != nullptr; } - bool initialized() const { return !streams.empty(); } - - /// Compatibility with QueryPipeline (Processors) - void setMaxThreads(size_t max_threads_) { max_threads = max_threads_; } - size_t getNumThreads() const { return max_threads; } - }; - - template - void executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, std::optional prepared_pipe, QueryPipeline & save_context_and_storage); + void executeImpl(QueryPipeline & pipeline, const BlockInputStreamPtr & prepared_input, std::optional prepared_pipe); /// Different stages of query execution. - /// dry_run - don't read from table, use empty header block instead. - void executeWithMultipleStreamsImpl(Pipeline & pipeline, const BlockInputStreamPtr & input, bool dry_run); - - template - void executeFetchColumns(QueryProcessingStage::Enum processing_stage, TPipeline & pipeline, + void executeFetchColumns( + QueryProcessingStage::Enum processing_stage, + QueryPipeline & pipeline, const PrewhereInfoPtr & prewhere_info, - const Names & columns_to_remove_after_prewhere, - QueryPipeline & save_context_and_storage); - - void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); - void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final); - void executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final); - void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); - void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression); - static void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression); - void executeOrder(Pipeline & pipeline, InputSortingInfoPtr sorting_info); - void executeWithFill(Pipeline & pipeline); - void executeMergeSorted(Pipeline & pipeline); - void executePreLimit(Pipeline & pipeline); - void executeUnion(Pipeline & pipeline, Block header); - void executeLimitBy(Pipeline & pipeline); - void executeLimit(Pipeline & pipeline); - void executeOffset(Pipeline & pipeline); - static void executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression); - void executeDistinct(Pipeline & pipeline, bool before_order, Names columns); - void executeExtremes(Pipeline & pipeline); - void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, const std::unordered_map & subqueries_for_sets); - void executeMergeSorted(Pipeline & pipeline, const SortDescription & sort_description, UInt64 limit); + const Names & columns_to_remove_after_prewhere); void executeWhere(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); void executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final); @@ -213,17 +133,12 @@ private: String generateFilterActions(ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) const; - /// Add ConvertingBlockInputStream to specified header. - static void unifyStreams(Pipeline & pipeline, Block header); - enum class Modificator { ROLLUP = 0, CUBE = 1 }; - void executeRollupOrCube(Pipeline & pipeline, Modificator modificator); - void executeRollupOrCube(QueryPipeline & pipeline, Modificator modificator); /** If there is a SETTINGS section in the SELECT query, then apply settings from it. diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 7fe124b31e6..7b86616555a 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -3,15 +3,9 @@ #include #include #include -#include -#include -#include -#include #include -#include #include #include -#include #include #include @@ -180,69 +174,10 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock( } -BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams(QueryPipeline & parent_pipeline) -{ - BlockInputStreams nested_streams; - - for (auto & interpreter : nested_interpreters) - { - BlockInputStreams streams = interpreter->executeWithMultipleStreams(parent_pipeline); - nested_streams.insert(nested_streams.end(), streams.begin(), streams.end()); - } - - /// Unify data structure. - if (nested_interpreters.size() > 1) - { - for (auto & stream : nested_streams) - stream = std::make_shared(stream, result_header,ConvertingBlockInputStream::MatchColumnsMode::Position); - parent_pipeline.addInterpreterContext(context); - } - - /// Update max_streams due to: - /// - max_distributed_connections for Distributed() engine - /// - max_streams_to_max_threads_ratio - /// - /// XXX: res.pipeline.getMaxThreads() cannot be used since it is capped to - /// number of streams, which is empty for non-Processors case. - max_streams = (*std::min_element(nested_interpreters.begin(), nested_interpreters.end(), [](const auto &a, const auto &b) - { - return a->getMaxStreams() < b->getMaxStreams(); - }))->getMaxStreams(); - - return nested_streams; -} - - BlockIO InterpreterSelectWithUnionQuery::execute() { BlockIO res; - BlockInputStreams nested_streams = executeWithMultipleStreams(res.pipeline); - BlockInputStreamPtr result_stream; - - if (nested_streams.empty()) - { - result_stream = std::make_shared(getSampleBlock()); - } - else if (nested_streams.size() == 1) - { - result_stream = nested_streams.front(); - nested_streams.clear(); - } - else - { - result_stream = std::make_shared(nested_streams, nullptr, max_streams); - nested_streams.clear(); - } - - res.in = result_stream; - res.pipeline.addInterpreterContext(context); - return res; -} - - -QueryPipeline InterpreterSelectWithUnionQuery::executeWithProcessors() -{ - QueryPipeline main_pipeline; + QueryPipeline & main_pipeline = res.pipeline; std::vector pipelines; bool has_main_pipeline = false; @@ -254,12 +189,12 @@ QueryPipeline InterpreterSelectWithUnionQuery::executeWithProcessors() if (!has_main_pipeline) { has_main_pipeline = true; - main_pipeline = interpreter->executeWithProcessors(); + main_pipeline = interpreter->execute().pipeline; headers.emplace_back(main_pipeline.getHeader()); } else { - pipelines.emplace_back(interpreter->executeWithProcessors()); + pipelines.emplace_back(interpreter->execute().pipeline); headers.emplace_back(pipelines.back().getHeader()); } } @@ -280,7 +215,7 @@ QueryPipeline InterpreterSelectWithUnionQuery::executeWithProcessors() main_pipeline.addInterpreterContext(context); - return main_pipeline; + return res; } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index c7a8e09578b..3b5fe533a84 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -29,12 +29,6 @@ public: BlockIO execute() override; - /// Execute the query without union of streams. - BlockInputStreams executeWithMultipleStreams(QueryPipeline & parent_pipeline); - - QueryPipeline executeWithProcessors() override; - bool canExecuteWithProcessors() const override { return true; } - bool ignoreLimits() const override { return options.ignore_limits; } bool ignoreQuota() const override { return options.ignore_quota; } diff --git a/src/Interpreters/InterpreterSetRoleQuery.cpp b/src/Interpreters/InterpreterSetRoleQuery.cpp index f8e0167d748..c627061dd51 100644 --- a/src/Interpreters/InterpreterSetRoleQuery.cpp +++ b/src/Interpreters/InterpreterSetRoleQuery.cpp @@ -62,7 +62,7 @@ void InterpreterSetRoleQuery::setRole(const ASTSetRoleQuery & query) void InterpreterSetRoleQuery::setDefaultRole(const ASTSetRoleQuery & query) { - context.checkAccess(AccessType::CREATE_USER | AccessType::DROP_USER); + context.checkAccess(AccessType::ALTER_USER); auto & access_control = context.getAccessControlManager(); std::vector to_users = ExtendedRoleSet{*query.to_users, access_control, context.getUserID()}.getMatchingIDs(access_control); diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 9938910cf1e..30005c7b169 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -50,7 +50,7 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() auto resolve_table_type = show_query->temporary ? Context::ResolveExternal : Context::ResolveOrdinary; auto table_id = context.resolveStorageID(*show_query, resolve_table_type); context.checkAccess(AccessType::SHOW_COLUMNS, table_id); - create_query = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(table_id.table_name); + create_query = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(table_id.table_name, context); } else if ((show_query = query_ptr->as())) { diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index f3d48651148..fedda7cab50 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -144,7 +144,7 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, auto access = context.getAccess(); for (auto & elem : DatabaseCatalog::instance().getDatabases()) { - for (auto iterator = elem.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!access->isGranted(log, getRequiredAccessType(action_type), elem.first, iterator->name())) continue; @@ -321,7 +321,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, context.checkAccess(AccessType::SYSTEM_RESTART_REPLICA, replica); auto table_ddl_guard = need_ddl_guard ? DatabaseCatalog::instance().getDDLGuard(replica.getDatabaseName(), replica.getTableName()) : nullptr; - auto [database, table] = DatabaseCatalog::instance().tryGetDatabaseAndTable(replica); + auto [database, table] = DatabaseCatalog::instance().tryGetDatabaseAndTable(replica, context); ASTPtr create_ast; /// Detach actions @@ -332,7 +332,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, { /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - create_ast = database->getCreateTableQuery(replica.table_name); + create_ast = database->getCreateTableQuery(replica.table_name, context); database->detachTable(replica.table_name); } @@ -369,7 +369,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context) for (auto & elem : catalog.getDatabases()) { DatabasePtr & database = elem.second; - for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (dynamic_cast(iterator->table().get())) replica_names.emplace_back(StorageID{database->getDatabaseName(), iterator->name()}); @@ -394,7 +394,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context) void InterpreterSystemQuery::syncReplica(ASTSystemQuery &) { context.checkAccess(AccessType::SYSTEM_SYNC_REPLICA, table_id); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); if (auto * storage_replicated = dynamic_cast(table.get())) { @@ -416,7 +416,7 @@ void InterpreterSystemQuery::flushDistributed(ASTSystemQuery &) { context.checkAccess(AccessType::SYSTEM_FLUSH_DISTRIBUTED, table_id); - if (auto * storage_distributed = dynamic_cast(DatabaseCatalog::instance().getTable(table_id).get())) + if (auto * storage_distributed = dynamic_cast(DatabaseCatalog::instance().getTable(table_id, context).get())) storage_distributed->flushClusterNodesAllData(); else throw Exception("Table " + table_id.getNameForLogs() + " is not distributed", ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index dc1ae6a7cad..489be488b4d 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -40,7 +40,7 @@ BlockIO InterpreterWatchQuery::execute() auto table_id = context.resolveStorageID(query, Context::ResolveOrdinary); /// Get storage - storage = DatabaseCatalog::instance().tryGetTable(table_id); + storage = DatabaseCatalog::instance().tryGetTable(table_id, context); if (!storage) throw Exception("Table " + table_id.getNameForLogs() + " doesn't exist.", diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 3bd883dcce8..7450890952a 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -181,7 +181,7 @@ StoragePtr JoinedTables::getLeftTableStorage() } /// Read from table. Even without table expression (implicit SELECT ... FROM system.one). - return DatabaseCatalog::instance().getTable(table_id); + return DatabaseCatalog::instance().getTable(table_id, context); } bool JoinedTables::resolveTables() @@ -261,7 +261,7 @@ std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & se if (table_to_join.database_and_table_name) { auto joined_table_id = context.resolveStorageID(table_to_join.database_and_table_name); - StoragePtr table = DatabaseCatalog::instance().tryGetTable(joined_table_id); + StoragePtr table = DatabaseCatalog::instance().tryGetTable(joined_table_id, context); if (table) { if (dynamic_cast(table.get()) || diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index e0f35925231..93bc87b5f5b 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -184,7 +184,7 @@ bool isStorageTouchedByMutations( /// For some reason it may copy context and and give it into ExpressionBlockInputStream /// after that we will use context from destroyed stack frame in our stream. InterpreterSelectQuery interpreter(select_query, context_copy, storage, SelectQueryOptions().ignoreLimits()); - BlockInputStreamPtr in = interpreter.execute().in; + BlockInputStreamPtr in = interpreter.execute().getInputStream(); Block block = in->read(); if (!block.rows()) @@ -687,7 +687,7 @@ void MutationsInterpreter::validate(TableStructureReadLockHolder &) } /// Do not use getSampleBlock in order to check the whole pipeline. - Block first_stage_header = select_interpreter->execute().in->getHeader(); + Block first_stage_header = select_interpreter->execute().getInputStream()->getHeader(); BlockInputStreamPtr in = std::make_shared(first_stage_header); addStreamsForLaterStages(stages, in)->getHeader(); } @@ -697,7 +697,7 @@ BlockInputStreamPtr MutationsInterpreter::execute(TableStructureReadLockHolder & if (!can_execute) throw Exception("Cannot execute mutations interpreter because can_execute flag set to false", ErrorCodes::LOGICAL_ERROR); - BlockInputStreamPtr in = select_interpreter->execute().in; + BlockInputStreamPtr in = select_interpreter->execute().getInputStream(); auto result_stream = addStreamsForLaterStages(stages, in); diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index cc3a4e3de34..b8c3920c3e4 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -139,7 +139,7 @@ bool PartLog::addNewParts(Context & current_context, const PartLog::MutableDataP } catch (...) { - tryLogCurrentException(part_log ? part_log->log : &Logger::get("PartLog"), __PRETTY_FUNCTION__); + tryLogCurrentException(part_log ? part_log->log : &Poco::Logger::get("PartLog"), __PRETTY_FUNCTION__); return false; } diff --git a/src/Interpreters/PredicateExpressionsOptimizer.cpp b/src/Interpreters/PredicateExpressionsOptimizer.cpp index d2f38b85b4b..b5d2c632135 100644 --- a/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -49,13 +49,6 @@ static ASTs splitConjunctionPredicate(const std::initializer_list { std::vector res; - auto remove_expression_at_index = [&res] (const size_t index) - { - if (index < res.size() - 1) - std::swap(res[index], res.back()); - res.pop_back(); - }; - for (const auto & predicate : predicates) { if (!predicate) @@ -65,14 +58,15 @@ static ASTs splitConjunctionPredicate(const std::initializer_list for (size_t idx = 0; idx < res.size();) { - const auto & expression = res.at(idx); + ASTPtr expression = res.at(idx); if (const auto * function = expression->as(); function && function->name == "and") { + res.erase(res.begin() + idx); + for (auto & child : function->arguments->children) res.emplace_back(child); - remove_expression_at_index(idx); continue; } ++idx; diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 9120f5959df..d86b5678f6d 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -84,7 +84,7 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as if (!is_unlimited_query && max_size && processes.size() >= max_size) { if (queue_max_wait_ms) - LOG_WARNING(&Logger::get("ProcessList"), "Too many simultaneous queries, will wait {} ms.", queue_max_wait_ms); + LOG_WARNING(&Poco::Logger::get("ProcessList"), "Too many simultaneous queries, will wait {} ms.", queue_max_wait_ms); if (!queue_max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(queue_max_wait_ms), [&]{ return processes.size() < max_size; })) throw Exception("Too many simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES); } @@ -231,7 +231,7 @@ ProcessListEntry::~ProcessListEntry() auto user_process_list_it = parent.user_to_queries.find(user); if (user_process_list_it == parent.user_to_queries.end()) { - LOG_ERROR(&Logger::get("ProcessList"), "Logical error: cannot find user in ProcessList"); + LOG_ERROR(&Poco::Logger::get("ProcessList"), "Logical error: cannot find user in ProcessList"); std::terminate(); } @@ -250,7 +250,7 @@ ProcessListEntry::~ProcessListEntry() if (!found) { - LOG_ERROR(&Logger::get("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser"); + LOG_ERROR(&Poco::Logger::get("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser"); std::terminate(); } parent.have_space.notify_all(); diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 848abc9aa8a..a4c8fd59245 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -31,7 +31,7 @@ public: /// store all set elements in explicit form. /// This is needed for subsequent use for index. Set(const SizeLimits & limits_, bool fill_set_elements_, bool transform_null_in_) - : log(&Logger::get("Set")), + : log(&Poco::Logger::get("Set")), limits(limits_), fill_set_elements(fill_set_elements_), transform_null_in(transform_null_in_) { } @@ -105,7 +105,7 @@ private: /// Types for set_elements. DataTypes set_elements_types; - Logger * log; + Poco::Logger * log; /// Limitations on the maximum size of the set SizeLimits limits; diff --git a/src/Interpreters/SubqueryForSet.cpp b/src/Interpreters/SubqueryForSet.cpp index 72831970de7..ac5c1e3d9eb 100644 --- a/src/Interpreters/SubqueryForSet.cpp +++ b/src/Interpreters/SubqueryForSet.cpp @@ -13,7 +13,7 @@ void SubqueryForSet::makeSource(std::shared_ptr { joined_block_aliases = std::move(joined_block_aliases_); source = std::make_shared(interpreter->getSampleBlock(), - [interpreter]() mutable { return interpreter->execute().in; }); + [interpreter]() mutable { return interpreter->execute().getInputStream(); }); sample_block = source->getHeader(); renameColumns(sample_block); diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index e7e65c4f19e..f1a72886901 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -41,7 +41,7 @@ std::shared_ptr createSystemLog( if (database != default_database_name) { /// System tables must be loaded before other tables, but loading order is undefined for all databases except `system` - LOG_ERROR(&Logger::get("SystemLog"), "Custom database name for a system table specified in config. Table `{}` will be created in `system` database instead of `{}`", table, database); + LOG_ERROR(&Poco::Logger::get("SystemLog"), "Custom database name for a system table specified in config. Table `{}` will be created in `system` database instead of `{}`", table, database); database = default_database_name; } @@ -76,12 +76,6 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi text_log = createSystemLog(global_context, "system", "text_log", config, "text_log"); metric_log = createSystemLog(global_context, "system", "metric_log", config, "metric_log"); - if (metric_log) - { - size_t collect_interval_milliseconds = config.getUInt64("metric_log.collect_interval_milliseconds"); - metric_log->startCollectMetric(collect_interval_milliseconds); - } - if (query_log) logs.emplace_back(query_log.get()); if (query_thread_log) @@ -112,6 +106,12 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi shutdown(); throw; } + + if (metric_log) + { + size_t collect_interval_milliseconds = config.getUInt64("metric_log.collect_interval_milliseconds"); + metric_log->startCollectMetric(collect_interval_milliseconds); + } } diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 3c0d2159ef9..dd2f815ce92 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -152,7 +152,7 @@ public: ASTPtr getCreateTableQuery() override; protected: - Logger * log; + Poco::Logger * log; private: /* Saving thread data */ @@ -207,7 +207,7 @@ SystemLog::SystemLog(Context & context_, , flush_interval_milliseconds(flush_interval_milliseconds_) { assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE); - log = &Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")"); + log = &Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")"); } @@ -431,7 +431,7 @@ void SystemLog::prepareTable() { String description = table_id.getNameForLogs(); - table = DatabaseCatalog::instance().tryGetTable(table_id); + table = DatabaseCatalog::instance().tryGetTable(table_id, context); if (table) { @@ -442,7 +442,7 @@ void SystemLog::prepareTable() { /// Rename the existing table. int suffix = 0; - while (DatabaseCatalog::instance().isTableExist({table_id.database_name, table_id.table_name + "_" + toString(suffix)})) + while (DatabaseCatalog::instance().isTableExist({table_id.database_name, table_id.table_name + "_" + toString(suffix)}, context)) ++suffix; auto rename = std::make_shared(); @@ -483,7 +483,7 @@ void SystemLog::prepareTable() interpreter.setInternal(true); interpreter.execute(); - table = DatabaseCatalog::instance().getTable(table_id); + table = DatabaseCatalog::instance().getTable(table_id, context); } is_prepared = true; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 2cc6730b90d..e1e2108c0fc 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -105,7 +105,7 @@ static void logQuery(const String & query, const Context & context, bool interna { if (internal) { - LOG_DEBUG(&Logger::get("executeQuery"), "(internal) {}", joinLines(query)); + LOG_DEBUG(&Poco::Logger::get("executeQuery"), "(internal) {}", joinLines(query)); } else { @@ -113,7 +113,7 @@ static void logQuery(const String & query, const Context & context, bool interna const auto & initial_query_id = context.getClientInfo().initial_query_id; const auto & current_user = context.getClientInfo().current_user; - LOG_DEBUG(&Logger::get("executeQuery"), "(from {}{}{}) {}", + LOG_DEBUG(&Poco::Logger::get("executeQuery"), "(from {}{}{}) {}", context.getClientInfo().current_address.toString(), (current_user != "default" ? ", user: " + context.getClientInfo().current_user : ""), (!initial_query_id.empty() && current_query_id != initial_query_id ? ", initial_query_id: " + initial_query_id : std::string()), @@ -145,10 +145,10 @@ static void setExceptionStackTrace(QueryLogElement & elem) static void logException(Context & context, QueryLogElement & elem) { if (elem.stack_trace.empty()) - LOG_ERROR(&Logger::get("executeQuery"), "{} (from {}) (in query: {})", + LOG_ERROR(&Poco::Logger::get("executeQuery"), "{} (from {}) (in query: {})", elem.exception, context.getClientInfo().current_address.toString(), joinLines(elem.query)); else - LOG_ERROR(&Logger::get("executeQuery"), "{} (from {}) (in query: {})" + LOG_ERROR(&Poco::Logger::get("executeQuery"), "{} (from {}) (in query: {})" ", Stack trace (when copying this message, always include the lines below):\n\n{}", elem.exception, context.getClientInfo().current_address.toString(), joinLines(elem.query), elem.stack_trace); } @@ -280,7 +280,6 @@ static std::tuple executeQueryImpl( /// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion. String query(begin, query_end); BlockIO res; - QueryPipeline & pipeline = res.pipeline; String query_for_logging; @@ -338,7 +337,6 @@ static std::tuple executeQueryImpl( context.resetInputCallbacks(); auto interpreter = InterpreterFactory::get(ast, context, stage); - bool use_processors = interpreter->canExecuteWithProcessors(); std::shared_ptr quota; if (!interpreter->ignoreQuota()) @@ -358,10 +356,9 @@ static std::tuple executeQueryImpl( limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); } - if (use_processors) - pipeline = interpreter->executeWithProcessors(); - else - res = interpreter->execute(); + res = interpreter->execute(); + QueryPipeline & pipeline = res.pipeline; + bool use_processors = pipeline.initialized(); if (res.pipeline.initialized()) use_processors = true; @@ -508,10 +505,10 @@ static std::tuple executeQueryImpl( if (elem.read_rows != 0) { - LOG_INFO(&Logger::get("executeQuery"), "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.", - elem.read_rows, formatReadableSizeWithBinarySuffix(elem.read_bytes), elapsed_seconds, + LOG_INFO(&Poco::Logger::get("executeQuery"), "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.", + elem.read_rows, ReadableSize(elem.read_bytes), elapsed_seconds, static_cast(elem.read_rows / elapsed_seconds), - formatReadableSizeWithBinarySuffix(elem.read_bytes / elapsed_seconds)); + ReadableSize(elem.read_bytes / elapsed_seconds)); } elem.thread_ids = std::move(info.thread_ids); @@ -588,7 +585,7 @@ static std::tuple executeQueryImpl( std::stringstream log_str; log_str << "Query pipeline:\n"; res.in->dumpTree(log_str); - LOG_DEBUG(&Logger::get("executeQuery"), log_str.str()); + LOG_DEBUG(&Poco::Logger::get("executeQuery"), log_str.str()); } } } diff --git a/src/Interpreters/getTableExpressions.cpp b/src/Interpreters/getTableExpressions.cpp index b5444f73b35..8467a98685d 100644 --- a/src/Interpreters/getTableExpressions.cpp +++ b/src/Interpreters/getTableExpressions.cpp @@ -96,7 +96,7 @@ static NamesAndTypesList getColumnsFromTableExpression(const ASTTableExpression else if (table_expression.database_and_table_name) { auto table_id = context.resolveStorageID(table_expression.database_and_table_name); - const auto & table = DatabaseCatalog::instance().getTable(table_id); + const auto & table = DatabaseCatalog::instance().getTable(table_id, context); const auto & columns = table->getColumns(); names_and_type_list = columns.getOrdinary(); materialized = columns.getMaterialized(); diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index e108db1af30..c94759897f5 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -96,7 +96,7 @@ std::shared_ptr interpretSubquery( else { auto table_id = context.resolveStorageID(table_expression); - const auto & storage = DatabaseCatalog::instance().getTable(table_id); + const auto & storage = DatabaseCatalog::instance().getTable(table_id, context); columns = storage->getColumns().getOrdinary(); select_query->replaceDatabaseAndTable(table_id); } diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index bf1f64f0a77..bf5701e1be4 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -48,7 +48,6 @@ static void loadDatabase( const String & database_path, bool force_restore_data) { - String database_attach_query; String database_metadata_file = database_path + ".sql"; diff --git a/src/Interpreters/tests/select_query.cpp b/src/Interpreters/tests/select_query.cpp index 8ce65ede05f..fb364d28086 100644 --- a/src/Interpreters/tests/select_query.cpp +++ b/src/Interpreters/tests/select_query.cpp @@ -24,8 +24,8 @@ int main(int, char **) try { Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); /// Pre-initialize the `DateLUT` so that the first initialization does not affect the measured execution speed. DateLUT::instance(); diff --git a/src/Parsers/ASTExtendedRoleSet.h b/src/Parsers/ASTExtendedRoleSet.h index 8d619e5d6a0..656f563bd9a 100644 --- a/src/Parsers/ASTExtendedRoleSet.h +++ b/src/Parsers/ASTExtendedRoleSet.h @@ -15,7 +15,10 @@ public: bool all = false; Strings except_names; bool except_current_user = false; - bool id_mode = false; /// If true then `names` and `except_names` keeps UUIDs, not names. + + bool id_mode = false; /// true if `names` and `except_names` keep UUIDs, not names. + bool can_contain_roles = true; /// true if this set can contain names of roles. + bool can_contain_users = true; /// true if this set can contain names of users. bool empty() const { return names.empty() && !current_user && !all; } void replaceCurrentUserTagWithName(const String & current_user_name); diff --git a/src/Parsers/ParserCreateUserQuery.cpp b/src/Parsers/ParserCreateUserQuery.cpp index 2f73e7fa75a..3bf7e508220 100644 --- a/src/Parsers/ParserCreateUserQuery.cpp +++ b/src/Parsers/ParserCreateUserQuery.cpp @@ -201,6 +201,7 @@ namespace return false; default_roles = typeid_cast>(ast); + default_roles->can_contain_users = false; return true; }); } diff --git a/src/Parsers/ParserSetRoleQuery.cpp b/src/Parsers/ParserSetRoleQuery.cpp index e6ff7893891..a69480f89eb 100644 --- a/src/Parsers/ParserSetRoleQuery.cpp +++ b/src/Parsers/ParserSetRoleQuery.cpp @@ -18,6 +18,7 @@ namespace return false; roles = typeid_cast>(ast); + roles->can_contain_users = false; return true; }); } @@ -34,6 +35,7 @@ namespace return false; to_users = typeid_cast>(ast); + to_users->can_contain_roles = false; return true; }); } diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index af061a373cc..dc59e0a2f5a 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -74,10 +74,6 @@ void PullingPipelineExecutor::cancel() /// Cancel execution if it wasn't finished. if (executor) executor->cancel(); - - /// Read all data and finish execution. - Chunk chunk; - while (pull(chunk)); } Chunk PullingPipelineExecutor::getTotals() diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index e2676eb64c2..12d4db1f4a8 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -149,7 +149,7 @@ Chunk IRowInputFormat::generate() { if (num_errors && (params.allow_errors_num > 0 || params.allow_errors_ratio > 0)) { - Logger * log = &Logger::get("IRowInputFormat"); + Poco::Logger * log = &Poco::Logger::get("IRowInputFormat"); LOG_TRACE(log, "Skipped {} rows with errors while reading the input stream", num_errors); } diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 78a291de262..8017667909b 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -612,7 +612,7 @@ private: try { Poco::URI url(base_url, "/schemas/ids/" + std::to_string(id)); - LOG_TRACE((&Logger::get("AvroConfluentRowInputFormat")), "Fetching schema id = {}", id); + LOG_TRACE((&Poco::Logger::get("AvroConfluentRowInputFormat")), "Fetching schema id = {}", id); /// One second for connect/send/receive. Just in case. ConnectionTimeouts timeouts({1, 0}, {1, 0}, {1, 0}); @@ -629,7 +629,7 @@ private: Poco::JSON::Parser parser; auto json_body = parser.parse(*response_body).extract(); auto schema = json_body->getValue("schema"); - LOG_TRACE((&Logger::get("AvroConfluentRowInputFormat")), "Succesfully fetched schema id = {}\n{}", id, schema); + LOG_TRACE((&Poco::Logger::get("AvroConfluentRowInputFormat")), "Succesfully fetched schema id = {}\n{}", id, schema); return avro::compileJsonSchemaFromString(schema); } catch (const Exception &) diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 8f1967d3704..94dfc3dc2c7 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -60,24 +60,25 @@ void MySQLOutputFormat::consume(Chunk chunk) void MySQLOutputFormat::finalize() { size_t affected_rows = 0; - std::stringstream human_readable_info; + std::string human_readable_info; if (QueryStatus * process_list_elem = context->getProcessListElement()) { CurrentThread::finalizePerformanceCounters(); QueryStatusInfo info = process_list_elem->getInfo(); affected_rows = info.written_rows; - human_readable_info << std::fixed << std::setprecision(3) - << "Read " << info.read_rows << " rows, " << formatReadableSizeWithBinarySuffix(info.read_bytes) << " in " << info.elapsed_seconds << " sec., " - << static_cast(info.read_rows / info.elapsed_seconds) << " rows/sec., " - << formatReadableSizeWithBinarySuffix(info.read_bytes / info.elapsed_seconds) << "/sec."; + human_readable_info = fmt::format( + "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.", + info.read_rows, ReadableSize(info.read_bytes), info.elapsed_seconds, + static_cast(info.read_rows / info.elapsed_seconds), + ReadableSize(info.read_bytes / info.elapsed_seconds)); } const auto & header = getPort(PortKind::Main).getHeader(); if (header.columns() == 0) - packet_sender->sendPacket(OK_Packet(0x0, context->mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info.str()), true); + packet_sender->sendPacket(OK_Packet(0x0, context->mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info), true); else if (context->mysql.client_capabilities & CLIENT_DEPRECATE_EOF) - packet_sender->sendPacket(OK_Packet(0xfe, context->mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info.str()), true); + packet_sender->sendPacket(OK_Packet(0xfe, context->mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info), true); else packet_sender->sendPacket(EOF_Packet(0, 0), true); } diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp index 05276f87540..49a3d018098 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp @@ -27,7 +27,7 @@ CollapsingSortedAlgorithm::CollapsingSortedAlgorithm( size_t max_block_size, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes, - Logger * log_) + Poco::Logger * log_) : IMergingAlgorithmWithSharedChunks(num_inputs, std::move(description_), out_row_sources_buf_, max_row_refs) , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) , sign_column_number(header.getPositionByName(sign_column)) diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h index 3cbe95d96e1..d95fac2f02b 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h @@ -35,7 +35,7 @@ public: size_t max_block_size, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes, - Logger * log_); + Poco::Logger * log_); Status merge() override; @@ -62,7 +62,7 @@ private: PODArray current_row_sources; /// Sources of rows with the current primary key size_t count_incorrect_data = 0; /// To prevent too many error messages from writing to the log. - Logger * log; + Poco::Logger * log; void reportIncorrectData(); void insertRow(RowRef & row); diff --git a/src/Processors/Merges/Algorithms/RowRef.h b/src/Processors/Merges/Algorithms/RowRef.h index efd04717e94..1b4da9781f8 100644 --- a/src/Processors/Merges/Algorithms/RowRef.h +++ b/src/Processors/Merges/Algorithms/RowRef.h @@ -83,7 +83,7 @@ public: { if (free_chunks.size() != chunks.size()) { - LOG_ERROR(&Logger::get("SharedChunkAllocator"), "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: {}", StackTrace().toString()); + LOG_ERROR(&Poco::Logger::get("SharedChunkAllocator"), "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: {}", StackTrace().toString()); return; } @@ -100,7 +100,7 @@ private: /// This may happen if allocator was removed before chunks. /// Log message and exit, because we don't want to throw exception in destructor. - LOG_ERROR(&Logger::get("SharedChunkAllocator"), "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: {}", StackTrace().toString()); + LOG_ERROR(&Poco::Logger::get("SharedChunkAllocator"), "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: {}", StackTrace().toString()); return; } diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index cdf7c4a1607..4e65504a101 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -29,7 +29,7 @@ public: max_block_size, out_row_sources_buf_, use_average_block_sizes, - &Logger::get("CollapsingSortedTransform")) + &Poco::Logger::get("CollapsingSortedTransform")) { } diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index defadd0caec..e680304ccee 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -43,7 +43,7 @@ void MergingSortedTransform::onFinish() const auto & merged_data = algorithm.getMergedData(); - auto * log = &Logger::get("MergingSortedTransform"); + auto * log = &Poco::Logger::get("MergingSortedTransform"); double seconds = total_stopwatch.elapsedSeconds(); @@ -53,7 +53,7 @@ void MergingSortedTransform::onFinish() LOG_DEBUG(log, "Merge sorted {} blocks, {} rows in {} sec., {} rows/sec., {}/sec", merged_data.totalChunks(), merged_data.totalMergedRows(), seconds, merged_data.totalMergedRows() / seconds, - formatReadableSizeWithBinarySuffix(merged_data.totalAllocatedBytes() / seconds)); + ReadableSize(merged_data.totalAllocatedBytes() / seconds)); } } diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index fc8fd575c35..d8eff53f567 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -542,9 +542,9 @@ void AggregatingTransform::initGenerate() size_t rows = variants.sizeWithoutOverflowRow(); LOG_TRACE(log, "Aggregated. {} to {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)", - src_rows, rows, formatReadableSizeWithBinarySuffix(src_bytes), + src_rows, rows, ReadableSize(src_bytes), elapsed_seconds, src_rows / elapsed_seconds, - formatReadableSizeWithBinarySuffix(src_bytes / elapsed_seconds)); + ReadableSize(src_bytes / elapsed_seconds)); if (params->aggregator.hasTemporaryFiles()) { @@ -593,7 +593,7 @@ void AggregatingTransform::initGenerate() for (const auto & file : files.files) processors.emplace_back(std::make_unique(header, file->path())); - LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), formatReadableSizeWithBinarySuffix(files.sum_size_compressed), formatReadableSizeWithBinarySuffix(files.sum_size_uncompressed)); + LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), ReadableSize(files.sum_size_compressed), ReadableSize(files.sum_size_uncompressed)); auto pipe = createMergingAggregatedMemoryEfficientPipe( header, params, files.files.size(), temporary_data_merge_threads); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index ebfb14a387a..c2693579c67 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -88,7 +88,7 @@ private: Processors processors; AggregatingTransformParamsPtr params; - Logger * log = &Logger::get("AggregatingTransform"); + Poco::Logger * log = &Poco::Logger::get("AggregatingTransform"); ColumnRawPtrs key_columns; Aggregator::AggregateColumns aggregate_columns; diff --git a/src/Processors/Transforms/ConvertingTransform.cpp b/src/Processors/Transforms/ConvertingTransform.cpp index c5bf5ad4d70..a0e9626b302 100644 --- a/src/Processors/Transforms/ConvertingTransform.cpp +++ b/src/Processors/Transforms/ConvertingTransform.cpp @@ -59,7 +59,11 @@ ConvertingTransform::ConvertingTransform( break; case MatchColumnsMode::Name: - if (source.has(res_elem.name)) + /// It may seem strange, but sometimes block may have columns with the same name. + /// For this specific case, try to get column from the same position if it has correct name first. + if (result_col_num < source.columns() && source.getByPosition(result_col_num).name == res_elem.name) + conversion[result_col_num] = result_col_num; + else if (source.has(res_elem.name)) conversion[result_col_num] = source.getPositionByName(res_elem.name); else throw Exception("Cannot find column " + backQuoteIfNeed(res_elem.name) + " in source stream", diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index aeb7a43b61b..f6df6042959 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -54,7 +54,7 @@ private: size_t bytes_to_transfer = 0; using Logger = Poco::Logger; - Logger * log = &Logger::get("CreatingSetsBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("CreatingSetsBlockInputStream"); bool is_initialized = false; diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index 34eb833c411..428fbb6d528 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -31,7 +31,7 @@ class MergeSorter; class BufferingToFileTransform : public IAccumulatingTransform { public: - BufferingToFileTransform(const Block & header, Logger * log_, std::string path_) + BufferingToFileTransform(const Block & header, Poco::Logger * log_, std::string path_) : IAccumulatingTransform(header, header), log(log_) , path(std::move(path_)), file_buf_out(path), compressed_buf_out(file_buf_out) , out_stream(std::make_shared(compressed_buf_out, 0, header)) @@ -80,7 +80,7 @@ public: } private: - Logger * log; + Poco::Logger * log; std::string path; WriteBufferFromFile file_buf_out; CompressedWriteBuffer compressed_buf_out; @@ -267,7 +267,7 @@ void MergeSortingTransform::remerge() new_chunks.emplace_back(std::move(chunk)); } - LOG_DEBUG(log, "Memory usage is lowered from {} to {}", formatReadableSizeWithBinarySuffix(sum_bytes_in_blocks), formatReadableSizeWithBinarySuffix(new_sum_bytes_in_blocks)); + LOG_DEBUG(log, "Memory usage is lowered from {} to {}", ReadableSize(sum_bytes_in_blocks), ReadableSize(new_sum_bytes_in_blocks)); /// If the memory consumption was not lowered enough - we will not perform remerge anymore. 2 is a guess. if (new_sum_bytes_in_blocks * 2 > sum_bytes_in_blocks) diff --git a/src/Processors/Transforms/MergeSortingTransform.h b/src/Processors/Transforms/MergeSortingTransform.h index 043cb3f36c1..22812e08b40 100644 --- a/src/Processors/Transforms/MergeSortingTransform.h +++ b/src/Processors/Transforms/MergeSortingTransform.h @@ -41,7 +41,7 @@ private: size_t sum_rows_in_blocks = 0; size_t sum_bytes_in_blocks = 0; - Logger * log = &Logger::get("MergeSortingTransform"); + Poco::Logger * log = &Poco::Logger::get("MergeSortingTransform"); /// If remerge doesn't save memory at least several times, mark it as useless and don't do it anymore. bool remerge_is_useful = true; diff --git a/src/Processors/Transforms/MergingAggregatedTransform.h b/src/Processors/Transforms/MergingAggregatedTransform.h index cb1ce01976c..73e0d8cd013 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.h +++ b/src/Processors/Transforms/MergingAggregatedTransform.h @@ -21,7 +21,7 @@ protected: private: AggregatingTransformParamsPtr params; - Logger * log = &Logger::get("MergingAggregatedTransform"); + Poco::Logger * log = &Poco::Logger::get("MergingAggregatedTransform"); size_t max_threads; AggregatedDataVariants data_variants; diff --git a/src/Processors/tests/processors_test_aggregation.cpp b/src/Processors/tests/processors_test_aggregation.cpp index e3316432ba8..9b8bee67d52 100644 --- a/src/Processors/tests/processors_test_aggregation.cpp +++ b/src/Processors/tests/processors_test_aggregation.cpp @@ -184,8 +184,8 @@ try auto thread_group = CurrentThread::getGroup(); Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); registerAggregateFunctions(); auto & factory = AggregateFunctionFactory::instance(); diff --git a/src/Processors/tests/processors_test_merge_sorting_transform.cpp b/src/Processors/tests/processors_test_merge_sorting_transform.cpp index 470bf79a174..5e6720f0167 100644 --- a/src/Processors/tests/processors_test_merge_sorting_transform.cpp +++ b/src/Processors/tests/processors_test_merge_sorting_transform.cpp @@ -125,8 +125,8 @@ int main(int, char **) try { Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); auto disk = std::make_shared("tmp", ".", 0); auto tmp_volume = std::make_shared("tmp", std::vector{disk}, 0); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index e5c784fd409..84d23f10a55 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -225,7 +225,7 @@ void HTTPHandler::pushDelayedResults(Output & used_output) HTTPHandler::HTTPHandler(IServer & server_, const std::string & name) : server(server_) - , log(&Logger::get(name)) + , log(&Poco::Logger::get(name)) { server_display_name = server.config().getString("display_name", getFQDNOrHostName()); } @@ -404,8 +404,6 @@ void HTTPHandler::processQuery( used_output.out_maybe_delayed_and_compressed = used_output.out_maybe_compressed; } - std::unique_ptr in_post_raw = std::make_unique(istr); - /// Request body can be compressed using algorithm specified in the Content-Encoding header. String http_request_compression_method_str = request.get("Content-Encoding", ""); std::unique_ptr in_post = wrapReadBufferWithCompressionMethod( diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index e916070be22..2f00aa0aa72 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes } HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_) - : log(&Logger::get(name_)), name(name_) + : log(&Poco::Logger::get(name_)), name(name_) { } diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index 55206018d41..273e337813e 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -18,7 +18,7 @@ class HTTPRequestHandlerFactoryMain : public Poco::Net::HTTPRequestHandlerFactor private: using TThis = HTTPRequestHandlerFactoryMain; - Logger * log; + Poco::Logger * log; std::string name; std::vector child_factories; diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index dabb0c12208..51b3d7eaef5 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -253,7 +253,7 @@ void MySQLHandler::comFieldList(ReadBuffer & payload) ComFieldList packet; packet.readPayload(payload); String database = connection_context.getCurrentDatabase(); - StoragePtr table_ptr = DatabaseCatalog::instance().getTable({database, packet.table}); + StoragePtr table_ptr = DatabaseCatalog::instance().getTable({database, packet.table}, connection_context); for (const NameAndTypePair & column: table_ptr->getColumns().getAll()) { ColumnDefinition column_definition( diff --git a/src/Server/MySQLHandlerFactory.cpp b/src/Server/MySQLHandlerFactory.cpp index 5d78ed81068..18e05f88402 100644 --- a/src/Server/MySQLHandlerFactory.cpp +++ b/src/Server/MySQLHandlerFactory.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes MySQLHandlerFactory::MySQLHandlerFactory(IServer & server_) : server(server_) - , log(&Logger::get("MySQLHandlerFactory")) + , log(&Poco::Logger::get("MySQLHandlerFactory")) { #if USE_SSL try diff --git a/src/Server/ReplicasStatusHandler.cpp b/src/Server/ReplicasStatusHandler.cpp index f2d1ffe2ee5..55746409feb 100644 --- a/src/Server/ReplicasStatusHandler.cpp +++ b/src/Server/ReplicasStatusHandler.cpp @@ -44,7 +44,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request if (db.second->getEngineName() == "Lazy") continue; - for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { const auto & table = iterator->table(); StorageReplicatedMergeTree * table_replicated = dynamic_cast(table.get()); @@ -96,7 +96,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request } catch (...) { - LOG_ERROR((&Logger::get("ReplicasStatusHandler")), "Cannot send exception to client"); + LOG_ERROR((&Poco::Logger::get("ReplicasStatusHandler")), "Cannot send exception to client"); } } } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b8f3660179d..6e9275540e5 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -474,7 +474,7 @@ void TCPHandler::processInsertQuery(const Settings & connection_settings) if (query_context->getSettingsRef().input_format_defaults_for_omitted_fields) { if (!table_id.empty()) - sendTableColumns(DatabaseCatalog::instance().getTable(table_id)->getColumns()); + sendTableColumns(DatabaseCatalog::instance().getTable(table_id, *query_context)->getColumns()); } } @@ -627,7 +627,7 @@ void TCPHandler::processTablesStatusRequest() for (const QualifiedTableName & table_name: request.tables) { auto resolved_id = connection_context.tryResolveStorageID({table_name.database, table_name.table}); - StoragePtr table = DatabaseCatalog::instance().tryGetTable(resolved_id); + StoragePtr table = DatabaseCatalog::instance().tryGetTable(resolved_id, connection_context); if (!table) continue; @@ -944,11 +944,11 @@ bool TCPHandler::receiveData(bool scalar) StoragePtr storage; /// If such a table does not exist, create it. if (resolved) - storage = DatabaseCatalog::instance().getTable(resolved); + storage = DatabaseCatalog::instance().getTable(resolved, *query_context); else { NamesAndTypesList columns = block.getNamesAndTypesList(); - auto temporary_table = TemporaryTableHolder(*query_context, ColumnsDescription{columns}); + auto temporary_table = TemporaryTableHolder(*query_context, ColumnsDescription{columns}, {}); storage = temporary_table.getTable(); query_context->addExternalTable(temporary_id.table_name, std::move(temporary_table)); } diff --git a/src/Server/TCPHandlerFactory.h b/src/Server/TCPHandlerFactory.h index a5532a8dc02..5ecd427bf8b 100644 --- a/src/Server/TCPHandlerFactory.h +++ b/src/Server/TCPHandlerFactory.h @@ -27,7 +27,7 @@ private: public: explicit TCPHandlerFactory(IServer & server_, bool secure_ = false) : server(server_) - , log(&Logger::get(std::string("TCP") + (secure_ ? "S" : "") + "HandlerFactory")) + , log(&Poco::Logger::get(std::string("TCP") + (secure_ ? "S" : "") + "HandlerFactory")) { } diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 4dd62db0965..0e84f68e5fe 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -88,7 +88,7 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( , default_sleep_time{storage.global_context->getSettingsRef().distributed_directory_monitor_sleep_time_ms.totalMilliseconds()} , sleep_time{default_sleep_time} , max_sleep_time{storage.global_context->getSettingsRef().distributed_directory_monitor_max_sleep_time_ms.totalMilliseconds()} - , log{&Logger::get(getLoggerName())} + , log{&Poco::Logger::get(getLoggerName())} , monitor_blocker(monitor_blocker_) , bg_pool(bg_pool_) { @@ -301,7 +301,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa } void StorageDistributedDirectoryMonitor::readHeader( - ReadBuffer & in, Settings & insert_settings, std::string & insert_query, ClientInfo & client_info, Logger * log) + ReadBuffer & in, Settings & insert_settings, std::string & insert_query, ClientInfo & client_info, Poco::Logger * log) { UInt64 query_size; readVarUInt(query_size, in); @@ -542,7 +542,7 @@ public: : in(file_name) , decompressing_in(in) , block_in(decompressing_in, ClickHouseRevision::get()) - , log{&Logger::get("DirectoryMonitorBlockInputStream")} + , log{&Poco::Logger::get("DirectoryMonitorBlockInputStream")} { Settings insert_settings; String insert_query; @@ -576,7 +576,7 @@ private: Block first_block; Block header; - Logger * log; + Poco::Logger * log; }; BlockInputStreamPtr StorageDistributedDirectoryMonitor::createStreamFromFile(const String & file_name) diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index e2a913ee1ef..418cd430243 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -68,14 +68,14 @@ private: std::chrono::time_point last_decrease_time {std::chrono::system_clock::now()}; std::atomic quit {false}; std::mutex mutex; - Logger * log; + Poco::Logger * log; ActionBlocker & monitor_blocker; BackgroundSchedulePool & bg_pool; BackgroundSchedulePoolTaskHolder task_handle; /// Read insert query and insert settings for backward compatible. - static void readHeader(ReadBuffer & in, Settings & insert_settings, std::string & insert_query, ClientInfo & client_info, Logger * log); + static void readHeader(ReadBuffer & in, Settings & insert_settings, std::string & insert_query, ClientInfo & client_info, Poco::Logger * log); friend class DirectoryMonitorBlockInputStream; }; diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index a171e289695..5516e85b143 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -87,7 +87,7 @@ DistributedBlockOutputStream::DistributedBlockOutputStream( bool insert_sync_, UInt64 insert_timeout_) : context(context_), storage(storage_), query_ast(query_ast_), query_string(queryToString(query_ast_)), cluster(cluster_), insert_sync(insert_sync_), - insert_timeout(insert_timeout_), log(&Logger::get("DistributedBlockOutputStream")) + insert_timeout(insert_timeout_), log(&Poco::Logger::get("DistributedBlockOutputStream")) { } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 34b479d1dea..73674d8210c 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -403,25 +403,6 @@ void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settin } } -BlockInputStreams IStorage::readStreams( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) -{ - ForceTreeShapedPipeline enable_tree_shape(query_info); - auto pipes = read(column_names, query_info, context, processed_stage, max_block_size, num_streams); - - BlockInputStreams res; - res.reserve(pipes.size()); - - for (auto & pipe : pipes) - res.emplace_back(std::make_shared(std::move(pipe))); - - return res; -} StorageID IStorage::getStorageID() const { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 87b84aebb20..38d81b0dc6f 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -308,16 +308,6 @@ public: throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** The same as read, but returns BlockInputStreams. - */ - BlockInputStreams readStreams( - const Names & /*column_names*/, - const SelectQueryInfo & /*query_info*/, - const Context & /*context*/, - QueryProcessingStage::Enum /*processed_stage*/, - size_t /*max_block_size*/, - unsigned /*num_streams*/); - /** Writes the data to a table. * Receives a description of the query, which can contain information about the data write method. * Returns an object by which you can write data sequentially. diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 93983fb6080..43984f81e05 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -19,6 +19,7 @@ struct KafkaSettings : public SettingsCollection M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.", 0) \ M(SettingString, kafka_topic_list, "", "A list of Kafka topics.", 0) \ M(SettingString, kafka_group_name, "", "A group of Kafka consumers.", 0) \ + M(SettingString, kafka_client_id, "", "A client id of Kafka consumer.", 0) \ M(SettingString, kafka_format, "", "The message format for Kafka engine.", 0) \ M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.", 0) \ M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \ diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 7731cf3c06a..0fc42e49582 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -34,6 +34,7 @@ #include #include #include +#include namespace DB @@ -118,6 +119,7 @@ StorageKafka::StorageKafka( const ColumnsDescription & columns_, const String & brokers_, const String & group_, + const String & client_id_, const Names & topics_, const String & format_name_, char row_delimiter_, @@ -132,12 +134,13 @@ StorageKafka::StorageKafka( , topics(global_context.getMacros()->expand(topics_)) , brokers(global_context.getMacros()->expand(brokers_)) , group(global_context.getMacros()->expand(group_)) + , client_id(client_id_.empty() ? getDefaultClientId(table_id_) : global_context.getMacros()->expand(client_id_)) , format_name(global_context.getMacros()->expand(format_name_)) , row_delimiter(row_delimiter_) , schema_name(global_context.getMacros()->expand(schema_name_)) , num_consumers(num_consumers_) , max_block_size(max_block_size_) - , log(&Logger::get("StorageKafka (" + table_id_.table_name + ")")) + , log(&Poco::Logger::get("StorageKafka (" + table_id_.table_name + ")")) , semaphore(0, num_consumers_) , skip_broken(skip_broken_) , intermediate_commit(intermediate_commit_) @@ -149,6 +152,13 @@ StorageKafka::StorageKafka( task->deactivate(); } +String StorageKafka::getDefaultClientId(const StorageID & table_id_) +{ + std::stringstream ss; + ss << VERSION_NAME << "-" << getFQDNOrHostName() << "-" << table_id_.database_name << "-" << table_id_.table_name; + return ss.str(); +} + Pipes StorageKafka::read( const Names & column_names, @@ -194,7 +204,7 @@ void StorageKafka::startup() { try { - pushReadBuffer(createReadBuffer()); + pushReadBuffer(createReadBuffer(i)); ++num_created_consumers; } catch (const cppkafka::Exception &) @@ -262,7 +272,7 @@ ProducerBufferPtr StorageKafka::createWriteBuffer(const Block & header) cppkafka::Configuration conf; conf.set("metadata.broker.list", brokers); conf.set("group.id", group); - conf.set("client.id", VERSION_FULL); + conf.set("client.id", client_id); // TODO: fill required settings updateConfiguration(conf); @@ -275,13 +285,22 @@ ProducerBufferPtr StorageKafka::createWriteBuffer(const Block & header) } -ConsumerBufferPtr StorageKafka::createReadBuffer() +ConsumerBufferPtr StorageKafka::createReadBuffer(const size_t consumer_number) { cppkafka::Configuration conf; conf.set("metadata.broker.list", brokers); conf.set("group.id", group); - conf.set("client.id", VERSION_FULL); + if (num_consumers > 1) + { + std::stringstream ss; + ss << client_id << "-" << consumer_number; + conf.set("client.id", ss.str()); + } + else + { + conf.set("client.id", client_id); + } conf.set("auto.offset.reset", "smallest"); // If no offset stored for this group, read all messages from the start @@ -324,10 +343,10 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & conf) } // No need to add any prefix, messages can be distinguished - conf.set_log_callback([this](cppkafka::KafkaHandleBase &, int level, const std::string & /* facility */, const std::string & message) + conf.set_log_callback([this](cppkafka::KafkaHandleBase &, int level, const std::string & facility, const std::string & message) { auto [poco_level, client_logs_level] = parseSyslogLevel(level); - LOG_IMPL(log, client_logs_level, poco_level, message); + LOG_IMPL(log, client_logs_level, poco_level, "[rdk:{}] {}", facility, message); }); // Configure interceptor to change thread name @@ -361,7 +380,7 @@ bool StorageKafka::checkDependencies(const StorageID & table_id) // Check the dependencies are ready? for (const auto & db_tab : dependencies) { - auto table = DatabaseCatalog::instance().tryGetTable(db_tab); + auto table = DatabaseCatalog::instance().tryGetTable(db_tab, global_context); if (!table) return false; @@ -429,7 +448,7 @@ void StorageKafka::threadFunc() bool StorageKafka::streamToViews() { auto table_id = getStorageID(); - auto table = DatabaseCatalog::instance().getTable(table_id); + auto table = DatabaseCatalog::instance().getTable(table_id, global_context); if (!table) throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); @@ -503,6 +522,7 @@ void registerStorageKafka(StorageFactory & factory) * - Kafka broker list * - List of topics * - Group ID (may be a constaint expression with a string result) + * - Client ID * - Message format (string) * - Row delimiter * - Schema (optional, if the format supports it) @@ -709,9 +729,12 @@ void registerStorageKafka(StorageFactory & factory) } } + // Get and check client id + String client_id = kafka_settings.kafka_client_id.value; + return StorageKafka::create( args.table_id, args.context, args.columns, - brokers, group, topics, format, row_delimiter, schema, num_consumers, max_block_size, skip_broken, intermediate_commit); + brokers, group, client_id, topics, format, row_delimiter, schema, num_consumers, max_block_size, skip_broken, intermediate_commit); }; factory.registerStorage("Kafka", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index c813ed0033d..1ea7d6dcad7 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -67,6 +67,7 @@ protected: const ColumnsDescription & columns_, const String & brokers_, const String & group_, + const String & client_id_, const Names & topics_, const String & format_name_, char row_delimiter_, @@ -83,6 +84,7 @@ private: Names topics; const String brokers; const String group; + const String client_id; const String format_name; char row_delimiter; /// optional row delimiter for generating char delimited stream in order to make various input stream parsers happy. const String schema_name; @@ -108,12 +110,13 @@ private: BackgroundSchedulePool::TaskHolder task; std::atomic stream_cancelled{false}; - ConsumerBufferPtr createReadBuffer(); + ConsumerBufferPtr createReadBuffer(const size_t consumer_number); // Update Kafka configuration with values from CH user configuration. void updateConfiguration(cppkafka::Configuration & conf); void threadFunc(); + static String getDefaultClientId(const StorageID & table_id_); bool streamToViews(); bool checkDependencies(const StorageID & table_id); }; diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index cd660407c89..3ee8c8409b8 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -111,7 +111,7 @@ MergeableBlocksPtr StorageLiveView::collectMergeableBlocks(const Context & conte InterpreterSelectQuery interpreter(mergeable_query->clone(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); - auto view_mergeable_stream = std::make_shared(interpreter.execute().in); + auto view_mergeable_stream = std::make_shared(interpreter.execute().getInputStream()); while (Block this_block = view_mergeable_stream->read()) base_blocks->push_back(this_block); @@ -148,7 +148,7 @@ BlockInputStreamPtr StorageLiveView::completeQuery(Pipes pipes) block_context->addExternalTable(getBlocksTableName(), TemporaryTableHolder(global_context, creator)); InterpreterSelectQuery select(getInnerBlocksQuery(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete)); - BlockInputStreamPtr data = std::make_shared(select.execute().in); + BlockInputStreamPtr data = std::make_shared(select.execute().getInputStream()); /// Squashing is needed here because the view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY @@ -218,7 +218,7 @@ void StorageLiveView::writeIntoLiveView( QueryProcessingStage::WithMergeableState); auto data_mergeable_stream = std::make_shared( - select_block.execute().in); + select_block.execute().getInputStream()); while (Block this_block = data_mergeable_stream->read()) new_mergeable_blocks->push_back(this_block); @@ -401,7 +401,7 @@ void StorageLiveView::noUsersThread(std::shared_ptr storage, co if (drop_table) { - if (DatabaseCatalog::instance().tryGetTable(table_id)) + if (DatabaseCatalog::instance().tryGetTable(table_id, storage->global_context)) { try { diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 801b0b42ec4..fe62de224da 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -53,7 +53,7 @@ public: { return getStorageID().table_name + "_blocks"; } - StoragePtr getParentStorage() const { return DatabaseCatalog::instance().getTable(select_table_id); } + StoragePtr getParentStorage() const { return DatabaseCatalog::instance().getTable(select_table_id, global_context); } ASTPtr getInnerQuery() const { return inner_query->clone(); } ASTPtr getInnerSubQuery() const diff --git a/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/src/Storages/MergeTree/BackgroundProcessingPool.cpp index ff195112929..8f6d7c19549 100644 --- a/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -51,7 +51,7 @@ BackgroundProcessingPool::BackgroundProcessingPool(int size_, , thread_name(thread_name_) , settings(pool_settings) { - logger = &Logger::get(log_name); + logger = &Poco::Logger::get(log_name); LOG_INFO(logger, "Create {} with {} threads", log_name, size); threads.resize(size); diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index c0e8c0d2331..c1aff6bdba5 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -21,7 +21,7 @@ class Service final : public InterserverIOEndpoint { public: Service(MergeTreeData & data_) - : data(data_), log(&Logger::get(data.getLogName() + " (Replicated PartsService)")) {} + : data(data_), log(&Poco::Logger::get(data.getLogName() + " (Replicated PartsService)")) {} Service(const Service &) = delete; Service & operator=(const Service &) = delete; @@ -36,7 +36,7 @@ private: /// StorageReplicatedMergeTree::shutdown() waits for all parts exchange handlers to finish, /// so Service will never access dangling reference to storage MergeTreeData & data; - Logger * log; + Poco::Logger * log; }; /** Client for getting the parts from the table *MergeTree. @@ -44,7 +44,7 @@ private: class Fetcher final { public: - Fetcher(MergeTreeData & data_) : data(data_), log(&Logger::get("Fetcher")) {} + Fetcher(MergeTreeData & data_) : data(data_), log(&Poco::Logger::get("Fetcher")) {} Fetcher(const Fetcher &) = delete; Fetcher & operator=(const Fetcher &) = delete; @@ -75,7 +75,7 @@ private: PooledReadWriteBufferFromHTTP & in); MergeTreeData & data; - Logger * log; + Poco::Logger * log; }; } diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 758401b7f4f..762dbc7d5b6 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -107,7 +107,7 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions( int rc = zookeeper.tryMulti(lock_ops, lock_responses); if (rc == Coordination::ZBADVERSION) { - LOG_TRACE(&Logger::get("EphemeralLocksInAllPartitions"), "Someone has inserted a block in a new partition while we were creating locks. Retry."); + LOG_TRACE(&Poco::Logger::get("EphemeralLocksInAllPartitions"), "Someone has inserted a block in a new partition while we were creating locks. Retry."); continue; } else if (rc != Coordination::ZOK) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 21768644940..287bf916c19 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -775,6 +775,7 @@ void IMergeTreeDataPart::remove() const } } + String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) const { /// Do not allow underscores in the prefix because they are used as separators. diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 6ae22885dfd..ee381709dd4 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -181,8 +181,11 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "empty", - [] (RPNElement & out, const Field &) + [] (RPNElement & out, const Field & value) { + if (value.getType() != Field::Types::String) + return false; + out.function = RPNElement::FUNCTION_IN_RANGE; out.range = Range(""); return true; @@ -190,8 +193,11 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "notEmpty", - [] (RPNElement & out, const Field &) + [] (RPNElement & out, const Field & value) { + if (value.getType() != Field::Types::String) + return false; + out.function = RPNElement::FUNCTION_NOT_IN_RANGE; out.range = Range(""); return true; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7b60369ef25..c7ab39d3e04 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -136,7 +136,7 @@ MergeTreeData::MergeTreeData( , relative_data_path(relative_data_path_) , broken_part_callback(broken_part_callback_) , log_name(table_id_.getNameForLogs()) - , log(&Logger::get(log_name)) + , log(&Poco::Logger::get(log_name)) , storage_settings(std::move(storage_settings_)) , data_parts_by_info(data_parts_indexes.get()) , data_parts_by_state_and_info(data_parts_indexes.get()) @@ -2810,8 +2810,7 @@ inline ReservationPtr checkAndReturnReservation(UInt64 expected_size, Reservatio if (reservation) return reservation; - throw Exception("Cannot reserve " + formatReadableSizeWithBinarySuffix(expected_size) + ", not enough space", - ErrorCodes::NOT_ENOUGH_SPACE); + throw Exception(fmt::format("Cannot reserve {}, not enough space", ReadableSize(expected_size)), ErrorCodes::NOT_ENOUGH_SPACE); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 96fcaa7d49a..13d748b5e30 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -643,6 +643,7 @@ public: Int64 minmax_idx_date_column_pos = -1; /// In a common case minmax index includes a date column. Int64 minmax_idx_time_column_pos = -1; /// In other cases, minmax index often includes a dateTime column. + /// TODO (alesap) generate this expressions on fly ExpressionActionsPtr primary_key_and_skip_indices_expr; ExpressionActionsPtr sorting_key_and_skip_indices_expr; @@ -693,7 +694,7 @@ protected: BrokenPartCallback broken_part_callback; String log_name; - Logger * log; + Poco::Logger * log; /// Storage settings. /// Use get and set to receive readonly versions. diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index fca3f824953..f41d52109a6 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -153,7 +153,7 @@ void FutureMergedMutatedPart::updatePath(const MergeTreeData & storage, const Re } MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, size_t background_pool_size_) - : data(data_), background_pool_size(background_pool_size_), log(&Logger::get(data.getLogName() + " (MergerMutator)")) + : data(data_), background_pool_size(background_pool_size_), log(&Poco::Logger::get(data.getLogName() + " (MergerMutator)")) { } @@ -371,14 +371,13 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( ", {} required now (+{}% on overhead); suppressing similar warnings for the next hour", parts.front()->name, (*prev_it)->name, - formatReadableSizeWithBinarySuffix(available_disk_space), - formatReadableSizeWithBinarySuffix(sum_bytes), + ReadableSize(available_disk_space), + ReadableSize(sum_bytes), static_cast((DISK_USAGE_COEFFICIENT_TO_SELECT - 1.0) * 100)); } if (out_disable_reason) - *out_disable_reason = "Insufficient available disk space, required " + - formatReadableSizeWithDecimalSuffix(required_disk_space); + *out_disable_reason = fmt::format("Insufficient available disk space, required {}", ReadableSize(required_disk_space)); return false; } @@ -715,13 +714,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor Pipe pipe(std::move(input)); - if (data.hasPrimaryKey() || data.hasIndices()) + if (data.hasSortingKey()) { - auto expr = std::make_shared(pipe.getHeader(), data.sorting_key_and_skip_indices_expr); + auto expr = std::make_shared(pipe.getHeader(), data.getSortingKey().expression); pipe.addSimpleTransform(std::move(expr)); - - auto materializing = std::make_shared(pipe.getHeader()); - pipe.addSimpleTransform(std::move(materializing)); } pipes.emplace_back(std::move(pipe)); @@ -797,6 +793,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merged_stream = std::make_shared(merged_stream, data, new_data_part, time_of_merge, force_ttl); const auto & index_factory = MergeTreeIndexFactory::instance(); + if (data.hasIndices()) + { + merged_stream = std::make_shared(merged_stream, data.primary_key_and_skip_indices_expr); + merged_stream = std::make_shared(merged_stream); + } + MergedBlockOutputStream to{ new_data_part, merging_columns, @@ -950,7 +952,15 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor /// Print overall profiling info. NOTE: it may duplicates previous messages { double elapsed_seconds = merge_entry->watch.elapsedSeconds(); - LOG_DEBUG(log, "Merge sorted {} rows, containing {} columns ({} merged, {} gathered) in {} sec., {} rows/sec., {}/sec.", merge_entry->rows_read, all_column_names.size(), merging_column_names.size(), gathering_column_names.size(), elapsed_seconds, merge_entry->rows_read / elapsed_seconds, formatReadableSizeWithBinarySuffix(merge_entry->bytes_read_uncompressed / elapsed_seconds)); + LOG_DEBUG(log, + "Merge sorted {} rows, containing {} columns ({} merged, {} gathered) in {} sec., {} rows/sec., {}/sec.", + merge_entry->rows_read, + all_column_names.size(), + merging_column_names.size(), + gathering_column_names.size(), + elapsed_seconds, + merge_entry->rows_read / elapsed_seconds, + ReadableSize(merge_entry->bytes_read_uncompressed / elapsed_seconds)); } if (merge_alg != MergeAlgorithm::Vertical) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 1ea6a72952c..385ada72fdd 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -229,7 +229,7 @@ private: MergeTreeData & data; const size_t background_pool_size; - Logger * log; + Poco::Logger * log; /// When the last time you wrote to the log that the disk space was running out (not to write about this too often). time_t disk_space_warning_time = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 8f3aa451a10..8a7fabc4203 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -84,7 +84,7 @@ namespace ErrorCodes MergeTreeDataSelectExecutor::MergeTreeDataSelectExecutor(const MergeTreeData & data_) - : data(data_), log(&Logger::get(data.getLogName() + " (SelectExecutor)")) + : data(data_), log(&Poco::Logger::get(data.getLogName() + " (SelectExecutor)")) { } @@ -1153,7 +1153,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( if (num_streams > settings.max_final_threads) num_streams = settings.max_final_threads; - if (num_streams <= 1 || sort_description.empty() || query_info.force_tree_shaped_pipeline) + if (num_streams <= 1 || sort_description.empty()) { Pipe pipe(std::move(pipes), get_merging_processor()); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 7d632e30f83..85d69ead181 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -44,7 +44,7 @@ public: private: const MergeTreeData & data; - Logger * log; + Poco::Logger * log; Pipes spreadMarkRangesAmongStreams( RangesInDataParts && parts, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index e199aa3b43a..ffaa227641e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -34,7 +34,7 @@ using BlocksWithPartition = std::vector; class MergeTreeDataWriter { public: - MergeTreeDataWriter(MergeTreeData & data_) : data(data_), log(&Logger::get(data.getLogName() + " (Writer)")) {} + MergeTreeDataWriter(MergeTreeData & data_) : data(data_), log(&Poco::Logger::get(data.getLogName() + " (Writer)")) {} /** Split the block to blocks, each of them must be written as separate part. * (split rows by partition) @@ -50,7 +50,7 @@ public: private: MergeTreeData & data; - Logger * log; + Poco::Logger * log; }; } diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 4e564b512d6..e84ff418bc3 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -179,7 +179,7 @@ bool MergeTreePartsMover::selectPartsForMove( if (!parts_to_move.empty()) { - LOG_TRACE(log, "Selected {} parts to move according to storage policy rules and {} parts according to TTL rules, {} total", parts_to_move_by_policy_rules, parts_to_move_by_ttl_rules, formatReadableSizeWithBinarySuffix(parts_to_move_total_size_bytes)); + LOG_TRACE(log, "Selected {} parts to move according to storage policy rules and {} parts according to TTL rules, {} total", parts_to_move_by_policy_rules, parts_to_move_by_ttl_rules, ReadableSize(parts_to_move_total_size_bytes)); return true; } else diff --git a/src/Storages/MergeTree/MergeTreePartsMover.h b/src/Storages/MergeTree/MergeTreePartsMover.h index 95a20dc1f77..8ddf5e165af 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.h +++ b/src/Storages/MergeTree/MergeTreePartsMover.h @@ -66,7 +66,7 @@ public: private: MergeTreeData * data; - Logger * log; + Poco::Logger * log; }; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index b266b65e19e..9ca1446ef64 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -169,7 +169,7 @@ void MergeTreeReadPool::profileFeedback(const ReadBufferFromFileBase::ProfileInf ProfileEvents::increment(ProfileEvents::SlowRead); LOG_DEBUG(log, "Slow read, event №{}: read {} bytes in {} sec., {}/s.", backoff_state.num_events, info.bytes_read, info.nanoseconds / 1e9, - formatReadableSizeWithBinarySuffix(throughput)); + ReadableSize(throughput)); if (backoff_state.num_events < backoff_settings.min_events) return; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index f639a6a4905..c43074f1962 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -133,7 +133,7 @@ private: mutable std::mutex mutex; - Logger * log = &Logger::get("MergeTreeReadPool"); + Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPool"); }; using MergeTreeReadPoolPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h index 211bf9701cf..ea603bd468f 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h @@ -69,7 +69,7 @@ private: Chunks chunks; - Logger * log = &Logger::get("MergeTreeReverseSelectProcessor"); + Poco::Logger * log = &Poco::Logger::get("MergeTreeReverseSelectProcessor"); }; } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 4c64bfb6a18..d2438e20192 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -67,7 +67,7 @@ private: String path; bool is_first_task = true; - Logger * log = &Logger::get("MergeTreeSelectProcessor"); + Poco::Logger * log = &Poco::Logger::get("MergeTreeSelectProcessor"); }; } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index dac559913aa..6155fef200a 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -45,7 +45,7 @@ private: /// Should read using direct IO bool read_with_direct_io; - Logger * log = &Logger::get("MergeTreeSequentialSource"); + Poco::Logger * log = &Poco::Logger::get("MergeTreeSequentialSource"); std::shared_ptr mark_cache; using MergeTreeReaderPtr = std::unique_ptr; diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index a8da0e8615c..61f99ac6d88 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -32,7 +32,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( const Context & context, const MergeTreeData & data, const Names & queried_columns_, - Logger * log_) + Poco::Logger * log_) : table_columns{ext::map(data.getColumns().getAllPhysical(), [] (const NameAndTypePair & col) { return col.name; })}, queried_columns{queried_columns_}, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index d5e44166536..d8c1103809f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -33,7 +33,7 @@ namespace ErrorCodes ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream( StorageReplicatedMergeTree & storage_, size_t quorum_, size_t quorum_timeout_ms_, size_t max_parts_per_block_, bool deduplicate_) : storage(storage_), quorum(quorum_), quorum_timeout_ms(quorum_timeout_ms_), max_parts_per_block(max_parts_per_block_), deduplicate(deduplicate_), - log(&Logger::get(storage.getLogName() + " (Replicated OutputStream)")) + log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) { /// The quorum value `1` has the same meaning as if it is disabled. if (quorum == 1) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index 0f6fc1e7cee..b8650c25c7d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -63,7 +63,7 @@ private: bool last_block_is_duplicate = false; using Logger = Poco::Logger; - Logger * log; + Poco::Logger * log; }; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 5fc2e2e3417..de91a5d5940 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplicatedMergeTree & storage_) : storage(storage_) , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeCleanupThread)") - , log(&Logger::get(log_name)) + , log(&Poco::Logger::get(log_name)) { task = storage.global_context.getSchedulePool().createTask(log_name, [this]{ run(); }); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h index 306ee29a5de..a787f99d907 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h @@ -34,7 +34,7 @@ public: private: StorageReplicatedMergeTree & storage; String log_name; - Logger * log; + Poco::Logger * log; BackgroundSchedulePool::TaskHolder task; pcg64 rng; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index eeb6b9bddd7..8f99f315620 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -26,7 +26,7 @@ static const auto PART_CHECK_ERROR_SLEEP_MS = 5 * 1000; ReplicatedMergeTreePartCheckThread::ReplicatedMergeTreePartCheckThread(StorageReplicatedMergeTree & storage_) : storage(storage_) , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreePartCheckThread)") - , log(&Logger::get(log_name)) + , log(&Poco::Logger::get(log_name)) { task = storage.global_context.getSchedulePool().createTask(log_name, [this] { run(); }); task->schedule(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h index 198c9714f64..e86356e1346 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h @@ -77,7 +77,7 @@ private: StorageReplicatedMergeTree & storage; String log_name; - Logger * log; + Poco::Logger * log; using StringSet = std::set; using PartToCheck = std::pair; /// The name of the part and the minimum time to check (or zero, if not important). diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c78f1f4f97a..4ea7ddda738 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -116,7 +116,7 @@ void ReplicatedMergeTreeQueue::initialize( zookeeper_path = zookeeper_path_; replica_path = replica_path_; logger_name = logger_name_; - log = &Logger::get(logger_name); + log = &Poco::Logger::get(logger_name); addVirtualParts(parts); } @@ -1030,11 +1030,18 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( if (!ignore_max_size && sum_parts_size_in_bytes > max_source_parts_size) { - String reason = "Not executing log entry " + entry.typeToString() + " for part " + entry.new_part_name - + " because source parts size (" + formatReadableSizeWithBinarySuffix(sum_parts_size_in_bytes) - + ") is greater than the current maximum (" + formatReadableSizeWithBinarySuffix(max_source_parts_size) + ")."; - LOG_DEBUG(log, reason); - out_postpone_reason = reason; + const char * format_str = "Not executing log entry {} for part {}" + " because source parts size ({}) is greater than the current maximum ({})."; + + LOG_DEBUG(log, format_str, + entry.typeToString(), entry.new_part_name, + ReadableSize(sum_parts_size_in_bytes), ReadableSize(max_source_parts_size)); + + /// Copy-paste of above because we need structured logging (instead of already formatted message). + out_postpone_reason = fmt::format(format_str, + entry.typeToString(), entry.new_part_name, + ReadableSize(sum_parts_size_in_bytes), ReadableSize(max_source_parts_size)); + return false; } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 8e58c8b7af2..4cbb86adb7b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -56,7 +56,7 @@ private: String zookeeper_path; String replica_path; String logger_name; - Logger * log = nullptr; + Poco::Logger * log = nullptr; /// Protects the queue, future_parts and other queue state variables. mutable std::mutex state_mutex; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 5b4f4f2a5be..93d652f2be0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -42,7 +42,7 @@ static String generateActiveNodeIdentifier() ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_) : storage(storage_) , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeRestartingThread)") - , log(&Logger::get(log_name)) + , log(&Poco::Logger::get(log_name)) , active_node_identifier(generateActiveNodeIdentifier()) { const auto storage_settings = storage.getSettings(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 88c6fe755d1..8641af07476 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -33,7 +33,7 @@ public: private: StorageReplicatedMergeTree & storage; String log_name; - Logger * log; + Poco::Logger * log; std::atomic need_stop {false}; /// The random data we wrote into `/replicas/me/is_active`. diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 3d7e26d3817..84cf3a32aa1 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -80,28 +80,6 @@ struct SelectQueryInfo /// Prepared sets are used for indices by storage engine. /// Example: x IN (1, 2, 3) PreparedSets sets; - - /// Temporary flag is needed to support old pipeline with input streams. - /// If enabled, then pipeline returned by storage must be a tree. - /// Processors from the tree can't return ExpandPipeline status. - mutable bool force_tree_shaped_pipeline = false; -}; - -/// RAII class to enable force_tree_shaped_pipeline for SelectQueryInfo. -/// Looks awful, but I hope it's temporary. -struct ForceTreeShapedPipeline -{ - explicit ForceTreeShapedPipeline(const SelectQueryInfo & info_) : info(info_) - { - force_tree_shaped_pipeline = info.force_tree_shaped_pipeline; - info.force_tree_shaped_pipeline = true; - } - - ~ForceTreeShapedPipeline() { info.force_tree_shaped_pipeline = force_tree_shaped_pipeline; } - -private: - bool force_tree_shaped_pipeline; - const SelectQueryInfo & info; }; } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index b3b3db4a2ce..77f2ba1c288 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -74,7 +74,7 @@ StorageBuffer::StorageBuffer( , max_thresholds(max_thresholds_) , destination_id(destination_id_) , allow_materialized(allow_materialized_) - , log(&Logger::get("StorageBuffer (" + table_id_.getFullTableName() + ")")) + , log(&Poco::Logger::get("StorageBuffer (" + table_id_.getFullTableName() + ")")) , bg_pool(global_context.getBufferFlushSchedulePool()) { setColumns(columns_); @@ -129,7 +129,7 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context { if (destination_id) { - auto destination = DatabaseCatalog::instance().getTable(destination_id); + auto destination = DatabaseCatalog::instance().getTable(destination_id, context); if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); @@ -153,7 +153,7 @@ Pipes StorageBuffer::read( if (destination_id) { - auto destination = DatabaseCatalog::instance().getTable(destination_id); + auto destination = DatabaseCatalog::instance().getTable(destination_id, context); if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); @@ -234,7 +234,7 @@ Pipes StorageBuffer::read( */ if (processed_stage > QueryProcessingStage::FetchColumns) for (auto & pipe : pipes_from_buffers) - pipe = InterpreterSelectQuery(query_info.query, context, std::move(pipe), SelectQueryOptions(processed_stage)).executeWithProcessors().getPipe(); + pipe = InterpreterSelectQuery(query_info.query, context, std::move(pipe), SelectQueryOptions(processed_stage)).execute().pipeline.getPipe(); if (query_info.prewhere_info) { @@ -334,7 +334,7 @@ public: StoragePtr destination; if (storage.destination_id) { - destination = DatabaseCatalog::instance().tryGetTable(storage.destination_id); + destination = DatabaseCatalog::instance().tryGetTable(storage.destination_id, storage.global_context); if (destination.get() == &storage) throw Exception("Destination table is myself. Write will cause infinite loop.", ErrorCodes::INFINITE_LOOP); } @@ -434,7 +434,7 @@ bool StorageBuffer::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, con if (!destination_id) return false; - auto destination = DatabaseCatalog::instance().getTable(destination_id); + auto destination = DatabaseCatalog::instance().getTable(destination_id, query_context); if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); @@ -602,7 +602,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc */ try { - writeBlockToDestination(block_to_write, DatabaseCatalog::instance().tryGetTable(destination_id)); + writeBlockToDestination(block_to_write, DatabaseCatalog::instance().tryGetTable(destination_id, global_context)); } catch (...) { @@ -739,7 +739,7 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const S std::optional StorageBuffer::totalRows() const { std::optional underlying_rows; - auto underlying = DatabaseCatalog::instance().tryGetTable(destination_id); + auto underlying = DatabaseCatalog::instance().tryGetTable(destination_id, global_context); if (underlying) underlying_rows = underlying->totalRows(); diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 6d6c1f66569..10a4482c801 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -75,7 +75,7 @@ public: { if (!destination_id) return false; - auto dest = DatabaseCatalog::instance().tryGetTable(destination_id); + auto dest = DatabaseCatalog::instance().tryGetTable(destination_id, global_context); if (dest && dest.get() != this) return dest->supportsPrewhere(); return false; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 96ab8266b37..2b10311d8cf 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -278,7 +278,7 @@ StorageDistributed::StorageDistributed( , remote_database(remote_database_) , remote_table(remote_table_) , global_context(std::make_unique(context_)) - , log(&Logger::get("StorageDistributed (" + id_.table_name + ")")) + , log(&Poco::Logger::get("StorageDistributed (" + id_.table_name + ")")) , cluster_name(global_context->getMacros()->expand(cluster_name_)) , has_sharding_key(sharding_key_) , storage_policy(storage_policy_) diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 5b91c800a1f..a7e3a073af4 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -128,7 +128,7 @@ public: ASTPtr remote_table_function_ptr; std::unique_ptr global_context; - Logger * log; + Poco::Logger * log; /// Used to implement TableFunctionRemote. std::shared_ptr owned_cluster; diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 33d8385cf85..fa5034d946c 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -85,7 +85,7 @@ private: mutable std::shared_mutex rwlock; - Logger * log = &Logger::get("StorageFile"); + Poco::Logger * log = &Poco::Logger::get("StorageFile"); }; } diff --git a/src/Storages/StorageHDFS.h b/src/Storages/StorageHDFS.h index 48defd72911..5b250247b84 100644 --- a/src/Storages/StorageHDFS.h +++ b/src/Storages/StorageHDFS.h @@ -45,7 +45,7 @@ private: Context & context; String compression_method; - Logger * log = &Logger::get("StorageHDFS"); + Poco::Logger * log = &Poco::Logger::get("StorageHDFS"); }; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 866e4a88185..3d8a04feba4 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -149,7 +149,7 @@ StorageMaterializedView::StorageMaterializedView( create_interpreter.setInternal(true); create_interpreter.execute(); - target_table_id = DatabaseCatalog::instance().getTable({manual_create_query->database, manual_create_query->table})->getStorageID(); + target_table_id = DatabaseCatalog::instance().getTable({manual_create_query->database, manual_create_query->table}, global_context)->getStorageID(); } if (!select_table_id.empty()) @@ -204,7 +204,7 @@ BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, const StorageID & target_table_id) { - if (DatabaseCatalog::instance().tryGetTable(target_table_id)) + if (DatabaseCatalog::instance().tryGetTable(target_table_id, global_context)) { /// We create and execute `drop` query for internal table. auto drop_query = std::make_shared(); @@ -362,12 +362,12 @@ void StorageMaterializedView::shutdown() StoragePtr StorageMaterializedView::getTargetTable() const { - return DatabaseCatalog::instance().getTable(target_table_id); + return DatabaseCatalog::instance().getTable(target_table_id, global_context); } StoragePtr StorageMaterializedView::tryGetTargetTable() const { - return DatabaseCatalog::instance().tryGetTable(target_table_id); + return DatabaseCatalog::instance().tryGetTable(target_table_id, global_context); } Strings StorageMaterializedView::getDataPaths() const diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 440c5acc9bf..a800df7a208 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -62,7 +62,7 @@ StorageMerge::StorageMerge( template StoragePtr StorageMerge::getFirstTable(F && predicate) const { - auto iterator = getDatabaseIterator(); + auto iterator = getDatabaseIterator(global_context); while (iterator->isValid()) { @@ -110,7 +110,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & { auto stage_in_source_tables = QueryProcessingStage::FetchColumns; - DatabaseTablesIteratorPtr iterator = getDatabaseIterator(); + DatabaseTablesIteratorPtr iterator = getDatabaseIterator(context); size_t selected_table_size = 0; @@ -238,20 +238,9 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer if (!storage) { - if (query_info.force_tree_shaped_pipeline) - { - /// This flag means that pipeline must be tree-shaped, - /// so we can't enable processors for InterpreterSelectQuery here. - auto stream = InterpreterSelectQuery(modified_query_info.query, *modified_context, std::make_shared(header), - SelectQueryOptions(processed_stage).analyze()).execute().in; - - pipes.emplace_back(std::make_shared(std::move(stream))); - return pipes; - } - auto pipe = InterpreterSelectQuery(modified_query_info.query, *modified_context, std::make_shared(header), - SelectQueryOptions(processed_stage).analyze()).executeWithProcessors().getPipe(); + SelectQueryOptions(processed_stage).analyze()).execute().pipeline.getPipe(); pipe.addInterpreterContext(modified_context); pipes.emplace_back(std::move(pipe)); return pipes; @@ -276,15 +265,8 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer InterpreterSelectQuery interpreter{modified_query_info.query, *modified_context, SelectQueryOptions(processed_stage)}; - if (query_info.force_tree_shaped_pipeline) { - BlockInputStreamPtr stream = interpreter.execute().in; - Pipe pipe(std::make_shared(std::move(stream))); - pipes.emplace_back(std::move(pipe)); - } - else - { - Pipe pipe = interpreter.executeWithProcessors().getPipe(); + Pipe pipe = interpreter.execute().pipeline.getPipe(); pipes.emplace_back(std::move(pipe)); } @@ -329,7 +311,7 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const String & query_id, const Settings & settings) const { StorageListWithLocks selected_tables; - auto iterator = getDatabaseIterator(); + auto iterator = getDatabaseIterator(global_context); while (iterator->isValid()) { @@ -349,7 +331,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( const ASTPtr & query, bool has_virtual_column, const String & query_id, const Settings & settings) const { StorageListWithLocks selected_tables; - DatabaseTablesIteratorPtr iterator = getDatabaseIterator(); + DatabaseTablesIteratorPtr iterator = getDatabaseIterator(global_context); auto virtual_column = ColumnString::create(); @@ -384,12 +366,12 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( } -DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator() const +DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const Context & context) const { checkStackSize(); auto database = DatabaseCatalog::instance().getDatabase(source_database); auto table_name_match = [this](const String & table_name_) { return table_name_regexp.match(table_name_); }; - return database->getTablesIterator(table_name_match); + return database->getTablesIterator(context, table_name_match); } diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index ebe40109d84..401c5facf0c 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -61,7 +61,7 @@ private: template StoragePtr getFirstTable(F && predicate) const; - DatabaseTablesIteratorPtr getDatabaseIterator() const; + DatabaseTablesIteratorPtr getDatabaseIterator(const Context & context) const; NamesAndTypesList getVirtuals() const override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ca9876abe3e..8b383672342 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -990,7 +990,7 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma case PartitionCommand::MoveDestinationType::TABLE: checkPartitionCanBeDropped(command.partition); String dest_database = context.resolveDatabase(command.to_database); - auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}); + auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, context); movePartitionToTable(dest_storage, command.partition, context); break; } @@ -1002,7 +1002,7 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma { checkPartitionCanBeDropped(command.partition); String from_database = context.resolveDatabase(command.from_database); - auto from_storage = DatabaseCatalog::instance().getTable({from_database, command.from_table}); + auto from_storage = DatabaseCatalog::instance().getTable({from_database, command.from_table}, context); replacePartitionFrom(from_storage, command.partition, command.replace, context); } break; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b272bb30764..cd513159544 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1570,7 +1570,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) auto clone_data_parts_from_source_table = [&] () -> size_t { - source_table = DatabaseCatalog::instance().tryGetTable(source_table_id); + source_table = DatabaseCatalog::instance().tryGetTable(source_table_id, global_context); if (!source_table) { LOG_DEBUG(log, "Can't use {} as source table for REPLACE PARTITION command. It does not exist.", source_table_id.getNameForLogs()); @@ -3485,7 +3485,7 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part case PartitionCommand::MoveDestinationType::TABLE: checkPartitionCanBeDropped(command.partition); String dest_database = query_context.resolveDatabase(command.to_database); - auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}); + auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, query_context); movePartitionToTable(dest_storage, command.partition, query_context); break; } @@ -3496,7 +3496,7 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part { checkPartitionCanBeDropped(command.partition); String from_database = query_context.resolveDatabase(command.from_database); - auto from_storage = DatabaseCatalog::instance().getTable({from_database, command.from_table}); + auto from_storage = DatabaseCatalog::instance().getTable({from_database, command.from_table}, query_context); replacePartitionFrom(from_storage, command.partition, command.replace, query_context); } break; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 22fe4656f17..e8fd89c4505 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -119,7 +119,6 @@ namespace return Chunk(std::move(columns), num_rows); } - reader->readSuffix(); reader.reset(); return {}; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 4682b873dc5..86bfed5ac84 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -191,8 +191,8 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path) backup_stream.readSuffix(); /// TODO Add speed, compressed bytes, data volume in memory, compression ratio ... Generalize all statistics logging in project. - LOG_INFO(&Logger::get("StorageSetOrJoinBase"), "Loaded from backup file {}. {} rows, {}. State has {} unique rows.", - file_path, backup_stream.getProfileInfo().rows, formatReadableSizeWithBinarySuffix(backup_stream.getProfileInfo().bytes), getSize()); + LOG_INFO(&Poco::Logger::get("StorageSetOrJoinBase"), "Loaded from backup file {}. {} rows, {}. State has {} unique rows.", + file_path, backup_stream.getProfileInfo().rows, ReadableSize(backup_stream.getProfileInfo().bytes), getSize()); } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 5c3e93fc7f8..b61d52657dd 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -221,7 +221,7 @@ StorageStripeLog::StorageStripeLog( , table_path(relative_path_) , max_compress_block_size(max_compress_block_size_) , file_checker(disk, table_path + "sizes.json") - , log(&Logger::get("StorageStripeLog")) + , log(&Poco::Logger::get("StorageStripeLog")) { setColumns(columns_); setConstraints(constraints_); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index c85f576bd43..ed8e5da081e 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -68,7 +68,7 @@ private: FileChecker file_checker; mutable std::shared_mutex rwlock; - Logger * log; + Poco::Logger * log; }; } diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 5bdfa1fc374..2a62068516e 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -334,7 +334,7 @@ StorageTinyLog::StorageTinyLog( , table_path(relative_path_) , max_compress_block_size(max_compress_block_size_) , file_checker(disk, table_path + "sizes.json") - , log(&Logger::get("StorageTinyLog")) + , log(&Poco::Logger::get("StorageTinyLog")) { setColumns(columns_); setConstraints(constraints_); diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index a8be3be2435..102ec76fda3 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -71,7 +71,7 @@ private: FileChecker file_checker; mutable std::shared_mutex rwlock; - Logger * log; + Poco::Logger * log; void addFiles(const String & column_name, const IDataType & type); }; diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 01c85c5238a..c6b37a50aa9 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -65,42 +65,24 @@ Pipes StorageView::read( current_inner_query = getRuntimeViewQuery(*query_info.query->as(), context); InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names); - /// FIXME res may implicitly use some objects owned be pipeline, but them will be destructed after return - if (query_info.force_tree_shaped_pipeline) + + auto pipeline = interpreter.execute().pipeline; + + /// It's expected that the columns read from storage are not constant. + /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. + pipeline.addSimpleTransform([](const Block & header) { - QueryPipeline pipeline; - BlockInputStreams streams = interpreter.executeWithMultipleStreams(pipeline); + return std::make_shared(header); + }); - for (auto & stream : streams) - { - stream = std::make_shared(stream); - stream = std::make_shared(stream, getSampleBlockForColumns(column_names), - ConvertingBlockInputStream::MatchColumnsMode::Name); - } - - for (auto & stream : streams) - pipes.emplace_back(std::make_shared(std::move(stream))); - } - else + /// And also convert to expected structure. + pipeline.addSimpleTransform([&](const Block & header) { - auto pipeline = interpreter.executeWithProcessors(); + return std::make_shared(header, getSampleBlockForColumns(column_names), + ConvertingTransform::MatchColumnsMode::Name); + }); - /// It's expected that the columns read from storage are not constant. - /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. - pipeline.addSimpleTransform([](const Block & header) - { - return std::make_shared(header); - }); - - /// And also convert to expected structure. - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, getSampleBlockForColumns(column_names), - ConvertingTransform::MatchColumnsMode::Name); - }); - - pipes = std::move(pipeline).getPipes(); - } + pipes = std::move(pipeline).getPipes(); return pipes; } diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index ab824fc8bdc..8eb8856512e 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -301,7 +301,7 @@ Pipes StorageSystemColumns::read( const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; - for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) { const String & table_name = iterator->name(); storages.emplace(std::piecewise_construct, diff --git a/src/Storages/System/StorageSystemGraphite.cpp b/src/Storages/System/StorageSystemGraphite.cpp index cde75aa5550..bfa25a99838 100644 --- a/src/Storages/System/StorageSystemGraphite.cpp +++ b/src/Storages/System/StorageSystemGraphite.cpp @@ -25,7 +25,7 @@ NamesAndTypesList StorageSystemGraphite::getNamesAndTypes() /* * Looking for (Replicated)*GraphiteMergeTree and get all configuration parameters for them */ -static StorageSystemGraphite::Configs getConfigs() +static StorageSystemGraphite::Configs getConfigs(const Context & context) { const Databases databases = DatabaseCatalog::instance().getDatabases(); StorageSystemGraphite::Configs graphite_configs; @@ -36,7 +36,7 @@ static StorageSystemGraphite::Configs getConfigs() if (db.second->getEngineName() == "Lazy") continue; - for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { const auto & table = iterator->table(); @@ -71,9 +71,9 @@ static StorageSystemGraphite::Configs getConfigs() return graphite_configs; } -void StorageSystemGraphite::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const +void StorageSystemGraphite::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - Configs graphite_configs = getConfigs(); + Configs graphite_configs = getConfigs(context); for (const auto & config : graphite_configs) { diff --git a/src/Storages/System/StorageSystemMutations.cpp b/src/Storages/System/StorageSystemMutations.cpp index d4a262860dc..685565d82e1 100644 --- a/src/Storages/System/StorageSystemMutations.cpp +++ b/src/Storages/System/StorageSystemMutations.cpp @@ -51,7 +51,7 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, const Contex const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); - for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 6356e6d699e..3b97cbb2d9b 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -111,7 +111,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; - for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) { String table_name = iterator->name(); StoragePtr storage = iterator->table(); diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 734d0098d8d..251b45e44b6 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -76,7 +76,7 @@ Pipes StorageSystemReplicas::read( if (db.second->getEngineName() == "Lazy") continue; const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); - for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; diff --git a/src/Storages/System/StorageSystemReplicationQueue.cpp b/src/Storages/System/StorageSystemReplicationQueue.cpp index a6f0af8cc6c..2c188cf3734 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -60,7 +60,7 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); - for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 2caf599fe51..2bf6595bf53 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -232,7 +232,7 @@ protected: const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); if (!tables_it || !tables_it->isValid()) - tables_it = database->getTablesIterator(); + tables_it = database->getTablesIterator(context); const bool need_lock_structure = needLockStructure(database, getPort().getHeader()); @@ -331,7 +331,7 @@ protected: if (columns_mask[src_index] || columns_mask[src_index + 1]) { - ASTPtr ast = database->tryGetCreateTableQuery(table_name); + ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); if (columns_mask[src_index++]) res_columns[res_index++]->insert(ast ? queryToString(ast) : ""); diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 711612cf9e5..19d1172f1ff 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -84,7 +84,7 @@ ColumnsDescription getStructureOfRemoteTableInShard( else { if (shard_info.isLocal()) - return DatabaseCatalog::instance().getTable(table_id)->getColumns(); + return DatabaseCatalog::instance().getTable(table_id, context)->getColumns(); /// Request for a table description query = "DESC TABLE " + table_id.getFullTableName(); diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 203104801aa..fff352210e7 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -68,7 +68,7 @@ using DiskImplementations = testing::Types; TYPED_TEST_SUITE(StorageLogTest, DiskImplementations); // Returns data written to table in Values format. -std::string writeData(int rows, DB::StoragePtr & table, DB::Context & context) +std::string writeData(int rows, DB::StoragePtr & table, const DB::Context & context) { using namespace DB; @@ -104,7 +104,7 @@ std::string writeData(int rows, DB::StoragePtr & table, DB::Context & context) } // Returns all table data in Values format. -std::string readData(DB::StoragePtr & table, DB::Context & context) +std::string readData(DB::StoragePtr & table, const DB::Context & context) { using namespace DB; @@ -136,7 +136,7 @@ std::string readData(DB::StoragePtr & table, DB::Context & context) TYPED_TEST(StorageLogTest, testReadWrite) { using namespace DB; - auto context_holder = getContext(); + const auto & context_holder = getContext(); std::string data; diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index df6ee2bd0a5..bf86322a676 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -18,7 +18,7 @@ using namespace DB; /// NOTE How to do better? struct State { - Context & context; + Context context; NamesAndTypesList columns{ {"column", std::make_shared()}, {"apply_id", std::make_shared()}, @@ -27,10 +27,11 @@ struct State {"create_time", std::make_shared()}, }; - explicit State(Context & context_) : context(context_) + explicit State() + : context(getContext().context) { registerFunctions(); - DatabasePtr database = std::make_shared("test"); + DatabasePtr database = std::make_shared("test", context); database->attachTable("table", StorageMemory::create(StorageID("test", "table"), ColumnsDescription{columns}, ConstraintsDescription{})); context.makeGlobalContext(); DatabaseCatalog::instance().attachDatabase("test", database); @@ -38,6 +39,11 @@ struct State } }; +State getState() +{ + static State state; + return state; +} static void check(const std::string & query, const std::string & expected, const Context & context, const NamesAndTypesList & columns) { @@ -54,8 +60,7 @@ static void check(const std::string & query, const std::string & expected, const TEST(TransformQueryForExternalDatabase, InWithSingleElement) { - auto context_holder = getContext(); - State state(context_holder.context); + const State & state = getState(); check("SELECT column FROM test.table WHERE 1 IN (1)", R"(SELECT "column" FROM "test"."table" WHERE 1)", @@ -70,8 +75,7 @@ TEST(TransformQueryForExternalDatabase, InWithSingleElement) TEST(TransformQueryForExternalDatabase, Like) { - auto context_holder = getContext(); - State state(context_holder.context); + const State & state = getState(); check("SELECT column FROM test.table WHERE column LIKE '%hello%'", R"(SELECT "column" FROM "test"."table" WHERE "column" LIKE '%hello%')", @@ -83,8 +87,7 @@ TEST(TransformQueryForExternalDatabase, Like) TEST(TransformQueryForExternalDatabase, Substring) { - auto context_holder = getContext(); - State state(context_holder.context); + const State & state = getState(); check("SELECT column FROM test.table WHERE left(column, 10) = RIGHT(column, 10) AND SUBSTRING(column FROM 1 FOR 2) = 'Hello'", R"(SELECT "column" FROM "test"."table")", @@ -93,8 +96,7 @@ TEST(TransformQueryForExternalDatabase, Substring) TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries) { - auto context_holder = getContext(); - State state(context_holder.context); + const State & state = getState(); check("SELECT column FROM test.table WHERE 1 = 1 AND toString(column) = '42' AND column = 42 AND left(column, 10) = RIGHT(column, 10) AND column IN (1, 42) AND SUBSTRING(column FROM 1 FOR 2) = 'Hello' AND column != 4", R"(SELECT "column" FROM "test"."table" WHERE 1 AND ("column" = 42) AND ("column" IN (1, 42)) AND ("column" != 4))", @@ -106,8 +108,7 @@ TEST(TransformQueryForExternalDatabase, MultipleAndSubqueries) TEST(TransformQueryForExternalDatabase, Issue7245) { - auto context_holder = getContext(); - State state(context_holder.context); + const State & state = getState(); check("select apply_id from test.table where apply_type = 2 and create_time > addDays(toDateTime('2019-01-01 01:02:03'),-7) and apply_status in (3,4)", R"(SELECT "apply_id", "apply_type", "apply_status", "create_time" FROM "test"."table" WHERE ("apply_type" = 2) AND ("create_time" > '2018-12-25 01:02:03') AND ("apply_status" IN (3, 4)))", diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 7a161c13bdc..cd924270f7c 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes } -static NamesAndTypesList chooseColumns(const String & source_database, const String & table_name_regexp_) +static NamesAndTypesList chooseColumns(const String & source_database, const String & table_name_regexp_, const Context & context) { OptimizedRegularExpression table_name_regexp(table_name_regexp_); auto table_name_match = [&](const String & table_name) { return table_name_regexp.match(table_name); }; @@ -31,7 +31,7 @@ static NamesAndTypesList chooseColumns(const String & source_database, const Str { auto database = DatabaseCatalog::instance().getDatabase(source_database); - auto iterator = database->getTablesIterator(table_name_match); + auto iterator = database->getTablesIterator(context, table_name_match); if (iterator->isValid()) any_table = iterator->table(); @@ -69,7 +69,7 @@ StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Co auto res = StorageMerge::create( StorageID(getDatabaseName(), table_name), - ColumnsDescription{chooseColumns(source_database, table_name_regexp)}, + ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)}, source_database, table_name_regexp, context); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 1a2735296e8..219546c92f1 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -30,6 +30,8 @@ from contextlib import closing MESSAGES_TO_RETRY = [ "DB::Exception: ZooKeeper session has been expired", "Coordination::Exception: Connection loss", + "Operation timed out", + "ConnectionPoolWithFailover: Connection failed at try", ] @@ -331,6 +333,19 @@ def run_tests_array(all_tests_with_params): server_logs_level = "warning" +def check_server_started(client, retry_count): + while retry_count > 0: + clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + (stdout, stderr) = clickhouse_proc.communicate("SELECT 1") + if clickhouse_proc.returncode != 0 or not stdout.startswith("1"): + retry_count -= 1 + sleep(0.5) + else: + return True + + return False + + def main(args): global SERVER_DIED global exit_code @@ -344,6 +359,9 @@ def main(args): return stdout.startswith('1') + if not check_server_started(args.client, args.server_check_retries): + raise Exception("clickhouse-server is not responding. Cannot execute 'SELECT 1' query.") + base_dir = os.path.abspath(args.queries) tmp_dir = os.path.abspath(args.tmp) @@ -360,7 +378,7 @@ def main(args): os.environ.setdefault("CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL", server_logs_level) if args.zookeeper is None: - code, out = commands.getstatusoutput(args.extract_from_config +" --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l') + code, out = commands.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l') try: if int(out) > 0: args.zookeeper = True @@ -484,7 +502,14 @@ def main(args): total_tests_run += tests_n if args.hung_check: - processlist = get_processlist(args.client) + + # Some queries may execute in background for some time after test was finished. This is normal. + for n in range(1, 60): + processlist = get_processlist(args.client) + if not processlist: + break + sleep(1) + if processlist: print(colored("\nFound hung queries in processlist:", args, "red", attrs=["bold"])) print(processlist) @@ -570,6 +595,7 @@ if __name__ == '__main__': parser.add_argument('--parallel', default='1/1', help='One parallel test run number/total') parser.add_argument('-j', '--jobs', default=1, nargs='?', type=int, help='Run all tests in parallel') parser.add_argument('-U', '--unified', default=3, type=int, help='output NUM lines of unified context') + parser.add_argument('-r', '--server-check-retries', default=30, type=int, help='Num of tries to execute SELECT 1 before tests started') parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests') parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests') diff --git a/tests/integration/test_default_role/__init__.py b/tests/integration/test_default_role/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_default_role/test.py b/tests/integration/test_default_role/test.py new file mode 100644 index 00000000000..2b6b4698b20 --- /dev/null +++ b/tests/integration/test_default_role/test.py @@ -0,0 +1,77 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV +import re + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance') + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + + instance.query("CREATE USER john") + instance.query("CREATE ROLE rx") + instance.query("CREATE ROLE ry") + + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def reset_users_and_roles(): + instance.query("CREATE USER OR REPLACE john") + yield + + +def test_set_default_roles(): + assert instance.query("SHOW CURRENT ROLES", user="john") == "" + + instance.query("GRANT rx, ry TO john") + assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['rx', 0, 1], ['ry', 0, 1]] ) + + instance.query("SET DEFAULT ROLE NONE TO john") + assert instance.query("SHOW CURRENT ROLES", user="john") == "" + + instance.query("SET DEFAULT ROLE rx TO john") + assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['rx', 0, 1]] ) + + instance.query("SET DEFAULT ROLE ry TO john") + assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['ry', 0, 1]] ) + + instance.query("SET DEFAULT ROLE ALL TO john") + assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['rx', 0, 1], ['ry', 0, 1]] ) + + instance.query("SET DEFAULT ROLE ALL EXCEPT rx TO john") + assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['ry', 0, 1]] ) + + +def test_alter_user(): + assert instance.query("SHOW CURRENT ROLES", user="john") == "" + + instance.query("GRANT rx, ry TO john") + assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['rx', 0, 1], ['ry', 0, 1]] ) + + instance.query("ALTER USER john DEFAULT ROLE NONE") + assert instance.query("SHOW CURRENT ROLES", user="john") == "" + + instance.query("ALTER USER john DEFAULT ROLE rx") + assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['rx', 0, 1]] ) + + instance.query("ALTER USER john DEFAULT ROLE ALL") + assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['rx', 0, 1], ['ry', 0, 1]] ) + + instance.query("ALTER USER john DEFAULT ROLE ALL EXCEPT rx") + assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['ry', 0, 1]] ) + + +def test_wrong_set_default_role(): + assert "There is no user `rx`" in instance.query_and_get_error("SET DEFAULT ROLE NONE TO rx") + assert "There is no user `ry`" in instance.query_and_get_error("SET DEFAULT ROLE rx TO ry") + assert "There is no role `john`" in instance.query_and_get_error("SET DEFAULT ROLE john TO john") + assert "There is no role `john`" in instance.query_and_get_error("ALTER USER john DEFAULT ROLE john") + assert "There is no role `john`" in instance.query_and_get_error("ALTER USER john DEFAULT ROLE ALL EXCEPT john") diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/configs/dictionaries/cache_strings_default_settings.xml b/tests/integration/test_dictionary_allow_read_expired_keys/configs/dictionaries/cache_strings_default_settings.xml new file mode 100644 index 00000000000..11807bc1ad6 --- /dev/null +++ b/tests/integration/test_dictionary_allow_read_expired_keys/configs/dictionaries/cache_strings_default_settings.xml @@ -0,0 +1,35 @@ + + + default_string + + + dictionary_node + 9000 + default + + test + strings
+
+ + + 2 + 1 + + + + 1000 + 10000 + + + + + key + + + value + String + > + + +
+
\ No newline at end of file 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 new file mode 100644 index 00000000000..7d762db2a6d --- /dev/null +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py @@ -0,0 +1,61 @@ +from __future__ import print_function +import pytest +import os +import random +import string +import time + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) + +dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) +main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_strings_default_settings.xml']) + + +def get_random_string(string_length=8): + alphabet = string.ascii_letters + string.digits + return ''.join((random.choice(alphabet) for _ in range(string_length))) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + dictionary_node.query("CREATE DATABASE IF NOT EXISTS test;") + dictionary_node.query("DROP TABLE IF EXISTS test.strings;") + dictionary_node.query(""" + CREATE TABLE test.strings + (key UInt64, value String) + ENGINE = Memory; + """) + + values_to_insert = ", ".join(["({}, '{}')".format(1000000 + number, get_random_string()) for number in range(100)]) + dictionary_node.query("INSERT INTO test.strings VALUES {}".format(values_to_insert)) + + yield cluster + finally: + cluster.shutdown() + +# @pytest.mark.skip(reason="debugging") +def test_return_real_values(started_cluster): + assert None != dictionary_node.get_process_pid("clickhouse"), "ClickHouse must be alive" + + first_batch = """ + SELECT count(*) + FROM + ( + SELECT + arrayJoin(arrayMap(x -> (x + 1000000), range(100))) AS id, + dictGetString('default_string', 'value', toUInt64(id)) AS value + ) + WHERE value = ''; + """ + + assert TSV("0") == TSV(main_node.query(first_batch)) + + # Waiting for cache to become expired + time.sleep(5) + + assert TSV("0") == TSV(main_node.query(first_batch)) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index f75a168d5db..e2c02239ae9 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -144,7 +144,7 @@ def test_mysql_federated(mysql_server, server_address): node.query('''CREATE TABLE mysql_federated.test (col UInt32) ENGINE = Log''', settings={"password": "123"}) node.query('''INSERT INTO mysql_federated.test VALUES (0), (1), (5)''', settings={"password": "123"}) - code, (_, stderr) = mysql_server.exec_run(''' + code, (stdout, stderr) = mysql_server.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');" @@ -152,6 +152,9 @@ def test_mysql_federated(mysql_server, server_address): -e "CREATE DATABASE mysql_federated;" '''.format(host=server_address, port=server_port), demux=True) + if code != 0: + print(stdout) + print(stderr) assert code == 0 code, (stdout, stderr) = mysql_server.exec_run(''' diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 13577864870..6684266ccab 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -12,8 +12,11 @@ from helpers.network import PartitionManager import json import subprocess import kafka.errors -from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer +from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnection from kafka.admin import NewTopic +from kafka.protocol.admin import DescribeGroupsResponse_v1, DescribeGroupsRequest_v1 +from kafka.protocol.group import MemberAssignment +import socket from google.protobuf.internal.encoder import _VarintBytes """ @@ -110,6 +113,32 @@ def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference else: return TSV(result) == TSV(reference) +# https://stackoverflow.com/a/57692111/1555175 +def describe_consumer_group(name): + client = BrokerConnection('localhost', 9092, 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 @@ -161,6 +190,9 @@ def test_kafka_settings_old_syntax(kafka_cluster): kafka_check_result(result, True) + members = describe_consumer_group('old') + assert members[0]['client_id'] == u'ClickHouse-instance-test-kafka' + # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:9092 --describe --members --group old --verbose")) @pytest.mark.timeout(180) def test_kafka_settings_new_syntax(kafka_cluster): @@ -172,6 +204,7 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_group_name = 'new', kafka_format = 'JSONEachRow', kafka_row_delimiter = '\\n', + kafka_client_id = '{instance} test 1234', kafka_skip_broken_messages = 1; ''') @@ -197,6 +230,8 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_check_result(result, True) + members = describe_consumer_group('new') + assert members[0]['client_id'] == u'instance test 1234' @pytest.mark.timeout(180) def test_kafka_consumer_hang(kafka_cluster): @@ -837,6 +872,7 @@ def test_kafka_virtual_columns2(kafka_cluster): SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'virt2_0,virt2_1', kafka_group_name = 'virt2', + kafka_num_consumers = 2, kafka_format = 'JSONEachRow'; CREATE MATERIALIZED VIEW test.view Engine=Log AS @@ -866,6 +902,11 @@ def test_kafka_virtual_columns2(kafka_cluster): time.sleep(10) + members = describe_consumer_group('virt2') + #pprint.pprint(members) + members[0]['client_id'] = u'ClickHouse-instance-test-kafka-0' + members[1]['client_id'] = u'ClickHouse-instance-test-kafka-1' + result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) expected = '''\ @@ -1475,6 +1516,7 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): # as it's a bit tricky to hit the proper moment - let's check in logs if we did it correctly assert instance.contains_in_log("Local: Waiting for coordinator") + assert instance.contains_in_log("All commit attempts failed") result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') print(result) @@ -1484,7 +1526,10 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): DROP TABLE test.view; ''') - assert TSV(result) == TSV('22\t22\t22') + # After https://github.com/edenhill/librdkafka/issues/2631 + # timeout triggers rebalance, making further commits to the topic after getting back online + # impossible. So we have a duplicate in that scenario, but we report that situation properly. + assert TSV(result) == TSV('42\t22\t22') diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index c6453beea6a..ae2af34c35b 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -834,6 +834,7 @@ def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): finally: node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) +@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), @@ -864,6 +865,8 @@ def test_double_move_while_select(started_cluster, name, positive): thread = threading.Thread(target=long_select) thread.start() + time.sleep(1) + node1.query("ALTER TABLE {name} MOVE PART '{part}' TO DISK 'jbod1'".format(name=name, part=parts[0])) # Fill jbod1 to force ClickHouse to make move of partition 1 to external. diff --git a/tests/performance/insert_select_default_small_block.xml b/tests/performance/insert_select_default_small_block.xml new file mode 100644 index 00000000000..12e67b09d2f --- /dev/null +++ b/tests/performance/insert_select_default_small_block.xml @@ -0,0 +1,38 @@ + + 1 + + +CREATE TABLE insert_small_block_performance +( + `x` String, + `a` DEFAULT SHA256(x), + `b` DEFAULT SHA256(toString(a)), + `c` DEFAULT SHA256(toString(b)), + `d` DEFAULT SHA256(toString(c)), + `e` DEFAULT SHA256(toString(d)), + `f` DEFAULT SHA256(toString(e)), + `g` DEFAULT SHA256(toString(f)), + `h` DEFAULT SHA256(toString(g)), + `i` DEFAULT SHA256(toString(h)), + `j` DEFAULT SHA256(toString(i)), + `k` DEFAULT SHA256(toString(j)), + `l` DEFAULT SHA256(toString(k)), + `m` DEFAULT SHA256(toString(l)), + `n` DEFAULT SHA256(toString(m)), + `o` DEFAULT SHA256(toString(n)), + `p` DEFAULT SHA256(toString(o)), + `q` DEFAULT SHA256(toString(p)), + `r` DEFAULT SHA256(toString(q)), + `s` DEFAULT SHA256(toString(r)), + `t` DEFAULT SHA256(toString(s)), + `u` DEFAULT SHA256(toString(t)), + `v` DEFAULT SHA256(toString(u)), + `w` DEFAULT SHA256(toString(v)) +) +ENGINE = Null; + + + INSERT INTO insert_small_block_performance (x) SELECT toString(number) FROM numbers(10000); + + DROP TABLE IF EXISTS insert_small_block_performance + diff --git a/tests/queries/0_stateless/00342_escape_sequences.reference b/tests/queries/0_stateless/00342_escape_sequences.reference index b7ecd06be78..50e0756d16f 100644 --- a/tests/queries/0_stateless/00342_escape_sequences.reference +++ b/tests/queries/0_stateless/00342_escape_sequences.reference @@ -1 +1 @@ -07080C0A0D090B5C27223FAA +07080C0A0D090B5C27225C3FAA diff --git a/tests/queries/0_stateless/00365_statistics_in_formats.sh b/tests/queries/0_stateless/00365_statistics_in_formats.sh index f0e23337806..fb79b23f78e 100755 --- a/tests/queries/0_stateless/00365_statistics_in_formats.sh +++ b/tests/queries/0_stateless/00365_statistics_in_formats.sh @@ -3,16 +3,16 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.numbers"; -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.numbers (number UInt64) engine = MergeTree order by number"; -$CLICKHOUSE_CLIENT --query="INSERT INTO test.numbers select * from system.numbers limit 10"; +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS numbers"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE numbers (number UInt64) engine = MergeTree order by number"; +$CLICKHOUSE_CLIENT --query="INSERT INTO numbers select * from system.numbers limit 10"; -$CLICKHOUSE_CLIENT --query="SELECT number FROM test.numbers LIMIT 10 FORMAT JSON" | grep 'rows_read'; -$CLICKHOUSE_CLIENT --query="SELECT number FROM test.numbers LIMIT 10 FORMAT JSONCompact" | grep 'rows_read'; -$CLICKHOUSE_CLIENT --query="SELECT number FROM test.numbers LIMIT 10 FORMAT XML" | grep 'rows_read'; +$CLICKHOUSE_CLIENT --query="SELECT number FROM numbers LIMIT 10 FORMAT JSON" | grep 'rows_read'; +$CLICKHOUSE_CLIENT --query="SELECT number FROM numbers LIMIT 10 FORMAT JSONCompact" | grep 'rows_read'; +$CLICKHOUSE_CLIENT --query="SELECT number FROM numbers LIMIT 10 FORMAT XML" | grep 'rows_read'; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM test.numbers LIMIT 10 FORMAT JSON" | grep 'rows_read'; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM test.numbers LIMIT 10 FORMAT JSONCompact" | grep 'rows_read'; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM test.numbers LIMIT 10 FORMAT XML" | grep 'rows_read'; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM numbers LIMIT 10 FORMAT JSON" | grep 'rows_read'; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM numbers LIMIT 10 FORMAT JSONCompact" | grep 'rows_read'; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM numbers LIMIT 10 FORMAT XML" | grep 'rows_read'; -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.numbers"; \ No newline at end of file +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS numbers"; diff --git a/tests/queries/0_stateless/00539_functions_for_working_with_json.reference b/tests/queries/0_stateless/00539_functions_for_working_with_json.reference index ee7fb68b7c2..c0399f8ab2e 100644 --- a/tests/queries/0_stateless/00539_functions_for_working_with_json.reference +++ b/tests/queries/0_stateless/00539_functions_for_working_with_json.reference @@ -9,5 +9,7 @@ test"string "test_string" "test\\"string" "test\\"string" + "{" + "[" ["]", "2", "3"] {"nested" : [1,2,3]} diff --git a/tests/queries/0_stateless/00539_functions_for_working_with_json.sql b/tests/queries/0_stateless/00539_functions_for_working_with_json.sql index 8a4d1794293..514b5f2e5ea 100644 --- a/tests/queries/0_stateless/00539_functions_for_working_with_json.sql +++ b/tests/queries/0_stateless/00539_functions_for_working_with_json.sql @@ -11,5 +11,7 @@ SELECT visitParamExtractRaw('{"myparam":"test_string"}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": "test_string"}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": "test\\"string"}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": "test\\"string", "other":123}', 'myparam'); +SELECT visitParamExtractRaw('{"myparam": "{"}', 'myparam'); +SELECT visitParamExtractRaw('{"myparam": "["}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": ["]", "2", "3"], "other":123}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": {"nested" : [1,2,3]}, "other":123}', 'myparam'); diff --git a/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql b/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql index d26e947b54f..b4579e6aac3 100644 --- a/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql +++ b/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql @@ -1,14 +1,14 @@ -- Check that settings are correctly passed through Distributed table -DROP TABLE IF EXISTS test.simple; -CREATE TABLE test.simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/test/simple', '1') ORDER BY d; +DROP TABLE IF EXISTS simple; +CREATE TABLE simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/test/simple', '1') ORDER BY d; -- TODO: replace '127.0.0.2' -> '127.0.0.1' after a fix -INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (1); -INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (1); +INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (1); +INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (1); SET insert_deduplicate=0; -INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (2); -INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (2); +INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (2); +INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (2); -SELECT * FROM remote('127.0.0.2', 'test', 'simple') ORDER BY d; -DROP TABLE test.simple; \ No newline at end of file +SELECT * FROM remote('127.0.0.2', currentDatabase(), 'simple') ORDER BY d; +DROP TABLE simple; diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index 44ab712bcf8..2f3514ae2f5 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -26,144 +26,143 @@ function query_with_retry echo "Query '$1' failed with '$result'" } -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.src;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.dst_r1;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.dst_r2;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r1;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r2;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.dst_r1 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/dst_1', '1') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.dst_r2 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/dst_1', '2') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE dst_r1 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/dst_1', '1') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE dst_r2 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/dst_1', '2') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (0, '0', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (2, '0', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (0, '0', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (2, '0', 1);" $CLICKHOUSE_CLIENT --query="SELECT 'Initial';" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (0, '1', 2);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2), (1, '2', 2);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (2, '1', 2);" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (0, '1', 2);" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2), (1, '2', 2);" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (2, '1', 2);" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.src;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM src;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'REPLACE simple';" -query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.src;" -query_with_retry "ALTER TABLE test.src DROP PARTITION 1;" +query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM src;" +query_with_retry "ALTER TABLE src DROP PARTITION 1;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.src;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM src;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'REPLACE empty';" -query_with_retry "ALTER TABLE test.src DROP PARTITION 1;" -query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.src;" +query_with_retry "ALTER TABLE src DROP PARTITION 1;" +query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM src;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'REPLACE recursive';" -query_with_retry "ALTER TABLE test.dst_r1 DROP PARTITION 1;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2), (1, '2', 2);" +query_with_retry "ALTER TABLE dst_r1 DROP PARTITION 1;" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2), (1, '2', 2);" $CLICKHOUSE_CLIENT --query="CREATE table test_block_numbers (m UInt64) ENGINE MergeTree() ORDER BY tuple();" -$CLICKHOUSE_CLIENT --query="INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database='test' AND table='dst_r1' AND active AND name LIKE '1_%';" +$CLICKHOUSE_CLIENT --query="INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database=currentDatabase() AND table='dst_r1' AND active AND name LIKE '1_%';" -query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database='test' AND table='dst_r1' AND active AND name LIKE '1_%';" +$CLICKHOUSE_CLIENT --query="INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database=currentDatabase() AND table='dst_r1' AND active AND name LIKE '1_%';" $CLICKHOUSE_CLIENT --query="SELECT (max(m) - min(m) > 1) AS new_block_is_generated FROM test_block_numbers;" $CLICKHOUSE_CLIENT --query="DROP TABLE test_block_numbers;" $CLICKHOUSE_CLIENT --query="SELECT 'ATTACH FROM';" -query_with_retry "ALTER TABLE test.dst_r1 DROP PARTITION 1;" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.src;" +query_with_retry "ALTER TABLE dst_r1 DROP PARTITION 1;" +$CLICKHOUSE_CLIENT --query="DROP TABLE src;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);" +$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r2 VALUES (1, '1', 2);" -query_with_retry "ALTER TABLE test.dst_r2 ATTACH PARTITION 1 FROM test.src;" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r2 VALUES (1, '1', 2);" +query_with_retry "ALTER TABLE dst_r2 ATTACH PARTITION 1 FROM src;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'REPLACE with fetch';" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.src;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2); -- trash part to be deleted" +$CLICKHOUSE_CLIENT --query="DROP TABLE src;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2); -- trash part to be deleted" # Stop replication at the second replica and remove source table to use fetch instead of copying -$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES test.dst_r2;" -query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.src;" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.src;" -$CLICKHOUSE_CLIENT --query="SYSTEM START REPLICATION QUEUES test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES dst_r2;" +query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM src;" +$CLICKHOUSE_CLIENT --query="DROP TABLE src;" +$CLICKHOUSE_CLIENT --query="SYSTEM START REPLICATION QUEUES dst_r2;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'REPLACE with fetch of merged';" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.src;" -query_with_retry "ALTER TABLE test.dst_r1 DROP PARTITION 1;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;" +query_with_retry "ALTER TABLE dst_r1 DROP PARTITION 1;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2); -- trash part to be deleted" +$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2); -- trash part to be deleted" -$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES test.dst_r2;" -query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.src;" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.src;" +$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES dst_r2;" +query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM src;" +$CLICKHOUSE_CLIENT --query="DROP TABLE src;" # do not wait other replicas to execute OPTIMIZE -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r1;" -query_with_retry "OPTIMIZE TABLE test.dst_r1 PARTITION 1;" "--replication_alter_partitions_sync=0 --optimize_throw_if_noop=1" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r1;" +query_with_retry "OPTIMIZE TABLE dst_r1 PARTITION 1;" "--replication_alter_partitions_sync=0 --optimize_throw_if_noop=1" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM test.dst_r1;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM dst_r1;" -$CLICKHOUSE_CLIENT --query="SYSTEM START REPLICATION QUEUES test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM START REPLICATION QUEUES dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'After restart';" -$CLICKHOUSE_CLIENT --query="USE test;" -$CLICKHOUSE_CLIENT --query="SYSTEM RESTART REPLICA test.dst_r1;" +$CLICKHOUSE_CLIENT --query="SYSTEM RESTART REPLICA dst_r1;" $CLICKHOUSE_CLIENT --query="SYSTEM RESTART REPLICAS;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'DETACH+ATTACH PARTITION';" -query_with_retry "ALTER TABLE test.dst_r1 DETACH PARTITION 0;" -query_with_retry "ALTER TABLE test.dst_r1 DETACH PARTITION 1;" -query_with_retry "ALTER TABLE test.dst_r1 DETACH PARTITION 2;" -query_with_retry "ALTER TABLE test.dst_r1 ATTACH PARTITION 1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +query_with_retry "ALTER TABLE dst_r1 DETACH PARTITION 0;" +query_with_retry "ALTER TABLE dst_r1 DETACH PARTITION 1;" +query_with_retry "ALTER TABLE dst_r1 DETACH PARTITION 2;" +query_with_retry "ALTER TABLE dst_r1 ATTACH PARTITION 1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.src;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.dst_r1;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.dst_r2;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r1;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r2;" diff --git a/tests/queries/0_stateless/00731_long_merge_tree_select_opened_files.sh b/tests/queries/0_stateless/00731_long_merge_tree_select_opened_files.sh index 5177cb6087c..5ae9dcf67ec 100755 --- a/tests/queries/0_stateless/00731_long_merge_tree_select_opened_files.sh +++ b/tests/queries/0_stateless/00731_long_merge_tree_select_opened_files.sh @@ -14,9 +14,11 @@ $CLICKHOUSE_CLIENT $settings -n -q " DROP TABLE IF EXISTS merge_tree_table; CREATE TABLE merge_tree_table (id UInt64, date Date, uid UInt32) ENGINE = MergeTree(date, id, 8192);" + $CLICKHOUSE_CLIENT $settings -q "INSERT INTO merge_tree_table SELECT (intHash64(number)) % 10000, toDate('2018-08-01'), rand() FROM system.numbers LIMIT 10000000;" -$CLICKHOUSE_CLIENT $settings -q "OPTIMIZE TABLE merge_tree_table FINAL;" +# If merge is already happening, OPTIMIZE will be noop. But we have to ensure that the data is merged. +for i in {1..100}; do $CLICKHOUSE_CLIENT $settings --optimize_throw_if_noop=1 -q "OPTIMIZE TABLE merge_tree_table FINAL;" && break; sleep 1; done # The query may open more files if query log will be flushed during the query. # To lower this chance, we also flush logs before the query. diff --git a/tests/queries/0_stateless/00763_lock_buffer.sh b/tests/queries/0_stateless/00763_lock_buffer.sh index 3146ce96132..d320ae51a37 100755 --- a/tests/queries/0_stateless/00763_lock_buffer.sh +++ b/tests/queries/0_stateless/00763_lock_buffer.sh @@ -4,20 +4,20 @@ set -e CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.mt_00763_2" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.buffer_00763_2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mt_00763_2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS buffer_00763_2" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.buffer_00763_2 (s String) ENGINE = Buffer(test, mt_00763_2, 1, 1, 1, 1, 1, 1, 1)" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE buffer_00763_2 (s String) ENGINE = Buffer('$CLICKHOUSE_DATABASE', mt_00763_2, 1, 1, 1, 1, 1, 1, 1)" function thread1() { - seq 1 500 | sed -r -e 's/.+/DROP TABLE IF EXISTS test.mt_00763_2; CREATE TABLE test.mt_00763_2 (s String) ENGINE = MergeTree ORDER BY s; INSERT INTO test.mt_00763_2 SELECT toString(number) FROM numbers(10);/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: + seq 1 500 | sed -r -e 's/.+/DROP TABLE IF EXISTS mt_00763_2; CREATE TABLE mt_00763_2 (s String) ENGINE = MergeTree ORDER BY s; INSERT INTO mt_00763_2 SELECT toString(number) FROM numbers(10);/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: } function thread2() { - seq 1 1000 | sed -r -e 's/.+/SELECT count() FROM test.buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473' + seq 1 1000 | sed -r -e 's/.+/SELECT count() FROM buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473' } thread1 & @@ -25,5 +25,5 @@ thread2 & wait -${CLICKHOUSE_CLIENT} --query="DROP TABLE test.mt_00763_2" -${CLICKHOUSE_CLIENT} --query="DROP TABLE test.buffer_00763_2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mt_00763_2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE buffer_00763_2" diff --git a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh index 059d70253b2..26bdb712ec5 100755 --- a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh +++ b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh @@ -6,21 +6,21 @@ CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.mt_00763_1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.buffer_00763_1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mt_00763_1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS buffer_00763_1" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.buffer_00763_1 (s String) ENGINE = Buffer(test, mt_00763_1, 1, 1, 1, 1, 1, 1, 1)" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.mt_00763_1 (x UInt32, s String) ENGINE = MergeTree ORDER BY x" -${CLICKHOUSE_CLIENT} --query="INSERT INTO test.mt_00763_1 VALUES (1, '1'), (2, '2'), (3, '3')" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE buffer_00763_1 (s String) ENGINE = Buffer($CLICKHOUSE_DATABASE, mt_00763_1, 1, 1, 1, 1, 1, 1, 1)" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE mt_00763_1 (x UInt32, s String) ENGINE = MergeTree ORDER BY x" +${CLICKHOUSE_CLIENT} --query="INSERT INTO mt_00763_1 VALUES (1, '1'), (2, '2'), (3, '3')" function thread1() { - seq 1 300 | sed -r -e 's/.+/ALTER TABLE test.mt_00763_1 MODIFY column s UInt32; ALTER TABLE test.mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: + seq 1 300 | sed -r -e 's/.+/ALTER TABLE mt_00763_1 MODIFY column s UInt32; ALTER TABLE mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: } function thread2() { - seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM test.buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' + seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' } thread1 & @@ -28,5 +28,5 @@ thread2 & wait -${CLICKHOUSE_CLIENT} --query="DROP TABLE test.mt_00763_1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE test.buffer_00763_1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mt_00763_1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE buffer_00763_1" diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql index 2e1df541f14..9f0850ff7e5 100644 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql @@ -1,15 +1,15 @@ -DROP TABLE IF EXISTS test.part_header_r1; -DROP TABLE IF EXISTS test.part_header_r2; +DROP TABLE IF EXISTS part_header_r1; +DROP TABLE IF EXISTS part_header_r2; SET replication_alter_partitions_sync = 2; -CREATE TABLE test.part_header_r1(x UInt32, y UInt32) +CREATE TABLE part_header_r1(x UInt32, y UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/part_header', '1') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 0, old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0; -CREATE TABLE test.part_header_r2(x UInt32, y UInt32) +CREATE TABLE part_header_r2(x UInt32, y UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/part_header', '2') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 1, old_parts_lifetime = 1, @@ -17,47 +17,47 @@ CREATE TABLE test.part_header_r2(x UInt32, y UInt32) cleanup_delay_period_random_add = 0; SELECT '*** Test fetches ***'; -INSERT INTO test.part_header_r1 VALUES (1, 1); -INSERT INTO test.part_header_r2 VALUES (2, 2); -SYSTEM SYNC REPLICA test.part_header_r1; -SYSTEM SYNC REPLICA test.part_header_r2; +INSERT INTO part_header_r1 VALUES (1, 1); +INSERT INTO part_header_r2 VALUES (2, 2); +SYSTEM SYNC REPLICA part_header_r1; +SYSTEM SYNC REPLICA part_header_r2; SELECT '*** replica 1 ***'; -SELECT x, y FROM test.part_header_r1 ORDER BY x; +SELECT x, y FROM part_header_r1 ORDER BY x; SELECT '*** replica 2 ***'; -SELECT x, y FROM test.part_header_r2 ORDER BY x; +SELECT x, y FROM part_header_r2 ORDER BY x; SELECT '*** Test merges ***'; -OPTIMIZE TABLE test.part_header_r1; -SYSTEM SYNC REPLICA test.part_header_r2; +OPTIMIZE TABLE part_header_r1; +SYSTEM SYNC REPLICA part_header_r2; SELECT '*** replica 1 ***'; -SELECT _part, x FROM test.part_header_r1 ORDER BY x; +SELECT _part, x FROM part_header_r1 ORDER BY x; SELECT '*** replica 2 ***'; -SELECT _part, x FROM test.part_header_r2 ORDER BY x; +SELECT _part, x FROM part_header_r2 ORDER BY x; SELECT sleep(3) FORMAT Null; SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; -SELECT name FROM system.parts WHERE active AND database = 'test' AND table = 'part_header_r1'; +SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r1'; SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test/part_header/replicas/1/parts'; SELECT '*** replica 2 ***'; -SELECT name FROM system.parts WHERE active AND database = 'test' AND table = 'part_header_r2'; +SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r2'; SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test/part_header/replicas/1/parts'; SELECT '*** Test ALTER ***'; -ALTER TABLE test.part_header_r1 MODIFY COLUMN y String; +ALTER TABLE part_header_r1 MODIFY COLUMN y String; SELECT '*** replica 1 ***'; -SELECT x, length(y) FROM test.part_header_r1 ORDER BY x; +SELECT x, length(y) FROM part_header_r1 ORDER BY x; SELECT '*** replica 2 ***'; -SELECT x, length(y) FROM test.part_header_r2 ORDER BY x; +SELECT x, length(y) FROM part_header_r2 ORDER BY x; SELECT '*** Test CLEAR COLUMN ***'; SET replication_alter_partitions_sync = 2; -ALTER TABLE test.part_header_r1 CLEAR COLUMN y IN PARTITION tuple(); +ALTER TABLE part_header_r1 CLEAR COLUMN y IN PARTITION tuple(); SELECT '*** replica 1 ***'; -SELECT x, length(y) FROM test.part_header_r1 ORDER BY x; +SELECT x, length(y) FROM part_header_r1 ORDER BY x; SELECT '*** replica 2 ***'; -SELECT x, length(y) FROM test.part_header_r2 ORDER BY x; +SELECT x, length(y) FROM part_header_r2 ORDER BY x; -DROP TABLE test.part_header_r1; -DROP TABLE test.part_header_r2; +DROP TABLE part_header_r1; +DROP TABLE part_header_r2; diff --git a/tests/queries/0_stateless/00926_multimatch.sql b/tests/queries/0_stateless/00926_multimatch.sql index d54e4fd2280..bf67533bb6c 100644 --- a/tests/queries/0_stateless/00926_multimatch.sql +++ b/tests/queries/0_stateless/00926_multimatch.sql @@ -89,4 +89,4 @@ SELECT [1, 2, 3, 11] = arraySort(multiMatchAllIndices('фабрикант', ['', SELECT [1] = multiMatchAllIndices(materialize('/odezhda-dlya-bega/'), ['/odezhda-dlya-bega/', 'kurtki-i-vetrovki-dlya-bega', 'futbolki-i-mayki-dlya-bega']); SELECT [] = multiMatchAllIndices(materialize('aaaa'), ['.*aa.*aaa.*', 'aaaaaa{2}', '\(aa\){3}']); SELECT 'All tests above must return 1, all tests below return something.'; -SELECT arraySort(multiMatchAllIndices(arrayJoin(['aaaa', 'aaaaaa', 'bbbb', 'aaaaaaaaaaaaaa']), ['.*aa.*aaa.*', 'aaaaaa{2}', '\(aa\){3}'])); +SELECT arraySort(multiMatchAllIndices(arrayJoin(['aaaa', 'aaaaaa', 'bbbb', 'aaaaaaaaaaaaaa']), ['.*aa.*aaa.*', 'aaaaaa{2}', '(aa){3}'])); diff --git a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql index 5b785d31497..a7ae0402b5a 100644 --- a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql +++ b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql @@ -1,214 +1,214 @@ ----- Group of very similar simple tests ------ select '----HORIZONTAL MERGE TESTS----'; -DROP TABLE IF EXISTS test.zero_rows_per_granule1; -DROP TABLE IF EXISTS test.zero_rows_per_granule2; +DROP TABLE IF EXISTS zero_rows_per_granule1; +DROP TABLE IF EXISTS zero_rows_per_granule2; -CREATE TABLE test.zero_rows_per_granule1 ( +CREATE TABLE zero_rows_per_granule1 ( p Date, k UInt64, v1 UInt64, v2 Int64 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/zero_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, write_final_mark = 0; -CREATE TABLE test.zero_rows_per_granule2 ( +CREATE TABLE zero_rows_per_granule2 ( p Date, k UInt64, v1 UInt64, v2 Int64 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/zero_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, write_final_mark = 0; -INSERT INTO test.zero_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +INSERT INTO zero_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -SYSTEM SYNC REPLICA test.zero_rows_per_granule2; +SYSTEM SYNC REPLICA zero_rows_per_granule2; SELECT 'Replica synced'; -SELECT COUNT(*) FROM test.zero_rows_per_granule1; +SELECT COUNT(*) FROM zero_rows_per_granule1; -SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database=currentDatabase() and active=1; -SELECT COUNT(*) FROM test.zero_rows_per_granule2; +SELECT COUNT(*) FROM zero_rows_per_granule2; -SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database=currentDatabase() and active=1; -INSERT INTO test.zero_rows_per_granule2 (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-19', 8, 7000, 8000); +INSERT INTO zero_rows_per_granule2 (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-19', 8, 7000, 8000); -SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database=currentDatabase() and active=1; -SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database=currentDatabase() and active=1; SELECT sleep(0.7) Format Null; -OPTIMIZE TABLE test.zero_rows_per_granule2 FINAL; +OPTIMIZE TABLE zero_rows_per_granule2 FINAL; SELECT 'Parts optimized'; -SYSTEM SYNC REPLICA test.zero_rows_per_granule1; +SYSTEM SYNC REPLICA zero_rows_per_granule1; SELECT 'Replica synced'; -SELECT COUNT(*) FROM test.zero_rows_per_granule2; +SELECT COUNT(*) FROM zero_rows_per_granule2; -SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database=currentDatabase() and active=1; -SELECT COUNT(*) FROM test.zero_rows_per_granule1; +SELECT COUNT(*) FROM zero_rows_per_granule1; -SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database=currentDatabase() and active=1; -DROP TABLE IF EXISTS test.zero_rows_per_granule1; -DROP TABLE IF EXISTS test.zero_rows_per_granule2; +DROP TABLE IF EXISTS zero_rows_per_granule1; +DROP TABLE IF EXISTS zero_rows_per_granule2; SELECT '-----'; -DROP TABLE IF EXISTS test.four_rows_per_granule1; -DROP TABLE IF EXISTS test.four_rows_per_granule2; +DROP TABLE IF EXISTS four_rows_per_granule1; +DROP TABLE IF EXISTS four_rows_per_granule2; -CREATE TABLE test.four_rows_per_granule1 ( +CREATE TABLE four_rows_per_granule1 ( p Date, k UInt64, v1 UInt64, v2 Int64 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/four_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; -CREATE TABLE test.four_rows_per_granule2 ( +CREATE TABLE four_rows_per_granule2 ( p Date, k UInt64, v1 UInt64, v2 Int64 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/four_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; -INSERT INTO test.four_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +INSERT INTO four_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -SELECT COUNT(*) FROM test.four_rows_per_granule1; +SELECT COUNT(*) FROM four_rows_per_granule1; -SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database=currentDatabase() and active=1; -SYSTEM SYNC REPLICA test.four_rows_per_granule2; +SYSTEM SYNC REPLICA four_rows_per_granule2; SELECT 'Replica synced'; -SELECT COUNT(*) FROM test.four_rows_per_granule2; +SELECT COUNT(*) FROM four_rows_per_granule2; -SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database=currentDatabase() and active=1; -DETACH TABLE test.four_rows_per_granule2; -ATTACH TABLE test.four_rows_per_granule2; +DETACH TABLE four_rows_per_granule2; +ATTACH TABLE four_rows_per_granule2; SELECT 'Table attached'; -INSERT INTO test.four_rows_per_granule2 (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-19', 8, 7000, 8000); +INSERT INTO four_rows_per_granule2 (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-19', 8, 7000, 8000); -SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database=currentDatabase() and active=1; -SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database=currentDatabase() and active=1; SELECT sleep(0.7) Format Null; -OPTIMIZE TABLE test.four_rows_per_granule2 FINAL; +OPTIMIZE TABLE four_rows_per_granule2 FINAL; SELECT 'Parts optimized'; -DETACH TABLE test.four_rows_per_granule2; +DETACH TABLE four_rows_per_granule2; -ATTACH TABLE test.four_rows_per_granule2; +ATTACH TABLE four_rows_per_granule2; -SELECT COUNT(*) FROM test.four_rows_per_granule2; +SELECT COUNT(*) FROM four_rows_per_granule2; ---SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database='test' and active=1; +--SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database=currentDatabase() and active=1; -SYSTEM SYNC REPLICA test.four_rows_per_granule1; +SYSTEM SYNC REPLICA four_rows_per_granule1; SELECT 'Replica synced'; -SELECT COUNT(*) FROM test.four_rows_per_granule1; +SELECT COUNT(*) FROM four_rows_per_granule1; ---SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database='test' and active=1; +--SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database=currentDatabase() and active=1; -DROP TABLE IF EXISTS test.four_rows_per_granule1; -DROP TABLE IF EXISTS test.four_rows_per_granule2; +DROP TABLE IF EXISTS four_rows_per_granule1; +DROP TABLE IF EXISTS four_rows_per_granule2; SELECT '-----'; -DROP TABLE IF EXISTS test.adaptive_granularity_alter1; -DROP TABLE IF EXISTS test.adaptive_granularity_alter2; +DROP TABLE IF EXISTS adaptive_granularity_alter1; +DROP TABLE IF EXISTS adaptive_granularity_alter2; -CREATE TABLE test.adaptive_granularity_alter1 ( +CREATE TABLE adaptive_granularity_alter1 ( p Date, k UInt64, v1 UInt64, v2 Int64 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/adaptive_granularity_alter', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; -CREATE TABLE test.adaptive_granularity_alter2 ( +CREATE TABLE adaptive_granularity_alter2 ( p Date, k UInt64, v1 UInt64, v2 Int64 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/adaptive_granularity_alter', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; -INSERT INTO test.adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +INSERT INTO adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -SELECT COUNT(*) FROM test.adaptive_granularity_alter1; +SELECT COUNT(*) FROM adaptive_granularity_alter1; -SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database=currentDatabase() and active=1; -SYSTEM SYNC REPLICA test.adaptive_granularity_alter2; +SYSTEM SYNC REPLICA adaptive_granularity_alter2; SELECT 'Replica synced'; -SELECT COUNT(*) FROM test.adaptive_granularity_alter2; +SELECT COUNT(*) FROM adaptive_granularity_alter2; -SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database=currentDatabase() and active=1; -ALTER TABLE test.adaptive_granularity_alter2 MODIFY COLUMN v1 Int16; +ALTER TABLE adaptive_granularity_alter2 MODIFY COLUMN v1 Int16; -DETACH TABLE test.adaptive_granularity_alter2; +DETACH TABLE adaptive_granularity_alter2; -ATTACH TABLE test.adaptive_granularity_alter2; +ATTACH TABLE adaptive_granularity_alter2; -SELECT COUNT(*) FROM test.adaptive_granularity_alter2; +SELECT COUNT(*) FROM adaptive_granularity_alter2; -SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database=currentDatabase() and active=1; -SYSTEM SYNC REPLICA test.adaptive_granularity_alter1; +SYSTEM SYNC REPLICA adaptive_granularity_alter1; SELECT 'Replica synced'; -SELECT COUNT(*) FROM test.adaptive_granularity_alter1; +SELECT COUNT(*) FROM adaptive_granularity_alter1; -SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database=currentDatabase() and active=1; -INSERT INTO test.adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 5, 3000, 4000), ('2018-05-17', 6, 5000, 6000), ('2018-05-19', 42, 42, 42); +INSERT INTO adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 5, 3000, 4000), ('2018-05-17', 6, 5000, 6000), ('2018-05-19', 42, 42, 42); -SELECT COUNT(*) FROM test.adaptive_granularity_alter1; +SELECT COUNT(*) FROM adaptive_granularity_alter1; -SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database=currentDatabase() and active=1; -SYSTEM SYNC REPLICA test.adaptive_granularity_alter2; +SYSTEM SYNC REPLICA adaptive_granularity_alter2; -SELECT COUNT(*) FROM test.adaptive_granularity_alter2; +SELECT COUNT(*) FROM adaptive_granularity_alter2; -SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database=currentDatabase() and active=1; -ALTER TABLE test.adaptive_granularity_alter1 MODIFY COLUMN v2 String; +ALTER TABLE adaptive_granularity_alter1 MODIFY COLUMN v2 String; -DETACH TABLE test.adaptive_granularity_alter1; +DETACH TABLE adaptive_granularity_alter1; -ATTACH TABLE test.adaptive_granularity_alter1; +ATTACH TABLE adaptive_granularity_alter1; -INSERT INTO test.adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 100, 1000, 'aaaa'), ('2018-05-16', 101, 3000, 'bbbb'), ('2018-05-17', 102, 5000, 'cccc'), ('2018-05-19', 103, 7000, 'dddd'); +INSERT INTO adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 100, 1000, 'aaaa'), ('2018-05-16', 101, 3000, 'bbbb'), ('2018-05-17', 102, 5000, 'cccc'), ('2018-05-19', 103, 7000, 'dddd'); SELECT sleep(0.7) Format Null; -OPTIMIZE TABLE test.adaptive_granularity_alter1 FINAL; +OPTIMIZE TABLE adaptive_granularity_alter1 FINAL; SELECT 'Parts optimized'; -SELECT k, v2 FROM test.adaptive_granularity_alter1 WHERE k >= 100 OR k = 42 ORDER BY k; +SELECT k, v2 FROM adaptive_granularity_alter1 WHERE k >= 100 OR k = 42 ORDER BY k; -SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1; +SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database=currentDatabase() and active=1; -SYSTEM SYNC REPLICA test.adaptive_granularity_alter2; +SYSTEM SYNC REPLICA adaptive_granularity_alter2; SELECT 'Replica synced'; -SELECT k, v2 FROM test.adaptive_granularity_alter2 WHERE k >= 100 OR k = 42 ORDER BY k; +SELECT k, v2 FROM adaptive_granularity_alter2 WHERE k >= 100 OR k = 42 ORDER BY k; -SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1; +SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database=currentDatabase() and active=1; -DROP TABLE IF EXISTS test.adaptive_granularity_alter1; -DROP TABLE IF EXISTS test.adaptive_granularity_alter2; +DROP TABLE IF EXISTS adaptive_granularity_alter1; +DROP TABLE IF EXISTS adaptive_granularity_alter2; diff --git a/tests/queries/0_stateless/00937_test_use_header_csv.sh b/tests/queries/0_stateless/00937_test_use_header_csv.sh index bd9204ffdcc..c0d4f2be522 100755 --- a/tests/queries/0_stateless/00937_test_use_header_csv.sh +++ b/tests/queries/0_stateless/00937_test_use_header_csv.sh @@ -3,10 +3,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.csv" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (d Date, u UInt8, str String) ENGINE = TinyLog" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS csv" +$CLICKHOUSE_CLIENT --query="CREATE TABLE csv (d Date, u UInt8, str String) ENGINE = TinyLog" -INSERT_QUERY='$CLICKHOUSE_CLIENT --query="INSERT INTO test.csv FORMAT CSVWithNames"' +INSERT_QUERY='$CLICKHOUSE_CLIENT --query="INSERT INTO csv FORMAT CSVWithNames"' USE_HEADER='--input_format_with_names_use_header=1' SKIP_UNKNOWN='--input_format_skip_unknown_fields=1' @@ -32,5 +32,5 @@ echo -ne 'str,u\nLine16,1\nLine17,2\n' | eval $INSERT_QUERY $US echo -ne 'd,str\n2019-04-18,Line18\n2019-04-18,Line19\n'| eval $INSERT_QUERY $USE_HEADER echo -ne 'unknown\n\n\n' | eval $INSERT_QUERY $USE_HEADER $SKIP_UNKNOWN -$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.csv" +$CLICKHOUSE_CLIENT --query="SELECT * FROM csv" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS csv" diff --git a/tests/queries/0_stateless/00942_mutate_index.sh b/tests/queries/0_stateless/00942_mutate_index.sh index 30ac7e8821b..d14c5659fc9 100755 --- a/tests/queries/0_stateless/00942_mutate_index.sh +++ b/tests/queries/0_stateless/00942_mutate_index.sh @@ -4,11 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh . $CURDIR/mergetree_mutations.lib -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" $CLICKHOUSE_CLIENT -n --query=" -CREATE TABLE test.minmax_idx +CREATE TABLE minmax_idx ( u64 UInt64, i64 Int64, @@ -19,7 +19,7 @@ ORDER BY u64 SETTINGS index_granularity = 2;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES +$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES (0, 1, 1), (1, 1, 2), (2, 1, 3), @@ -31,13 +31,13 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES (8, 1, 9), (9, 1, 10)" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 1;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 5;" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx UPDATE i64 = 5 WHERE i64 = 1;" -wait_for_mutation "minmax_idx" "mutation_2.txt" "test" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx UPDATE i64 = 5 WHERE i64 = 1;" +wait_for_mutation "minmax_idx" "mutation_2.txt" "$CLICKHOUSE_DATABASE" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 1;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 5;" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" +$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx" diff --git a/tests/queries/0_stateless/00943_materialize_index.sh b/tests/queries/0_stateless/00943_materialize_index.sh index b406f3894eb..c3d5a6bb219 100755 --- a/tests/queries/0_stateless/00943_materialize_index.sh +++ b/tests/queries/0_stateless/00943_materialize_index.sh @@ -4,11 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh . $CURDIR/mergetree_mutations.lib -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" $CLICKHOUSE_CLIENT -n --query=" -CREATE TABLE test.minmax_idx +CREATE TABLE minmax_idx ( u64 UInt64, i64 Int64, @@ -19,7 +19,7 @@ ORDER BY u64 SETTINGS index_granularity = 2;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES +$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES (0, 2, 1), (1, 1, 1), (2, 1, 1), @@ -31,35 +31,35 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES (8, 1, 2), (9, 1, 2)" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT -n --query=" -ALTER TABLE test.minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;" +ALTER TABLE minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" -wait_for_mutation "minmax_idx" "mutation_3.txt" "test" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" +wait_for_mutation "minmax_idx" "mutation_3.txt" "$CLICKHOUSE_DATABASE" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 2;" -wait_for_mutation "minmax_idx" "mutation_4.txt" "test" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 2;" +wait_for_mutation "minmax_idx" "mutation_4.txt" "$CLICKHOUSE_DATABASE" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 1;" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 2;" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx CLEAR INDEX idx IN PARTITION 1;" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx CLEAR INDEX idx IN PARTITION 2;" sleep 0.5 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx;" -wait_for_mutation "minmax_idx" "mutation_5.txt" "test" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx;" +wait_for_mutation "minmax_idx" "mutation_5.txt" "$CLICKHOUSE_DATABASE" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" +$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx" diff --git a/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/tests/queries/0_stateless/00944_clear_index_in_partition.sh index 9f53ec16c35..d6ca4106a0e 100755 --- a/tests/queries/0_stateless/00944_clear_index_in_partition.sh +++ b/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -4,11 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh . $CURDIR/mergetree_mutations.lib -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" $CLICKHOUSE_CLIENT -n --query=" -CREATE TABLE test.minmax_idx +CREATE TABLE minmax_idx ( u64 UInt64, i64 Int64, @@ -20,7 +20,7 @@ ORDER BY u64 SETTINGS index_granularity = 2;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES +$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES (0, 2, 1), (1, 1, 1), (2, 1, 1), @@ -32,19 +32,19 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES (8, 1, 2), (9, 1, 2)" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" -wait_for_mutation "minmax_idx" "mutation_3.txt" "test" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" +wait_for_mutation "minmax_idx" "mutation_3.txt" "$CLICKHOUSE_DATABASE" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" +$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx" diff --git a/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh b/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh index 28e7bd0b94e..b8b133cde3d 100755 --- a/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh @@ -4,12 +4,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh . $CURDIR/mergetree_mutations.lib -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions1;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions2;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS indices_mutaions1;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS indices_mutaions2;" $CLICKHOUSE_CLIENT -n --query=" -CREATE TABLE test.indices_mutaions1 +CREATE TABLE indices_mutaions1 ( u64 UInt64, i64 Int64, @@ -19,7 +19,7 @@ CREATE TABLE test.indices_mutaions1 PARTITION BY i32 ORDER BY u64 SETTINGS index_granularity = 2; -CREATE TABLE test.indices_mutaions2 +CREATE TABLE indices_mutaions2 ( u64 UInt64, i64 Int64, @@ -31,7 +31,7 @@ ORDER BY u64 SETTINGS index_granularity = 2;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.indices_mutaions1 VALUES +$CLICKHOUSE_CLIENT --query="INSERT INTO indices_mutaions1 VALUES (0, 2, 1), (1, 1, 1), (2, 1, 1), @@ -43,18 +43,18 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.indices_mutaions1 VALUES (8, 1, 2), (9, 1, 2)" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2 +$CLICKHOUSE_CLIENT --query="ALTER TABLE indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2 +$CLICKHOUSE_CLIENT --query="ALTER TABLE indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.indices_mutaions1" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.indices_mutaions2" +$CLICKHOUSE_CLIENT --query="DROP TABLE indices_mutaions1" +$CLICKHOUSE_CLIENT --query="DROP TABLE indices_mutaions2" diff --git a/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh b/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh index 19d2b487955..067c19658ab 100755 --- a/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh +++ b/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh @@ -3,17 +3,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.a;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.b;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS a;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS b;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.a (x UInt64) ENGINE = Memory;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.b (x UInt64) ENGINE = Memory;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE a (x UInt64) ENGINE = Memory;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE b (x UInt64) ENGINE = Memory;" function thread1() { for attempt_thread1 in {1..10} do - seq 1 500000 | $CLICKHOUSE_CLIENT --query_id=11 --query="INSERT INTO test.a FORMAT TSV" & + seq 1 500000 | $CLICKHOUSE_CLIENT --query_id=11 --query="INSERT INTO a FORMAT TSV" & while true; do $CLICKHOUSE_CLIENT --query="KILL QUERY WHERE query_id='11' SYNC" | grep -q "cant_cancel" && sleep .1 || break ||: done @@ -27,7 +27,7 @@ function thread2() { for attempt_thread2 in {1..10} do - seq 1 500000 | $CLICKHOUSE_CLIENT --query_id=22 --query="INSERT INTO test.b FORMAT TSV" & + seq 1 500000 | $CLICKHOUSE_CLIENT --query_id=22 --query="INSERT INTO b FORMAT TSV" & while true; do $CLICKHOUSE_CLIENT --query="KILL QUERY WHERE query_id='22' SYNC" | grep -q "cant_cancel" && sleep .1 || break ||: done @@ -46,5 +46,5 @@ bash -c thread2 > /dev/null 2>&1 & wait echo OK -$CLICKHOUSE_CLIENT --query "DROP TABLE test.a" -$CLICKHOUSE_CLIENT --query "DROP TABLE test.b" +$CLICKHOUSE_CLIENT --query "DROP TABLE a" +$CLICKHOUSE_CLIENT --query "DROP TABLE b" diff --git a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 019e95cb359..bd132202979 100644 --- a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -4,7 +4,7 @@ a 2 1 0 a 3 1 0 b 13 2 0 b 15 2 0 -SELECT \n co, \n co2, \n co3, \n num\nFROM \n(\n SELECT \n co, \n co2, \n co3, \n count() AS num\n FROM \n (\n SELECT \n 1 AS co, \n 2 AS co2, \n 3 AS co3\n )\n GROUP BY \n co, \n co2, \n co3\n WITH CUBE\n HAVING (co != 0) AND (co2 != 2)\n)\nWHERE (co != 0) AND (co2 != 2) +SELECT \n co, \n co2, \n co3, \n num\nFROM \n(\n SELECT \n co, \n co2, \n co3, \n count() AS num\n FROM \n (\n SELECT \n 1 AS co, \n 2 AS co2, \n 3 AS co3\n )\n GROUP BY \n co, \n co2, \n co3\n WITH CUBE\n HAVING (co2 != 2) AND (co != 0)\n)\nWHERE (co != 0) AND (co2 != 2) 1 0 3 1 1 0 0 1 SELECT alias AS name\nFROM \n(\n SELECT name AS alias\n FROM system.settings\n WHERE alias = \'enable_optimize_predicate_expression\'\n)\nANY INNER JOIN \n(\n SELECT name\n FROM system.settings\n) USING (name)\nWHERE name = \'enable_optimize_predicate_expression\' @@ -26,3 +26,4 @@ SELECT dummy\nFROM \n(\n SELECT dummy\n FROM system.one\n WHERE arrayMa 0 SELECT \n id, \n value, \n value_1\nFROM \n(\n SELECT \n 1 AS id, \n 2 AS value\n)\nALL INNER JOIN \n(\n SELECT \n 1 AS id, \n 3 AS value_1\n) USING (id)\nWHERE arrayMap(x -> ((x + value) + value_1), [1]) = [6] 1 2 3 +SELECT dummy\nFROM system.one\nWHERE (dummy > 0) AND (dummy < 0) diff --git a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index db6c78e3cc6..18552a6591d 100644 --- a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -74,3 +74,13 @@ SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [ ANALYZE SELECT * FROM (SELECT 1 AS id, 2 AS value) INNER JOIN (SELECT 1 AS id, 3 AS value_1) USING id WHERE arrayMap(x -> x + value + value_1, [1]) = [6]; SELECT * FROM (SELECT 1 AS id, 2 AS value) INNER JOIN (SELECT 1 AS id, 3 AS value_1) USING id WHERE arrayMap(x -> x + value + value_1, [1]) = [6]; + +-- check order is preserved +ANALYZE SELECT * FROM system.one HAVING dummy > 0 AND dummy < 0; + +-- from #10613 +SELECT name, count() AS cnt +FROM remote('127.{1,2}', system.settings) +GROUP BY name +HAVING (max(value) > '9') AND (min(changed) = 0) +FORMAT Null; diff --git a/tests/queries/0_stateless/01075_allowed_client_hosts.reference b/tests/queries/0_stateless/01075_allowed_client_hosts.reference index 73f54c6027a..3fdea9d1cda 100644 --- a/tests/queries/0_stateless/01075_allowed_client_hosts.reference +++ b/tests/queries/0_stateless/01075_allowed_client_hosts.reference @@ -8,10 +8,10 @@ CREATE USER test_user_01075 HOST LOCAL, IP \'2001:db8:11a3:9d7:1f34:8a2e:7a0:765 CREATE USER test_user_01075 HOST LOCAL CREATE USER test_user_01075 HOST NONE CREATE USER test_user_01075 HOST LIKE \'@.somesite.com\' -CREATE USER test_user_01075 HOST REGEXP \'.*.anothersite.com\' -CREATE USER test_user_01075 HOST REGEXP \'.*.anothersite.com\', \'.*.anothersite.org\' -CREATE USER test_user_01075 HOST REGEXP \'.*.anothersite2.com\', \'.*.anothersite2.org\' -CREATE USER test_user_01075 HOST REGEXP \'.*.anothersite3.com\', \'.*.anothersite3.org\' +CREATE USER test_user_01075 HOST REGEXP \'.*\\\\.anothersite\\\\.com\' +CREATE USER test_user_01075 HOST REGEXP \'.*\\\\.anothersite\\\\.com\', \'.*\\\\.anothersite\\\\.org\' +CREATE USER test_user_01075 HOST REGEXP \'.*\\\\.anothersite2\\\\.com\', \'.*\\\\.anothersite2\\\\.org\' +CREATE USER test_user_01075 HOST REGEXP \'.*\\\\.anothersite3\\\\.com\', \'.*\\\\.anothersite3\\\\.org\' CREATE USER `test_user_01075_x@localhost` HOST LOCAL CREATE USER test_user_01075_x CREATE USER `test_user_01075_x@192.168.23.15` HOST LIKE \'192.168.23.15\' diff --git a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.reference b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.reference index 337ab7fe2e3..89a55127234 100644 --- a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.reference +++ b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.reference @@ -1 +1,5 @@ +replica_01108_1 +replica_01108_2 +replica_01108_3 +replica_01108_4 1180 40 diff --git a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh index 01c9ef0a277..65b738aed8e 100755 --- a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh +++ b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh @@ -53,7 +53,7 @@ export -f rename_thread_2; export -f restart_thread_1; export -f restart_thread_2; -TIMEOUT=30 +TIMEOUT=10 timeout $TIMEOUT bash -c rename_thread_1 2> /dev/null & timeout $TIMEOUT bash -c rename_thread_2 2> /dev/null & @@ -63,6 +63,12 @@ timeout $TIMEOUT bash -c restart_thread_2 2> /dev/null & wait sleep 3 +for i in `seq 4`; do + $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA replica_01108_$i" >/dev/null 2>&1 + $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA replica_01108_${i}_tmp" >/dev/null 2>&1 +done + +$CLICKHOUSE_CLIENT -q "SELECT replaceOne(name, '_tmp', '') FROM system.tables WHERE database = currentDatabase() AND match(name, '^replica_01108_')" $CLICKHOUSE_CLIENT -q "SELECT sum(n), count(n) FROM merge(currentDatabase(), '^replica_01108_') GROUP BY position(_table, 'tmp')" diff --git a/tests/queries/0_stateless/01122_totals_rollup_having_block_header.sql b/tests/queries/0_stateless/01122_totals_rollup_having_block_header.sql index 4f4f3355912..6fb877c350a 100644 --- a/tests/queries/0_stateless/01122_totals_rollup_having_block_header.sql +++ b/tests/queries/0_stateless/01122_totals_rollup_having_block_header.sql @@ -1,14 +1,14 @@ -DROP TABLE IF EXISTS test.rollup_having; -CREATE TABLE test.rollup_having ( +DROP TABLE IF EXISTS rollup_having; +CREATE TABLE rollup_having ( a Nullable(String), b Nullable(String) ) ENGINE = Memory; -INSERT INTO test.rollup_having VALUES (NULL, NULL); -INSERT INTO test.rollup_having VALUES ('a', NULL); -INSERT INTO test.rollup_having VALUES ('a', 'b'); +INSERT INTO rollup_having VALUES (NULL, NULL); +INSERT INTO rollup_having VALUES ('a', NULL); +INSERT INTO rollup_having VALUES ('a', 'b'); -SELECT a, b, count(*) FROM test.rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL; -- { serverError 48 } -SELECT a, b, count(*) FROM test.rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL and b IS NOT NULL; -- { serverError 48 } +SELECT a, b, count(*) FROM rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL; -- { serverError 48 } +SELECT a, b, count(*) FROM rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL and b IS NOT NULL; -- { serverError 48 } -DROP TABLE test.rollup_having; +DROP TABLE rollup_having; diff --git a/tests/queries/0_stateless/01124_view_bad_types.sql b/tests/queries/0_stateless/01124_view_bad_types.sql index 81fc53930c1..715f7b375f1 100644 --- a/tests/queries/0_stateless/01124_view_bad_types.sql +++ b/tests/queries/0_stateless/01124_view_bad_types.sql @@ -1,11 +1,11 @@ -DROP TABLE IF EXISTS test.table; -CREATE TABLE test.table (x UInt16) ENGINE = TinyLog; -INSERT INTO test.table SELECT * FROM system.numbers LIMIT 10; +DROP TABLE IF EXISTS source_table; +CREATE TABLE source_table (x UInt16) ENGINE = TinyLog; +INSERT INTO source_table SELECT * FROM system.numbers LIMIT 10; -DROP TABLE IF EXISTS test.view; -CREATE VIEW test.view (x UInt64) AS SELECT * FROM test.table; +DROP TABLE IF EXISTS dest_view; +CREATE VIEW dest_view (x UInt64) AS SELECT * FROM source_table; -SELECT x, any(x) FROM test.view GROUP BY x ORDER BY x; +SELECT x, any(x) FROM dest_view GROUP BY x ORDER BY x; -DROP TABLE test.view; -DROP TABLE test.table; +DROP TABLE dest_view; +DROP TABLE source_table; diff --git a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.reference b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.reference index ed22b7e1e35..e4872ddeddd 100644 --- a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.reference +++ b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.reference @@ -1,4 +1,4 @@ 0 0 -100000 -200000 +800000 +1600000 diff --git a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh index 7e08c930f67..c9a4de647b9 100755 --- a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh +++ b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh @@ -54,7 +54,7 @@ EOL echo "create table null_01278 as data_01278 Engine=Null();" | execute for i in $(seq 1 $TEST_01278_PARTS); do echo "create table part_01278_$i as data_01278 Engine=Buffer(currentDatabase(), null_01278, 1, 86400, 86400, 1e5, 1e6, 10e6, 100e6);" - echo "create materialized view mv_01278_$i to part_01278_$i as select * from data_01278 where key%$TEST_01278_PARTS+1 == $i;" + echo "create materialized view mv_01278_$i to part_01278_$i as select * from data_01278 where key%$TEST_01278_PARTS+1 != $i;" done | execute echo "create table out_01278 as data_01278 Engine=Merge(currentDatabase(), 'part_01278_');" | execute diff --git a/tests/queries/0_stateless/01284_escape_sequences_php_mysql_style.reference b/tests/queries/0_stateless/01284_escape_sequences_php_mysql_style.reference new file mode 100644 index 00000000000..19a5fc680ff --- /dev/null +++ b/tests/queries/0_stateless/01284_escape_sequences_php_mysql_style.reference @@ -0,0 +1,3 @@ +a\\_\\c\\l\\i\\c\\k\\h\\o\\u\\s a\\_\\c\\l\\i\\c\\k\\h\\o\\u\\s\\e +1 0 1 1 +1 1 0 0 0 diff --git a/tests/queries/0_stateless/01284_escape_sequences_php_mysql_style.sql b/tests/queries/0_stateless/01284_escape_sequences_php_mysql_style.sql new file mode 100644 index 00000000000..36ad7b4506c --- /dev/null +++ b/tests/queries/0_stateless/01284_escape_sequences_php_mysql_style.sql @@ -0,0 +1,6 @@ +SELECT 'a\_\c\l\i\c\k\h\o\u\s\e', 'a\\_\\c\\l\\i\\c\\k\\h\\o\\u\\s\\e'; +select 'aXb' like 'a_b', 'aXb' like 'a\_b', 'a_b' like 'a\_b', 'a_b' like 'a\\_b'; +SELECT match('Hello', '\w+'), match('Hello', '\\w+'), match('Hello', '\\\w+'), match('Hello', '\w\+'), match('Hello', 'w+'); + +SELECT match('Hello', '\He\l\l\o'); -- { serverError 427 } +SELECT match('Hello', '\H\e\l\l\o'); -- { serverError 427 } diff --git a/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.reference b/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.reference new file mode 100644 index 00000000000..0d221d16995 --- /dev/null +++ b/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.reference @@ -0,0 +1,20 @@ +INSERT +1 0 +1 1 +1 1 +INSERT +1 0 +1 1 +1 0 +1 1 +1 2 +1 3 +1 1 +1 1 +1 3 +OPTIMIZE +1 3 +1 3 +OPTIMIZE +1 3 +1 3 diff --git a/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.sql b/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.sql new file mode 100644 index 00000000000..110c5b65cab --- /dev/null +++ b/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.sql @@ -0,0 +1,34 @@ +DROP TABLE IF EXISTS data_01285; + +SET max_threads=1; + + +CREATE TABLE data_01285 ( + key Int, + value SimpleAggregateFunction(max, Nullable(Int)), + INDEX value_idx assumeNotNull(value) TYPE minmax GRANULARITY 1 +) +ENGINE=AggregatingMergeTree() +ORDER BY key; + +SELECT 'INSERT'; +INSERT INTO data_01285 SELECT 1, number FROM numbers(2); +SELECT * FROM data_01285; +SELECT * FROM data_01285 WHERE assumeNotNull(value) = 1; +SELECT 'INSERT'; +INSERT INTO data_01285 SELECT 1, number FROM numbers(4); +SELECT * FROM data_01285; +SELECT * FROM data_01285 WHERE assumeNotNull(value) = 1; +SELECT * FROM data_01285 WHERE assumeNotNull(value) = 3; +SELECT 'OPTIMIZE'; +OPTIMIZE TABLE data_01285 FINAL; +SELECT * FROM data_01285; +-- before the fix value_idx contains one range {0, 0} +-- and hence cannot find these record. +SELECT * FROM data_01285 WHERE assumeNotNull(value) = 3; +-- one more time just in case +SELECT 'OPTIMIZE'; +OPTIMIZE TABLE data_01285 FINAL; +SELECT * FROM data_01285; +-- and this passes even without fix. +SELECT * FROM data_01285 WHERE assumeNotNull(value) = 3; diff --git a/tests/queries/1_stateful/00155_date_datetime_key_condition.reference b/tests/queries/0_stateless/01285_date_datetime_key_condition.reference similarity index 100% rename from tests/queries/1_stateful/00155_date_datetime_key_condition.reference rename to tests/queries/0_stateless/01285_date_datetime_key_condition.reference diff --git a/tests/queries/0_stateless/01285_date_datetime_key_condition.sql b/tests/queries/0_stateless/01285_date_datetime_key_condition.sql new file mode 100644 index 00000000000..fe1454cd7c4 --- /dev/null +++ b/tests/queries/0_stateless/01285_date_datetime_key_condition.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS date_datetime_key_condition; + +CREATE TABLE date_datetime_key_condition (dt DateTime) ENGINE = MergeTree() ORDER BY dt; +INSERT INTO date_datetime_key_condition VALUES ('2020-01-01 00:00:00'), ('2020-01-01 10:00:00'), ('2020-01-02 00:00:00'); + +-- partial +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt > toDate('2020-01-01') AND dt < toDate('2020-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt >= toDate('2020-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt < toDate('2020-01-02'); + +-- inside +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt > toDate('2019-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt < toDate('2021-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt >= toDate('2019-01-02') AND dt < toDate('2021-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt > toDate('2019-01-02') OR dt <= toDate('2021-01-02'); + +-- outside +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt < toDate('2019-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt > toDate('2021-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt < toDate('2019-01-02') OR dt > toDate('2021-01-02'); + +DROP TABLE date_datetime_key_condition; diff --git a/tests/queries/0_stateless/01285_engine_join_donmikel.reference b/tests/queries/0_stateless/01285_engine_join_donmikel.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/01285_engine_join_donmikel.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/01285_engine_join_donmikel.sh b/tests/queries/0_stateless/01285_engine_join_donmikel.sh new file mode 100755 index 00000000000..8a867921dbe --- /dev/null +++ b/tests/queries/0_stateless/01285_engine_join_donmikel.sh @@ -0,0 +1,65 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --multiquery --query " +DROP TABLE IF EXISTS NmSubj; +DROP TABLE IF EXISTS events; + +create table NmSubj +( + NmId UInt32, + SubjectId UInt32 +) + engine = Join(All, inner, NmId); + +create table events +( + EventDate Date, + EventDateTime DateTime, + EventId String, + SessionId FixedString(36), + PageViewId FixedString(36), + UserId UInt64, + UniqUserId FixedString(36), + UrlReferrer String, + Param1 String, + Param2 String, + Param3 String, + Param4 String, + Param5 String, + Param6 String, + Param7 String, + Param8 String, + Param9 String, + Param10 String, + ApplicationType UInt8, + Locale String, + Lang String, + Version String, + Path String, + QueryString String, + UserHostAddress UInt32 +) + engine = MergeTree() + PARTITION BY (toYYYYMM(EventDate), EventId) + ORDER BY (EventId, EventDate, Locale, ApplicationType, intHash64(UserId)) + SAMPLE BY intHash64(UserId) + SETTINGS index_granularity = 8192; + +insert into NmSubj values (1, 1), (2, 2), (3, 3); +" + +$CLICKHOUSE_CLIENT --query "INSERT INTO events FORMAT TSV" < ${CURDIR}/01285_engine_join_donmikel.tsv + +$CLICKHOUSE_CLIENT --query " +SELECT toInt32(count() / 24) as Count +FROM events as e INNER JOIN NmSubj as ns +ON ns.NmId = toUInt32(e.Param1) +WHERE e.EventDate = today() - 7 AND e.EventId = 'GCO' AND ns.SubjectId = 2073" + +$CLICKHOUSE_CLIENT --multiquery --query " +DROP TABLE NmSubj; +DROP TABLE events; +" diff --git a/tests/queries/0_stateless/01285_engine_join_donmikel.tsv b/tests/queries/0_stateless/01285_engine_join_donmikel.tsv new file mode 100644 index 00000000000..a6f47144ff2 --- /dev/null +++ b/tests/queries/0_stateless/01285_engine_join_donmikel.tsv @@ -0,0 +1,100 @@ +2020-02-17 2020-02-17 19:40:09 EMC wFxiX8iZ3Vns8pO6bx0WvYftGEOM9WMff Ku5ofnKwBaQ0CtcdZa6sAnSQbyufu0zfd 0 X16qWFb_h8JONLGxs7oiXFZTdBFfTniRMG mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=33&userId=274574634&email=Z21haWwucnU%3D&source=mn 657583482 +2020-02-17 2020-02-17 20:23:53 EMC foco1fj5Li6ey>MSfn?jKHBBdiwKXiBmy7Ni ICZBjN7en5snnszHKbTVUU*xyOhuZEgI1EK 0 I7XAKnCMsD9tdFHmmcCrI~KqiKQdo7Gxd5 mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=E17727463464LnJ1&source=mn 2992671877 +2020-02-17 2020-02-17 19:49:04 EMC q3AxEdMQhsr1NewVSKjHzYvh63XdxtGcBZX GVGMPZhenJd83JarI7Qicq5BKRfijT0ls 18536997 D_myvD3U4bmy5anwdNRFaxm KNoDfcCDL9U6 http&o=mail.ru/redir?u=http&o=mail.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6userId mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=AD0581918974621haWwuY29t&source=mn 1656424571 +2020-02-17 2020-02-17 21:55:32 EMC ef7tm7k7Yvpb4XIfvf7ON-s4Xv3ztD7SkB KIDkUzEgcAMu%ObAQAkI3jmm3638bSnX0 22341123 6NcB26yJrLhWgldC_HvV2fUOHNLRFvkLbA mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=E943634627468871D581940274695&userId=2746887723D&source=mn 1429586160 +2020-02-17 2020-02-17 21:37:47 EMC 3Y_KZLEJgn4bw?omv4EizIs1lUMz0Li0Ws0 I3QaVG9UvXWvWReLSHuDCs3DdgTbX8av6 28213800 KnoofunrqFKdIta8pMuR!FNX2yu1I6gNo https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://click.mail&v=2&s=3745772 mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=5YWxpbC5jb20%3D&source=mn 872136829 +2020-02-17 2020-02-17 19:15:34 EMC JcHEtSzMb70K}04FW#ShpZw1ot_fb7wxj8 KG9bmRo6yweN2EX48CUBQcsOqzbHn6g3u 30047107 UKb5rJ5D\bFncBHxCioYswtJKAlv6Ua5MVSrj mn 48 Ru ru 7.0.32 /confirm/vipstatus ?key=75ydQ%3D%3D&source=mn 626057004 +2020-02-17 2020-02-17 18:15:04 EMC _sjptcVU5CF4sTnXh0C6DCYcOo0rNowf 0a0AcabOllf7\rN9X9aa7rVmJ0J1oZGJY2 30648090 Z2K_4kffe21iW1hl2jby1RXAAh5XtPLP3 mn 48 Ru ru 7.0.32 /confirm/vipstatus ?key=5&ts=15819420%3D&source=mn 3682305993 +2020-02-17 2020-02-17 16:10:20 EMC WM44OUvEZse_9hQ0NJsn0k47pX6_UnaS7 LgCWwTMZ+GwiQEbNT95zUnkMCXYqPs1Xt1 0 QUuSVpqq#SgMdveJDW0bQ6s8jn8MLXzL1w mn 187 Ru ru 7.0.32 /confirm/vipstatus ?key=CCD&source=mn 726391852 +2020-02-17 2020-02-17 17:30:44 EMC PGfQVZHuAY0A_kUA\04ceYRjWqgPJtfhCn 4TR9oRKCNrouy:zJBLBi8J4=u7Mkt9jgzgmB 0 xbCoBX9pTDaWiRxW@F3fOgKD9lX58oWsc mn 187 Ru ru 7.0.32 /confirm/vipstatus ?key=DB3D&source=pcap 2498835913 +2020-02-17 2020-02-17 19:10:51 EMC gnd66xWlKf2H2tiZdbivjoz4ILmJ4lp_ ysgw6WLdbNWSHQrybWjhH9uJqnlZMWAv 0 nDQDqsnm/dlpbFMPJE93b6EETHzFxUSzVv pcap 187 Ru ru 7.0.32 /confirm/vipstatus ?key=4&userId=274634688&userId=274688&userId=27465D85CD4F634627468877266792YXJ1&source=mn 816668334 +2020-02-17 2020-02-17 20:52:00 EMC pBzkcTpn50WpO]BZH7^js6TpPGp7Uid7mFg jQVdKDl4+Sp_j5CZc PtVoMGTIGOfYO6Q2 0 aQGyrF9s3qUMPq_dUnOiM6QVMX5PNQ_eiV pcap 187 Ru ru 7.0.32 /confirm/vipstatus ?key=50&email=YWlsLmNvbQ%3D%3D&source=pcap 1770779506 +2020-02-17 2020-02-17 21:31:14 EMC cyTu3wNRCI1TzkxCybFNz\rCkrdDZQxYtlW gmtJF_hWE8M79bWSaKkn8UT5Xdwfb0uK8T 0 RSJUU0SBDIXjpYORna59cWouc20W5VIP mn 187 Ru ru 7.0.32 /confirm/vipstatus ?key=4CD%3D&source=mn 3448881839 +2020-02-17 2020-02-17 17:15:46 EMC I3bvr8ce2SU_&UuDX4J6KRJl5uipnL2Or myxD8nQXFYnKqBoPGGo4Hy6hBTXRL04K 0 ntHjLMvhnb7b09Tu7jRkM8KgS5ni0w4oH mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=36274695&userId=2560&userId=27463465&userId=27468872EDB908&ts=158193819382 2447621965 +2020-02-17 2020-02-17 18:21:21 EMC _468tOVLPpIkgKvPGX8ypjfTZV09k8w5 0KakGIWk9uaL0Rv3]1T5wPeHPklVWzhytg 0 QAJxAuQADoEQqY0HUT56P0O4K5y55K_L https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://mail&v=2&s=92Ymail&v=2&s=3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D66email&v=2&s=93D158192YWlsLnJ1%26source%3Dmn&c=swm&r=https%3A%2F%2Fwww.wildberries.ru mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=5819392158193634&userId=274688&ts=15819408BD1haWwucnU%3D&source=mn 1547300784 +2020-02-17 2020-02-17 19:07:33 EMC Y8yBoZ7Cp2GUTQ8srLMZJFIZ85WgiBeau wmIgwW88aSGvvjKwV,8EooQonI_90taLrh 0 KXVqerLQyaMaMtuT1FzUjrLev8Fw11mkfO pcap 1 Ru ru 7.0.32 /confirm/vipstatus ?key=72846239707&userId=274688&ts=1581921haWwucnU%3D&source=pcap 2777789095 +2020-02-17 2020-02-17 11:27:36 EMC n66D1ZN9C6y7LASQucHNdkw_QseVRnt32 yeqFeZFa,7cQ_JyTO3yJJKS QaCKEWERPx4Z 24388614 6h91HeiPyIKw8yV0p\tcaK99w_zmnmXU77RI https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://click.mail.ru/redir?u=https://click.mail&v=2&s=3D158192YWlsLnJ1%26source%3Dmn&c=swm&r=http&o=mail.ru/redir?u=https://click.mail.ru/redir?u=https://mail%3DYWlsLnJ1%26source mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=6167462E4CD570A0312D6BF1BtYWlsLnJ1&source=mn 662531688 +2020-02-17 2020-02-17 14:11:06 EMC jamANU2zei_lBkfEk_Im5ow0gWB5G0gL1S So7Jc6gheYgclfoq8l0hqsPExrMNSXPLV 20685668 nwRXATVEY2PaYr34qwtZX0Q6ah_sceXjbL https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=http&o=mail&v=2&s=9444%26email.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69103Dmn&c=swm&r=http&o=mail&v=2&s=92YWlsLnJ1%26source%3Dpc&c=swm&r=https mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=420%3D&source=mn 3233174651 +2020-02-17 2020-02-17 14:32:04 EMC lsruHZ0Ywg1yDdrio.j5piNMTnqNxeaOQJ G6K62xDbm9aJ_kNlLqEo4AhVf46wDtWi 32046518 wgpf1yqVtVhLL7yT9sp1RbqaXGF81GmSiQ6 mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=1581943D&source=pcap 3183159832 +2020-02-17 2020-02-17 15:11:37 EMC 5EW8o4vD850H>TQam3NA2@egQm32jpASIF KJKN5QAHdoqw_RGol5hLvSq6753cKLswM 32971244 8Vh3DVlCjfXs[5hqVMGDDP2iBjoiqhoACB https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://mail&v=2&s=3D27457AD3D274576userId%3D27461haWwucnU%253D%26source%3Dmn&c=swm&r=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6ts%3D158193Fkey%3D669%26email&v=2&s=3Fkey%3D69D01 mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=5634&userId=274695&ts=15819440446274695&userId=27465&email=Z21haWwucnU%3D&source=pcap 574798405 +2020-02-17 2020-02-17 13:59:25 EMC i6LAlVL3U0GUFFtjiRgfPyjyattjzds6y2 IYv_jZ2k+bNO33nbLKRRGcJvTHVIueEDI 21779069 eUgOCUGbEZM15\'Of0Vz6zQmAT0T0zX3XSO http&o=mail.ru/redir?u=http&o=mail.ru/redir?u=https%3A%2F%2Fwww.wildberries mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=D1F9sail=YWxpbC5ydQ%3D%3D&source=mn 3475274476 +2020-02-17 2020-02-17 06:40:47 EMC A_3372QT5xhTmjNrokk3yZCtuOm52FfZT oOD7es9hGUs7jB3_)jVKh3Oo10IQvAu01 27168564 UYEHX0itbLKZjChWxLsP4a7LUXQLX4iB mn 48 Ru ru 7.0.32 /confirm/vipstatus ?key=467468874634&email=dG92&userId=274695&email=email=a2B7AD60&email=bWFpbC5ydQ%3D%3D&source=mn 3084609784 +2020-02-17 2020-02-17 14:10:13 EMC WSlfONgzmGj8kylbItxLW8MX4u60oapxy U8TMovlAcJTcA~u3KoOFG7pchP3CBPbcB4 28904171 nBfRyelBAkOfR70tIzBusuS0M8_KbRsOGI pcap 187 Ru ru 7.0.32 /confirm/vipstatus ?key=4462F5B37274695&email=Ymail=Z21haWwucnU%3D&source=mn 2020344465 +2020-02-17 2020-02-17 01:43:57 EMC khjsNWq7 Iq564D1u\'vWuKZHN4kfq0Kyfv hcAK5RGOjZHYbFJwufN7\tZeXCRWAEKoT2 0 G0Zi9TAk`t0aUcHRiOcb1|RgoT8xMOH4KV mn 187 Ru ru 7.0.32 /confirm/vipstatus ?key=1581940095&ts=158193819382&userId=27465D8B8F1936256C35B4B466&userId=2746887C1A97&userId=274695&userId=274634&userId 902363005 +2020-02-17 2020-02-17 02:17:35 EMC H56Q4qwa\'eYiFCmGR40Pp7,m6fDiE2nJSmB Efk7W3ZrgNsEi.anvkrteuN>lF2UlX0RZFzr 0 QhyRjVGEsD_4YaQiaLorQ6y0fbCxkEVzQ5 pcap 187 Ru ru 7.0.32 /confirm/vipstatus ?key=44695&email=YW5hcnU%3D&source=mn 2161496531 +2020-02-17 2020-02-17 03:20:48 EMC 1wBv9Q7rDQuImuif>0v1t8OXWSIXymKrI sNujW7iatZLPxEOpI1fZdT4p0QRaotsXPZ 0 LSKNxAcrTMsADK9VI7AMqm05blzLWXCJ05 pcap 187 Ru ru 7.0.32 /confirm/vipstatus ?key=AE1FAC2BFA1E75hce=mn 985607445 +2020-02-17 2020-02-17 06:18:16 EMC 6dqrbyyKHWPBEyPZGbJRKjxtmt0Aq8IsON k57VHYX59Ws9u[L3nNEruOGCVgGH_l27J1 0 _lwuZQycqGeyMxkRSM7pR1lHNU6_V0nVWG pcap 187 Ru ru 7.0.32 /confirm/vipstatus ?key=B27465D88&userId=274572BE3274577273AF0&email=a274578&userId=274692&userId=274688&userId=27465D8192 130983489 +2020-02-17 2020-02-17 09:20:13 EMC UCLTP56_YcQ9rWVJa8rif2Aq5NNqVThlwH 7Rx9OsBs7BemVxuArwZ7N5BqcFwK7CSI 0 JOIbkOPdPPbPCtqRV6wQHsdWYNnmarCxm mn 187 Ru ru 7.0.32 /confirm/vipstatus ?key=A6E819420%3D&source=pcap 2567294448 +2020-02-17 2020-02-17 13:05:00 EMC 28AZ4n0zpsVuwzenZ6*gd0VNMjnT8Kmfncv PFiGZRLTjK1CxfLJajtGVFOATfWnTE00 0 QYPUzvwV=w8ev0E77B|RK6Z<66ZX5y1YIK1J mn 187 Ru ru 7.0.32 /confirm/vipstatus ?key=C19213C0060312E9E28A82&email=bmFA2095&userId=2746346&userId=27457455266&ts=15819213D&source=pcap 2881908356 +2020-02-17 2020-02-17 13:40:53 EMC uDOkWKoTLCbB2SFYoepda0 itJJ9zJgyZVY d50Y87RQCO70ci0u{QDZcA8dU31qm0IS1 0 0MuqyXoTtKYpmheID1AiXWz9sZ0zroDj mn 187 Ru ru 7.0.32 /confirm/vipstatus ?key=A3274695&userId=2746346470&userId=274634&userId=256AEA56926&ts=158194407&userId=27463467720%3D&source=pcap 700256219 +2020-02-17 2020-02-17 13:56:10 EMC rOXnSk45QcoEkw03yMYbGUhD_YZ_yCx9OU DyGXuVg8;P6Tq8CDp2_0XI`zPz4wuG8iJJh 0 3IM74UyhDyv2Jamik02pDQJu_L10vfPCKTm mn 187 Ru ru 7.0.32 /confirm/vipstatus ?key=459&email=bmFA81942095&userId=27468872095&userId=27457F1944672Y29t&source=pcap 607085190 +2020-02-17 2020-02-17 13:57:55 EMC xo09ZqxjuPBo_2aYOgnG\\sH3HDRtPH7xW 1I3ceRxYeGTFgNy30ojqJ]qrOgqu1Ag8yO 0 PwQkOkG2N1_p*ZFjOwmM6FviVATMGY5_rbK mn 187 Ru ru 7.0.32 /confirm/vipstatus ?key=158193979967F628BBA48E274634&email=dGV4LnJ1&source=mn 3851759707 +2020-02-17 2020-02-17 14:09:55 EMC vmlDYHpCg270cyeLJZBJTwBBnnJHH99Av l9FPMTHyyKlHijgeO]2oJK90OO8hWvntXS8 0 0Xu7Ha8eqQdwcG3pm#GNMw6HzEsxoAszdK mn 187 Ru ru 7.0.32 /confirm/vipstatus ?key=BtYWlsLnJ1&source=mn 2783694582 +2020-02-17 2020-02-17 15:31:52 EMC WIzjCT0n1XqAlOuRi2YskEx619ZjBaFe4g z_NGqjbLB0IhQV1SxDRMNG0DlMUxPlqaQ6 0 bv75_7V5B4pBB2aNK\fRwnBDQm0ngrcFPLN mn 187 Ru ru 7.0.32 /confirm/vipstatus ?key=158193628FA603193692&email=bmFA934&userId=265&ts=158192YW5haWwucnU%3D&source=pcap 722494088 +2020-02-17 2020-02-17 06:28:37 EMC 434gE67EZuzQL3ssU31NIt0eoeEpm8YplE7RGkXuG https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https%3A%2F%2Fwwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6source%3Dpc&c=swm&r=http&o=mail.ru/redir?u=https://click.mail.ru/redir?u=http&o=mail&v=2&s=974578Fvipstatus%3Fkey mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=1581938C8193634&email=a29t&source=mn 4267472950 +2020-02-17 2020-02-17 03:43:33 EMC TAl7tSOQaATLUUMZTx90iv0sqRoADhEn2r zvCR7ostHdMiQuVI8uxOyYGf0mm6k3ckHSL 32488481 9kFERwTF"AWKxn0YPOUv_tA8c5X2cdy4bA pcap 1 Ru ru 7.0.32 /confirm/vipstatus ?key=1581936292&email=a256F095&ts=158192&userId=274695&userId=27465D84D65D8B8E7EA692YW5haWwucnU%3D&source 1722142578 +2020-02-17 2020-02-17 04:06:30 EMC kL0p32eE J8hj51lGJQdDpzRytjcc2_AnA eO7MVT5A47viQfb3A01JapoD001J0l_UAt 23078127 KDZSswsocsumoOtSx\\phTvuffYOTHGNoUa mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=7724D&source=mn 2669624134 +2020-02-17 2020-02-17 04:23:38 EMC q3PyeDUQPYw7QN9BwE1eP\nDzWsZDAv3GTN u3GnfRRy>bOqpqFvSDikeB8V0zHacc0ur 26965394 CxxiDtfVM9T6WuleL1P4mMbzF9k0TLWLF pcap 1 Ru ru 7.0.32 /confirm/vipstatus ?key=58193695&ts=1581942095&email=bWFpbC5ydQ%3D%3D&source=mn 2552851257 +2020-02-17 2020-02-17 05:36:35 EMC ipti_lG94H8P5RgFwKzaoxUwWNEV9beK zolMT7vt_ttY|BmlD2PSejtg6O7WzJ2S_ 31107154 ha6rDRxktOKPjl2gzz7R2TpQu2KKta9Q mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=4942095&ts=15819369266&userId=274692158193634&userId=2745523552D695 1459132007 +2020-02-17 2020-02-17 00:45:31 EMC hXw0ETb7piXc>PKMzqxhV_F_UdWZnFB5O VXB5E4w28OrrZT0X1Sn9SrTWuJhjff7I 19656120 v06fJehG\'_m057sFduiQBx(mS5JnuSD6jph mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=B193695&ts=15819346165D85jb20%3D&source=pcap 1300966980 +2020-02-17 2020-02-17 03:44:40 EMC 0Q0EMlWcxIACIHUsZX4mEhJnxPpCJV6Qe ZqVPZ1Yb>rFSB\tzEMNdTEjoWIHLaQ2o5vI_E 17361913 AuRaBJOGvceRXWiMb9\'wolUnAGR0hxXQgOS pcap 1 Ru ru 7.0.32 /confirm/vipstatus ?key=725634&email=bWFpbC5ydQ%3D%3D&source=mn 1552491257 +2020-02-17 2020-02-17 02:35:40 EMC mSk0tnq0soJQslU0KlkSmfnXFu6fqRw0A hjlyHsr0&gybjsAJ0WRqdLJz32l3lmiEgDN 32664890 YAoxYQxl"Ce8HvY2j90VWzKQxlGCTNRAyJ mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=6459&ts=158194464089&userId=27465D82&userId=27455229t&source=mn 3051884197 +2020-02-17 2020-02-17 13:44:36 EMC znOwy_e4iHG8(NdCh2qgz1tjBR0ZjsFus eNSFOPi9oax3Ru7x01JA4TWeX66Ev8T9U 18979240 LosZ3fZjVY8hXjH8AJgKx:O3FNAMVDRGlZ mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=71581944027465D819384019215819397327457AbWFpbC5ydQ%3D%3D&source=mn 1828619156 +2020-02-17 2020-02-17 02:07:55 EMC yvDmRiaYkm43iWRNnnW3YDPBqq55XWVXr 7gpmg2PirkSp]TYhcY7i6]DawnwV9tqC0b 22858229 yDybPZi3Oc7m,JVxfltuRtWm8TmiHQJOjY https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://click.mail%3DZG1haWwucnU%253D%26source%3Dmn&c=swm&r=http&o=mail%3DZG1haWwucnU%253D%26source%3Dmn&c=swm&r=http&o=mail&v=2&s=9158192YWlsLnJ1%26source%3Dmn&c=swm&r=https://click.mail&v=2&s=3Dmn&c=swm&r=https://click.mail.ru/redir?u=http&o=mail%3DYm_ce%3Dmn mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=158189721581939265D89&email=dG1haWwuY29t&source=pcap 1735264694 +2020-02-17 2020-02-17 00:39:49 EMC BOA0kaQben9xa\fOjQ8YZt2L9UV_FYiMKYa3n 0BL0DaCp3weXj7aIq4ZBhaKzr0Uwpdpm 21820865 xATcz14a2hYJcGJb2gRsIIQ9K2A5fOGo https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://mail.ru/redir?u=https://mail&v=2&s=90BtYWlsLnJ1%26source%3Dmn&c=swm&r=https://click.mail.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6ts%3D158192YWlsLnJ1%26source%3Dmn&c=swm&r=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6userId%3D27457%26ts%3D158192YWlsLnJ1%26source%3Dmn&c=swm&r=http&o=mail.ru/redir?u=http&o=mail.ru/redir?u=http mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=77256160&userId=274634&email=bWFpbC5ydQ%3D%3D&source=mn 1855925025 +2020-02-17 2020-02-17 01:38:01 EMC uNpwiDVW2_p6HEMl4YIHw0z_vOpEqPyir napvpdbMVeSjy KJz5spT94Yu6wGCm_xgkX 27148457 R8wZY_PRklNkxX0Dn6uA{sGHEPaMT3kc8 mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=3C3274695&userId=2745528194401908193467BD9189&ts=1581926&ts=1581939772B67B6AE4CD2DBD910B392YW5haWwucnU 2249892028 +2020-02-17 2020-02-17 23:32:22 EMC uKrr0IYx5Kt1\'QPtv@toS9nQ4ZcrvAD0l8 4dWFNMPK50HO0z6ZPSoX6g02Ceu35kLzjtB 25390020 CXq72Vsbolj2.WJUoa8UoTZlw2wHUHxWX https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=http&o=mail%3DYWlsLnJ1%26source%3Dmn&c=swm&r=https%3A%2F%2Fwwwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6email&v=2&s=3A%2F%2Fwwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D67972 mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=BE304F9CDBC6&ts=1581934&email=YW5haWwuY29t&source=mn 3489237198 +2020-02-17 2020-02-17 22:36:42 EMC hQkEQSurISp_uy9ORd05 yjXxg7QByQH9 FURnnJ51xl2NiQHcN\r6hpIkfYa17e26Mt8 25551409 NBIMBrwi mVoYYuxJq_8Df0CM8CZkrZezFE mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=158194463465&userId=25616C3D&source=mn 2913713687 +2020-02-17 2020-02-17 20:19:16 EMC Xahxt0V3`JlMo+tyHm8MxJFT9NEywdtAZeD YV0_62kSEM9X 0GW7eHjXxF95KfKoarfR 27982291 iuEEgZrUYV04*QSEQg_uWbYo8cewVWPriGO mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=DFD92Y29t&source=mn 1136043092 +2020-02-17 2020-02-17 20:05:44 EMC gAraBOGNzDEdkz8qxSB68sQXrgyTFpX6 KC9GBYXCc2sL;fDL4#6DMT%2ELzdoqHtjMp 27849879 hmvHfcpQ7YcXyk6byq3rSqEUfDuei6M62h mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=71&source=mn 795793408 +2020-02-17 2020-02-17 01:03:41 EMC Sujwqk4khZF0dd3dRrn0zsOYRdkd9ynAR NaKyF0O3rR3r0wXNLf_3KSANWona7Ftry 30475228 calDaEsa910bkYOkR\bbKjQeUvAOoR7CQJH https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=http&o=mail.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6source%3Dmn&c=swm&r=http&o=mail%3DYWlsLnJ1%26source%3Dmn&c=swm&r=https://click.mail&v mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=B1581918970&userId=274634&userId=27457&email=YW5hce=mn 3165615449 +2020-02-17 2020-02-17 18:40:14 EMC zBHwSrKIMfAFFJxuvWtIsZ2DA_nAX98Aa NQiRPz123luXCH0fvH?59tbjHdGAlVgiHhc 26251034 0KbAptV7u8SZ?V4N7WHvx[0vY5D_saLU2G https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://click.mail%3DYWlsLnJ1%26source%3Dmn&c=swm&r=https://click.mail&v=2&s=9158192Ymail&v=2&s=936source%3Dmn&c=swm&r=http&o=mail%3DbW9%26userId%3D2746158192YWlsLnJ1%26source mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=5&userId=274552746887BF4LnJ1&source=mn 115808044 +2020-02-17 2020-02-17 22:03:35 EMC dvrVwEyocDVpJ3YEozYAH1XuC51DQcby59 wcCGF71wVXRj0pz2h131p\\1c7jnQYzejlC 24153343 mHlXjDvel9Ab8kZxtR6ELtFOSq7X1Q8_7 https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=http&o=mail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D65 mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=A63468874692&email=ce=pcap 1125992029 +2020-02-17 2020-02-17 13:11:31 EMC ELifUEthNP3xZTZ5RZd2tIgUWpctZJNKkdp 3qxE4qStGm1jIRPc9,JCHKxNAJqeRRgVaJ5 32523658 opEAQkCl*DGc0VozrkcYlaUmWvfN0IAYa https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=http&o=mail%3DYWlsLnJ1%26source%3Dmn&c=swm&r=https://click.mail&v=2&s=3%26userId%3D274570%26userId%3D274615819383%26userId%3D2745746193%26userId%3D2745745776source%3Dmn&c=swm&r=http&o=mail&v=2&s=34Fww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6source%3Dmn&c=swm&r=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6126source%3Dmn&c=swm&r=https://click.mail%3DbWFpbC5ydQ%253D%253D mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=616&userId=27465&email=YWlsLmNvbQ%3D%3D&source=pcap 660510007 +2020-02-17 2020-02-17 23:08:20 EMC Ww8aun1EXuPT3stlfv12_A4nVx4RfDfGaK Gj6ixLDK{FkfiIQ0ZqHK9IsJMWys80zowLQy 28692974 nB20Riv63uMc]E9s0Bv7x1XcxILAKq2zwc pcap 1 Ru ru 7.0.32 /confirm/vipstatus ?key=3D&source=pcap 458549627 +2020-02-17 2020-02-17 15:45:11 EMC _nkKyF0NV20B2AAxI@oOj8uSGus_lOrqBS 0vXB63kVoAv97D8cUCQSR\bmet_h2UhiQ4X 31753811 zA7KKsesYWPrI1KD"FFmdemMIJ3AcxwZr https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://click.mail%3DZG1haWwucnU%253D%26source%3Dmn&c=swm&r=http&o=mail&v=2&s=9%26ts%3D1581944577457EC6source%3Dmn&c=swm&r=https://click.mail.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D67D492YWlsLnJ1%26source%3Dmn&c=swm&r=https%3A%2F%2Fwwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D685764%26email&v=2&s=9%26email.ru/redir?u=https%3A%2F%2Fwwww.wildberries.ru%2Fconfirm mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=DA8189&email=a2DC9serId=274634&userId=25695&userId=274688&userId=27465 4039013194 +2020-02-17 2020-02-17 14:54:17 EMC wHTfVQKqlnQp!PXzWG80qc0oWU1pvtKeL EaNVlRXuzZ33\fLB0Fmjet29f3xaJplvaiX 30992376 vJnOh3mpOb0h!xPw1slS0e[Ehj1AIx920Jo https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://click.mail.ru/redir?u=https://click.mail&v=2&s=3c pcap 1 Ru ru 7.0.32 /confirm/vipstatus ?key=7462B60&email=cnU%3D&source=mn 2569826295 +2020-02-17 2020-02-17 22:51:48 EMC C1a6zgET p1v0mSMHJJgMxp8w7uJ7zN14J9Z 5cn50cUip60T6nRihVzsSRarErtllzgH6 26715992 0D4M2m041DlURNuFG9QJtIcjHVkp5_TEZ https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://click.mail&v=2&s=9446158194419447 mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=DEBE34&email=Z21haWwucnU%3D&source=mn 1504060974 +2020-02-17 2020-02-17 13:53:54 EMC 5a2rniMpduiEUzYFsgZjKHqgoZWbDYts ziqQkEkD3tYgLGhe&QwROPlb3hkAIZW_UI 32696355 0rSWK9UmGDPuRvPwXqs3mK9cF0balOqbS http&o=mail.ru/redir?u=http&o=mail.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6email pcap 1 Ru ru 7.0.32 /confirm/vipstatus ?key=77274552E&ts=158192158192YXNvbQ%3D%3D&source=pcap 1675780127 +2020-02-17 2020-02-17 20:08:49 EMC 0IBozTjy/SMIkY5tY\\_GAd\'0VIzFgFK2VSY V0fFnEjkPzK3.y_ng3Nfs1oJtBLCWPq35z 33102601 NQhlj1k9bLy234WJi\rKtOd2239rJDYJSKh https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=http&o=mail.ru/redir?u=http&o=mail.ru/redir?u=https%3A%2F%2Fwwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D61haWwucnU%253D%253D%26source%3Dmn&c=swm&r=https://click.mail%3DYWlsLnJ1%26source%3Dmn&c=swm&r=http&o=mail.ru/redir?u=https://click.mail.ru/redir?u=https%3A%2F%2Fwwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69%26userId%3D2746193A%2F%2Fwwwwwww.wildberries.ru%2Fconfirm%2Fvipstatus mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=D8BD7B082&ts=15819392YW5haWwuY29t&source=mn 832387252 +2020-02-17 2020-02-17 21:39:56 EMC 3aiRtPZF[Sjelo0e2EVnauTwl0tsW9SRi7f i8OkH4au\rfMiBV0E4frxZu9WJpRG5bhp4 29554136 gzPJTobNMGgG1kUYVe9sOwfpHmKiF1p0 https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=http&o=mail%3DbW97B365 mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=AE&ts=158194466&ts=15819213CD274688&ts=1581934688&email=Ymn 74272104 +2020-02-17 2020-02-17 17:21:43 EMC yqM2AqiUS2Lmo0S5KNYR-RKGxDrME0w5U T1Q2fyBqs8qhiWB_JBE3Ko_Ro0VD79pT0 18406884 mVoGr7MMzfVX0HtUKAYeCVX68_NOaiQY https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https%3A%2F%2Fwwwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6email%3DbWFpbC5ydQ%253D%253D%26source%3Dmn&c=swm&r=http&o=mail&v=2&s=9158192YWlsLnJ1%26source%3Dmn&c=swm&r=https mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=5819392&userId=27465D8B446887AF95&email=bWFpbC5ydQ%3D%3D&source=pcap 1422459332 +2020-02-17 2020-02-17 15:06:14 EMC NXf_EKmdwAkZZIKr0N0w4HuVTpNqZ30Ldk0 PpN_0d7ywY45Y_Wm#L2S01pN5klv0Wbsa 21385681 JH3MQSLtn6gUIe92ilJPT.EFDSucrCuGHZ mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=A4627465D819397D96AE65&email=dG1haWwuY29t&source=pcap 959722117 +2020-02-17 2020-02-17 19:52:47 EMC OY90Zcjc8ebLcK_uULQHDNvpwO1hN3MwIat ASdCffiid6DweXoVjmpL8~5DHOzhFnPRuW 33422761 4T1zxu2F4evId\rETTryZBYsCBkTxsXLeuC9 mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=E97265D8A70F&ts=15819403193634&userId=2746887&email=ce=mn 997133189 +2020-02-17 2020-02-17 17:55:20 EMC ManEZ_qtzd0v\bdxdB07LpVWqoZUfpDzZW MnNRUoUE2ASWag8ClKK5l0yPRafHb0gsw 29428950 clyoRyW5\00F6wDd5P188B6k_OdKCbUFQqT mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=52666799692&email=dGF7470E634&email=ce=mn 3015790084 +2020-02-17 2020-02-17 20:07:34 EMC pr7D2hoMay0bwJjvH6UaPLZpeDXtq0Njs bDlkkbc3OnwzdcNZHDcBkR|V0ApitIVgPOv 22120090 TC2aVmxXNfOhSL0y2oqzb=drQElfeo0_OQ mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=581943D&source=mn 2666495492 +2020-02-17 2020-02-17 18:20:19 EMC a_EpwBGWRXNSTY_FyAhRB25SpX8Gw8JaW 0XDllB5u!HDENidqN/CRrkxwnjTXLeEKIyb 32607759 OKByJYVH3pLb:BWhWwvLnnpw0urPjP04i https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6userId%3D27461203Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6email&v=2&s=3D158192YWlsLnJ1%26source%3Dpc&c=swm&r=https mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=BC66&ts=158193926&userId=256BFC35FD2AEAZ21haWwucnU%3D&source=mn 590291581 +2020-02-17 2020-02-17 10:27:11 EMC bn8x3TAHKlMOsdlmzybH0S*iDc0hYull7Zt 7PY8hsBErbSTnpoU7iPxD8CFjIMxK8eN8F 30559777 lgByHEG2ffmexKgJXv5rC6CW78KV4Kia mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=6667&email=bWFpbC5ydQ%3D%3D&source=mn 1898373016 +2020-02-17 2020-02-17 14:09:12 EMC jYFR0DCE2eTkEFCuAIyAHVYLUy9ENh9Cqc dhe0lvmTDchj8XhgtsZ3mCPzv5hTGPM8Q 24813187 Y7MmPfU2YhtaY85K"2xIfqNraWWqBp5QvK https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https%3A%2F%2Fwwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6source%3Dmn&c=swm&r=http&o=mail&v=2&s=96email%3DZG1haWwucnU%253D%26source%3Dmn&c=swm&r=http&o=mail&v=2&s=944576userId%3D27461819443Fkey%3D646192YWlsLnJ1%26source%3Dmn&c=swm&r=https://click.mail.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6userId%3D274615819360 mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=AD9397158192&userId=25616AEBD63465D88&email=bmRleC5ydQ%3D%3D&source=mn 1726149098 +2020-02-17 2020-02-17 18:27:00 EMC VnzEdMoo2lhW?Yr6AQyNVw yFnUj_l0pBiG EnQ5CpnJt3LM1hiuC7lsqJChM0i4JMZ0 28169388 zBJ8p4pSlPo_bkk4\t2ObECW2ReYnHoWrQH https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6userId%3D27461203Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6email&v=2&s=3D158192YWlsLnJ1%26source%3Dmn&c=swm&r=https mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=397729t&source=mn 1102313974 +2020-02-17 2020-02-17 19:49:07 EMC fxpHcK20ouDlcjYFRhFT4DMtbMML0qb8ds hBF63Vf_qZrpsfD4EOfd6YGUBvflIJqF 22835697 TkBqOH2Jjp0wH35uDnjAxqgyb2P4HqtznK https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6userId%3D2746158192YWlsLnJ1%26source%3Dmn&c=swm&r=https://click.mail&v=2&s=9158192YWlsLnJ1%26source%3Dmn&c=swm&r=https://click.mail&v=2&s=92YWlsLnJ1%26source%3Dmn&c=swm&r=https://click.mail%3DbWFpbC5ydQ%253D%253D%26source%3Dpc&c=swm&r=http&o=mail%3DZG1haWwucnU%253D%26source%3Dmn&c=swm&r=https://click.mail&v=2&s=9%26source%3Dpc&c pcap 1 Ru ru 7.0.32 /confirm/vipstatus ?key=ADDABAA48E95&userId=2746346887D9420%3D&source=mn 3497823576 +2020-02-17 2020-02-17 13:28:56 EMC hZbZqj2vRV2FpCXlvxIEyqdeQrwceKg35 8fpYiVLE9gfSIzfP_$dzlj1H8tMzVbcg59 20129530 8bY0GHO5KMpHW9cIAke4yQQ94oQfvoHBxY mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=C5ydQ%3D%3D&source=mn 3102735519 +2020-02-17 2020-02-17 21:37:19 EMC TAlSFJzIsXzTWbuEymOTU320iUcNXmLT5j J2Vf06b6CMLFAvwH_wsUNOv4O9nnCy20 17587521 nEczrivzdrNaBYYvdnQRUT2pBBo8hu3kCvI mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=66&email=YWlsLnJ1&source=mn 1613554206 +2020-02-17 2020-02-17 21:58:32 EMC XPXxpWq0wHcK2FB4$JaOa9xIGhEaajLeS l59HKsiwgQuXgAweGZRZrEkN0Pqk_plZ4c 24296347 3s8mV_3W&hisrcwCm6QXCg_0zgLyr6nnqPB https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6userId%3D274616 pcap 1 Ru ru 7.0.32 /confirm/vipstatus ?key=467105695&email=a2YWlsLnJ1&source=pcap 2033387311 +2020-02-17 2020-02-17 16:54:16 EMC 2beVLC_ZYMmZuZbJZ1mjV LQeoXV0BsZKq J4Y22CvJx36nq6KNDMXBg!H5lUx1Z7BoFf 31434491 dabaBlCY351pXUW0HdErDTW_XGkqwmta https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6userId%3D27461203Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6Fconfirm%2Fvipstatus%3Fkey%3D661076userId%3D27457CD15819444613 mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=76031938874634&userId=2564BFA93626&userId=256&email=ce=mn 1613641575 +2020-02-17 2020-02-17 00:27:36 EMC Au3PRMFCW0Ij8iL0ZtsrryAvKJowqXJ8k ntqK0ayP8ltR0twuGraE`Vwgcmux6jFwg 21307924 ruixkHLKjmENcmpbvvbrrLU8FDLIpWOFtU https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://click.mail.ru/redir?u=https%3A%2F%2Fwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6email.ru/redir?u=https%3A%2F%2Fwwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D6CD2745785dQ mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=A41E772FFC5ydQ%3D%3D&source=pcap 2395792653 +2020-02-17 2020-02-17 22:03:46 EMC Tc9EmNpb5Xy0edNdTCLg7X2eLmGuTdEkIt YsCMp8oiUBPyhHBsDmEZs0xvixA2L6atY 23431515 Bh67PWI6jKZ6Q102nTeN(YP6e8kLoqiVA https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://mail&v=2&s=96userId%3D274616 pcap 1 Ru ru 7.0.32 /confirm/vipstatus ?key=5D819446&userId=27465D8191897C7462F400&userId=27463465D8192&email=Z21haWwuY29t&source=pcap 2211382541 +2020-02-17 2020-02-17 06:34:15 EMC S13SmJGaHVUBXqd3bs\tRpB3G_hXZBJjxOt6 FP_GWMlfw2REi3Up5Cqu0qWcA99YlCduy 27883082 xJ1eD6ixour7M0K_l6Ziv6awoxMpnzsaW https://click.mail%3DYmail%3Dberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D69444%26email.ru/redir?u=https://mail%3Dby%3D6%26ts%3D158193DBDB3A%2F%2Fwwww.wildberries.ru%2Fconfirm%2Fvipstatus%3Fkey%3D63158192YWlsLnJ1%26source%3Dmn&c=swm&r=https://click.mail&v=2&s=92YWlsLnJ1%26source mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=15819362hQG1haWwuY29t&source=pcap 2045856196 +2020-02-17 2020-02-17 00:36:06 EMC 2mjWiSh70SN_1xOBst7vbYq1z5EL26Z0i 7e5aIak4Pca2woYGZIJi!P_gRQdnPpHDP 28784173 b5hw0LlyfMCyYstJ9XbB6Pq7VN5hO9kIy mn 1 Ru ru 7.0.32 /confirm/vipstatus ?key=15819387AE3634&email=cnU%3D&source=mn 1255281910 diff --git a/tests/queries/0_stateless/01286_constraints_on_default.reference b/tests/queries/0_stateless/01286_constraints_on_default.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/01286_constraints_on_default.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/01286_constraints_on_default.sql b/tests/queries/0_stateless/01286_constraints_on_default.sql new file mode 100644 index 00000000000..d150bac15b5 --- /dev/null +++ b/tests/queries/0_stateless/01286_constraints_on_default.sql @@ -0,0 +1,29 @@ +DROP TABLE IF EXISTS default_constraints; +CREATE TABLE default_constraints +( + x UInt8, + y UInt8 DEFAULT x + 1, + CONSTRAINT c CHECK y < 5 +) ENGINE = Memory; + +INSERT INTO default_constraints (x) SELECT number FROM system.numbers LIMIT 5; -- { serverError 469 } +INSERT INTO default_constraints (x) VALUES (0),(1),(2),(3),(4); -- { serverError 469 } + +SELECT y, throwIf(NOT y < 5) FROM default_constraints; +SELECT count() FROM default_constraints; + +DROP TABLE default_constraints; + + +CREATE TEMPORARY TABLE default_constraints +( + x UInt8, + y UInt8 DEFAULT x + 1, + CONSTRAINT c CHECK y < 5 +); + +INSERT INTO default_constraints (x) SELECT number FROM system.numbers LIMIT 5; -- { serverError 469 } +INSERT INTO default_constraints (x) VALUES (0),(1),(2),(3),(4); -- { serverError 469 } + +SELECT y, throwIf(NOT y < 5) FROM default_constraints; +SELECT count() FROM default_constraints; diff --git a/tests/queries/0_stateless/01287_max_execution_speed.reference b/tests/queries/0_stateless/01287_max_execution_speed.reference new file mode 100644 index 00000000000..2e85f1f5335 --- /dev/null +++ b/tests/queries/0_stateless/01287_max_execution_speed.reference @@ -0,0 +1,8 @@ +Ok (1) +Ok (2) +2000000 +1 +Ok (3) +2000000 +1 +Ok (4) diff --git a/tests/queries/0_stateless/01287_max_execution_speed.sql b/tests/queries/0_stateless/01287_max_execution_speed.sql new file mode 100644 index 00000000000..7e8f6681c84 --- /dev/null +++ b/tests/queries/0_stateless/01287_max_execution_speed.sql @@ -0,0 +1,44 @@ +SET min_execution_speed = 100000000000, timeout_before_checking_execution_speed = 0.1; +SELECT count() FROM system.numbers; -- { serverError 160 } +SELECT 'Ok (1)'; +SET min_execution_speed = 0; + +SET min_execution_speed_bytes = 800000000000, timeout_before_checking_execution_speed = 0.1; +SELECT count() FROM system.numbers; -- { serverError 160 } +SELECT 'Ok (2)'; +SET min_execution_speed_bytes = 0; + +SET max_execution_speed = 1000000; +SET max_block_size = 100; + +CREATE TEMPORARY TABLE times (t DateTime); + +INSERT INTO times SELECT now(); +SELECT count() FROM numbers(2000000); +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; +SELECT 'Ok (3)'; +SET max_execution_speed = 0; + +SET max_execution_speed_bytes = 8000000; +TRUNCATE TABLE times; + +INSERT INTO times SELECT now(); +SELECT count() FROM numbers(2000000); +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; +SELECT 'Ok (4)'; +SET max_execution_speed_bytes = 0; + +-- Note that 'min_execution_speed' does not count sleeping due to throttling +-- with 'max_execution_speed' and similar limits like 'priority' and 'max_network_bandwidth' + +-- Note: I have to disable this part of the test because it actually can work slower under sanitizers, +-- with debug builds and in presense of random system hickups in our CI environment. + +--SET max_execution_speed = 1000000, min_execution_speed = 2000000; +-- And this query will work despite the fact that the above settings look contradictory. +--SELECT count() FROM numbers(1000000); +--SELECT 'Ok (5)'; diff --git a/tests/queries/0_stateless/01288_shard_max_network_bandwidth.reference b/tests/queries/0_stateless/01288_shard_max_network_bandwidth.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/tests/queries/0_stateless/01288_shard_max_network_bandwidth.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/tests/queries/0_stateless/01288_shard_max_network_bandwidth.sql b/tests/queries/0_stateless/01288_shard_max_network_bandwidth.sql new file mode 100644 index 00000000000..09c043784bb --- /dev/null +++ b/tests/queries/0_stateless/01288_shard_max_network_bandwidth.sql @@ -0,0 +1,15 @@ +-- Limit to 10 MB/sec +SET max_network_bandwidth = 10000000; + +-- Lower max_block_size, so we can start throttling sooner. Otherwise query will be executed too quickly. +SET max_block_size = 100; + +CREATE TEMPORARY TABLE times (t DateTime); + +-- rand64 is uncompressable data. Each number will take 8 bytes of bandwidth. +-- This query should execute in no less than 1.6 seconds if throttled. +INSERT INTO times SELECT now(); +SELECT sum(ignore(*)) FROM (SELECT rand64() FROM remote('127.0.0.{2,3}', numbers(2000000))); +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; diff --git a/tests/queries/0_stateless/01289_min_execution_speed_not_too_early.reference b/tests/queries/0_stateless/01289_min_execution_speed_not_too_early.reference new file mode 100644 index 00000000000..e69de29bb2d 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 new file mode 100644 index 00000000000..e3a18d0c515 --- /dev/null +++ b/tests/queries/0_stateless/01289_min_execution_speed_not_too_early.sql @@ -0,0 +1,19 @@ +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; + +-- Exception about execution speed is not thrown from these queries. +SELECT * FROM ES LIMIT 1 format Null; +SELECT * FROM ES LIMIT 10 format Null; +SELECT * FROM ES LIMIT 100 format Null; +SELECT * FROM ES LIMIT 1000 format Null; +SELECT * FROM ES LIMIT 10000 format Null; +SELECT * FROM ES LIMIT 100000 format Null; +SELECT * FROM ES LIMIT 1000000 format Null; + +DROP TABLE ES; diff --git a/tests/queries/0_stateless/01290_empty_array_index_analysis.reference b/tests/queries/0_stateless/01290_empty_array_index_analysis.reference new file mode 100644 index 00000000000..5037a64c0f0 --- /dev/null +++ b/tests/queries/0_stateless/01290_empty_array_index_analysis.reference @@ -0,0 +1,50 @@ +--- notEmpty + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- empty + [] 1 +--- = [] + [] 1 +--- != [] + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- > [] + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- < [] +--- >= [] + [] 1 + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- <= [] + [] 1 +--- +--- notEmpty + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- empty + [] 1 +--- = [] + [] 1 +--- != [] + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- > [] + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- < [] +--- >= [] + [] 1 + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- <= [] + [] 1 +--- diff --git a/tests/queries/0_stateless/01290_empty_array_index_analysis.sql b/tests/queries/0_stateless/01290_empty_array_index_analysis.sql new file mode 100644 index 00000000000..b1b6067945d --- /dev/null +++ b/tests/queries/0_stateless/01290_empty_array_index_analysis.sql @@ -0,0 +1,66 @@ +drop table if exists count_lc_test; + +CREATE TABLE count_lc_test +( + `s` LowCardinality(String), + `arr` Array(LowCardinality(String)), + `num` UInt64 +) +ENGINE = MergeTree +ORDER BY (s, arr); + +INSERT INTO count_lc_test(num, arr) VALUES (1,[]),(2,['a']),(3,['a','b','c']),(4,['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa']); + +SELECT '--- notEmpty'; +select * from count_lc_test where notEmpty(arr); +SELECT '--- empty'; +select * from count_lc_test where empty(arr); +SELECT '--- = []'; +select * from count_lc_test where arr = []; +SELECT '--- != []'; +select * from count_lc_test where arr != []; +SELECT '--- > []'; +select * from count_lc_test where arr > []; +SELECT '--- < []'; +select * from count_lc_test where arr < []; +SELECT '--- >= []'; +select * from count_lc_test where arr >= []; +SELECT '--- <= []'; +select * from count_lc_test where arr <= []; +SELECT '---'; + +DROP TABLE count_lc_test; + + +drop table if exists count_lc_test; + +CREATE TABLE count_lc_test +( + `s` LowCardinality(String), + `arr` Array(String), + `num` UInt64 +) +ENGINE = MergeTree +ORDER BY (s, arr); + +INSERT INTO count_lc_test(num, arr) VALUES (1,[]),(2,['a']),(3,['a','b','c']),(4,['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa']); + +SELECT '--- notEmpty'; +select * from count_lc_test where notEmpty(arr); +SELECT '--- empty'; +select * from count_lc_test where empty(arr); +SELECT '--- = []'; +select * from count_lc_test where arr = []; +SELECT '--- != []'; +select * from count_lc_test where arr != []; +SELECT '--- > []'; +select * from count_lc_test where arr > []; +SELECT '--- < []'; +select * from count_lc_test where arr < []; +SELECT '--- >= []'; +select * from count_lc_test where arr >= []; +SELECT '--- <= []'; +select * from count_lc_test where arr <= []; +SELECT '---'; + +DROP TABLE count_lc_test; diff --git a/tests/queries/0_stateless/01290_max_execution_speed_distributed.reference b/tests/queries/0_stateless/01290_max_execution_speed_distributed.reference new file mode 100644 index 00000000000..ad0e80d8e69 --- /dev/null +++ b/tests/queries/0_stateless/01290_max_execution_speed_distributed.reference @@ -0,0 +1,3 @@ +2000000 +1 +1 diff --git a/tests/queries/0_stateless/01290_max_execution_speed_distributed.sql b/tests/queries/0_stateless/01290_max_execution_speed_distributed.sql new file mode 100644 index 00000000000..8282390ca90 --- /dev/null +++ b/tests/queries/0_stateless/01290_max_execution_speed_distributed.sql @@ -0,0 +1,13 @@ +SET max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0.001, max_block_size = 100; + +CREATE TEMPORARY TABLE times (t DateTime); + +INSERT INTO times SELECT now(); +SELECT count('special query for 01290_max_execution_speed_distributed') FROM remote('127.0.0.{2,3}', numbers(1000000)); +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; + +-- Check that the query was also throttled on "remote" servers. +SYSTEM FLUSH LOGS; +SELECT DISTINCT query_duration_ms >= 500 FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%special query for 01290_max_execution_speed_distributed%' AND type = 2; diff --git a/tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.reference b/tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.reference new file mode 100644 index 00000000000..257cc5642cb --- /dev/null +++ b/tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.reference @@ -0,0 +1 @@ +foo diff --git a/tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.sql b/tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.sql new file mode 100644 index 00000000000..f0992f2e438 --- /dev/null +++ b/tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS data; +DROP TABLE IF EXISTS dist; + +create table data (key String) Engine=Memory(); +create table dist (key LowCardinality(String)) engine=Distributed(test_cluster_two_shards, currentDatabase(), data); +insert into data values ('foo'); +set distributed_aggregation_memory_efficient=1; +select * from dist group by key; + +DROP TABLE data; +DROP TABLE dist; diff --git a/tests/queries/0_stateless/01291_geo_types.reference b/tests/queries/0_stateless/01291_geo_types.reference new file mode 100644 index 00000000000..baf7b25a231 --- /dev/null +++ b/tests/queries/0_stateless/01291_geo_types.reference @@ -0,0 +1 @@ +(0,0) [(0,0),(10,0),(10,10),(0,10)] [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]] [[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]] diff --git a/tests/queries/0_stateless/01291_geo_types.sql b/tests/queries/0_stateless/01291_geo_types.sql new file mode 100644 index 00000000000..0d923f08ccd --- /dev/null +++ b/tests/queries/0_stateless/01291_geo_types.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS geo; + +SET allow_experimental_geo_types = 1; + +CREATE TABLE geo (a Point, b Ring, c Polygon, d MultiPolygon) ENGINE=Memory(); + +INSERT INTO geo VALUES((0, 0), [(0, 0), (10, 0), (10, 10), (0, 10)], [[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]], [[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]); + +SELECT * from geo; diff --git a/tests/queries/0_stateless/01291_unsupported_conversion_from_decimal.reference b/tests/queries/0_stateless/01291_unsupported_conversion_from_decimal.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01291_unsupported_conversion_from_decimal.sql b/tests/queries/0_stateless/01291_unsupported_conversion_from_decimal.sql new file mode 100644 index 00000000000..256c6424901 --- /dev/null +++ b/tests/queries/0_stateless/01291_unsupported_conversion_from_decimal.sql @@ -0,0 +1,5 @@ +SELECT toIntervalSecond(now64()); -- { serverError 70 } +SELECT CAST(now64() AS IntervalSecond); -- { serverError 70 } + +SELECT toIntervalSecond(now64()); -- { serverError 70 } +SELECT CAST(now64() AS IntervalSecond); -- { serverError 70 } diff --git a/tests/queries/1_stateful/00155_date_datetime_key_condition.sql b/tests/queries/1_stateful/00155_date_datetime_key_condition.sql deleted file mode 100644 index cee1c7cfb46..00000000000 --- a/tests/queries/1_stateful/00155_date_datetime_key_condition.sql +++ /dev/null @@ -1,22 +0,0 @@ -DROP TABLE IF EXISTS test.date_datetime_key_condition; - -CREATE TABLE test.date_datetime_key_condition (dt DateTime) ENGINE = MergeTree() ORDER BY dt; -INSERT INTO test.date_datetime_key_condition VALUES ('2020-01-01 00:00:00'), ('2020-01-01 10:00:00'), ('2020-01-02 00:00:00'); - --- partial -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt > toDate('2020-01-01') AND dt < toDate('2020-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt >= toDate('2020-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt < toDate('2020-01-02'); - --- inside -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt > toDate('2019-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt < toDate('2021-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt >= toDate('2019-01-02') AND dt < toDate('2021-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt > toDate('2019-01-02') OR dt <= toDate('2021-01-02'); - --- outside -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt < toDate('2019-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt > toDate('2021-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt < toDate('2019-01-02') OR dt > toDate('2021-01-02'); - -DROP TABLE test.date_datetime_key_condition; \ No newline at end of file diff --git a/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.reference b/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.reference new file mode 100644 index 00000000000..53bb58224b9 --- /dev/null +++ b/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.reference @@ -0,0 +1,4 @@ +4392010 +1 +4392010 +1 diff --git a/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql b/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql new file mode 100644 index 00000000000..37e91296f14 --- /dev/null +++ b/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql @@ -0,0 +1,16 @@ +SET max_execution_speed = 4000000, timeout_before_checking_execution_speed = 0.001; + +CREATE TEMPORARY TABLE times (t DateTime); + +INSERT INTO times SELECT now(); +SELECT count() FROM test.hits SAMPLE 1 / 2; +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; +TRUNCATE TABLE times; + +INSERT INTO times SELECT now(); +SELECT count() FROM merge(test, '^hits$') SAMPLE 1 / 2; +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; diff --git a/utils/generate-ya-make/generate-ya-make.sh b/utils/generate-ya-make/generate-ya-make.sh index ec4511f56c1..84be2177386 100755 --- a/utils/generate-ya-make/generate-ya-make.sh +++ b/utils/generate-ya-make/generate-ya-make.sh @@ -8,5 +8,6 @@ ROOT_PATH=$(git rev-parse --show-toplevel) EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing' find "${ROOT_PATH}" -name 'ya.make.in' | while read path; do - (cd $(dirname "${path}") && perl -pne 's/<\?(.+?)\?>/`$1`/e' < "${path}" > "${path/.in/}") + echo "# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it." > "${path/.in/}" + (cd $(dirname "${path}") && perl -pne 's/<\?(.+?)\?>/`$1`/e' < "${path}" >> "${path/.in/}") done diff --git a/utils/zookeeper-cli/zookeeper-cli.cpp b/utils/zookeeper-cli/zookeeper-cli.cpp index 0a503e77250..6fd7b39ab68 100644 --- a/utils/zookeeper-cli/zookeeper-cli.cpp +++ b/utils/zookeeper-cli/zookeeper-cli.cpp @@ -66,8 +66,8 @@ int main(int argc, char ** argv) } Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); zkutil::ZooKeeper zk(argv[1]); LineReader lr({}, '\\'); diff --git a/website/templates/common_meta.html b/website/templates/common_meta.html index 8b65e67d780..84bd93d5175 100644 --- a/website/templates/common_meta.html +++ b/website/templates/common_meta.html @@ -22,6 +22,9 @@ +{% if config and (config.extra.single_page or config.extra.version_prefix) %} + +{% endif %} {% if config and page %} {% for code, name in config.extra.languages.items() %} diff --git a/website/templates/docs/amp.html b/website/templates/docs/amp.html index a60270d23d0..0cff614035f 100644 --- a/website/templates/docs/amp.html +++ b/website/templates/docs/amp.html @@ -7,6 +7,9 @@ {% include "templates/docs/ld_json.html" %} + {% if config.extra.single_page or config.extra.version_prefix %} + + {% endif %}