diff --git a/CHANGELOG.md b/CHANGELOG.md index 885b4fc656f..ca14134753a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,8 +1,288 @@ +## ClickHouse release v20.4 + +### ClickHouse release v20.4.2.9, 2020-05-12 + +#### Backward Incompatible Change +* System tables (e.g. system.query_log, system.trace_log, system.metric_log) are using compact data part format for parts smaller than 10 MiB in size. Compact data part format is supported since version 20.3. If you are going to downgrade to version less than 20.3, you should manually delete table data for system logs in `/var/lib/clickhouse/data/system/`. +* When string comparison involves FixedString and compared arguments are of different sizes, do comparison as if smaller string is padded to the length of the larger. This is intented for SQL compatibility if we imagine that FixedString data type corresponds to SQL CHAR. This closes [#9272](https://github.com/ClickHouse/ClickHouse/issues/9272). [#10363](https://github.com/ClickHouse/ClickHouse/pull/10363) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Make SHOW CREATE TABLE multiline. Now it is more readable and more like MySQL. [#10049](https://github.com/ClickHouse/ClickHouse/pull/10049) ([Azat Khuzhin](https://github.com/azat)) +* Added a setting `validate_polygons` that is used in `pointInPolygon` function and enabled by default. [#9857](https://github.com/ClickHouse/ClickHouse/pull/9857) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### New Feature +* Add support for secured connection from ClickHouse to Zookeeper [#10184](https://github.com/ClickHouse/ClickHouse/pull/10184) ([Konstantin Lebedev](https://github.com/xzkostyan)) +* Support custom HTTP handlers. See ISSUES-5436 for description. [#7572](https://github.com/ClickHouse/ClickHouse/pull/7572) ([Winter Zhang](https://github.com/zhang2014)) +* Add MessagePack Input/Output format. [#9889](https://github.com/ClickHouse/ClickHouse/pull/9889) ([Kruglov Pavel](https://github.com/Avogar)) +* Add Regexp input format. [#9196](https://github.com/ClickHouse/ClickHouse/pull/9196) ([Kruglov Pavel](https://github.com/Avogar)) +* Added output format `Markdown` for embedding tables in markdown documents. [#10317](https://github.com/ClickHouse/ClickHouse/pull/10317) ([Kruglov Pavel](https://github.com/Avogar)) +* Added support for custom settings section in dictionaries. Also fixes issue [#2829](https://github.com/ClickHouse/ClickHouse/issues/2829). [#10137](https://github.com/ClickHouse/ClickHouse/pull/10137) ([Artem Streltsov](https://github.com/kekekekule)) +* Added custom settings support in DDL-queries for CREATE DICTIONARY [#10465](https://github.com/ClickHouse/ClickHouse/pull/10465) ([Artem Streltsov](https://github.com/kekekekule)) +* Add simple server-wide memory profiler that will collect allocation contexts when server memory usage becomes higher than the next allocation threshold. [#10444](https://github.com/ClickHouse/ClickHouse/pull/10444) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add setting `always_fetch_merged_part` which restrict replica to merge parts by itself and always prefer dowloading from other replicas. [#10379](https://github.com/ClickHouse/ClickHouse/pull/10379) ([alesapin](https://github.com/alesapin)) +* Add function JSONExtractKeysAndValuesRaw which extracts raw data from JSON objects [#10378](https://github.com/ClickHouse/ClickHouse/pull/10378) ([hcz](https://github.com/hczhcz)) +* Add memory usage from OS to `system.asynchronous_metrics`. [#10361](https://github.com/ClickHouse/ClickHouse/pull/10361) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added generic variants for functions `least` and `greatest`. Now they work with arbitrary number of arguments of arbitrary types. This fixes [#4767](https://github.com/ClickHouse/ClickHouse/issues/4767) [#10318](https://github.com/ClickHouse/ClickHouse/pull/10318) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Now ClickHouse controls timeouts of dictionary sources on its side. Two new settings added to cache dictionary configuration: `strict_max_lifetime_seconds`, which is `max_lifetime` by default, and `query_wait_timeout_milliseconds`, which is one minute by default. The first settings is also useful with `allow_read_expired_keys` settings (to forbid reading very expired keys). [#10337](https://github.com/ClickHouse/ClickHouse/pull/10337) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Add log_queries_min_type to filter which entries will be written to query_log [#10053](https://github.com/ClickHouse/ClickHouse/pull/10053) ([Azat Khuzhin](https://github.com/azat)) +* Added function `isConstant`. This function checks whether its argument is constant expression and returns 1 or 0. It is intended for development, debugging and demonstration purposes. [#10198](https://github.com/ClickHouse/ClickHouse/pull/10198) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* add joinGetOrNull to return NULL when key is missing instead of returning the default value. [#10094](https://github.com/ClickHouse/ClickHouse/pull/10094) ([Amos Bird](https://github.com/amosbird)) +* Consider `NULL` to be equal to `NULL` in `IN` operator, if the option `transform_null_in` is set. [#10085](https://github.com/ClickHouse/ClickHouse/pull/10085) ([achimbab](https://github.com/achimbab)) +* Add `ALTER TABLE ... RENAME COLUMN` for MergeTree table engines family. [#9948](https://github.com/ClickHouse/ClickHouse/pull/9948) ([alesapin](https://github.com/alesapin)) +* Support parallel distributed INSERT SELECT. [#9759](https://github.com/ClickHouse/ClickHouse/pull/9759) ([vxider](https://github.com/Vxider)) +* Add ability to query Distributed over Distributed (w/o `distributed_group_by_no_merge`) ... [#9923](https://github.com/ClickHouse/ClickHouse/pull/9923) ([Azat Khuzhin](https://github.com/azat)) +* Add function `arrayReduceInRanges` which aggregates array elements in given ranges. [#9598](https://github.com/ClickHouse/ClickHouse/pull/9598) ([hcz](https://github.com/hczhcz)) +* Add Dictionary Status on prometheus exporter. [#9622](https://github.com/ClickHouse/ClickHouse/pull/9622) ([Guillaume Tassery](https://github.com/YiuRULE)) +* Add function arrayAUC [#8698](https://github.com/ClickHouse/ClickHouse/pull/8698) ([taiyang-li](https://github.com/taiyang-li)) +* Support `DROP VIEW` statement for better TPC-H compatibility. [#9831](https://github.com/ClickHouse/ClickHouse/pull/9831) ([Amos Bird](https://github.com/amosbird)) +* Add 'strict_order' option to windowFunnel() [#9773](https://github.com/ClickHouse/ClickHouse/pull/9773) ([achimbab](https://github.com/achimbab)) +* Support `DATE` and `TIMESTAMP` SQL operators, e.g. `SELECT date '2001-01-01'` [#9691](https://github.com/ClickHouse/ClickHouse/pull/9691) ([Artem Zuikov](https://github.com/4ertus2)) + +#### Experimental Feature +* Added experimental database engine Atomic. It supports non-blocking `DROP` and `RENAME TABLE` queries and atomic `EXCHANGE TABLES t1 AND t2` query [#7512](https://github.com/ClickHouse/ClickHouse/pull/7512) ([tavplubix](https://github.com/tavplubix)) +* Initial support for ReplicatedMergeTree over S3 (it works in suboptimal way) [#10126](https://github.com/ClickHouse/ClickHouse/pull/10126) ([Pavel Kovalenko](https://github.com/Jokser)) + +#### Bug Fix +* 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 bug, which caused HTTP requests to get stuck on client closing connection when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)) +* Fix segfault in StorageBuffer when exception is thrown on server startup. Fixes [#10550](https://github.com/ClickHouse/ClickHouse/issues/10550) [#10609](https://github.com/ClickHouse/ClickHouse/pull/10609) ([tavplubix](https://github.com/tavplubix)) +* The query`SYSTEM DROP DNS CACHE` now also drops caches 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)) +* Fix usage of multiple `IN` operators with an identical set in one query. Fixes [#10539](https://github.com/ClickHouse/ClickHouse/issues/10539) [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)) +* Fix 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)) +* Fix 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)) +* Fix logic for aggregation_memory_efficient_merge_threads setting. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)) +* Fix disappearing totals. Totals could have being filtered if query 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)) +* Fix 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)) +* Fix 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)) +* Fix index corruption, which may occur in some cases after merging compact parts into another compact part. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)) +* Fix 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)) +* Fix overflow at beginning of unix epoch for timezones with fractional offset from UTC. Fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Better diagnostics for input formats. Fixes [#10204](https://github.com/ClickHouse/ClickHouse/issues/10204) [#10418](https://github.com/ClickHouse/ClickHouse/pull/10418) ([tavplubix](https://github.com/tavplubix)) +* Fix numeric overflow in `simpleLinearRegression()` over large integers [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)) +* Fix use-after-free in Distributed shutdown, avoid waiting for sending all batches [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)) +* Add CA certificates to clickhouse-server docker image [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)) +* Fix a rare endless loop that might have occurred when using the `addressToLine` function or AggregateFunctionState columns. [#10466](https://github.com/ClickHouse/ClickHouse/pull/10466) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Handle zookeeper "no node error" during distributed query [#10050](https://github.com/ClickHouse/ClickHouse/pull/10050) ([Daniel Chen](https://github.com/Phantomape)) +* Fix bug when server cannot attach table after column's default was altered. [#10441](https://github.com/ClickHouse/ClickHouse/pull/10441) ([alesapin](https://github.com/alesapin)) +* Implicitly cast the default expression type to the column type for the ALIAS columns [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)) +* Don't remove metadata directory if `ATTACH DATABASE` fails [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)) +* Avoid dependency on system tzdata. Fixes loading of `Africa/Casablanca` timezone on CentOS 8. Fixes [#10211](https://github.com/ClickHouse/ClickHouse/issues/10211) [#10425](https://github.com/ClickHouse/ClickHouse/pull/10425) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix some issues if data is inserted with quorum and then gets deleted (DROP PARTITION, TTL, etc.). It led to stuck of INSERTs or false-positive exceptions in SELECTs. Fixes [#9946](https://github.com/ClickHouse/ClickHouse/issues/9946) [#10188](https://github.com/ClickHouse/ClickHouse/pull/10188) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Check the 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)) +* Prefer `fallback_to_stale_replicas` over `skip_unavailable_shards`, otherwise when both settings specified and there are no up-to-date replicas the query will fail (patch from @alex-zaitsev ) [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)) +* Fix the issue when a query with ARRAY JOIN, ORDER BY and LIMIT may return incomplete result. Fixes [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([Vadim Plakhtinskiy](https://github.com/VadimPlh)) +* Add database name to dictionary name after DETACH/ATTACH. Fixes system.dictionaries table and `SYSTEM RELOAD` query [#10415](https://github.com/ClickHouse/ClickHouse/pull/10415) ([Azat Khuzhin](https://github.com/azat)) +* Fix possible incorrect result for extremes in processors pipeline. [#10131](https://github.com/ClickHouse/ClickHouse/pull/10131) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix possible segfault when the setting `distributed_group_by_no_merge` is enabled (introduced in 20.3.7.46 by [#10131](https://github.com/ClickHouse/ClickHouse/issues/10131)). [#10399](https://github.com/ClickHouse/ClickHouse/pull/10399) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix wrong flattening of `Array(Tuple(...))` data types. Fixes [#10259](https://github.com/ClickHouse/ClickHouse/issues/10259) [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix column names of constants inside JOIN that may clash with names of constants outside of JOIN [#9950](https://github.com/ClickHouse/ClickHouse/pull/9950) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Fix order of columns after Block::sortColumns() [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)) +* Fix possible `Pipeline stuck` error in `ConcatProcessor` which may happen in remote query. [#10381](https://github.com/ClickHouse/ClickHouse/pull/10381) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Don't make disk reservations for aggregations. Fixes [#9241](https://github.com/ClickHouse/ClickHouse/issues/9241) [#10375](https://github.com/ClickHouse/ClickHouse/pull/10375) ([Azat Khuzhin](https://github.com/azat)) +* Fix wrong behaviour of datetime functions for timezones that has altered between positive and negative offsets from UTC (e.g. Pacific/Kiritimati). Fixes [#7202](https://github.com/ClickHouse/ClickHouse/issues/7202) [#10369](https://github.com/ClickHouse/ClickHouse/pull/10369) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Avoid infinite loop in `dictIsIn` function. Fixes #515 [#10365](https://github.com/ClickHouse/ClickHouse/pull/10365) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Disable GROUP BY sharding_key optimization by default and fix it for WITH ROLLUP/CUBE/TOTALS [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)) +* Check for error code when checking parts and don't mark part as broken if the error is like "not enough memory". Fixes [#6269](https://github.com/ClickHouse/ClickHouse/issues/6269) [#10364](https://github.com/ClickHouse/ClickHouse/pull/10364) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Show information about not loaded dictionaries in system tables. [#10234](https://github.com/ClickHouse/ClickHouse/pull/10234) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fix 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 `DROP` vs `OPTIMIZE` race in `ReplicatedMergeTree`. `DROP` could left some garbage in replica path in ZooKeeper if there was concurrent `OPTIMIZE` query. [#10312](https://github.com/ClickHouse/ClickHouse/pull/10312) ([tavplubix](https://github.com/tavplubix)) +* Fix 'Logical error: CROSS JOIN has expressions' error for queries with comma and names joins mix. Fixes [#9910](https://github.com/ClickHouse/ClickHouse/issues/9910) [#10311](https://github.com/ClickHouse/ClickHouse/pull/10311) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix queries with `max_bytes_before_external_group_by`. [#10302](https://github.com/ClickHouse/ClickHouse/pull/10302) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix the issue with limiting maximum recursion depth in parser in certain cases. This fixes [#10283](https://github.com/ClickHouse/ClickHouse/issues/10283) This fix may introduce minor incompatibility: long and deep queries via clickhouse-client may refuse to work, and you should adjust settings `max_query_size` and `max_parser_depth` accordingly. [#10295](https://github.com/ClickHouse/ClickHouse/pull/10295) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Allow to use `count(*)` with multiple JOINs. Fixes [#9853](https://github.com/ClickHouse/ClickHouse/issues/9853) [#10291](https://github.com/ClickHouse/ClickHouse/pull/10291) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix 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)) +* Fix 'Cannot add column' error while creating `range_hashed` dictionary using DDL query. Fixes [#10093](https://github.com/ClickHouse/ClickHouse/issues/10093). [#10235](https://github.com/ClickHouse/ClickHouse/pull/10235) ([alesapin](https://github.com/alesapin)) +* Fix rare possible exception `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fixed bug where ClickHouse would throw "Unknown function lambda." error message when user tries to run ALTER UPDATE/DELETE on tables with ENGINE = Replicated*. Check for nondeterministic functions now handles lambda expressions correctly. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)) +* Fixed reasonably rare segfault in StorageSystemTables that happens when SELECT ... FROM system.tables is run on a database with Lazy engine. [#10209](https://github.com/ClickHouse/ClickHouse/pull/10209) ([Alexander Kazakov](https://github.com/Akazz)) +* Fix possible infinite query execution when the query actually should stop on LIMIT, while reading from infinite source like `system.numbers` or `system.zeros`. [#10206](https://github.com/ClickHouse/ClickHouse/pull/10206) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fixed "generateRandom" function for Date type. This fixes [#9973](https://github.com/ClickHouse/ClickHouse/issues/9973). Fix an edge case when dates with year 2106 are inserted to MergeTree tables with old-style partitioning but partitions are named with year 1970. [#10218](https://github.com/ClickHouse/ClickHouse/pull/10218) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Convert types if the table definition of a View does not correspond to the SELECT query. This fixes [#10180](https://github.com/ClickHouse/ClickHouse/issues/10180) and [#10022](https://github.com/ClickHouse/ClickHouse/issues/10022) [#10217](https://github.com/ClickHouse/ClickHouse/pull/10217) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix `parseDateTimeBestEffort` for strings in RFC-2822 when day of week is Tuesday or Thursday. This fixes [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082) [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix column names of constants inside JOIN that may clash with names of constants outside of JOIN. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix move-to-prewhere optimization in presense of arrayJoin functions (in certain cases). This fixes [#10092](https://github.com/ClickHouse/ClickHouse/issues/10092) [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix issue with separator appearing in SCRAMBLE for native mysql-connector-java (JDBC) [#10140](https://github.com/ClickHouse/ClickHouse/pull/10140) ([BohuTANG](https://github.com/BohuTANG)) +* Fix using the current database for an access checking when the database isn't specified. [#10192](https://github.com/ClickHouse/ClickHouse/pull/10192) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fix ALTER of tables with compact parts. [#10130](https://github.com/ClickHouse/ClickHouse/pull/10130) ([Anton Popov](https://github.com/CurtizJ)) +* Add the ability to relax the restriction on non-deterministic functions usage in mutations with `allow_nondeterministic_mutations` setting. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)) +* Fix `DROP TABLE` invoked for dictionary [#10165](https://github.com/ClickHouse/ClickHouse/pull/10165) ([Azat Khuzhin](https://github.com/azat)) +* Convert blocks if structure does not match when doing `INSERT` into Distributed table [#10135](https://github.com/ClickHouse/ClickHouse/pull/10135) ([Azat Khuzhin](https://github.com/azat)) +* The number of rows was logged incorrectly (as sum across all parts) when inserted block is split by parts with partition key. [#10138](https://github.com/ClickHouse/ClickHouse/pull/10138) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add some arguments check and support identifier arguments for MySQL Database Engine [#10077](https://github.com/ClickHouse/ClickHouse/pull/10077) ([Winter Zhang](https://github.com/zhang2014)) +* Fix incorrect `index_granularity_bytes` check while creating new replica. Fixes [#10098](https://github.com/ClickHouse/ClickHouse/issues/10098). [#10121](https://github.com/ClickHouse/ClickHouse/pull/10121) ([alesapin](https://github.com/alesapin)) +* Fix bug in `CHECK TABLE` query when table contain skip indices. [#10068](https://github.com/ClickHouse/ClickHouse/pull/10068) ([alesapin](https://github.com/alesapin)) +* Fix Distributed-over-Distributed with the only one shard in a nested table [#9997](https://github.com/ClickHouse/ClickHouse/pull/9997) ([Azat Khuzhin](https://github.com/azat)) +* Fix possible rows loss for queries with `JOIN` and `UNION ALL`. Fixes [#9826](https://github.com/ClickHouse/ClickHouse/issues/9826), [#10113](https://github.com/ClickHouse/ClickHouse/issues/10113). ... [#10099](https://github.com/ClickHouse/ClickHouse/pull/10099) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix bug in dictionary when local clickhouse server is used as source. It may caused memory corruption if types in dictionary and source are not compatible. [#10071](https://github.com/ClickHouse/ClickHouse/pull/10071) ([alesapin](https://github.com/alesapin)) +* Fixed replicated tables startup when updating from an old ClickHouse version where `/table/replicas/replica_name/metadata` node doesn't exist. Fixes [#10037](https://github.com/ClickHouse/ClickHouse/issues/10037). [#10095](https://github.com/ClickHouse/ClickHouse/pull/10095) ([alesapin](https://github.com/alesapin)) +* Fix error `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform`. It happened when setting `distributed_aggregation_memory_efficient` was enabled, and distributed query read aggregating data with mixed single and two-level aggregation from different shards. [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix deadlock when database with materialized view failed attach at start [#10054](https://github.com/ClickHouse/ClickHouse/pull/10054) ([Azat Khuzhin](https://github.com/azat)) +* Fix a segmentation fault that could occur in GROUP BY over string keys containing trailing zero bytes ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). ... [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Fix wrong results of distributed queries when alias could override qualified column name. Fixes [#9672](https://github.com/ClickHouse/ClickHouse/issues/9672) [#9714](https://github.com/ClickHouse/ClickHouse/issues/9714) [#9972](https://github.com/ClickHouse/ClickHouse/pull/9972) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix possible deadlock in `SYSTEM RESTART REPLICAS` [#9955](https://github.com/ClickHouse/ClickHouse/pull/9955) ([tavplubix](https://github.com/tavplubix)) +* Fix the number of threads used for remote query execution (performance regression, since 20.3). This happened when query from `Distributed` table was executed simultaneously on local and remote shards. Fixes [#9965](https://github.com/ClickHouse/ClickHouse/issues/9965) [#9971](https://github.com/ClickHouse/ClickHouse/pull/9971) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fixed `DeleteOnDestroy` logic in `ATTACH PART` which could lead to automatic removal of attached part and added few tests [#9410](https://github.com/ClickHouse/ClickHouse/pull/9410) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fix a bug with `ON CLUSTER` DDL queries freezing on server startup. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)) +* Fix bug in which the necessary tables weren't retrieved at one of the processing stages of queries to some databases. Fixes [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)) +* Fix 'Not found column in block' error when `JOIN` appears with `TOTALS`. Fixes [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839) [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix parsing multiple hosts set in the CREATE USER command [#9924](https://github.com/ClickHouse/ClickHouse/pull/9924) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fix `TRUNCATE` for Join table engine ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)) +* Fix race condition between drop and optimize in `ReplicatedMergeTree`. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)) +* Fix `DISTINCT` for Distributed when `optimize_skip_unused_shards` is set. [#9808](https://github.com/ClickHouse/ClickHouse/pull/9808) ([Azat Khuzhin](https://github.com/azat)) +* Fix "scalar doesn't exist" error in ALTERs ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). ... [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)) +* Fix error with qualified names in `distributed_product_mode=\'local\'`. Fixes [#4756](https://github.com/ClickHouse/ClickHouse/issues/4756) [#9891](https://github.com/ClickHouse/ClickHouse/pull/9891) ([Artem Zuikov](https://github.com/4ertus2)) +* For INSERT queries shards now do clamp the settings from the initiator to their constraints instead of throwing an exception. This fix allows to send INSERT queries to a shard with another constraints. This change improves fix [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)) +* Add some retries when commiting offsets to Kafka broker, since it can reject commit if during `offsets.commit.timeout.ms` there were no enough replicas available for the `__consumer_offsets` topic [#9884](https://github.com/ClickHouse/ClickHouse/pull/9884) ([filimonov](https://github.com/filimonov)) +* Fix Distributed engine behavior when virtual columns of the underlying table used in `WHERE` [#9847](https://github.com/ClickHouse/ClickHouse/pull/9847) ([Azat Khuzhin](https://github.com/azat)) +* Fixed some cases when timezone of the function argument wasn't used properly. [#9574](https://github.com/ClickHouse/ClickHouse/pull/9574) ([Vasily Nemkov](https://github.com/Enmk)) +* Fix 'Different expressions with the same alias' error when query has PREWHERE and WHERE on distributed table and `SET distributed_product_mode = 'local'`. [#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix mutations excessive memory consumption for tables with a composite primary key. This fixes [#9850](https://github.com/ClickHouse/ClickHouse/issues/9850). [#9860](https://github.com/ClickHouse/ClickHouse/pull/9860) ([alesapin](https://github.com/alesapin)) +* Fix calculating grants for introspection functions from the setting `allow_introspection_functions`. [#9840](https://github.com/ClickHouse/ClickHouse/pull/9840) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fix max_distributed_connections (w/ and w/o Processors) [#9673](https://github.com/ClickHouse/ClickHouse/pull/9673) ([Azat Khuzhin](https://github.com/azat)) +* Fix possible exception `Got 0 in totals chunk, expected 1` on client. It happened for queries with `JOIN` in case if right joined table had zero rows. Example: `select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy limit 0 FORMAT TabSeparated;`. Fixes [#9777](https://github.com/ClickHouse/ClickHouse/issues/9777). ... [#9823](https://github.com/ClickHouse/ClickHouse/pull/9823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix 'COMMA to CROSS JOIN rewriter is not enabled or cannot rewrite query' error in case of subqueries with COMMA JOIN out of tables lists (i.e. in WHERE). Fixes [#9782](https://github.com/ClickHouse/ClickHouse/issues/9782) [#9830](https://github.com/ClickHouse/ClickHouse/pull/9830) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix server crashing when `optimize_skip_unused_shards` is set and expression for key can't be converted to its field type [#9804](https://github.com/ClickHouse/ClickHouse/pull/9804) ([Azat Khuzhin](https://github.com/azat)) +* Fix empty string handling in `splitByString`. [#9767](https://github.com/ClickHouse/ClickHouse/pull/9767) ([hcz](https://github.com/hczhcz)) +* Fix broken `ALTER TABLE DELETE COLUMN` query for compact parts. [#9779](https://github.com/ClickHouse/ClickHouse/pull/9779) ([alesapin](https://github.com/alesapin)) +* Fixed missing `rows_before_limit_at_least` for queries over http (with processors pipeline). Fixes [#9730](https://github.com/ClickHouse/ClickHouse/issues/9730) [#9757](https://github.com/ClickHouse/ClickHouse/pull/9757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix excessive memory consumption in `ALTER` queries (mutations). This fixes [#9533](https://github.com/ClickHouse/ClickHouse/issues/9533) and [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670). [#9754](https://github.com/ClickHouse/ClickHouse/pull/9754) ([alesapin](https://github.com/alesapin)) +* Fix possible permanent "Cannot schedule a task" error. [#9154](https://github.com/ClickHouse/ClickHouse/pull/9154) ([Azat Khuzhin](https://github.com/azat)) +* Fix bug in backquoting in external dictionaries DDL. Fixes [#9619](https://github.com/ClickHouse/ClickHouse/issues/9619). [#9734](https://github.com/ClickHouse/ClickHouse/pull/9734) ([alesapin](https://github.com/alesapin)) +* Fixed data race in `text_log`. It does not correspond to any real bug. [#9726](https://github.com/ClickHouse/ClickHouse/pull/9726) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix bug in a replication that doesn't allow replication to work if the user has executed mutations on the previous version. This fixes [#9645](https://github.com/ClickHouse/ClickHouse/issues/9645). [#9652](https://github.com/ClickHouse/ClickHouse/pull/9652) ([alesapin](https://github.com/alesapin)) +* Fixed incorrect internal function names for `sumKahan` and `sumWithOverflow`. It led to exception while using this functions in remote queries. [#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)) +* Add setting `use_compact_format_in_distributed_parts_names` which allows to write files for `INSERT` queries into `Distributed` table with more compact format. This fixes [#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([alesapin](https://github.com/alesapin)) +* Fix RIGHT and FULL JOIN with LowCardinality in JOIN keys. [#9610](https://github.com/ClickHouse/ClickHouse/pull/9610) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix possible exceptions `Size of filter doesn't match size of column` and `Invalid number of rows in Chunk` in `MergeTreeRangeReader`. They could appear while executing `PREWHERE` in some cases. [#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([Anton Popov](https://github.com/CurtizJ)) +* Allow `ALTER ON CLUSTER` of Distributed tables with internal replication. This fixes [#3268](https://github.com/ClickHouse/ClickHouse/issues/3268) [#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)) +* Fix issue when timezone was not preserved if you write a simple arithmetic expression like `time + 1` (in contrast to an expression like `time + INTERVAL 1 SECOND`). This fixes [#5743](https://github.com/ClickHouse/ClickHouse/issues/5743) [#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Improvement +* Use time zone when comparing DateTime with string literal. This fixes [#5206](https://github.com/ClickHouse/ClickHouse/issues/5206). [#10515](https://github.com/ClickHouse/ClickHouse/pull/10515) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Print verbose diagnostic info if Decimal value cannot be parsed from text input format. [#10205](https://github.com/ClickHouse/ClickHouse/pull/10205) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add tasks/memory metrics for distributed/buffer schedule pools [#10449](https://github.com/ClickHouse/ClickHouse/pull/10449) ([Azat Khuzhin](https://github.com/azat)) +* Display result as soon as it's ready for SELECT DISTINCT queries in clickhouse-local and HTTP interface. This fixes [#8951](https://github.com/ClickHouse/ClickHouse/issues/8951) [#9559](https://github.com/ClickHouse/ClickHouse/pull/9559) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Allow to use `SAMPLE OFFSET` query instead of `cityHash64(PRIMARY KEY) % N == n` for splitting in `clickhouse-copier`. To use this feature, pass `--experimental-use-sample-offset 1` as a command line argument. [#10414](https://github.com/ClickHouse/ClickHouse/pull/10414) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Allow to parse BOM in TSV if the first column cannot contain BOM in its value. This fixes [#10301](https://github.com/ClickHouse/ClickHouse/issues/10301) [#10424](https://github.com/ClickHouse/ClickHouse/pull/10424) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add Avro nested fields insert support [#10354](https://github.com/ClickHouse/ClickHouse/pull/10354) ([Andrew Onyshchuk](https://github.com/oandrew)) +* Allowed to alter column in non-modifying data mode when the same type is specified. [#10382](https://github.com/ClickHouse/ClickHouse/pull/10382) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Auto `distributed_group_by_no_merge` on GROUP BY sharding key (if `optimize_skip_unused_shards` is set) [#10341](https://github.com/ClickHouse/ClickHouse/pull/10341) ([Azat Khuzhin](https://github.com/azat)) +* Optimize queries with LIMIT/LIMIT BY/ORDER BY for distributed with GROUP BY sharding_key [#10373](https://github.com/ClickHouse/ClickHouse/pull/10373) ([Azat Khuzhin](https://github.com/azat)) +* Added a setting `max_server_memory_usage` to limit total memory usage of the server. The metric `MemoryTracking` is now calculated without a drift. The setting `max_memory_usage_for_all_queries` is now obsolete and does nothing. This closes [#10293](https://github.com/ClickHouse/ClickHouse/issues/10293). [#10362](https://github.com/ClickHouse/ClickHouse/pull/10362) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add config option `system_tables_lazy_load`. If it's set to false, then system tables with logs are loaded at the server startup. [Alexander Burmak](https://github.com/Alex-Burmak), [Svyatoslav Tkhon Il Pak](https://github.com/DeifyTheGod), [#9642](https://github.com/ClickHouse/ClickHouse/pull/9642) [#10359](https://github.com/ClickHouse/ClickHouse/pull/10359) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Use background thread pool (background_schedule_pool_size) for distributed sends [#10263](https://github.com/ClickHouse/ClickHouse/pull/10263) ([Azat Khuzhin](https://github.com/azat)) +* Use background thread pool for background buffer flushes. [#10315](https://github.com/ClickHouse/ClickHouse/pull/10315) ([Azat Khuzhin](https://github.com/azat)) +* Support for one special case of removing incompletely written parts. This fixes [#9940](https://github.com/ClickHouse/ClickHouse/issues/9940). [#10221](https://github.com/ClickHouse/ClickHouse/pull/10221) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Use isInjective() over manual list of such functions for GROUP BY optimization. [#10342](https://github.com/ClickHouse/ClickHouse/pull/10342) ([Azat Khuzhin](https://github.com/azat)) +* Avoid printing error message in log if client sends RST packet immediately on connect. It is typical behaviour of IPVS balancer with keepalived and VRRP. This fixes [#1851](https://github.com/ClickHouse/ClickHouse/issues/1851) [#10274](https://github.com/ClickHouse/ClickHouse/pull/10274) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Allow to parse `+inf` for floating point types. This closes [#1839](https://github.com/ClickHouse/ClickHouse/issues/1839) [#10272](https://github.com/ClickHouse/ClickHouse/pull/10272) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Implemented `generateRandom` table function for Nested types. This closes [#9903](https://github.com/ClickHouse/ClickHouse/issues/9903) [#10219](https://github.com/ClickHouse/ClickHouse/pull/10219) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Provide `max_allowed_packed` in MySQL compatibility interface that will help some clients to communicate with ClickHouse via MySQL protocol. [#10199](https://github.com/ClickHouse/ClickHouse/pull/10199) ([BohuTANG](https://github.com/BohuTANG)) +* Allow literals for GLOBAL IN (i.e. `SELECT * FROM remote('localhost', system.one) WHERE dummy global in (0)`) [#10196](https://github.com/ClickHouse/ClickHouse/pull/10196) ([Azat Khuzhin](https://github.com/azat)) +* Fix various small issues in interactive mode of clickhouse-client [#10194](https://github.com/ClickHouse/ClickHouse/pull/10194) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Avoid superfluous dictionaries load (system.tables, DROP/SHOW CREATE TABLE) [#10164](https://github.com/ClickHouse/ClickHouse/pull/10164) ([Azat Khuzhin](https://github.com/azat)) +* Update to RWLock: timeout parameter for getLock() + implementation reworked to be phase fair [#10073](https://github.com/ClickHouse/ClickHouse/pull/10073) ([Alexander Kazakov](https://github.com/Akazz)) +* Enhanced compatibility with native mysql-connector-java(JDBC) [#10021](https://github.com/ClickHouse/ClickHouse/pull/10021) ([BohuTANG](https://github.com/BohuTANG)) +* The function `toString` is considered monotonic and can be used for index analysis even when applied in tautological cases with String or LowCardinality(String) argument. [#10110](https://github.com/ClickHouse/ClickHouse/pull/10110) ([Amos Bird](https://github.com/amosbird)) +* Add `ON CLUSTER` clause support to commands `{CREATE|DROP} USER/ROLE/ROW POLICY/SETTINGS PROFILE/QUOTA`, `GRANT`. [#9811](https://github.com/ClickHouse/ClickHouse/pull/9811) ([Vitaly Baranov](https://github.com/vitlibar)) +* Virtual hosted-style support for S3 URI [#9998](https://github.com/ClickHouse/ClickHouse/pull/9998) ([Pavel Kovalenko](https://github.com/Jokser)) +* Now layout type for dictionaries with no arguments can be specified without round brackets in dictionaries DDL-queries. Fixes [#10057](https://github.com/ClickHouse/ClickHouse/issues/10057). [#10064](https://github.com/ClickHouse/ClickHouse/pull/10064) ([alesapin](https://github.com/alesapin)) +* Add ability to use number ranges with leading zeros in filepath [#9989](https://github.com/ClickHouse/ClickHouse/pull/9989) ([Olga Khvostikova](https://github.com/stavrolia)) +* Better memory usage in CROSS JOIN. [#10029](https://github.com/ClickHouse/ClickHouse/pull/10029) ([Artem Zuikov](https://github.com/4ertus2)) +* Try to connect to all shards in cluster when getting structure of remote table and skip_unavailable_shards is set. [#7278](https://github.com/ClickHouse/ClickHouse/pull/7278) ([nvartolomei](https://github.com/nvartolomei)) +* Add `total_rows`/`total_bytes` into the `system.tables` table. [#9919](https://github.com/ClickHouse/ClickHouse/pull/9919) ([Azat Khuzhin](https://github.com/azat)) +* System log tables now use polymorpic parts by default. [#9905](https://github.com/ClickHouse/ClickHouse/pull/9905) ([Anton Popov](https://github.com/CurtizJ)) +* Add type column into system.settings/merge_tree_settings [#9909](https://github.com/ClickHouse/ClickHouse/pull/9909) ([Azat Khuzhin](https://github.com/azat)) +* Check for available CPU instructions at server startup as early as possible. [#9888](https://github.com/ClickHouse/ClickHouse/pull/9888) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Remove `ORDER BY` stage from mutations because we read from a single ordered part in a single thread. Also add check that the rows in mutation are ordered by sorting key and this order is not violated. [#9886](https://github.com/ClickHouse/ClickHouse/pull/9886) ([alesapin](https://github.com/alesapin)) +* Implement operator LIKE for FixedString at left hand side. This is needed to better support TPC-DS queries. [#9890](https://github.com/ClickHouse/ClickHouse/pull/9890) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add `force_optimize_skip_unused_shards_no_nested` that will disable `force_optimize_skip_unused_shards` for nested Distributed table [#9812](https://github.com/ClickHouse/ClickHouse/pull/9812) ([Azat Khuzhin](https://github.com/azat)) +* Now columns size is calculated only once for MergeTree data parts. [#9827](https://github.com/ClickHouse/ClickHouse/pull/9827) ([alesapin](https://github.com/alesapin)) +* Evaluate constant expressions for `optimize_skip_unused_shards` (i.e. `SELECT * FROM foo_dist WHERE key=xxHash32(0)`) [#8846](https://github.com/ClickHouse/ClickHouse/pull/8846) ([Azat Khuzhin](https://github.com/azat)) +* Check for using `Date` or `DateTime` column from TTL expressions was removed. [#9967](https://github.com/ClickHouse/ClickHouse/pull/9967) ([Vladimir Chebotarev](https://github.com/excitoon)) +* DiskS3 hard links optimal implementation. [#9760](https://github.com/ClickHouse/ClickHouse/pull/9760) ([Pavel Kovalenko](https://github.com/Jokser)) +* If `set multiple_joins_rewriter_version = 2` enables second version of multiple JOIN rewrites that keeps not clashed column names as is. It supports multiple JOINs with `USING` and allow `select *` for JOINs with subqueries. [#9739](https://github.com/ClickHouse/ClickHouse/pull/9739) ([Artem Zuikov](https://github.com/4ertus2)) +* Implementation of "non-blocking" alter for StorageMergeTree [#9606](https://github.com/ClickHouse/ClickHouse/pull/9606) ([alesapin](https://github.com/alesapin)) +* Add MergeTree full support for DiskS3 [#9646](https://github.com/ClickHouse/ClickHouse/pull/9646) ([Pavel Kovalenko](https://github.com/Jokser)) +* Extend `splitByString` to support empty strings as separators. [#9742](https://github.com/ClickHouse/ClickHouse/pull/9742) ([hcz](https://github.com/hczhcz)) +* Add a `timestamp_ns` column to `system.trace_log`. It contains a high-definition timestamp of the trace event, and allows to build timelines of thread profiles ("flame charts"). [#9696](https://github.com/ClickHouse/ClickHouse/pull/9696) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* When the setting `send_logs_level` is enabled, avoid intermixing of log messages and query progress. [#9634](https://github.com/ClickHouse/ClickHouse/pull/9634) ([Azat Khuzhin](https://github.com/azat)) +* Added support of `MATERIALIZE TTL IN PARTITION`. [#9581](https://github.com/ClickHouse/ClickHouse/pull/9581) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Support complex types inside Avro nested fields [#10502](https://github.com/ClickHouse/ClickHouse/pull/10502) ([Andrew Onyshchuk](https://github.com/oandrew)) + +#### Performance Improvement +* Better insert logic for right table for Partial MergeJoin. [#10467](https://github.com/ClickHouse/ClickHouse/pull/10467) ([Artem Zuikov](https://github.com/4ertus2)) +* Improved performance of row-oriented formats (more than 10% for CSV and more than 35% for Avro in case of narrow tables). [#10503](https://github.com/ClickHouse/ClickHouse/pull/10503) ([Andrew Onyshchuk](https://github.com/oandrew)) +* Improved performance of queries with explicitly defined sets at right side of IN operator and tuples on the left side. [#10385](https://github.com/ClickHouse/ClickHouse/pull/10385) ([Anton Popov](https://github.com/CurtizJ)) +* Use less memory for hash table in HashJoin. [#10416](https://github.com/ClickHouse/ClickHouse/pull/10416) ([Artem Zuikov](https://github.com/4ertus2)) +* Special HashJoin over StorageDictionary. Allow rewrite `dictGet()` functions with JOINs. It's not backward incompatible itself but could uncover [#8400](https://github.com/ClickHouse/ClickHouse/issues/8400) on some installations. [#10133](https://github.com/ClickHouse/ClickHouse/pull/10133) ([Artem Zuikov](https://github.com/4ertus2)) +* Enable parallel insert of materialized view when its target table supports. [#10052](https://github.com/ClickHouse/ClickHouse/pull/10052) ([vxider](https://github.com/Vxider)) +* Improved performance of index analysis with monotonic functions. [#9607](https://github.com/ClickHouse/ClickHouse/pull/9607)[#10026](https://github.com/ClickHouse/ClickHouse/pull/10026) ([Anton Popov](https://github.com/CurtizJ)) +* Using SSE2 or SSE4.2 SIMD intrinsics to speed up tokenization in bloom filters. [#9968](https://github.com/ClickHouse/ClickHouse/pull/9968) ([Vasily Nemkov](https://github.com/Enmk)) +* Improved performance of queries with explicitly defined sets at right side of `IN` operator. This fixes performance regression in version 20.3. [#9740](https://github.com/ClickHouse/ClickHouse/pull/9740) ([Anton Popov](https://github.com/CurtizJ)) +* Now clickhouse-copier splits each partition in number of pieces and copies them independently. [#9075](https://github.com/ClickHouse/ClickHouse/pull/9075) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Adding more aggregation methods. For example TPC-H query 1 will now pick `FixedHashMap` and gets 25% performance gain [#9829](https://github.com/ClickHouse/ClickHouse/pull/9829) ([Amos Bird](https://github.com/amosbird)) +* Use single row counter for multiple streams in pre-limit transform. This helps to avoid uniting pipeline streams in queries with `limit` but without `order by` (like `select f(x) from (select x from t limit 1000000000)`) and use multiple threads for further processing. [#9602](https://github.com/ClickHouse/ClickHouse/pull/9602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + +#### Build/Testing/Packaging Improvement +* Use a fork of AWS SDK libraries from ClickHouse-Extras [#10527](https://github.com/ClickHouse/ClickHouse/pull/10527) ([Pavel Kovalenko](https://github.com/Jokser)) +* Add integration tests for new ALTER RENAME COLUMN query. [#10654](https://github.com/ClickHouse/ClickHouse/pull/10654) ([vzakaznikov](https://github.com/vzakaznikov)) +* Fix possible signed integer overflow in invocation of function `now64` with wrong arguments. This fixes [#8973](https://github.com/ClickHouse/ClickHouse/issues/8973) [#10511](https://github.com/ClickHouse/ClickHouse/pull/10511) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Split fuzzer and sanitizer configurations to make build config compatible with Oss-fuzz. [#10494](https://github.com/ClickHouse/ClickHouse/pull/10494) ([kyprizel](https://github.com/kyprizel)) +* Fixes for clang-tidy on clang-10. [#10420](https://github.com/ClickHouse/ClickHouse/pull/10420) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Display absolute paths in error messages. Otherwise KDevelop fails to navigate to correct file and opens a new file instead. [#10434](https://github.com/ClickHouse/ClickHouse/pull/10434) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added `ASAN_OPTIONS` environment variable to investigate errors in CI stress tests with Address sanitizer. [#10440](https://github.com/ClickHouse/ClickHouse/pull/10440) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Enable ThinLTO for clang builds (experimental). [#10435](https://github.com/ClickHouse/ClickHouse/pull/10435) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Remove accidential dependency on Z3 that may be introduced if the system has Z3 solver installed. [#10426](https://github.com/ClickHouse/ClickHouse/pull/10426) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Move integration tests docker files to docker/ directory. [#10335](https://github.com/ClickHouse/ClickHouse/pull/10335) ([Ilya Yatsishin](https://github.com/qoega)) +* Allow to use `clang-10` in CI. It ensures that [#10238](https://github.com/ClickHouse/ClickHouse/issues/10238) is fixed. [#10384](https://github.com/ClickHouse/ClickHouse/pull/10384) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Update OpenSSL to upstream master. Fixed the issue when TLS connections may fail with the message `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error` and `SSL Exception: error:2400006E:random number generator::error retrieving entropy`. The issue was present in version 20.1. [#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) ([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)) +* Add performance test for [Parallel INSERT for materialized view](https://github.com/ClickHouse/ClickHouse/pull/10052). [#10345](https://github.com/ClickHouse/ClickHouse/pull/10345) ([vxider](https://github.com/Vxider)) +* Fix flaky test `test_settings_constraints_distributed.test_insert_clamps_settings`. [#10346](https://github.com/ClickHouse/ClickHouse/pull/10346) ([Vitaly Baranov](https://github.com/vitlibar)) +* Add util to test results upload in CI ClickHouse [#10330](https://github.com/ClickHouse/ClickHouse/pull/10330) ([Ilya Yatsishin](https://github.com/qoega)) +* Convert test results to JSONEachRow format in junit_to_html tool [#10323](https://github.com/ClickHouse/ClickHouse/pull/10323) ([Ilya Yatsishin](https://github.com/qoega)) +* Update cctz. [#10215](https://github.com/ClickHouse/ClickHouse/pull/10215) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Allow to create HTML report from the purest JUnit XML report. [#10247](https://github.com/ClickHouse/ClickHouse/pull/10247) ([Ilya Yatsishin](https://github.com/qoega)) +* Update the check for minimal compiler version. Fix the root cause of the issue [#10250](https://github.com/ClickHouse/ClickHouse/issues/10250) [#10256](https://github.com/ClickHouse/ClickHouse/pull/10256) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Initial support for live view tables over distributed [#10179](https://github.com/ClickHouse/ClickHouse/pull/10179) ([vzakaznikov](https://github.com/vzakaznikov)) +* Fix (false) MSan report in MergeTreeIndexFullText. The issue first appeared in [#9968](https://github.com/ClickHouse/ClickHouse/issues/9968). [#10801](https://github.com/ClickHouse/ClickHouse/pull/10801) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* clickhouse-docker-util [#10151](https://github.com/ClickHouse/ClickHouse/pull/10151) ([filimonov](https://github.com/filimonov)) +* Update pdqsort to recent version [#10171](https://github.com/ClickHouse/ClickHouse/pull/10171) ([Ivan](https://github.com/abyss7)) +* Update libdivide to v3.0 [#10169](https://github.com/ClickHouse/ClickHouse/pull/10169) ([Ivan](https://github.com/abyss7)) +* Add check with enabled polymorphic parts. [#10086](https://github.com/ClickHouse/ClickHouse/pull/10086) ([Anton Popov](https://github.com/CurtizJ)) +* Add cross-compile build for FreeBSD. This fixes [#9465](https://github.com/ClickHouse/ClickHouse/issues/9465) [#9643](https://github.com/ClickHouse/ClickHouse/pull/9643) ([Ivan](https://github.com/abyss7)) +* Add performance test for [#6924](https://github.com/ClickHouse/ClickHouse/issues/6924) [#6980](https://github.com/ClickHouse/ClickHouse/pull/6980) ([filimonov](https://github.com/filimonov)) +* Add support of `/dev/null` in the `File` engine for better performance testing [#8455](https://github.com/ClickHouse/ClickHouse/pull/8455) ([Amos Bird](https://github.com/amosbird)) +* Move all folders inside /dbms one level up [#9974](https://github.com/ClickHouse/ClickHouse/pull/9974) ([Ivan](https://github.com/abyss7)) +* Add a test that checks that read from MergeTree with single thread is performed in order. Addition to [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670) [#9762](https://github.com/ClickHouse/ClickHouse/pull/9762) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix the `00964_live_view_watch_events_heartbeat.py` test to avoid race condition. [#9944](https://github.com/ClickHouse/ClickHouse/pull/9944) ([vzakaznikov](https://github.com/vzakaznikov)) +* Fix integration test `test_settings_constraints` [#9962](https://github.com/ClickHouse/ClickHouse/pull/9962) ([Vitaly Baranov](https://github.com/vitlibar)) +* Every function in its own file, part 12. [#9922](https://github.com/ClickHouse/ClickHouse/pull/9922) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added performance test for the case of extremely slow analysis of array of tuples. [#9872](https://github.com/ClickHouse/ClickHouse/pull/9872) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Update zstd to 1.4.4. It has some minor improvements in performance and compression ratio. If you run replicas with different versions of ClickHouse you may see reasonable error messages `Data after merge is not byte-identical to data on another replicas.` with explanation. These messages are Ok and you should not worry. [#10663](https://github.com/ClickHouse/ClickHouse/pull/10663) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix TSan report in `system.stack_trace`. [#9832](https://github.com/ClickHouse/ClickHouse/pull/9832) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Removed dependency on `clock_getres`. [#9833](https://github.com/ClickHouse/ClickHouse/pull/9833) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added identifier names check with clang-tidy. [#9799](https://github.com/ClickHouse/ClickHouse/pull/9799) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Update "builder" docker image. This image is not used in CI but is useful for developers. [#9809](https://github.com/ClickHouse/ClickHouse/pull/9809) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Remove old `performance-test` tool that is no longer used in CI. `clickhouse-performance-test` is great but now we are using way superior tool that is doing comparison testing with sophisticated statistical formulas to achieve confident results regardless to various changes in environment. [#9796](https://github.com/ClickHouse/ClickHouse/pull/9796) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added most of clang-static-analyzer checks. [#9765](https://github.com/ClickHouse/ClickHouse/pull/9765) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Update Poco to 1.9.3 in preparation for MongoDB URI support. [#6892](https://github.com/ClickHouse/ClickHouse/pull/6892) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Fix build with `-DUSE_STATIC_LIBRARIES=0 -DENABLE_JEMALLOC=0` [#9651](https://github.com/ClickHouse/ClickHouse/pull/9651) ([Artem Zuikov](https://github.com/4ertus2)) +* For change log script, if merge commit was cherry-picked to release branch, take PR name from commit description. [#9708](https://github.com/ClickHouse/ClickHouse/pull/9708) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Support `vX.X-conflicts` tag in backport script. [#9705](https://github.com/ClickHouse/ClickHouse/pull/9705) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix `auto-label` for backporting script. [#9685](https://github.com/ClickHouse/ClickHouse/pull/9685) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Use libc++ in Darwin cross-build to make it consistent with native build. [#9665](https://github.com/ClickHouse/ClickHouse/pull/9665) ([Hui Wang](https://github.com/huiwang)) +* Fix flacky test `01017_uniqCombined_memory_usage`. Continuation of [#7236](https://github.com/ClickHouse/ClickHouse/issues/7236). [#9667](https://github.com/ClickHouse/ClickHouse/pull/9667) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix build for native MacOS Clang compiler [#9649](https://github.com/ClickHouse/ClickHouse/pull/9649) ([Ivan](https://github.com/abyss7)) +* Allow to add various glitches around `pthread_mutex_lock`, `pthread_mutex_unlock` functions. [#9635](https://github.com/ClickHouse/ClickHouse/pull/9635) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add support for `clang-tidy` in `packager` script. [#9625](https://github.com/ClickHouse/ClickHouse/pull/9625) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add ability to use unbundled msgpack. [#10168](https://github.com/ClickHouse/ClickHouse/pull/10168) ([Azat Khuzhin](https://github.com/azat)) + + ## ClickHouse release v20.3 ### ClickHouse release v20.3.8.53, 2020-04-23 -### Bug Fix +#### Bug Fix * Fixed wrong behaviour of datetime functions for timezones that has altered between positive and negative offsets from UTC (e.g. Pacific/Kiritimati). This fixes [#7202](https://github.com/ClickHouse/ClickHouse/issues/7202) [#10369](https://github.com/ClickHouse/ClickHouse/pull/10369) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix possible segfault with `distributed_group_by_no_merge` enabled (introduced in 20.3.7.46 by [#10131](https://github.com/ClickHouse/ClickHouse/issues/10131)). [#10399](https://github.com/ClickHouse/ClickHouse/pull/10399) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) * Fix 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)) @@ -18,7 +298,7 @@ * Fix 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). Author: [Vadim Plakhtinskiy](https://github.com/VadimPlh). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Check the 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)) -### Performance Improvement +#### Performance Improvement * Improved performance of queries with explicitly defined sets at right side of `IN` operator and tuples in the left side. This fixes performance regression in version 20.3. [#9740](https://github.com/ClickHouse/ClickHouse/pull/9740), [#10385](https://github.com/ClickHouse/ClickHouse/pull/10385) ([Anton Popov](https://github.com/CurtizJ)) ### ClickHouse release v20.3.7.46, 2020-04-17 @@ -59,7 +339,6 @@ * Fix bug in `CHECK TABLE` query when table contain skip indices. [#10068](https://github.com/ClickHouse/ClickHouse/pull/10068) ([alesapin](https://github.com/alesapin)). * Fix error `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform`. It happened when setting `distributed_aggregation_memory_efficient` was enabled, and distributed query read aggregating data with different level from different shards (mixed single and two level aggregation). [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix a segmentation fault that could occur in GROUP BY over string keys containing trailing zero bytes ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Fix parallel distributed INSERT SELECT for remote table. This PR fixes the solution provided in [#9759](https://github.com/ClickHouse/ClickHouse/pull/9759). [#9999](https://github.com/ClickHouse/ClickHouse/pull/9999) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix the number of threads used for remote query execution (performance regression, since 20.3). This happened when query from `Distributed` table was executed simultaneously on local and remote shards. Fixes [#9965](https://github.com/ClickHouse/ClickHouse/issues/9965). [#9971](https://github.com/ClickHouse/ClickHouse/pull/9971) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix bug in which the necessary tables weren't retrieved at one of the processing stages of queries to some databases. Fixes [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)). * Fix 'Not found column in block' error when `JOIN` appears with `TOTALS`. Fixes [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839). [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)). diff --git a/CMakeLists.txt b/CMakeLists.txt index fb36aff6603..53dfd1df1cb 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -385,9 +385,6 @@ if (OS_LINUX AND NOT ENABLE_JEMALLOC) endif () if (USE_OPENCL) - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -DUSE_OPENCL=1") - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -DUSE_OPENCL=1") - if (OS_DARWIN) set(OPENCL_LINKER_FLAGS "-framework OpenCL") set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${OPENCL_LINKER_FLAGS}") diff --git a/cmake/find/opencl.cmake b/cmake/find/opencl.cmake index b1bf4630990..0f307350cb8 100644 --- a/cmake/find/opencl.cmake +++ b/cmake/find/opencl.cmake @@ -1,13 +1,19 @@ +# TODO: enable by default +if(0) + option(ENABLE_OPENCL "Enable OpenCL support" ${ENABLE_LIBRARIES}) +endif() + if(ENABLE_OPENCL) # Intel OpenCl driver: sudo apt install intel-opencl-icd -# TODO It's possible to add it as submodules: https://github.com/intel/compute-runtime/releases +# @sa https://github.com/intel/compute-runtime/releases # OpenCL applications should link wiht ICD loader # sudo apt install opencl-headers ocl-icd-libopencl1 # sudo ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so +# TODO: add https://github.com/OCL-dev/ocl-icd as submodule instead -find_package(OpenCL REQUIRED) +find_package(OpenCL) if(OpenCL_FOUND) set(USE_OPENCL 1) endif() diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 05c055107ac..af88a3954d7 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -27,7 +27,7 @@ function configure kill -0 $left_pid disown $left_pid set +m - while ! clickhouse-client --port 9001 --query "select 1" ; do kill -0 $left_pid ; echo . ; sleep 1 ; done + while ! clickhouse-client --port 9001 --query "select 1" && kill -0 $left_pid ; do echo . ; sleep 1 ; done echo server for setup started clickhouse-client --port 9001 --query "create database test" ||: @@ -71,9 +71,9 @@ function restart set +m - while ! clickhouse-client --port 9001 --query "select 1" ; do kill -0 $left_pid ; echo . ; sleep 1 ; done + while ! clickhouse-client --port 9001 --query "select 1" && kill -0 $left_pid ; do echo . ; sleep 1 ; done echo left ok - while ! clickhouse-client --port 9002 --query "select 1" ; do kill -0 $right_pid ; echo . ; sleep 1 ; done + while ! clickhouse-client --port 9002 --query "select 1" && kill -0 $right_pid ; do echo . ; sleep 1 ; done echo right ok clickhouse-client --port 9001 --query "select * from system.tables where database != 'system'" @@ -263,7 +263,7 @@ done wait unset IFS -parallel --verbose --null < analyze-commands.txt +parallel --null < analyze-commands.txt } # Analyze results @@ -314,6 +314,25 @@ create table queries_old_format engine File(TSVWithNamesAndTypes, 'queries.rep') from queries ; +-- save all test runs as JSON for the new comparison page +create table all_query_funs_json engine File(JSON, 'report/all-query-runs.json') as + select test, query, versions_runs[1] runs_left, versions_runs[2] runs_right + from ( + select + test, query, + groupArrayInsertAt(runs, version) versions_runs + from ( + select + replaceAll(_file, '-queries.tsv', '') test, + query, version, + groupArray(time) runs + from file('*-queries.tsv', TSV, 'query text, run int, version UInt32, time float') + group by test, query, version + ) + group by test, query + ) + ; + create table changed_perf_tsv engine File(TSV, 'report/changed-perf.tsv') as select left, right, diff, stat_threshold, changed_fail, test, query from queries where changed_show order by abs(diff) desc; @@ -542,7 +561,7 @@ case "$stage" in # to collect the logs. Prefer not to restart, because addresses might change # and we won't be able to process trace_log data. Start in a subshell, so that # it doesn't interfere with the watchdog through `wait`. - ( time get_profiles || restart || get_profiles ||: ) + ( get_profiles || restart || get_profiles ||: ) # Kill the whole process group, because somehow when the subshell is killed, # the sleep inside remains alive and orphaned. diff --git a/docs/_description_templates/template-system-table.md b/docs/_description_templates/template-system-table.md index 9c05bf4cc5d..137766a34b6 100644 --- a/docs/_description_templates/template-system-table.md +++ b/docs/_description_templates/template-system-table.md @@ -1,10 +1,10 @@ -## system.table\_name {#system-tables_table-name} +## system.table_name {#system-tables_table-name} Description. Columns: -- `column_name` ([data\_type\_name](path/to/data_type.md)) — Description. +- `column_name` ([data_type_name](path/to/data_type.md)) — Description. **Example** diff --git a/docs/en/development/build-osx.md b/docs/en/development/build-osx.md index 281f961fc2d..02da3ba1390 100644 --- a/docs/en/development/build-osx.md +++ b/docs/en/development/build-osx.md @@ -5,7 +5,7 @@ toc_title: How to Build ClickHouse on Mac OS X # How to Build ClickHouse on Mac OS X {#how-to-build-clickhouse-on-mac-os-x} -Build should work on Mac OS X 10.15 (Catalina) +Build should work on Mac OS X 10.15 (Catalina). ## Install Homebrew {#install-homebrew} diff --git a/docs/en/engines/index.md b/docs/en/engines/index.md index c4b0b299858..fe7e46fb534 100644 --- a/docs/en/engines/index.md +++ b/docs/en/engines/index.md @@ -3,4 +3,4 @@ toc_folder_title: Engines toc_priority: 25 --- - +{## [Original article](https://clickhouse.tech/docs/en/engines/) ##} diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 8f8c09f6632..e917b8ef58c 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -50,7 +50,7 @@ sudo rpm --import https://repo.clickhouse.tech/CLICKHOUSE-KEY.GPG sudo yum-config-manager --add-repo https://repo.clickhouse.tech/rpm/stable/x86_64 ``` -If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). The `prestable` tag is sometimes available too. +If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). `prestable` is sometimes also available. Then run these commands to install packages: diff --git a/docs/en/getting-started/playground.md b/docs/en/getting-started/playground.md index 36595d1998a..bed1618314b 100644 --- a/docs/en/getting-started/playground.md +++ b/docs/en/getting-started/playground.md @@ -8,27 +8,27 @@ toc_title: Playground [ClickHouse Playground](https://play.clickhouse.tech) allows people to experiment with ClickHouse by running queries instantly, without setting up their server or cluster. Several example datasets are available in the Playground as well as sample queries that show ClickHouse features. There's also a selection of ClickHouse LTS releases to experiment with. -ClickHouse Playground gives the experience of m2.small [Managed Service for ClickHouse](https://cloud.yandex.com/services/managed-clickhouse) instance hosted in [Yandex.Cloud](https://cloud.yandex.com/). More information about [cloud providers](../commercial/cloud.md). +ClickHouse Playground gives the experience of m2.small [Managed Service for ClickHouse](https://cloud.yandex.com/services/managed-clickhouse) instance (4 vCPU, 32 GB RAM) hosted in [Yandex.Cloud](https://cloud.yandex.com/). More information about [cloud providers](../commercial/cloud.md). You can make queries to playground using any HTTP client, for example [curl](https://curl.haxx.se) or [wget](https://www.gnu.org/software/wget/), or set up a connection using [JDBC](../interfaces/jdbc.md) or [ODBC](../interfaces/odbc.md) drivers. More information about software products that support ClickHouse is available [here](../interfaces/index.md). ## Credentials -| Parameter | Value | -|:------------------|:----------------------------------------| -| HTTPS endpoint | `https://play-api.clickhouse.tech:8443` | -| Native endpoint | `play-api.clickhouse.tech:9440` | -| User | `playground` | -| Password | `clickhouse` | - -!!! note "Note" - Note that all endpoints require a secure TLS connection. +| Parameter | Value | +|:--------------------|:----------------------------------------| +| HTTPS endpoint | `https://play-api.clickhouse.tech:8443` | +| Native TCP endpoint | `play-api.clickhouse.tech:9440` | +| User | `playground` | +| Password | `clickhouse` | There are additional endpoints with specific ClickHouse releases to experiment with their differences (ports and user/password are the same as above): * 20.3 LTS: `play-api-v20-3.clickhouse.tech` * 19.14 LTS: `play-api-v19-14.clickhouse.tech` +!!! note "Note" + All these endpoints require a secure TLS connection. + ## Limitations The queries are executed as a read-only user. It implies some limitations: @@ -50,7 +50,7 @@ HTTPS endpoint example with `curl`: curl "https://play-api.clickhouse.tech:8443/?query=SELECT+'Play+ClickHouse!';&user=playground&password=clickhouse&database=datasets" ``` -TCP endpoint example with [../interfaces/cli.md]: +TCP endpoint example with [CLI](../interfaces/cli.md): ``` bash clickhouse client --secure -h play-api.clickhouse.tech --port 9440 -u playground --password clickhouse -q "SELECT 'Play ClickHouse!'" ``` diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index c3fa509165e..8868522e977 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -11,7 +11,7 @@ toc_title: Adopters | Company | Industry | Usecase | Cluster Size | (Un)Compressed Data Size\* | Reference | |---------------------------------------------------------------------|---------------------------------|-----------------------|------------------------------------------------------------|------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | [2gis](https://2gis.ru){.favicon} | Maps | Monitoring | — | — | [Talk in Russian, July 2019](https://youtu.be/58sPkXfq6nw) | -| [Aloha Browser](https://alohabrowser.com/){.favicon} | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://github.com/yandex/clickhouse-presentations/blob/master/meetup22/aloha.pdf) | +| [Aloha Browser](https://alohabrowser.com/){.favicon} | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.tech/meetup22/aloha.pdf) | | [Amadeus](https://amadeus.com/){.favicon} | Travel | Analytics | — | — | [Press Release, April 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | | [Appsflyer](https://www.appsflyer.com){.favicon} | Mobile analytics | Main product | — | — | [Talk in Russian, July 2019](https://www.youtube.com/watch?v=M3wbRlcpBbY) | | [ArenaData](https://arenadata.tech/){.favicon} | Data Platform | Main product | — | — | [Slides in Russian, December 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup38/indexes.pdf) | diff --git a/docs/en/introduction/performance.md b/docs/en/introduction/performance.md index 73aacc293b1..d2780aedccb 100644 --- a/docs/en/introduction/performance.md +++ b/docs/en/introduction/performance.md @@ -27,4 +27,4 @@ Under the same conditions, ClickHouse can handle several hundred queries per sec We recommend inserting data in packets of at least 1000 rows, or no more than a single request per second. When inserting to a MergeTree table from a tab-separated dump, the insertion speed can be from 50 to 200 MB/s. If the inserted rows are around 1 Kb in size, the speed will be from 50,000 to 200,000 rows per second. If the rows are small, the performance can be higher in rows per second (on Banner System data -`>` 500,000 rows per second; on Graphite data -`>` 1,000,000 rows per second). To improve performance, you can make multiple INSERT queries in parallel, which scales linearly. -[Original article](https://clickhouse.tech/docs/en/introduction/performance/) +{## [Original article](https://clickhouse.tech/docs/en/introduction/performance/) ##} diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 3a11785d6ba..93acf3cae7a 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -207,7 +207,7 @@ If `http_port` is specified, the OpenSSL configuration is ignored even if it is **Example** ``` xml -0000 +9999 ``` ## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} @@ -733,7 +733,7 @@ Example 9004 ``` -## tmp\_path {#server-settings-tmp_path} +## tmp_path {#tmp-path} Path to temporary data for processing large queries. @@ -746,16 +746,17 @@ Path to temporary data for processing large queries. /var/lib/clickhouse/tmp/ ``` -## tmp\_policy {#server-settings-tmp-policy} +## tmp_policy {#tmp-policy} -Policy from [`storage_configuration`](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) to store temporary files. -If not set [`tmp_path`](#server-settings-tmp_path) is used, otherwise it is ignored. +Policy from [storage_configuration](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) to store temporary files. + +If not set, [tmp_path](#tmp-path) is used, otherwise it is ignored. !!! note "Note" - - `move_factor` is ignored -- `keep_free_space_bytes` is ignored -- `max_data_part_size_bytes` is ignored -- you must have exactly one volume in that policy + - `move_factor` is ignored. + - `keep_free_space_bytes` is ignored. + - `max_data_part_size_bytes` is ignored. + - Уou must have exactly one volume in that policy. ## uncompressed\_cache\_size {#server-settings-uncompressed_cache_size} diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 67097103cc9..e6e692315f0 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1026,27 +1026,32 @@ Possible values: Default value: 0. -## optimize\_skip\_unused\_shards {#settings-optimize_skip_unused_shards} +## optimize_skip_unused_shards {#optimize-skip-unused-shards} -Enables or disables skipping of unused shards for SELECT queries that have sharding key condition in PREWHERE/WHERE (assumes that the data is distributed by sharding key, otherwise do nothing). - -Default value: 0 - -## force\_optimize\_skip\_unused\_shards {#settings-force_optimize_skip_unused_shards} - -Enables or disables query execution if [`optimize_skip_unused_shards`](#settings-optimize_skip_unused_shards) enabled and skipping of unused shards is not possible. If the skipping is not possible and the setting is enabled exception will be thrown. +Enables or disables skipping of unused shards for [SELECT](../../sql-reference/statements/select/index.md) queries that have sharding key condition in `WHERE/PREWHERE` (assuming that the data is distributed by sharding key, otherwise does nothing). Possible values: -- 0 - Disabled (do not throws) -- 1 - Disable query execution only if the table has sharding key -- 2 - Disable query execution regardless sharding key is defined for the table +- 0 — Disabled. +- 1 — Enabled. + +Default value: 0 + +## force_optimize_skip_unused_shards {#force-optimize-skip-unused-shards} + +Enables or disables query execution if [optimize_skip_unused_shards](#optimize-skip-unused-shards) is enabled and skipping of unused shards is not possible. If the skipping is not possible and the setting is enabled, an exception will be thrown. + +Possible values: + +- 0 — Disabled. ClickHouse doesn't throw an exception. +- 1 — Enabled. Query execution is disabled only if the table has a sharding key. +- 2 — Enabled. Query execution is disabled regardless of whether a sharding key is defined for the table. Default value: 0 ## force\_optimize\_skip\_unused\_shards\_no\_nested {#settings-force_optimize_skip_unused_shards_no_nested} -Reset [`optimize_skip_unused_shards`](#settings-force_optimize_skip_unused_shards) for nested `Distributed` table +Reset [`optimize_skip_unused_shards`](#optimize-skip-unused-shards) for nested `Distributed` table Possible values: @@ -1250,7 +1255,9 @@ Default value: Empty ## background\_pool\_size {#background_pool_size} -Sets the number of threads performing background operations in table engines (for example, merges in [MergeTree engine](../../engines/table-engines/mergetree-family/index.md) tables). This setting is applied at ClickHouse server start and can’t be changed in a user session. By adjusting this setting, you manage CPU and disk load. Smaller pool size utilizes less CPU and disk resources, but background processes advance slower which might eventually impact query performance. +Sets the number of threads performing background operations in table engines (for example, merges in [MergeTree engine](../../engines/table-engines/mergetree-family/index.md) tables). This setting is applied from `default` profile at ClickHouse server start and can’t be changed in a user session. By adjusting this setting, you manage CPU and disk load. Smaller pool size utilizes less CPU and disk resources, but background processes advance slower which might eventually impact query performance. + +Before changing it, please also take a look at related [MergeTree settings](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree), such as `number_of_free_entries_in_pool_to_lower_max_size_of_merge` and `number_of_free_entries_in_pool_to_execute_mutation`. Possible values: diff --git a/docs/en/operations/system-tables.md b/docs/en/operations/system-tables.md index e36101c61c5..a186e4f2cda 100644 --- a/docs/en/operations/system-tables.md +++ b/docs/en/operations/system-tables.md @@ -536,26 +536,26 @@ Contains logging entries. Logging level which goes to this table can be limited Columns: -- `event_date` (`Date`) - Date of the entry. -- `event_time` (`DateTime`) - Time of the entry. -- `microseconds` (`UInt32`) - Microseconds of the entry. +- `event_date` (Date) — Date of the entry. +- `event_time` (DateTime) — Time of the entry. +- `microseconds` (UInt32) — Microseconds of the entry. - `thread_name` (String) — Name of the thread from which the logging was done. - `thread_id` (UInt64) — OS thread ID. -- `level` (`Enum8`) - Entry level. - - `'Fatal' = 1` - - `'Critical' = 2` - - `'Error' = 3` - - `'Warning' = 4` - - `'Notice' = 5` - - `'Information' = 6` - - `'Debug' = 7` - - `'Trace' = 8` -- `query_id` (`String`) - ID of the query. -- `logger_name` (`LowCardinality(String)`) - Name of the logger (i.e. `DDLWorker`) -- `message` (`String`) - The message itself. -- `revision` (`UInt32`) - ClickHouse revision. -- `source_file` (`LowCardinality(String)`) - Source file from which the logging was done. -- `source_line` (`UInt64`) - Source line from which the logging was done. +- `level` (`Enum8`) — Entry level. Possible values: + - `1` or `'Fatal'`. + - `2` or `'Critical'`. + - `3` or `'Error'`. + - `4` or `'Warning'`. + - `5` or `'Notice'`. + - `6` or `'Information'`. + - `7` or `'Debug'`. + - `8` or `'Trace'`. +- `query_id` (String) — ID of the query. +- `logger_name` (LowCardinality(String)) — Name of the logger (i.e. `DDLWorker`). +- `message` (String) — The message itself. +- `revision` (UInt32) — ClickHouse revision. +- `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} diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 2d838c964fc..bdadf97cd11 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -58,6 +58,7 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [range\_hashed](#range-hashed) - [complex\_key\_hashed](#complex-key-hashed) - [complex\_key\_cache](#complex-key-cache) +- [complex\_key\_direct](#complex-key-direct) - [ip\_trie](#ip-trie) ### flat {#flat} @@ -317,6 +318,10 @@ or LAYOUT(DIRECT()) ``` +### complex\_key\_direct {#complex-key-direct} + +This type of storage is for use with composite [keys](external-dicts-dict-structure.md). Similar to `direct`. + ### ip\_trie {#ip-trie} This type of storage is for mapping network prefixes (IP addresses) to metadata such as ASN. diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index a36a4a4b017..5d89d6d335b 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -53,16 +53,16 @@ An exception is thrown when dividing by zero or when dividing a minimal negative Differs from ‘intDiv’ in that it returns zero when dividing by zero or when dividing a minimal negative number by minus one. -## modulo(a, b), a % b operator {#moduloa-b-a-b-operator} +## modulo(a, b), a % b operator {#modulo} Calculates the remainder after division. If arguments are floating-point numbers, they are pre-converted to integers by dropping the decimal portion. The remainder is taken in the same sense as in C++. Truncated division is used for negative numbers. An exception is thrown when dividing by zero or when dividing a minimal negative number by minus one. -## moduloOrZero(a, b) {#moduloorzeroa-b} +## moduloOrZero(a, b) {#modulo-or-zero} -Differs from ‘modulo’ in that it returns zero when the divisor is zero. +Differs from [modulo](#modulo) in that it returns zero when the divisor is zero. ## negate(a), -a operator {#negatea-a-operator} diff --git a/docs/en/sql-reference/statements/alter.md b/docs/en/sql-reference/statements/alter.md index 6b1adcdb033..44485555bda 100644 --- a/docs/en/sql-reference/statements/alter.md +++ b/docs/en/sql-reference/statements/alter.md @@ -201,17 +201,17 @@ All changes on replicated tables are broadcasting to ZooKeeper so will be applie The following operations with [partitions](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) are available: -- [DETACH PARTITION](#alter_detach-partition) – Moves a partition to the `detached` directory and forget it. -- [DROP PARTITION](#alter_drop-partition) – Deletes a partition. -- [ATTACH PART\|PARTITION](#alter_attach-partition) – Adds a part or partition from the `detached` directory to the table. -- [ATTACH PARTITION FROM](#alter_attach-partition-from) – Copies the data partition from one table to another and adds. -- [REPLACE PARTITION](#alter_replace-partition) - Copies the data partition from one table to another and replaces. -- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition)(#alter_move_to_table-partition) - Move the data partition from one table to another. -- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) - Resets the value of a specified column in a partition. -- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - Resets the specified secondary index in a partition. -- [FREEZE PARTITION](#alter_freeze-partition) – Creates a backup of a partition. -- [FETCH PARTITION](#alter_fetch-partition) – Downloads a partition from another server. -- [MOVE PARTITION\|PART](#alter_move-partition) – Move partition/data part to another disk or volume. +- [DETACH PARTITION](#alter_detach-partition) — Moves a partition to the `detached` directory and forget it. +- [DROP PARTITION](#alter_drop-partition) — Deletes a partition. +- [ATTACH PART\|PARTITION](#alter_attach-partition) — Adds a part or partition from the `detached` directory to the table. +- [ATTACH PARTITION FROM](#alter_attach-partition-from) — Copies the data partition from one table to another and adds. +- [REPLACE PARTITION](#alter_replace-partition) — Copies the data partition from one table to another and replaces. +- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition) — Moves the data partition from one table to another. +- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) — Resets the value of a specified column in a partition. +- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) — Resets the specified secondary index in a partition. +- [FREEZE PARTITION](#alter_freeze-partition) — Creates a backup of a partition. +- [FETCH PARTITION](#alter_fetch-partition) — Downloads a partition from another server. +- [MOVE PARTITION\|PART](#alter_move-partition) — Move partition/data part to another disk or volume. @@ -307,13 +307,13 @@ For the query to run successfully, the following conditions must be met: ALTER TABLE table_source MOVE PARTITION partition_expr TO TABLE table_dest ``` -This query move the data partition from the `table_source` to `table_dest` with deleting the data from `table_source`. +This query moves the data partition from the `table_source` to `table_dest` with deleting the data from `table_source`. For the query to run successfully, the following conditions must be met: - Both tables must have the same structure. - Both tables must have the same partition key. -- Both tables must be the same engine family. (replicated or non-replicated) +- Both tables must be the same engine family (replicated or non-replicated). - Both tables must have the same storage policy. #### CLEAR COLUMN IN PARTITION {#alter_clear-column-partition} diff --git a/docs/en/whats-new/roadmap.md b/docs/en/whats-new/roadmap.md index e50d663fcb0..c2ebc5260e6 100644 --- a/docs/en/whats-new/roadmap.md +++ b/docs/en/whats-new/roadmap.md @@ -5,13 +5,12 @@ toc_title: Roadmap # Roadmap {#roadmap} -## Q1 2020 {#q1-2020} - -- Role-based access control - ## Q2 2020 {#q2-2020} - Integration with external authentication services + +## Q3 2020 {#q3-2020} + - Resource pools for more precise distribution of cluster capacity between users {## [Original article](https://clickhouse.tech/docs/en/roadmap/) ##} diff --git a/docs/es/interfaces/third-party/integrations.md b/docs/es/interfaces/third-party/integrations.md index 5e9e8f841bf..716e774871b 100644 --- a/docs/es/interfaces/third-party/integrations.md +++ b/docs/es/interfaces/third-party/integrations.md @@ -25,6 +25,7 @@ toc_title: Integrations - Message queues - [Kafka](https://kafka.apache.org) - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (uses [Go client](https://github.com/ClickHouse/clickhouse-go/)) + - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - Stream processing - [Flink](https://flink.apache.org) - [flink-clickhouse-sink](https://github.com/ivi-ru/flink-clickhouse-sink) diff --git a/docs/es/operations/server-configuration-parameters/settings.md b/docs/es/operations/server-configuration-parameters/settings.md index 7ca3b628bbc..29f726ee635 100644 --- a/docs/es/operations/server-configuration-parameters/settings.md +++ b/docs/es/operations/server-configuration-parameters/settings.md @@ -209,7 +209,7 @@ Si `http_port` se especifica, la configuración de OpenSSL se ignora incluso si **Ejemplo** ``` xml -0000 +9999 ``` ## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} diff --git a/docs/fa/interfaces/third-party/integrations.md b/docs/fa/interfaces/third-party/integrations.md index 2e3bb52fa3d..657432c7958 100644 --- a/docs/fa/interfaces/third-party/integrations.md +++ b/docs/fa/interfaces/third-party/integrations.md @@ -27,6 +27,7 @@ toc_title: "\u06CC\u06A9\u067E\u0627\u0631\u0686\u06AF\u06CC" - صف پیام - [کافکا](https://kafka.apache.org) - [در حال بارگذاری](https://github.com/housepower/clickhouse_sinker) (استفاده [برو کارگیر](https://github.com/ClickHouse/clickhouse-go/)) + - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - پردازش جریان - [لرزش](https://flink.apache.org) - [سینک فلینک-کلیک](https://github.com/ivi-ru/flink-clickhouse-sink) diff --git a/docs/fa/operations/server-configuration-parameters/settings.md b/docs/fa/operations/server-configuration-parameters/settings.md index 673f24c1494..1459c20dd5c 100644 --- a/docs/fa/operations/server-configuration-parameters/settings.md +++ b/docs/fa/operations/server-configuration-parameters/settings.md @@ -210,7 +210,7 @@ toc_title: "\u062A\u0646\u0638\u06CC\u0645\u0627\u062A \u06A9\u0627\u0631\u06AF\ **مثال** ``` xml -0000 +9999 ``` ## نقلقولهای جدید از این نویسنده {#server_configuration_parameters-http_server_default_response} diff --git a/docs/fr/interfaces/third-party/integrations.md b/docs/fr/interfaces/third-party/integrations.md index 6edb04ce518..f252fd6229b 100644 --- a/docs/fr/interfaces/third-party/integrations.md +++ b/docs/fr/interfaces/third-party/integrations.md @@ -27,6 +27,7 @@ toc_title: "Int\xE9gration" - Files d'attente de messages - [Kafka](https://kafka.apache.org) - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (utiliser [Allez client](https://github.com/ClickHouse/clickhouse-go/)) + - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - Traitement de flux - [Flink](https://flink.apache.org) - [flink-clickhouse-évier](https://github.com/ivi-ru/flink-clickhouse-sink) diff --git a/docs/fr/operations/server-configuration-parameters/settings.md b/docs/fr/operations/server-configuration-parameters/settings.md index 741bec44421..45be3c5c009 100644 --- a/docs/fr/operations/server-configuration-parameters/settings.md +++ b/docs/fr/operations/server-configuration-parameters/settings.md @@ -209,7 +209,7 @@ Si `http_port` est spécifié, la configuration OpenSSL est ignorée même si el **Exemple** ``` xml -0000 +9999 ``` ## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} diff --git a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 20c944deef4..559d8adaa6f 100644 --- a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -60,6 +60,7 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [range\_hashed](#range-hashed) - [complex\_key\_hashed](#complex-key-hashed) - [complex\_key\_cache](#complex-key-cache) +- [complex\_key\_direct](#complex-key-direct) - [ip\_trie](#ip-trie) ### plat {#flat} @@ -319,6 +320,11 @@ ou LAYOUT(DIRECT()) ``` +### complex\_key\_cache {#complex-key-cache} + +Ce type de stockage est pour une utilisation avec composite [touches](external-dicts-dict-structure.md). Semblable à `direct`. + + ### ip\_trie {#ip-trie} Ce type de stockage permet de mapper des préfixes de réseau (adresses IP) à des métadonnées telles que ASN. diff --git a/docs/ja/interfaces/third-party/integrations.md b/docs/ja/interfaces/third-party/integrations.md index 59876cd63a0..3e38d578093 100644 --- a/docs/ja/interfaces/third-party/integrations.md +++ b/docs/ja/interfaces/third-party/integrations.md @@ -27,6 +27,7 @@ toc_title: "\u7D71\u5408" - メッセージキュ - [カフカ](https://kafka.apache.org) - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (用途 [Goクライアント](https://github.com/ClickHouse/clickhouse-go/)) + - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - ストリーム処理 - [フリンク](https://flink.apache.org) - [フリンク-クリックハウス-シンク](https://github.com/ivi-ru/flink-clickhouse-sink) diff --git a/docs/ja/operations/server-configuration-parameters/settings.md b/docs/ja/operations/server-configuration-parameters/settings.md index 2b460e8aca6..98a31fa5f60 100644 --- a/docs/ja/operations/server-configuration-parameters/settings.md +++ b/docs/ja/operations/server-configuration-parameters/settings.md @@ -209,7 +209,7 @@ HTTP経由でサーバーに接続するためのポート。 **例** ``` xml -0000 +9999 ``` ## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index fb36b9c87ec..04712328844 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -38,7 +38,7 @@ sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 ``` -Для использования наиболее свежих версий нужно заменить `stable` на `testing` (рекомендуется для тестовых окружений). +Для использования наиболее свежих версий нужно заменить `stable` на `testing` (рекомендуется для тестовых окружений). Также иногда доступен `prestable`. Для, собственно, установки пакетов необходимо выполнить следующие команды: diff --git a/docs/ru/interfaces/third-party/integrations.md b/docs/ru/interfaces/third-party/integrations.md index bc4b9f22950..39449b54df8 100644 --- a/docs/ru/interfaces/third-party/integrations.md +++ b/docs/ru/interfaces/third-party/integrations.md @@ -20,6 +20,7 @@ - Очереди сообщений - [Kafka](https://kafka.apache.org) - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (использует [Go client](https://github.com/ClickHouse/clickhouse-go/)) + - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - Потоковая обработка - [Flink](https://flink.apache.org) - [flink-clickhouse-sink](https://github.com/ivi-ru/flink-clickhouse-sink) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 2e0067f99b0..c664580c659 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -195,7 +195,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat **Пример** ``` xml -0000 +9999 ``` ## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} @@ -686,7 +686,7 @@ TCP порт для защищённого обмена данными с кли 9004 ``` -## tmp\_path {#tmp-path} +## tmp_path {#tmp-path} Путь ко временным данным для обработки больших запросов. @@ -698,6 +698,17 @@ TCP порт для защищённого обмена данными с кли ``` xml /var/lib/clickhouse/tmp/ ``` +## tmp_policy {#tmp-policy} + +Политика из [storage_configuration](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) для хранения временных файлов. + +Если политика не задана, используется [tmp_path](#tmp-path). В противном случае `tmp_path` игнорируется. + +!!! note "Примечание" + - `move_factor` игнорируется. + - `keep_free_space_bytes` игнорируется. + - `max_data_part_size_bytes` игнорируется. + - В данной политике у вас должен быть ровно один том. ## uncompressed\_cache\_size {#server-settings-uncompressed_cache_size} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 16d840fe4b1..56c3042bfa3 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1025,6 +1025,29 @@ ClickHouse генерирует исключение Значение по умолчанию: 0. +## optimize_skip_unused_shards {#optimize-skip-unused-shards} + +Включает или отключает пропуск неиспользуемых шардов для запросов [SELECT](../../sql-reference/statements/select/index.md) , в которых условие ключа шардирования задано в секции `WHERE/PREWHERE`. Предполагается, что данные распределены с помощью ключа шардирования, в противном случае настройка ничего не делает. + +Возможные значения: + +- 0 — Выключена. +- 1 — Включена. + +Значение по умолчанию: 0 + +## force_optimize_skip_unused_shards {#force-optimize-skip-unused-shards} + +Разрешает или запрещает выполнение запроса, если настройка [optimize_skip_unused_shards](#optimize-skip-unused-shards) включена, а пропуск неиспользуемых шардов невозможен. Если данная настройка включена и пропуск невозможен, ClickHouse генерирует исключение. + +Возможные значения: + +- 0 — Выключена. ClickHouse не генерирует исключение. +- 1 — Включена. Выполнение запроса запрещается, только если у таблицы есть ключ шардирования. +- 2 — Включена. Выполнение запроса запрещается, даже если для таблицы не определен ключ шардирования. + +Значение по умолчанию: 0 + ## optimize\_throw\_if\_noop {#setting-optimize_throw_if_noop} Включает или отключает генерирование исключения в в случаях, когда запрос [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) не выполняет мёрж. diff --git a/docs/ru/operations/system-tables.md b/docs/ru/operations/system-tables.md index bc4a4c2751f..ae0e67a4515 100644 --- a/docs/ru/operations/system-tables.md +++ b/docs/ru/operations/system-tables.md @@ -517,6 +517,33 @@ CurrentMetric_ReplicatedChecks: 0 - `query` (String) – текст запроса. Для запросов `INSERT` не содержит встаявляемые данные. - `query_id` (String) – идентификатор запроса, если был задан. +## system.text\_log {#system-tables-text-log} + +Содержит записи логов. Уровень логирования для таблицы может быть ограничен параметром сервера `text_log.level`. + +Столбцы: + +- `event_date` (Date) — Дата создания записи. +- `event_time` (DateTime) — Время создания записи. +- `microseconds` (UInt32) — Время создания записи в микросекундах. +- `thread_name` (String) — Название потока, из которого была сделана запись. +- `thread_id` (UInt64) — Идентификатор потока ОС. +- `level` (Enum8) — Уровень логирования записи. Возможные значения: + - `1` или `'Fatal'`. + - `2` или `'Critical'`. + - `3` или `'Error'`. + - `4` или `'Warning'`. + - `5` или `'Notice'`. + - `6` или `'Information'`. + - `7` или `'Debug'`. + - `8` или `'Trace'`. +- `query_id` (String) — Идентификатор запроса. +- `logger_name` (LowCardinality(String)) — Название логгера (`DDLWorker`). +- `message` (String) — Само тело записи. +- `revision` (UInt32) — Ревизия ClickHouse. +- `source_file` (LowCardinality(String)) — Исходный файл, из которого была сделана запись. +- `source_line` (UInt64) — Исходная строка, из которой была сделана запись. + ## system.query\_log {#system_tables-query_log} Содержит информацию о выполнении запросов. Для каждого запроса вы можете увидеть время начала обработки, продолжительность обработки, сообщения об ошибках и другую информацию. diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index aedc701482d..9256fab5e0c 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -53,6 +53,7 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [range\_hashed](#range-hashed) - [complex\_key\_hashed](#complex-key-hashed) - [complex\_key\_cache](#complex-key-cache) +- [complex\_key\_direct](#complex-key-direct) - [ip\_trie](#ip-trie) ### flat {#flat} @@ -315,6 +316,10 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) LAYOUT(DIRECT()) ``` +### complex\_key\_direct {#complex-key-direct} + +Тип размещения предназначен для использования с составными [ключами](external-dicts-dict-structure.md). Аналогичен `direct`. + ### ip\_trie {#ip-trie} Тип размещения предназначен для сопоставления префиксов сети (IP адресов) с метаданными, такими как ASN. diff --git a/docs/ru/sql-reference/functions/arithmetic-functions.md b/docs/ru/sql-reference/functions/arithmetic-functions.md index 85c597143fd..8513737f025 100644 --- a/docs/ru/sql-reference/functions/arithmetic-functions.md +++ b/docs/ru/sql-reference/functions/arithmetic-functions.md @@ -48,13 +48,17 @@ SELECT toTypeName(0), toTypeName(0 + 0), toTypeName(0 + 0 + 0), toTypeName(0 + 0 Отличается от intDiv тем, что при делении на ноль или при делении минимального отрицательного числа на минус единицу, возвращается ноль. -## modulo(a, b), оператор a % b {#moduloa-b-operator-a-b} +## modulo(a, b), оператор a % b {#modulo} Вычисляет остаток от деления. Если аргументы - числа с плавающей запятой, то они предварительно преобразуются в целые числа, путём отбрасывания дробной части. Берётся остаток в том же смысле, как это делается в C++. По факту, для отрицательных чисел, используется truncated division. При делении на ноль или при делении минимального отрицательного числа на минус единицу, кидается исключение. +## moduloOrZero(a, b) {#modulo-or-zero} + +В отличие от [modulo](#modulo), возвращает ноль при делении на ноль. + ## negate(a), оператор -a {#negatea-operator-a} Вычисляет число, обратное по знаку. Результат всегда имеет знаковый тип. diff --git a/docs/ru/sql-reference/functions/ext-dict-functions.md b/docs/ru/sql-reference/functions/ext-dict-functions.md index f8d2ebb2b98..a260ec1e16e 100644 --- a/docs/ru/sql-reference/functions/ext-dict-functions.md +++ b/docs/ru/sql-reference/functions/ext-dict-functions.md @@ -1,6 +1,6 @@ # Функции для работы с внешними словарями {#ext_dict_functions} -Информацию о подключении и настройке внешних словарей смотрите в разделе [Внешние словари](../../sql-reference/functions/ext-dict-functions.md). +Информацию о подключении и настройке внешних словарей смотрите в разделе [Внешние словари](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). ## dictGet {#dictget} diff --git a/docs/ru/sql-reference/statements/alter.md b/docs/ru/sql-reference/statements/alter.md index c4afb6981c4..1cf061c174f 100644 --- a/docs/ru/sql-reference/statements/alter.md +++ b/docs/ru/sql-reference/statements/alter.md @@ -204,17 +204,17 @@ ALTER TABLE [db].name DROP CONSTRAINT constraint_name; Для работы с [партициями](../../sql-reference/statements/alter.md) доступны следующие операции: -- [DETACH PARTITION](#alter_detach-partition) – перенести партицию в директорию `detached`; -- [DROP PARTITION](#alter_drop-partition) – удалить партицию; -- [ATTACH PARTITION\|PART](#alter_attach-partition) – добавить партицию/кусок в таблицу из директории `detached`; -- [ATTACH PARTITION FROM](#alter_attach-partition-from) – скопировать партицию из другой таблицы; -- [REPLACE PARTITION](#alter_replace-partition) – скопировать партицию из другой таблицы с заменой; -- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition) (\#alter\_move\_to\_table-partition) - переместить партицию в другую таблицу; -- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) – удалить все значения в столбце для заданной партиции; -- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - очистить построенные вторичные индексы для заданной партиции; -- [FREEZE PARTITION](#alter_freeze-partition) – создать резервную копию партиции; -- [FETCH PARTITION](#alter_fetch-partition) – скачать партицию с другого сервера; -- [MOVE PARTITION\|PART](#alter_move-partition) – переместить партицию/кускок на другой диск или том. +- [DETACH PARTITION](#alter_detach-partition) — перенести партицию в директорию `detached`; +- [DROP PARTITION](#alter_drop-partition) — удалить партицию; +- [ATTACH PARTITION\|PART](#alter_attach-partition) — добавить партицию/кусок в таблицу из директории `detached`; +- [ATTACH PARTITION FROM](#alter_attach-partition-from) — скопировать партицию из другой таблицы; +- [REPLACE PARTITION](#alter_replace-partition) — скопировать партицию из другой таблицы с заменой; +- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition) — переместить партицию в другую таблицу; +- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) — удалить все значения в столбце для заданной партиции; +- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) — очистить построенные вторичные индексы для заданной партиции; +- [FREEZE PARTITION](#alter_freeze-partition) — создать резервную копию партиции; +- [FETCH PARTITION](#alter_fetch-partition) — скачать партицию с другого сервера; +- [MOVE PARTITION\|PART](#alter_move-partition) — переместить партицию/кускок на другой диск или том. #### DETACH PARTITION {#alter_detach-partition} @@ -312,12 +312,14 @@ ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1 ALTER TABLE table_source MOVE PARTITION partition_expr TO TABLE table_dest ``` -Перемещает партицию из таблицы `table_source` в таблицу `table_dest` (добавляет к существующим данным в `table_dest`), с удалением данных из таблицы `table_source`. +Перемещает партицию из таблицы `table_source` в таблицу `table_dest` (добавляет к существующим данным в `table_dest`) с удалением данных из таблицы `table_source`. Следует иметь в виду: - Таблицы должны иметь одинаковую структуру. - Для таблиц должен быть задан одинаковый ключ партиционирования. +- Движки таблиц должны быть одинакового семейства (реплицированные или нереплицированные). +- Для таблиц должна быть задана одинаковая политика хранения. #### CLEAR COLUMN IN PARTITION {#alter_clear-column-partition} diff --git a/docs/tools/mdx_clickhouse.py b/docs/tools/mdx_clickhouse.py index 6e41c7d07bb..393658be2d7 100755 --- a/docs/tools/mdx_clickhouse.py +++ b/docs/tools/mdx_clickhouse.py @@ -119,6 +119,11 @@ class PatchedMacrosPlugin(macros.plugin.MacrosPlugin): def on_page_markdown(self, markdown, page, config, files): markdown = super(PatchedMacrosPlugin, self).on_page_markdown(markdown, page, config, files) + + if os.path.islink(page.file.abs_src_path): + lang = config.data['theme']['language'] + page.canonical_url = page.canonical_url.replace(f'/{lang}/', '/en/', 1) + if config.data['extra'].get('version_prefix') or config.data['extra'].get('single_page'): return markdown if self.skip_git_log: diff --git a/docs/tools/release.sh b/docs/tools/release.sh index 8c61d16966f..ff3d2971e6e 100755 --- a/docs/tools/release.sh +++ b/docs/tools/release.sh @@ -44,7 +44,7 @@ then if [[ ! -z "${CLOUDFLARE_TOKEN}" ]] then sleep 1m - git diff --stat="9999,9999" --diff-filter=M HEAD~1 | grep '|' | awk '$1 ~ /\.html$/ { if ($3>4) { url="https://clickhouse.tech/"$1; sub(/\/index.html/, "/", url); print "\""url"\""; }}' | split -l 25 /dev/stdin PURGE + git diff --stat="9999,9999" --diff-filter=M HEAD~1 | grep '|' | awk '$1 ~ /\.html$/ { if ($3>8) { url="https://content.clickhouse.tech/"$1; sub(/\/index.html/, "/", url); print "\""url"\""; }}' | split -l 25 /dev/stdin PURGE for FILENAME in $(ls PURGE*) do POST_DATA=$(cat "${FILENAME}" | sed -n -e 'H;${x;s/\n/,/g;s/^,//;p;}' | awk '{print "{\"files\":["$0"]}";}') diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 246c49738e6..79ee2e96f82 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -1,7 +1,7 @@ Babel==2.8.0 backports-abc==0.5 backports.functools-lru-cache==1.6.1 -beautifulsoup4==4.9.0 +beautifulsoup4==4.9.1 certifi==2020.4.5.1 chardet==3.0.4 click==7.1.2 @@ -21,7 +21,7 @@ mkdocs-htmlproofer-plugin==0.0.3 mkdocs-macros-plugin==0.4.7 nltk==3.5 nose==1.3.7 -protobuf==3.11.3 +protobuf==3.12.0 numpy==1.18.4 Pygments==2.5.2 pymdown-extensions==7.1 @@ -31,7 +31,7 @@ repackage==0.7.3 requests==2.23.0 singledispatch==3.4.0.3 six==1.14.0 -soupsieve==2.0 +soupsieve==2.0.1 termcolor==1.1.0 tornado==5.1.1 Unidecode==1.1.1 diff --git a/docs/tools/website.py b/docs/tools/website.py index 050a31c3803..ed950bd06e3 100644 --- a/docs/tools/website.py +++ b/docs/tools/website.py @@ -1,9 +1,11 @@ +import concurrent.futures import hashlib import json import logging import os import shutil import subprocess +import sys import bs4 import closure @@ -20,25 +22,31 @@ def adjust_markdown_html(content): content, features='html.parser' ) + for a in soup.find_all('a'): + a_class = a.attrs.get('class') + if a_class and 'headerlink' in a_class: + a.string = '\xa0' for details in soup.find_all('details'): for summary in details.find_all('summary'): if summary.parent != details: summary.extract() details.insert(0, summary) for div in soup.find_all('div'): - div.attrs['role'] = 'alert' div_class = div.attrs.get('class') - for a in div.find_all('a'): - a_class = a.attrs.get('class') - if a_class: - a.attrs['class'] = a_class + ['alert-link'] - else: - a.attrs['class'] = 'alert-link' + is_admonition = div_class and 'admonition' in div.attrs.get('class') + if is_admonition: + for a in div.find_all('a'): + a_class = a.attrs.get('class') + if a_class: + a.attrs['class'] = a_class + ['alert-link'] + else: + a.attrs['class'] = 'alert-link' for p in div.find_all('p'): p_class = p.attrs.get('class') - if p_class and ('admonition-title' in p_class): - p.attrs['class'] = p_class + ['alert-heading', 'display-5', 'mb-2'] - if div_class and 'admonition' in div.attrs.get('class'): + if is_admonition and p_class and ('admonition-title' in p_class): + p.attrs['class'] = p_class + ['alert-heading', 'display-6', 'mb-2'] + if is_admonition: + div.attrs['role'] = 'alert' if ('info' in div_class) or ('note' in div_class): mode = 'alert-primary' elif ('attention' in div_class) or ('warning' in div_class): @@ -49,7 +57,7 @@ def adjust_markdown_html(content): mode = 'alert-info' else: mode = 'alert-secondary' - div.attrs['class'] = div_class + ['alert', 'lead', 'pb-0', 'mb-4', mode] + div.attrs['class'] = div_class + ['alert', 'pb-0', 'mb-4', mode] return str(soup) @@ -138,6 +146,7 @@ def get_js_in(args): f"'{args.website_dir}/js/jquery.js'", f"'{args.website_dir}/js/popper.js'", f"'{args.website_dir}/js/bootstrap.js'", + f"'{args.website_dir}/js/sentry.js'", f"'{args.website_dir}/js/base.js'", f"'{args.website_dir}/js/index.js'", f"'{args.website_dir}/js/docsearch.js'", @@ -145,6 +154,28 @@ def get_js_in(args): ] +def minify_file(path, css_digest, js_digest): + if not ( + path.endswith('.html') or + path.endswith('.css') + ): + return + + logging.info('Minifying %s', path) + with open(path, 'rb') as f: + content = f.read().decode('utf-8') + if path.endswith('.html'): + content = minify_html(content) + content = content.replace('base.css?css_digest', f'base.css?{css_digest}') + content = content.replace('base.js?js_digest', f'base.js?{js_digest}') + elif path.endswith('.css'): + content = cssmin.cssmin(content) + elif path.endswith('.js'): + content = jsmin.jsmin(content) + with open(path, 'wb') as f: + f.write(content.encode('utf-8')) + + def minify_website(args): css_in = ' '.join(get_css_in(args)) css_out = f'{args.output_dir}/css/base.css' @@ -190,28 +221,17 @@ def minify_website(args): if args.minify: logging.info('Minifying website') - for root, _, filenames in os.walk(args.output_dir): - for filename in filenames: - path = os.path.join(root, filename) - if not ( - filename.endswith('.html') or - filename.endswith('.css') - ): - continue - - logging.info('Minifying %s', path) - with open(path, 'rb') as f: - content = f.read().decode('utf-8') - if filename.endswith('.html'): - content = minify_html(content) - content = content.replace('base.css?css_digest', f'base.css?{css_digest}') - content = content.replace('base.js?js_digest', f'base.js?{js_digest}') - elif filename.endswith('.css'): - content = cssmin.cssmin(content) - elif filename.endswith('.js'): - content = jsmin.jsmin(content) - with open(path, 'wb') as f: - f.write(content.encode('utf-8')) + with concurrent.futures.ThreadPoolExecutor() as executor: + futures = [] + for root, _, filenames in os.walk(args.output_dir): + for filename in filenames: + path = os.path.join(root, filename) + futures.append(executor.submit(minify_file, path, css_digest, js_digest)) + for future in futures: + exc = future.exception() + if exc: + logging.error(exc) + sys.exit(1) def process_benchmark_results(args): diff --git a/docs/tr/interfaces/third-party/integrations.md b/docs/tr/interfaces/third-party/integrations.md index 91f0151fa8a..8a1d5c239f6 100644 --- a/docs/tr/interfaces/third-party/integrations.md +++ b/docs/tr/interfaces/third-party/integrations.md @@ -27,6 +27,7 @@ toc_title: Entegrasyonlar - Mesaj kuyrukları - [Kafka](https://kafka.apache.org) - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (kullanma [Go client](https://github.com/ClickHouse/clickhouse-go/)) + - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - Akış işleme - [Flink](https://flink.apache.org) - [flink-clickhouse-lavabo](https://github.com/ivi-ru/flink-clickhouse-sink) diff --git a/docs/tr/operations/server-configuration-parameters/settings.md b/docs/tr/operations/server-configuration-parameters/settings.md index a944261de8d..cc5ef3e8e21 100644 --- a/docs/tr/operations/server-configuration-parameters/settings.md +++ b/docs/tr/operations/server-configuration-parameters/settings.md @@ -209,7 +209,7 @@ Eğer `http_port` belirtilmişse, OpenSSL yapılandırması ayarlanmış olsa bi **Örnek** ``` xml -0000 +9999 ``` ## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} diff --git a/docs/zh/engines/index.md b/docs/zh/engines/index.md index d9bb984abf2..c1176e2a77d 100644 --- a/docs/zh/engines/index.md +++ b/docs/zh/engines/index.md @@ -1,7 +1,5 @@ --- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "\u53D1\u52A8\u673A" +toc_folder_title: "\u5f15\u64ce" toc_priority: 25 --- diff --git a/docs/zh/interfaces/third-party/integrations.md b/docs/zh/interfaces/third-party/integrations.md index 0d551829991..014fdc88304 100644 --- a/docs/zh/interfaces/third-party/integrations.md +++ b/docs/zh/interfaces/third-party/integrations.md @@ -19,6 +19,7 @@ - 消息队列 - [卡夫卡](https://kafka.apache.org) - [clickhouse\_sinker](https://github.com/housepower/clickhouse_sinker) (使用 [去客户](https://github.com/ClickHouse/clickhouse-go/)) + - [stream-loader-clickhouse](https://github.com/adform/stream-loader) - 流处理 - [Flink](https://flink.apache.org) - [flink-clickhouse-sink](https://github.com/ivi-ru/flink-clickhouse-sink) diff --git a/docs/zh/operations/server-configuration-parameters/settings.md b/docs/zh/operations/server-configuration-parameters/settings.md index 23db483217a..2c9d611b6a7 100644 --- a/docs/zh/operations/server-configuration-parameters/settings.md +++ b/docs/zh/operations/server-configuration-parameters/settings.md @@ -207,7 +207,7 @@ ClickHouse每x秒重新加载内置字典。 这使得编辑字典 “on the fly **示例** ``` xml -0000 +9999 ``` ## http\_server\_default\_response {#server_configuration_parameters-http_server_default_response} diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 0c18301df0c..53947283faf 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -398,6 +398,10 @@ private: ignore_error = config().getBool("ignore-error", false); } + ClientInfo & client_info = context.getClientInfo(); + client_info.setInitialQuery(); + client_info.quota_key = config().getString("quota_key", ""); + connect(); /// Initialize DateLUT here to avoid counting time spent here as query execution time. @@ -606,9 +610,7 @@ private: server_version = toString(server_version_major) + "." + toString(server_version_minor) + "." + toString(server_version_patch); - if ( - server_display_name = connection->getServerDisplayName(connection_parameters.timeouts); - server_display_name.length() == 0) + if (server_display_name = connection->getServerDisplayName(connection_parameters.timeouts); server_display_name.empty()) { server_display_name = config().getString("host", "localhost"); } @@ -914,7 +916,7 @@ private: query_id, QueryProcessingStage::Complete, &context.getSettingsRef(), - nullptr, + &context.getClientInfo(), true); sendExternalTables(); @@ -946,7 +948,15 @@ private: if (!parsed_insert_query.data && (is_interactive || (!stdin_is_a_tty && std_in.eof()))) throw Exception("No data to insert", ErrorCodes::NO_DATA_TO_INSERT); - connection->sendQuery(connection_parameters.timeouts, query_without_data, query_id, QueryProcessingStage::Complete, &context.getSettingsRef(), nullptr, true); + connection->sendQuery( + connection_parameters.timeouts, + query_without_data, + query_id, + QueryProcessingStage::Complete, + &context.getSettingsRef(), + &context.getClientInfo(), + true); + sendExternalTables(); /// Receive description of table structure. @@ -1719,6 +1729,7 @@ public: */ ("password", po::value()->implicit_value("\n", ""), "password") ("ask-password", "ask-password") + ("quota_key", po::value(), "A string to differentiate quotas when the user have keyed quotas configured on server") ("query_id", po::value(), "query_id") ("query,q", po::value(), "query") ("database,d", po::value(), "database") @@ -1854,6 +1865,8 @@ public: config().setString("password", options["password"].as()); if (options.count("ask-password")) config().setBool("ask-password", true); + if (options.count("quota_key")) + config().setString("quota_key", options["quota_key"].as()); if (options.count("multiline")) config().setBool("multiline", true); if (options.count("multiquery")) diff --git a/programs/client/ConnectionParameters.cpp b/programs/client/ConnectionParameters.cpp index e1611af249d..50cac3b7800 100644 --- a/programs/client/ConnectionParameters.cpp +++ b/programs/client/ConnectionParameters.cpp @@ -29,8 +29,10 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati "port", config.getInt(is_secure ? "tcp_port_secure" : "tcp_port", is_secure ? DBMS_DEFAULT_SECURE_PORT : DBMS_DEFAULT_PORT)); default_database = config.getString("database", ""); + /// changed the default value to "default" to fix the issue when the user in the prompt is blank user = config.getString("user", "default"); + bool password_prompt = false; if (config.getBool("ask-password", false)) { @@ -52,6 +54,7 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati if (auto result = readpassphrase(prompt.c_str(), buf, sizeof(buf), 0)) password = result; } + compression = config.getBool("compression", true) ? Protocol::Compression::Enable : Protocol::Compression::Disable; timeouts = ConnectionTimeouts( diff --git a/programs/client/ConnectionParameters.h b/programs/client/ConnectionParameters.h index 834f08df9cb..a169df8390a 100644 --- a/programs/client/ConnectionParameters.h +++ b/programs/client/ConnectionParameters.h @@ -23,7 +23,6 @@ struct ConnectionParameters ConnectionTimeouts timeouts; ConnectionParameters() {} - ConnectionParameters(const Poco::Util::AbstractConfiguration & config); }; diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index d2d19a395bb..06790038a5e 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -279,7 +279,7 @@ void LocalServer::processQueries() context->makeSessionContext(); context->makeQueryContext(); - context->setUser("default", "", Poco::Net::SocketAddress{}, ""); + context->setUser("default", "", Poco::Net::SocketAddress{}); context->setCurrentQueryId(""); applyCmdSettings(); diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 3d25d645456..bceeec306cf 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -283,8 +283,10 @@ void HTTPHandler::processQuery( } std::string query_id = params.get("query_id", ""); - context.setUser(user, password, request.clientAddress(), quota_key); + context.setUser(user, password, request.clientAddress()); context.setCurrentQueryId(query_id); + if (!quota_key.empty()) + context.setQuotaKey(quota_key); /// The user could specify session identifier and session timeout. /// It allows to modify settings, create temporary tables and reuse them in subsequent requests. diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 13cb294d03f..9015c16c140 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -61,13 +61,12 @@ #include #include "MySQLHandlerFactory.h" -#if USE_OPENCL -#include "Common/BitonicSort.h" -#endif - #if !defined(ARCADIA_BUILD) -# include "config_core.h" -# include "Common/config_version.h" +# include "config_core.h" +# include "Common/config_version.h" +# if USE_OPENCL +# include "Common/BitonicSort.h" // Y_IGNORE +# endif #endif #if defined(OS_LINUX) @@ -225,8 +224,10 @@ int Server::main(const std::vector & /*args*/) registerDictionaries(); registerDisks(); +#if !defined(ARCADIA_BUILD) #if USE_OPENCL - BitonicSort::getInstance().configure(); + BitonicSort::getInstance().configure(); +#endif #endif CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::get()); @@ -379,7 +380,7 @@ int Server::main(const std::vector & /*args*/) std::string tmp_path = config().getString("tmp_path", path + "tmp/"); std::string tmp_policy = config().getString("tmp_policy", ""); const VolumePtr & volume = global_context->setTemporaryStorage(tmp_path, tmp_policy); - for (const DiskPtr & disk : volume->disks) + for (const DiskPtr & disk : volume->getDisks()) setupTmpPath(log, disk->getPath()); } diff --git a/programs/server/TCPHandler.cpp b/programs/server/TCPHandler.cpp index 93fa78499a1..a792af30cf2 100644 --- a/programs/server/TCPHandler.cpp +++ b/programs/server/TCPHandler.cpp @@ -735,7 +735,7 @@ void TCPHandler::receiveHello() << (!user.empty() ? ", user: " + user : "") << "."); - connection_context.setUser(user, password, socket().peerAddress(), ""); + connection_context.setUser(user, password, socket().peerAddress()); } diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 7f12e02ae3a..64cd7f27cd9 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -61,8 +61,11 @@ void Connection::connect(const ConnectionTimeouts & timeouts) if (connected) disconnect(); - LOG_TRACE(log_wrapper.get(), "Connecting. Database: " << (default_database.empty() ? "(not specified)" : default_database) << ". User: " << user - << (static_cast(secure) ? ". Secure" : "") << (static_cast(compression) ? "" : ". Uncompressed")); + LOG_TRACE(log_wrapper.get(), "Connecting. Database: " + << (default_database.empty() ? "(not specified)" : default_database) + << ". User: " << user + << (static_cast(secure) ? ". Secure" : "") + << (static_cast(compression) ? "" : ". Uncompressed")); if (static_cast(secure)) { @@ -165,12 +168,14 @@ void Connection::sendHello() || has_control_character(password)) throw Exception("Parameters 'default_database', 'user' and 'password' must not contain ASCII control characters", ErrorCodes::BAD_ARGUMENTS); + auto client_revision = ClickHouseRevision::get(); + writeVarUInt(Protocol::Client::Hello, *out); writeStringBinary((DBMS_NAME " ") + client_name, *out); writeVarUInt(DBMS_VERSION_MAJOR, *out); writeVarUInt(DBMS_VERSION_MINOR, *out); // NOTE For backward compatibility of the protocol, client cannot send its version_patch. - writeVarUInt(ClickHouseRevision::get(), *out); + writeVarUInt(client_revision, *out); writeStringBinary(default_database, *out); writeStringBinary(user, *out); writeStringBinary(password, *out); @@ -394,23 +399,10 @@ void Connection::sendQuery( /// Client info. if (server_revision >= DBMS_MIN_REVISION_WITH_CLIENT_INFO) { - ClientInfo client_info_to_send; - - if (!client_info || client_info->empty()) - { - /// No client info passed - means this query initiated by me. - client_info_to_send.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; - client_info_to_send.fillOSUserHostNameAndVersionInfo(); - client_info_to_send.client_name = (DBMS_NAME " ") + client_name; - } + if (client_info && !client_info->empty()) + client_info->write(*out, server_revision); else - { - /// This query is initiated by another query. - client_info_to_send = *client_info; - client_info_to_send.query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - } - - client_info_to_send.write(*out, server_revision); + ClientInfo().write(*out, server_revision); } /// Per query settings. diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index 1ecb432c827..be52234b904 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -15,8 +15,8 @@ namespace DB * * void thread() * { - * auto connection = pool.get(); - * connection->sendQuery("SELECT 'Hello, world!' AS world"); + * auto connection = pool.get(); + * connection->sendQuery(...); * } */ diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index b0ff2104ab1..282aaabd119 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -94,7 +94,7 @@ void MultiplexedConnections::sendQuery( const String & query, const String & query_id, UInt64 stage, - const ClientInfo * client_info, + const ClientInfo & client_info, bool with_pending_data) { std::lock_guard lock(cancel_mutex); @@ -126,14 +126,14 @@ void MultiplexedConnections::sendQuery( { modified_settings.parallel_replica_offset = i; replica_states[i].connection->sendQuery(timeouts, query, query_id, - stage, &modified_settings, client_info, with_pending_data); + stage, &modified_settings, &client_info, with_pending_data); } } else { /// Use single replica. - replica_states[0].connection->sendQuery(timeouts, query, query_id, stage, - &modified_settings, client_info, with_pending_data); + replica_states[0].connection->sendQuery(timeouts, query, query_id, + stage, &modified_settings, &client_info, with_pending_data); } sent_query = true; diff --git a/src/Client/MultiplexedConnections.h b/src/Client/MultiplexedConnections.h index 9d825adb227..eaec7f744bc 100644 --- a/src/Client/MultiplexedConnections.h +++ b/src/Client/MultiplexedConnections.h @@ -36,10 +36,10 @@ public: void sendQuery( const ConnectionTimeouts & timeouts, const String & query, - const String & query_id = "", - UInt64 stage = QueryProcessingStage::Complete, - const ClientInfo * client_info = nullptr, - bool with_pending_data = false); + const String & query_id, + UInt64 stage, + const ClientInfo & client_info, + bool with_pending_data); /// Get packet from any replica. Packet receivePacket(); diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 9b128fcffec..74f1438de14 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -21,7 +21,7 @@ #if !defined(ARCADIA_BUILD) # include # if USE_OPENCL -# include "Common/BitonicSort.h" +# include "Common/BitonicSort.h" // Y_IGNORE # endif #else #undef USE_OPENCL @@ -38,6 +38,7 @@ namespace ErrorCodes { extern const int PARAMETER_OUT_OF_BOUND; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int OPENCL_ERROR; extern const int LOGICAL_ERROR; } @@ -120,6 +121,30 @@ namespace }; } +template +void ColumnVector::getSpecialPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, + IColumn::SpecialSort special_sort) const +{ + if (special_sort == IColumn::SpecialSort::OPENCL_BITONIC) + { +#if !defined(ARCADIA_BUILD) +#if USE_OPENCL + if (!limit || limit >= data.size()) + { + res.resize(data.size()); + + if (data.empty() || BitonicSort::getInstance().sort(data, res, !reverse)) + return; + } +#else + throw DB::Exception("'special_sort = bitonic' specified but OpenCL not available", DB::ErrorCodes::OPENCL_ERROR); +#endif +#endif + } + + getPermutation(reverse, limit, nan_direction_hint, res); +} + template void ColumnVector::getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const { @@ -144,12 +169,6 @@ void ColumnVector::getPermutation(bool reverse, size_t limit, int nan_directi } else { -#if USE_OPENCL - /// If bitonic sort if specified as preferred than `nan_direction_hint` equals specific value 42. - if (nan_direction_hint == 42 && BitonicSort::getInstance().sort(data, res, !reverse)) - return; -#endif - /// A case for radix sort if constexpr (is_arithmetic_v && !std::is_same_v) { diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 2fd177625cc..43b7c607f64 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -189,6 +189,8 @@ public: } void getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; + void getSpecialPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, + IColumn::SpecialSort) const override; void reserve(size_t n) override { diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 11ade9b3b84..2a38fd5365b 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -245,6 +245,17 @@ public: */ virtual void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const = 0; + enum class SpecialSort + { + NONE = 0, + OPENCL_BITONIC, + }; + + virtual void getSpecialPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, SpecialSort) const + { + getPermutation(reverse, limit, nan_direction_hint, res); + } + /** Copies each element according offsets parameter. * (i-th element should be copied offsets[i] - offsets[i - 1] times.) * It is necessary in ARRAY JOIN operation. @@ -306,8 +317,9 @@ public: static MutablePtr mutate(Ptr ptr) { - MutablePtr res = ptr->shallowMutate(); - res->forEachSubcolumn([](WrappedPtr & subcolumn) { subcolumn = IColumn::mutate(std::move(subcolumn)); }); + MutablePtr res = ptr->shallowMutate(); /// Now use_count is 2. + ptr.reset(); /// Reset use_count to 1. + res->forEachSubcolumn([](WrappedPtr & subcolumn) { subcolumn = IColumn::mutate(std::move(subcolumn).detach()); }); return res; } diff --git a/src/Common/BitonicSort.h b/src/Common/BitonicSort.h index 840ab477a45..6bf10ebe835 100644 --- a/src/Common/BitonicSort.h +++ b/src/Common/BitonicSort.h @@ -11,25 +11,32 @@ #include #endif -#include -#include -#include -#include -#include -#include - #include #include #include #include #include -#include "oclBasics.cpp" +#include "oclBasics.h" #include "bitonicSortKernels.cl" class BitonicSort { public: + using KernelType = OCL::KernelType; + + enum Types + { + KernelInt8 = 0, + KernelUInt8, + KernelInt16, + KernelUInt16, + KernelInt32, + KernelUInt32, + KernelInt64, + KernelUInt64, + KernelMax + }; static BitonicSort & getInstance() { @@ -39,40 +46,50 @@ public: /// Sorts given array in specified order. Returns `true` if given sequence was sorted, `false` otherwise. template - bool sort(const DB::PaddedPODArray & data, DB::IColumn::Permutation & res, cl_uint sort_ascending) + bool sort(const DB::PaddedPODArray & data, DB::IColumn::Permutation & res, cl_uint sort_ascending [[maybe_unused]]) const { - size_t s = data.size(); - - /// Getting the nearest power of 2. - size_t power = 1; - - if (s <= 8) power = 8; - else while (power < s) power <<= 1; - - /// Allocates more space for additional stubs to be added if needed. - std::vector pairs_content(power); - std::vector pairs_indices(power); - for (UInt32 i = 0; i < s; ++i) + if constexpr ( + std::is_same_v || + std::is_same_v || + std::is_same_v || + std::is_same_v || + std::is_same_v || + std::is_same_v || + std::is_same_v || + std::is_same_v) { - pairs_content[i] = data[i]; - pairs_indices[i] = i; - } + size_t data_size = data.size(); - bool result = sort(pairs_content.data(), pairs_indices.data(), s, power - s, sort_ascending); + /// Getting the nearest power of 2. + size_t power = 8; + while (power < data_size) + power <<= 1; - if (!result) return false; + /// Allocates more space for additional stubs to be added if needed. + std::vector pairs_content(power); + std::vector pairs_indices(power); - for (size_t i = 0, shift = 0; i < power; ++i) - { - if (pairs_indices[i] >= s) + memcpy(&pairs_content[0], &data[0], sizeof(T) * data_size); + for (UInt32 i = 0; i < data_size; ++i) + pairs_indices[i] = i; + + fillWithStubs(pairs_content.data(), pairs_indices.data(), data_size, power - data_size, sort_ascending); + sort(pairs_content.data(), pairs_indices.data(), power, sort_ascending); + + for (size_t i = 0, shift = 0; i < power; ++i) { - ++shift; - continue; + if (pairs_indices[i] >= data_size) + { + ++shift; + continue; + } + res[i - shift] = pairs_indices[i]; } - res[i - shift] = pairs_indices[i]; + + return true; } - return true; + return false; } /// Creating a configuration instance with making all OpenCl required variables @@ -84,29 +101,36 @@ public: cl_platform_id platform = OCL::getPlatformID(settings); cl_device_id device = OCL::getDeviceID(platform, settings); cl_context gpu_context = OCL::makeContext(device, settings); - cl_command_queue command_queue = OCL::makeCommandQueue(device, gpu_context, settings); + cl_command_queue command_queue = OCL::makeCommandQueue<2>(device, gpu_context, settings); cl_program program = OCL::makeProgram(bitonic_sort_kernels, gpu_context, device, settings); /// Creating kernels for each specified data type. cl_int error = 0; + kernels.resize(KernelMax); - kernels["char"] = std::shared_ptr(clCreateKernel(program, "bitonicSort_char", &error), - clReleaseKernel); - kernels["uchar"] = std::shared_ptr(clCreateKernel(program, "bitonicSort_uchar", &error), - clReleaseKernel); - kernels["short"] = std::shared_ptr(clCreateKernel(program, "bitonicSort_short", &error), - clReleaseKernel); - kernels["ushort"] = std::shared_ptr(clCreateKernel(program, "bitonicSort_ushort", &error), - clReleaseKernel); - kernels["int"] = std::shared_ptr(clCreateKernel(program, "bitonicSort_int", &error), - clReleaseKernel); - kernels["uint"] = std::shared_ptr(clCreateKernel(program, "bitonicSort_uint", &error), - clReleaseKernel); - kernels["long"] = std::shared_ptr(clCreateKernel(program, "bitonicSort_long", &error), - clReleaseKernel); - kernels["ulong"] = std::shared_ptr(clCreateKernel(program, "bitonicSort_ulong", &error), - clReleaseKernel); + kernels[KernelInt8] = std::shared_ptr(clCreateKernel(program, "bitonicSort_char", &error), clReleaseKernel); + OCL::checkError(error); + + kernels[KernelUInt8] = std::shared_ptr(clCreateKernel(program, "bitonicSort_uchar", &error), clReleaseKernel); + OCL::checkError(error); + + kernels[KernelInt16] = std::shared_ptr(clCreateKernel(program, "bitonicSort_short", &error), clReleaseKernel); + OCL::checkError(error); + + kernels[KernelUInt16] = std::shared_ptr(clCreateKernel(program, "bitonicSort_ushort", &error), clReleaseKernel); + OCL::checkError(error); + + kernels[KernelInt32] = std::shared_ptr(clCreateKernel(program, "bitonicSort_int", &error), clReleaseKernel); + OCL::checkError(error); + + kernels[KernelUInt32] = std::shared_ptr(clCreateKernel(program, "bitonicSort_uint", &error), clReleaseKernel); + OCL::checkError(error); + + kernels[KernelInt64] = std::shared_ptr(clCreateKernel(program, "bitonicSort_long", &error), clReleaseKernel); + OCL::checkError(error); + + kernels[KernelUInt64] = std::shared_ptr(clCreateKernel(program, "bitonicSort_ulong", &error), clReleaseKernel); OCL::checkError(error); configuration = std::shared_ptr(new OCL::Configuration(device, gpu_context, command_queue, program)); @@ -114,97 +138,24 @@ public: private: /// Dictionary with kernels for each type from list: uchar, char, ushort, short, uint, int, ulong and long. - std::map> kernels; + std::vector> kernels; /// Current configuration with core OpenCL instances. std::shared_ptr configuration = nullptr; - /// Returns `true` if given sequence was sorted, `false` otherwise. - template - bool sort(T * p_input, cl_uint * indices, cl_int array_size, cl_int number_of_stubs, cl_uint sort_ascending) - { - if (typeid(T).name() == typeid(cl_char).name()) - sort_char(reinterpret_cast(p_input), indices, array_size, number_of_stubs, sort_ascending); - else if (typeid(T) == typeid(cl_uchar)) - sort_uchar(reinterpret_cast(p_input), indices, array_size, number_of_stubs, sort_ascending); - else if (typeid(T) == typeid(cl_short)) - sort_short(reinterpret_cast(p_input), indices, array_size, number_of_stubs, sort_ascending); - else if (typeid(T) == typeid(cl_ushort)) - sort_ushort(reinterpret_cast(p_input), indices, array_size, number_of_stubs, sort_ascending); - else if (typeid(T) == typeid(cl_int)) - sort_int(reinterpret_cast(p_input), indices, array_size, number_of_stubs, sort_ascending); - else if (typeid(T) == typeid(cl_uint)) - sort_uint(reinterpret_cast(p_input), indices, array_size, number_of_stubs, sort_ascending); - else if (typeid(T) == typeid(cl_long)) - sort_long(reinterpret_cast(p_input), indices, array_size, number_of_stubs, sort_ascending); - else if (typeid(T) == typeid(cl_ulong)) - sort_ulong(reinterpret_cast(p_input), indices, array_size, number_of_stubs, sort_ascending); - else - return false; - - return true; - } - - /// Specific functions for each integer type. - void sort_char(cl_char * p_input, cl_uint * indices, cl_int array_size, cl_int number_of_stubs, cl_uint sort_ascending) - { - cl_char stubs_value = sort_ascending ? CHAR_MAX : CHAR_MIN; - fillWithStubs(number_of_stubs, stubs_value, p_input, indices, array_size); - sort(kernels["char"].get(), p_input, indices, array_size + number_of_stubs, sort_ascending); - } - - void sort_uchar(cl_uchar * p_input, cl_uint * indices, cl_int array_size, cl_int number_of_stubs, cl_uint sort_ascending) - { - cl_uchar stubs_value = sort_ascending ? UCHAR_MAX : 0; - fillWithStubs(number_of_stubs, stubs_value, p_input, indices, array_size); - sort(kernels["uchar"].get(), p_input, indices, array_size + number_of_stubs, sort_ascending); - } - - void sort_short(cl_short * p_input, cl_uint * indices, cl_int array_size, cl_int number_of_stubs, cl_uint sort_ascending) - { - cl_short stubs_value = sort_ascending ? SHRT_MAX : SHRT_MIN; - fillWithStubs(number_of_stubs, stubs_value, p_input, indices, array_size); - sort(kernels["short"].get(), p_input, indices, array_size + number_of_stubs, sort_ascending); - } - - void sort_ushort(cl_ushort * p_input, cl_uint * indices, cl_int array_size, cl_int number_of_stubs, cl_uint sort_ascending) - { - cl_ushort stubs_value = sort_ascending ? USHRT_MAX : 0; - fillWithStubs(number_of_stubs, stubs_value, p_input, indices, array_size); - sort(kernels["ushort"].get(), p_input, indices, array_size + number_of_stubs, sort_ascending); - } - - void sort_int(cl_int * p_input, cl_uint * indices, cl_int array_size, cl_int number_of_stubs, cl_uint sort_ascending) - { - cl_int stubs_value = sort_ascending ? INT_MAX : INT_MIN; - fillWithStubs(number_of_stubs, stubs_value, p_input, indices, array_size); - sort(kernels["int"].get(), p_input, indices, array_size + number_of_stubs, sort_ascending); - } - - void sort_uint(cl_uint * p_input, cl_uint * indices, cl_int array_size, cl_int number_of_stubs, cl_uint sort_ascending) - { - cl_uint stubs_value = sort_ascending ? UINT_MAX : 0; - fillWithStubs(number_of_stubs, stubs_value, p_input, indices, array_size); - sort(kernels["uint"].get(), p_input, indices, array_size + number_of_stubs, sort_ascending); - } - - void sort_long(cl_long * p_input, cl_uint * indices, cl_int array_size, cl_int number_of_stubs, cl_uint sort_ascending) - { - cl_long stubs_value = sort_ascending ? LONG_MAX : LONG_MIN; - fillWithStubs(number_of_stubs, stubs_value, p_input, indices, array_size); - sort(kernels["long"].get(), p_input, indices, array_size + number_of_stubs, sort_ascending); - } - - void sort_ulong(cl_ulong * p_input, cl_uint * indices, cl_int array_size, cl_int number_of_stubs, cl_uint sort_ascending) - { - cl_ulong stubs_value = sort_ascending ? ULONG_MAX : 0; - fillWithStubs(number_of_stubs, stubs_value, p_input, indices, array_size); - sort(kernels["ulong"].get(), p_input, indices, array_size + number_of_stubs, sort_ascending); - } + cl_kernel getKernel(Int8) const { return kernels[KernelInt8].get(); } + cl_kernel getKernel(UInt8) const { return kernels[KernelUInt8].get(); } + cl_kernel getKernel(Int16) const { return kernels[KernelInt16].get(); } + cl_kernel getKernel(UInt16) const { return kernels[KernelUInt16].get(); } + cl_kernel getKernel(Int32) const { return kernels[KernelInt32].get(); } + cl_kernel getKernel(UInt32) const { return kernels[KernelUInt32].get(); } + cl_kernel getKernel(Int64) const { return kernels[KernelInt64].get(); } + cl_kernel getKernel(UInt64) const { return kernels[KernelUInt64].get(); } /// Sorts p_input inplace with indices. Works only with arrays which size equals to power of two. template - void sort(cl_kernel kernel, T * p_input, cl_uint * indices, cl_int array_size, cl_uint sort_ascending) + void sort(T * p_input, cl_uint * indices, cl_int array_size, cl_uint sort_ascending) const { + cl_kernel kernel = getKernel(T(0)); cl_int error = CL_SUCCESS; cl_int num_stages = 0; @@ -246,7 +197,7 @@ private: } template - void configureKernel(cl_kernel kernel, int number_of_argument, void * source) + void configureKernel(cl_kernel kernel, int number_of_argument, void * source) const { cl_int error = clSetKernelArg(kernel, number_of_argument, sizeof(T), source); OCL::checkError(error); @@ -254,9 +205,9 @@ private: /// Fills given sequences from `arraySize` index with `numberOfStubs` values. template - void fillWithStubs(cl_int number_of_stubs, T value, T * p_input, - cl_uint * indices, cl_int array_size) + void fillWithStubs(T * p_input, cl_uint * indices, cl_int array_size, cl_int number_of_stubs, cl_uint sort_ascending) const { + T value = sort_ascending ? std::numeric_limits::max() : std::numeric_limits::min(); for (cl_int index = 0; index < number_of_stubs; ++index) { p_input[array_size + index] = value; @@ -264,7 +215,7 @@ private: } } - BitonicSort() {} - BitonicSort(BitonicSort const &); - void operator=(BitonicSort const &); + BitonicSort() = default; + BitonicSort(BitonicSort const &) = delete; + void operator = (BitonicSort const &) = delete; }; diff --git a/src/Common/COW.h b/src/Common/COW.h index 3c2171436bf..23733a8635d 100644 --- a/src/Common/COW.h +++ b/src/Common/COW.h @@ -217,6 +217,9 @@ protected: operator const immutable_ptr & () const { return value; } operator immutable_ptr & () { return value; } + /// Get internal immutable ptr. Does not change internal use counter. + immutable_ptr detach() && { return std::move(value); } + operator bool() const { return value != nullptr; } bool operator! () const { return value == nullptr; } diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 374315005de..cb4c591041c 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -495,6 +495,9 @@ namespace ErrorCodes extern const int ATOMIC_RENAME_FAIL = 521; extern const int OPENCL_ERROR = 522; extern const int UNKNOWN_ROW_POLICY = 523; + extern const int ALTER_OF_COLUMN_IS_FORBIDDEN = 524; + extern const int INCORRECT_DISK_INDEX = 525; + extern const int UNKNOWN_VOLUME_TYPE = 526; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/src/Common/WeakHash.h b/src/Common/WeakHash.h index 87ddec4deff..bfea75eddf1 100644 --- a/src/Common/WeakHash.h +++ b/src/Common/WeakHash.h @@ -17,6 +17,8 @@ public: explicit WeakHash32(size_t size) : data(size, ~UInt32(0)) {} WeakHash32(const WeakHash32 & other) { data.assign(other.data); } + void reset(size_t size) { data.assign(size, ~UInt32(0)); } + const Container & getData() const { return data; } Container & getData() { return data; } diff --git a/src/Common/XDBCBridgeHelper.h b/src/Common/XDBCBridgeHelper.h index 41aeb421394..a3c538cb065 100644 --- a/src/Common/XDBCBridgeHelper.h +++ b/src/Common/XDBCBridgeHelper.h @@ -151,16 +151,22 @@ public: LOG_TRACE(log, BridgeHelperMixin::serviceAlias() + " is not running, will try to start it"); startBridge(); bool started = false; - for (size_t counter : ext::range(1, 20)) + + uint64_t milliseconds_to_wait = 10; /// Exponential backoff + uint64_t counter = 0; + while (milliseconds_to_wait < 10000) { + ++counter; LOG_TRACE(log, "Checking " + BridgeHelperMixin::serviceAlias() + " is running, try " << counter); if (checkBridgeIsRunning()) { started = true; break; } - std::this_thread::sleep_for(std::chrono::milliseconds(10)); + std::this_thread::sleep_for(std::chrono::milliseconds(milliseconds_to_wait)); + milliseconds_to_wait *= 2; } + if (!started) throw Exception(BridgeHelperMixin::getName() + "BridgeHelper: " + BridgeHelperMixin::serviceAlias() + " is not responding", ErrorCodes::EXTERNAL_SERVER_IS_NOT_RESPONDING); diff --git a/src/Common/oclBasics.cpp b/src/Common/oclBasics.h similarity index 90% rename from src/Common/oclBasics.cpp rename to src/Common/oclBasics.h index cd90cd945c1..7c977830e82 100644 --- a/src/Common/oclBasics.cpp +++ b/src/Common/oclBasics.h @@ -1,3 +1,5 @@ +#pragma once + #include #if USE_OPENCL @@ -15,24 +17,18 @@ #include #include -#ifndef CL_VERSION_2_0 -#define CL_USE_DEPRECATED_OPENCL_1_2_APIS -#endif - - -using KernelType = std::remove_reference::type; - namespace DB { - namespace ErrorCodes - { - extern const int OPENCL_ERROR; - } +namespace ErrorCodes +{ + extern const int OPENCL_ERROR; +} } struct OCL { + using KernelType = std::remove_reference::type; /** * Structure which represents the most essential settings of common OpenCl entities. @@ -209,7 +205,7 @@ struct OCL static void checkError(cl_int error) { if (error != CL_SUCCESS) - throw DB::Exception("OpenCL error " + opencl_error_to_str(error), DB::ErrorCodes::OPENCL_ERROR); + throw DB::Exception("OpenCL error: " + opencl_error_to_str(error), DB::ErrorCodes::OPENCL_ERROR); } @@ -221,22 +217,18 @@ struct OCL cl_int error = clGetPlatformIDs(settings.number_of_platform_entries, &platform, settings.number_of_available_platforms); checkError(error); - return platform; } - static cl_device_id getDeviceID(cl_platform_id & platform, const Settings & settings) { cl_device_id device; cl_int error = clGetDeviceIDs(platform, CL_DEVICE_TYPE_GPU, settings.number_of_devices_entries, &device, settings.number_of_available_devices); OCL::checkError(error); - return device; } - static cl_context makeContext(cl_device_id & device, const Settings & settings) { cl_int error; @@ -244,32 +236,43 @@ struct OCL &device, settings.context_callback, settings.context_callback_data, &error); OCL::checkError(error); - return gpu_context; } - + template static cl_command_queue makeCommandQueue(cl_device_id & device, cl_context & context, const Settings & settings [[maybe_unused]]) { cl_int error; -#ifdef CL_USE_DEPRECATED_OPENCL_1_2_APIS - cl_command_queue command_queue = clCreateCommandQueue(context, device, settings.command_queue_properties, &error); -#else - cl_command_queue command_queue = clCreateCommandQueueWithProperties(context, device, nullptr, &error); -#endif - OCL::checkError(error); + cl_command_queue command_queue; + if constexpr (version == 1) + { +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wdeprecated-declarations" + command_queue = clCreateCommandQueue(context, device, settings.command_queue_properties, &error); +#pragma GCC diagnostic pop + } + else + { +#ifdef CL_VERSION_2_0 + command_queue = clCreateCommandQueueWithProperties(context, device, nullptr, &error); +#else + throw DB::Exception("Binary is built with OpenCL version < 2.0", DB::ErrorCodes::OPENCL_ERROR); +#endif + } + + OCL::checkError(error); return command_queue; } - static cl_program makeProgram(const char * source_code, cl_context context, cl_device_id device_id, const Settings & settings) { cl_int error = 0; size_t source_size = strlen(source_code); - cl_program program = clCreateProgramWithSource(context, settings.number_of_program_source_pointers, &source_code, &source_size, &error); + cl_program program = clCreateProgramWithSource(context, settings.number_of_program_source_pointers, + &source_code, &source_size, &error); checkError(error); error = clBuildProgram(program, settings.number_of_devices_entries, &device_id, settings.build_options, @@ -291,39 +294,30 @@ struct OCL } checkError(error); - return program; } - /// Configuring buffer for given input data template - static cl_mem createBuffer(K * p_input, cl_int array_size, cl_context context, - cl_int elements_size = sizeof(K)) + static cl_mem createBuffer(K * p_input, cl_int array_size, cl_context context, cl_int elements_size = sizeof(K)) { cl_int error = CL_SUCCESS; - cl_mem cl_input_buffer = - clCreateBuffer - ( + cl_mem cl_input_buffer = clCreateBuffer( context, CL_MEM_USE_HOST_PTR, zeroCopySizeAlignment(elements_size * array_size), p_input, - &error - ); + &error); checkError(error); - return cl_input_buffer; } - static size_t zeroCopySizeAlignment(size_t required_size) { return required_size + (~required_size + 1) % 64; } - /// Manipulating with common OpenCL variables. static void finishCommandQueue(cl_command_queue command_queue) @@ -333,10 +327,8 @@ struct OCL OCL::checkError(error); } - template - static void releaseData(T * origin, cl_int array_size, cl_mem cl_buffer, - cl_command_queue command_queue, size_t offset = 0) + static void releaseData(T * origin, cl_int array_size, cl_mem cl_buffer, cl_command_queue command_queue, size_t offset = 0) { cl_int error = CL_SUCCESS; @@ -357,7 +349,6 @@ struct OCL error = clReleaseMemObject(cl_buffer); checkError(error); } - }; #endif diff --git a/src/Common/renameat2.cpp b/src/Common/renameat2.cpp index 23f2788ac76..323b72267a6 100644 --- a/src/Common/renameat2.cpp +++ b/src/Common/renameat2.cpp @@ -4,6 +4,7 @@ #if defined(linux) || defined(__linux) || defined(__linux__) #include +#include #include #include #include @@ -51,13 +52,11 @@ static void renameat2(const std::string & old_path, const std::string & new_path { if (old_path.empty() || new_path.empty()) throw Exception("Cannot rename " + old_path + " to " + new_path + ": path is empty", ErrorCodes::LOGICAL_ERROR); - if (old_path[0] != '/' || new_path[0] != '/') - throw Exception("Cannot rename " + old_path + " to " + new_path + ": path is relative", ErrorCodes::LOGICAL_ERROR); /// int olddirfd (ignored for absolute oldpath), const char *oldpath, /// int newdirfd (ignored for absolute newpath), const char *newpath, /// unsigned int flags - if (0 == syscall(__NR_renameat2, 0, old_path.c_str(), 0, new_path.c_str(), flags)) + if (0 == syscall(__NR_renameat2, AT_FDCWD, old_path.c_str(), AT_FDCWD, new_path.c_str(), flags)) return; if (errno == EEXIST) diff --git a/src/Common/tests/CMakeLists.txt b/src/Common/tests/CMakeLists.txt index 495663bc5d8..72c47d1ef49 100644 --- a/src/Common/tests/CMakeLists.txt +++ b/src/Common/tests/CMakeLists.txt @@ -37,7 +37,7 @@ target_link_libraries (radix_sort PRIVATE clickhouse_common_io) if (USE_OPENCL) add_executable (bitonic_sort bitonic_sort.cpp) - target_link_libraries (bitonic_sort PRIVATE clickhouse_common_io ${OPENCL_LINKER_FLAGS}) + target_link_libraries (bitonic_sort PRIVATE clickhouse_common_io ${OPENCL_LINKER_FLAGS} ${OpenCL_LIBRARIES}) endif () add_executable (arena_with_free_lists arena_with_free_lists.cpp) diff --git a/src/Common/tests/bitonic_sort.cpp b/src/Common/tests/bitonic_sort.cpp index adaef94ed4c..2545662c8cb 100644 --- a/src/Common/tests/bitonic_sort.cpp +++ b/src/Common/tests/bitonic_sort.cpp @@ -1,8 +1,6 @@ #include #include -#if USE_OPENCL - #if !defined(__APPLE__) && !defined(__FreeBSD__) #include #endif @@ -16,13 +14,10 @@ #include "Common/BitonicSort.h" -using Key = cl_ulong; - - /// Generates vector of size 8 for testing. /// Vector contains max possible value, min possible value and duplicate values. template -static void generateTest(std::vector& data, Type min_value, Type max_value) +static void generateTest(std::vector & data, Type min_value, Type max_value) { int size = 10; @@ -62,8 +57,7 @@ static void check(const std::vector & indices, bool reverse = true) template -static void sortBitonicSortWithPodArrays(const std::vector& data, - std::vector & indices, bool ascending = true) +static void sortBitonicSortWithPodArrays(const std::vector & data, std::vector & indices, bool ascending = true) { DB::PaddedPODArray pod_array_data = DB::PaddedPODArray(data.size()); DB::IColumn::Permutation pod_array_indices = DB::IColumn::Permutation(data.size()); @@ -74,7 +68,6 @@ static void sortBitonicSortWithPodArrays(const std::vector& data, *(pod_array_indices.data() + index) = index; } - BitonicSort::getInstance().configure(); BitonicSort::getInstance().sort(pod_array_data, pod_array_indices, ascending); for (size_t index = 0; index < data.size(); ++index) @@ -83,7 +76,7 @@ static void sortBitonicSortWithPodArrays(const std::vector& data, template -static void testBitonicSort(std::string test_name, Type min_value, Type max_value) +static void testBitonicSort(const std::string & test_name, Type min_value, Type max_value) { std::cerr << test_name << std::endl; @@ -102,147 +95,80 @@ static void testBitonicSort(std::string test_name, Type min_value, Type max_valu static void straightforwardTests() { - testBitonicSort("Test 01: cl_char.", CHAR_MIN, CHAR_MAX); - testBitonicSort("Test 02: cl_uchar.", 0, UCHAR_MAX); - testBitonicSort("Test 03: cl_short.", SHRT_MIN, SHRT_MAX); - testBitonicSort("Test 04: cl_ushort.", 0, USHRT_MAX); - testBitonicSort("Test 05: cl_int.", INT_MIN, INT_MAX); - testBitonicSort("Test 06: cl_uint.", 0, UINT_MAX); - testBitonicSort("Test 07: cl_long.", LONG_MIN, LONG_MAX); - testBitonicSort("Test 08: cl_ulong.", 0, ULONG_MAX); + testBitonicSort("Test 01: Int8.", CHAR_MIN, CHAR_MAX); + testBitonicSort("Test 02: UInt8.", 0, UCHAR_MAX); + testBitonicSort("Test 03: Int16.", SHRT_MIN, SHRT_MAX); + testBitonicSort("Test 04: UInt16.", 0, USHRT_MAX); + testBitonicSort("Test 05: Int32.", INT_MIN, INT_MAX); + testBitonicSort("Test 06: UInt32.", 0, UINT_MAX); + testBitonicSort("Test 07: Int64.", LONG_MIN, LONG_MAX); + testBitonicSort("Test 08: UInt64.", 0, ULONG_MAX); } -static void NO_INLINE sort1(Key * data, size_t size) +template +static void bitonicSort(std::vector & data) { - std::sort(data, data + size); -} - - -static void NO_INLINE sort2(std::vector & data, std::vector & indices) -{ - BitonicSort::getInstance().configure(); + size_t size = data.size(); + std::vector indices(size); + for (size_t i = 0; i < size; ++i) + indices[i] = i; sortBitonicSortWithPodArrays(data, indices); - std::vector result(data.size()); - for (size_t index = 0; index < data.size(); ++index) - result[index] = data[indices[index]]; + std::vector result(size); + for (size_t i = 0; i < size; ++i) + result[i] = data[indices[i]]; data = std::move(result); } -int main(int argc, char ** argv) +template +static bool checkSort(const std::vector & data, size_t size) { - straightforwardTests(); + std::vector copy1(data.begin(), data.begin() + size); + std::vector copy2(data.begin(), data.begin() + size); - if (argc < 3) - { - std::cerr << "Not enough arguments were passed\n"; - return 1; - } + std::sort(copy1.data(), copy1.data() + size); + bitonicSort(copy2); - size_t n = DB::parse(argv[1]); - size_t method = DB::parse(argv[2]); + for (size_t i = 0; i < size; ++i) + if (copy1[i] != copy2[i]) + return false; - std::vector data(n); - std::vector indices(n); - - { - Stopwatch watch; - - for (auto & elem : data) - elem = static_cast(rand()); - - for (size_t i = 0; i < n; ++i) - indices[i] = i; - - watch.stop(); - double elapsed = watch.elapsedSeconds(); - std::cerr - << "Filled in " << elapsed - << " (" << n / elapsed << " elem/sec., " - << n * sizeof(Key) / elapsed / 1048576 << " MB/sec.)" - << std::endl; - } - - if (n <= 100) - { - std::cerr << std::endl; - for (const auto & elem : data) - std::cerr << elem << ' '; - std::cerr << std::endl; - for (const auto & index : indices) - std::cerr << index << ' '; - std::cerr << std::endl; - } - - { - Stopwatch watch; - - if (method == 1) sort1(data.data(), n); - if (method == 2) sort2(data, indices); - - watch.stop(); - double elapsed = watch.elapsedSeconds(); - std::cerr - << "Sorted in " << elapsed - << " (" << n / elapsed << " elem/sec., " - << n * sizeof(Key) / elapsed / 1048576 << " MB/sec.)" - << std::endl; - } - - { - Stopwatch watch; - - size_t i = 1; - while (i < n) - { - if (!(data[i - 1] <= data[i])) - break; - ++i; - } - - watch.stop(); - double elapsed = watch.elapsedSeconds(); - std::cerr - << "Checked in " << elapsed - << " (" << n / elapsed << " elem/sec., " - << n * sizeof(Key) / elapsed / 1048576 << " MB/sec.)" - << std::endl - << "Result: " << (i == n ? "Ok." : "Fail!") << std::endl; - } - - if (n <= 1000) - { - std::cerr << std::endl; - - std::cerr << data[0] << ' '; - for (size_t i = 1; i < n; ++i) - { - if (!(data[i - 1] <= data[i])) - std::cerr << "*** "; - std::cerr << data[i] << ' '; - } - - std::cerr << std::endl; - - for (const auto & index : indices) - std::cerr << index << ' '; - std::cerr << std::endl; - } - - return 0; + return true; } -#else int main() { - std::cerr << "Openc CL disabled."; + BitonicSort::getInstance().configure(); + + straightforwardTests(); + + size_t size = 1100; + std::vector data(size); + for (size_t i = 0; i < size; ++i) + data[i] = rand(); + + for (size_t i = 0; i < 128; ++i) + { + if (!checkSort(data, i)) + { + std::cerr << "fail at length " << i << std::endl; + return 1; + } + } + + for (size_t i = 128; i < size; i += 7) + { + if (!checkSort(data, i)) + { + std::cerr << "fail at length " << i << std::endl; + return 1; + } + } return 0; } - -#endif diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 20dd25b6b2d..b548b573a90 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -959,7 +959,7 @@ public: if (auth_response->empty()) { - context.setUser(user_name, "", address, ""); + context.setUser(user_name, "", address); return; } @@ -982,7 +982,7 @@ public: { password_sha1[i] = digest[i] ^ static_cast((*auth_response)[i]); } - context.setUser(user_name, password_sha1, address, ""); + context.setUser(user_name, password_sha1, address); } private: String scramble; @@ -1124,7 +1124,7 @@ public: password.pop_back(); } - context.setUser(user_name, password, address, ""); + context.setUser(user_name, password, address); } private: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d0e86df45c4..28b068339ce 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -56,6 +56,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, min_insert_block_size_bytes_for_materialized_views, 0, "Like min_insert_block_size_bytes, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_bytes)", 0) \ M(SettingUInt64, max_joined_block_size_rows, DEFAULT_BLOCK_SIZE, "Maximum block size for JOIN result (if join algorithm supports it). 0 means unlimited.", 0) \ M(SettingUInt64, max_insert_threads, 0, "The maximum number of threads to execute the INSERT SELECT query. Values 0 or 1 means that INSERT SELECT is not run in parallel. Higher values will lead to higher memory usage. Parallel INSERT SELECT has effect only if the SELECT part is run on parallel, see 'max_threads' setting.", 0) \ + M(SettingUInt64, max_final_threads, 16, "The maximum number of threads to read from table with FINAL.", 0) \ M(SettingMaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.", 0) \ M(SettingMaxThreads, max_alter_threads, 0, "The maximum number of threads to execute the ALTER requests. By default, it is determined automatically.", 0) \ M(SettingUInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.", 0) \ @@ -436,7 +437,6 @@ struct Settings : public SettingsCollection 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) \ - DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) /** Set multiple settings from "profile" (in server configuration file (users.xml), profiles contain groups of multiple settings). diff --git a/src/Core/config_core.h.in b/src/Core/config_core.h.in index 054ee9a80b7..620c23c21cc 100644 --- a/src/Core/config_core.h.in +++ b/src/Core/config_core.h.in @@ -8,3 +8,4 @@ #cmakedefine01 USE_EMBEDDED_COMPILER #cmakedefine01 USE_INTERNAL_LLVM_LIBRARY #cmakedefine01 USE_SSL +#cmakedefine01 USE_OPENCL diff --git a/src/DataStreams/RemoteBlockInputStream.cpp b/src/DataStreams/RemoteBlockInputStream.cpp index fc1578d1749..e9b8d26f975 100644 --- a/src/DataStreams/RemoteBlockInputStream.cpp +++ b/src/DataStreams/RemoteBlockInputStream.cpp @@ -347,7 +347,10 @@ void RemoteBlockInputStream::sendQuery() established = true; auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings); - multiplexed_connections->sendQuery(timeouts, query, query_id, stage, &context.getClientInfo(), true); + ClientInfo modified_client_info = context.getClientInfo(); + modified_client_info.query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + + multiplexed_connections->sendQuery(timeouts, query, query_id, stage, modified_client_info, true); established = false; sent_query = true; diff --git a/src/DataStreams/RemoteBlockOutputStream.cpp b/src/DataStreams/RemoteBlockOutputStream.cpp index 5200500cd00..327e0204892 100644 --- a/src/DataStreams/RemoteBlockOutputStream.cpp +++ b/src/DataStreams/RemoteBlockOutputStream.cpp @@ -21,14 +21,17 @@ namespace ErrorCodes RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const ConnectionTimeouts & timeouts, const String & query_, - const Settings * settings_, - const ClientInfo * client_info_) - : connection(connection_), query(query_), settings(settings_), client_info(client_info_) + const Settings & settings_, + const ClientInfo & client_info_) + : connection(connection_), query(query_) { - /** Send query and receive "header", that describe table structure. + ClientInfo modified_client_info = client_info_; + modified_client_info.query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + + /** Send query and receive "header", that describes table structure. * Header is needed to know, what structure is required for blocks to be passed to 'write' method. */ - connection.sendQuery(timeouts, query, "", QueryProcessingStage::Complete, settings, client_info); + connection.sendQuery(timeouts, query, "", QueryProcessingStage::Complete, &settings_, &modified_client_info); while (true) { diff --git a/src/DataStreams/RemoteBlockOutputStream.h b/src/DataStreams/RemoteBlockOutputStream.h index f7f1ab46806..40387180997 100644 --- a/src/DataStreams/RemoteBlockOutputStream.h +++ b/src/DataStreams/RemoteBlockOutputStream.h @@ -22,8 +22,8 @@ public: RemoteBlockOutputStream(Connection & connection_, const ConnectionTimeouts & timeouts, const String & query_, - const Settings * settings_ = nullptr, - const ClientInfo * client_info_ = nullptr); + const Settings & settings_, + const ClientInfo & client_info_); Block getHeader() const override { return header; } @@ -38,8 +38,6 @@ public: private: Connection & connection; String query; - const Settings * settings; - const ClientInfo * client_info; Block header; bool finished = false; }; diff --git a/src/DataTypes/DataTypeString.cpp b/src/DataTypes/DataTypeString.cpp index efaf844a845..a7bfb2b635d 100644 --- a/src/DataTypes/DataTypeString.cpp +++ b/src/DataTypes/DataTypeString.cpp @@ -376,15 +376,19 @@ void registerDataTypeString(DataTypeFactory & factory) /// These synonyms are added for compatibility. factory.registerAlias("CHAR", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("CHARACTER", "String", DataTypeFactory::CaseInsensitive); factory.registerAlias("VARCHAR", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("VARCHAR2", "String", DataTypeFactory::CaseInsensitive); /// Oracle factory.registerAlias("TEXT", "String", DataTypeFactory::CaseInsensitive); factory.registerAlias("TINYTEXT", "String", DataTypeFactory::CaseInsensitive); factory.registerAlias("MEDIUMTEXT", "String", DataTypeFactory::CaseInsensitive); factory.registerAlias("LONGTEXT", "String", DataTypeFactory::CaseInsensitive); factory.registerAlias("BLOB", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("CLOB", "String", DataTypeFactory::CaseInsensitive); factory.registerAlias("TINYBLOB", "String", DataTypeFactory::CaseInsensitive); factory.registerAlias("MEDIUMBLOB", "String", DataTypeFactory::CaseInsensitive); factory.registerAlias("LONGBLOB", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("BYTEA", "String", DataTypeFactory::CaseInsensitive); /// PostgreSQL } } diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index 0d48845b4fe..dde78210f45 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -186,6 +186,8 @@ void registerDataTypeDecimal(DataTypeFactory & factory) factory.registerDataType("Decimal", create, DataTypeFactory::CaseInsensitive); factory.registerAlias("DEC", "Decimal", DataTypeFactory::CaseInsensitive); + factory.registerAlias("NUMERIC", "Decimal", DataTypeFactory::CaseInsensitive); + factory.registerAlias("FIXED", "Decimal", DataTypeFactory::CaseInsensitive); } /// Explicit template instantiations. diff --git a/src/DataTypes/DataTypesNumber.cpp b/src/DataTypes/DataTypesNumber.cpp index 5739a64d815..18b819c4aa6 100644 --- a/src/DataTypes/DataTypesNumber.cpp +++ b/src/DataTypes/DataTypesNumber.cpp @@ -23,11 +23,17 @@ void registerDataTypeNumbers(DataTypeFactory & factory) /// These synonyms are added for compatibility. factory.registerAlias("TINYINT", "Int8", DataTypeFactory::CaseInsensitive); + factory.registerAlias("BOOL", "Int8", DataTypeFactory::CaseInsensitive); + factory.registerAlias("BOOLEAN", "Int8", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INT1", "Int8", DataTypeFactory::CaseInsensitive); /// MySQL + factory.registerAlias("BYTE", "Int8", DataTypeFactory::CaseInsensitive); /// MS Access factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT", "Int32", DataTypeFactory::CaseInsensitive); factory.registerAlias("INTEGER", "Int32", DataTypeFactory::CaseInsensitive); factory.registerAlias("BIGINT", "Int64", DataTypeFactory::CaseInsensitive); factory.registerAlias("FLOAT", "Float32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("REAL", "Float32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("SINGLE", "Float32", DataTypeFactory::CaseInsensitive); /// MS Access factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::CaseInsensitive); } diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index bafa65b10ba..2894b65274f 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -1,12 +1,14 @@ #include #include #include +#include #include #include #include #include #include #include +#include namespace DB @@ -227,7 +229,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) { - SCOPE_EXIT({ Poco::File(table_metadata_tmp_path).remove(); }); + SCOPE_EXIT({ std::error_code code; std::filesystem::remove(table_metadata_tmp_path, code); }); std::unique_lock lock{mutex}; auto actual_table_id = getTableUnlocked(table_id.table_name, lock)->getStorageID(); @@ -323,7 +325,7 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & try { String link = path_to_table_symlinks + escapeForFileName(table_name); - String data = global_context.getPath() + actual_data_path; + String data = Poco::Path(global_context.getPath()).makeAbsolute().toString() + actual_data_path; Poco::File{data}.linkTo(link, Poco::File::LINK_SYMBOLIC); } catch (...) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 352b9db2672..2c093ae90ad 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -29,7 +29,6 @@ #include #include #include -#include namespace DB diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 8c736bc0e56..7bf147dd539 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -74,7 +74,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( , load_all_query{query_builder.composeLoadAllQuery()} { /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). - context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0), {}); + context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0)); context = copyContextAndApplySettings(path_to_settings, context, config); /// Query context is needed because some code in executeQuery function may assume it exists. diff --git a/src/Dictionaries/ComplexKeyDirectDictionary.cpp b/src/Dictionaries/ComplexKeyDirectDictionary.cpp new file mode 100644 index 00000000000..81058f28ff4 --- /dev/null +++ b/src/Dictionaries/ComplexKeyDirectDictionary.cpp @@ -0,0 +1,604 @@ +#include "ComplexKeyDirectDictionary.h" +#include +#include "DictionaryBlockInputStream.h" +#include "DictionaryFactory.h" +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; + extern const int BAD_ARGUMENTS; + extern const int UNSUPPORTED_METHOD; +} + + +ComplexKeyDirectDictionary::ComplexKeyDirectDictionary( + const std::string & database_, + const std::string & name_, + const DictionaryStructure & dict_struct_, + DictionarySourcePtr source_ptr_, + BlockPtr saved_block_) + : database(database_) + , name(name_) + , full_name{database_.empty() ? name_ : (database_ + "." + name_)} + , dict_struct(dict_struct_) + , source_ptr{std::move(source_ptr_)} + , saved_block{std::move(saved_block_)} +{ + if (!this->source_ptr->supportsSelectiveLoad()) + throw Exception{full_name + ": source cannot be used with ComplexKeyDirectDictionary", ErrorCodes::UNSUPPORTED_METHOD}; + + + createAttributes(); +} + +#define DECLARE(TYPE) \ + void ComplexKeyDirectDictionary::get##TYPE(const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType & out) const \ + { \ + dict_struct.validateKeyTypes(key_types); \ + const auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ +\ + const auto null_value = std::get(attribute.null_values); \ +\ + getItemsImpl( \ + attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return null_value; }); \ + } +DECLARE(UInt8) +DECLARE(UInt16) +DECLARE(UInt32) +DECLARE(UInt64) +DECLARE(UInt128) +DECLARE(Int8) +DECLARE(Int16) +DECLARE(Int32) +DECLARE(Int64) +DECLARE(Float32) +DECLARE(Float64) +DECLARE(Decimal32) +DECLARE(Decimal64) +DECLARE(Decimal128) +#undef DECLARE + +void ComplexKeyDirectDictionary::getString( + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ColumnString * out) const +{ + dict_struct.validateKeyTypes(key_types); + const auto & attribute = getAttribute(attribute_name); + checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + + const auto & null_value = std::get(attribute.null_values); + getItemsStringImpl( + attribute, + key_columns, + [&](const size_t, const String value) { const auto ref = StringRef{value}; out->insertData(ref.data, ref.size); }, + [&](const size_t) { return String(null_value.data, null_value.size); }); +} + +#define DECLARE(TYPE) \ + void ComplexKeyDirectDictionary::get##TYPE( \ + const std::string & attribute_name, \ + const Columns & key_columns, \ + const DataTypes & key_types, \ + const PaddedPODArray & def, \ + ResultArrayType & out) const \ + { \ + dict_struct.validateKeyTypes(key_types); \ + const auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ +\ + getItemsImpl( \ + attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t row) { return def[row]; }); \ + } +DECLARE(UInt8) +DECLARE(UInt16) +DECLARE(UInt32) +DECLARE(UInt64) +DECLARE(UInt128) +DECLARE(Int8) +DECLARE(Int16) +DECLARE(Int32) +DECLARE(Int64) +DECLARE(Float32) +DECLARE(Float64) +DECLARE(Decimal32) +DECLARE(Decimal64) +DECLARE(Decimal128) +#undef DECLARE + +void ComplexKeyDirectDictionary::getString( + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, const ColumnString * const def, ColumnString * const out) const +{ + dict_struct.validateKeyTypes(key_types); + + const auto & attribute = getAttribute(attribute_name); + checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + + getItemsStringImpl( + attribute, + key_columns, + [&](const size_t, const String value) { const auto ref = StringRef{value}; out->insertData(ref.data, ref.size); }, + [&](const size_t row) { const auto ref = def->getDataAt(row); return String(ref.data, ref.size); }); +} + +#define DECLARE(TYPE) \ + void ComplexKeyDirectDictionary::get##TYPE( \ + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, const TYPE def, ResultArrayType & out) const \ + { \ + dict_struct.validateKeyTypes(key_types); \ + const auto & attribute = getAttribute(attribute_name); \ + checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ +\ + getItemsImpl( \ + attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \ + } +DECLARE(UInt8) +DECLARE(UInt16) +DECLARE(UInt32) +DECLARE(UInt64) +DECLARE(UInt128) +DECLARE(Int8) +DECLARE(Int16) +DECLARE(Int32) +DECLARE(Int64) +DECLARE(Float32) +DECLARE(Float64) +DECLARE(Decimal32) +DECLARE(Decimal64) +DECLARE(Decimal128) +#undef DECLARE + +void ComplexKeyDirectDictionary::getString( + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, const String & def, ColumnString * const out) const +{ + dict_struct.validateKeyTypes(key_types); + + const auto & attribute = getAttribute(attribute_name); + checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + + ComplexKeyDirectDictionary::getItemsStringImpl( + attribute, + key_columns, + [&](const size_t, const String value) { const auto ref = StringRef{value}; out->insertData(ref.data, ref.size); }, + [&](const size_t) { return def; }); +} + + +void ComplexKeyDirectDictionary::has(const Columns & key_columns, const DataTypes & key_types, PaddedPODArray & out) const +{ + dict_struct.validateKeyTypes(key_types); + const auto & attribute = attributes.front(); + + switch (attribute.type) + { + case AttributeUnderlyingType::utUInt8: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utUInt16: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utUInt32: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utUInt64: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utUInt128: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utInt8: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utInt16: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utInt32: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utInt64: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utFloat32: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utFloat64: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utString: + has(attribute, key_columns, out); + break; + + case AttributeUnderlyingType::utDecimal32: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utDecimal64: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::utDecimal128: + has(attribute, key_columns, out); + break; + } +} + + +void ComplexKeyDirectDictionary::createAttributes() +{ + const auto size = dict_struct.attributes.size(); + attributes.reserve(size); + + for (const auto & attribute : dict_struct.attributes) + { + attribute_index_by_name.emplace(attribute.name, attributes.size()); + attribute_name_by_index.emplace(attributes.size(), attribute.name); + attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value, attribute.name)); + + if (attribute.hierarchical) + throw Exception{full_name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(), + ErrorCodes::TYPE_MISMATCH}; + } +} + + +template +void ComplexKeyDirectDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) +{ + attribute.null_values = T(null_value.get>()); +} + +template <> +void ComplexKeyDirectDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) +{ + attribute.string_arena = std::make_unique(); + const String & string = null_value.get(); + const char * string_in_arena = attribute.string_arena->insert(string.data(), string.size()); + attribute.null_values.emplace(string_in_arena, string.size()); +} + + +ComplexKeyDirectDictionary::Attribute ComplexKeyDirectDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value, const std::string & attr_name) +{ + Attribute attr{type, {}, {}, attr_name}; + + switch (type) + { + case AttributeUnderlyingType::utUInt8: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utUInt16: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utUInt32: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utUInt64: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utUInt128: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utInt8: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utInt16: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utInt32: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utInt64: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utFloat32: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utFloat64: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utString: + createAttributeImpl(attr, null_value); + break; + + case AttributeUnderlyingType::utDecimal32: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utDecimal64: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::utDecimal128: + createAttributeImpl(attr, null_value); + break; + } + + return attr; +} + +template +StringRef ComplexKeyDirectDictionary::placeKeysInPool( + const size_t row, const Columns & key_columns, StringRefs & keys, const std::vector & key_attributes, Pool & pool) const +{ + const auto keys_size = key_columns.size(); + size_t sum_keys_size{}; + + for (size_t j = 0; j < keys_size; ++j) + { + keys[j] = key_columns[j]->getDataAt(row); + sum_keys_size += keys[j].size; + if (key_attributes[j].underlying_type == AttributeUnderlyingType::utString) + sum_keys_size += sizeof(size_t) + 1; + } + + auto place = pool.alloc(sum_keys_size); + + auto key_start = place; + for (size_t j = 0; j < keys_size; ++j) + { + if (key_attributes[j].underlying_type == AttributeUnderlyingType::utString) + { + auto start = key_start; + auto key_size = keys[j].size + 1; + memcpy(key_start, &key_size, sizeof(size_t)); + key_start += sizeof(size_t); + memcpy(key_start, keys[j].data, keys[j].size); + key_start += keys[j].size; + *key_start = '\0'; + ++key_start; + keys[j].data = start; + keys[j].size += sizeof(size_t) + 1; + } + else + { + memcpy(key_start, keys[j].data, keys[j].size); + keys[j].data = key_start; + key_start += keys[j].size; + } + } + + return {place, sum_keys_size}; +} + + +template +void ComplexKeyDirectDictionary::getItemsImpl( + const Attribute & attribute, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const +{ + const auto rows = key_columns.front()->size(); + const auto keys_size = dict_struct.key->size(); + StringRefs keys_array(keys_size); + MapType value_by_key; + Arena temporary_keys_pool; + std::vector to_load(rows); + PODArray keys(rows); + + for (const auto row : ext::range(0, rows)) + { + const StringRef key = placeKeysInPool(row, key_columns, keys_array, *dict_struct.key, temporary_keys_pool); + keys[row] = key; + value_by_key[key] = get_default(row); + to_load[row] = row; + } + + auto stream = source_ptr->loadKeys(key_columns, to_load); + const auto attributes_size = attributes.size(); + + stream->readPrefix(); + + while (const auto block = stream->read()) + { + const auto columns = ext::map( + ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; }); + + const auto attribute_columns = ext::map(ext::range(0, attributes_size), [&](const size_t attribute_idx) + { + return block.safeGetByPosition(keys_size + attribute_idx).column; + }); + for (const size_t attribute_idx : ext::range(0, attributes.size())) + { + const IColumn & attribute_column = *attribute_columns[attribute_idx]; + Arena pool; + + StringRefs keys_temp(keys_size); + + const auto columns_size = columns.front()->size(); + + for (const auto row_idx : ext::range(0, columns_size)) + { + const StringRef key = placeKeysInPool(row_idx, columns, keys_temp, *dict_struct.key, pool); + if (value_by_key.has(key) && attribute.name == attribute_name_by_index.at(attribute_idx)) + { + if (attribute.type == AttributeUnderlyingType::utFloat32) + { + value_by_key[key] = static_cast(attribute_column[row_idx].template get()); + } + else + { + value_by_key[key] = static_cast(attribute_column[row_idx].template get()); + } + + } + } + } + } + + stream->readSuffix(); + + for (const auto row : ext::range(0, rows)) + { + set_value(row, value_by_key[keys[row]]); + } + + query_count.fetch_add(rows, std::memory_order_relaxed); +} + +template +void ComplexKeyDirectDictionary::getItemsStringImpl( + const Attribute & attribute, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const +{ + const auto rows = key_columns.front()->size(); + const auto keys_size = dict_struct.key->size(); + StringRefs keys_array(keys_size); + MapType value_by_key; + Arena temporary_keys_pool; + std::vector to_load(rows); + PODArray keys(rows); + + for (const auto row : ext::range(0, rows)) + { + const StringRef key = placeKeysInPool(row, key_columns, keys_array, *dict_struct.key, temporary_keys_pool); + keys[row] = key; + value_by_key[key] = get_default(row); + to_load[row] = row; + } + + auto stream = source_ptr->loadKeys(key_columns, to_load); + const auto attributes_size = attributes.size(); + + stream->readPrefix(); + + while (const auto block = stream->read()) + { + const auto columns = ext::map( + ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; }); + + const auto attribute_columns = ext::map(ext::range(0, attributes_size), [&](const size_t attribute_idx) + { + return block.safeGetByPosition(keys_size + attribute_idx).column; + }); + for (const size_t attribute_idx : ext::range(0, attributes.size())) + { + const IColumn & attribute_column = *attribute_columns[attribute_idx]; + Arena pool; + + StringRefs keys_temp(keys_size); + + const auto columns_size = columns.front()->size(); + + for (const auto row_idx : ext::range(0, columns_size)) + { + const StringRef key = placeKeysInPool(row_idx, columns, keys_temp, *dict_struct.key, pool); + if (value_by_key.has(key) && attribute.name == attribute_name_by_index.at(attribute_idx)) + { + const String from_source = attribute_column[row_idx].template get(); + value_by_key[key] = from_source; + } + } + } + } + + stream->readSuffix(); + + for (const auto row : ext::range(0, rows)) + { + set_value(row, value_by_key[keys[row]]); + } + + query_count.fetch_add(rows, std::memory_order_relaxed); +} + + +const ComplexKeyDirectDictionary::Attribute & ComplexKeyDirectDictionary::getAttribute(const std::string & attribute_name) const +{ + const auto it = attribute_index_by_name.find(attribute_name); + if (it == std::end(attribute_index_by_name)) + throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; + + return attributes[it->second]; +} + + +template +void ComplexKeyDirectDictionary::has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray & out) const +{ + const auto rows = key_columns.front()->size(); + const auto keys_size = dict_struct.key->size(); + StringRefs keys_array(keys_size); + MapType has_key; + Arena temporary_keys_pool; + std::vector to_load(rows); + PODArray keys(rows); + + for (const auto row : ext::range(0, rows)) + { + const StringRef key = placeKeysInPool(row, key_columns, keys_array, *dict_struct.key, temporary_keys_pool); + keys[row] = key; + has_key[key] = 0; + to_load[row] = row; + } + + auto stream = source_ptr->loadKeys(key_columns, to_load); + + stream->readPrefix(); + + while (const auto block = stream->read()) + { + const auto columns = ext::map( + ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; }); + + for (const size_t attribute_idx : ext::range(0, attributes.size())) + { + Arena pool; + + StringRefs keys_temp(keys_size); + + const auto columns_size = columns.front()->size(); + + for (const auto row_idx : ext::range(0, columns_size)) + { + const StringRef key = placeKeysInPool(row_idx, columns, keys_temp, *dict_struct.key, pool); + if (has_key.has(key) && attribute.name == attribute_name_by_index.at(attribute_idx)) + { + has_key[key] = 1; + } + } + } + } + + stream->readSuffix(); + + for (const auto row : ext::range(0, rows)) + { + out[row] = has_key[keys[row]]; + } + + query_count.fetch_add(rows, std::memory_order_relaxed); +} + + +BlockInputStreamPtr ComplexKeyDirectDictionary::getBlockInputStream(const Names & /* column_names */, size_t /* max_block_size */) const +{ + return source_ptr->loadAll(); +} + + +void registerDictionaryComplexKeyDirect(DictionaryFactory & factory) +{ + auto create_layout = [=](const std::string & full_name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr) -> DictionaryPtr + { + if (!dict_struct.key) + throw Exception{"'key' is required for dictionary of layout 'complex_key_direct'", ErrorCodes::BAD_ARGUMENTS}; + + if (dict_struct.range_min || dict_struct.range_max) + throw Exception{full_name + + ": elements .structure.range_min and .structure.range_max should be defined only " + "for a dictionary of layout 'range_hashed'", + ErrorCodes::BAD_ARGUMENTS}; + + const String database = config.getString(config_prefix + ".database", ""); + const String name = config.getString(config_prefix + ".name"); + + if (config.has(config_prefix + ".lifetime.min") || config.has(config_prefix + ".lifetime.max")) + throw Exception{"'lifetime' parameter is redundant for the dictionary' of layout 'direct'", ErrorCodes::BAD_ARGUMENTS}; + + + return std::make_unique(database, name, dict_struct, std::move(source_ptr)); + }; + factory.registerLayout("complex_key_direct", create_layout, false); +} + + +} diff --git a/src/Dictionaries/ComplexKeyDirectDictionary.h b/src/Dictionaries/ComplexKeyDirectDictionary.h new file mode 100644 index 00000000000..e814c5dde82 --- /dev/null +++ b/src/Dictionaries/ComplexKeyDirectDictionary.h @@ -0,0 +1,225 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include "DictionaryStructure.h" +#include "IDictionary.h" +#include "IDictionarySource.h" + + +namespace DB +{ +using BlockPtr = std::shared_ptr; + +class ComplexKeyDirectDictionary final : public IDictionaryBase +{ +public: + ComplexKeyDirectDictionary( + const std::string & database_, + const std::string & name_, + const DictionaryStructure & dict_struct_, + DictionarySourcePtr source_ptr_, + BlockPtr saved_block_ = nullptr); + + const std::string & getDatabase() const override { return database; } + const std::string & getName() const override { return name; } + const std::string & getFullName() const override { return full_name; } + + std::string getTypeName() const override { return "ComplexKeyDirect"; } + + size_t getBytesAllocated() const override { return 0; } + + size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); } + + double getHitRate() const override { return 1.0; } + + size_t getElementCount() const override { return 0; } + + double getLoadFactor() const override { return 0; } + + std::string getKeyDescription() const { return key_description; } + + std::shared_ptr clone() const override + { + return std::make_shared(database, name, dict_struct, source_ptr->clone(), saved_block); + } + + const IDictionarySource * getSource() const override { return source_ptr.get(); } + + const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } + + const DictionaryStructure & getStructure() const override { return dict_struct; } + + bool isInjective(const std::string & attribute_name) const override + { + return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective; + } + + template + using ResultArrayType = std::conditional_t, DecimalPaddedPODArray, PaddedPODArray>; + +#define DECLARE(TYPE) \ + void get##TYPE(const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType & out) const; + DECLARE(UInt8) + DECLARE(UInt16) + DECLARE(UInt32) + DECLARE(UInt64) + DECLARE(UInt128) + DECLARE(Int8) + DECLARE(Int16) + DECLARE(Int32) + DECLARE(Int64) + DECLARE(Float32) + DECLARE(Float64) + DECLARE(Decimal32) + DECLARE(Decimal64) + DECLARE(Decimal128) +#undef DECLARE + + void getString( + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ColumnString * out) const; + +#define DECLARE(TYPE) \ + void get##TYPE( \ + const std::string & attribute_name, \ + const Columns & key_columns, \ + const DataTypes & key_types, \ + const PaddedPODArray & def, \ + ResultArrayType & out) const; + DECLARE(UInt8) + DECLARE(UInt16) + DECLARE(UInt32) + DECLARE(UInt64) + DECLARE(UInt128) + DECLARE(Int8) + DECLARE(Int16) + DECLARE(Int32) + DECLARE(Int64) + DECLARE(Float32) + DECLARE(Float64) + DECLARE(Decimal32) + DECLARE(Decimal64) + DECLARE(Decimal128) +#undef DECLARE + + void getString( + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, const ColumnString * const def, ColumnString * const out) const; + +#define DECLARE(TYPE) \ + void get##TYPE(const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, const TYPE def, ResultArrayType & out) const; + DECLARE(UInt8) + DECLARE(UInt16) + DECLARE(UInt32) + DECLARE(UInt64) + DECLARE(UInt128) + DECLARE(Int8) + DECLARE(Int16) + DECLARE(Int32) + DECLARE(Int64) + DECLARE(Float32) + DECLARE(Float64) + DECLARE(Decimal32) + DECLARE(Decimal64) + DECLARE(Decimal128) +#undef DECLARE + + void getString( + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, const String & def, ColumnString * const out) const; + + void has(const Columns & key_columns, const DataTypes & key_types, PaddedPODArray & out) const; + + BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + +private: + template + using MapType = HashMapWithSavedHash; + + struct Attribute final + { + AttributeUnderlyingType type; + std::variant< + UInt8, + UInt16, + UInt32, + UInt64, + UInt128, + Int8, + Int16, + Int32, + Int64, + Decimal32, + Decimal64, + Decimal128, + Float32, + Float64, + StringRef> + null_values; + std::unique_ptr string_arena; + std::string name; + }; + + void createAttributes(); + + template + void addAttributeSize(const Attribute & attribute); + + void calculateBytesAllocated(); + + template + void createAttributeImpl(Attribute & attribute, const Field & null_value); + + Attribute createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value, const std::string & name); + + template + StringRef placeKeysInPool( + const size_t row, const Columns & key_columns, StringRefs & keys, const std::vector & key_attributes, Pool & pool) const; + + template + void getItemsStringImpl( + const Attribute & attribute, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const; + + template + void getItemsImpl( + const Attribute & attribute, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const; + + template + void resize(Attribute & attribute, const Key id); + + template + void setAttributeValueImpl(Attribute & attribute, const Key id, const T & value); + + void setAttributeValue(Attribute & attribute, const Key id, const Field & value); + + const Attribute & getAttribute(const std::string & attribute_name) const; + + template + void has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray & out) const; + + const std::string database; + const std::string name; + const std::string full_name; + const DictionaryStructure dict_struct; + const DictionarySourcePtr source_ptr; + const DictionaryLifetime dict_lifetime; + + std::map attribute_index_by_name; + std::map attribute_name_by_index; + std::vector attributes; + + mutable std::atomic query_count{0}; + + BlockPtr saved_block; + const std::string key_description{dict_struct.getKeyDescription()}; +}; + +} diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index dbdcab1ed8d..61d8e21341b 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -28,6 +28,9 @@ DirectDictionary::DirectDictionary( , source_ptr{std::move(source_ptr_)} , saved_block{std::move(saved_block_)} { + if (!this->source_ptr->supportsSelectiveLoad()) + throw Exception{full_name + ": source cannot be used with DirectDictionary", ErrorCodes::UNSUPPORTED_METHOD}; + createAttributes(); } diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index eabf630ac1e..88832a5cc40 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -414,7 +414,8 @@ void checkAST(const ASTCreateQuery & query) if (query.dictionary->layout == nullptr) throw Exception("Cannot create dictionary with empty layout", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); - const auto is_direct_layout = !strcasecmp(query.dictionary->layout->layout_type.data(), "direct"); + const auto is_direct_layout = !strcasecmp(query.dictionary->layout->layout_type.data(), "direct") || + !strcasecmp(query.dictionary->layout->layout_type.data(), "complex_key_direct"); if (query.dictionary->lifetime == nullptr && !is_direct_layout) throw Exception("Cannot create dictionary with empty lifetime", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); diff --git a/src/Dictionaries/registerDictionaries.cpp b/src/Dictionaries/registerDictionaries.cpp index 299da521519..ad6adbc86fb 100644 --- a/src/Dictionaries/registerDictionaries.cpp +++ b/src/Dictionaries/registerDictionaries.cpp @@ -25,6 +25,7 @@ void registerDictionaries() registerDictionaryRangeHashed(factory); registerDictionaryComplexKeyHashed(factory); registerDictionaryComplexKeyCache(factory); + registerDictionaryComplexKeyDirect(factory); #if !defined(ARCADIA_BUILD) registerDictionaryTrie(factory); #endif diff --git a/src/Dictionaries/registerDictionaries.h b/src/Dictionaries/registerDictionaries.h index eda5ca074f4..a3a4a175d41 100644 --- a/src/Dictionaries/registerDictionaries.h +++ b/src/Dictionaries/registerDictionaries.h @@ -20,6 +20,7 @@ class DictionaryFactory; void registerDictionaryRangeHashed(DictionaryFactory & factory); void registerDictionaryComplexKeyHashed(DictionaryFactory & factory); void registerDictionaryComplexKeyCache(DictionaryFactory & factory); +void registerDictionaryComplexKeyDirect(DictionaryFactory & factory); void registerDictionaryTrie(DictionaryFactory & factory); void registerDictionaryFlat(DictionaryFactory & factory); void registerDictionaryHashed(DictionaryFactory & factory); diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index 7005f9b90f8..e47b55d5254 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -25,6 +25,7 @@ SRCS( ComplexKeyCacheDictionary_setAttributeValue.cpp ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp ComplexKeyHashedDictionary.cpp + ComplexKeyDirectDictionary.cpp DictionaryBlockInputStreamBase.cpp DictionaryFactory.cpp DictionarySourceFactory.cpp diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index a1c2641e2f3..7e089ba8b51 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -12,11 +12,13 @@ namespace DB { + namespace ErrorCodes { extern const int UNKNOWN_ELEMENT_IN_CONFIG; extern const int EXCESSIVE_ELEMENT_IN_CONFIG; extern const int PATH_ACCESS_DENIED; + extern const int INCORRECT_DISK_INDEX; } std::mutex DiskLocal::reservation_mutex; @@ -34,7 +36,9 @@ public: UInt64 getSize() const override { return size; } - DiskPtr getDisk() const override { return disk; } + DiskPtr getDisk(size_t i) const override; + + Disks getDisks() const override { return {disk}; } void update(UInt64 new_size) override; @@ -282,6 +286,15 @@ void DiskLocal::copy(const String & from_path, const std::shared_ptr & to IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } +DiskPtr DiskLocalReservation::getDisk(size_t i) const +{ + if (i != 0) + { + throw Exception("Can't use i != 0 with single disk reservation", ErrorCodes::INCORRECT_DISK_INDEX); + } + return disk; +} + void DiskLocalReservation::update(UInt64 new_size) { std::lock_guard lock(DiskLocal::reservation_mutex); diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index b464d8652d6..0ae8763eef3 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -55,7 +55,7 @@ DiskSelectorPtr DiskSelector::updateFromConfig( constexpr auto default_disk_name = "default"; std::set old_disks_minus_new_disks; - for (const auto & [disk_name, _] : result->disks) + for (const auto & [disk_name, _] : result->getDisksMap()) { old_disks_minus_new_disks.insert(disk_name); } @@ -65,10 +65,10 @@ DiskSelectorPtr DiskSelector::updateFromConfig( if (!std::all_of(disk_name.begin(), disk_name.end(), isWordCharASCII)) throw Exception("Disk name can contain only alphanumeric and '_' (" + disk_name + ")", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); - if (result->disks.count(disk_name) == 0) + if (result->getDisksMap().count(disk_name) == 0) { auto disk_config_prefix = config_prefix + "." + disk_name; - result->disks.emplace(disk_name, factory.create(disk_name, config, disk_config_prefix, context)); + result->addToDiskMap(disk_name, factory.create(disk_name, config, disk_config_prefix, context)); } else { diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index 430ba97c003..85b67f55d0a 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -29,6 +29,10 @@ public: /// Get all disks with names const auto & getDisksMap() const { return disks; } + void addToDiskMap(String name, DiskPtr disk) + { + disks.emplace(name, disk); + } private: std::map disks; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 7d0b429720e..011c75402f4 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -206,8 +206,11 @@ public: /// Get reservation size. virtual UInt64 getSize() const = 0; - /// Get disk where reservation take place. - virtual DiskPtr getDisk() const = 0; + /// Get i-th disk where reservation take place. + virtual DiskPtr getDisk(size_t i = 0) const = 0; + + /// Get all disks, used in reservation + virtual Disks getDisks() const = 0; /// Changes amount of reserved space. virtual void update(UInt64 new_size) = 0; diff --git a/src/Disks/IVolume.h b/src/Disks/IVolume.h index 504dded7696..a762958a33f 100644 --- a/src/Disks/IVolume.h +++ b/src/Disks/IVolume.h @@ -8,6 +8,17 @@ namespace DB { +enum class VolumeType +{ + JBOD, + SINGLE_DISK, + UNKNOWN +}; + +class IVolume; +using VolumePtr = std::shared_ptr; +using Volumes = std::vector; + /** * Disks group by some (user) criteria. For example, * - VolumeJBOD("slow_disks", [d1, d2], 100) @@ -22,7 +33,7 @@ namespace DB class IVolume : public Space { public: - IVolume(String name_, Disks disks_): disks(std::move(disks_)), name(std::move(name_)) + IVolume(String name_, Disks disks_): disks(std::move(disks_)), name(name_) { } @@ -37,16 +48,17 @@ public: /// Volume name from config const String & getName() const override { return name; } + virtual VolumeType getType() const = 0; /// Return biggest unreserved space across all disks UInt64 getMaxUnreservedFreeSpace() const; - Disks disks; + DiskPtr getDisk(size_t i = 0) const { return disks[i]; } + const Disks & getDisks() const { return disks; } + protected: + Disks disks; const String name; }; -using VolumePtr = std::shared_ptr; -using Volumes = std::vector; - } diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 7643516e197..1b058354545 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -28,6 +28,7 @@ namespace ErrorCodes extern const int FILE_ALREADY_EXISTS; extern const int CANNOT_SEEK_THROUGH_FILE; extern const int UNKNOWN_FORMAT; + extern const int INCORRECT_DISK_INDEX; } namespace @@ -369,7 +370,16 @@ public: UInt64 getSize() const override { return size; } - DiskPtr getDisk() const override { return disk; } + DiskPtr getDisk(size_t i) const override + { + if (i != 0) + { + throw Exception("Can't use i != 0 with single disk reservation", ErrorCodes::INCORRECT_DISK_INDEX); + } + return disk; + } + + Disks getDisks() const override { return {disk}; } void update(UInt64 new_size) override { diff --git a/src/Disks/SingleDiskVolume.cpp b/src/Disks/SingleDiskVolume.cpp new file mode 100644 index 00000000000..47140407026 --- /dev/null +++ b/src/Disks/SingleDiskVolume.cpp @@ -0,0 +1,6 @@ +#include + +namespace DB +{ + +} diff --git a/src/Disks/SingleDiskVolume.h b/src/Disks/SingleDiskVolume.h new file mode 100644 index 00000000000..c7a6776a7d2 --- /dev/null +++ b/src/Disks/SingleDiskVolume.h @@ -0,0 +1,26 @@ +#pragma once + +#include + +namespace DB +{ + +class SingleDiskVolume : public IVolume +{ +public: + SingleDiskVolume(const String & name_, DiskPtr disk): IVolume(name_, {disk}) + { + } + + ReservationPtr reserve(UInt64 bytes) override + { + return disks[0]->reserve(bytes); + } + + VolumeType getType() const override { return VolumeType::SINGLE_DISK; } + +}; + +using VolumeSingleDiskPtr = std::shared_ptr; + +} diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index 8518e1516db..e1ae718d9b9 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -55,7 +55,7 @@ StoragePolicy::StoragePolicy( std::set disk_names; for (const auto & volume : volumes) { - for (const auto & disk : volume->disks) + for (const auto & disk : volume->getDisks()) { if (disk_names.find(disk->getName()) != disk_names.end()) throw Exception( @@ -102,7 +102,7 @@ bool StoragePolicy::isDefaultPolicy() const if (volumes[0]->getName() != "default") return false; - const auto & disks = volumes[0]->disks; + const auto & disks = volumes[0]->getDisks(); if (disks.size() != 1) return false; @@ -117,7 +117,7 @@ Disks StoragePolicy::getDisks() const { Disks res; for (const auto & volume : volumes) - for (const auto & disk : volume->disks) + for (const auto & disk : volume->getDisks()) res.push_back(disk); return res; } @@ -130,17 +130,17 @@ DiskPtr StoragePolicy::getAnyDisk() const if (volumes.empty()) throw Exception("StoragePolicy has no volumes. It's a bug.", ErrorCodes::LOGICAL_ERROR); - if (volumes[0]->disks.empty()) + if (volumes[0]->getDisks().empty()) throw Exception("Volume '" + volumes[0]->getName() + "' has no disks. It's a bug.", ErrorCodes::LOGICAL_ERROR); - return volumes[0]->disks[0]; + return volumes[0]->getDisks()[0]; } DiskPtr StoragePolicy::getDiskByName(const String & disk_name) const { for (auto && volume : volumes) - for (auto && disk : volume->disks) + for (auto && disk : volume->getDisks()) if (disk->getName() == disk_name) return disk; return {}; @@ -181,7 +181,7 @@ ReservationPtr StoragePolicy::makeEmptyReservationOnLargestDisk() const DiskPtr max_disk; for (const auto & volume : volumes) { - for (const auto & disk : volume->disks) + for (const auto & disk : volume->getDisks()) { auto avail_space = disk->getAvailableSpace(); if (avail_space > max_space) @@ -207,10 +207,10 @@ void StoragePolicy::checkCompatibleWith(const StoragePolicyPtr & new_storage_pol throw Exception("New storage policy shall contain volumes of old one", ErrorCodes::LOGICAL_ERROR); std::unordered_set new_disk_names; - for (const auto & disk : new_storage_policy->getVolumeByName(volume->getName())->disks) + for (const auto & disk : new_storage_policy->getVolumeByName(volume->getName())->getDisks()) new_disk_names.insert(disk->getName()); - for (const auto & disk : volume->disks) + for (const auto & disk : volume->getDisks()) if (new_disk_names.count(disk->getName()) == 0) throw Exception("New storage policy shall contain disks of old one", ErrorCodes::LOGICAL_ERROR); } @@ -222,7 +222,7 @@ size_t StoragePolicy::getVolumeIndexByDisk(const DiskPtr & disk_ptr) const for (size_t i = 0; i < volumes.size(); ++i) { const auto & volume = volumes[i]; - for (const auto & disk : volume->disks) + for (const auto & disk : volume->getDisks()) if (disk->getName() == disk_ptr->getName()) return i; } diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index a41a62f1223..00ac2c2c9bb 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Disks/VolumeJBOD.h b/src/Disks/VolumeJBOD.h index 7399d3cf065..16a96ec9fb9 100644 --- a/src/Disks/VolumeJBOD.h +++ b/src/Disks/VolumeJBOD.h @@ -25,6 +25,8 @@ public: DiskSelectorPtr disk_selector ); + VolumeType getType() const override { return VolumeType::JBOD; } + /// Next disk (round-robin) /// /// - Used with policy for temporary data diff --git a/src/Disks/createVolume.cpp b/src/Disks/createVolume.cpp new file mode 100644 index 00000000000..111f05b8db8 --- /dev/null +++ b/src/Disks/createVolume.cpp @@ -0,0 +1,17 @@ +#include "createVolume.h" + +namespace DB +{ + +VolumePtr createVolumeFromReservation(const ReservationPtr & reservation, VolumePtr other_volume) +{ + if (other_volume->getType() == VolumeType::JBOD || other_volume->getType() == VolumeType::SINGLE_DISK) + { + /// Since reservation on JBOD chosies one of disks and makes reservation there, volume + /// for such type of reservation will be with one disk. + return std::make_shared(other_volume->getName(), reservation->getDisk()); + } + return nullptr; +} + +} diff --git a/src/Disks/createVolume.h b/src/Disks/createVolume.h new file mode 100644 index 00000000000..52085ec16bc --- /dev/null +++ b/src/Disks/createVolume.h @@ -0,0 +1,12 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +VolumePtr createVolumeFromReservation(const ReservationPtr & reservation, VolumePtr other_volume); + +} diff --git a/src/Disks/ya.make b/src/Disks/ya.make index a14024e7af3..0187bd54b48 100644 --- a/src/Disks/ya.make +++ b/src/Disks/ya.make @@ -5,6 +5,7 @@ PEERDIR( ) SRCS( + createVolume.cpp DiskFactory.cpp DiskLocal.cpp DiskMemory.cpp @@ -12,6 +13,7 @@ SRCS( IDisk.cpp IVolume.cpp registerDisks.cpp + SingleDiskVolume.cpp StoragePolicy.cpp VolumeJBOD.cpp ) diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 118c389666f..8e538bf780b 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -31,6 +31,7 @@ #include #include #include +#include #include #include #include @@ -174,6 +175,7 @@ private: !executeDispatchSimple(block, arguments, result, dict) && !executeDispatchComplex(block, arguments, result, dict) && !executeDispatchComplex(block, arguments, result, dict) && + !executeDispatchComplex(block, arguments, result, dict) && #if !defined(ARCADIA_BUILD) !executeDispatchComplex(block, arguments, result, dict) && #endif @@ -324,6 +326,7 @@ private: !executeDispatch(block, arguments, result, dict) && !executeDispatchComplex(block, arguments, result, dict) && !executeDispatchComplex(block, arguments, result, dict) && + !executeDispatchComplex(block, arguments, result, dict) && #if !defined(ARCADIA_BUILD) !executeDispatchComplex(block, arguments, result, dict) && #endif @@ -501,6 +504,7 @@ private: !executeDispatch(block, arguments, result, dict) && !executeDispatchComplex(block, arguments, result, dict) && !executeDispatchComplex(block, arguments, result, dict) && + !executeDispatchComplex(block, arguments, result, dict) && #if !defined(ARCADIA_BUILD) !executeDispatchComplex(block, arguments, result, dict) && #endif @@ -834,6 +838,7 @@ private: !executeDispatch(block, arguments, result, dict) && !executeDispatchComplex(block, arguments, result, dict) && !executeDispatchComplex(block, arguments, result, dict) && + !executeDispatchComplex(block, arguments, result, dict) && #if !defined(ARCADIA_BUILD) !executeDispatchComplex(block, arguments, result, dict) && #endif @@ -1088,6 +1093,7 @@ private: !executeDispatch(block, arguments, result, dict) && !executeDispatchComplex(block, arguments, result, dict) && !executeDispatchComplex(block, arguments, result, dict) && + !executeDispatchComplex(block, arguments, result, dict) && #if !defined(ARCADIA_BUILD) !executeDispatchComplex(block, arguments, result, dict) && #endif diff --git a/src/Functions/ignoreExceptNull.cpp b/src/Functions/ignoreExceptNull.cpp deleted file mode 100644 index ff009533e37..00000000000 --- a/src/Functions/ignoreExceptNull.cpp +++ /dev/null @@ -1,56 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - -/** ignoreExceptNull(...) is a function that takes any arguments, and always returns 0 except Null. - */ - class FunctionIgnoreExceptNull : public IFunction - { - public: - static constexpr auto name = "ignoreExceptNull"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - bool isVariadic() const override - { - return true; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - String getName() const override - { - return name; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - /// This function is mainly used in query analysis instead of "in" functions - /// in the case when only header is needed and set for in is not calculated. - /// Because of that function must return the same column type as "in" function, which is ColumnUInt8. - auto res = ColumnUInt8::create(input_rows_count, 0); - block.getByPosition(result).column = std::move(res); - } - }; - - - void registerFunctionIgnoreExceptNull(FunctionFactory & factory) - { - factory.registerFunction(); - } - -} diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index a89535c675a..81b75f67764 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -21,62 +22,34 @@ namespace ErrorCodes * notIn(x, set) - and NOT IN. */ -template +template struct FunctionInName; -template <> -struct FunctionInName -{ - static constexpr auto name = "in"; -}; +template <> struct FunctionInName { static constexpr auto name = "in"; }; +template <> struct FunctionInName { static constexpr auto name = "globalIn"; }; +template <> struct FunctionInName { static constexpr auto name = "notIn"; }; +template <> struct FunctionInName { static constexpr auto name = "globalNotIn"; }; +template <> struct FunctionInName { static constexpr auto name = "nullIn"; }; +template <> struct FunctionInName { static constexpr auto name = "globalNullIn"; }; +template <> struct FunctionInName { static constexpr auto name = "notNullIn"; }; +template <> struct FunctionInName { static constexpr auto name = "globalNotNullIn"; }; +template <> struct FunctionInName { static constexpr auto name = "inIgnoreSet"; }; +template <> struct FunctionInName { static constexpr auto name = "globalInIgnoreSet"; }; +template <> struct FunctionInName { static constexpr auto name = "notInIgnoreSet"; }; +template <> struct FunctionInName { static constexpr auto name = "globalNotInIgnoreSet"; }; +template <> struct FunctionInName { static constexpr auto name = "nullInIgnoreSet"; }; +template <> struct FunctionInName { static constexpr auto name = "globalNullInIgnoreSet"; }; +template <> struct FunctionInName { static constexpr auto name = "notNullInIgnoreSet"; }; +template <> struct FunctionInName { static constexpr auto name = "globalNotNullInIgnoreSet"; }; -template <> -struct FunctionInName -{ - static constexpr auto name = "globalIn"; -}; - -template <> -struct FunctionInName -{ - static constexpr auto name = "notIn"; -}; - -template <> -struct FunctionInName -{ - static constexpr auto name = "globalNotIn"; -}; - -template <> -struct FunctionInName -{ - static constexpr auto name = "nullIn"; -}; - -template <> -struct FunctionInName -{ - static constexpr auto name = "globalNullIn"; -}; - -template <> -struct FunctionInName -{ - static constexpr auto name = "notNullIn"; -}; - -template <> -struct FunctionInName -{ - static constexpr auto name = "globalNotNullIn"; -}; - -template +template class FunctionIn : public IFunction { public: - static constexpr auto name = FunctionInName::name; + /// ignore_set flag means that we don't use set from the second argument, just return zero column. + /// It is needed to perform type analysis without creation of set. + static constexpr auto name = FunctionInName::name; + static FunctionPtr create(const Context &) { return std::make_shared(); @@ -97,13 +70,21 @@ public: return std::make_shared(); } - bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForConstants() const override + { + /// Never return constant for -IgnoreSet functions to avoid constant folding. + return !ignore_set; + } bool useDefaultImplementationForNulls() const override { return null_is_skipped; } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, [[maybe_unused]] size_t input_rows_count) override { - /// NOTE: after updating this code, check that FunctionIgnoreExceptNull returns the same type of column. + if constexpr (ignore_set) + { + block.getByPosition(result).column = ColumnUInt8::create(input_rows_count, 0u); + return; + } /// Second argument must be ColumnSet. ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column; @@ -146,17 +127,23 @@ public: } }; +template +static void registerFunctionsInImpl(FunctionFactory & factory) +{ + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); +} void registerFunctionsIn(FunctionFactory & factory) { - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); + registerFunctionsInImpl(factory); + registerFunctionsInImpl(factory); } } diff --git a/src/Functions/randomFixedString.cpp b/src/Functions/randomFixedString.cpp new file mode 100644 index 00000000000..9fb7550346b --- /dev/null +++ b/src/Functions/randomFixedString.cpp @@ -0,0 +1,90 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int DECIMAL_OVERFLOW; +} + + +/* Generate random fixed string with fully random bytes (including zero). */ +class FunctionRandomFixedString : public IFunction +{ +public: + static constexpr auto name = "randomFixedString"; + + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return false; } + + size_t getNumberOfArguments() const override { return 1; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!isUnsignedInteger(arguments[0].type)) + throw Exception("First argument for function " + getName() + " must be unsigned integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!arguments[0].column || !isColumnConst(*arguments[0].column)) + throw Exception("First argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN); + + const size_t n = assert_cast(*arguments[0].column).getValue(); + return std::make_shared(n); + } + + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const size_t n = assert_cast(*block.getByPosition(arguments[0]).column).getValue(); + + auto col_to = ColumnFixedString::create(n); + ColumnFixedString::Chars & data_to = col_to->getChars(); + + if (input_rows_count == 0) + { + block.getByPosition(result).column = std::move(col_to); + return; + } + + size_t total_size; + if (common::mulOverflow(input_rows_count, n, total_size)) + throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + + /// Fill random bytes. + data_to.resize(total_size); + pcg64_fast rng(randomSeed()); /// TODO It is inefficient. We should use SIMD PRNG instead. + + auto * pos = data_to.data(); + auto * end = pos + data_to.size(); + while (pos < end) + { + unalignedStore(pos, rng()); + pos += sizeof(UInt64); // We have padding in column buffers that we can overwrite. + } + + block.getByPosition(result).column = std::move(col_to); + } +}; + +void registerFunctionRandomFixedString(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/randomString.cpp b/src/Functions/randomString.cpp index 2b95825d449..4ea470f0a65 100644 --- a/src/Functions/randomString.cpp +++ b/src/Functions/randomString.cpp @@ -4,8 +4,8 @@ #include #include #include -#include #include +#include namespace DB diff --git a/src/Functions/randomStringUTF8.cpp b/src/Functions/randomStringUTF8.cpp new file mode 100644 index 00000000000..51f57d63ea9 --- /dev/null +++ b/src/Functions/randomStringUTF8.cpp @@ -0,0 +1,148 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int TOO_LARGE_STRING_SIZE; +} + + +/* Generate string with a UTF-8 encoded text. + * Take a single argument - length of result string in Unicode code points. + * ATTENTION: Method generate only assignable code points (excluded 4-13 planes). + * See https://en.wikipedia.org/wiki/Plane_(Unicode) */ + +class FunctionRandomStringUTF8 : public IFunction +{ +public: + static constexpr auto name = "randomStringUTF8"; + + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return false; } + + size_t getNumberOfArguments() const override { return 1; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isNumber(*arguments[0])) + throw Exception("First argument of function " + getName() + " must have numeric type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + auto col_to = ColumnString::create(); + ColumnString::Chars & data_to = col_to->getChars(); + ColumnString::Offsets & offsets_to = col_to->getOffsets(); + + if (input_rows_count == 0) + { + block.getByPosition(result).column = std::move(col_to); + return; + } + + offsets_to.resize(input_rows_count); + + const IColumn & length_column = *block.getByPosition(arguments[0]).column; + size_t summary_utf8_len = 0; + for (size_t row_num = 0; row_num < input_rows_count; ++row_num) + { + size_t utf8_len = length_column.getUInt(row_num); + summary_utf8_len += utf8_len; + } + + /* As we generate only assigned planes, the mathematical expectation of the number of bytes + * per generated code point ~= 3.85. So, reserving for coefficient 4 will not be an overhead + */ + + if (summary_utf8_len > (1 << 29)) + throw Exception("Too large string size in function " + getName(), ErrorCodes::TOO_LARGE_STRING_SIZE); + + size_t size_in_bytes_with_margin = summary_utf8_len * 4 + input_rows_count; + data_to.resize(size_in_bytes_with_margin); + pcg64_fast rng(randomSeed()); // TODO It is inefficient. We should use SIMD PRNG instead. + + const auto generate_code_point = [](UInt32 rand) -> UInt32 { + /// We want to generate number in [0x0, 0x70000) and shift it if need + + /// Generate highest byte in [0, 6] + /// https://lemire.me/blog/2016/06/27/a-fast-alternative-to-the-modulo-reduction/ + UInt32 code_point = (rand >> 16) * 7u; + code_point &= ~0xFFFF; + code_point |= rand & 0xFFFF; // and other bytes obtaining in a simple way + + if (code_point >= 0x40000) + { + code_point += 0xa0000; // shift if it is in 14-16 plane + return code_point; + } + + if (0xD7FF < code_point && code_point < 0xE000) // this range will not be valid in isValidUTF8 + { + /// The distribution will be slightly non-uniform but we don't care. + return 0u; + } + + return code_point; + }; + + IColumn::Offset offset = 0; + for (size_t row_num = 0; row_num < input_rows_count; ++row_num) + { + size_t utf8_len = length_column.getUInt(row_num); + auto * pos = data_to.data() + offset; + + size_t last_writen_bytes = 0; + size_t i = 0; + for (; i < utf8_len; i += 2) + { + UInt64 rand = rng(); + + UInt32 code_point1 = generate_code_point(rand); + UInt32 code_point2 = generate_code_point(rand >> 32); + + /// We have padding in column buffers that we can overwrite. + pos += UTF8::convert(code_point1, pos, sizeof(int)); + last_writen_bytes = UTF8::convert(code_point2, pos, sizeof(int)); + pos += last_writen_bytes; + } + offset = pos - data_to.data() + 1; + if (i > utf8_len) + { + offset -= last_writen_bytes; + } + offsets_to[row_num] = offset; + } + + /// Put zero bytes in between. + auto * pos = data_to.data(); + for (size_t row_num = 0; row_num < input_rows_count; ++row_num) + pos[offsets_to[row_num] - 1] = 0; + + block.getByPosition(result).column = std::move(col_to); + } +}; + +void registerFunctionRandomStringUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} +} diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index c8c6d6d79e2..3e6099ee25a 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -27,7 +27,6 @@ void registerFunctionSleep(FunctionFactory &); void registerFunctionSleepEachRow(FunctionFactory &); void registerFunctionMaterialize(FunctionFactory &); void registerFunctionIgnore(FunctionFactory &); -void registerFunctionIgnoreExceptNull(FunctionFactory &); void registerFunctionIdentity(FunctionFactory &); void registerFunctionArrayJoin(FunctionFactory &); void registerFunctionReplicate(FunctionFactory &); @@ -84,7 +83,6 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionSleepEachRow(factory); registerFunctionMaterialize(factory); registerFunctionIgnore(factory); - registerFunctionIgnoreExceptNull(factory); registerFunctionIdentity(factory); registerFunctionArrayJoin(factory); registerFunctionReplicate(factory); diff --git a/src/Functions/registerFunctionsRandom.cpp b/src/Functions/registerFunctionsRandom.cpp index 5826fe78419..3638474c4fe 100644 --- a/src/Functions/registerFunctionsRandom.cpp +++ b/src/Functions/registerFunctionsRandom.cpp @@ -8,6 +8,8 @@ void registerFunctionRandConstant(FunctionFactory & factory); void registerFunctionGenerateUUIDv4(FunctionFactory & factory); void registerFunctionRandomPrintableASCII(FunctionFactory & factory); void registerFunctionRandomString(FunctionFactory & factory); +void registerFunctionRandomFixedString(FunctionFactory & factory); +void registerFunctionRandomStringUTF8(FunctionFactory & factory); void registerFunctionsRandom(FunctionFactory & factory) { @@ -17,6 +19,8 @@ void registerFunctionsRandom(FunctionFactory & factory) registerFunctionGenerateUUIDv4(factory); registerFunctionRandomPrintableASCII(factory); registerFunctionRandomString(factory); + registerFunctionRandomFixedString(factory); + registerFunctionRandomStringUTF8(factory); } } diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 67eeb8cc884..8e53ffe493d 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -205,7 +205,6 @@ SRCS( ifNull.cpp IFunction.cpp ignore.cpp - ignoreExceptNull.cpp in.cpp intDiv.cpp intDivOrZero.cpp @@ -284,6 +283,8 @@ SRCS( rand.cpp randomPrintableASCII.cpp randomString.cpp + randomStringUTF8.cpp + randomFixedString.cpp regexpQuoteMeta.cpp registerFunctionsArithmetic.cpp registerFunctionsComparison.cpp diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 4d032e552ca..81f19d0b3dc 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -381,11 +381,13 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (!data.only_consts) { /// We are in the part of the tree that we are not going to compute. You just need to define types. - /// Do not subquery and create sets. We treat "IN" as "ignoreExceptNull" function. + /// Do not subquery and create sets. We replace "in*" function to "in*IgnoreSet". + + auto argument_name = node.arguments->children.at(0)->getColumnName(); data.addAction(ExpressionAction::applyFunction( - FunctionFactory::instance().get("ignoreExceptNull", data.context), - { node.arguments->children.at(0)->getColumnName() }, + FunctionFactory::instance().get(node.name + "IgnoreSet", data.context), + { argument_name, argument_name }, column_name.get(ast))); } return; diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index 909582aa308..378375dcc18 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -116,6 +116,14 @@ void ClientInfo::read(ReadBuffer & in, const UInt64 client_protocol_revision) } +void ClientInfo::setInitialQuery() +{ + query_kind = QueryKind::INITIAL_QUERY; + fillOSUserHostNameAndVersionInfo(); + client_name = (DBMS_NAME " ") + client_name; +} + + void ClientInfo::fillOSUserHostNameAndVersionInfo() { os_user.resize(256, '\0'); diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 492cc299a06..7a4df63c17a 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -84,6 +84,10 @@ public: void write(WriteBuffer & out, const UInt64 server_protocol_revision) const; void read(ReadBuffer & in, const UInt64 client_protocol_revision); + /// Initialize parameters on client initiating query. + void setInitialQuery(); + +private: void fillOSUserHostNameAndVersionInfo(); }; diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 151dfc5c9bb..7fad9af8960 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -328,8 +328,8 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting /// In case of internal_replication we will be appending names to dir_name_for_internal_replication std::string dir_name_for_internal_replication; + std::string dir_name_for_internal_replication_with_local; - auto first = true; for (const auto & replica_key : replica_keys) { if (startsWith(replica_key, "weight") || startsWith(replica_key, "internal_replication")) @@ -340,18 +340,20 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting replica_addresses.emplace_back(config, partial_prefix + replica_key, current_shard_num, current_replica_num); ++current_replica_num; - if (!replica_addresses.back().is_local) + if (internal_replication) { - if (internal_replication) + auto dir_name = replica_addresses.back().toFullString(settings.use_compact_format_in_distributed_parts_names); + if (!replica_addresses.back().is_local) { - auto dir_name = replica_addresses.back().toFullString(settings.use_compact_format_in_distributed_parts_names); - if (first) + if (dir_name_for_internal_replication.empty()) dir_name_for_internal_replication = dir_name; else dir_name_for_internal_replication += "," + dir_name; } - - if (first) first = false; + if (dir_name_for_internal_replication_with_local.empty()) + dir_name_for_internal_replication_with_local = dir_name; + else + dir_name_for_internal_replication_with_local += "," + dir_name; } } else @@ -383,8 +385,16 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting if (weight) slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size()); - shards_info.push_back({std::move(dir_name_for_internal_replication), current_shard_num, weight, - std::move(shard_local_addresses), std::move(shard_pool), std::move(all_replicas_pools), internal_replication}); + shards_info.push_back({ + std::move(dir_name_for_internal_replication), + std::move(dir_name_for_internal_replication_with_local), + current_shard_num, + weight, + std::move(shard_local_addresses), + std::move(shard_pool), + std::move(all_replicas_pools), + internal_replication + }); } else throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); @@ -433,8 +443,16 @@ Cluster::Cluster(const Settings & settings, const std::vectortmp_volume = tmp_policy->getVolume(0); } - if (shared->tmp_volume->disks.empty()) + if (shared->tmp_volume->getDisks().empty()) throw Exception("No disks volume for temporary files", ErrorCodes::NO_ELEMENTS_IN_CONFIG); return shared->tmp_volume; @@ -648,15 +648,13 @@ ConfigurationPtr Context::getUsersConfig() } -void Context::setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key) +void Context::setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address) { auto lock = getLock(); client_info.current_user = name; client_info.current_password = password; client_info.current_address = address; - if (!quota_key.empty()) - client_info.quota_key = quota_key; auto new_user_id = getAccessControlManager().find(name); std::shared_ptr new_access; @@ -686,6 +684,12 @@ std::shared_ptr Context::getUser() const return getAccess()->getUser(); } +void Context::setQuotaKey(String quota_key_) +{ + auto lock = getLock(); + client_info.quota_key = std::move(quota_key_); +} + String Context::getUserName() const { return getAccess()->getUserName(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 31d7b1f1e13..4ba135d0b1a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -250,7 +250,8 @@ public: /// Sets the current user, checks the password and that the specified host is allowed. /// Must be called before getClientInfo. - void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key); + void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address); + void setQuotaKey(String quota_key_); UserPtr getUser() const; String getUserName() const; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6c7a17ffd77..aa124d9843f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -93,6 +93,7 @@ #include #include #include +#include #include #include #include @@ -659,16 +660,16 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, const Co return order_descr; } -static UInt64 getLimitUIntValue(const ASTPtr & node, const Context & context) +static UInt64 getLimitUIntValue(const ASTPtr & node, const Context & context, const std::string & expr) { const auto & [field, type] = evaluateConstantExpression(node, context); if (!isNativeNumber(type)) - throw Exception("Illegal type " + type->getName() + " of LIMIT expression, must be numeric type", ErrorCodes::INVALID_LIMIT_EXPRESSION); + throw Exception("Illegal type " + type->getName() + " of " + expr + " expression, must be numeric type", ErrorCodes::INVALID_LIMIT_EXPRESSION); Field converted = convertFieldToType(field, DataTypeUInt64()); if (converted.isNull()) - throw Exception("The value " + applyVisitor(FieldVisitorToString(), field) + " of LIMIT expression is not representable as UInt64", ErrorCodes::INVALID_LIMIT_EXPRESSION); + throw Exception("The value " + applyVisitor(FieldVisitorToString(), field) + " of " + expr + " expression is not representable as UInt64", ErrorCodes::INVALID_LIMIT_EXPRESSION); return converted.safeGet(); } @@ -681,11 +682,12 @@ static std::pair getLimitLengthAndOffset(const ASTSelectQuery & if (query.limitLength()) { - length = getLimitUIntValue(query.limitLength(), context); + length = getLimitUIntValue(query.limitLength(), context, "LIMIT"); if (query.limitOffset() && length) - offset = getLimitUIntValue(query.limitOffset(), context); + offset = getLimitUIntValue(query.limitOffset(), context, "OFFSET"); } - + else if (query.limitOffset()) + offset = getLimitUIntValue(query.limitOffset(), context, "OFFSET"); return {length, offset}; } @@ -693,7 +695,7 @@ static std::pair getLimitLengthAndOffset(const ASTSelectQuery & static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & context) { /// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY, neither ARRAY JOIN. - if (!query.distinct && !query.limitBy() && !query.limit_with_ties && !query.arrayJoinExpressionList()) + if (!query.distinct && !query.limitBy() && !query.limit_with_ties && !query.arrayJoinExpressionList() && query.limitLength()) { auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); return limit_length + limit_offset; @@ -1070,6 +1072,8 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (!(pipeline_with_processors && has_prelimit)) /// Limit is no longer needed if there is prelimit. executeLimit(pipeline); + + executeOffset(pipeline); } } @@ -2063,10 +2067,9 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting const Settings & settings = context->getSettingsRef(); - /// TODO: 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); + IBlockInputStream::LocalLimits limits; + limits.mode = IBlockInputStream::LIMITS_CURRENT; + limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); if (input_sorting_info) { @@ -2103,6 +2106,8 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting return std::make_shared(header, output_order_descr, limit); }); + /// NOTE limits are not applied to the size of temporary sets in FinishSortingTransform + pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr { return std::make_shared( @@ -2122,6 +2127,15 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting return std::make_shared(header, output_order_descr, limit); }); + pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr + { + if (stream_type == QueryPipeline::StreamType::Totals) + return nullptr; + + auto transform = std::make_shared(header, limits); + return transform; + }); + /// Merge the sorted blocks. pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr { @@ -2345,8 +2359,8 @@ void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline) Names columns; for (const auto & elem : query.limitBy()->children) columns.emplace_back(elem->getColumnName()); - UInt64 length = getLimitUIntValue(query.limitByLength(), *context); - UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context) : 0); + UInt64 length = getLimitUIntValue(query.limitByLength(), *context, "LIMIT"); + UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context, "OFFSET") : 0); pipeline.transform([&](auto & stream) { @@ -2364,8 +2378,8 @@ void InterpreterSelectQuery::executeLimitBy(QueryPipeline & pipeline) for (const auto & elem : query.limitBy()->children) columns.emplace_back(elem->getColumnName()); - UInt64 length = getLimitUIntValue(query.limitByLength(), *context); - UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context) : 0); + UInt64 length = getLimitUIntValue(query.limitByLength(), *context, "LIMIT"); + UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context, "OFFSET") : 0); pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr { @@ -2444,6 +2458,7 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline) } } +void InterpreterSelectQuery::executeOffset(Pipeline & /* pipeline */) {} void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline) { @@ -2539,6 +2554,26 @@ void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline) } +void InterpreterSelectQuery::executeOffset(QueryPipeline & pipeline) +{ + auto & query = getSelectQuery(); + /// If there is not a LIMIT but an offset + if (!query.limitLength() && query.limitOffset()) + { + UInt64 limit_length; + UInt64 limit_offset; + std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context); + + pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipeline::StreamType::Main) + return nullptr; + return std::make_shared(header, limit_offset, 1); + }); + } +} + + void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline) { if (!context->getSettingsRef().extremes) diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index c50f4a2f7b7..1415143dd63 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -185,6 +185,7 @@ private: 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); @@ -203,6 +204,7 @@ private: void executePreLimit(QueryPipeline & pipeline, bool do_not_skip_offset); void executeLimitBy(QueryPipeline & pipeline); void executeLimit(QueryPipeline & pipeline); + void executeOffset(QueryPipeline & pipeline); static void executeProjection(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); void executeDistinct(QueryPipeline & pipeline, bool before_order, Names columns); void executeExtremes(QueryPipeline & pipeline); diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index f5a024c7768..b2a4eec7883 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -165,9 +165,9 @@ private: // Queue is bounded. But its size is quite large to not block in all normal cases. std::vector queue; // An always-incrementing index of the first message currently in the queue. - // We use it to give a global sequential index to every message, so that we can wait - // until a particular message is flushed. This is used to implement synchronous log - // flushing for SYSTEM FLUSH LOGS. + // We use it to give a global sequential index to every message, so that we + // can wait until a particular message is flushed. This is used to implement + // synchronous log flushing for SYSTEM FLUSH LOGS. uint64_t queue_front_index = 0; bool is_shutdown = false; std::condition_variable flush_event; @@ -175,6 +175,8 @@ private: uint64_t requested_flush_before = 0; // Flushed log up to this index, exclusive uint64_t flushed_before = 0; + // Logged overflow message at this queue front index + uint64_t logged_queue_full_at_index = -1; void savingThreadFunction(); @@ -244,9 +246,22 @@ void SystemLog::add(const LogElement & element) if (queue.size() >= DBMS_SYSTEM_LOG_QUEUE_SIZE) { - // TextLog sets its logger level to 0, so this log is a noop and there - // is no recursive logging. - LOG_ERROR(log, "Queue is full for system log '" + demangle(typeid(*this).name()) + "'."); + // Ignore all further entries until the queue is flushed. + // Log a message about that. Don't spam it -- this might be especially + // problematic in case of trace log. Remember what the front index of the + // queue was when we last logged the message. If it changed, it means the + // queue was flushed, and we can log again. + if (queue_front_index != logged_queue_full_at_index) + { + logged_queue_full_at_index = queue_front_index; + + // TextLog sets its logger level to 0, so this log is a noop and + // there is no recursive logging. + LOG_ERROR(log, "Queue is full for system log '" + << demangle(typeid(*this).name()) << "'" + << " at " << queue_front_index); + } + return; } @@ -325,9 +340,16 @@ void SystemLog::savingThreadFunction() uint64_t to_flush_end = 0; { + LOG_TRACE(log, "Sleeping"); std::unique_lock lock(mutex); - flush_event.wait_for(lock, std::chrono::milliseconds(flush_interval_milliseconds), - [&] () { return requested_flush_before > flushed_before || is_shutdown; }); + const bool predicate = flush_event.wait_for(lock, + std::chrono::milliseconds(flush_interval_milliseconds), + [&] () + { + return requested_flush_before > flushed_before + || is_shutdown; + } + ); queue_front_index += queue.size(); to_flush_end = queue_front_index; @@ -337,6 +359,13 @@ void SystemLog::savingThreadFunction() queue.swap(to_flush); exit_this_thread = is_shutdown; + + LOG_TRACE(log, "Woke up" + << (predicate ? " by condition" : " by timeout (" + + toString(flush_interval_milliseconds) + " ms)") + << ", " << to_flush.size() << " elements to flush" + << " up to " << to_flush_end + << (is_shutdown ? ", shutdown requested" : "")); } if (to_flush.empty()) @@ -351,6 +380,7 @@ void SystemLog::savingThreadFunction() tryLogCurrentException(__PRETTY_FUNCTION__); } } + LOG_TRACE(log, "Terminating"); } @@ -359,11 +389,13 @@ void SystemLog::flushImpl(const std::vector & to_flush, { try { - LOG_TRACE(log, "Flushing system log"); + LOG_TRACE(log, "Flushing system log, " + << to_flush.size() << " entries to flush"); - /// We check for existence of the table and create it as needed at every flush. - /// This is done to allow user to drop the table at any moment (new empty table will be created automatically). - /// BTW, flush method is called from single thread. + /// We check for existence of the table and create it as needed at every + /// flush. This is done to allow user to drop the table at any moment + /// (new empty table will be created automatically). BTW, flush method + /// is called from single thread. prepareTable(); Block block = LogElement::createBlock(); @@ -389,9 +421,13 @@ void SystemLog::flushImpl(const std::vector & to_flush, tryLogCurrentException(__PRETTY_FUNCTION__); } - std::unique_lock lock(mutex); - flushed_before = to_flush_end; - flush_event.notify_all(); + { + std::unique_lock lock(mutex); + flushed_before = to_flush_end; + flush_event.notify_all(); + } + + LOG_TRACE(log, "Flushed system log"); } diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index 0e98dc0eb4b..ec0865c2fb5 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { extern const int BAD_COLLATION; - extern const int OPENCL_ERROR; } static bool isCollationRequired(const SortColumnDescription & description) @@ -134,20 +133,12 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) else if (!isColumnConst(*column)) { int nan_direction_hint = description[0].nulls_direction; + auto special_sort = description[0].special_sort; - /// If in Settings `special_sort` option has been set as `bitonic_sort`, - /// then via `nan_direction_hint` variable a flag which specifies bitonic sort as preferred - /// will be passed to `getPermutation` method with value 42. - if (description[0].special_sort == SpecialSort::OPENCL_BITONIC) - { -#ifdef USE_OPENCL - nan_direction_hint = 42; -#else - throw DB::Exception("Bitonic sort specified as preferred, but OpenCL not available", DB::ErrorCodes::OPENCL_ERROR); -#endif - } - - column->getPermutation(reverse, limit, nan_direction_hint, perm); + if (special_sort == SpecialSort::OPENCL_BITONIC) + column->getSpecialPermutation(reverse, limit, nan_direction_hint, perm, IColumn::SpecialSort::OPENCL_BITONIC); + else + column->getPermutation(reverse, limit, nan_direction_hint, perm); } else /// we don't need to do anything with const column diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index 6a88700e3b8..9e65543babe 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -154,6 +154,11 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F if (limit_with_ties) s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << " WITH TIES" << (s.hilite ? hilite_none : ""); } + else if (limitOffset()) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "OFFSET " << (s.hilite ? hilite_none : ""); + limitOffset()->formatImpl(s, state, frame); + } if (settings()) { diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 89579d07177..70a8b282a72 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1107,6 +1107,7 @@ const char * ParserAlias::restricted_keywords[] = "HAVING", "ORDER", "LIMIT", + "OFFSET", "SETTINGS", "FORMAT", "UNION", diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 927ac45001e..d2d7bbf9f21 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -241,6 +241,11 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (top_length && limit_length) throw Exception("Can not use TOP and LIMIT together", ErrorCodes::TOP_AND_LIMIT_TOGETHER); } + else if (s_offset.ignore(pos, expected)) + { + if (!exp_elem.parse(pos, limit_offset, expected)) + return false; + } /// Because TOP n in totally equals LIMIT n if (top_length) diff --git a/src/Processors/Chunk.cpp b/src/Processors/Chunk.cpp index 1f73c9f276a..340cd4a43d7 100644 --- a/src/Processors/Chunk.cpp +++ b/src/Processors/Chunk.cpp @@ -46,7 +46,7 @@ Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_ Chunk Chunk::clone() const { - return Chunk(getColumns(), getNumRows()); + return Chunk(getColumns(), getNumRows(), chunk_info); } void Chunk::setColumns(Columns columns_, UInt64 num_rows_) diff --git a/src/Processors/ISimpleTransform.cpp b/src/Processors/ISimpleTransform.cpp index fc8f1ba30f4..ac8f2f8b7ae 100644 --- a/src/Processors/ISimpleTransform.cpp +++ b/src/Processors/ISimpleTransform.cpp @@ -29,10 +29,10 @@ ISimpleTransform::Status ISimpleTransform::prepare() } /// Output if has data. - if (transformed) + if (has_output) { - output.pushData(std::move(current_data)); - transformed = false; + output.pushData(std::move(output_data)); + has_output = false; if (!no_more_data_needed) return Status::PortFull; @@ -56,27 +56,17 @@ ISimpleTransform::Status ISimpleTransform::prepare() return Status::Finished; } - if (!input.hasData()) - { - input.setNeeded(); - return Status::NeedData; - } + input.setNeeded(); - current_data = input.pullData(true); + if (!input.hasData()) + return Status::NeedData; + + input_data = input.pullData(set_input_not_needed_after_read); has_input = true; - if (current_data.exception) - { - /// Skip transform in case of exception. - has_input = false; - transformed = true; - + if (input_data.exception) /// No more data needed. Exception will be thrown (or swallowed) later. input.setNotNeeded(); - } - - if (set_input_not_needed_after_read) - input.setNotNeeded(); } /// Now transform. @@ -85,29 +75,35 @@ ISimpleTransform::Status ISimpleTransform::prepare() void ISimpleTransform::work() { - if (current_data.exception) + if (input_data.exception) + { + /// Skip transform in case of exception. + output_data = std::move(input_data); + has_input = false; + has_output = true; return; + } try { - transform(current_data.chunk); + transform(input_data.chunk, output_data.chunk); } catch (DB::Exception &) { - current_data.exception = std::current_exception(); - transformed = true; + output_data.exception = std::current_exception(); + has_output = true; has_input = false; return; } has_input = !needInputData(); - if (!skip_empty_chunks || current_data.chunk) - transformed = true; + if (!skip_empty_chunks || output_data.chunk) + has_output = true; - if (transformed && !current_data.chunk) + if (has_output && !output_data.chunk && getOutputPort().getHeader()) /// Support invariant that chunks must have the same number of columns as header. - current_data.chunk = Chunk(getOutputPort().getHeader().cloneEmpty().getColumns(), 0); + output_data.chunk = Chunk(getOutputPort().getHeader().cloneEmpty().getColumns(), 0); } } diff --git a/src/Processors/ISimpleTransform.h b/src/Processors/ISimpleTransform.h index 1f0c5a5b040..ee92b574d7c 100644 --- a/src/Processors/ISimpleTransform.h +++ b/src/Processors/ISimpleTransform.h @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + /** Has one input and one output. * Simply pull a block from input, transform it, and push it to output. */ @@ -15,18 +20,29 @@ protected: InputPort & input; OutputPort & output; - Port::Data current_data; + Port::Data input_data; + Port::Data output_data; bool has_input = false; - bool transformed = false; + bool has_output = false; bool no_more_data_needed = false; const bool skip_empty_chunks; /// Set input port NotNeeded after chunk was pulled. /// Input port will become needed again only after data was transformed. /// This allows to escape caching chunks in input port, which can lead to uneven data distribution. - bool set_input_not_needed_after_read = false; + bool set_input_not_needed_after_read = true; + + virtual void transform(Chunk &) + { + throw Exception("Method transform is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + + virtual void transform(Chunk & input_chunk, Chunk & output_chunk) + { + transform(input_chunk); + output_chunk.swap(input_chunk); + } - virtual void transform(Chunk & chunk) = 0; virtual bool needInputData() const { return true; } void stopReading() { no_more_data_needed = true; } diff --git a/src/Processors/ISource.cpp b/src/Processors/ISource.cpp index 7c620a98a74..90f3962b83e 100644 --- a/src/Processors/ISource.cpp +++ b/src/Processors/ISource.cpp @@ -56,11 +56,6 @@ void ISource::work() finished = true; throw; } -// { -// current_chunk = std::current_exception(); -// has_input = true; -// got_exception = true; -// } } } diff --git a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp index 269c69f2747..be9bf3e354c 100644 --- a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp @@ -289,10 +289,10 @@ void AggregatingSortedAlgorithm::initialize(Chunks chunks) initializeQueue(std::move(chunks)); } -void AggregatingSortedAlgorithm::consume(Chunk chunk, size_t source_num) +void AggregatingSortedAlgorithm::consume(Chunk & chunk, size_t source_num) { preprocessChunk(chunk, columns_definition); - updateCursor(std::move(chunk), source_num); + updateCursor(chunk, source_num); } IMergingAlgorithm::Status AggregatingSortedAlgorithm::merge() diff --git a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h index b4819ad030c..fe1710adc8b 100644 --- a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h @@ -20,7 +20,7 @@ public: SortDescription description_, size_t max_block_size); void initialize(Chunks chunks) override; - void consume(Chunk chunk, size_t source_num) override; + void consume(Chunk & chunk, size_t source_num) override; Status merge() override; struct SimpleAggregateDescription; diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp index cd3a193105d..8e799664fae 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp @@ -23,6 +23,7 @@ CollapsingSortedAlgorithm::CollapsingSortedAlgorithm( size_t num_inputs, SortDescription description_, const String & sign_column, + bool only_positive_sign_, size_t max_block_size, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes, @@ -30,6 +31,7 @@ CollapsingSortedAlgorithm::CollapsingSortedAlgorithm( : 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)) + , only_positive_sign(only_positive_sign_) , log(log_) { } @@ -76,7 +78,7 @@ void CollapsingSortedAlgorithm::insertRows() if (last_is_positive || count_positive != count_negative) { - if (count_positive <= count_negative) + if (count_positive <= count_negative && !only_positive_sign) { insertRow(first_negative_row); diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h index 4158f55a7cd..3cbe95d96e1 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h @@ -31,6 +31,7 @@ public: size_t num_inputs, SortDescription description_, const String & sign_column, + bool only_positive_sign_, /// For select final. Skip rows with sum(sign) < 0. size_t max_block_size, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes, @@ -42,6 +43,7 @@ private: MergedData merged_data; const size_t sign_column_number; + const bool only_positive_sign; static constexpr size_t max_row_refs = 4; /// first_negative, last_positive, last, current. RowRef first_negative_row; diff --git a/src/Processors/Merges/Algorithms/IMergingAlgorithm.h b/src/Processors/Merges/Algorithms/IMergingAlgorithm.h index 263acee4c2d..b49209e462e 100644 --- a/src/Processors/Merges/Algorithms/IMergingAlgorithm.h +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithm.h @@ -21,7 +21,7 @@ public: }; virtual void initialize(Chunks chunks) = 0; - virtual void consume(Chunk chunk, size_t source_num) = 0; + virtual void consume(Chunk & chunk, size_t source_num) = 0; virtual Status merge() = 0; IMergingAlgorithm() = default; diff --git a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp index 6777109982e..751a08ce69f 100644 --- a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp @@ -28,15 +28,15 @@ void IMergingAlgorithmWithDelayedChunk::initializeQueue(Chunks chunks) queue = SortingHeap(cursors); } -void IMergingAlgorithmWithDelayedChunk::updateCursor(Chunk chunk, size_t source_num) +void IMergingAlgorithmWithDelayedChunk::updateCursor(Chunk & chunk, size_t source_num) { auto & source_chunk = source_chunks[source_num]; /// Extend lifetime of last chunk. - last_chunk = std::move(source_chunk); + last_chunk.swap(source_chunk); last_chunk_sort_columns = std::move(cursors[source_num].sort_columns); - source_chunk = std::move(chunk); + source_chunk.swap(chunk); cursors[source_num].reset(source_chunk.getColumns(), {}); queue.push(cursors[source_num]); diff --git a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h index d02b9dfcb7a..f7d5f630238 100644 --- a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h @@ -24,7 +24,7 @@ protected: ColumnRawPtrs last_chunk_sort_columns; /// Point to last_chunk if valid. void initializeQueue(Chunks chunks); - void updateCursor(Chunk chunk, size_t source_num); + void updateCursor(Chunk & chunk, size_t source_num); private: /// Chunks currently being merged. diff --git a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp index bcea74b5f38..1fe61653ecc 100644 --- a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp @@ -39,7 +39,7 @@ void IMergingAlgorithmWithSharedChunks::initialize(Chunks chunks) auto & source_chunk = source_chunks[source_num]; - source_chunk = chunk_allocator.alloc(std::move(chunks[source_num])); + source_chunk = chunk_allocator.alloc(chunks[source_num]); cursors[source_num] = SortCursorImpl(source_chunk->getColumns(), description, source_num); source_chunk->all_columns = cursors[source_num].all_columns; @@ -49,12 +49,12 @@ void IMergingAlgorithmWithSharedChunks::initialize(Chunks chunks) queue = SortingHeap(cursors); } -void IMergingAlgorithmWithSharedChunks::consume(Chunk chunk, size_t source_num) +void IMergingAlgorithmWithSharedChunks::consume(Chunk & chunk, size_t source_num) { prepareChunk(chunk); auto & source_chunk = source_chunks[source_num]; - source_chunk = chunk_allocator.alloc(std::move(chunk)); + source_chunk = chunk_allocator.alloc(chunk); cursors[source_num].reset(source_chunk->getColumns(), {}); source_chunk->all_columns = cursors[source_num].all_columns; diff --git a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h index 1ef7f540f96..a3dbadc458d 100644 --- a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h @@ -16,7 +16,7 @@ public: size_t max_row_refs); void initialize(Chunks chunks) override; - void consume(Chunk chunk, size_t source_num) override; + void consume(Chunk & chunk, size_t source_num) override; private: SortDescription description; diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp index 77e9a9b7285..f1d6c56809d 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp @@ -74,10 +74,10 @@ void MergingSortedAlgorithm::initialize(Chunks chunks) queue_without_collation = SortingHeap(cursors); } -void MergingSortedAlgorithm::consume(Chunk chunk, size_t source_num) +void MergingSortedAlgorithm::consume(Chunk & chunk, size_t source_num) { prepareChunk(chunk); - source_chunks[source_num] = std::move(chunk); + source_chunks[source_num].swap(chunk); cursors[source_num].reset(source_chunks[source_num].getColumns(), {}); if (has_collation) diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h index 6ff48b520bd..5b361c1000e 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h @@ -23,7 +23,7 @@ public: void addInput(); void initialize(Chunks chunks) override; - void consume(Chunk chunk, size_t source_num) override; + void consume(Chunk & chunk, size_t source_num) override; Status merge() override; const MergedData & getMergedData() const { return merged_data; } diff --git a/src/Processors/Merges/Algorithms/RowRef.h b/src/Processors/Merges/Algorithms/RowRef.h index 5aeae952067..02a07130bce 100644 --- a/src/Processors/Merges/Algorithms/RowRef.h +++ b/src/Processors/Merges/Algorithms/RowRef.h @@ -63,7 +63,7 @@ public: free_chunks.push_back(i); } - SharedChunkPtr alloc(Chunk && chunk) + SharedChunkPtr alloc(Chunk & chunk) { if (free_chunks.empty()) throw Exception("Not enough space in SharedChunkAllocator. " @@ -72,7 +72,7 @@ public: auto pos = free_chunks.back(); free_chunks.pop_back(); - chunks[pos] = std::move(chunk); + chunks[pos].swap(chunk); chunks[pos].position = pos; chunks[pos].allocator = this; @@ -109,11 +109,6 @@ private: return; } - /// Release memory. It is not obligatory. - ptr->clear(); - ptr->all_columns.clear(); - ptr->sort_columns.clear(); - free_chunks.push_back(ptr->position); } diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 5bcbf778d64..89154044ae5 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -632,10 +632,10 @@ void SummingSortedAlgorithm::initialize(Chunks chunks) initializeQueue(std::move(chunks)); } -void SummingSortedAlgorithm::consume(Chunk chunk, size_t source_num) +void SummingSortedAlgorithm::consume(Chunk & chunk, size_t source_num) { preprocessChunk(chunk); - updateCursor(std::move(chunk), source_num); + updateCursor(chunk, source_num); } IMergingAlgorithm::Status SummingSortedAlgorithm::merge() diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h index a38df215ccc..fc5431f1a08 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h @@ -23,7 +23,7 @@ public: size_t max_block_size); void initialize(Chunks chunks) override; - void consume(Chunk chunk, size_t source_num) override; + void consume(Chunk & chunk, size_t source_num) override; Status merge() override; struct AggregateDescription; diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index d4f40c60938..cdf7c4a1607 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -15,6 +15,7 @@ public: size_t num_inputs, SortDescription description_, const String & sign_column, + bool only_positive_sign, size_t max_block_size, WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false) @@ -24,6 +25,7 @@ public: num_inputs, std::move(description_), sign_column, + only_positive_sign, max_block_size, out_row_sources_buf_, use_average_block_sizes, diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index 0dc4cd41991..2037a88733a 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { @@ -135,7 +136,6 @@ IProcessor::Status IMergingTransformBase::prepare() if (state.is_finished) { - if (is_port_full) return Status::PortFull; @@ -158,11 +158,11 @@ IProcessor::Status IMergingTransformBase::prepare() if (!input.hasData()) return Status::NeedData; - auto chunk = input.pull(); - if (!chunk.hasRows() && !input.isFinished()) + state.input_chunk = input.pull(); + if (!state.input_chunk.hasRows() && !input.isFinished()) return Status::NeedData; - state.input_chunk = std::move(chunk); + state.has_input = true; } state.need_data = false; @@ -174,4 +174,83 @@ IProcessor::Status IMergingTransformBase::prepare() return Status::Ready; } +static void filterChunk(Chunk & chunk, size_t selector_position) +{ + if (!chunk.getChunkInfo()) + throw Exception("IMergingTransformBase expected ChunkInfo for input chunk", ErrorCodes::LOGICAL_ERROR); + + const auto * chunk_info = typeid_cast(chunk.getChunkInfo().get()); + if (!chunk_info) + throw Exception("IMergingTransformBase expected SelectorInfo for input chunk", ErrorCodes::LOGICAL_ERROR); + + const auto & selector = chunk_info->selector; + + IColumn::Filter filter; + filter.resize_fill(selector.size()); + + size_t num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + + size_t num_result_rows = 0; + + for (size_t row = 0; row < num_rows; ++row) + { + if (selector[row] == selector_position) + { + ++num_result_rows; + filter[row] = 1; + } + } + + for (auto & column : columns) + column = column->filter(filter, num_result_rows); + + chunk.clear(); + chunk.setColumns(std::move(columns), num_result_rows); +} + +bool IMergingTransformBase::filterChunks() +{ + if (state.selector_position < 0) + return true; + + bool has_empty_chunk = false; + + if (!state.init_chunks.empty()) + { + for (size_t i = 0; i < input_states.size(); ++i) + { + auto & chunk = state.init_chunks[i]; + if (!chunk || input_states[i].is_filtered) + continue; + + filterChunk(chunk, state.selector_position); + + if (!chunk.hasRows()) + { + chunk.clear(); + has_empty_chunk = true; + input_states[i].is_initialized = false; + is_initialized = false; + } + else + input_states[i].is_filtered = true; + } + } + + if (state.has_input) + { + filterChunk(state.input_chunk, state.selector_position); + if (!state.input_chunk.hasRows()) + { + state.has_input = false; + state.need_data = true; + has_empty_chunk = true; + } + } + + return !has_empty_chunk; +} + + } diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index f9c2dba8271..12a366bf21b 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -18,6 +18,8 @@ public: const Block & output_header, bool have_all_inputs_); + OutputPort & getOutputPort() { return outputs.front(); } + /// Methods to add additional input port. It is possible to do only before the first call of `prepare`. void addInput(); /// Need to be called after all inputs are added. (only if have_all_inputs was not specified). @@ -25,20 +27,29 @@ public: Status prepare() override; + /// Set position which will be used in selector if input chunk has attached SelectorInfo (see SelectorInfo.h). + /// Columns will be filtered, keep only rows labeled with this position. + /// It is used in parallel final. + void setSelectorPosition(size_t position) { state.selector_position = position; } + protected: virtual void onNewInput(); /// Is called when new input is added. Only if have_all_inputs = false. virtual void onFinish() {} /// Is called when all data is processed. + bool filterChunks(); /// Filter chunks if selector position was set. For parallel final. + /// Processor state. struct State { Chunk output_chunk; Chunk input_chunk; + bool has_input = false; bool is_finished = false; bool need_data = false; size_t next_input_to_read = 0; Chunks init_chunks; + ssize_t selector_position = -1; }; State state; @@ -50,6 +61,7 @@ private: InputPort & port; bool is_initialized = false; + bool is_filtered = false; }; std::vector input_states; @@ -78,14 +90,18 @@ public: void work() override { + if (!filterChunks()) + return; + if (!state.init_chunks.empty()) algorithm.initialize(std::move(state.init_chunks)); - if (state.input_chunk) + if (state.has_input) { // std::cerr << "Consume chunk with " << state.input_chunk.getNumRows() // << " for input " << state.next_input_to_read << std::endl; - algorithm.consume(std::move(state.input_chunk), state.next_input_to_read); + algorithm.consume(state.input_chunk, state.next_input_to_read); + state.has_input = false; } IMergingAlgorithm::Status status = algorithm.merge(); @@ -120,4 +136,6 @@ private: using IMergingTransformBase::state; }; +using MergingTransformPtr = std::shared_ptr; + } diff --git a/src/Processors/OffsetTransform.cpp b/src/Processors/OffsetTransform.cpp new file mode 100644 index 00000000000..f380a5a5159 --- /dev/null +++ b/src/Processors/OffsetTransform.cpp @@ -0,0 +1,186 @@ +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +OffsetTransform::OffsetTransform( + const Block & header_, size_t offset_, size_t num_streams) + : IProcessor(InputPorts(num_streams, header_), OutputPorts(num_streams, header_)) + , offset(offset_) +{ + ports_data.resize(num_streams); + + size_t cur_stream = 0; + for (auto & input : inputs) + { + ports_data[cur_stream].input_port = &input; + ++cur_stream; + } + + cur_stream = 0; + for (auto & output : outputs) + { + ports_data[cur_stream].output_port = &output; + ++cur_stream; + } +} + + +IProcessor::Status OffsetTransform::prepare( + const PortNumbers & updated_input_ports, + const PortNumbers & updated_output_ports) +{ + bool has_full_port = false; + + auto process_pair = [&](size_t pos) + { + auto status = preparePair(ports_data[pos]); + + switch (status) + { + case IProcessor::Status::Finished: + { + if (!ports_data[pos].is_finished) + { + ports_data[pos].is_finished = true; + ++num_finished_port_pairs; + } + + return; + } + case IProcessor::Status::PortFull: + { + has_full_port = true; + return; + } + case IProcessor::Status::NeedData: + return; + default: + throw Exception( + "Unexpected status for OffsetTransform::preparePair : " + IProcessor::statusToName(status), + ErrorCodes::LOGICAL_ERROR); + + } + }; + + for (auto pos : updated_input_ports) + process_pair(pos); + + for (auto pos : updated_output_ports) + process_pair(pos); + + /// All ports are finished. It may happen even before we reached the limit (has less data then limit). + if (num_finished_port_pairs == ports_data.size()) + return Status::Finished; + + if (has_full_port) + return Status::PortFull; + + return Status::NeedData; +} + +OffsetTransform::Status OffsetTransform::prepare() +{ + if (ports_data.size() != 1) + throw Exception("prepare without arguments is not supported for multi-port OffsetTransform.", + ErrorCodes::LOGICAL_ERROR); + + return prepare({0}, {0}); +} + +OffsetTransform::Status OffsetTransform::preparePair(PortsData & data) +{ + auto & output = *data.output_port; + auto & input = *data.input_port; + + /// Check can output. + bool output_finished = false; + if (output.isFinished()) + { + output_finished = true; + } + + if (!output_finished && !output.canPush()) + { + input.setNotNeeded(); + return Status::PortFull; + } + + /// Check can input. + + if (input.isFinished()) + { + output.finish(); + return Status::Finished; + } + + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + + data.current_chunk = input.pull(true); + + auto rows = data.current_chunk.getNumRows(); + + if (rows_before_limit_at_least) + rows_before_limit_at_least->add(rows); + + /// Process block. + + rows_read += rows; + + if (rows_read < offset) + { + data.current_chunk.clear(); + + if (input.isFinished()) + { + output.finish(); + return Status::Finished; + } + + /// Now, we pulled from input, and it must be empty. + input.setNeeded(); + return Status::NeedData; + } + + if (!(rows_read >= offset + rows)) + splitChunk(data); + + output.push(std::move(data.current_chunk)); + + return Status::PortFull; +} + + +void OffsetTransform::splitChunk(PortsData & data) const +{ + size_t num_rows = data.current_chunk.getNumRows(); + size_t num_columns = data.current_chunk.getNumColumns(); + + /// return a piece of the block + size_t start = std::max( + static_cast(0), + static_cast(offset) - static_cast(rows_read) + static_cast(num_rows)); + + size_t length = static_cast(rows_read) - static_cast(offset); + + if (length == num_rows) + return; + + auto columns = data.current_chunk.detachColumns(); + + for (size_t i = 0; i < num_columns; ++i) + columns[i] = columns[i]->cut(start, length); + + data.current_chunk.setColumns(std::move(columns), length); +} + +} + diff --git a/src/Processors/OffsetTransform.h b/src/Processors/OffsetTransform.h new file mode 100644 index 00000000000..3fee4e791a5 --- /dev/null +++ b/src/Processors/OffsetTransform.h @@ -0,0 +1,52 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/// Implementation for OFFSET N (without limit) +/// This processor support multiple inputs and outputs (the same number). +/// Each pair of input and output port works independently. +class OffsetTransform : public IProcessor +{ +private: + + size_t offset; + + size_t rows_read = 0; /// including the last read block + RowsBeforeLimitCounterPtr rows_before_limit_at_least; + + /// State of port's pair. + /// Chunks from different port pairs are not mixed for berret cache locality. + struct PortsData + { + Chunk current_chunk; + + InputPort * input_port = nullptr; + OutputPort * output_port = nullptr; + bool is_finished = false; + }; + + std::vector ports_data; + size_t num_finished_port_pairs = 0; + +public: + OffsetTransform(const Block & header_, size_t offset_, size_t num_streams = 1); + + String getName() const override { return "Offset"; } + + Status prepare(const PortNumbers & /*updated_input_ports*/, const PortNumbers & /*updated_output_ports*/) override; + Status prepare() override; /// Compatibility for TreeExecutor. + Status preparePair(PortsData & data); + void splitChunk(PortsData & data) const; + + InputPort & getInputPort() { return inputs.front(); } + OutputPort & getOutputPort() { return outputs.front(); } + + void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { rows_before_limit_at_least.swap(counter); } +}; + +} diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index d9b21dbc854..5d92e909a52 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -96,6 +96,15 @@ Pipe::Pipe(Pipes && pipes, ProcessorPtr transform) processors.emplace_back(std::move(transform)); } +Pipe::Pipe(OutputPort * port) : output_port(port) +{ +} + +void Pipe::addProcessors(const Processors & processors_) +{ + processors.insert(processors.end(), processors_.begin(), processors_.end()); +} + void Pipe::addSimpleTransform(ProcessorPtr transform) { checkSimpleTransform(*transform); diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 42bbd4e06d0..984fa7605c6 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -22,6 +22,8 @@ public: /// Transform must have the number of inputs equals to the number of pipes. And single output. /// Will connect pipes outputs with transform inputs automatically. Pipe(Pipes && pipes, ProcessorPtr transform); + /// Create pipe from output port. If pipe was created that way, it possibly will not have tree shape. + Pipe(OutputPort * port); Pipe(const Pipe & other) = delete; Pipe(Pipe && other) = default; @@ -29,6 +31,9 @@ public: Pipe & operator=(const Pipe & other) = delete; Pipe & operator=(Pipe && other) = default; + /// Append processors to pipe. After this, it possibly will not have tree shape. + void addProcessors(const Processors & processors_); + OutputPort & getPort() const { return *output_port; } const Block & getHeader() const { return output_port->getHeader(); } diff --git a/src/Processors/Sources/SourceWithProgress.cpp b/src/Processors/Sources/SourceWithProgress.cpp index 80844da16cd..8d7a0a3d946 100644 --- a/src/Processors/Sources/SourceWithProgress.cpp +++ b/src/Processors/Sources/SourceWithProgress.cpp @@ -15,7 +15,9 @@ namespace ErrorCodes void SourceWithProgress::work() { if (!limits.speed_limits.checkTimeLimit(total_stopwatch.elapsed(), limits.timeout_overflow_mode)) + { cancel(); + } else { was_progress_called = false; @@ -57,7 +59,13 @@ void SourceWithProgress::progress(const Progress & value) /// The total amount of data processed or intended for processing in all sources, possibly on remote servers. ProgressValues progress = process_list_elem->getProgressIn(); - size_t total_rows_estimate = std::max(progress.read_rows, progress.total_rows_to_read); + + /// If the mode is "throw" and estimate of total rows is known, then throw early if an estimate is too high. + /// If the mode is "break", then allow to read before limit even if estimate is very high. + + size_t rows_to_check_limit = progress.read_rows; + if (limits.size_limits.overflow_mode == OverflowMode::THROW && progress.total_rows_to_read > progress.read_rows) + rows_to_check_limit = progress.total_rows_to_read; /// Check the restrictions on the /// * amount of data to read @@ -67,9 +75,11 @@ void SourceWithProgress::progress(const Progress & value) if (limits.mode == LimitsMode::LIMITS_TOTAL) { - if (!limits.size_limits.check(total_rows_estimate, progress.read_bytes, "rows to read", + if (!limits.size_limits.check(rows_to_check_limit, progress.read_bytes, "rows or bytes to read", ErrorCodes::TOO_MANY_ROWS, ErrorCodes::TOO_MANY_BYTES)) + { cancel(); + } } size_t total_rows = progress.total_rows_to_read; diff --git a/src/Processors/Transforms/AddingSelectorTransform.cpp b/src/Processors/Transforms/AddingSelectorTransform.cpp new file mode 100644 index 00000000000..f75a5920072 --- /dev/null +++ b/src/Processors/Transforms/AddingSelectorTransform.cpp @@ -0,0 +1,76 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +AddingSelectorTransform::AddingSelectorTransform( + const Block & header, size_t num_outputs_, ColumnNumbers key_columns_) + : ISimpleTransform(header, header, false) + , num_outputs(num_outputs_) + , key_columns(std::move(key_columns_)) + , hash(0) +{ + setInputNotNeededAfterRead(false); + + if (num_outputs <= 1) + throw Exception("SplittingByHashTransform expects more than 1 outputs, got " + std::to_string(num_outputs), + ErrorCodes::LOGICAL_ERROR); + + if (key_columns.empty()) + throw Exception("SplittingByHashTransform cannot split by empty set of key columns", + ErrorCodes::LOGICAL_ERROR); + + for (auto & column : key_columns) + if (column >= header.columns()) + throw Exception("Invalid column number: " + std::to_string(column) + + ". There is only " + std::to_string(header.columns()) + " columns in header", + ErrorCodes::LOGICAL_ERROR); +} + +static void calculateWeakHash32(const Chunk & chunk, const ColumnNumbers & key_columns, WeakHash32 & hash) +{ + auto num_rows = chunk.getNumRows(); + const auto & columns = chunk.getColumns(); + + hash.reset(num_rows); + + for (const auto & column_number : key_columns) + columns[column_number]->updateWeakHash32(hash); +} + +static IColumn::Selector fillSelector(const WeakHash32 & hash, size_t num_outputs) +{ + /// Row from interval [(2^32 / num_outputs) * i, (2^32 / num_outputs) * (i + 1)) goes to bucket with number i. + + const auto & hash_data = hash.getData(); + size_t num_rows = hash_data.size(); + IColumn::Selector selector(num_rows); + + for (size_t row = 0; row < num_rows; ++row) + { + selector[row] = hash_data[row]; /// [0, 2^32) + selector[row] *= num_outputs; /// [0, num_outputs * 2^32), selector stores 64 bit values. + selector[row] >>= 32u; /// [0, num_outputs) + } + + return selector; +} + +void AddingSelectorTransform::transform(Chunk & input_chunk, Chunk & output_chunk) +{ + auto chunk_info = std::make_shared(); + + calculateWeakHash32(input_chunk, key_columns, hash); + chunk_info->selector = fillSelector(hash, num_outputs); + + input_chunk.swap(output_chunk); + output_chunk.setChunkInfo(std::move(chunk_info)); +} + +} diff --git a/src/Processors/Transforms/AddingSelectorTransform.h b/src/Processors/Transforms/AddingSelectorTransform.h new file mode 100644 index 00000000000..bad97adfa76 --- /dev/null +++ b/src/Processors/Transforms/AddingSelectorTransform.h @@ -0,0 +1,26 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ + +/// Add IColumn::Selector to chunk (see SelectorInfo.h). +/// Selector is filled by formula (WeakHash(key_columns) * num_outputs / MAX_INT). +class AddingSelectorTransform : public ISimpleTransform +{ +public: + AddingSelectorTransform(const Block & header, size_t num_outputs_, ColumnNumbers key_columns_); + String getName() const override { return "AddingSelector"; } + void transform(Chunk & input_chunk, Chunk & output_chunk) override; + +private: + size_t num_outputs; + ColumnNumbers key_columns; + + WeakHash32 hash; +}; + +} diff --git a/src/Processors/Transforms/CopyTransform.cpp b/src/Processors/Transforms/CopyTransform.cpp new file mode 100644 index 00000000000..c9047c942d6 --- /dev/null +++ b/src/Processors/Transforms/CopyTransform.cpp @@ -0,0 +1,108 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +CopyTransform::CopyTransform(const Block & header, size_t num_outputs) + : IProcessor(InputPorts(1, header), OutputPorts(num_outputs, header)) +{ + if (num_outputs <= 1) + throw Exception("CopyTransform expects more than 1 outputs, got " + std::to_string(num_outputs), ErrorCodes::LOGICAL_ERROR); +} + +IProcessor::Status CopyTransform::prepare() +{ + Status status = Status::Ready; + + while (status == Status::Ready) + { + status = !has_data ? prepareConsume() + : prepareGenerate(); + } + + return status; +} + +IProcessor::Status CopyTransform::prepareConsume() +{ + auto & input = getInputPort(); + + /// Check all outputs are finished or ready to get data. + + bool all_finished = true; + for (auto & output : outputs) + { + if (output.isFinished()) + continue; + + all_finished = false; + } + + if (all_finished) + { + input.close(); + return Status::Finished; + } + + /// Try get chunk from input. + + if (input.isFinished()) + { + for (auto & output : outputs) + output.finish(); + + return Status::Finished; + } + + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + + chunk = input.pull(); + has_data = true; + was_output_processed.assign(outputs.size(), false); + + return Status::Ready; +} + +IProcessor::Status CopyTransform::prepareGenerate() +{ + bool all_outputs_processed = true; + + size_t chunk_number = 0; + for (auto & output : outputs) + { + auto & was_processed = was_output_processed[chunk_number]; + ++chunk_number; + + if (was_processed) + continue; + + if (output.isFinished()) + continue; + + if (!output.canPush()) + { + all_outputs_processed = false; + continue; + } + + output.push(chunk.clone()); + was_processed = true; + } + + if (all_outputs_processed) + { + has_data = false; + return Status::Ready; + } + + return Status::PortFull; +} + +} diff --git a/src/Processors/Transforms/CopyTransform.h b/src/Processors/Transforms/CopyTransform.h new file mode 100644 index 00000000000..cf56fdf10d9 --- /dev/null +++ b/src/Processors/Transforms/CopyTransform.h @@ -0,0 +1,28 @@ +#pragma once +#include + +namespace DB +{ + +/// Transform which has single input and num_outputs outputs. +/// Read chunk from input and copy it to all outputs. +class CopyTransform : public IProcessor +{ +public: + CopyTransform(const Block & header, size_t num_outputs); + + String getName() const override { return "Copy"; } + Status prepare() override; + + InputPort & getInputPort() { return inputs.front(); } + +private: + Chunk chunk; + bool has_data = false; + std::vector was_output_processed; + + Status prepareGenerate(); + Status prepareConsume(); +}; + +} diff --git a/src/Processors/Transforms/LimitsCheckingTransform.cpp b/src/Processors/Transforms/LimitsCheckingTransform.cpp index c3ac019f2b6..56edd5f0317 100644 --- a/src/Processors/Transforms/LimitsCheckingTransform.cpp +++ b/src/Processors/Transforms/LimitsCheckingTransform.cpp @@ -44,7 +44,9 @@ void LimitsCheckingTransform::transform(Chunk & chunk) if (limits.mode == LimitsMode::LIMITS_CURRENT && !limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES)) + { stopReading(); + } if (quota) checkQuota(chunk); @@ -56,13 +58,16 @@ void LimitsCheckingTransform::checkQuota(Chunk & chunk) switch (limits.mode) { case LimitsMode::LIMITS_TOTAL: - /// Checked in `progress` method. + /// Checked in SourceWithProgress::progress method. break; case LimitsMode::LIMITS_CURRENT: { UInt64 total_elapsed = info.total_stopwatch.elapsedNanoseconds(); - quota->used({Quota::RESULT_ROWS, chunk.getNumRows()}, {Quota::RESULT_BYTES, chunk.bytes()}, {Quota::EXECUTION_TIME, total_elapsed - prev_elapsed}); + quota->used( + {Quota::RESULT_ROWS, chunk.getNumRows()}, + {Quota::RESULT_BYTES, chunk.bytes()}, + {Quota::EXECUTION_TIME, total_elapsed - prev_elapsed}); prev_elapsed = total_elapsed; break; } diff --git a/src/Processors/Transforms/MergeSortingTransform.h b/src/Processors/Transforms/MergeSortingTransform.h index a8786e5a034..043cb3f36c1 100644 --- a/src/Processors/Transforms/MergeSortingTransform.h +++ b/src/Processors/Transforms/MergeSortingTransform.h @@ -38,6 +38,9 @@ private: VolumePtr tmp_volume; size_t min_free_disk_space; + size_t sum_rows_in_blocks = 0; + size_t sum_bytes_in_blocks = 0; + Logger * log = &Logger::get("MergeSortingTransform"); /// If remerge doesn't save memory at least several times, mark it as useless and don't do it anymore. diff --git a/src/Processors/Transforms/SelectorInfo.h b/src/Processors/Transforms/SelectorInfo.h new file mode 100644 index 00000000000..2876d64ed28 --- /dev/null +++ b/src/Processors/Transforms/SelectorInfo.h @@ -0,0 +1,14 @@ +#pragma once +#include +#include + +namespace DB +{ + +/// ChunkInfo with IColumn::Selector. It is added by AddingSelectorTransform. +struct SelectorInfo : public ChunkInfo +{ + IColumn::Selector selector; +}; + +} diff --git a/src/Processors/Transforms/SortingTransform.h b/src/Processors/Transforms/SortingTransform.h index 49bdf303c7f..9178991f324 100644 --- a/src/Processors/Transforms/SortingTransform.h +++ b/src/Processors/Transforms/SortingTransform.h @@ -66,8 +66,8 @@ class SortingTransform : public IProcessor public: /// limit - if not 0, allowed to return just first 'limit' rows in sorted order. SortingTransform(const Block & header, - const SortDescription & description_, - size_t max_merged_block_size_, UInt64 limit_); + const SortDescription & description_, + size_t max_merged_block_size_, UInt64 limit_); ~SortingTransform() override; @@ -83,9 +83,6 @@ protected: size_t max_merged_block_size; UInt64 limit; - size_t sum_rows_in_blocks = 0; - size_t sum_bytes_in_blocks = 0; - /// Before operation, will remove constant columns from blocks. And after, place constant columns back. /// (to avoid excessive virtual function calls and because constants cannot be serialized in Native format for temporary files) /// Save original block structure here. diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 02402df56e3..8659fffd741 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -98,6 +98,7 @@ SRCS( Merges/ReplacingSortedTransform.h Merges/SummingSortedTransform.h Merges/VersionedCollapsingTransform.h + OffsetTransform.cpp Pipe.cpp Port.cpp QueryPipeline.cpp @@ -106,8 +107,10 @@ SRCS( Sources/SourceFromInputStream.cpp Sources/SourceWithProgress.cpp Transforms/AddingMissedTransform.cpp + Transforms/AddingSelectorTransform.cpp Transforms/AggregatingTransform.cpp Transforms/ConvertingTransform.cpp + Transforms/CopyTransform.cpp Transforms/CreatingSetsTransform.cpp Transforms/CubeTransform.cpp Transforms/DistinctTransform.cpp diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 2a5e47fdf04..d90a7974968 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -215,8 +215,10 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri auto pools = createPoolsForAddresses(name, pool_factory); const auto settings = storage.global_context.getSettings(); - return pools.size() == 1 ? pools.front() : std::make_shared(pools, LoadBalancing::RANDOM, - settings.distributed_replica_error_half_life.totalSeconds(), settings.distributed_replica_error_cap); + return pools.size() == 1 ? pools.front() : std::make_shared(pools, + settings.load_balancing, + settings.distributed_replica_error_half_life.totalSeconds(), + settings.distributed_replica_error_cap); } @@ -274,7 +276,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa ClientInfo client_info; readHeader(in, insert_settings, insert_query, client_info, log); - RemoteBlockOutputStream remote{*connection, timeouts, insert_query, &insert_settings, &client_info}; + RemoteBlockOutputStream remote{*connection, timeouts, insert_query, insert_settings, client_info}; remote.writePrefix(); remote.writePrepared(in); @@ -463,7 +465,7 @@ struct StorageDistributedDirectoryMonitor::Batch if (first) { first = false; - remote = std::make_unique(*connection, timeouts, insert_query, &insert_settings, &client_info); + remote = std::make_unique(*connection, timeouts, insert_query, insert_settings, client_info); remote->writePrefix(); } diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 55d8d4da9d8..c7b0436a9a3 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -167,6 +167,7 @@ std::string DistributedBlockOutputStream::getCurrentStateDescription() void DistributedBlockOutputStream::initWritingJobs(const Block & first_block) { + const Settings & settings = context.getSettingsRef(); const auto & addresses_with_failovers = cluster->getShardsAddresses(); const auto & shards_info = cluster->getShardsInfo(); size_t num_shards = shards_info.size(); @@ -180,14 +181,14 @@ void DistributedBlockOutputStream::initWritingJobs(const Block & first_block) const auto & shard_info = shards_info[shard_index]; auto & shard_jobs = per_shard_jobs[shard_index]; - /// If hasInternalReplication, than prefer local replica - if (!shard_info.hasInternalReplication() || !shard_info.isLocal()) + /// If hasInternalReplication, than prefer local replica (if !prefer_localhost_replica) + if (!shard_info.hasInternalReplication() || !shard_info.isLocal() || !settings.prefer_localhost_replica) { const auto & replicas = addresses_with_failovers[shard_index]; for (size_t replica_index : ext::range(0, replicas.size())) { - if (!replicas[replica_index].is_local) + if (!replicas[replica_index].is_local || !settings.prefer_localhost_replica) { shard_jobs.replicas_jobs.emplace_back(shard_index, replica_index, false, first_block); ++remote_jobs_count; @@ -198,7 +199,7 @@ void DistributedBlockOutputStream::initWritingJobs(const Block & first_block) } } - if (shard_info.isLocal()) + if (shard_info.isLocal() && settings.prefer_localhost_replica) { shard_jobs.replicas_jobs.emplace_back(shard_index, 0, true, first_block); ++local_jobs_count; @@ -275,12 +276,12 @@ ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutp } const Block & shard_block = (num_shards > 1) ? job.current_shard_block : current_block; + const Settings & settings = context.getSettingsRef(); - if (!job.is_local_job) + if (!job.is_local_job || !settings.prefer_localhost_replica) { if (!job.stream) { - const Settings & settings = context.getSettingsRef(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings); if (shard_info.hasInternalReplication()) { @@ -311,14 +312,14 @@ ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutp if (throttler) job.connection_entry->setThrottler(throttler); - job.stream = std::make_shared(*job.connection_entry, timeouts, query_string, &settings, &context.getClientInfo()); + job.stream = std::make_shared(*job.connection_entry, timeouts, query_string, settings, context.getClientInfo()); job.stream->writePrefix(); } CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; job.stream->write(shard_block); } - else + else // local { if (!job.stream) { @@ -507,31 +508,25 @@ void DistributedBlockOutputStream::writeSplitAsync(const Block & block) void DistributedBlockOutputStream::writeAsyncImpl(const Block & block, const size_t shard_id) { const auto & shard_info = cluster->getShardsInfo()[shard_id]; + const auto & settings = context.getSettingsRef(); if (shard_info.hasInternalReplication()) { - if (shard_info.getLocalNodeCount() > 0) - { + if (shard_info.isLocal() && settings.prefer_localhost_replica) /// Prefer insert into current instance directly writeToLocal(block, shard_info.getLocalNodeCount()); - } else - { - if (shard_info.dir_name_for_internal_replication.empty()) - throw Exception("Directory name for async inserts is empty, table " + storage.getStorageID().getNameForLogs(), ErrorCodes::LOGICAL_ERROR); - - writeToShard(block, {shard_info.dir_name_for_internal_replication}); - } + writeToShard(block, {shard_info.pathForInsert(settings.prefer_localhost_replica)}); } else { - if (shard_info.getLocalNodeCount() > 0) + if (shard_info.isLocal()) writeToLocal(block, shard_info.getLocalNodeCount()); std::vector dir_names; for (const auto & address : cluster->getShardsAddresses()[shard_id]) - if (!address.is_local) - dir_names.push_back(address.toFullString(context.getSettingsRef().use_compact_format_in_distributed_parts_names)); + if (!address.is_local || !settings.prefer_localhost_replica) + dir_names.push_back(address.toFullString(settings.use_compact_format_in_distributed_parts_names)); if (!dir_names.empty()) writeToShard(block, dir_names); diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 2e2797983b6..5c4657403b7 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -32,6 +33,7 @@ #include #include #include +#include namespace DB @@ -67,6 +69,46 @@ namespace conf.set(key_name, config.getString(key_path)); } } + + rd_kafka_resp_err_t rdKafkaOnThreadStart(rd_kafka_t *, rd_kafka_thread_type_t thread_type, const char *, void * ctx) + { + StorageKafka * self = reinterpret_cast(ctx); + + const auto & storage_id = self->getStorageID(); + const auto & table = storage_id.getTableName(); + + switch (thread_type) + { + case RD_KAFKA_THREAD_MAIN: + setThreadName(("rdk:m/" + table.substr(0, 9)).c_str()); + break; + case RD_KAFKA_THREAD_BACKGROUND: + setThreadName(("rdk:bg/" + table.substr(0, 8)).c_str()); + break; + case RD_KAFKA_THREAD_BROKER: + setThreadName(("rdk:b/" + table.substr(0, 9)).c_str()); + break; + } + return RD_KAFKA_RESP_ERR_NO_ERROR; + } + + rd_kafka_resp_err_t rdKafkaOnNew(rd_kafka_t * rk, const rd_kafka_conf_t *, void * ctx, char * /*errstr*/, size_t /*errstr_size*/) + { + return rd_kafka_interceptor_add_on_thread_start(rk, "setThreadName", rdKafkaOnThreadStart, ctx); + } + + rd_kafka_resp_err_t rdKafkaOnConfDup(rd_kafka_conf_t * new_conf, const rd_kafka_conf_t * /*old_conf*/, size_t /*filter_cnt*/, const char ** /*filter*/, void * ctx) + { + rd_kafka_resp_err_t status; + + // cppkafka copies configuration multiple times + status = rd_kafka_conf_interceptor_add_on_conf_dup(new_conf, "setThreadName", rdKafkaOnConfDup, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + return status; + + status = rd_kafka_conf_interceptor_add_on_new(new_conf, "setThreadName", rdKafkaOnNew, ctx); + return status; + } } StorageKafka::StorageKafka( @@ -278,6 +320,33 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & conf) if (config.has(topic_config_key)) loadFromConfig(conf, config, topic_config_key); } + + // 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) + { + auto [poco_level, client_logs_level] = parseSyslogLevel(level); + LOG_SIMPLE(log, message, client_logs_level, poco_level); + }); + + // Configure interceptor to change thread name + // + // TODO: add interceptors support into the cppkafka. + // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatibliity overrides it to noop. + { + // This should be safe, since we wait the rdkafka object anyway. + void * self = reinterpret_cast(this); + + int status; + + status = rd_kafka_conf_interceptor_add_on_new(conf.get_handle(), "setThreadName", rdKafkaOnNew, self); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(log, "Cannot set new interceptor"); + + // cppkafka always copy the configuration + status = rd_kafka_conf_interceptor_add_on_conf_dup(conf.get_handle(), "setThreadName", rdKafkaOnConfDup, self); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(log, "Cannot set dup conf interceptor"); + } } bool StorageKafka::checkDependencies(const StorageID & table_id) diff --git a/src/Storages/Kafka/parseSyslogLevel.cpp b/src/Storages/Kafka/parseSyslogLevel.cpp new file mode 100644 index 00000000000..692bc03000b --- /dev/null +++ b/src/Storages/Kafka/parseSyslogLevel.cpp @@ -0,0 +1,32 @@ +#include "parseSyslogLevel.h" +#include + +/// Must be in a sepearate compilation unit due to macros overlaps: +/// - syslog (LOG_DEBUG/...) +/// - logger_useful.h (LOG_DEBUG()/...) +std::pair parseSyslogLevel(const int level) +{ + using DB::LogsLevel; + using Poco::Message; + + switch (level) + { + case LOG_EMERG: [[fallthrough]]; + case LOG_ALERT: + return std::make_pair(Message::PRIO_FATAL, LogsLevel::error); + case LOG_CRIT: + return std::make_pair(Message::PRIO_CRITICAL, LogsLevel::error); + case LOG_ERR: + return std::make_pair(Message::PRIO_ERROR, LogsLevel::error); + case LOG_WARNING: + return std::make_pair(Message::PRIO_WARNING, LogsLevel::warning); + case LOG_NOTICE: + return std::make_pair(Message::PRIO_NOTICE, LogsLevel::information); + case LOG_INFO: + return std::make_pair(Message::PRIO_INFORMATION, LogsLevel::information); + case LOG_DEBUG: + return std::make_pair(Message::PRIO_DEBUG, LogsLevel::debug); + default: + return std::make_pair(Message::PRIO_TRACE, LogsLevel::trace); + } +} diff --git a/src/Storages/Kafka/parseSyslogLevel.h b/src/Storages/Kafka/parseSyslogLevel.h new file mode 100644 index 00000000000..f0fb4968631 --- /dev/null +++ b/src/Storages/Kafka/parseSyslogLevel.h @@ -0,0 +1,7 @@ +#pragma once + +#include +#include +#include + +std::pair parseSyslogLevel(const int level); diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 70c06efff5c..2b491f9ede6 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -115,7 +116,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo { String file_name = it.first; - auto disk = part->disk; + auto disk = part->volume->getDisk(); String path = part->getFullRelativePath() + file_name; UInt64 size = disk->getFileSize(path); @@ -316,7 +317,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( assertEOF(in); - MergeTreeData::MutableDataPartPtr new_data_part = data.createPart(part_name, reservation->getDisk(), part_relative_path); + auto volume = std::make_shared("volume_" + part_name, disk); + MergeTreeData::MutableDataPartPtr new_data_part = data.createPart(part_name, volume, part_relative_path); new_data_part->is_temp = true; new_data_part->modification_time = time(nullptr); new_data_part->loadColumnsChecksumsIndexes(true, false); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 59bd2830789..44ad4af626e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -137,11 +137,11 @@ void IMergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other) IMergeTreeDataPart::IMergeTreeDataPart( - MergeTreeData & storage_, const String & name_, const DiskPtr & disk_, const std::optional & relative_path_, Type part_type_) + MergeTreeData & storage_, const String & name_, const VolumePtr & volume_, const std::optional & relative_path_, Type part_type_) : storage(storage_) , name(name_) , info(MergeTreePartInfo::fromPartName(name_, storage.format_version)) - , disk(disk_) + , volume(volume_) , relative_path(relative_path_.value_or(name_)) , index_granularity_info(storage_, part_type_) , part_type(part_type_) @@ -152,13 +152,13 @@ IMergeTreeDataPart::IMergeTreeDataPart( const MergeTreeData & storage_, const String & name_, const MergeTreePartInfo & info_, - const DiskPtr & disk_, + const VolumePtr & volume_, const std::optional & relative_path_, Type part_type_) : storage(storage_) , name(name_) , info(info_) - , disk(disk_) + , volume(volume_) , relative_path(relative_path_.value_or(name_)) , index_granularity_info(storage_, part_type_) , part_type(part_type_) @@ -245,7 +245,7 @@ void IMergeTreeDataPart::removeIfNeeded() { auto path = getFullRelativePath(); - if (!disk->exists(path)) + if (!volume->getDisk()->exists(path)) return; if (is_temp) @@ -392,7 +392,7 @@ String IMergeTreeDataPart::getFullPath() const if (relative_path.empty()) throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - return storage.getFullPathOnDisk(disk) + relative_path + "/"; + return storage.getFullPathOnDisk(volume->getDisk()) + relative_path + "/"; } String IMergeTreeDataPart::getFullRelativePath() const @@ -452,7 +452,7 @@ void IMergeTreeDataPart::loadIndex() } String index_path = getFullRelativePath() + "primary.idx"; - auto index_file = openForReading(disk, index_path); + auto index_file = openForReading(volume->getDisk(), index_path); for (size_t i = 0; i < index_granularity.getMarksCount(); ++i) //-V756 for (size_t j = 0; j < key_size; ++j) @@ -468,7 +468,7 @@ void IMergeTreeDataPart::loadIndex() } if (!index_file->eof()) - throw Exception("Index file " + fullPath(disk, index_path) + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE); + throw Exception("Index file " + fullPath(volume->getDisk(), index_path) + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE); index.assign(std::make_move_iterator(loaded_index.begin()), std::make_move_iterator(loaded_index.end())); } @@ -489,9 +489,9 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() else { String path = getFullRelativePath(); - partition.load(storage, disk, path); + partition.load(storage, volume->getDisk(), path); if (!isEmpty()) - minmax_idx.load(storage, disk, path); + minmax_idx.load(storage, volume->getDisk(), path); } String calculated_partition_id = partition.getID(storage.partition_key_sample); @@ -505,23 +505,23 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() void IMergeTreeDataPart::loadChecksums(bool require) { String path = getFullRelativePath() + "checksums.txt"; - if (disk->exists(path)) + if (volume->getDisk()->exists(path)) { - auto buf = openForReading(disk, path); + auto buf = openForReading(volume->getDisk(), path); if (checksums.read(*buf)) { assertEOF(*buf); bytes_on_disk = checksums.getTotalSizeOnDisk(); } else - bytes_on_disk = calculateTotalSizeOnDisk(disk, getFullRelativePath()); + bytes_on_disk = calculateTotalSizeOnDisk(volume->getDisk(), getFullRelativePath()); } else { if (require) throw Exception("No checksums.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); - bytes_on_disk = calculateTotalSizeOnDisk(disk, getFullRelativePath()); + bytes_on_disk = calculateTotalSizeOnDisk(volume->getDisk(), getFullRelativePath()); } } @@ -534,10 +534,10 @@ void IMergeTreeDataPart::loadRowsCount() } else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT) { - if (!disk->exists(path)) + if (!volume->getDisk()->exists(path)) throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); - auto buf = openForReading(disk, path); + auto buf = openForReading(volume->getDisk(), path); readIntText(rows_count, *buf); assertEOF(*buf); } @@ -582,9 +582,9 @@ void IMergeTreeDataPart::loadRowsCount() void IMergeTreeDataPart::loadTTLInfos() { String path = getFullRelativePath() + "ttl.txt"; - if (disk->exists(path)) + if (volume->getDisk()->exists(path)) { - auto in = openForReading(disk, path); + auto in = openForReading(volume->getDisk(), path); assertString("ttl format version: ", *in); size_t format_version; readText(format_version, *in); @@ -609,7 +609,7 @@ void IMergeTreeDataPart::loadTTLInfos() void IMergeTreeDataPart::loadColumns(bool require) { String path = getFullRelativePath() + "columns.txt"; - if (!disk->exists(path)) + if (!volume->getDisk()->exists(path)) { /// We can get list of columns only from columns.txt in compact parts. if (require || part_type == Type::COMPACT) @@ -617,21 +617,21 @@ void IMergeTreeDataPart::loadColumns(bool require) /// If there is no file with a list of columns, write it down. for (const NameAndTypePair & column : storage.getColumns().getAllPhysical()) - if (disk->exists(getFullRelativePath() + getFileNameForColumn(column) + ".bin")) + if (volume->getDisk()->exists(getFullRelativePath() + getFileNameForColumn(column) + ".bin")) columns.push_back(column); if (columns.empty()) throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); { - auto buf = disk->writeFile(path + ".tmp", 4096); + auto buf = volume->getDisk()->writeFile(path + ".tmp", 4096); columns.writeText(*buf); } - disk->moveFile(path + ".tmp", path); + volume->getDisk()->moveFile(path + ".tmp", path); } else { - columns.readText(*disk->readFile(path)); + columns.readText(*volume->getDisk()->readFile(path)); } size_t pos = 0; @@ -659,29 +659,29 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ String from = getFullRelativePath(); String to = storage.relative_data_path + new_relative_path + "/"; - if (!disk->exists(from)) - throw Exception("Part directory " + fullPath(disk, from) + " doesn't exist. Most likely it is logical error.", ErrorCodes::FILE_DOESNT_EXIST); + if (!volume->getDisk()->exists(from)) + throw Exception("Part directory " + fullPath(volume->getDisk(), from) + " doesn't exist. Most likely it is logical error.", ErrorCodes::FILE_DOESNT_EXIST); - if (disk->exists(to)) + if (volume->getDisk()->exists(to)) { if (remove_new_dir_if_exists) { Names files; - disk->listFiles(to, files); + volume->getDisk()->listFiles(to, files); - LOG_WARNING(storage.log, "Part directory " << fullPath(disk, to) << " already exists" + LOG_WARNING(storage.log, "Part directory " << fullPath(volume->getDisk(), to) << " already exists" << " and contains " << files.size() << " files. Removing it."); - disk->removeRecursive(to); + volume->getDisk()->removeRecursive(to); } else { - throw Exception("Part directory " + fullPath(disk, to) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS); + throw Exception("Part directory " + fullPath(volume->getDisk(), to) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS); } } - disk->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); - disk->moveFile(from, to); + volume->getDisk()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); + volume->getDisk()->moveFile(from, to); relative_path = new_relative_path; } @@ -710,29 +710,29 @@ void IMergeTreeDataPart::remove() const String to = storage.relative_data_path + "delete_tmp_" + name; // TODO directory delete_tmp_ is never removed if server crashes before returning from this function - if (disk->exists(to)) + if (volume->getDisk()->exists(to)) { - LOG_WARNING(storage.log, "Directory " << fullPath(disk, to) << " (to which part must be renamed before removing) already exists." + LOG_WARNING(storage.log, "Directory " << fullPath(volume->getDisk(), to) << " (to which part must be renamed before removing) already exists." " Most likely this is due to unclean restart. Removing it."); try { - disk->removeRecursive(to + "/"); + volume->getDisk()->removeRecursive(to + "/"); } catch (...) { - LOG_ERROR(storage.log, "Cannot recursively remove directory " << fullPath(disk, to) << ". Exception: " << getCurrentExceptionMessage(false)); + LOG_ERROR(storage.log, "Cannot recursively remove directory " << fullPath(volume->getDisk(), to) << ". Exception: " << getCurrentExceptionMessage(false)); throw; } } try { - disk->moveFile(from, to); + volume->getDisk()->moveFile(from, to); } catch (const Poco::FileNotFoundException &) { - LOG_ERROR(storage.log, "Directory " << fullPath(disk, to) << " (part to remove) doesn't exist or one of nested files has gone." + LOG_ERROR(storage.log, "Directory " << fullPath(volume->getDisk(), to) << " (part to remove) doesn't exist or one of nested files has gone." " Most likely this is due to manual removing. This should be discouraged. Ignoring."); return; @@ -741,7 +741,7 @@ void IMergeTreeDataPart::remove() const if (checksums.empty()) { /// If the part is not completely written, we cannot use fast path by listing files. - disk->removeRecursive(to + "/"); + volume->getDisk()->removeRecursive(to + "/"); } else { @@ -754,25 +754,25 @@ void IMergeTreeDataPart::remove() const # pragma GCC diagnostic ignored "-Wunused-variable" #endif for (const auto & [file, _] : checksums.files) - disk->remove(to + "/" + file); + volume->getDisk()->remove(to + "/" + file); #if !__clang__ # pragma GCC diagnostic pop #endif for (const auto & file : {"checksums.txt", "columns.txt"}) - disk->remove(to + "/" + file); - disk->removeIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_PATH); + volume->getDisk()->remove(to + "/" + file); + volume->getDisk()->removeIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_PATH); - disk->remove(to); + volume->getDisk()->remove(to); } catch (...) { /// Recursive directory removal does many excessive "stat" syscalls under the hood. - LOG_ERROR(storage.log, "Cannot quickly remove directory " << fullPath(disk, to) << " by removing files; fallback to recursive removal. Reason: " + LOG_ERROR(storage.log, "Cannot quickly remove directory " << fullPath(volume->getDisk(), to) << " by removing files; fallback to recursive removal. Reason: " << getCurrentExceptionMessage(false)); - disk->removeRecursive(to + "/"); + volume->getDisk()->removeRecursive(to + "/"); } } } @@ -793,7 +793,7 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) { res = "detached/" + (prefix.empty() ? "" : prefix + "_") + name + (try_no ? "_try" + DB::toString(try_no) : ""); - if (!disk->exists(getFullRelativePath() + res)) + if (!volume->getDisk()->exists(getFullRelativePath() + res)) return res; LOG_WARNING(storage.log, "Directory " << res << " (to detach to) already exists." @@ -817,16 +817,16 @@ void IMergeTreeDataPart::makeCloneInDetached(const String & prefix) const String destination_path = storage.relative_data_path + getRelativePathForDetachedPart(prefix); /// Backup is not recursive (max_level is 0), so do not copy inner directories - localBackup(disk, getFullRelativePath(), destination_path, 0); - disk->removeIfExists(destination_path + "/" + DELETE_ON_DESTROY_MARKER_PATH); + localBackup(volume->getDisk(), getFullRelativePath(), destination_path, 0); + volume->getDisk()->removeIfExists(destination_path + "/" + DELETE_ON_DESTROY_MARKER_PATH); } void IMergeTreeDataPart::makeCloneOnDiskDetached(const ReservationPtr & reservation) const { assertOnDisk(); auto reserved_disk = reservation->getDisk(); - if (reserved_disk->getName() == disk->getName()) - throw Exception("Can not clone data part " + name + " to same disk " + disk->getName(), ErrorCodes::LOGICAL_ERROR); + if (reserved_disk->getName() == volume->getDisk()->getName()) + throw Exception("Can not clone data part " + name + " to same disk " + volume->getDisk()->getName(), ErrorCodes::LOGICAL_ERROR); String path_to_clone = storage.relative_data_path + "detached/"; @@ -834,8 +834,8 @@ void IMergeTreeDataPart::makeCloneOnDiskDetached(const ReservationPtr & reservat throw Exception("Path " + fullPath(reserved_disk, path_to_clone + relative_path) + " already exists. Can not clone ", ErrorCodes::DIRECTORY_ALREADY_EXISTS); reserved_disk->createDirectory(path_to_clone); - disk->copy(getFullRelativePath(), reserved_disk, path_to_clone); - disk->removeIfExists(path_to_clone + "/" + DELETE_ON_DESTROY_MARKER_PATH); + volume->getDisk()->copy(getFullRelativePath(), reserved_disk, path_to_clone); + volume->getDisk()->removeIfExists(path_to_clone + "/" + DELETE_ON_DESTROY_MARKER_PATH); } void IMergeTreeDataPart::checkConsistencyBase() const @@ -865,7 +865,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const } } - checksums.checkSizes(disk, path); + checksums.checkSizes(volume->getDisk(), path); } else { @@ -879,17 +879,17 @@ void IMergeTreeDataPart::checkConsistencyBase() const /// Check that the primary key index is not empty. if (!storage.primary_key_columns.empty()) - check_file_not_empty(disk, path + "primary.idx"); + check_file_not_empty(volume->getDisk(), path + "primary.idx"); if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { - check_file_not_empty(disk, path + "count.txt"); + check_file_not_empty(volume->getDisk(), path + "count.txt"); if (storage.partition_key_expr) - check_file_not_empty(disk, path + "partition.dat"); + check_file_not_empty(volume->getDisk(), path + "partition.dat"); for (const String & col_name : storage.minmax_idx_columns) - check_file_not_empty(disk, path + "minmax_" + escapeForFileName(col_name) + ".idx"); + check_file_not_empty(volume->getDisk(), path + "minmax_" + escapeForFileName(col_name) + ".idx"); } } } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 338c1db1f06..f0ae8b40c5b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -31,6 +31,9 @@ struct FutureMergedMutatedPart; class IReservation; using ReservationPtr = std::unique_ptr; +class IVolume; +using VolumePtr = std::shared_ptr; + class IMergeTreeReader; class IMergeTreeDataPartWriter; @@ -60,14 +63,14 @@ public: const MergeTreeData & storage_, const String & name_, const MergeTreePartInfo & info_, - const DiskPtr & disk, + const VolumePtr & volume, const std::optional & relative_path, Type part_type_); IMergeTreeDataPart( MergeTreeData & storage_, const String & name_, - const DiskPtr & disk, + const VolumePtr & volume, const std::optional & relative_path, Type part_type_); @@ -155,7 +158,7 @@ public: String name; MergeTreePartInfo info; - DiskPtr disk; + VolumePtr volume; mutable String relative_path; MergeTreeIndexGranularityInfo index_granularity_info; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 9928583df80..763f4fff879 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -63,18 +63,16 @@ void IMergeTreeDataPartWriter::Stream::addToChecksums(MergeTreeData::DataPart::C IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( - DiskPtr disk_, - const String & part_path_, - const MergeTreeData & storage_, + const MergeTreeData::DataPartPtr & data_part_, const NamesAndTypesList & columns_list_, const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : disk(std::move(disk_)) - , part_path(part_path_) - , storage(storage_) + : data_part(data_part_) + , part_path(data_part_->getFullRelativePath()) + , storage(data_part_->storage) , columns_list(columns_list_) , marks_file_extension(marks_file_extension_) , index_granularity(index_granularity_) @@ -87,6 +85,7 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( if (settings.blocks_are_granules_size && !index_granularity.empty()) throw Exception("Can't take information about index granularity from blocks, when non empty index_granularity array specified", ErrorCodes::LOGICAL_ERROR); + auto disk = data_part->volume->getDisk(); if (!disk->exists(part_path)) disk->createDirectories(part_path); } @@ -165,7 +164,7 @@ void IMergeTreeDataPartWriter::initPrimaryIndex() { if (storage.hasPrimaryKey()) { - index_file_stream = disk->writeFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); + index_file_stream = data_part->volume->getDisk()->writeFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); index_stream = std::make_unique(*index_file_stream); } @@ -180,7 +179,7 @@ void IMergeTreeDataPartWriter::initSkipIndices() skip_indices_streams.emplace_back( std::make_unique( stream_name, - disk, + data_part->volume->getDisk(), part_path + stream_name, INDEX_FILE_EXTENSION, part_path + stream_name, marks_file_extension, default_codec, settings.max_compress_block_size, diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 142ad6ca14e..ffdba570544 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -61,9 +61,7 @@ public: using StreamPtr = std::unique_ptr; IMergeTreeDataPartWriter( - DiskPtr disk, - const String & part_path, - const MergeTreeData & storage, + const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns_list, const std::vector & indices_to_recalc, const String & marks_file_extension, @@ -118,7 +116,7 @@ protected: using SerializationState = IDataType::SerializeBinaryBulkStatePtr; using SerializationStates = std::unordered_map; - DiskPtr disk; + MergeTreeData::DataPartPtr data_part; String part_path; const MergeTreeData & storage; NamesAndTypesList columns_list; diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index c016ec325da..329a8ee4508 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -9,7 +9,7 @@ namespace DB IMergedBlockOutputStream::IMergedBlockOutputStream( const MergeTreeDataPartPtr & data_part) : storage(data_part->storage) - , disk(data_part->disk) + , volume(data_part->volume) , part_path(data_part->getFullRelativePath()) { } @@ -82,7 +82,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( { if (checksums.files.count(removed_file)) { - data_part->disk->remove(data_part->getFullRelativePath() + removed_file); + data_part->volume->getDisk()->remove(data_part->getFullRelativePath() + removed_file); checksums.files.erase(removed_file); } } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index 7b808ef6784..6a06d4b0c75 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -37,7 +37,7 @@ protected: protected: const MergeTreeData & storage; - DiskPtr disk; + VolumePtr volume; String part_path; static Block getBlockAndPermute(const Block & block, const Names & names, const IColumn::Permutation * permutation); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index edc492efdbe..de9d3f6e981 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -109,6 +109,7 @@ namespace ErrorCodes extern const int UNEXPECTED_AST_STRUCTURE; extern const int UNKNOWN_DISK; extern const int NOT_ENOUGH_SPACE; + extern const int ALTER_OF_COLUMN_IS_FORBIDDEN; } @@ -928,7 +929,8 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) if (!MergeTreePartInfo::tryParsePartName(part_name, &part_info, format_version)) return; - auto part = createPart(part_name, part_info, part_disk_ptr, part_name); + auto single_disk_volume = std::make_shared("volume_" + part_name, part_disk_ptr); + auto part = createPart(part_name, part_info, single_disk_volume, part_name); bool broken = false; String part_path = relative_data_path + "/" + part_name; @@ -1473,13 +1475,13 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S { throw Exception( "Trying to ALTER RENAME key " + backQuoteIfNeed(command.column_name) + " column which is a part of key expression", - ErrorCodes::ILLEGAL_COLUMN); + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); } } else if (command.isModifyingData(getInMemoryMetadata())) { if (columns_alter_type_forbidden.count(command.column_name)) - throw Exception("Trying to ALTER key column " + command.column_name, ErrorCodes::ILLEGAL_COLUMN); + throw Exception("ALTER of key column " + command.column_name + " is forbidden", ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); if (columns_alter_type_metadata_only.count(command.column_name)) { @@ -1487,7 +1489,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S { auto it = old_types.find(command.column_name); if (it == old_types.end() || !isMetadataOnlyConversion(it->second, command.data_type.get())) - throw Exception("ALTER of key column " + command.column_name + " must be metadata-only", ErrorCodes::ILLEGAL_COLUMN); + throw Exception("ALTER of key column " + command.column_name + " must be metadata-only", + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); } } } @@ -1550,12 +1553,12 @@ MergeTreeDataPartType MergeTreeData::choosePartType(size_t bytes_uncompressed, s MergeTreeData::MutableDataPartPtr MergeTreeData::createPart(const String & name, MergeTreeDataPartType type, const MergeTreePartInfo & part_info, - const DiskPtr & disk, const String & relative_path) const + const VolumePtr & volume, const String & relative_path) const { if (type == MergeTreeDataPartType::COMPACT) - return std::make_shared(*this, name, part_info, disk, relative_path); + return std::make_shared(*this, name, part_info, volume, relative_path); else if (type == MergeTreeDataPartType::WIDE) - return std::make_shared(*this, name, part_info, disk, relative_path); + return std::make_shared(*this, name, part_info, volume, relative_path); else throw Exception("Unknown type in part " + relative_path, ErrorCodes::UNKNOWN_PART_TYPE); } @@ -1573,18 +1576,18 @@ static MergeTreeDataPartType getPartTypeFromMarkExtension(const String & mrk_ext } MergeTreeData::MutableDataPartPtr MergeTreeData::createPart( - const String & name, const DiskPtr & disk, const String & relative_path) const + const String & name, const VolumePtr & volume, const String & relative_path) const { - return createPart(name, MergeTreePartInfo::fromPartName(name, format_version), disk, relative_path); + return createPart(name, MergeTreePartInfo::fromPartName(name, format_version), volume, relative_path); } MergeTreeData::MutableDataPartPtr MergeTreeData::createPart( const String & name, const MergeTreePartInfo & part_info, - const DiskPtr & disk, const String & relative_path) const + const VolumePtr & volume, const String & relative_path) const { MergeTreeDataPartType type; auto full_path = relative_data_path + relative_path + "/"; - auto mrk_ext = MergeTreeIndexGranularityInfo::getMarksExtensionFromFilesystem(disk, full_path); + auto mrk_ext = MergeTreeIndexGranularityInfo::getMarksExtensionFromFilesystem(volume->getDisk(), full_path); if (mrk_ext) type = getPartTypeFromMarkExtension(*mrk_ext); @@ -1594,7 +1597,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createPart( type = choosePartType(0, 0); } - return createPart(name, type, part_info, disk, relative_path); + return createPart(name, type, part_info, volume, relative_path); } void MergeTreeData::changeSettings( @@ -2312,7 +2315,7 @@ void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy) auto part_it = data_parts_indexes.insert(part_copy).first; modifyPartState(part_it, DataPartState::Committed); - auto disk = original_active_part->disk; + auto disk = original_active_part->volume->getDisk(); String marker_path = original_active_part->getFullRelativePath() + DELETE_ON_DESTROY_MARKER_PATH; try { @@ -2377,7 +2380,7 @@ MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const String & part_na static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part) { - auto disk = part->disk; + auto disk = part->volume->getDisk(); String full_part_path = part->getFullRelativePath(); /// Earlier the list of columns was written incorrectly. Delete it and re-create. @@ -2402,9 +2405,9 @@ static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part) } } -MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const DiskPtr & disk, const String & relative_path) const +MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const VolumePtr & volume, const String & relative_path) const { - MutableDataPartPtr part = createPart(Poco::Path(relative_path).getFileName(), disk, relative_path); + MutableDataPartPtr part = createPart(Poco::Path(relative_path).getFileName(), volume, relative_path); loadPartAndFixMetadataImpl(part); return part; } @@ -2517,7 +2520,7 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & parts.erase(std::remove_if(parts.begin(), parts.end(), [&](auto part_ptr) { - return part_ptr->disk->getName() == disk->getName(); + return part_ptr->volume->getDisk()->getName() == disk->getName(); }), parts.end()); if (parts.empty()) @@ -2568,9 +2571,9 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String parts.erase(std::remove_if(parts.begin(), parts.end(), [&](auto part_ptr) { - for (const auto & disk : volume->disks) + for (const auto & disk : volume->getDisks()) { - if (part_ptr->disk->getName() == disk->getName()) + if (part_ptr->volume->getDisk()->getName() == disk->getName()) { return true; } @@ -2846,7 +2849,8 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const for (const auto & part_names : renamed_parts.old_and_new_names) { LOG_DEBUG(log, "Checking part " << part_names.second); - MutableDataPartPtr part = createPart(part_names.first, name_to_disk[part_names.first], source_dir + part_names.second); + auto single_disk_volume = std::make_shared("volume_" + part_names.first, name_to_disk[part_names.first]); + MutableDataPartPtr part = createPart(part_names.first, single_disk_volume, source_dir + part_names.second); loadPartAndFixMetadataImpl(part); loaded_parts.push_back(part); } @@ -2960,12 +2964,12 @@ bool MergeTreeData::TTLEntry::isPartInDestination(StoragePolicyPtr policy, const { if (destination_type == PartDestinationType::VOLUME) { - for (const auto & disk : policy->getVolumeByName(destination_name)->disks) - if (disk->getName() == part.disk->getName()) + for (const auto & disk : policy->getVolumeByName(destination_name)->getDisks()) + if (disk->getName() == part.volume->getDisk()->getName()) return true; } else if (destination_type == PartDestinationType::DISK) - return policy->getDiskByName(destination_name)->getName() == part.disk->getName(); + return policy->getDiskByName(destination_name)->getName() == part.volume->getDisk()->getName(); return false; } @@ -3179,7 +3183,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( bool does_storage_policy_allow_same_disk = false; for (const DiskPtr & disk : getStoragePolicy()->getDisks()) { - if (disk->getName() == src_part->disk->getName()) + if (disk->getName() == src_part->volume->getDisk()->getName()) { does_storage_policy_allow_same_disk = true; break; @@ -3192,7 +3196,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( String dst_part_name = src_part->getNewName(dst_part_info); String tmp_dst_part_name = tmp_part_prefix + dst_part_name; - auto reservation = reserveSpace(src_part->getBytesOnDisk(), src_part->disk); + auto reservation = reserveSpace(src_part->getBytesOnDisk(), src_part->volume->getDisk()); auto disk = reservation->getDisk(); String src_part_path = src_part->getFullRelativePath(); String dst_part_path = relative_data_path + tmp_dst_part_name; @@ -3204,7 +3208,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( localBackup(disk, src_part_path, dst_part_path); disk->removeIfExists(dst_part_path + "/" + DELETE_ON_DESTROY_MARKER_PATH); - auto dst_data_part = createPart(dst_part_name, dst_part_info, reservation->getDisk(), tmp_dst_part_name); + auto single_disk_volume = std::make_shared(disk->getName(), disk); + auto dst_data_part = createPart(dst_part_name, dst_part_info, single_disk_volume, tmp_dst_part_name); dst_data_part->is_temp = true; @@ -3276,7 +3281,7 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & if (!matcher(part)) continue; - part->disk->createDirectories(shadow_path); + part->volume->getDisk()->createDirectories(shadow_path); String backup_path = shadow_path + (!with_name.empty() @@ -3287,8 +3292,8 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & LOG_DEBUG(log, "Freezing part " << part->name << " snapshot will be placed at " + backup_path); String backup_part_path = backup_path + relative_data_path + part->relative_path; - localBackup(part->disk, part->getFullRelativePath(), backup_part_path); - part->disk->removeIfExists(backup_part_path + "/" + DELETE_ON_DESTROY_MARKER_PATH); + localBackup(part->volume->getDisk(), part->getFullRelativePath(), backup_part_path); + part->volume->getDisk()->removeIfExists(backup_part_path + "/" + DELETE_ON_DESTROY_MARKER_PATH); part->is_frozen.store(true, std::memory_order_relaxed); ++parts_processed; @@ -3409,7 +3414,7 @@ bool MergeTreeData::areBackgroundMovesNeeded() const if (policy->getVolumes().size() > 1) return true; - return policy->getVolumes().size() == 1 && policy->getVolumes()[0]->disks.size() > 1 && !move_ttl_entries.empty(); + return policy->getVolumes().size() == 1 && policy->getVolumes()[0]->getDisks().size() > 1 && !move_ttl_entries.empty(); } bool MergeTreeData::movePartsToSpace(const DataPartsVector & parts, SpacePtr space) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index bd70a25169f..394260096a7 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -194,14 +194,14 @@ public: /// After this method setColumns must be called MutableDataPartPtr createPart(const String & name, MergeTreeDataPartType type, const MergeTreePartInfo & part_info, - const DiskPtr & disk, const String & relative_path) const; + const VolumePtr & volume, const String & relative_path) const; /// After this methods 'loadColumnsChecksumsIndexes' must be called MutableDataPartPtr createPart(const String & name, - const DiskPtr & disk, const String & relative_path) const; + const VolumePtr & volume, const String & relative_path) const; MutableDataPartPtr createPart(const String & name, const MergeTreePartInfo & part_info, - const DiskPtr & disk, const String & relative_path) const; + const VolumePtr & volume, const String & relative_path) const; /// Auxiliary object to add a set of parts into the working set in two steps: /// * First, as PreCommitted parts (the parts are ready, but not yet in the active set). @@ -539,7 +539,7 @@ public: bool hasAnyTTL() const override { return hasRowsTTL() || hasAnyMoveTTL() || hasAnyColumnTTL(); } /// Check that the part is not broken and calculate the checksums for it if they are not present. - MutableDataPartPtr loadPartAndFixMetadata(const DiskPtr & disk, const String & relative_path) const; + MutableDataPartPtr loadPartAndFixMetadata(const VolumePtr & volume, const String & relative_path) const; /** Create local backup (snapshot) for parts with specified prefix. * Backup is created in directory clickhouse_dir/shadow/i/, where i - incremental number, diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 61eb038f6b0..124983d3503 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -610,11 +610,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor all_columns, data.sorting_key_expr, data.skip_indices, data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names); + auto single_disk_volume = std::make_shared("volume_" + future_part.name, disk); MergeTreeData::MutableDataPartPtr new_data_part = data.createPart( future_part.name, future_part.type, future_part.part_info, - disk, + single_disk_volume, TMP_PREFIX + future_part.name); new_data_part->setColumns(all_columns); @@ -752,7 +753,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor case MergeTreeData::MergingParams::Collapsing: merged_transform = std::make_unique( - header, pipes.size(), sort_description, data.merging_params.sign_column, + header, pipes.size(), sort_description, data.merging_params.sign_column, false, merge_block_size, rows_sources_write_buf.get(), blocks_are_granules_size); break; @@ -1028,8 +1029,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor in->setProgressCallback(MergeProgressCallback(merge_entry, watch_prev_elapsed, stage_progress)); } + auto single_disk_volume = std::make_shared("volume_" + future_part.name, space_reservation->getDisk()); auto new_data_part = data.createPart( - future_part.name, future_part.type, future_part.part_info, space_reservation->getDisk(), "tmp_mut_" + future_part.name); + future_part.name, future_part.type, future_part.part_info, single_disk_volume, "tmp_mut_" + future_part.name); new_data_part->is_temp = true; new_data_part->ttl_infos = source_part->ttl_infos; @@ -1039,7 +1041,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor new_data_part->setColumns(getColumnsForNewDataPart(source_part, updated_header, all_columns, for_file_renames)); new_data_part->partition.assign(source_part->partition); - auto disk = new_data_part->disk; + auto disk = new_data_part->volume->getDisk(); String new_part_tmp_path = new_data_part->getFullRelativePath(); /// Note: this is done before creating input streams, because otherwise data.data_parts_mutex @@ -1644,7 +1646,7 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart( MergeTreeData::MutableDataPartPtr new_data_part, bool need_remove_expired_values) { - auto disk = new_data_part->disk; + auto disk = new_data_part->volume->getDisk(); if (need_remove_expired_values) { /// Write a file with ttl infos in json format. @@ -1674,7 +1676,7 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart( new_data_part->minmax_idx = source_part->minmax_idx; new_data_part->modification_time = time(nullptr); new_data_part->setBytesOnDisk( - MergeTreeData::DataPart::calculateTotalSizeOnDisk(new_data_part->disk, new_data_part->getFullRelativePath())); + MergeTreeData::DataPart::calculateTotalSizeOnDisk(new_data_part->volume->getDisk(), new_data_part->getFullRelativePath())); new_data_part->calculateColumnsSizesOnDisk(); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 134b2fc1ef0..32acc266e42 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -20,9 +20,9 @@ namespace ErrorCodes MergeTreeDataPartCompact::MergeTreeDataPartCompact( MergeTreeData & storage_, const String & name_, - const DiskPtr & disk_, + const VolumePtr & volume_, const std::optional & relative_path_) - : IMergeTreeDataPart(storage_, name_, disk_, relative_path_, Type::COMPACT) + : IMergeTreeDataPart(storage_, name_, volume_, relative_path_, Type::COMPACT) { } @@ -30,9 +30,9 @@ MergeTreeDataPartCompact::MergeTreeDataPartCompact( const MergeTreeData & storage_, const String & name_, const MergeTreePartInfo & info_, - const DiskPtr & disk_, + const VolumePtr & volume_, const std::optional & relative_path_) - : IMergeTreeDataPart(storage_, name_, info_, disk_, relative_path_, Type::COMPACT) + : IMergeTreeDataPart(storage_, name_, info_, volume_, relative_path_, Type::COMPACT) { } @@ -68,7 +68,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( { return *getColumnPosition(lhs.name) < *getColumnPosition(rhs.name); }); return std::make_unique( - disk, getFullRelativePath(), storage, ordered_columns_list, indices_to_recalc, + shared_from_this(), ordered_columns_list, indices_to_recalc, index_granularity_info.marks_file_extension, default_codec, writer_settings, computed_index_granularity); } @@ -99,12 +99,12 @@ void MergeTreeDataPartCompact::loadIndexGranularity() throw Exception("MergeTreeDataPartCompact cannot be created with non-adaptive granulary.", ErrorCodes::NOT_IMPLEMENTED); auto marks_file_path = index_granularity_info.getMarksFilePath(full_path + "data"); - if (!disk->exists(marks_file_path)) - throw Exception("Marks file '" + fullPath(disk, marks_file_path) + "' doesn't exist", ErrorCodes::NO_FILE_IN_DATA_PART); + if (!volume->getDisk()->exists(marks_file_path)) + throw Exception("Marks file '" + fullPath(volume->getDisk(), marks_file_path) + "' doesn't exist", ErrorCodes::NO_FILE_IN_DATA_PART); - size_t marks_file_size = disk->getFileSize(marks_file_path); + size_t marks_file_size = volume->getDisk()->getFileSize(marks_file_path); - auto buffer = disk->readFile(marks_file_path, marks_file_size); + auto buffer = volume->getDisk()->readFile(marks_file_path, marks_file_size); while (!buffer->eof()) { /// Skip offsets for columns @@ -146,9 +146,9 @@ void MergeTreeDataPartCompact::checkConsistency(bool require_part_metadata) cons if (require_part_metadata) { if (!checksums.files.count(mrk_file_name)) - throw Exception("No marks file checksum for column in part " + fullPath(disk, path), ErrorCodes::NO_FILE_IN_DATA_PART); + throw Exception("No marks file checksum for column in part " + fullPath(volume->getDisk(), path), ErrorCodes::NO_FILE_IN_DATA_PART); if (!checksums.files.count(DATA_FILE_NAME_WITH_EXTENSION)) - throw Exception("No data file checksum for in part " + fullPath(disk, path), ErrorCodes::NO_FILE_IN_DATA_PART); + throw Exception("No data file checksum for in part " + fullPath(volume->getDisk(), path), ErrorCodes::NO_FILE_IN_DATA_PART); } } else @@ -156,24 +156,24 @@ void MergeTreeDataPartCompact::checkConsistency(bool require_part_metadata) cons { /// count.txt should be present even in non custom-partitioned parts auto file_path = path + "count.txt"; - if (!disk->exists(file_path) || disk->getFileSize(file_path) == 0) - throw Exception("Part " + path + " is broken: " + fullPath(disk, file_path) + " is empty", ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); + if (!volume->getDisk()->exists(file_path) || volume->getDisk()->getFileSize(file_path) == 0) + throw Exception("Part " + path + " is broken: " + fullPath(volume->getDisk(), file_path) + " is empty", ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); } /// Check that marks are nonempty and have the consistent size with columns number. auto mrk_file_path = path + mrk_file_name; - if (disk->exists(mrk_file_name)) + if (volume->getDisk()->exists(mrk_file_name)) { - UInt64 file_size = disk->getFileSize(mrk_file_name); + UInt64 file_size = volume->getDisk()->getFileSize(mrk_file_name); if (!file_size) - throw Exception("Part " + path + " is broken: " + fullPath(disk, mrk_file_name) + " is empty.", + throw Exception("Part " + path + " is broken: " + fullPath(volume->getDisk(), mrk_file_name) + " is empty.", ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); UInt64 expected_file_size = index_granularity_info.getMarkSizeInBytes(columns.size()) * index_granularity.getMarksCount(); if (expected_file_size != file_size) throw Exception( - "Part " + path + " is broken: bad size of marks file '" + fullPath(disk, mrk_file_name) + "': " + std::to_string(file_size) + ", must be: " + std::to_string(expected_file_size), + "Part " + path + " is broken: bad size of marks file '" + fullPath(volume->getDisk(), mrk_file_name) + "': " + std::to_string(file_size) + ", must be: " + std::to_string(expected_file_size), ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index fa98a2c863f..3ce1fd830d5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -26,13 +26,13 @@ public: const MergeTreeData & storage_, const String & name_, const MergeTreePartInfo & info_, - const DiskPtr & disk_, + const VolumePtr & volume_, const std::optional & relative_path_ = {}); MergeTreeDataPartCompact( MergeTreeData & storage_, const String & name_, - const DiskPtr & disk_, + const VolumePtr & volume_, const std::optional & relative_path_ = {}); MergeTreeReaderPtr getReader( diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index dfc57d5d5d0..d96b61b4bea 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -19,9 +19,9 @@ namespace ErrorCodes MergeTreeDataPartWide::MergeTreeDataPartWide( MergeTreeData & storage_, const String & name_, - const DiskPtr & disk_, + const VolumePtr & volume_, const std::optional & relative_path_) - : IMergeTreeDataPart(storage_, name_, disk_, relative_path_, Type::WIDE) + : IMergeTreeDataPart(storage_, name_, volume_, relative_path_, Type::WIDE) { } @@ -29,9 +29,9 @@ MergeTreeDataPartWide::MergeTreeDataPartWide( const MergeTreeData & storage_, const String & name_, const MergeTreePartInfo & info_, - const DiskPtr & disk_, + const VolumePtr & volume_, const std::optional & relative_path_) - : IMergeTreeDataPart(storage_, name_, info_, disk_, relative_path_, Type::WIDE) + : IMergeTreeDataPart(storage_, name_, info_, volume_, relative_path_, Type::WIDE) { } @@ -59,7 +59,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter( const MergeTreeIndexGranularity & computed_index_granularity) const { return std::make_unique( - disk, getFullRelativePath(), storage, columns_list, indices_to_recalc, + shared_from_this(), columns_list, indices_to_recalc, index_granularity_info.marks_file_extension, default_codec, writer_settings, computed_index_granularity); } @@ -99,7 +99,7 @@ ColumnSize MergeTreeDataPartWide::getColumnSizeImpl( void MergeTreeDataPartWide::loadIndexGranularity() { String full_path = getFullRelativePath(); - index_granularity_info.changeGranularityIfRequired(disk, full_path); + index_granularity_info.changeGranularityIfRequired(volume->getDisk(), full_path); if (columns.empty()) @@ -107,10 +107,10 @@ void MergeTreeDataPartWide::loadIndexGranularity() /// We can use any column, it doesn't matter std::string marks_file_path = index_granularity_info.getMarksFilePath(full_path + getFileNameForColumn(columns.front())); - if (!disk->exists(marks_file_path)) - throw Exception("Marks file '" + fullPath(disk, marks_file_path) + "' doesn't exist", ErrorCodes::NO_FILE_IN_DATA_PART); + if (!volume->getDisk()->exists(marks_file_path)) + throw Exception("Marks file '" + fullPath(volume->getDisk(), marks_file_path) + "' doesn't exist", ErrorCodes::NO_FILE_IN_DATA_PART); - size_t marks_file_size = disk->getFileSize(marks_file_path); + size_t marks_file_size = volume->getDisk()->getFileSize(marks_file_path); if (!index_granularity_info.is_adaptive) { @@ -119,7 +119,7 @@ void MergeTreeDataPartWide::loadIndexGranularity() } else { - auto buffer = disk->readFile(marks_file_path, marks_file_size); + auto buffer = volume->getDisk()->readFile(marks_file_path, marks_file_size); while (!buffer->eof()) { buffer->seek(sizeof(size_t) * 2, SEEK_CUR); /// skip offset_in_compressed file and offset_in_decompressed_block @@ -129,7 +129,7 @@ void MergeTreeDataPartWide::loadIndexGranularity() } if (index_granularity.getMarksCount() * index_granularity_info.getMarkSizeInBytes() != marks_file_size) - throw Exception("Cannot read all marks from file " + fullPath(disk, marks_file_path), ErrorCodes::CANNOT_READ_ALL_DATA); + throw Exception("Cannot read all marks from file " + fullPath(volume->getDisk(), marks_file_path), ErrorCodes::CANNOT_READ_ALL_DATA); } index_granularity.setInitialized(); @@ -158,10 +158,10 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const String mrk_file_name = file_name + index_granularity_info.marks_file_extension; String bin_file_name = file_name + ".bin"; if (!checksums.files.count(mrk_file_name)) - throw Exception("No " + mrk_file_name + " file checksum for column " + name_type.name + " in part " + fullPath(disk, path), + throw Exception("No " + mrk_file_name + " file checksum for column " + name_type.name + " in part " + fullPath(volume->getDisk(), path), ErrorCodes::NO_FILE_IN_DATA_PART); if (!checksums.files.count(bin_file_name)) - throw Exception("No " + bin_file_name + " file checksum for column " + name_type.name + " in part " + fullPath(disk, path), + throw Exception("No " + bin_file_name + " file checksum for column " + name_type.name + " in part " + fullPath(volume->getDisk(), path), ErrorCodes::NO_FILE_IN_DATA_PART); }, stream_path); } @@ -179,12 +179,12 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const auto file_path = path + IDataType::getFileNameForStream(name_type.name, substream_path) + index_granularity_info.marks_file_extension; /// Missing file is Ok for case when new column was added. - if (disk->exists(file_path)) + if (volume->getDisk()->exists(file_path)) { - UInt64 file_size = disk->getFileSize(file_path); + UInt64 file_size = volume->getDisk()->getFileSize(file_path); if (!file_size) - throw Exception("Part " + path + " is broken: " + fullPath(disk, file_path) + " is empty.", + throw Exception("Part " + path + " is broken: " + fullPath(volume->getDisk(), file_path) + " is empty.", ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); if (!marks_size) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index d66e2108fc1..ba9e0765510 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -19,13 +19,13 @@ public: const MergeTreeData & storage_, const String & name_, const MergeTreePartInfo & info_, - const DiskPtr & disk, + const VolumePtr & volume, const std::optional & relative_path = {}); MergeTreeDataPartWide( MergeTreeData & storage_, const String & name_, - const DiskPtr & disk, + const VolumePtr & volume, const std::optional & relative_path = {}); MergeTreeReaderPtr getReader( diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 2f708ac6954..1a7a757c149 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -6,26 +6,23 @@ namespace DB MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( - DiskPtr disk_, - const String & part_path_, - const MergeTreeData & storage_, + const MergeTreeData::DataPartPtr & data_part_, const NamesAndTypesList & columns_list_, const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) -: IMergeTreeDataPartWriter(disk_, part_path_, - storage_, columns_list_, - indices_to_recalc_, marks_file_extension_, - default_codec_, settings_, index_granularity_) + : IMergeTreeDataPartWriter(data_part_, columns_list_, + indices_to_recalc_, marks_file_extension_, + default_codec_, settings_, index_granularity_) { using DataPart = MergeTreeDataPartCompact; String data_file_name = DataPart::DATA_FILE_NAME; stream = std::make_unique( data_file_name, - disk_, + data_part->volume->getDisk(), part_path + data_file_name, DataPart::DATA_FILE_EXTENSION, part_path + data_file_name, marks_file_extension, default_codec, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 45d72d90b1e..07caba94712 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -8,9 +8,7 @@ class MergeTreeDataPartWriterCompact : public IMergeTreeDataPartWriter { public: MergeTreeDataPartWriterCompact( - DiskPtr disk, - const String & part_path, - const MergeTreeData & storage, + const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns_list, const std::vector & indices_to_recalc, const String & marks_file_extension, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index e5eececacfb..4cdf57a4700 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -13,18 +13,16 @@ namespace } MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( - DiskPtr disk_, - const String & part_path_, - const MergeTreeData & storage_, + const MergeTreeData::DataPartPtr & data_part_, const NamesAndTypesList & columns_list_, const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : IMergeTreeDataPartWriter(disk_, part_path_, - storage_, columns_list_, indices_to_recalc_, - marks_file_extension_, default_codec_, settings_, index_granularity_) + : IMergeTreeDataPartWriter(data_part_, columns_list_, + indices_to_recalc_, marks_file_extension_, + default_codec_, settings_, index_granularity_) { const auto & columns = storage.getColumns(); for (const auto & it : columns_list) @@ -46,7 +44,7 @@ void MergeTreeDataPartWriterWide::addStreams( column_streams[stream_name] = std::make_unique( stream_name, - disk, + data_part->volume->getDisk(), part_path + stream_name, DATA_FILE_EXTENSION, part_path + stream_name, marks_file_extension, effective_codec, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 4e4f4806d53..acd7f749d00 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -11,9 +11,7 @@ public: using ColumnToSize = std::map; MergeTreeDataPartWriterWide( - DiskPtr disk, - const String & part_path, - const MergeTreeData & storage, + const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns_list, const std::vector & indices_to_recalc, const String & marks_file_extension, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index d52bc0e45bf..662fb067e48 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -41,22 +41,25 @@ namespace std #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 namespace ProfileEvents { @@ -622,6 +625,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( res = spreadMarkRangesAmongStreamsFinal( std::move(parts_with_ranges), + num_streams, column_names_to_read, max_block_size, settings.use_uncompressed_cache, @@ -1042,6 +1046,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, + size_t num_streams, const Names & column_names, UInt64 max_block_size, bool use_uncompressed_cache, @@ -1104,71 +1109,122 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( for (size_t i = 0; i < sort_columns_size; ++i) sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); - /// Converts pipes to BlockInputsStreams. - /// It is temporary, till not all merging streams are implemented as processors. - auto streams_to_merge = [&pipes]() + auto get_merging_processor = [&]() -> MergingTransformPtr { - size_t num_streams = pipes.size(); - - BlockInputStreams streams; - streams.reserve(num_streams); - - for (size_t i = 0; i < num_streams; ++i) - streams.emplace_back(std::make_shared(std::move(pipes[i]))); - - pipes.clear(); - return streams; - }; - - BlockInputStreamPtr merged; - ProcessorPtr merged_processor; - switch (data.merging_params.mode) - { - case MergeTreeData::MergingParams::Ordinary: + switch (data.merging_params.mode) { - merged_processor = std::make_shared(header, pipes.size(), - sort_description, max_block_size); - break; + case MergeTreeData::MergingParams::Ordinary: + { + return std::make_shared(header, pipes.size(), + sort_description, max_block_size); + } + + case MergeTreeData::MergingParams::Collapsing: + return std::make_shared(header, pipes.size(), + sort_description, data.merging_params.sign_column, true, max_block_size); + + case MergeTreeData::MergingParams::Summing: + return std::make_shared(header, pipes.size(), + sort_description, data.merging_params.columns_to_sum, max_block_size); + + case MergeTreeData::MergingParams::Aggregating: + return std::make_shared(header, pipes.size(), + sort_description, max_block_size); + + case MergeTreeData::MergingParams::Replacing: + return std::make_shared(header, pipes.size(), + sort_description, data.merging_params.version_column, max_block_size); + + case MergeTreeData::MergingParams::VersionedCollapsing: + return std::make_shared(header, pipes.size(), + sort_description, data.merging_params.sign_column, max_block_size); + + case MergeTreeData::MergingParams::Graphite: + throw Exception("GraphiteMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR); } - case MergeTreeData::MergingParams::Collapsing: - merged = std::make_shared( - streams_to_merge(), sort_description, data.merging_params.sign_column); - break; + __builtin_unreachable(); + }; - case MergeTreeData::MergingParams::Summing: - merged_processor = std::make_shared(header, pipes.size(), - sort_description, data.merging_params.columns_to_sum, max_block_size); - break; + if (num_streams > settings.max_final_threads) + num_streams = settings.max_final_threads; - case MergeTreeData::MergingParams::Aggregating: - merged_processor = std::make_shared(header, pipes.size(), - sort_description, max_block_size); - break; - - case MergeTreeData::MergingParams::Replacing: /// TODO Make ReplacingFinalBlockInputStream - merged_processor = std::make_shared(header, pipes.size(), - sort_description, data.merging_params.version_column, max_block_size); - break; - - case MergeTreeData::MergingParams::VersionedCollapsing: /// TODO Make VersionedCollapsingFinalBlockInputStream - merged_processor = std::make_shared(header, pipes.size(), - sort_description, data.merging_params.sign_column, max_block_size); - break; - - case MergeTreeData::MergingParams::Graphite: - throw Exception("GraphiteMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR); - } - - if (merged_processor) + if (num_streams <= 1 || sort_description.empty() || query_info.force_tree_shaped_pipeline) { - Pipe pipe(std::move(pipes), std::move(merged_processor)); + + Pipe pipe(std::move(pipes), get_merging_processor()); pipes = Pipes(); pipes.emplace_back(std::move(pipe)); + + return pipes; } - if (merged) - pipes.emplace_back(std::make_shared(merged)); + ColumnNumbers key_columns; + key_columns.reserve(sort_description.size()); + + for (auto & desc : sort_description) + { + if (!desc.column_name.empty()) + key_columns.push_back(header.getPositionByName(desc.column_name)); + else + key_columns.emplace_back(desc.column_number); + } + + Processors selectors; + Processors copiers; + selectors.reserve(pipes.size()); + + for (auto & pipe : pipes) + { + auto selector = std::make_shared(pipe.getHeader(), num_streams, key_columns); + auto copier = std::make_shared(pipe.getHeader(), num_streams); + connect(pipe.getPort(), selector->getInputPort()); + connect(selector->getOutputPort(), copier->getInputPort()); + selectors.emplace_back(std::move(selector)); + copiers.emplace_back(std::move(copier)); + } + + Processors merges; + std::vector input_ports; + merges.reserve(num_streams); + input_ports.reserve(num_streams); + + for (size_t i = 0; i < num_streams; ++i) + { + auto merge = get_merging_processor(); + merge->setSelectorPosition(i); + input_ports.emplace_back(merge->getInputs().begin()); + merges.emplace_back(std::move(merge)); + } + + /// Connect outputs of i-th splitter with i-th input port of every merge. + for (auto & resize : copiers) + { + size_t input_num = 0; + for (auto & output : resize->getOutputs()) + { + connect(output, *input_ports[input_num]); + ++input_ports[input_num]; + ++input_num; + } + } + + Processors processors; + for (auto & pipe : pipes) + { + auto pipe_processors = std::move(pipe).detachProcessors(); + processors.insert(processors.end(), pipe_processors.begin(), pipe_processors.end()); + } + + pipes.clear(); + pipes.reserve(num_streams); + for (auto & merge : merges) + pipes.emplace_back(&merge->getOutputs().front()); + + pipes.front().addProcessors(processors); + pipes.front().addProcessors(selectors); + pipes.front().addProcessors(copiers); + pipes.front().addProcessors(merges); return pipes; } @@ -1324,7 +1380,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( const MarkRanges & ranges, const Settings & settings) const { - if (!part->disk->exists(part->getFullRelativePath() + index->getFileName() + ".idx")) + if (!part->volume->getDisk()->exists(part->getFullRelativePath() + index->getFileName() + ".idx")) { LOG_DEBUG(log, "File for index " << backQuote(index->name) << " does not exist. Skipping it."); return ranges; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 09b7958c59f..92fa98fd914 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -73,6 +73,7 @@ private: Pipes spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, + size_t num_streams, const Names & column_names, UInt64 max_block_size, bool use_uncompressed_cache, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 1814e40e10e..a78e2e5ae32 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -231,12 +232,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa NamesAndTypesList columns = data.getColumns().getAllPhysical().filter(block.getNames()); ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr)); + VolumePtr volume = data.getStoragePolicy()->getVolume(0); auto new_data_part = data.createPart( part_name, data.choosePartType(expected_size, block.rows()), new_part_info, - reservation->getDisk(), + createVolumeFromReservation(reservation, volume), TMP_PREFIX + part_name); new_data_part->setColumns(columns); @@ -247,13 +249,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa /// The name could be non-unique in case of stale files from previous runs. String full_path = new_data_part->getFullRelativePath(); - if (new_data_part->disk->exists(full_path)) + if (new_data_part->volume->getDisk()->exists(full_path)) { - LOG_WARNING(log, "Removing old temporary directory " + fullPath(new_data_part->disk, full_path)); - new_data_part->disk->removeRecursive(full_path); + LOG_WARNING(log, "Removing old temporary directory " + fullPath(new_data_part->volume->getDisk(), full_path)); + new_data_part->volume->getDisk()->removeRecursive(full_path); } - new_data_part->disk->createDirectories(full_path); + new_data_part->volume->getDisk()->createDirectories(full_path); /// If we need to calculate some columns to sort. if (data.hasSortingKey() || data.hasSkipIndices()) diff --git a/src/Storages/MergeTree/MergeTreeIndexReader.cpp b/src/Storages/MergeTree/MergeTreeIndexReader.cpp index 55eeb41ceb5..d8f13e49b31 100644 --- a/src/Storages/MergeTree/MergeTreeIndexReader.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexReader.cpp @@ -7,7 +7,7 @@ namespace DB MergeTreeIndexReader::MergeTreeIndexReader( MergeTreeIndexPtr index_, MergeTreeData::DataPartPtr part_, size_t marks_count_, const MarkRanges & all_mark_ranges_) : index(index_), stream( - part_->disk, + part_->volume->getDisk(), part_->getFullRelativePath() + index->getFileName(), ".idx", marks_count_, all_mark_ranges_, MergeTreeReaderSettings{}, nullptr, nullptr, diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index c6a00b46345..1609e1e517e 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -108,7 +108,7 @@ bool MergeTreePartsMover::selectPartsForMove( /// Do not check last volume for (size_t i = 0; i != volumes.size() - 1; ++i) { - for (const auto & disk : volumes[i]->disks) + for (const auto & disk : volumes[i]->getDisks()) { UInt64 required_maximum_available_space = disk->getTotalSpace() * policy->getMoveFactor(); UInt64 unreserved_space = disk->getUnreservedSpace(); @@ -129,7 +129,7 @@ bool MergeTreePartsMover::selectPartsForMove( continue; auto ttl_entry = part->storage.selectTTLEntryForTTLInfos(part->ttl_infos, time_of_move); - auto to_insert = need_to_move.find(part->disk); + auto to_insert = need_to_move.find(part->volume->getDisk()); ReservationPtr reservation; if (ttl_entry) { @@ -196,8 +196,9 @@ MergeTreeData::DataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEnt LOG_TRACE(log, "Cloning part " << moving_part.part->name); moving_part.part->makeCloneOnDiskDetached(moving_part.reserved_space); + auto single_disk_volume = std::make_shared("volume_" + moving_part.part->name, moving_part.reserved_space->getDisk()); MergeTreeData::MutableDataPartPtr cloned_part = - data->createPart(moving_part.part->name, moving_part.reserved_space->getDisk(), "detached/" + moving_part.part->name); + data->createPart(moving_part.part->name, single_disk_volume, "detached/" + moving_part.part->name); LOG_TRACE(log, "Part " << moving_part.part->name << " was cloned to " << cloned_part->getFullPath()); cloned_part->loadColumnsChecksumsIndexes(true, true); diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index a895149e12e..c4a05a8bfac 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -28,7 +28,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( uncompressed_cache_, mark_cache_, std::move(mark_ranges_), std::move(settings_), std::move(avg_value_size_hints_)) , marks_loader( - data_part->disk, + data_part->volume->getDisk(), mark_cache, data_part->index_granularity_info.getMarksFilePath(data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME), data_part->getMarksCount(), data_part->index_granularity_info, @@ -40,10 +40,10 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( if (uncompressed_cache) { auto buffer = std::make_unique( - fullPath(data_part->disk, full_data_path), + fullPath(data_part->volume->getDisk(), full_data_path), [this, full_data_path, buffer_size]() { - return data_part->disk->readFile( + return data_part->volume->getDisk()->readFile( full_data_path, buffer_size, 0, @@ -62,7 +62,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( { auto buffer = std::make_unique( - data_part->disk->readFile(full_data_path, buffer_size, 0, settings.min_bytes_to_use_direct_io, 0)); + data_part->volume->getDisk()->readFile(full_data_path, buffer_size, 0, settings.min_bytes_to_use_direct_io, 0)); if (profile_callback_) buffer->setProfileCallback(profile_callback_, clock_type_); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 1a03acb5758..34bf095e57e 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -176,7 +176,7 @@ void MergeTreeReaderWide::addStreams(const String & name, const IDataType & type return; streams.emplace(stream_name, std::make_unique( - data_part->disk, data_part->getFullRelativePath() + stream_name, DATA_FILE_EXTENSION, + data_part->volume->getDisk(), data_part->getFullRelativePath() + stream_name, DATA_FILE_EXTENSION, data_part->getMarksCount(), all_mark_ranges, settings, mark_cache, uncompressed_cache, data_part->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION), &data_part->index_granularity_info, diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 2b482ac7c29..e0760e87d00 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -49,7 +49,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( } } - disk->createDirectories(part_path); + volume->getDisk()->createDirectories(part_path); writer = data_part->getWriter(columns_list, skip_indices, default_codec, writer_settings); writer->initPrimaryIndex(); @@ -99,14 +99,14 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part)) { - new_part->partition.store(storage, disk, part_path, checksums); + new_part->partition.store(storage, volume->getDisk(), part_path, checksums); if (new_part->minmax_idx.initialized) - new_part->minmax_idx.store(storage, disk, part_path, checksums); + new_part->minmax_idx.store(storage, volume->getDisk(), part_path, checksums); else if (rows_count) throw Exception("MinMax index was not initialized for new non-empty part " + new_part->name + ". It is a bug.", ErrorCodes::LOGICAL_ERROR); - auto count_out = disk->writeFile(part_path + "count.txt", 4096); + auto count_out = volume->getDisk()->writeFile(part_path + "count.txt", 4096); HashingWriteBuffer count_out_hashing(*count_out); writeIntText(rows_count, count_out_hashing); count_out_hashing.next(); @@ -117,7 +117,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( if (!new_part->ttl_infos.empty()) { /// Write a file with ttl infos in json format. - auto out = disk->writeFile(part_path + "ttl.txt", 4096); + auto out = volume->getDisk()->writeFile(part_path + "ttl.txt", 4096); HashingWriteBuffer out_hashing(*out); new_part->ttl_infos.write(out_hashing); checksums.files["ttl.txt"].file_size = out_hashing.count(); @@ -128,13 +128,13 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( { /// Write a file with a description of columns. - auto out = disk->writeFile(part_path + "columns.txt", 4096); + auto out = volume->getDisk()->writeFile(part_path + "columns.txt", 4096); part_columns.writeText(*out); } { /// Write file with checksums. - auto out = disk->writeFile(part_path + "checksums.txt", 4096); + auto out = volume->getDisk()->writeFile(part_path + "checksums.txt", 4096); checksums.write(*out); } diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index c12ed12c2e0..58ff2af9466 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -167,7 +167,7 @@ IMergeTreeDataPart::Checksums checkDataPart( std::function is_cancelled) { return checkDataPart( - data_part->disk, + data_part->volume->getDisk(), data_part->getFullRelativePath(), data_part->getColumns(), data_part->getType(), diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f8202cbcac3..eb75f53ef9c 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -579,7 +579,7 @@ void StorageDistributed::startup() if (!volume) return; - for (const DiskPtr & disk : volume->disks) + for (const DiskPtr & disk : volume->getDisks()) createDirectoryMonitors(disk->getPath()); for (const String & path : getDataPaths()) @@ -607,7 +607,7 @@ Strings StorageDistributed::getDataPaths() const if (relative_data_path.empty()) return paths; - for (const DiskPtr & disk : volume->disks) + for (const DiskPtr & disk : volume->getDisks()) paths.push_back(disk->getPath() + relative_data_path); return paths; @@ -811,7 +811,7 @@ void StorageDistributed::rename(const String & new_path_to_table_data, const Sto } void StorageDistributed::renameOnDisk(const String & new_path_to_table_data) { - for (const DiskPtr & disk : volume->disks) + for (const DiskPtr & disk : volume->getDisks()) { const String path(disk->getPath()); auto new_path = path + new_path_to_table_data; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 91add9a8104..e22e81d5041 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -94,16 +94,16 @@ void StorageMergeTree::startup() /// NOTE background task will also do the above cleanups periodically. time_after_previous_cleanup.restart(); - auto & pool = global_context.getBackgroundPool(); - - merging_mutating_task_handle = pool.createTask([this] { return mergeMutateTask(); }); + auto & merge_pool = global_context.getBackgroundPool(); + merging_mutating_task_handle = merge_pool.createTask([this] { return mergeMutateTask(); }); /// Ensure that thread started only after assignment to 'merging_mutating_task_handle' is done. - pool.startTask(merging_mutating_task_handle); + merge_pool.startTask(merging_mutating_task_handle); if (areBackgroundMovesNeeded()) { - moving_task_handle = pool.createTask([this] { return movePartsTask(); }); - pool.startTask(moving_task_handle); + auto & move_pool = global_context.getBackgroundMovePool(); + moving_task_handle = move_pool.createTask([this] { return movePartsTask(); }); + move_pool.startTask(moving_task_handle); } } @@ -289,7 +289,7 @@ public: /// if we mutate part, than we should reserve space on the same disk, because mutations possible can create hardlinks if (is_mutation) - reserved_space = storage.tryReserveSpace(total_size, future_part_.parts[0]->disk); + reserved_space = storage.tryReserveSpace(total_size, future_part_.parts[0]->volume); else { IMergeTreeDataPart::TTLInfos ttl_infos; @@ -297,7 +297,7 @@ public: for (auto & part_ptr : future_part_.parts) { ttl_infos.update(part_ptr->ttl_infos); - max_volume_index = std::max(max_volume_index, storage.getStoragePolicy()->getVolumeIndexByDisk(part_ptr->disk)); + max_volume_index = std::max(max_volume_index, storage.getStoragePolicy()->getVolumeIndexByDisk(part_ptr->volume->getDisk())); } reserved_space = storage.tryReserveSpacePreferringTTLRules(total_size, ttl_infos, time(nullptr), max_volume_index); @@ -1250,7 +1250,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, const Context & c for (auto & part : data_parts) { - auto disk = part->disk; + auto disk = part->volume->getDisk(); String part_path = part->getFullRelativePath(); /// If the checksums file is not present, calculate the checksums and write them to disk. String checksums_path = part_path + "checksums.txt"; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1317b05d9fe..e6fc32ecbf9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1050,7 +1050,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) for (auto & part_ptr : parts) { ttl_infos.update(part_ptr->ttl_infos); - max_volume_index = std::max(max_volume_index, getStoragePolicy()->getVolumeIndexByDisk(part_ptr->disk)); + max_volume_index = std::max(max_volume_index, getStoragePolicy()->getVolumeIndexByDisk(part_ptr->volume->getDisk())); } ReservationPtr reserved_space = reserveSpacePreferringTTLRules(estimated_space_for_merge, ttl_infos, time(nullptr), max_volume_index); @@ -1189,7 +1189,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM /// Once we mutate part, we must reserve space on the same disk, because mutations can possibly create hardlinks. /// Can throw an exception. - ReservationPtr reserved_space = reserveSpace(estimated_space_for_result, source_part->disk); + ReservationPtr reserved_space = reserveSpace(estimated_space_for_result, source_part->volume); auto table_lock = lockStructureForShare( false, RWLockImpl::NO_QUERY, storage_settings_ptr->lock_acquire_timeout_for_background_operations); diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 2dfbf415100..2418594899e 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -111,7 +111,7 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns_, const Sto columns_[i++]->insert(info.database); columns_[i++]->insert(info.table); columns_[i++]->insert(info.engine); - columns_[i++]->insert(part->disk->getName()); + columns_[i++]->insert(part->volume->getDisk()->getName()); columns_[i++]->insert(part->getFullPath()); if (has_state_column) diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index baa608727aa..b8acdc5f995 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -138,7 +138,7 @@ void StorageSystemPartsColumns::processNextStorage(MutableColumns & columns_, co columns_[j++]->insert(info.database); columns_[j++]->insert(info.table); columns_[j++]->insert(info.engine); - columns_[j++]->insert(part->disk->getName()); + columns_[j++]->insert(part->volume->getDisk()->getName()); columns_[j++]->insert(part->getFullPath()); columns_[j++]->insert(column.name); diff --git a/src/Storages/System/StorageSystemStoragePolicies.cpp b/src/Storages/System/StorageSystemStoragePolicies.cpp index 69962d42d18..81b6ddd465a 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.cpp +++ b/src/Storages/System/StorageSystemStoragePolicies.cpp @@ -55,8 +55,8 @@ Pipes StorageSystemStoragePolicies::read( col_volume_name->insert(volumes[i]->getName()); col_priority->insert(i + 1); Array disks; - disks.reserve(volumes[i]->disks.size()); - for (const auto & disk_ptr : volumes[i]->disks) + disks.reserve(volumes[i]->getDisks().size()); + for (const auto & disk_ptr : volumes[i]->getDisks()) disks.push_back(disk_ptr->getName()); col_disks->insert(disks); col_max_part_size->insert(volumes[i]->max_data_part_size); diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/dictionary.py b/tests/integration/test_dictionaries_all_layouts_and_sources/dictionary.py index 58ccbc32c87..178cdcf44ef 100644 --- a/tests/integration/test_dictionaries_all_layouts_and_sources/dictionary.py +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/dictionary.py @@ -13,6 +13,7 @@ class Layout(object): 'complex_key_cache': '128', 'range_hashed': '', 'direct': '', + 'complex_key_direct': '' } def __init__(self, name): @@ -307,7 +308,7 @@ class Dictionary(object): def generate_config(self): with open(self.config_path, 'w') as result: - if self.structure.layout.get_str() != '': + if 'direct' not in self.structure.layout.get_str(): result.write(''' diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py index cc899ffd0cc..2debb8f11dd 100644 --- a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py @@ -112,7 +112,8 @@ LAYOUTS = [ Layout("complex_key_hashed"), Layout("complex_key_cache"), Layout("range_hashed"), - Layout("direct") + Layout("direct"), + Layout("complex_key_direct") ] SOURCES = [ diff --git a/tests/integration/test_insert_distributed_load_balancing/__init__.py b/tests/integration/test_insert_distributed_load_balancing/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_insert_distributed_load_balancing/configs/remote_servers.xml b/tests/integration/test_insert_distributed_load_balancing/configs/remote_servers.xml new file mode 100644 index 00000000000..61bc5af1f7d --- /dev/null +++ b/tests/integration/test_insert_distributed_load_balancing/configs/remote_servers.xml @@ -0,0 +1,18 @@ + + + + + true + + n2 + 9000 + + + n1 + 9000 + + + + + + diff --git a/tests/integration/test_insert_distributed_load_balancing/test.py b/tests/integration/test_insert_distributed_load_balancing/test.py new file mode 100644 index 00000000000..99d74ddc8df --- /dev/null +++ b/tests/integration/test_insert_distributed_load_balancing/test.py @@ -0,0 +1,67 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name +# pylint: disable=line-too-long + +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +n1 = cluster.add_instance('n1', main_configs=['configs/remote_servers.xml']) +n2 = cluster.add_instance('n2', main_configs=['configs/remote_servers.xml']) + +@pytest.fixture(scope='module', autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def create_tables(**dist_settings): + n1.query('DROP TABLE IF EXISTS data') + n2.query('DROP TABLE IF EXISTS data') + n1.query('DROP TABLE IF EXISTS dist') + + n1.query('CREATE TABLE data (key Int) Engine=Memory()') + n2.query('CREATE TABLE data (key Int) Engine=Memory()') + n1.query(""" + CREATE TABLE dist AS data + Engine=Distributed( + integration_test_cluster, + currentDatabase(), + data, + rand() + ) + """, settings=dist_settings) + +def insert_data(**settings): + create_tables(**settings) + n1.query('INSERT INTO dist SELECT * FROM numbers(10)', settings=settings) + n1.query('SYSTEM FLUSH DISTRIBUTED dist') + +def test_prefer_localhost_replica_1(): + insert_data() + assert int(n1.query('SELECT count() FROM data')) == 10 + assert int(n2.query('SELECT count() FROM data')) == 0 + +def test_prefer_localhost_replica_1_load_balancing_in_order(): + insert_data(load_balancing='in_order') + assert int(n1.query('SELECT count() FROM data')) == 10 + assert int(n2.query('SELECT count() FROM data')) == 0 + +def test_prefer_localhost_replica_0_load_balancing_nearest_hostname(): + insert_data(load_balancing='nearest_hostname', prefer_localhost_replica=0) + assert int(n1.query('SELECT count() FROM data')) == 10 + assert int(n2.query('SELECT count() FROM data')) == 0 + +def test_prefer_localhost_replica_0_load_balancing_in_order(): + insert_data(load_balancing='in_order', prefer_localhost_replica=0) + assert int(n1.query('SELECT count() FROM data')) == 0 + assert int(n2.query('SELECT count() FROM data')) == 10 + +def test_prefer_localhost_replica_0_load_balancing_in_order_sync(): + insert_data(load_balancing='in_order', prefer_localhost_replica=0, insert_distributed_sync=1) + assert int(n1.query('SELECT count() FROM data')) == 0 + assert int(n2.query('SELECT count() FROM data')) == 10 diff --git a/tests/performance/parallel_final.xml b/tests/performance/parallel_final.xml new file mode 100644 index 00000000000..bd6a921fc68 --- /dev/null +++ b/tests/performance/parallel_final.xml @@ -0,0 +1,52 @@ + + + + 1024 + 16 + + + + + collapsing + + collapsing_final_16p_ord + collapsing_final_16p_rnd + collapsing_final_16p_int_keys_ord + collapsing_final_16p_int_keys_rnd + collapsing_final_16p_str_keys_ord + collapsing_final_16p_str_keys_rnd + collapsing_final_1024p_ord + collapsing_final_1024p_rnd + + + + + create table collapsing_final_16p_ord (key1 UInt32, key2 String, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1, key2) partition by intDiv(key1, 8192 * 64) + create table collapsing_final_16p_rnd (key1 UInt32, key2 String, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1, key2) partition by key1 % 16 + create table collapsing_final_16p_int_keys_ord (key1 UInt32, key2 UInt32, key3 UInt32, key4 UInt32, key5 UInt32, key6 UInt32, key7 UInt32, key8 UInt32, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1, key2, key3, key4, key5, key6, key7, key8) partition by intDiv(key1, 8192 * 64) + create table collapsing_final_16p_int_keys_rnd (key1 UInt32, key2 UInt32, key3 UInt32, key4 UInt32, key5 UInt32, key6 UInt32, key7 UInt32, key8 UInt32, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1, key2, key3, key4, key5, key6, key7, key8) partition by key1 % 16 + create table collapsing_final_16p_str_keys_ord (key1 UInt32, key2 String, key3 String, key4 String, key5 String, key6 String, key7 String, key8 String, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1, key2, key3, key4, key5, key6, key7, key8) partition by intDiv(key1, 8192 * 64) + create table collapsing_final_16p_str_keys_rnd (key1 UInt32, key2 String, key3 String, key4 String, key5 String, key6 String, key7 String, key8 String, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1, key2, key3, key4, key5, key6, key7, key8) partition by key1 % 16 + create table collapsing_final_1024p_ord (key1 UInt32, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1) partition by intDiv(key1, 8192 * 2) + create table collapsing_final_1024p_rnd (key1 UInt32, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1) partition by key1 % 1024 + + + insert into collapsing_final_16p_ord select number, number, 1, number from numbers_mt(8388608) + insert into collapsing_final_16p_rnd select sipHash64(number), number, 1, number from numbers_mt(8388608) + insert into collapsing_final_16p_int_keys_ord select number, number, number, number, number, number, number, number, 1, number from numbers_mt(8388608) + insert into collapsing_final_16p_int_keys_rnd select sipHash64(number), number, number, number, number, number, number, number, 1, number from numbers_mt(8388608) + insert into collapsing_final_16p_str_keys_ord select number, number, number, number, number, number, number, number, 1, number from numbers_mt(8388608) + insert into collapsing_final_16p_str_keys_rnd select sipHash64(number), number, number, number, number, number, number, number, 1, number from numbers_mt(8388608) + + + insert into collapsing_final_1024p_ord select number, 1, number from numbers_mt(16777216) + insert into collapsing_final_1024p_rnd select number, 1, number from numbers_mt(16777216) + + optimize table {collapsing} final + + SELECT count() FROM {collapsing} final + SELECT sum(s) FROM {collapsing} final group by key1 limit 10 + SELECT sum(s) FROM {collapsing} final group by key1 % 8192 limit 10 + + DROP TABLE IF EXISTS {collapsing} + diff --git a/tests/performance/random_fixed_string.xml b/tests/performance/random_fixed_string.xml new file mode 100644 index 00000000000..3883acf5f62 --- /dev/null +++ b/tests/performance/random_fixed_string.xml @@ -0,0 +1,9 @@ + + + + + SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomFixedString(10)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomFixedString(100)) + SELECT count() FROM zeros(100000) WHERE NOT ignore(randomFixedString(1000)) + SELECT count() FROM zeros(10000) WHERE NOT ignore(randomFixedString(10000)) + diff --git a/tests/performance/random_string_utf8.xml b/tests/performance/random_string_utf8.xml new file mode 100644 index 00000000000..60e3b69e79f --- /dev/null +++ b/tests/performance/random_string_utf8.xml @@ -0,0 +1,12 @@ + + + + + SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomStringUTF8(10)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomStringUTF8(100)) + SELECT count() FROM zeros(100000) WHERE NOT ignore(randomStringUTF8(1000)) + SELECT count() FROM zeros(10000) WHERE NOT ignore(randomStringUTF8(10000)) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(randomStringUTF8(rand() % 10)) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(randomStringUTF8(rand() % 100)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomStringUTF8(rand() % 1000)) + diff --git a/tests/queries/0_stateless/00029_test_zookeeper_optimize_exception.sh b/tests/queries/0_stateless/00029_test_zookeeper_optimize_exception.sh index 7c55f59a3f5..0600da63af4 100755 --- a/tests/queries/0_stateless/00029_test_zookeeper_optimize_exception.sh +++ b/tests/queries/0_stateless/00029_test_zookeeper_optimize_exception.sh @@ -5,6 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test_optimize_exception" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test_optimize_exception_replicated" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE test_optimize_exception (date Date) ENGINE=MergeTree() PARTITION BY toYYYYMM(date) ORDER BY date" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE test_optimize_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test/optimize', 'r1') PARTITION BY toYYYYMM(date) ORDER BY date" @@ -23,4 +24,5 @@ echo `${CLICKHOUSE_CLIENT} --optimize_throw_if_noop 1 --server_logs_file=/dev/nu | grep -c 'Code: 388. DB::Exception: .* DB::Exception:.* Cannot select parts for optimization' ${CLICKHOUSE_CLIENT} --query="DROP TABLE test_optimize_exception NO DELAY" +${CLICKHOUSE_CLIENT} --query="DROP TABLE test_optimize_exception_replicated NO DELAY" sleep 1 diff --git a/tests/queries/0_stateless/00191_aggregating_merge_tree_and_final.sql b/tests/queries/0_stateless/00191_aggregating_merge_tree_and_final.sql index 47b96006113..776edeeb43c 100644 --- a/tests/queries/0_stateless/00191_aggregating_merge_tree_and_final.sql +++ b/tests/queries/0_stateless/00191_aggregating_merge_tree_and_final.sql @@ -4,11 +4,11 @@ CREATE TABLE aggregating_00191 (d Date DEFAULT '2000-01-01', k UInt64, u Aggrega INSERT INTO aggregating_00191 (k, u) SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000) GROUP BY k; INSERT INTO aggregating_00191 (k, u) SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100) + 50) AS u FROM (SELECT * FROM system.numbers LIMIT 500, 1000) GROUP BY k; -SELECT k, finalizeAggregation(u) FROM aggregating_00191 FINAL; +SELECT k, finalizeAggregation(u) FROM aggregating_00191 FINAL order by k; OPTIMIZE TABLE aggregating_00191; SELECT k, finalizeAggregation(u) FROM aggregating_00191; -SELECT k, finalizeAggregation(u) FROM aggregating_00191 FINAL; +SELECT k, finalizeAggregation(u) FROM aggregating_00191 FINAL order by k; DROP TABLE aggregating_00191; diff --git a/tests/queries/0_stateless/00219_full_right_join_column_order.sql b/tests/queries/0_stateless/00219_full_right_join_column_order.sql index 76098261358..78dfe236923 100644 --- a/tests/queries/0_stateless/00219_full_right_join_column_order.sql +++ b/tests/queries/0_stateless/00219_full_right_join_column_order.sql @@ -1,8 +1,8 @@ SET any_join_distinct_right_table_keys = 1; -SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) js1 ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) js2 USING a, b; -SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) js1 ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) js2 USING b, a; +SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) js1 ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) js2 USING a, b ORDER BY a, b; +SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) js1 ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) js2 USING b, a ORDER BY a, b; -SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) js1 ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b UNION ALL SELECT 1 AS a, 2000 AS b) js2 USING a, b; -SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) js1 ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b UNION ALL SELECT 1 AS a, 2000 AS b) js2 USING b, a; +SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) js1 ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b UNION ALL SELECT 1 AS a, 2000 AS b) js2 USING a, b ORDER BY a, b; +SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) js1 ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b UNION ALL SELECT 1 AS a, 2000 AS b) js2 USING b, a ORDER BY a, b; diff --git a/tests/queries/0_stateless/00564_versioned_collapsing_merge_tree.sql b/tests/queries/0_stateless/00564_versioned_collapsing_merge_tree.sql index 5b7f59f8b65..b7824e7efdc 100644 --- a/tests/queries/0_stateless/00564_versioned_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/00564_versioned_collapsing_merge_tree.sql @@ -3,7 +3,7 @@ create table mult_tab (date Date, value String, version UInt64, sign Int8) engin insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; select 'table with 2 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, sign; optimize table mult_tab; select 'table with 2 blocks optimized'; select * from mult_tab; @@ -15,7 +15,7 @@ create table mult_tab (date Date, value String, version UInt64, sign Int8) engin insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; select 'table with 2 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, sign; optimize table mult_tab; select 'table with 2 blocks optimized'; select * from mult_tab; @@ -27,7 +27,7 @@ create table mult_tab (date Date, value String, version UInt64, sign Int8) engin insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, -1, 1) from system.numbers limit 10; select 'table with 2 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, sign; optimize table mult_tab; select 'table with 2 blocks optimized'; select * from mult_tab; @@ -39,7 +39,7 @@ create table mult_tab (date Date, value String, version UInt64, sign Int8) engin insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 1, if(number % 2, -1, 1) from system.numbers limit 10; select 'table with 2 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, version, sign; optimize table mult_tab; select 'table with 2 blocks optimized'; select * from mult_tab; @@ -53,7 +53,7 @@ insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(numb insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, -1, 1) from system.numbers limit 10; select 'table with 4 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, sign; optimize table mult_tab; select 'table with 4 blocks optimized'; select * from mult_tab; @@ -68,7 +68,7 @@ insert into mult_tab select '2018-01-31', 'str_' || toString(number), 1, if(numb insert into mult_tab select '2018-01-31', 'str_' || toString(number), 1, if(number % 3 = 1, 1, -1) from system.numbers limit 10; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 1, if(number % 3 = 2, 1, -1) from system.numbers limit 10; select 'table with 5 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, sign; optimize table mult_tab; select 'table with 5 blocks optimized'; select * from mult_tab; @@ -80,7 +80,7 @@ create table mult_tab (date Date, value String, version UInt64, sign Int8) engin insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 1000000; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, -1, 1) from system.numbers limit 1000000; select 'table with 2 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, sign; optimize table mult_tab; select 'table with 2 blocks optimized'; select * from mult_tab; @@ -88,14 +88,14 @@ select * from mult_tab; select '-------------------------'; drop table if exists mult_tab; -create table mult_tab (date Date, value UInt64, version UInt64, sign Int8) engine = VersionedCollapsingMergeTree(date, (date), 8192, sign, version); -insert into mult_tab select '2018-01-31', number, 0, if(number < 64, 1, -1) from system.numbers limit 128; -insert into mult_tab select '2018-01-31', number, 0, if(number < 64, -1, 1) from system.numbers limit 128; +create table mult_tab (date Date, value UInt64, key UInt64, version UInt64, sign Int8) engine = VersionedCollapsingMergeTree(date, (date), 8192, sign, version); +insert into mult_tab select '2018-01-31', number, number, 0, if(number < 64, 1, -1) from system.numbers limit 128; +insert into mult_tab select '2018-01-31', number, number + 128, 0, if(number < 64, -1, 1) from system.numbers limit 128; select 'table with 2 blocks final'; -select * from mult_tab final settings max_block_size=33; +select date, value, version, sign from mult_tab final order by date, key, sign settings max_block_size=33; optimize table mult_tab; select 'table with 2 blocks optimized'; -select * from mult_tab; +select date, value, version, sign from mult_tab; select '-------------------------'; select 'Vertival merge'; @@ -106,7 +106,7 @@ create table mult_tab (date Date, value String, version UInt64, sign Int8) engin insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; select 'table with 2 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, sign; optimize table mult_tab; select 'table with 2 blocks optimized'; select * from mult_tab; @@ -118,7 +118,7 @@ create table mult_tab (date Date, value String, version UInt64, sign Int8) engin insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; select 'table with 2 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, sign; optimize table mult_tab; select 'table with 2 blocks optimized'; select * from mult_tab; @@ -130,7 +130,7 @@ create table mult_tab (date Date, value String, version UInt64, sign Int8) engin insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, -1, 1) from system.numbers limit 10; select 'table with 2 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, sign; optimize table mult_tab; select 'table with 2 blocks optimized'; select * from mult_tab; @@ -142,7 +142,7 @@ create table mult_tab (date Date, value String, version UInt64, sign Int8) engin insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 1, if(number % 2, -1, 1) from system.numbers limit 10; select 'table with 2 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, version, sign; optimize table mult_tab; select 'table with 2 blocks optimized'; select * from mult_tab; @@ -156,7 +156,7 @@ insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(numb insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 10; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, -1, 1) from system.numbers limit 10; select 'table with 4 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, sign; optimize table mult_tab; select 'table with 4 blocks optimized'; select * from mult_tab; @@ -171,7 +171,7 @@ insert into mult_tab select '2018-01-31', 'str_' || toString(number), 1, if(numb insert into mult_tab select '2018-01-31', 'str_' || toString(number), 1, if(number % 3 = 1, 1, -1) from system.numbers limit 10; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 1, if(number % 3 = 2, 1, -1) from system.numbers limit 10; select 'table with 5 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, sign; optimize table mult_tab; select 'table with 5 blocks optimized'; select * from mult_tab; @@ -183,7 +183,7 @@ create table mult_tab (date Date, value String, version UInt64, sign Int8) engin insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, 1, -1) from system.numbers limit 1000000; insert into mult_tab select '2018-01-31', 'str_' || toString(number), 0, if(number % 2, -1, 1) from system.numbers limit 1000000; select 'table with 2 blocks final'; -select * from mult_tab final; +select * from mult_tab final order by date, value, sign; optimize table mult_tab; select 'table with 2 blocks optimized'; select * from mult_tab; @@ -191,13 +191,13 @@ select * from mult_tab; select '-------------------------'; drop table if exists mult_tab; -create table mult_tab (date Date, value UInt64, version UInt64, sign Int8) engine = VersionedCollapsingMergeTree(sign, version) order by (date) settings enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 0; -insert into mult_tab select '2018-01-31', number, 0, if(number < 64, 1, -1) from system.numbers limit 128; -insert into mult_tab select '2018-01-31', number, 0, if(number < 64, -1, 1) from system.numbers limit 128; +create table mult_tab (date Date, value UInt64, key UInt64, version UInt64, sign Int8) engine = VersionedCollapsingMergeTree(sign, version) order by (date) settings enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 0; +insert into mult_tab select '2018-01-31', number, number, 0, if(number < 64, 1, -1) from system.numbers limit 128; +insert into mult_tab select '2018-01-31', number, number + 128, 0, if(number < 64, -1, 1) from system.numbers limit 128; select 'table with 2 blocks final'; -select * from mult_tab final settings max_block_size=33; +select date, value, version, sign from mult_tab final order by date, key, sign settings max_block_size=33; optimize table mult_tab; select 'table with 2 blocks optimized'; -select * from mult_tab; +select date, value, version, sign from mult_tab; DROP TABLE mult_tab; diff --git a/tests/queries/0_stateless/00700_decimal_arithm.sql b/tests/queries/0_stateless/00700_decimal_arithm.sql index 8cdc81e21b4..3fa2aade743 100644 --- a/tests/queries/0_stateless/00700_decimal_arithm.sql +++ b/tests/queries/0_stateless/00700_decimal_arithm.sql @@ -11,12 +11,17 @@ CREATE TABLE IF NOT EXISTS decimal g Decimal(9, 3), h decimal(18, 9), i deciMAL(38, 18), - j dec(4,2) + j dec(4, 2), + k NumEriC(23, 4), + l numeric(9, 3), + m NUMEric(18, 9), + n FixED(12, 6), + o fixed(8, 6) ) ENGINE = Memory; -INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0); -INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (42, 42, 42, 0.42, 0.42, 0.42, 42.42, 42.42, 42.42, 42.42); -INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, -0.42, -0.42, -42.42, -42.42, -42.42, -42.42); +INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j, k, l, m, n, o) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0); +INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j, k, l, m, n, o) VALUES (42, 42, 42, 0.42, 0.42, 0.42, 42.42, 42.42, 42.42, 42.42, 42.42, 42.42, 42.42, 42.42, 42.42); +INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j, k, l, m, n, o) VALUES (-42, -42, -42, -0.42, -0.42, -0.42, -42.42, -42.42, -42.42, -42.42, -42.42, -42.42, -42.42, -42.42, -42.42); SELECT a + a, a - a, a * a, a / a, intDiv(a, a), intDivOrZero(a, a) FROM decimal WHERE a = 42; SELECT b + b, b - b, b * b, b / b, intDiv(b, b), intDivOrZero(b, b) FROM decimal WHERE b = 42; diff --git a/tests/queries/0_stateless/00714_alter_uuid.sql b/tests/queries/0_stateless/00714_alter_uuid.sql index 6dca79c37ab..00a8f793e0e 100644 --- a/tests/queries/0_stateless/00714_alter_uuid.sql +++ b/tests/queries/0_stateless/00714_alter_uuid.sql @@ -39,7 +39,7 @@ ORDER BY (created_at, id0, id1); SET send_logs_level = 'none'; -ALTER TABLE uuid MODIFY COLUMN id0 UUID; -- { serverError 44 } -ALTER TABLE uuid MODIFY COLUMN id1 UUID; -- { serverError 44 } +ALTER TABLE uuid MODIFY COLUMN id0 UUID; -- { serverError 524 } +ALTER TABLE uuid MODIFY COLUMN id1 UUID; -- { serverError 524 } DROP TABLE uuid; diff --git a/tests/queries/0_stateless/00915_simple_aggregate_function.sql b/tests/queries/0_stateless/00915_simple_aggregate_function.sql index 1866e2bc8c5..ba4935a6518 100644 --- a/tests/queries/0_stateless/00915_simple_aggregate_function.sql +++ b/tests/queries/0_stateless/00915_simple_aggregate_function.sql @@ -5,13 +5,13 @@ create table simple (id UInt64,val SimpleAggregateFunction(sum,Double)) engine=A insert into simple select number,number from system.numbers limit 10; select * from simple; -select * from simple final; +select * from simple final order by id; select toTypeName(val) from simple limit 1; -- merge insert into simple select number,number from system.numbers limit 10; -select * from simple final; +select * from simple final order by id; optimize table simple final; select * from simple; @@ -33,7 +33,7 @@ insert into simple values(1,null,'2','2.2.2.2', 2, ([1,3], [1,1])); insert into simple values(10,'10','10','10.10.10.10', 4, ([2,3], [1,1])); insert into simple values(10,'2222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222','20','20.20.20.20', 1, ([2, 4], [1,1])); -select * from simple final; +select * from simple final order by id; select toTypeName(nullable_str),toTypeName(low_str),toTypeName(ip),toTypeName(status), toTypeName(tup) from simple limit 1; optimize table simple final; diff --git a/tests/queries/0_stateless/00933_alter_ttl.sql b/tests/queries/0_stateless/00933_alter_ttl.sql index f7f141a9408..d3298b3fbe1 100644 --- a/tests/queries/0_stateless/00933_alter_ttl.sql +++ b/tests/queries/0_stateless/00933_alter_ttl.sql @@ -21,6 +21,6 @@ drop table if exists ttl; create table ttl (d Date, a Int) engine = MergeTree order by tuple() partition by toDayOfMonth(d); alter table ttl modify column a Int ttl d + interval 1 day; desc table ttl; -alter table ttl modify column d Int ttl d + interval 1 day; -- { serverError 44} +alter table ttl modify column d Int ttl d + interval 1 day; -- { serverError 524 } drop table if exists ttl; diff --git a/tests/queries/0_stateless/00979_live_view_watch_continuous_aggregates.py b/tests/queries/0_stateless/00979_live_view_watch_continuous_aggregates.py index 5aec9ac0a8b..d184fea7183 100755 --- a/tests/queries/0_stateless/00979_live_view_watch_continuous_aggregates.py +++ b/tests/queries/0_stateless/00979_live_view_watch_continuous_aggregates.py @@ -33,28 +33,28 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(r'0.*1' + end_of_block) client2.send("INSERT INTO test.mt VALUES ('2019-01-01 00:00:00','New York',60),('2019-01-01 00:10:00','New York',70)") client2.expect(prompt) - client1.expect(r'"2019-01-01 00:00:00","New York",65,2') + client1.expect(r'"2019-01-01 00:00:00","New York",65') client2.send("INSERT INTO test.mt VALUES ('2019-01-01 00:00:00','Moscow',30),('2019-01-01 00:10:00', 'Moscow', 40)") client2.expect(prompt) - client1.expect(r'"2019-01-01 00:00:00","Moscow",35,3') - client1.expect(r'"2019-01-01 00:00:00","New York",65,3') + client1.expect(r'"2019-01-01 00:00:00","Moscow",35') + client1.expect(r'"2019-01-01 00:00:00","New York",65') client2.send("INSERT INTO test.mt VALUES ('2019-01-02 00:00:00','New York',50),('2019-01-02 00:10:00','New York',60)") client2.expect(prompt) - client1.expect(r'"2019-01-01 00:00:00","Moscow",35,4') - client1.expect(r'"2019-01-01 00:00:00","New York",65,4') - client1.expect(r'"2019-01-02 00:00:00","New York",55,4') + client1.expect(r'"2019-01-01 00:00:00","Moscow",35') + client1.expect(r'"2019-01-01 00:00:00","New York",65') + client1.expect(r'"2019-01-02 00:00:00","New York",55') client2.send("INSERT INTO test.mt VALUES ('2019-01-02 00:00:00','Moscow',20),('2019-01-02 00:10:00', 'Moscow', 30)") client2.expect(prompt) - client1.expect(r'"2019-01-01 00:00:00","Moscow",35,5') - client1.expect(r'"2019-01-01 00:00:00","New York",65,5') - client1.expect(r'"2019-01-02 00:00:00","Moscow",25,5') - client1.expect(r'"2019-01-02 00:00:00","New York",55,5') + client1.expect(r'"2019-01-01 00:00:00","Moscow",35') + client1.expect(r'"2019-01-01 00:00:00","New York",65') + client1.expect(r'"2019-01-02 00:00:00","Moscow",25') + client1.expect(r'"2019-01-02 00:00:00","New York",55') client2.send("INSERT INTO test.mt VALUES ('2019-01-02 00:03:00','New York',40),('2019-01-02 00:06:00','New York',30)") client2.expect(prompt) - client1.expect(r'"2019-01-01 00:00:00","Moscow",35,6') - client1.expect(r'"2019-01-01 00:00:00","New York",65,6') - client1.expect(r'"2019-01-02 00:00:00","Moscow",25,6') - client1.expect(r'"2019-01-02 00:00:00","New York",45,6') + client1.expect(r'"2019-01-01 00:00:00","Moscow",35') + client1.expect(r'"2019-01-01 00:00:00","New York",65') + client1.expect(r'"2019-01-02 00:00:00","Moscow",25') + client1.expect(r'"2019-01-02 00:00:00","New York",45') # send Ctrl-C client1.send('\x03', eol='') match = client1.expect('(%s)|([#\$] )' % prompt) diff --git a/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql b/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql index a9f7bf7ecd7..0b5845d3b04 100644 --- a/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql +++ b/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql @@ -4,7 +4,7 @@ drop table if exists tst; create table tst (timestamp DateTime, val Nullable(Int8)) engine SummingMergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); insert into tst values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2); -select * from tst final; +select * from tst final order by timestamp; select '-- 2 2'; select count() from tst; @@ -34,7 +34,7 @@ drop table if exists tst; create table tst (timestamp DateTime, val Nullable(Int8)) engine SummingMergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); insert into tst values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2), ('2018-02-01 00:00:00', 3), ('2018-02-02 00:00:00', 4); -select * from tst final; +select * from tst final order by timestamp; select '-- 4 2'; select count() from tst; @@ -64,7 +64,7 @@ drop table if exists tst; create table tst (timestamp DateTime, val Int8) engine SummingMergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); insert into tst values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2); -select * from tst final; +select * from tst final order by timestamp; select '-- 2 2'; select count() from tst; @@ -96,7 +96,7 @@ drop table if exists tst; create table tst (timestamp DateTime, val Int8) engine SummingMergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); insert into tst values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2), ('2018-02-01 00:00:00', 3), ('2018-02-02 00:00:00', 4); -select * from tst final; +select * from tst final order by timestamp; select '-- 4 2'; select count() from tst; diff --git a/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh b/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh index b6522b12895..15bb851fc77 100755 --- a/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh @@ -105,7 +105,7 @@ sleep 1 counter=0 while [[ $($CLICKHOUSE_CLIENT --query "select * from system.mutations where table like 'concurrent_mutate_mt_%' and is_done=0" 2>&1) ]]; do - if [ "$counter" -gt 20 ] + if [ "$counter" -gt 40 ] then break fi diff --git a/tests/queries/0_stateless/01131_max_rows_to_sort.reference b/tests/queries/0_stateless/01131_max_rows_to_sort.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01131_max_rows_to_sort.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01131_max_rows_to_sort.sql b/tests/queries/0_stateless/01131_max_rows_to_sort.sql new file mode 100644 index 00000000000..a6109700045 --- /dev/null +++ b/tests/queries/0_stateless/01131_max_rows_to_sort.sql @@ -0,0 +1,7 @@ +SET max_rows_to_sort = 100; +SELECT * FROM system.numbers ORDER BY number; -- { serverError 396 } + +SET sort_overflow_mode = 'break'; +SET max_block_size = 1000; + +SELECT count() >= 100 AND count() <= 1000 FROM (SELECT * FROM system.numbers ORDER BY number); diff --git a/tests/queries/0_stateless/01132_max_rows_to_read.reference b/tests/queries/0_stateless/01132_max_rows_to_read.reference new file mode 100644 index 00000000000..5087d15b87c --- /dev/null +++ b/tests/queries/0_stateless/01132_max_rows_to_read.reference @@ -0,0 +1,57 @@ +19 +20 +19 +20 +20 +20 +20 +20 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 diff --git a/tests/queries/0_stateless/01132_max_rows_to_read.sql b/tests/queries/0_stateless/01132_max_rows_to_read.sql new file mode 100644 index 00000000000..b7923a27d04 --- /dev/null +++ b/tests/queries/0_stateless/01132_max_rows_to_read.sql @@ -0,0 +1,30 @@ +SET max_block_size = 10; +SET max_rows_to_read = 20; +SET read_overflow_mode = 'throw'; + +SELECT count() FROM numbers(30); -- { serverError 158 } +SELECT count() FROM numbers(19); +SELECT count() FROM numbers(20); +SELECT count() FROM numbers(21); -- { serverError 158 } + +-- check early exception if the estimated number of rows is high +SELECT * FROM numbers(30); -- { serverError 158 } + +SET read_overflow_mode = 'break'; + +SELECT count() FROM numbers(19); +SELECT count() FROM numbers(20); +SELECT count() FROM numbers(21); +SELECT count() FROM numbers(29); +SELECT count() FROM numbers(30); +SELECT count() FROM numbers(31); + +-- check that partial result is returned even if the estimated number of rows is high +SELECT * FROM numbers(30); + +-- the same for uneven block sizes +-- NOTE: currently it outputs less amount of data; it will be better to output the latest block also +SET max_block_size = 11; +SELECT * FROM numbers(30); +SET max_block_size = 9; +SELECT * FROM numbers(30); diff --git a/tests/queries/0_stateless/01133_max_result_rows.reference b/tests/queries/0_stateless/01133_max_result_rows.reference new file mode 100644 index 00000000000..7134ee81514 --- /dev/null +++ b/tests/queries/0_stateless/01133_max_result_rows.reference @@ -0,0 +1,173 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +20 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +20 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +100 +100 +200 diff --git a/tests/queries/0_stateless/01133_max_result_rows.sql b/tests/queries/0_stateless/01133_max_result_rows.sql new file mode 100644 index 00000000000..db359959aed --- /dev/null +++ b/tests/queries/0_stateless/01133_max_result_rows.sql @@ -0,0 +1,24 @@ +SET max_block_size = 10; +SET max_result_rows = 20; +SET result_overflow_mode = 'throw'; + +SELECT DISTINCT intDiv(number, 10) FROM numbers(300); -- { serverError 396 } +SELECT DISTINCT intDiv(number, 10) FROM numbers(190); +SELECT DISTINCT intDiv(number, 10) FROM numbers(200); +SELECT DISTINCT intDiv(number, 10) FROM numbers(210); -- { serverError 396 } + +SET result_overflow_mode = 'break'; + +SELECT DISTINCT intDiv(number, 10) FROM numbers(300); +SELECT DISTINCT intDiv(number, 10) FROM numbers(190); +SELECT DISTINCT intDiv(number, 10) FROM numbers(200); +SELECT DISTINCT intDiv(number, 10) FROM numbers(210); + +SET max_block_size = 10; +SET max_result_rows = 1; +SELECT number FROM system.numbers; +SELECT count() FROM numbers(100); +-- subquery result is not the total result +SELECT count() FROM (SELECT * FROM numbers(100)); +-- remote query result is not the total result +SELECT count() FROM remote('127.0.0.{1,2}', numbers(100)); diff --git a/tests/queries/0_stateless/01134_set_overflow_mode.reference b/tests/queries/0_stateless/01134_set_overflow_mode.reference new file mode 100644 index 00000000000..2e981dc7c97 --- /dev/null +++ b/tests/queries/0_stateless/01134_set_overflow_mode.reference @@ -0,0 +1,13 @@ +1 +0 +1 +0 +--- +1 +0 +1 +0 +1 +0 +1 +0 diff --git a/tests/queries/0_stateless/01134_set_overflow_mode.sql b/tests/queries/0_stateless/01134_set_overflow_mode.sql new file mode 100644 index 00000000000..791bc6d7f9e --- /dev/null +++ b/tests/queries/0_stateless/01134_set_overflow_mode.sql @@ -0,0 +1,17 @@ +SET max_block_size = 10; +SET max_rows_in_set = 20; +SET set_overflow_mode = 'throw'; + +SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(300)); -- { serverError 191 } +SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(190)); +SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(200)); +SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(210)); -- { serverError 191 } + +SET set_overflow_mode = 'break'; + +SELECT '---'; + +SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(300)); +SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(190)); +SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(200)); +SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(210)); diff --git a/tests/queries/0_stateless/01137_order_by_func_final.sql b/tests/queries/0_stateless/01137_order_by_func_final.sql index 32a9085e9ee..afbced30131 100644 --- a/tests/queries/0_stateless/01137_order_by_func_final.sql +++ b/tests/queries/0_stateless/01137_order_by_func_final.sql @@ -5,6 +5,6 @@ INSERT INTO pk_func SELECT '2020-05-05 01:00:00', number FROM numbers(100000); INSERT INTO pk_func SELECT '2020-05-06 01:00:00', number FROM numbers(100000); INSERT INTO pk_func SELECT '2020-05-07 01:00:00', number FROM numbers(100000); -SELECT toDate(d), ui FROM pk_func FINAL; +SELECT toDate(d), ui FROM pk_func FINAL order by d; DROP TABLE pk_func; diff --git a/tests/queries/0_stateless/01198_client_quota_key.reference b/tests/queries/0_stateless/01198_client_quota_key.reference new file mode 100644 index 00000000000..c25611c15c6 --- /dev/null +++ b/tests/queries/0_stateless/01198_client_quota_key.reference @@ -0,0 +1,2 @@ +1 +Hello diff --git a/tests/queries/0_stateless/01198_client_quota_key.sh b/tests/queries/0_stateless/01198_client_quota_key.sh new file mode 100755 index 00000000000..f4b66aea6ac --- /dev/null +++ b/tests/queries/0_stateless/01198_client_quota_key.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --quota_key Hello --query_id test_quota_key --log_queries 1 --multiquery --query "SELECT 1; SYSTEM FLUSH LOGS; SELECT DISTINCT quota_key FROM system.query_log WHERE event_date >= yesterday() AND event_time >= now() - 300 AND query_id = 'test_quota_key'" diff --git a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql index b5ad162fdfb..5c62d5d9107 100644 --- a/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql +++ b/tests/queries/0_stateless/01213_alter_rename_primary_key_zookeeper.sql @@ -17,11 +17,11 @@ INSERT INTO table_for_rename_pk SELECT toDate('2019-10-01') + number % 3, number SELECT key1, value1 FROM table_for_rename_pk WHERE key1 = 1 AND key2 = 1 AND key3 = 1; -ALTER TABLE table_for_rename_pk RENAME COLUMN key1 TO renamed_key1; --{serverError 44} +ALTER TABLE table_for_rename_pk RENAME COLUMN key1 TO renamed_key1; --{serverError 524} -ALTER TABLE table_for_rename_pk RENAME COLUMN key3 TO renamed_key3; --{serverError 44} +ALTER TABLE table_for_rename_pk RENAME COLUMN key3 TO renamed_key3; --{serverError 524} -ALTER TABLE table_for_rename_pk RENAME COLUMN key2 TO renamed_key2; --{serverError 44} +ALTER TABLE table_for_rename_pk RENAME COLUMN key2 TO renamed_key2; --{serverError 524} DROP TABLE IF EXISTS table_for_rename_pk NO DELAY; SELECT sleep(1) FORMAT Null; @@ -45,12 +45,12 @@ PRIMARY KEY (key1, key2); INSERT INTO table_for_rename_with_primary_key SELECT toDate('2019-10-01') + number % 3, number, number, number, toString(number), toString(number) from numbers(9); -ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key1 TO renamed_key1; --{serverError 44} +ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key1 TO renamed_key1; --{serverError 524} -ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key2 TO renamed_key2; --{serverError 44} +ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key2 TO renamed_key2; --{serverError 524} -ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key3 TO renamed_key3; --{serverError 44} +ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key3 TO renamed_key3; --{serverError 524} -ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN value1 TO renamed_value1; --{serverError 44} +ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN value1 TO renamed_value1; --{serverError 524} DROP TABLE IF EXISTS table_for_rename_with_primary_key; diff --git a/tests/queries/0_stateless/01272_offset_without_limit.reference b/tests/queries/0_stateless/01272_offset_without_limit.reference new file mode 100644 index 00000000000..780c35c9d70 --- /dev/null +++ b/tests/queries/0_stateless/01272_offset_without_limit.reference @@ -0,0 +1,45 @@ +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +20 +21 +22 +23 +24 +25 +26 +27 +28 +29 +30 +31 +32 +33 +34 +35 +36 +37 +38 +39 +40 +41 +42 +43 +44 +45 +46 +47 +48 +49 diff --git a/tests/queries/0_stateless/01272_offset_without_limit.sql b/tests/queries/0_stateless/01272_offset_without_limit.sql new file mode 100644 index 00000000000..769808b2edd --- /dev/null +++ b/tests/queries/0_stateless/01272_offset_without_limit.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS offset_without_limit; + +CREATE TABLE offset_without_limit ( + value UInt32 +) Engine = MergeTree() + PRIMARY KEY value + ORDER BY value; + +INSERT INTO offset_without_limit SELECT * FROM system.numbers LIMIT 50; + +SELECT value FROM offset_without_limit ORDER BY value OFFSET 5; + +DROP TABLE offset_without_limit; diff --git a/tests/queries/0_stateless/01277_random_fixed_string.reference b/tests/queries/0_stateless/01277_random_fixed_string.reference new file mode 100644 index 00000000000..68189063594 --- /dev/null +++ b/tests/queries/0_stateless/01277_random_fixed_string.reference @@ -0,0 +1,2 @@ +FixedString(10) +1 diff --git a/tests/queries/0_stateless/01277_random_fixed_string.sql b/tests/queries/0_stateless/01277_random_fixed_string.sql new file mode 100644 index 00000000000..99782c1ac34 --- /dev/null +++ b/tests/queries/0_stateless/01277_random_fixed_string.sql @@ -0,0 +1,5 @@ +SELECT randomFixedString('string'); -- { serverError 43 } +SELECT randomFixedString(0); -- { serverError 69 } +SELECT randomFixedString(rand() % 10); -- { serverError 44 } +SELECT toTypeName(randomFixedString(10)); +SELECT DISTINCT c > 30000 FROM (SELECT arrayJoin(arrayMap(x -> reinterpretAsUInt8(substring(randomFixedString(100), x + 1, 1)), range(100))) AS byte, count() AS c FROM numbers(100000) GROUP BY byte ORDER BY byte); diff --git a/tests/queries/0_stateless/01278_random_string_utf8.reference b/tests/queries/0_stateless/01278_random_string_utf8.reference new file mode 100644 index 00000000000..36ae0ace76a --- /dev/null +++ b/tests/queries/0_stateless/01278_random_string_utf8.reference @@ -0,0 +1,4 @@ +100 +String +1 + diff --git a/tests/queries/0_stateless/01278_random_string_utf8.sql b/tests/queries/0_stateless/01278_random_string_utf8.sql new file mode 100644 index 00000000000..f2c0a6c41c7 --- /dev/null +++ b/tests/queries/0_stateless/01278_random_string_utf8.sql @@ -0,0 +1,5 @@ +SELECT randomStringUTF8('string'); -- { serverError 43 } +SELECT lengthUTF8(randomStringUTF8(100)); +SELECT toTypeName(randomStringUTF8(10)); +SELECT isValidUTF8(randomStringUTF8(100000)); +SELECT randomStringUTF8(0); diff --git a/tests/queries/0_stateless/01280_null_in.reference b/tests/queries/0_stateless/01280_null_in.reference new file mode 100644 index 00000000000..04402256766 --- /dev/null +++ b/tests/queries/0_stateless/01280_null_in.reference @@ -0,0 +1,8 @@ +0 +0 +1 +1 +0 +0 +1 +1 diff --git a/tests/queries/0_stateless/01280_null_in.sql b/tests/queries/0_stateless/01280_null_in.sql new file mode 100644 index 00000000000..76fe4db6786 --- /dev/null +++ b/tests/queries/0_stateless/01280_null_in.sql @@ -0,0 +1,9 @@ +SELECT count(in(NULL, [])); +SELECT count(notIn(NULL, [])); +SELECT count(nullIn(NULL, [])); +SELECT count(notNullIn(NULL, [])); + +SELECT count(in(NULL, tuple(NULL))); +SELECT count(notIn(NULL, tuple(NULL))); +SELECT count(nullIn(NULL, tuple(NULL))); +SELECT count(notNullIn(NULL, tuple(NULL))); diff --git a/tests/queries/bugs/leak_when_memory_limit_exceeded.sql b/tests/queries/bugs/leak_when_memory_limit_exceeded.sql new file mode 100644 index 00000000000..68811ad061e --- /dev/null +++ b/tests/queries/bugs/leak_when_memory_limit_exceeded.sql @@ -0,0 +1,22 @@ +-- max_memory_usage = 10000000000 (10 GB default) +-- Intel® Xeon® E5-1650 v3 Hexadcore 128 GB DDR4 ECC +-- Estimated time: ~ 250 seconds +-- Read rows: ~ 272 000 000 +SELECT + key, + uniqState(uuid_1) uuid_1_st, + uniqState(uuid_2) uuid_2_st, + uniqState(uuid_3) uuid_3_st +FROM ( + SELECT + rand64() value, + toString(value) value_str, + UUIDNumToString(toFixedString(substring(value_str, 1, 16), 16)) uuid_1, -- Any UUID + UUIDNumToString(toFixedString(substring(value_str, 2, 16), 16)) uuid_2, -- More memory + UUIDNumToString(toFixedString(substring(value_str, 3, 16), 16)) uuid_3, -- And more memory + modulo(value, 5000000) key -- Cardinality in my case + FROM numbers(550000000) +) +GROUP BY + key +LIMIT 100; diff --git a/website/css/base.css b/website/css/base.css index 1546337d4cf..a63d1d7b166 100644 --- a/website/css/base.css +++ b/website/css/base.css @@ -39,6 +39,12 @@ a:hover, a:active { line-height: 1.2; } +.display-6 { + font-size: 1.75rem; + font-weight: 300; + line-height: 1.2; +} + .bg-dark-alt, .bg-dark-alt:focus { background: #36363F; } diff --git a/website/css/docs.css b/website/css/docs.css index dfbf6d12b2d..18bf40de13b 100644 --- a/website/css/docs.css +++ b/website/css/docs.css @@ -33,7 +33,6 @@ summary { } .headerlink { - display: none; text-decoration: none !important; margin-left: .5rem; } @@ -171,11 +170,15 @@ summary { } @media (prefers-color-scheme: dark) { - body[data-spy] { + body[data-spy], body.amp, #to-full-website { background: #1c1c1c; color: #efefef; } + .invert-dark { + filter: invert(100%); + } + #sidebar { background: #333; } @@ -191,6 +194,7 @@ summary { #content td { border: 1px solid #444451; } + #content code { background: #444; color: #eee; diff --git a/website/js/base.js b/website/js/base.js index 92d34cf2661..46f06f70f30 100644 --- a/website/js/base.js +++ b/website/js/base.js @@ -1,27 +1,29 @@ (function () { + Sentry.init({ dsn: 'https://2b95b52c943f4ad99baccab7a9048e4d@o388870.ingest.sentry.io/5246103' }); $(document).click(function (event) { var target = $(event.target); var target_id = target.attr('id'); var selector = target.attr('href'); var is_tab = target.attr('role') === 'tab'; var is_collapse = target.attr('data-toggle') === 'collapse'; + var is_rating = target.attr('role') === 'rating'; var navbar_toggle = $('#navbar-toggle'); - navbar_toggle.collapse('hide'); $('.algolia-autocomplete .ds-dropdown-menu').hide(); - if (target_id && target_id.startsWith('logo-')) { selector = '#'; } - - if (selector && selector.startsWith('#') && !is_tab && !is_collapse) { + if (selector && selector.startsWith('#') && !is_tab && !is_collapse && !is_rating) { event.preventDefault(); var dst = window.location.href.replace(window.location.hash, ''); var offset = 0; if (selector !== '#') { - offset = $(selector).offset().top - $('#top-nav').height() * 1.5; - dst += selector; + var destination = $(selector); + if (destination.length) { + offset = destination.offset().top - $('#top-nav').height() * 1.5; + dst += selector; + } } $('html, body').animate({ scrollTop: offset @@ -32,10 +34,13 @@ var top_nav = $('#top-nav.sticky-top'); if (window.location.hash.length > 1 && top_nav.length) { - var offset = $(window.location.hash).offset().top - top_nav.height() * 1.5; - $('html, body').animate({ - scrollTop: offset - }, 70); + var hash_destination = $(window.location.hash); + if (hash_destination.length) { + var offset = hash_destination.offset().top - top_nav.height() * 1.5; + $('html, body').animate({ + scrollTop: offset + }, 70); + } } $('img').each(function() { diff --git a/website/js/docs.js b/website/js/docs.js index c83beb9ba7d..f5a497ca033 100644 --- a/website/js/docs.js +++ b/website/js/docs.js @@ -7,6 +7,7 @@ function onResize() { $('body').attr('data-offset', window_height.toString()); var sidebar = $('#sidebar'); var languages = $('#languages-dropdown') + var edit = $('#edit-link'); var single_page_switch = $('#single-page-switch'); if ((sidebar.width() - single_page_switch.width() - sidebar.find('.dropdown-toggle').width()) >= 36) { single_page_switch.addClass('float-right'); @@ -15,10 +16,13 @@ function onResize() { } if (is_wide) { sidebar.removeClass('collapse'); + edit.detach().appendTo($('#edit-wrapper')); languages.detach().appendTo($('#languages-wrapper')); } else { sidebar.addClass('collapse'); - languages.detach().insertBefore(single_page_switch); + edit.detach().insertBefore(single_page_switch); + edit.addClass('float-right'); + languages.detach().insertBefore(edit); languages.addClass('float-right'); single_page_switch.removeClass('float-right'); } @@ -78,10 +82,10 @@ $(document).ready(function () { var headers = $('#content h1, #content h2, #content h3, #content h4, #content h5, #content h6'); headers.mouseenter(function() { - $(this).find('.headerlink').show(); + $(this).find('.headerlink').html('¶'); }); headers.mouseleave(function() { - $(this).find('.headerlink').hide(); + $(this).find('.headerlink').html(' '); }); if ($('#docsearch-input').length) { @@ -101,4 +105,64 @@ $(document).ready(function () { debug: false }); } + + var rating_stars = $('#rating-stars'); + if (rating_stars.length) { + var key = ''; + var stars_text = rating_stars.text().replace(/^\s+|\s+$/g, ''); + var url_match = window.location.pathname.match(/^\/docs(?:\/v[0-9]+\.[0-9]+)?(?:\/[a-z]{2})(.*)/); + if (url_match && url_match.length > 1) { + key = url_match[1]; + if (key.length > 2) { + key = key.replace(/^\/|\/$/g, ''); + } + key = 'rating_' + key; + var local_stars = localStorage.getItem(key); + if (local_stars) { + stars_text = local_stars; + rating_stars.addClass('text-orange'); + } + } + var titles = JSON.parse(rating_stars.attr('data-titles').replace(/'/g, '"')); + var documentation = rating_stars.attr('data-documentation'); + var replacement = ''; + for (var i = 0; i < 5; i++) { + var star = stars_text.charAt(i); + replacement += '' + star + ''; + } + rating_stars.html(replacement); + $('[data-toggle="tooltip"]').tooltip(); + var rating_star_item = $('.rating-star'); + rating_star_item.hover(function() { + var current = $(this); + current.text('★'); + current.prevAll().text('★'); + current.nextAll().text('☆'); + }); + rating_stars.mouseleave(function() { + for (var i = 0; i < 5; i++) { + rating_stars.children().eq(i).text(stars_text[i]); + } + }); + rating_star_item.click(function(e) { + e.preventDefault(); + rating_stars.addClass('text-orange'); + stars_text = rating_stars.text(); + localStorage.setItem(key, stars_text); + $.ajax({ + url: window.location.pathname + 'rate/', + type: 'POST', + dataType: 'json', + data: JSON.stringify({rating: $(this).prevAll().length + 1}), + success: function () { + }, + error: function () { + rating_stars.removeClass('text-orange'); + localStorage.removeItem(key); + } + }); + }); + } }); diff --git a/website/js/index.js b/website/js/index.js index 0e5a62dadd9..21c3147db98 100644 --- a/website/js/index.js +++ b/website/js/index.js @@ -23,18 +23,17 @@ $(document).ready(function () { }); if (valid) { var data = JSON.stringify(serializeForm(meetup_form)); - console.log(data); $.ajax({ url: '/meet-form/', type: 'POST', dataType: 'json', data: data, success: function () { - meetup_form.html('

Thanks!

We\'ll be in touch soon.

') + meetup_form.html('

Thanks!

We\'ll be in touch soon.

'); $('#meetup-form-error').html(''); }, error: function () { - $('#meetup-form-error').html('
Error! Unfortunately it didn\'t work for some reason, please try again or use the email address below.
') + $('#meetup-form-error').html('
Error! Unfortunately it didn\'t work for some reason, please try again or use the email address below.
'); } }); } diff --git a/website/js/sentry.js b/website/js/sentry.js new file mode 100644 index 00000000000..7d171824be7 --- /dev/null +++ b/website/js/sentry.js @@ -0,0 +1,5699 @@ +/*! @sentry/browser 5.15.5 (2accf045) | https://github.com/getsentry/sentry-javascript */ +var Sentry = (function (exports) { + /*! ***************************************************************************** + Copyright (c) Microsoft Corporation. All rights reserved. + Licensed under the Apache License, Version 2.0 (the "License"); you may not use + this file except in compliance with the License. You may obtain a copy of the + License at http://www.apache.org/licenses/LICENSE-2.0 + + THIS CODE IS PROVIDED ON AN *AS IS* BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, EITHER EXPRESS OR IMPLIED, INCLUDING WITHOUT LIMITATION ANY IMPLIED + WARRANTIES OR CONDITIONS OF TITLE, FITNESS FOR A PARTICULAR PURPOSE, + MERCHANTABLITY OR NON-INFRINGEMENT. + + See the Apache Version 2.0 License for specific language governing permissions + and limitations under the License. + ***************************************************************************** */ + /* global Reflect, Promise */ + + var extendStatics = function(d, b) { + extendStatics = Object.setPrototypeOf || + ({ __proto__: [] } instanceof Array && function (d, b) { d.__proto__ = b; }) || + function (d, b) { for (var p in b) if (b.hasOwnProperty(p)) d[p] = b[p]; }; + return extendStatics(d, b); + }; + + function __extends(d, b) { + extendStatics(d, b); + function __() { this.constructor = d; } + d.prototype = b === null ? Object.create(b) : (__.prototype = b.prototype, new __()); + } + + var __assign = function() { + __assign = Object.assign || function __assign(t) { + for (var s, i = 1, n = arguments.length; i < n; i++) { + s = arguments[i]; + for (var p in s) if (Object.prototype.hasOwnProperty.call(s, p)) t[p] = s[p]; + } + return t; + }; + return __assign.apply(this, arguments); + }; + + function __rest(s, e) { + var t = {}; + for (var p in s) if (Object.prototype.hasOwnProperty.call(s, p) && e.indexOf(p) < 0) + t[p] = s[p]; + if (s != null && typeof Object.getOwnPropertySymbols === "function") + for (var i = 0, p = Object.getOwnPropertySymbols(s); i < p.length; i++) if (e.indexOf(p[i]) < 0) + t[p[i]] = s[p[i]]; + return t; + } + + function __decorate(decorators, target, key, desc) { + var c = arguments.length, r = c < 3 ? target : desc === null ? desc = Object.getOwnPropertyDescriptor(target, key) : desc, d; + if (typeof Reflect === "object" && typeof Reflect.decorate === "function") r = Reflect.decorate(decorators, target, key, desc); + else for (var i = decorators.length - 1; i >= 0; i--) if (d = decorators[i]) r = (c < 3 ? d(r) : c > 3 ? d(target, key, r) : d(target, key)) || r; + return c > 3 && r && Object.defineProperty(target, key, r), r; + } + + function __param(paramIndex, decorator) { + return function (target, key) { decorator(target, key, paramIndex); } + } + + function __metadata(metadataKey, metadataValue) { + if (typeof Reflect === "object" && typeof Reflect.metadata === "function") return Reflect.metadata(metadataKey, metadataValue); + } + + function __awaiter(thisArg, _arguments, P, generator) { + return new (P || (P = Promise))(function (resolve, reject) { + function fulfilled(value) { try { step(generator.next(value)); } catch (e) { reject(e); } } + function rejected(value) { try { step(generator["throw"](value)); } catch (e) { reject(e); } } + function step(result) { result.done ? resolve(result.value) : new P(function (resolve) { resolve(result.value); }).then(fulfilled, rejected); } + step((generator = generator.apply(thisArg, _arguments || [])).next()); + }); + } + + function __generator(thisArg, body) { + var _ = { label: 0, sent: function() { if (t[0] & 1) throw t[1]; return t[1]; }, trys: [], ops: [] }, f, y, t, g; + return g = { next: verb(0), "throw": verb(1), "return": verb(2) }, typeof Symbol === "function" && (g[Symbol.iterator] = function() { return this; }), g; + function verb(n) { return function (v) { return step([n, v]); }; } + function step(op) { + if (f) throw new TypeError("Generator is already executing."); + while (_) try { + if (f = 1, y && (t = op[0] & 2 ? y["return"] : op[0] ? y["throw"] || ((t = y["return"]) && t.call(y), 0) : y.next) && !(t = t.call(y, op[1])).done) return t; + if (y = 0, t) op = [op[0] & 2, t.value]; + switch (op[0]) { + case 0: case 1: t = op; break; + case 4: _.label++; return { value: op[1], done: false }; + case 5: _.label++; y = op[1]; op = [0]; continue; + case 7: op = _.ops.pop(); _.trys.pop(); continue; + default: + if (!(t = _.trys, t = t.length > 0 && t[t.length - 1]) && (op[0] === 6 || op[0] === 2)) { _ = 0; continue; } + if (op[0] === 3 && (!t || (op[1] > t[0] && op[1] < t[3]))) { _.label = op[1]; break; } + if (op[0] === 6 && _.label < t[1]) { _.label = t[1]; t = op; break; } + if (t && _.label < t[2]) { _.label = t[2]; _.ops.push(op); break; } + if (t[2]) _.ops.pop(); + _.trys.pop(); continue; + } + op = body.call(thisArg, _); + } catch (e) { op = [6, e]; y = 0; } finally { f = t = 0; } + if (op[0] & 5) throw op[1]; return { value: op[0] ? op[1] : void 0, done: true }; + } + } + + function __exportStar(m, exports) { + for (var p in m) if (!exports.hasOwnProperty(p)) exports[p] = m[p]; + } + + function __values(o) { + var m = typeof Symbol === "function" && o[Symbol.iterator], i = 0; + if (m) return m.call(o); + return { + next: function () { + if (o && i >= o.length) o = void 0; + return { value: o && o[i++], done: !o }; + } + }; + } + + function __read(o, n) { + var m = typeof Symbol === "function" && o[Symbol.iterator]; + if (!m) return o; + var i = m.call(o), r, ar = [], e; + try { + while ((n === void 0 || n-- > 0) && !(r = i.next()).done) ar.push(r.value); + } + catch (error) { e = { error: error }; } + finally { + try { + if (r && !r.done && (m = i["return"])) m.call(i); + } + finally { if (e) throw e.error; } + } + return ar; + } + + function __spread() { + for (var ar = [], i = 0; i < arguments.length; i++) + ar = ar.concat(__read(arguments[i])); + return ar; + } + + function __await(v) { + return this instanceof __await ? (this.v = v, this) : new __await(v); + } + + function __asyncGenerator(thisArg, _arguments, generator) { + if (!Symbol.asyncIterator) throw new TypeError("Symbol.asyncIterator is not defined."); + var g = generator.apply(thisArg, _arguments || []), i, q = []; + return i = {}, verb("next"), verb("throw"), verb("return"), i[Symbol.asyncIterator] = function () { return this; }, i; + function verb(n) { if (g[n]) i[n] = function (v) { return new Promise(function (a, b) { q.push([n, v, a, b]) > 1 || resume(n, v); }); }; } + function resume(n, v) { try { step(g[n](v)); } catch (e) { settle(q[0][3], e); } } + function step(r) { r.value instanceof __await ? Promise.resolve(r.value.v).then(fulfill, reject) : settle(q[0][2], r); } + function fulfill(value) { resume("next", value); } + function reject(value) { resume("throw", value); } + function settle(f, v) { if (f(v), q.shift(), q.length) resume(q[0][0], q[0][1]); } + } + + function __asyncDelegator(o) { + var i, p; + return i = {}, verb("next"), verb("throw", function (e) { throw e; }), verb("return"), i[Symbol.iterator] = function () { return this; }, i; + function verb(n, f) { i[n] = o[n] ? function (v) { return (p = !p) ? { value: __await(o[n](v)), done: n === "return" } : f ? f(v) : v; } : f; } + } + + function __asyncValues(o) { + if (!Symbol.asyncIterator) throw new TypeError("Symbol.asyncIterator is not defined."); + var m = o[Symbol.asyncIterator], i; + return m ? m.call(o) : (o = typeof __values === "function" ? __values(o) : o[Symbol.iterator](), i = {}, verb("next"), verb("throw"), verb("return"), i[Symbol.asyncIterator] = function () { return this; }, i); + function verb(n) { i[n] = o[n] && function (v) { return new Promise(function (resolve, reject) { v = o[n](v), settle(resolve, reject, v.done, v.value); }); }; } + function settle(resolve, reject, d, v) { Promise.resolve(v).then(function(v) { resolve({ value: v, done: d }); }, reject); } + } + + function __makeTemplateObject(cooked, raw) { + if (Object.defineProperty) { Object.defineProperty(cooked, "raw", { value: raw }); } else { cooked.raw = raw; } + return cooked; + } + function __importStar(mod) { + if (mod && mod.__esModule) return mod; + var result = {}; + if (mod != null) for (var k in mod) if (Object.hasOwnProperty.call(mod, k)) result[k] = mod[k]; + result.default = mod; + return result; + } + + function __importDefault(mod) { + return (mod && mod.__esModule) ? mod : { default: mod }; + } + + var tslib_1 = /*#__PURE__*/Object.freeze({ + __extends: __extends, + get __assign () { return __assign; }, + __rest: __rest, + __decorate: __decorate, + __param: __param, + __metadata: __metadata, + __awaiter: __awaiter, + __generator: __generator, + __exportStar: __exportStar, + __values: __values, + __read: __read, + __spread: __spread, + __await: __await, + __asyncGenerator: __asyncGenerator, + __asyncDelegator: __asyncDelegator, + __asyncValues: __asyncValues, + __makeTemplateObject: __makeTemplateObject, + __importStar: __importStar, + __importDefault: __importDefault + }); + + /** Console logging verbosity for the SDK. */ + var LogLevel; + (function (LogLevel) { + /** No logs will be generated. */ + LogLevel[LogLevel["None"] = 0] = "None"; + /** Only SDK internal errors will be logged. */ + LogLevel[LogLevel["Error"] = 1] = "Error"; + /** Information useful for debugging the SDK will be logged. */ + LogLevel[LogLevel["Debug"] = 2] = "Debug"; + /** All SDK actions will be logged. */ + LogLevel[LogLevel["Verbose"] = 3] = "Verbose"; + })(LogLevel || (LogLevel = {})); + + /** JSDoc */ + (function (Severity) { + /** JSDoc */ + Severity["Fatal"] = "fatal"; + /** JSDoc */ + Severity["Error"] = "error"; + /** JSDoc */ + Severity["Warning"] = "warning"; + /** JSDoc */ + Severity["Log"] = "log"; + /** JSDoc */ + Severity["Info"] = "info"; + /** JSDoc */ + Severity["Debug"] = "debug"; + /** JSDoc */ + Severity["Critical"] = "critical"; + })(exports.Severity || (exports.Severity = {})); + // tslint:disable:completed-docs + // tslint:disable:no-unnecessary-qualifier no-namespace + (function (Severity) { + /** + * Converts a string-based level into a {@link Severity}. + * + * @param level string representation of Severity + * @returns Severity + */ + function fromString(level) { + switch (level) { + case 'debug': + return Severity.Debug; + case 'info': + return Severity.Info; + case 'warn': + case 'warning': + return Severity.Warning; + case 'error': + return Severity.Error; + case 'fatal': + return Severity.Fatal; + case 'critical': + return Severity.Critical; + case 'log': + default: + return Severity.Log; + } + } + Severity.fromString = fromString; + })(exports.Severity || (exports.Severity = {})); + + /** The status of an Span. */ + var SpanStatus; + (function (SpanStatus) { + /** The operation completed successfully. */ + SpanStatus["Ok"] = "ok"; + /** Deadline expired before operation could complete. */ + SpanStatus["DeadlineExceeded"] = "deadline_exceeded"; + /** 401 Unauthorized (actually does mean unauthenticated according to RFC 7235) */ + SpanStatus["Unauthenticated"] = "unauthenticated"; + /** 403 Forbidden */ + SpanStatus["PermissionDenied"] = "permission_denied"; + /** 404 Not Found. Some requested entity (file or directory) was not found. */ + SpanStatus["NotFound"] = "not_found"; + /** 429 Too Many Requests */ + SpanStatus["ResourceExhausted"] = "resource_exhausted"; + /** Client specified an invalid argument. 4xx. */ + SpanStatus["InvalidArgument"] = "invalid_argument"; + /** 501 Not Implemented */ + SpanStatus["Unimplemented"] = "unimplemented"; + /** 503 Service Unavailable */ + SpanStatus["Unavailable"] = "unavailable"; + /** Other/generic 5xx. */ + SpanStatus["InternalError"] = "internal_error"; + /** Unknown. Any non-standard HTTP status code. */ + SpanStatus["UnknownError"] = "unknown_error"; + /** The operation was cancelled (typically by the user). */ + SpanStatus["Cancelled"] = "cancelled"; + /** Already exists (409) */ + SpanStatus["AlreadyExists"] = "already_exists"; + /** Operation was rejected because the system is not in a state required for the operation's */ + SpanStatus["FailedPrecondition"] = "failed_precondition"; + /** The operation was aborted, typically due to a concurrency issue. */ + SpanStatus["Aborted"] = "aborted"; + /** Operation was attempted past the valid range. */ + SpanStatus["OutOfRange"] = "out_of_range"; + /** Unrecoverable data loss or corruption */ + SpanStatus["DataLoss"] = "data_loss"; + })(SpanStatus || (SpanStatus = {})); + // tslint:disable:no-unnecessary-qualifier no-namespace + (function (SpanStatus) { + /** + * Converts a HTTP status code into a {@link SpanStatus}. + * + * @param httpStatus The HTTP response status code. + * @returns The span status or {@link SpanStatus.UnknownError}. + */ + // tslint:disable-next-line:completed-docs + function fromHttpCode(httpStatus) { + if (httpStatus < 400) { + return SpanStatus.Ok; + } + if (httpStatus >= 400 && httpStatus < 500) { + switch (httpStatus) { + case 401: + return SpanStatus.Unauthenticated; + case 403: + return SpanStatus.PermissionDenied; + case 404: + return SpanStatus.NotFound; + case 409: + return SpanStatus.AlreadyExists; + case 413: + return SpanStatus.FailedPrecondition; + case 429: + return SpanStatus.ResourceExhausted; + default: + return SpanStatus.InvalidArgument; + } + } + if (httpStatus >= 500 && httpStatus < 600) { + switch (httpStatus) { + case 501: + return SpanStatus.Unimplemented; + case 503: + return SpanStatus.Unavailable; + case 504: + return SpanStatus.DeadlineExceeded; + default: + return SpanStatus.InternalError; + } + } + return SpanStatus.UnknownError; + } + SpanStatus.fromHttpCode = fromHttpCode; + })(SpanStatus || (SpanStatus = {})); + + /** The status of an event. */ + (function (Status) { + /** The status could not be determined. */ + Status["Unknown"] = "unknown"; + /** The event was skipped due to configuration or callbacks. */ + Status["Skipped"] = "skipped"; + /** The event was sent to Sentry successfully. */ + Status["Success"] = "success"; + /** The client is currently rate limited and will try again later. */ + Status["RateLimit"] = "rate_limit"; + /** The event could not be processed. */ + Status["Invalid"] = "invalid"; + /** A server-side error ocurred during submission. */ + Status["Failed"] = "failed"; + })(exports.Status || (exports.Status = {})); + // tslint:disable:completed-docs + // tslint:disable:no-unnecessary-qualifier no-namespace + (function (Status) { + /** + * Converts a HTTP status code into a {@link Status}. + * + * @param code The HTTP response status code. + * @returns The send status or {@link Status.Unknown}. + */ + function fromHttpCode(code) { + if (code >= 200 && code < 300) { + return Status.Success; + } + if (code === 429) { + return Status.RateLimit; + } + if (code >= 400 && code < 500) { + return Status.Invalid; + } + if (code >= 500) { + return Status.Failed; + } + return Status.Unknown; + } + Status.fromHttpCode = fromHttpCode; + })(exports.Status || (exports.Status = {})); + + /** + * Consumes the promise and logs the error when it rejects. + * @param promise A promise to forget. + */ + + var setPrototypeOf = Object.setPrototypeOf || ({ __proto__: [] } instanceof Array ? setProtoOf : mixinProperties); // tslint:disable-line:no-unbound-method + /** + * setPrototypeOf polyfill using __proto__ + */ + function setProtoOf(obj, proto) { + // @ts-ignore + obj.__proto__ = proto; + return obj; + } + /** + * setPrototypeOf polyfill using mixin + */ + function mixinProperties(obj, proto) { + for (var prop in proto) { + if (!obj.hasOwnProperty(prop)) { + // @ts-ignore + obj[prop] = proto[prop]; + } + } + return obj; + } + + /** An error emitted by Sentry SDKs and related utilities. */ + var SentryError = /** @class */ (function (_super) { + __extends(SentryError, _super); + function SentryError(message) { + var _newTarget = this.constructor; + var _this = _super.call(this, message) || this; + _this.message = message; + // tslint:disable:no-unsafe-any + _this.name = _newTarget.prototype.constructor.name; + setPrototypeOf(_this, _newTarget.prototype); + return _this; + } + return SentryError; + }(Error)); + + /** + * Checks whether given value's type is one of a few Error or Error-like + * {@link isError}. + * + * @param wat A value to be checked. + * @returns A boolean representing the result. + */ + function isError(wat) { + switch (Object.prototype.toString.call(wat)) { + case '[object Error]': + return true; + case '[object Exception]': + return true; + case '[object DOMException]': + return true; + default: + return isInstanceOf(wat, Error); + } + } + /** + * Checks whether given value's type is ErrorEvent + * {@link isErrorEvent}. + * + * @param wat A value to be checked. + * @returns A boolean representing the result. + */ + function isErrorEvent(wat) { + return Object.prototype.toString.call(wat) === '[object ErrorEvent]'; + } + /** + * Checks whether given value's type is DOMError + * {@link isDOMError}. + * + * @param wat A value to be checked. + * @returns A boolean representing the result. + */ + function isDOMError(wat) { + return Object.prototype.toString.call(wat) === '[object DOMError]'; + } + /** + * Checks whether given value's type is DOMException + * {@link isDOMException}. + * + * @param wat A value to be checked. + * @returns A boolean representing the result. + */ + function isDOMException(wat) { + return Object.prototype.toString.call(wat) === '[object DOMException]'; + } + /** + * Checks whether given value's type is a string + * {@link isString}. + * + * @param wat A value to be checked. + * @returns A boolean representing the result. + */ + function isString(wat) { + return Object.prototype.toString.call(wat) === '[object String]'; + } + /** + * Checks whether given value's is a primitive (undefined, null, number, boolean, string) + * {@link isPrimitive}. + * + * @param wat A value to be checked. + * @returns A boolean representing the result. + */ + function isPrimitive(wat) { + return wat === null || (typeof wat !== 'object' && typeof wat !== 'function'); + } + /** + * Checks whether given value's type is an object literal + * {@link isPlainObject}. + * + * @param wat A value to be checked. + * @returns A boolean representing the result. + */ + function isPlainObject(wat) { + return Object.prototype.toString.call(wat) === '[object Object]'; + } + /** + * Checks whether given value's type is an Event instance + * {@link isEvent}. + * + * @param wat A value to be checked. + * @returns A boolean representing the result. + */ + function isEvent(wat) { + // tslint:disable-next-line:strict-type-predicates + return typeof Event !== 'undefined' && isInstanceOf(wat, Event); + } + /** + * Checks whether given value's type is an Element instance + * {@link isElement}. + * + * @param wat A value to be checked. + * @returns A boolean representing the result. + */ + function isElement(wat) { + // tslint:disable-next-line:strict-type-predicates + return typeof Element !== 'undefined' && isInstanceOf(wat, Element); + } + /** + * Checks whether given value's type is an regexp + * {@link isRegExp}. + * + * @param wat A value to be checked. + * @returns A boolean representing the result. + */ + function isRegExp(wat) { + return Object.prototype.toString.call(wat) === '[object RegExp]'; + } + /** + * Checks whether given value has a then function. + * @param wat A value to be checked. + */ + function isThenable(wat) { + // tslint:disable:no-unsafe-any + return Boolean(wat && wat.then && typeof wat.then === 'function'); + // tslint:enable:no-unsafe-any + } + /** + * Checks whether given value's type is a SyntheticEvent + * {@link isSyntheticEvent}. + * + * @param wat A value to be checked. + * @returns A boolean representing the result. + */ + function isSyntheticEvent(wat) { + // tslint:disable-next-line:no-unsafe-any + return isPlainObject(wat) && 'nativeEvent' in wat && 'preventDefault' in wat && 'stopPropagation' in wat; + } + /** + * Checks whether given value's type is an instance of provided constructor. + * {@link isInstanceOf}. + * + * @param wat A value to be checked. + * @param base A constructor to be used in a check. + * @returns A boolean representing the result. + */ + function isInstanceOf(wat, base) { + try { + // tslint:disable-next-line:no-unsafe-any + return wat instanceof base; + } + catch (_e) { + return false; + } + } + + /** + * Truncates given string to the maximum characters count + * + * @param str An object that contains serializable values + * @param max Maximum number of characters in truncated string + * @returns string Encoded + */ + function truncate(str, max) { + if (max === void 0) { max = 0; } + // tslint:disable-next-line:strict-type-predicates + if (typeof str !== 'string' || max === 0) { + return str; + } + return str.length <= max ? str : str.substr(0, max) + "..."; + } + /** + * Join values in array + * @param input array of values to be joined together + * @param delimiter string to be placed in-between values + * @returns Joined values + */ + function safeJoin(input, delimiter) { + if (!Array.isArray(input)) { + return ''; + } + var output = []; + // tslint:disable-next-line:prefer-for-of + for (var i = 0; i < input.length; i++) { + var value = input[i]; + try { + output.push(String(value)); + } + catch (e) { + output.push('[value cannot be serialized]'); + } + } + return output.join(delimiter); + } + /** + * Checks if the value matches a regex or includes the string + * @param value The string value to be checked against + * @param pattern Either a regex or a string that must be contained in value + */ + function isMatchingPattern(value, pattern) { + if (!isString(value)) { + return false; + } + if (isRegExp(pattern)) { + return pattern.test(value); + } + if (typeof pattern === 'string') { + return value.indexOf(pattern) !== -1; + } + return false; + } + + /** + * Requires a module which is protected against bundler minification. + * + * @param request The module path to resolve + */ + function dynamicRequire(mod, request) { + // tslint:disable-next-line: no-unsafe-any + return mod.require(request); + } + /** + * Checks whether we're in the Node.js or Browser environment + * + * @returns Answer to given question + */ + function isNodeEnv() { + // tslint:disable:strict-type-predicates + return Object.prototype.toString.call(typeof process !== 'undefined' ? process : 0) === '[object process]'; + } + var fallbackGlobalObject = {}; + /** + * Safely get global scope object + * + * @returns Global scope object + */ + function getGlobalObject() { + return (isNodeEnv() + ? global + : typeof window !== 'undefined' + ? window + : typeof self !== 'undefined' + ? self + : fallbackGlobalObject); + } + /** + * UUID4 generator + * + * @returns string Generated UUID4. + */ + function uuid4() { + var global = getGlobalObject(); + var crypto = global.crypto || global.msCrypto; + if (!(crypto === void 0) && crypto.getRandomValues) { + // Use window.crypto API if available + var arr = new Uint16Array(8); + crypto.getRandomValues(arr); + // set 4 in byte 7 + // tslint:disable-next-line:no-bitwise + arr[3] = (arr[3] & 0xfff) | 0x4000; + // set 2 most significant bits of byte 9 to '10' + // tslint:disable-next-line:no-bitwise + arr[4] = (arr[4] & 0x3fff) | 0x8000; + var pad = function (num) { + var v = num.toString(16); + while (v.length < 4) { + v = "0" + v; + } + return v; + }; + return (pad(arr[0]) + pad(arr[1]) + pad(arr[2]) + pad(arr[3]) + pad(arr[4]) + pad(arr[5]) + pad(arr[6]) + pad(arr[7])); + } + // http://stackoverflow.com/questions/105034/how-to-create-a-guid-uuid-in-javascript/2117523#2117523 + return 'xxxxxxxxxxxx4xxxyxxxxxxxxxxxxxxx'.replace(/[xy]/g, function (c) { + // tslint:disable-next-line:no-bitwise + var r = (Math.random() * 16) | 0; + // tslint:disable-next-line:no-bitwise + var v = c === 'x' ? r : (r & 0x3) | 0x8; + return v.toString(16); + }); + } + /** + * Parses string form of URL into an object + * // borrowed from https://tools.ietf.org/html/rfc3986#appendix-B + * // intentionally using regex and not href parsing trick because React Native and other + * // environments where DOM might not be available + * @returns parsed URL object + */ + function parseUrl(url) { + if (!url) { + return {}; + } + var match = url.match(/^(([^:\/?#]+):)?(\/\/([^\/?#]*))?([^?#]*)(\?([^#]*))?(#(.*))?$/); + if (!match) { + return {}; + } + // coerce to undefined values to empty string so we don't get 'undefined' + var query = match[6] || ''; + var fragment = match[8] || ''; + return { + host: match[4], + path: match[5], + protocol: match[2], + relative: match[5] + query + fragment, + }; + } + /** + * Extracts either message or type+value from an event that can be used for user-facing logs + * @returns event's description + */ + function getEventDescription(event) { + if (event.message) { + return event.message; + } + if (event.exception && event.exception.values && event.exception.values[0]) { + var exception = event.exception.values[0]; + if (exception.type && exception.value) { + return exception.type + ": " + exception.value; + } + return exception.type || exception.value || event.event_id || ''; + } + return event.event_id || ''; + } + /** JSDoc */ + function consoleSandbox(callback) { + var global = getGlobalObject(); + var levels = ['debug', 'info', 'warn', 'error', 'log', 'assert']; + if (!('console' in global)) { + return callback(); + } + var originalConsole = global.console; + var wrappedLevels = {}; + // Restore all wrapped console methods + levels.forEach(function (level) { + if (level in global.console && originalConsole[level].__sentry_original__) { + wrappedLevels[level] = originalConsole[level]; + originalConsole[level] = originalConsole[level].__sentry_original__; + } + }); + // Perform callback manipulations + var result = callback(); + // Revert restoration to wrapped state + Object.keys(wrappedLevels).forEach(function (level) { + originalConsole[level] = wrappedLevels[level]; + }); + return result; + } + /** + * Adds exception values, type and value to an synthetic Exception. + * @param event The event to modify. + * @param value Value of the exception. + * @param type Type of the exception. + * @hidden + */ + function addExceptionTypeValue(event, value, type) { + event.exception = event.exception || {}; + event.exception.values = event.exception.values || []; + event.exception.values[0] = event.exception.values[0] || {}; + event.exception.values[0].value = event.exception.values[0].value || value || ''; + event.exception.values[0].type = event.exception.values[0].type || type || 'Error'; + } + /** + * Adds exception mechanism to a given event. + * @param event The event to modify. + * @param mechanism Mechanism of the mechanism. + * @hidden + */ + function addExceptionMechanism(event, mechanism) { + if (mechanism === void 0) { mechanism = {}; } + // TODO: Use real type with `keyof Mechanism` thingy and maybe make it better? + try { + // @ts-ignore + // tslint:disable:no-non-null-assertion + event.exception.values[0].mechanism = event.exception.values[0].mechanism || {}; + Object.keys(mechanism).forEach(function (key) { + // @ts-ignore + event.exception.values[0].mechanism[key] = mechanism[key]; + }); + } + catch (_oO) { + // no-empty + } + } + /** + * A safe form of location.href + */ + function getLocationHref() { + try { + return document.location.href; + } + catch (oO) { + return ''; + } + } + /** + * Given a child DOM element, returns a query-selector statement describing that + * and its ancestors + * e.g. [HTMLElement] => body > div > input#foo.btn[name=baz] + * @returns generated DOM path + */ + function htmlTreeAsString(elem) { + // try/catch both: + // - accessing event.target (see getsentry/raven-js#838, #768) + // - `htmlTreeAsString` because it's complex, and just accessing the DOM incorrectly + // - can throw an exception in some circumstances. + try { + var currentElem = elem; + var MAX_TRAVERSE_HEIGHT = 5; + var MAX_OUTPUT_LEN = 80; + var out = []; + var height = 0; + var len = 0; + var separator = ' > '; + var sepLength = separator.length; + var nextStr = void 0; + while (currentElem && height++ < MAX_TRAVERSE_HEIGHT) { + nextStr = _htmlElementAsString(currentElem); + // bail out if + // - nextStr is the 'html' element + // - the length of the string that would be created exceeds MAX_OUTPUT_LEN + // (ignore this limit if we are on the first iteration) + if (nextStr === 'html' || (height > 1 && len + out.length * sepLength + nextStr.length >= MAX_OUTPUT_LEN)) { + break; + } + out.push(nextStr); + len += nextStr.length; + currentElem = currentElem.parentNode; + } + return out.reverse().join(separator); + } + catch (_oO) { + return ''; + } + } + /** + * Returns a simple, query-selector representation of a DOM element + * e.g. [HTMLElement] => input#foo.btn[name=baz] + * @returns generated DOM path + */ + function _htmlElementAsString(el) { + var elem = el; + var out = []; + var className; + var classes; + var key; + var attr; + var i; + if (!elem || !elem.tagName) { + return ''; + } + out.push(elem.tagName.toLowerCase()); + if (elem.id) { + out.push("#" + elem.id); + } + className = elem.className; + if (className && isString(className)) { + classes = className.split(/\s+/); + for (i = 0; i < classes.length; i++) { + out.push("." + classes[i]); + } + } + var attrWhitelist = ['type', 'name', 'title', 'alt']; + for (i = 0; i < attrWhitelist.length; i++) { + key = attrWhitelist[i]; + attr = elem.getAttribute(key); + if (attr) { + out.push("[" + key + "=\"" + attr + "\"]"); + } + } + return out.join(''); + } + var INITIAL_TIME = Date.now(); + var prevNow = 0; + var performanceFallback = { + now: function () { + var now = Date.now() - INITIAL_TIME; + if (now < prevNow) { + now = prevNow; + } + prevNow = now; + return now; + }, + timeOrigin: INITIAL_TIME, + }; + var crossPlatformPerformance = (function () { + if (isNodeEnv()) { + try { + var perfHooks = dynamicRequire(module, 'perf_hooks'); + return perfHooks.performance; + } + catch (_) { + return performanceFallback; + } + } + if (getGlobalObject().performance) { + // Polyfill for performance.timeOrigin. + // + // While performance.timing.navigationStart is deprecated in favor of performance.timeOrigin, performance.timeOrigin + // is not as widely supported. Namely, performance.timeOrigin is undefined in Safari as of writing. + // tslint:disable-next-line:strict-type-predicates + if (performance.timeOrigin === undefined) { + // As of writing, performance.timing is not available in Web Workers in mainstream browsers, so it is not always a + // valid fallback. In the absence of a initial time provided by the browser, fallback to INITIAL_TIME. + // @ts-ignore + // tslint:disable-next-line:deprecation + performance.timeOrigin = (performance.timing && performance.timing.navigationStart) || INITIAL_TIME; + } + } + return getGlobalObject().performance || performanceFallback; + })(); + /** + * Returns a timestamp in seconds with milliseconds precision since the UNIX epoch calculated with the monotonic clock. + */ + function timestampWithMs() { + return (crossPlatformPerformance.timeOrigin + crossPlatformPerformance.now()) / 1000; + } + var defaultRetryAfter = 60 * 1000; // 60 seconds + /** + * Extracts Retry-After value from the request header or returns default value + * @param now current unix timestamp + * @param header string representation of 'Retry-After' header + */ + function parseRetryAfterHeader(now, header) { + if (!header) { + return defaultRetryAfter; + } + var headerDelay = parseInt("" + header, 10); + if (!isNaN(headerDelay)) { + return headerDelay * 1000; + } + var headerDate = Date.parse("" + header); + if (!isNaN(headerDate)) { + return headerDate - now; + } + return defaultRetryAfter; + } + var defaultFunctionName = ''; + /** + * Safely extract function name from itself + */ + function getFunctionName(fn) { + try { + if (!fn || typeof fn !== 'function') { + return defaultFunctionName; + } + return fn.name || defaultFunctionName; + } + catch (e) { + // Just accessing custom props in some Selenium environments + // can cause a "Permission denied" exception (see raven-js#495). + return defaultFunctionName; + } + } + + // TODO: Implement different loggers for different environments + var global$1 = getGlobalObject(); + /** Prefix for logging strings */ + var PREFIX = 'Sentry Logger '; + /** JSDoc */ + var Logger = /** @class */ (function () { + /** JSDoc */ + function Logger() { + this._enabled = false; + } + /** JSDoc */ + Logger.prototype.disable = function () { + this._enabled = false; + }; + /** JSDoc */ + Logger.prototype.enable = function () { + this._enabled = true; + }; + /** JSDoc */ + Logger.prototype.log = function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + if (!this._enabled) { + return; + } + consoleSandbox(function () { + global$1.console.log(PREFIX + "[Log]: " + args.join(' ')); // tslint:disable-line:no-console + }); + }; + /** JSDoc */ + Logger.prototype.warn = function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + if (!this._enabled) { + return; + } + consoleSandbox(function () { + global$1.console.warn(PREFIX + "[Warn]: " + args.join(' ')); // tslint:disable-line:no-console + }); + }; + /** JSDoc */ + Logger.prototype.error = function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + if (!this._enabled) { + return; + } + consoleSandbox(function () { + global$1.console.error(PREFIX + "[Error]: " + args.join(' ')); // tslint:disable-line:no-console + }); + }; + return Logger; + }()); + // Ensure we only have a single logger instance, even if multiple versions of @sentry/utils are being used + global$1.__SENTRY__ = global$1.__SENTRY__ || {}; + var logger = global$1.__SENTRY__.logger || (global$1.__SENTRY__.logger = new Logger()); + + // tslint:disable:no-unsafe-any + /** + * Memo class used for decycle json objects. Uses WeakSet if available otherwise array. + */ + var Memo = /** @class */ (function () { + function Memo() { + // tslint:disable-next-line + this._hasWeakSet = typeof WeakSet === 'function'; + this._inner = this._hasWeakSet ? new WeakSet() : []; + } + /** + * Sets obj to remember. + * @param obj Object to remember + */ + Memo.prototype.memoize = function (obj) { + if (this._hasWeakSet) { + if (this._inner.has(obj)) { + return true; + } + this._inner.add(obj); + return false; + } + // tslint:disable-next-line:prefer-for-of + for (var i = 0; i < this._inner.length; i++) { + var value = this._inner[i]; + if (value === obj) { + return true; + } + } + this._inner.push(obj); + return false; + }; + /** + * Removes object from internal storage. + * @param obj Object to forget + */ + Memo.prototype.unmemoize = function (obj) { + if (this._hasWeakSet) { + this._inner.delete(obj); + } + else { + for (var i = 0; i < this._inner.length; i++) { + if (this._inner[i] === obj) { + this._inner.splice(i, 1); + break; + } + } + } + }; + return Memo; + }()); + + /** + * Wrap a given object method with a higher-order function + * + * @param source An object that contains a method to be wrapped. + * @param name A name of method to be wrapped. + * @param replacement A function that should be used to wrap a given method. + * @returns void + */ + function fill(source, name, replacement) { + if (!(name in source)) { + return; + } + var original = source[name]; + var wrapped = replacement(original); + // Make sure it's a function first, as we need to attach an empty prototype for `defineProperties` to work + // otherwise it'll throw "TypeError: Object.defineProperties called on non-object" + // tslint:disable-next-line:strict-type-predicates + if (typeof wrapped === 'function') { + try { + wrapped.prototype = wrapped.prototype || {}; + Object.defineProperties(wrapped, { + __sentry_original__: { + enumerable: false, + value: original, + }, + }); + } + catch (_Oo) { + // This can throw if multiple fill happens on a global object like XMLHttpRequest + // Fixes https://github.com/getsentry/sentry-javascript/issues/2043 + } + } + source[name] = wrapped; + } + /** + * Encodes given object into url-friendly format + * + * @param object An object that contains serializable values + * @returns string Encoded + */ + function urlEncode(object) { + return Object.keys(object) + .map( + // tslint:disable-next-line:no-unsafe-any + function (key) { return encodeURIComponent(key) + "=" + encodeURIComponent(object[key]); }) + .join('&'); + } + /** + * Transforms any object into an object literal with all it's attributes + * attached to it. + * + * @param value Initial source that we have to transform in order to be usable by the serializer + */ + function getWalkSource(value) { + if (isError(value)) { + var error = value; + var err = { + message: error.message, + name: error.name, + stack: error.stack, + }; + for (var i in error) { + if (Object.prototype.hasOwnProperty.call(error, i)) { + err[i] = error[i]; + } + } + return err; + } + if (isEvent(value)) { + var event_1 = value; + var source = {}; + source.type = event_1.type; + // Accessing event.target can throw (see getsentry/raven-js#838, #768) + try { + source.target = isElement(event_1.target) + ? htmlTreeAsString(event_1.target) + : Object.prototype.toString.call(event_1.target); + } + catch (_oO) { + source.target = ''; + } + try { + source.currentTarget = isElement(event_1.currentTarget) + ? htmlTreeAsString(event_1.currentTarget) + : Object.prototype.toString.call(event_1.currentTarget); + } + catch (_oO) { + source.currentTarget = ''; + } + // tslint:disable-next-line:strict-type-predicates + if (typeof CustomEvent !== 'undefined' && isInstanceOf(value, CustomEvent)) { + source.detail = event_1.detail; + } + for (var i in event_1) { + if (Object.prototype.hasOwnProperty.call(event_1, i)) { + source[i] = event_1; + } + } + return source; + } + return value; + } + /** Calculates bytes size of input string */ + function utf8Length(value) { + // tslint:disable-next-line:no-bitwise + return ~-encodeURI(value).split(/%..|./).length; + } + /** Calculates bytes size of input object */ + function jsonSize(value) { + return utf8Length(JSON.stringify(value)); + } + /** JSDoc */ + function normalizeToSize(object, + // Default Node.js REPL depth + depth, + // 100kB, as 200kB is max payload size, so half sounds reasonable + maxSize) { + if (depth === void 0) { depth = 3; } + if (maxSize === void 0) { maxSize = 100 * 1024; } + var serialized = normalize(object, depth); + if (jsonSize(serialized) > maxSize) { + return normalizeToSize(object, depth - 1, maxSize); + } + return serialized; + } + /** Transforms any input value into a string form, either primitive value or a type of the input */ + function serializeValue(value) { + var type = Object.prototype.toString.call(value); + // Node.js REPL notation + if (typeof value === 'string') { + return value; + } + if (type === '[object Object]') { + return '[Object]'; + } + if (type === '[object Array]') { + return '[Array]'; + } + var normalized = normalizeValue(value); + return isPrimitive(normalized) ? normalized : type; + } + /** + * normalizeValue() + * + * Takes unserializable input and make it serializable friendly + * + * - translates undefined/NaN values to "[undefined]"/"[NaN]" respectively, + * - serializes Error objects + * - filter global objects + */ + // tslint:disable-next-line:cyclomatic-complexity + function normalizeValue(value, key) { + if (key === 'domain' && value && typeof value === 'object' && value._events) { + return '[Domain]'; + } + if (key === 'domainEmitter') { + return '[DomainEmitter]'; + } + if (typeof global !== 'undefined' && value === global) { + return '[Global]'; + } + if (typeof window !== 'undefined' && value === window) { + return '[Window]'; + } + if (typeof document !== 'undefined' && value === document) { + return '[Document]'; + } + // React's SyntheticEvent thingy + if (isSyntheticEvent(value)) { + return '[SyntheticEvent]'; + } + // tslint:disable-next-line:no-tautology-expression + if (typeof value === 'number' && value !== value) { + return '[NaN]'; + } + if (value === void 0) { + return '[undefined]'; + } + if (typeof value === 'function') { + return "[Function: " + getFunctionName(value) + "]"; + } + return value; + } + /** + * Walks an object to perform a normalization on it + * + * @param key of object that's walked in current iteration + * @param value object to be walked + * @param depth Optional number indicating how deep should walking be performed + * @param memo Optional Memo class handling decycling + */ + function walk(key, value, depth, memo) { + if (depth === void 0) { depth = +Infinity; } + if (memo === void 0) { memo = new Memo(); } + // If we reach the maximum depth, serialize whatever has left + if (depth === 0) { + return serializeValue(value); + } + // If value implements `toJSON` method, call it and return early + // tslint:disable:no-unsafe-any + if (value !== null && value !== undefined && typeof value.toJSON === 'function') { + return value.toJSON(); + } + // tslint:enable:no-unsafe-any + // If normalized value is a primitive, there are no branches left to walk, so we can just bail out, as theres no point in going down that branch any further + var normalized = normalizeValue(value, key); + if (isPrimitive(normalized)) { + return normalized; + } + // Create source that we will use for next itterations, either objectified error object (Error type with extracted keys:value pairs) or the input itself + var source = getWalkSource(value); + // Create an accumulator that will act as a parent for all future itterations of that branch + var acc = Array.isArray(value) ? [] : {}; + // If we already walked that branch, bail out, as it's circular reference + if (memo.memoize(value)) { + return '[Circular ~]'; + } + // Walk all keys of the source + for (var innerKey in source) { + // Avoid iterating over fields in the prototype if they've somehow been exposed to enumeration. + if (!Object.prototype.hasOwnProperty.call(source, innerKey)) { + continue; + } + // Recursively walk through all the child nodes + acc[innerKey] = walk(innerKey, source[innerKey], depth - 1, memo); + } + // Once walked through all the branches, remove the parent from memo storage + memo.unmemoize(value); + // Return accumulated values + return acc; + } + /** + * normalize() + * + * - Creates a copy to prevent original input mutation + * - Skip non-enumerablers + * - Calls `toJSON` if implemented + * - Removes circular references + * - Translates non-serializeable values (undefined/NaN/Functions) to serializable format + * - Translates known global objects/Classes to a string representations + * - Takes care of Error objects serialization + * - Optionally limit depth of final output + */ + function normalize(input, depth) { + try { + // tslint:disable-next-line:no-unsafe-any + return JSON.parse(JSON.stringify(input, function (key, value) { return walk(key, value, depth); })); + } + catch (_oO) { + return '**non-serializable**'; + } + } + /** + * Given any captured exception, extract its keys and create a sorted + * and truncated list that will be used inside the event message. + * eg. `Non-error exception captured with keys: foo, bar, baz` + */ + function extractExceptionKeysForMessage(exception, maxLength) { + if (maxLength === void 0) { maxLength = 40; } + // tslint:disable:strict-type-predicates + var keys = Object.keys(getWalkSource(exception)); + keys.sort(); + if (!keys.length) { + return '[object has no keys]'; + } + if (keys[0].length >= maxLength) { + return truncate(keys[0], maxLength); + } + for (var includedKeys = keys.length; includedKeys > 0; includedKeys--) { + var serialized = keys.slice(0, includedKeys).join(', '); + if (serialized.length > maxLength) { + continue; + } + if (includedKeys === keys.length) { + return serialized; + } + return truncate(serialized, maxLength); + } + return ''; + } + + // Slightly modified (no IE8 support, ES6) and transcribed to TypeScript + + /** SyncPromise internal states */ + var States; + (function (States) { + /** Pending */ + States["PENDING"] = "PENDING"; + /** Resolved / OK */ + States["RESOLVED"] = "RESOLVED"; + /** Rejected / Error */ + States["REJECTED"] = "REJECTED"; + })(States || (States = {})); + /** + * Thenable class that behaves like a Promise and follows it's interface + * but is not async internally + */ + var SyncPromise = /** @class */ (function () { + function SyncPromise(executor) { + var _this = this; + this._state = States.PENDING; + this._handlers = []; + /** JSDoc */ + this._resolve = function (value) { + _this._setResult(States.RESOLVED, value); + }; + /** JSDoc */ + this._reject = function (reason) { + _this._setResult(States.REJECTED, reason); + }; + /** JSDoc */ + this._setResult = function (state, value) { + if (_this._state !== States.PENDING) { + return; + } + if (isThenable(value)) { + value.then(_this._resolve, _this._reject); + return; + } + _this._state = state; + _this._value = value; + _this._executeHandlers(); + }; + // TODO: FIXME + /** JSDoc */ + this._attachHandler = function (handler) { + _this._handlers = _this._handlers.concat(handler); + _this._executeHandlers(); + }; + /** JSDoc */ + this._executeHandlers = function () { + if (_this._state === States.PENDING) { + return; + } + var cachedHandlers = _this._handlers.slice(); + _this._handlers = []; + cachedHandlers.forEach(function (handler) { + if (handler.done) { + return; + } + if (_this._state === States.RESOLVED) { + if (handler.onfulfilled) { + handler.onfulfilled(_this._value); + } + } + if (_this._state === States.REJECTED) { + if (handler.onrejected) { + handler.onrejected(_this._value); + } + } + handler.done = true; + }); + }; + try { + executor(this._resolve, this._reject); + } + catch (e) { + this._reject(e); + } + } + /** JSDoc */ + SyncPromise.prototype.toString = function () { + return '[object SyncPromise]'; + }; + /** JSDoc */ + SyncPromise.resolve = function (value) { + return new SyncPromise(function (resolve) { + resolve(value); + }); + }; + /** JSDoc */ + SyncPromise.reject = function (reason) { + return new SyncPromise(function (_, reject) { + reject(reason); + }); + }; + /** JSDoc */ + SyncPromise.all = function (collection) { + return new SyncPromise(function (resolve, reject) { + if (!Array.isArray(collection)) { + reject(new TypeError("Promise.all requires an array as input.")); + return; + } + if (collection.length === 0) { + resolve([]); + return; + } + var counter = collection.length; + var resolvedCollection = []; + collection.forEach(function (item, index) { + SyncPromise.resolve(item) + .then(function (value) { + resolvedCollection[index] = value; + counter -= 1; + if (counter !== 0) { + return; + } + resolve(resolvedCollection); + }) + .then(null, reject); + }); + }); + }; + /** JSDoc */ + SyncPromise.prototype.then = function (onfulfilled, onrejected) { + var _this = this; + return new SyncPromise(function (resolve, reject) { + _this._attachHandler({ + done: false, + onfulfilled: function (result) { + if (!onfulfilled) { + // TODO: ¯\_(ツ)_/¯ + // TODO: FIXME + resolve(result); + return; + } + try { + resolve(onfulfilled(result)); + return; + } + catch (e) { + reject(e); + return; + } + }, + onrejected: function (reason) { + if (!onrejected) { + reject(reason); + return; + } + try { + resolve(onrejected(reason)); + return; + } + catch (e) { + reject(e); + return; + } + }, + }); + }); + }; + /** JSDoc */ + SyncPromise.prototype.catch = function (onrejected) { + return this.then(function (val) { return val; }, onrejected); + }; + /** JSDoc */ + SyncPromise.prototype.finally = function (onfinally) { + var _this = this; + return new SyncPromise(function (resolve, reject) { + var val; + var isRejected; + return _this.then(function (value) { + isRejected = false; + val = value; + if (onfinally) { + onfinally(); + } + }, function (reason) { + isRejected = true; + val = reason; + if (onfinally) { + onfinally(); + } + }).then(function () { + if (isRejected) { + reject(val); + return; + } + resolve(val); + }); + }); + }; + return SyncPromise; + }()); + + /** A simple queue that holds promises. */ + var PromiseBuffer = /** @class */ (function () { + function PromiseBuffer(_limit) { + this._limit = _limit; + /** Internal set of queued Promises */ + this._buffer = []; + } + /** + * Says if the buffer is ready to take more requests + */ + PromiseBuffer.prototype.isReady = function () { + return this._limit === undefined || this.length() < this._limit; + }; + /** + * Add a promise to the queue. + * + * @param task Can be any PromiseLike + * @returns The original promise. + */ + PromiseBuffer.prototype.add = function (task) { + var _this = this; + if (!this.isReady()) { + return SyncPromise.reject(new SentryError('Not adding Promise due to buffer limit reached.')); + } + if (this._buffer.indexOf(task) === -1) { + this._buffer.push(task); + } + task + .then(function () { return _this.remove(task); }) + .then(null, function () { + return _this.remove(task).then(null, function () { + // We have to add this catch here otherwise we have an unhandledPromiseRejection + // because it's a new Promise chain. + }); + }); + return task; + }; + /** + * Remove a promise to the queue. + * + * @param task Can be any PromiseLike + * @returns Removed promise. + */ + PromiseBuffer.prototype.remove = function (task) { + var removedTask = this._buffer.splice(this._buffer.indexOf(task), 1)[0]; + return removedTask; + }; + /** + * This function returns the number of unresolved promises in the queue. + */ + PromiseBuffer.prototype.length = function () { + return this._buffer.length; + }; + /** + * This will drain the whole queue, returns true if queue is empty or drained. + * If timeout is provided and the queue takes longer to drain, the promise still resolves but with false. + * + * @param timeout Number in ms to wait until it resolves with false. + */ + PromiseBuffer.prototype.drain = function (timeout) { + var _this = this; + return new SyncPromise(function (resolve) { + var capturedSetTimeout = setTimeout(function () { + if (timeout && timeout > 0) { + resolve(false); + } + }, timeout); + SyncPromise.all(_this._buffer) + .then(function () { + clearTimeout(capturedSetTimeout); + resolve(true); + }) + .then(null, function () { + resolve(true); + }); + }); + }; + return PromiseBuffer; + }()); + + /** + * Tells whether current environment supports Fetch API + * {@link supportsFetch}. + * + * @returns Answer to the given question. + */ + function supportsFetch() { + if (!('fetch' in getGlobalObject())) { + return false; + } + try { + // tslint:disable-next-line:no-unused-expression + new Headers(); + // tslint:disable-next-line:no-unused-expression + new Request(''); + // tslint:disable-next-line:no-unused-expression + new Response(); + return true; + } + catch (e) { + return false; + } + } + /** + * isNativeFetch checks if the given function is a native implementation of fetch() + */ + function isNativeFetch(func) { + return func && /^function fetch\(\)\s+\{\s+\[native code\]\s+\}$/.test(func.toString()); + } + /** + * Tells whether current environment supports Fetch API natively + * {@link supportsNativeFetch}. + * + * @returns true if `window.fetch` is natively implemented, false otherwise + */ + function supportsNativeFetch() { + if (!supportsFetch()) { + return false; + } + var global = getGlobalObject(); + // Fast path to avoid DOM I/O + // tslint:disable-next-line:no-unbound-method + if (isNativeFetch(global.fetch)) { + return true; + } + // window.fetch is implemented, but is polyfilled or already wrapped (e.g: by a chrome extension) + // so create a "pure" iframe to see if that has native fetch + var result = false; + var doc = global.document; + // tslint:disable-next-line:no-unbound-method deprecation + if (doc && typeof doc.createElement === "function") { + try { + var sandbox = doc.createElement('iframe'); + sandbox.hidden = true; + doc.head.appendChild(sandbox); + if (sandbox.contentWindow && sandbox.contentWindow.fetch) { + // tslint:disable-next-line:no-unbound-method + result = isNativeFetch(sandbox.contentWindow.fetch); + } + doc.head.removeChild(sandbox); + } + catch (err) { + logger.warn('Could not create sandbox iframe for pure fetch check, bailing to window.fetch: ', err); + } + } + return result; + } + /** + * Tells whether current environment supports Referrer Policy API + * {@link supportsReferrerPolicy}. + * + * @returns Answer to the given question. + */ + function supportsReferrerPolicy() { + // Despite all stars in the sky saying that Edge supports old draft syntax, aka 'never', 'always', 'origin' and 'default + // https://caniuse.com/#feat=referrer-policy + // It doesn't. And it throw exception instead of ignoring this parameter... + // REF: https://github.com/getsentry/raven-js/issues/1233 + if (!supportsFetch()) { + return false; + } + try { + // tslint:disable:no-unused-expression + new Request('_', { + referrerPolicy: 'origin', + }); + return true; + } + catch (e) { + return false; + } + } + /** + * Tells whether current environment supports History API + * {@link supportsHistory}. + * + * @returns Answer to the given question. + */ + function supportsHistory() { + // NOTE: in Chrome App environment, touching history.pushState, *even inside + // a try/catch block*, will cause Chrome to output an error to console.error + // borrowed from: https://github.com/angular/angular.js/pull/13945/files + var global = getGlobalObject(); + var chrome = global.chrome; + // tslint:disable-next-line:no-unsafe-any + var isChromePackagedApp = chrome && chrome.app && chrome.app.runtime; + var hasHistoryApi = 'history' in global && !!global.history.pushState && !!global.history.replaceState; + return !isChromePackagedApp && hasHistoryApi; + } + + /* tslint:disable:only-arrow-functions no-unsafe-any */ + var global$2 = getGlobalObject(); + /** + * Instrument native APIs to call handlers that can be used to create breadcrumbs, APM spans etc. + * - Console API + * - Fetch API + * - XHR API + * - History API + * - DOM API (click/typing) + * - Error API + * - UnhandledRejection API + */ + var handlers = {}; + var instrumented = {}; + /** Instruments given API */ + function instrument(type) { + if (instrumented[type]) { + return; + } + instrumented[type] = true; + switch (type) { + case 'console': + instrumentConsole(); + break; + case 'dom': + instrumentDOM(); + break; + case 'xhr': + instrumentXHR(); + break; + case 'fetch': + instrumentFetch(); + break; + case 'history': + instrumentHistory(); + break; + case 'error': + instrumentError(); + break; + case 'unhandledrejection': + instrumentUnhandledRejection(); + break; + default: + logger.warn('unknown instrumentation type:', type); + } + } + /** + * Add handler that will be called when given type of instrumentation triggers. + * Use at your own risk, this might break without changelog notice, only used internally. + * @hidden + */ + function addInstrumentationHandler(handler) { + // tslint:disable-next-line:strict-type-predicates + if (!handler || typeof handler.type !== 'string' || typeof handler.callback !== 'function') { + return; + } + handlers[handler.type] = handlers[handler.type] || []; + handlers[handler.type].push(handler.callback); + instrument(handler.type); + } + /** JSDoc */ + function triggerHandlers(type, data) { + var e_1, _a; + if (!type || !handlers[type]) { + return; + } + try { + for (var _b = __values(handlers[type] || []), _c = _b.next(); !_c.done; _c = _b.next()) { + var handler = _c.value; + try { + handler(data); + } + catch (e) { + logger.error("Error while triggering instrumentation handler.\nType: " + type + "\nName: " + getFunctionName(handler) + "\nError: " + e); + } + } + } + catch (e_1_1) { e_1 = { error: e_1_1 }; } + finally { + try { + if (_c && !_c.done && (_a = _b.return)) _a.call(_b); + } + finally { if (e_1) throw e_1.error; } + } + } + /** JSDoc */ + function instrumentConsole() { + if (!('console' in global$2)) { + return; + } + ['debug', 'info', 'warn', 'error', 'log', 'assert'].forEach(function (level) { + if (!(level in global$2.console)) { + return; + } + fill(global$2.console, level, function (originalConsoleLevel) { + return function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + triggerHandlers('console', { args: args, level: level }); + // this fails for some browsers. :( + if (originalConsoleLevel) { + Function.prototype.apply.call(originalConsoleLevel, global$2.console, args); + } + }; + }); + }); + } + /** JSDoc */ + function instrumentFetch() { + if (!supportsNativeFetch()) { + return; + } + fill(global$2, 'fetch', function (originalFetch) { + return function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + var commonHandlerData = { + args: args, + fetchData: { + method: getFetchMethod(args), + url: getFetchUrl(args), + }, + startTimestamp: Date.now(), + }; + triggerHandlers('fetch', __assign({}, commonHandlerData)); + return originalFetch.apply(global$2, args).then(function (response) { + triggerHandlers('fetch', __assign({}, commonHandlerData, { endTimestamp: Date.now(), response: response })); + return response; + }, function (error) { + triggerHandlers('fetch', __assign({}, commonHandlerData, { endTimestamp: Date.now(), error: error })); + throw error; + }); + }; + }); + } + /** Extract `method` from fetch call arguments */ + function getFetchMethod(fetchArgs) { + if (fetchArgs === void 0) { fetchArgs = []; } + if ('Request' in global$2 && isInstanceOf(fetchArgs[0], Request) && fetchArgs[0].method) { + return String(fetchArgs[0].method).toUpperCase(); + } + if (fetchArgs[1] && fetchArgs[1].method) { + return String(fetchArgs[1].method).toUpperCase(); + } + return 'GET'; + } + /** Extract `url` from fetch call arguments */ + function getFetchUrl(fetchArgs) { + if (fetchArgs === void 0) { fetchArgs = []; } + if (typeof fetchArgs[0] === 'string') { + return fetchArgs[0]; + } + if ('Request' in global$2 && isInstanceOf(fetchArgs[0], Request)) { + return fetchArgs[0].url; + } + return String(fetchArgs[0]); + } + /** JSDoc */ + function instrumentXHR() { + if (!('XMLHttpRequest' in global$2)) { + return; + } + var xhrproto = XMLHttpRequest.prototype; + fill(xhrproto, 'open', function (originalOpen) { + return function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + var url = args[1]; + this.__sentry_xhr__ = { + method: isString(args[0]) ? args[0].toUpperCase() : args[0], + url: args[1], + }; + // if Sentry key appears in URL, don't capture it as a request + if (isString(url) && this.__sentry_xhr__.method === 'POST' && url.match(/sentry_key/)) { + this.__sentry_own_request__ = true; + } + return originalOpen.apply(this, args); + }; + }); + fill(xhrproto, 'send', function (originalSend) { + return function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + var xhr = this; // tslint:disable-line:no-this-assignment + var commonHandlerData = { + args: args, + startTimestamp: Date.now(), + xhr: xhr, + }; + triggerHandlers('xhr', __assign({}, commonHandlerData)); + xhr.addEventListener('readystatechange', function () { + if (xhr.readyState === 4) { + try { + // touching statusCode in some platforms throws + // an exception + if (xhr.__sentry_xhr__) { + xhr.__sentry_xhr__.status_code = xhr.status; + } + } + catch (e) { + /* do nothing */ + } + triggerHandlers('xhr', __assign({}, commonHandlerData, { endTimestamp: Date.now() })); + } + }); + return originalSend.apply(this, args); + }; + }); + } + var lastHref; + /** JSDoc */ + function instrumentHistory() { + if (!supportsHistory()) { + return; + } + var oldOnPopState = global$2.onpopstate; + global$2.onpopstate = function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + var to = global$2.location.href; + // keep track of the current URL state, as we always receive only the updated state + var from = lastHref; + lastHref = to; + triggerHandlers('history', { + from: from, + to: to, + }); + if (oldOnPopState) { + return oldOnPopState.apply(this, args); + } + }; + /** @hidden */ + function historyReplacementFunction(originalHistoryFunction) { + return function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + var url = args.length > 2 ? args[2] : undefined; + if (url) { + // coerce to string (this is what pushState does) + var from = lastHref; + var to = String(url); + // keep track of the current URL state, as we always receive only the updated state + lastHref = to; + triggerHandlers('history', { + from: from, + to: to, + }); + } + return originalHistoryFunction.apply(this, args); + }; + } + fill(global$2.history, 'pushState', historyReplacementFunction); + fill(global$2.history, 'replaceState', historyReplacementFunction); + } + /** JSDoc */ + function instrumentDOM() { + if (!('document' in global$2)) { + return; + } + // Capture breadcrumbs from any click that is unhandled / bubbled up all the way + // to the document. Do this before we instrument addEventListener. + global$2.document.addEventListener('click', domEventHandler('click', triggerHandlers.bind(null, 'dom')), false); + global$2.document.addEventListener('keypress', keypressEventHandler(triggerHandlers.bind(null, 'dom')), false); + // After hooking into document bubbled up click and keypresses events, we also hook into user handled click & keypresses. + ['EventTarget', 'Node'].forEach(function (target) { + var proto = global$2[target] && global$2[target].prototype; + if (!proto || !proto.hasOwnProperty || !proto.hasOwnProperty('addEventListener')) { + return; + } + fill(proto, 'addEventListener', function (original) { + return function (eventName, fn, options) { + if (fn && fn.handleEvent) { + if (eventName === 'click') { + fill(fn, 'handleEvent', function (innerOriginal) { + return function (event) { + domEventHandler('click', triggerHandlers.bind(null, 'dom'))(event); + return innerOriginal.call(this, event); + }; + }); + } + if (eventName === 'keypress') { + fill(fn, 'handleEvent', function (innerOriginal) { + return function (event) { + keypressEventHandler(triggerHandlers.bind(null, 'dom'))(event); + return innerOriginal.call(this, event); + }; + }); + } + } + else { + if (eventName === 'click') { + domEventHandler('click', triggerHandlers.bind(null, 'dom'), true)(this); + } + if (eventName === 'keypress') { + keypressEventHandler(triggerHandlers.bind(null, 'dom'))(this); + } + } + return original.call(this, eventName, fn, options); + }; + }); + fill(proto, 'removeEventListener', function (original) { + return function (eventName, fn, options) { + var callback = fn; + try { + callback = callback && (callback.__sentry_wrapped__ || callback); + } + catch (e) { + // ignore, accessing __sentry_wrapped__ will throw in some Selenium environments + } + return original.call(this, eventName, callback, options); + }; + }); + }); + } + var debounceDuration = 1000; + var debounceTimer = 0; + var keypressTimeout; + var lastCapturedEvent; + /** + * Wraps addEventListener to capture UI breadcrumbs + * @param name the event name (e.g. "click") + * @param handler function that will be triggered + * @param debounce decides whether it should wait till another event loop + * @returns wrapped breadcrumb events handler + * @hidden + */ + function domEventHandler(name, handler, debounce) { + if (debounce === void 0) { debounce = false; } + return function (event) { + // reset keypress timeout; e.g. triggering a 'click' after + // a 'keypress' will reset the keypress debounce so that a new + // set of keypresses can be recorded + keypressTimeout = undefined; + // It's possible this handler might trigger multiple times for the same + // event (e.g. event propagation through node ancestors). Ignore if we've + // already captured the event. + if (!event || lastCapturedEvent === event) { + return; + } + lastCapturedEvent = event; + if (debounceTimer) { + clearTimeout(debounceTimer); + } + if (debounce) { + debounceTimer = setTimeout(function () { + handler({ event: event, name: name }); + }); + } + else { + handler({ event: event, name: name }); + } + }; + } + /** + * Wraps addEventListener to capture keypress UI events + * @param handler function that will be triggered + * @returns wrapped keypress events handler + * @hidden + */ + function keypressEventHandler(handler) { + // TODO: if somehow user switches keypress target before + // debounce timeout is triggered, we will only capture + // a single breadcrumb from the FIRST target (acceptable?) + return function (event) { + var target; + try { + target = event.target; + } + catch (e) { + // just accessing event properties can throw an exception in some rare circumstances + // see: https://github.com/getsentry/raven-js/issues/838 + return; + } + var tagName = target && target.tagName; + // only consider keypress events on actual input elements + // this will disregard keypresses targeting body (e.g. tabbing + // through elements, hotkeys, etc) + if (!tagName || (tagName !== 'INPUT' && tagName !== 'TEXTAREA' && !target.isContentEditable)) { + return; + } + // record first keypress in a series, but ignore subsequent + // keypresses until debounce clears + if (!keypressTimeout) { + domEventHandler('input', handler)(event); + } + clearTimeout(keypressTimeout); + keypressTimeout = setTimeout(function () { + keypressTimeout = undefined; + }, debounceDuration); + }; + } + var _oldOnErrorHandler = null; + /** JSDoc */ + function instrumentError() { + _oldOnErrorHandler = global$2.onerror; + global$2.onerror = function (msg, url, line, column, error) { + triggerHandlers('error', { + column: column, + error: error, + line: line, + msg: msg, + url: url, + }); + if (_oldOnErrorHandler) { + return _oldOnErrorHandler.apply(this, arguments); + } + return false; + }; + } + var _oldOnUnhandledRejectionHandler = null; + /** JSDoc */ + function instrumentUnhandledRejection() { + _oldOnUnhandledRejectionHandler = global$2.onunhandledrejection; + global$2.onunhandledrejection = function (e) { + triggerHandlers('unhandledrejection', e); + if (_oldOnUnhandledRejectionHandler) { + return _oldOnUnhandledRejectionHandler.apply(this, arguments); + } + return true; + }; + } + + /** Regular expression used to parse a Dsn. */ + var DSN_REGEX = /^(?:(\w+):)\/\/(?:(\w+)(?::(\w+))?@)([\w\.-]+)(?::(\d+))?\/(.+)/; + /** Error message */ + var ERROR_MESSAGE = 'Invalid Dsn'; + /** The Sentry Dsn, identifying a Sentry instance and project. */ + var Dsn = /** @class */ (function () { + /** Creates a new Dsn component */ + function Dsn(from) { + if (typeof from === 'string') { + this._fromString(from); + } + else { + this._fromComponents(from); + } + this._validate(); + } + /** + * Renders the string representation of this Dsn. + * + * By default, this will render the public representation without the password + * component. To get the deprecated private representation, set `withPassword` + * to true. + * + * @param withPassword When set to true, the password will be included. + */ + Dsn.prototype.toString = function (withPassword) { + if (withPassword === void 0) { withPassword = false; } + // tslint:disable-next-line:no-this-assignment + var _a = this, host = _a.host, path = _a.path, pass = _a.pass, port = _a.port, projectId = _a.projectId, protocol = _a.protocol, user = _a.user; + return (protocol + "://" + user + (withPassword && pass ? ":" + pass : '') + + ("@" + host + (port ? ":" + port : '') + "/" + (path ? path + "/" : path) + projectId)); + }; + /** Parses a string into this Dsn. */ + Dsn.prototype._fromString = function (str) { + var match = DSN_REGEX.exec(str); + if (!match) { + throw new SentryError(ERROR_MESSAGE); + } + var _a = __read(match.slice(1), 6), protocol = _a[0], user = _a[1], _b = _a[2], pass = _b === void 0 ? '' : _b, host = _a[3], _c = _a[4], port = _c === void 0 ? '' : _c, lastPath = _a[5]; + var path = ''; + var projectId = lastPath; + var split = projectId.split('/'); + if (split.length > 1) { + path = split.slice(0, -1).join('/'); + projectId = split.pop(); + } + this._fromComponents({ host: host, pass: pass, path: path, projectId: projectId, port: port, protocol: protocol, user: user }); + }; + /** Maps Dsn components into this instance. */ + Dsn.prototype._fromComponents = function (components) { + this.protocol = components.protocol; + this.user = components.user; + this.pass = components.pass || ''; + this.host = components.host; + this.port = components.port || ''; + this.path = components.path || ''; + this.projectId = components.projectId; + }; + /** Validates this Dsn and throws on error. */ + Dsn.prototype._validate = function () { + var _this = this; + ['protocol', 'user', 'host', 'projectId'].forEach(function (component) { + if (!_this[component]) { + throw new SentryError(ERROR_MESSAGE); + } + }); + if (this.protocol !== 'http' && this.protocol !== 'https') { + throw new SentryError(ERROR_MESSAGE); + } + if (this.port && isNaN(parseInt(this.port, 10))) { + throw new SentryError(ERROR_MESSAGE); + } + }; + return Dsn; + }()); + + /** + * Holds additional event information. {@link Scope.applyToEvent} will be + * called by the client before an event will be sent. + */ + var Scope = /** @class */ (function () { + function Scope() { + /** Flag if notifiying is happening. */ + this._notifyingListeners = false; + /** Callback for client to receive scope changes. */ + this._scopeListeners = []; + /** Callback list that will be called after {@link applyToEvent}. */ + this._eventProcessors = []; + /** Array of breadcrumbs. */ + this._breadcrumbs = []; + /** User */ + this._user = {}; + /** Tags */ + this._tags = {}; + /** Extra */ + this._extra = {}; + /** Contexts */ + this._context = {}; + } + /** + * Add internal on change listener. Used for sub SDKs that need to store the scope. + * @hidden + */ + Scope.prototype.addScopeListener = function (callback) { + this._scopeListeners.push(callback); + }; + /** + * @inheritDoc + */ + Scope.prototype.addEventProcessor = function (callback) { + this._eventProcessors.push(callback); + return this; + }; + /** + * This will be called on every set call. + */ + Scope.prototype._notifyScopeListeners = function () { + var _this = this; + if (!this._notifyingListeners) { + this._notifyingListeners = true; + setTimeout(function () { + _this._scopeListeners.forEach(function (callback) { + callback(_this); + }); + _this._notifyingListeners = false; + }); + } + }; + /** + * This will be called after {@link applyToEvent} is finished. + */ + Scope.prototype._notifyEventProcessors = function (processors, event, hint, index) { + var _this = this; + if (index === void 0) { index = 0; } + return new SyncPromise(function (resolve, reject) { + var processor = processors[index]; + // tslint:disable-next-line:strict-type-predicates + if (event === null || typeof processor !== 'function') { + resolve(event); + } + else { + var result = processor(__assign({}, event), hint); + if (isThenable(result)) { + result + .then(function (final) { return _this._notifyEventProcessors(processors, final, hint, index + 1).then(resolve); }) + .then(null, reject); + } + else { + _this._notifyEventProcessors(processors, result, hint, index + 1) + .then(resolve) + .then(null, reject); + } + } + }); + }; + /** + * @inheritDoc + */ + Scope.prototype.setUser = function (user) { + this._user = user || {}; + this._notifyScopeListeners(); + return this; + }; + /** + * @inheritDoc + */ + Scope.prototype.setTags = function (tags) { + this._tags = __assign({}, this._tags, tags); + this._notifyScopeListeners(); + return this; + }; + /** + * @inheritDoc + */ + Scope.prototype.setTag = function (key, value) { + var _a; + this._tags = __assign({}, this._tags, (_a = {}, _a[key] = value, _a)); + this._notifyScopeListeners(); + return this; + }; + /** + * @inheritDoc + */ + Scope.prototype.setExtras = function (extras) { + this._extra = __assign({}, this._extra, extras); + this._notifyScopeListeners(); + return this; + }; + /** + * @inheritDoc + */ + Scope.prototype.setExtra = function (key, extra) { + var _a; + this._extra = __assign({}, this._extra, (_a = {}, _a[key] = extra, _a)); + this._notifyScopeListeners(); + return this; + }; + /** + * @inheritDoc + */ + Scope.prototype.setFingerprint = function (fingerprint) { + this._fingerprint = fingerprint; + this._notifyScopeListeners(); + return this; + }; + /** + * @inheritDoc + */ + Scope.prototype.setLevel = function (level) { + this._level = level; + this._notifyScopeListeners(); + return this; + }; + /** + * @inheritDoc + */ + Scope.prototype.setTransaction = function (transaction) { + this._transaction = transaction; + if (this._span) { + this._span.transaction = transaction; + } + this._notifyScopeListeners(); + return this; + }; + /** + * @inheritDoc + */ + Scope.prototype.setContext = function (key, context) { + var _a; + this._context = __assign({}, this._context, (_a = {}, _a[key] = context, _a)); + this._notifyScopeListeners(); + return this; + }; + /** + * @inheritDoc + */ + Scope.prototype.setSpan = function (span) { + this._span = span; + this._notifyScopeListeners(); + return this; + }; + /** + * Internal getter for Span, used in Hub. + * @hidden + */ + Scope.prototype.getSpan = function () { + return this._span; + }; + /** + * Inherit values from the parent scope. + * @param scope to clone. + */ + Scope.clone = function (scope) { + var newScope = new Scope(); + if (scope) { + newScope._breadcrumbs = __spread(scope._breadcrumbs); + newScope._tags = __assign({}, scope._tags); + newScope._extra = __assign({}, scope._extra); + newScope._context = __assign({}, scope._context); + newScope._user = scope._user; + newScope._level = scope._level; + newScope._span = scope._span; + newScope._transaction = scope._transaction; + newScope._fingerprint = scope._fingerprint; + newScope._eventProcessors = __spread(scope._eventProcessors); + } + return newScope; + }; + /** + * @inheritDoc + */ + Scope.prototype.clear = function () { + this._breadcrumbs = []; + this._tags = {}; + this._extra = {}; + this._user = {}; + this._context = {}; + this._level = undefined; + this._transaction = undefined; + this._fingerprint = undefined; + this._span = undefined; + this._notifyScopeListeners(); + return this; + }; + /** + * @inheritDoc + */ + Scope.prototype.addBreadcrumb = function (breadcrumb, maxBreadcrumbs) { + var mergedBreadcrumb = __assign({ timestamp: timestampWithMs() }, breadcrumb); + this._breadcrumbs = + maxBreadcrumbs !== undefined && maxBreadcrumbs >= 0 + ? __spread(this._breadcrumbs, [mergedBreadcrumb]).slice(-maxBreadcrumbs) + : __spread(this._breadcrumbs, [mergedBreadcrumb]); + this._notifyScopeListeners(); + return this; + }; + /** + * @inheritDoc + */ + Scope.prototype.clearBreadcrumbs = function () { + this._breadcrumbs = []; + this._notifyScopeListeners(); + return this; + }; + /** + * Applies fingerprint from the scope to the event if there's one, + * uses message if there's one instead or get rid of empty fingerprint + */ + Scope.prototype._applyFingerprint = function (event) { + // Make sure it's an array first and we actually have something in place + event.fingerprint = event.fingerprint + ? Array.isArray(event.fingerprint) + ? event.fingerprint + : [event.fingerprint] + : []; + // If we have something on the scope, then merge it with event + if (this._fingerprint) { + event.fingerprint = event.fingerprint.concat(this._fingerprint); + } + // If we have no data at all, remove empty array default + if (event.fingerprint && !event.fingerprint.length) { + delete event.fingerprint; + } + }; + /** + * Applies the current context and fingerprint to the event. + * Note that breadcrumbs will be added by the client. + * Also if the event has already breadcrumbs on it, we do not merge them. + * @param event Event + * @param hint May contain additional informartion about the original exception. + * @hidden + */ + Scope.prototype.applyToEvent = function (event, hint) { + if (this._extra && Object.keys(this._extra).length) { + event.extra = __assign({}, this._extra, event.extra); + } + if (this._tags && Object.keys(this._tags).length) { + event.tags = __assign({}, this._tags, event.tags); + } + if (this._user && Object.keys(this._user).length) { + event.user = __assign({}, this._user, event.user); + } + if (this._context && Object.keys(this._context).length) { + event.contexts = __assign({}, this._context, event.contexts); + } + if (this._level) { + event.level = this._level; + } + if (this._transaction) { + event.transaction = this._transaction; + } + if (this._span) { + event.contexts = __assign({ trace: this._span.getTraceContext() }, event.contexts); + } + this._applyFingerprint(event); + event.breadcrumbs = __spread((event.breadcrumbs || []), this._breadcrumbs); + event.breadcrumbs = event.breadcrumbs.length > 0 ? event.breadcrumbs : undefined; + return this._notifyEventProcessors(__spread(getGlobalEventProcessors(), this._eventProcessors), event, hint); + }; + return Scope; + }()); + /** + * Retruns the global event processors. + */ + function getGlobalEventProcessors() { + var global = getGlobalObject(); + global.__SENTRY__ = global.__SENTRY__ || {}; + global.__SENTRY__.globalEventProcessors = global.__SENTRY__.globalEventProcessors || []; + return global.__SENTRY__.globalEventProcessors; + } + /** + * Add a EventProcessor to be kept globally. + * @param callback EventProcessor to add + */ + function addGlobalEventProcessor(callback) { + getGlobalEventProcessors().push(callback); + } + + /** + * API compatibility version of this hub. + * + * WARNING: This number should only be incresed when the global interface + * changes a and new methods are introduced. + * + * @hidden + */ + var API_VERSION = 3; + /** + * Default maximum number of breadcrumbs added to an event. Can be overwritten + * with {@link Options.maxBreadcrumbs}. + */ + var DEFAULT_BREADCRUMBS = 100; + /** + * Absolute maximum number of breadcrumbs added to an event. The + * `maxBreadcrumbs` option cannot be higher than this value. + */ + var MAX_BREADCRUMBS = 100; + /** + * @inheritDoc + */ + var Hub = /** @class */ (function () { + /** + * Creates a new instance of the hub, will push one {@link Layer} into the + * internal stack on creation. + * + * @param client bound to the hub. + * @param scope bound to the hub. + * @param version number, higher number means higher priority. + */ + function Hub(client, scope, _version) { + if (scope === void 0) { scope = new Scope(); } + if (_version === void 0) { _version = API_VERSION; } + this._version = _version; + /** Is a {@link Layer}[] containing the client and scope */ + this._stack = []; + this._stack.push({ client: client, scope: scope }); + } + /** + * Internal helper function to call a method on the top client if it exists. + * + * @param method The method to call on the client. + * @param args Arguments to pass to the client function. + */ + Hub.prototype._invokeClient = function (method) { + var _a; + var args = []; + for (var _i = 1; _i < arguments.length; _i++) { + args[_i - 1] = arguments[_i]; + } + var top = this.getStackTop(); + if (top && top.client && top.client[method]) { + (_a = top.client)[method].apply(_a, __spread(args, [top.scope])); + } + }; + /** + * @inheritDoc + */ + Hub.prototype.isOlderThan = function (version) { + return this._version < version; + }; + /** + * @inheritDoc + */ + Hub.prototype.bindClient = function (client) { + var top = this.getStackTop(); + top.client = client; + if (client && client.setupIntegrations) { + client.setupIntegrations(); + } + }; + /** + * @inheritDoc + */ + Hub.prototype.pushScope = function () { + // We want to clone the content of prev scope + var stack = this.getStack(); + var parentScope = stack.length > 0 ? stack[stack.length - 1].scope : undefined; + var scope = Scope.clone(parentScope); + this.getStack().push({ + client: this.getClient(), + scope: scope, + }); + return scope; + }; + /** + * @inheritDoc + */ + Hub.prototype.popScope = function () { + return this.getStack().pop() !== undefined; + }; + /** + * @inheritDoc + */ + Hub.prototype.withScope = function (callback) { + var scope = this.pushScope(); + try { + callback(scope); + } + finally { + this.popScope(); + } + }; + /** + * @inheritDoc + */ + Hub.prototype.getClient = function () { + return this.getStackTop().client; + }; + /** Returns the scope of the top stack. */ + Hub.prototype.getScope = function () { + return this.getStackTop().scope; + }; + /** Returns the scope stack for domains or the process. */ + Hub.prototype.getStack = function () { + return this._stack; + }; + /** Returns the topmost scope layer in the order domain > local > process. */ + Hub.prototype.getStackTop = function () { + return this._stack[this._stack.length - 1]; + }; + /** + * @inheritDoc + */ + Hub.prototype.captureException = function (exception, hint) { + var eventId = (this._lastEventId = uuid4()); + var finalHint = hint; + // If there's no explicit hint provided, mimick the same thing that would happen + // in the minimal itself to create a consistent behavior. + // We don't do this in the client, as it's the lowest level API, and doing this, + // would prevent user from having full control over direct calls. + if (!hint) { + var syntheticException = void 0; + try { + throw new Error('Sentry syntheticException'); + } + catch (exception) { + syntheticException = exception; + } + finalHint = { + originalException: exception, + syntheticException: syntheticException, + }; + } + this._invokeClient('captureException', exception, __assign({}, finalHint, { event_id: eventId })); + return eventId; + }; + /** + * @inheritDoc + */ + Hub.prototype.captureMessage = function (message, level, hint) { + var eventId = (this._lastEventId = uuid4()); + var finalHint = hint; + // If there's no explicit hint provided, mimick the same thing that would happen + // in the minimal itself to create a consistent behavior. + // We don't do this in the client, as it's the lowest level API, and doing this, + // would prevent user from having full control over direct calls. + if (!hint) { + var syntheticException = void 0; + try { + throw new Error(message); + } + catch (exception) { + syntheticException = exception; + } + finalHint = { + originalException: message, + syntheticException: syntheticException, + }; + } + this._invokeClient('captureMessage', message, level, __assign({}, finalHint, { event_id: eventId })); + return eventId; + }; + /** + * @inheritDoc + */ + Hub.prototype.captureEvent = function (event, hint) { + var eventId = (this._lastEventId = uuid4()); + this._invokeClient('captureEvent', event, __assign({}, hint, { event_id: eventId })); + return eventId; + }; + /** + * @inheritDoc + */ + Hub.prototype.lastEventId = function () { + return this._lastEventId; + }; + /** + * @inheritDoc + */ + Hub.prototype.addBreadcrumb = function (breadcrumb, hint) { + var top = this.getStackTop(); + if (!top.scope || !top.client) { + return; + } + var _a = (top.client.getOptions && top.client.getOptions()) || {}, _b = _a.beforeBreadcrumb, beforeBreadcrumb = _b === void 0 ? null : _b, _c = _a.maxBreadcrumbs, maxBreadcrumbs = _c === void 0 ? DEFAULT_BREADCRUMBS : _c; + if (maxBreadcrumbs <= 0) { + return; + } + var timestamp = timestampWithMs(); + var mergedBreadcrumb = __assign({ timestamp: timestamp }, breadcrumb); + var finalBreadcrumb = beforeBreadcrumb + ? consoleSandbox(function () { return beforeBreadcrumb(mergedBreadcrumb, hint); }) + : mergedBreadcrumb; + if (finalBreadcrumb === null) { + return; + } + top.scope.addBreadcrumb(finalBreadcrumb, Math.min(maxBreadcrumbs, MAX_BREADCRUMBS)); + }; + /** + * @inheritDoc + */ + Hub.prototype.setUser = function (user) { + var top = this.getStackTop(); + if (!top.scope) { + return; + } + top.scope.setUser(user); + }; + /** + * @inheritDoc + */ + Hub.prototype.setTags = function (tags) { + var top = this.getStackTop(); + if (!top.scope) { + return; + } + top.scope.setTags(tags); + }; + /** + * @inheritDoc + */ + Hub.prototype.setExtras = function (extras) { + var top = this.getStackTop(); + if (!top.scope) { + return; + } + top.scope.setExtras(extras); + }; + /** + * @inheritDoc + */ + Hub.prototype.setTag = function (key, value) { + var top = this.getStackTop(); + if (!top.scope) { + return; + } + top.scope.setTag(key, value); + }; + /** + * @inheritDoc + */ + Hub.prototype.setExtra = function (key, extra) { + var top = this.getStackTop(); + if (!top.scope) { + return; + } + top.scope.setExtra(key, extra); + }; + /** + * @inheritDoc + */ + Hub.prototype.setContext = function (name, context) { + var top = this.getStackTop(); + if (!top.scope) { + return; + } + top.scope.setContext(name, context); + }; + /** + * @inheritDoc + */ + Hub.prototype.configureScope = function (callback) { + var top = this.getStackTop(); + if (top.scope && top.client) { + callback(top.scope); + } + }; + /** + * @inheritDoc + */ + Hub.prototype.run = function (callback) { + var oldHub = makeMain(this); + try { + callback(this); + } + finally { + makeMain(oldHub); + } + }; + /** + * @inheritDoc + */ + Hub.prototype.getIntegration = function (integration) { + var client = this.getClient(); + if (!client) { + return null; + } + try { + return client.getIntegration(integration); + } + catch (_oO) { + logger.warn("Cannot retrieve integration " + integration.id + " from the current Hub"); + return null; + } + }; + /** + * @inheritDoc + */ + Hub.prototype.startSpan = function (spanOrSpanContext, forceNoChild) { + if (forceNoChild === void 0) { forceNoChild = false; } + return this._callExtensionMethod('startSpan', spanOrSpanContext, forceNoChild); + }; + /** + * @inheritDoc + */ + Hub.prototype.traceHeaders = function () { + return this._callExtensionMethod('traceHeaders'); + }; + /** + * Calls global extension method and binding current instance to the function call + */ + // @ts-ignore + Hub.prototype._callExtensionMethod = function (method) { + var args = []; + for (var _i = 1; _i < arguments.length; _i++) { + args[_i - 1] = arguments[_i]; + } + var carrier = getMainCarrier(); + var sentry = carrier.__SENTRY__; + // tslint:disable-next-line: strict-type-predicates + if (sentry && sentry.extensions && typeof sentry.extensions[method] === 'function') { + return sentry.extensions[method].apply(this, args); + } + logger.warn("Extension method " + method + " couldn't be found, doing nothing."); + }; + return Hub; + }()); + /** Returns the global shim registry. */ + function getMainCarrier() { + var carrier = getGlobalObject(); + carrier.__SENTRY__ = carrier.__SENTRY__ || { + extensions: {}, + hub: undefined, + }; + return carrier; + } + /** + * Replaces the current main hub with the passed one on the global object + * + * @returns The old replaced hub + */ + function makeMain(hub) { + var registry = getMainCarrier(); + var oldHub = getHubFromCarrier(registry); + setHubOnCarrier(registry, hub); + return oldHub; + } + /** + * Returns the default hub instance. + * + * If a hub is already registered in the global carrier but this module + * contains a more recent version, it replaces the registered version. + * Otherwise, the currently registered hub will be returned. + */ + function getCurrentHub() { + // Get main carrier (global for every environment) + var registry = getMainCarrier(); + // If there's no hub, or its an old API, assign a new one + if (!hasHubOnCarrier(registry) || getHubFromCarrier(registry).isOlderThan(API_VERSION)) { + setHubOnCarrier(registry, new Hub()); + } + // Prefer domains over global if they are there (applicable only to Node environment) + if (isNodeEnv()) { + return getHubFromActiveDomain(registry); + } + // Return hub that lives on a global object + return getHubFromCarrier(registry); + } + /** + * Try to read the hub from an active domain, fallback to the registry if one doesnt exist + * @returns discovered hub + */ + function getHubFromActiveDomain(registry) { + try { + var property = 'domain'; + var carrier = getMainCarrier(); + var sentry = carrier.__SENTRY__; + // tslint:disable-next-line: strict-type-predicates + if (!sentry || !sentry.extensions || !sentry.extensions[property]) { + return getHubFromCarrier(registry); + } + var domain = sentry.extensions[property]; + var activeDomain = domain.active; + // If there no active domain, just return global hub + if (!activeDomain) { + return getHubFromCarrier(registry); + } + // If there's no hub on current domain, or its an old API, assign a new one + if (!hasHubOnCarrier(activeDomain) || getHubFromCarrier(activeDomain).isOlderThan(API_VERSION)) { + var registryHubTopStack = getHubFromCarrier(registry).getStackTop(); + setHubOnCarrier(activeDomain, new Hub(registryHubTopStack.client, Scope.clone(registryHubTopStack.scope))); + } + // Return hub that lives on a domain + return getHubFromCarrier(activeDomain); + } + catch (_Oo) { + // Return hub that lives on a global object + return getHubFromCarrier(registry); + } + } + /** + * This will tell whether a carrier has a hub on it or not + * @param carrier object + */ + function hasHubOnCarrier(carrier) { + if (carrier && carrier.__SENTRY__ && carrier.__SENTRY__.hub) { + return true; + } + return false; + } + /** + * This will create a new {@link Hub} and add to the passed object on + * __SENTRY__.hub. + * @param carrier object + * @hidden + */ + function getHubFromCarrier(carrier) { + if (carrier && carrier.__SENTRY__ && carrier.__SENTRY__.hub) { + return carrier.__SENTRY__.hub; + } + carrier.__SENTRY__ = carrier.__SENTRY__ || {}; + carrier.__SENTRY__.hub = new Hub(); + return carrier.__SENTRY__.hub; + } + /** + * This will set passed {@link Hub} on the passed object's __SENTRY__.hub attribute + * @param carrier object + * @param hub Hub + */ + function setHubOnCarrier(carrier, hub) { + if (!carrier) { + return false; + } + carrier.__SENTRY__ = carrier.__SENTRY__ || {}; + carrier.__SENTRY__.hub = hub; + return true; + } + + /** + * This calls a function on the current hub. + * @param method function to call on hub. + * @param args to pass to function. + */ + function callOnHub(method) { + var args = []; + for (var _i = 1; _i < arguments.length; _i++) { + args[_i - 1] = arguments[_i]; + } + var hub = getCurrentHub(); + if (hub && hub[method]) { + // tslint:disable-next-line:no-unsafe-any + return hub[method].apply(hub, __spread(args)); + } + throw new Error("No hub defined or " + method + " was not found on the hub, please open a bug report."); + } + /** + * Captures an exception event and sends it to Sentry. + * + * @param exception An exception-like object. + * @returns The generated eventId. + */ + function captureException(exception) { + var syntheticException; + try { + throw new Error('Sentry syntheticException'); + } + catch (exception) { + syntheticException = exception; + } + return callOnHub('captureException', exception, { + originalException: exception, + syntheticException: syntheticException, + }); + } + /** + * Captures a message event and sends it to Sentry. + * + * @param message The message to send to Sentry. + * @param level Define the level of the message. + * @returns The generated eventId. + */ + function captureMessage(message, level) { + var syntheticException; + try { + throw new Error(message); + } + catch (exception) { + syntheticException = exception; + } + return callOnHub('captureMessage', message, level, { + originalException: message, + syntheticException: syntheticException, + }); + } + /** + * Captures a manually created event and sends it to Sentry. + * + * @param event The event to send to Sentry. + * @returns The generated eventId. + */ + function captureEvent(event) { + return callOnHub('captureEvent', event); + } + /** + * Callback to set context information onto the scope. + * @param callback Callback function that receives Scope. + */ + function configureScope(callback) { + callOnHub('configureScope', callback); + } + /** + * Records a new breadcrumb which will be attached to future events. + * + * Breadcrumbs will be added to subsequent events to provide more context on + * user's actions prior to an error or crash. + * + * @param breadcrumb The breadcrumb to record. + */ + function addBreadcrumb(breadcrumb) { + callOnHub('addBreadcrumb', breadcrumb); + } + /** + * Sets context data with the given name. + * @param name of the context + * @param context Any kind of data. This data will be normailzed. + */ + function setContext(name, context) { + callOnHub('setContext', name, context); + } + /** + * Set an object that will be merged sent as extra data with the event. + * @param extras Extras object to merge into current context. + */ + function setExtras(extras) { + callOnHub('setExtras', extras); + } + /** + * Set an object that will be merged sent as tags data with the event. + * @param tags Tags context object to merge into current context. + */ + function setTags(tags) { + callOnHub('setTags', tags); + } + /** + * Set key:value that will be sent as extra data with the event. + * @param key String of extra + * @param extra Any kind of data. This data will be normailzed. + */ + function setExtra(key, extra) { + callOnHub('setExtra', key, extra); + } + /** + * Set key:value that will be sent as tags data with the event. + * @param key String key of tag + * @param value String value of tag + */ + function setTag(key, value) { + callOnHub('setTag', key, value); + } + /** + * Updates user context information for future events. + * + * @param user User context object to be set in the current context. Pass `null` to unset the user. + */ + function setUser(user) { + callOnHub('setUser', user); + } + /** + * Creates a new scope with and executes the given operation within. + * The scope is automatically removed once the operation + * finishes or throws. + * + * This is essentially a convenience function for: + * + * pushScope(); + * callback(); + * popScope(); + * + * @param callback that will be enclosed into push/popScope. + */ + function withScope(callback) { + callOnHub('withScope', callback); + } + + var SENTRY_API_VERSION = '7'; + /** Helper class to provide urls to different Sentry endpoints. */ + var API = /** @class */ (function () { + /** Create a new instance of API */ + function API(dsn) { + this.dsn = dsn; + this._dsnObject = new Dsn(dsn); + } + /** Returns the Dsn object. */ + API.prototype.getDsn = function () { + return this._dsnObject; + }; + /** Returns a string with auth headers in the url to the store endpoint. */ + API.prototype.getStoreEndpoint = function () { + return "" + this._getBaseUrl() + this.getStoreEndpointPath(); + }; + /** Returns the store endpoint with auth added in url encoded. */ + API.prototype.getStoreEndpointWithUrlEncodedAuth = function () { + var dsn = this._dsnObject; + var auth = { + sentry_key: dsn.user, + sentry_version: SENTRY_API_VERSION, + }; + // Auth is intentionally sent as part of query string (NOT as custom HTTP header) + // to avoid preflight CORS requests + return this.getStoreEndpoint() + "?" + urlEncode(auth); + }; + /** Returns the base path of the url including the port. */ + API.prototype._getBaseUrl = function () { + var dsn = this._dsnObject; + var protocol = dsn.protocol ? dsn.protocol + ":" : ''; + var port = dsn.port ? ":" + dsn.port : ''; + return protocol + "//" + dsn.host + port; + }; + /** Returns only the path component for the store endpoint. */ + API.prototype.getStoreEndpointPath = function () { + var dsn = this._dsnObject; + return (dsn.path ? "/" + dsn.path : '') + "/api/" + dsn.projectId + "/store/"; + }; + /** Returns an object that can be used in request headers. */ + API.prototype.getRequestHeaders = function (clientName, clientVersion) { + var dsn = this._dsnObject; + var header = ["Sentry sentry_version=" + SENTRY_API_VERSION]; + header.push("sentry_client=" + clientName + "/" + clientVersion); + header.push("sentry_key=" + dsn.user); + if (dsn.pass) { + header.push("sentry_secret=" + dsn.pass); + } + return { + 'Content-Type': 'application/json', + 'X-Sentry-Auth': header.join(', '), + }; + }; + /** Returns the url to the report dialog endpoint. */ + API.prototype.getReportDialogEndpoint = function (dialogOptions) { + if (dialogOptions === void 0) { dialogOptions = {}; } + var dsn = this._dsnObject; + var endpoint = "" + this._getBaseUrl() + (dsn.path ? "/" + dsn.path : '') + "/api/embed/error-page/"; + var encodedOptions = []; + encodedOptions.push("dsn=" + dsn.toString()); + for (var key in dialogOptions) { + if (key === 'user') { + if (!dialogOptions.user) { + continue; + } + if (dialogOptions.user.name) { + encodedOptions.push("name=" + encodeURIComponent(dialogOptions.user.name)); + } + if (dialogOptions.user.email) { + encodedOptions.push("email=" + encodeURIComponent(dialogOptions.user.email)); + } + } + else { + encodedOptions.push(encodeURIComponent(key) + "=" + encodeURIComponent(dialogOptions[key])); + } + } + if (encodedOptions.length) { + return endpoint + "?" + encodedOptions.join('&'); + } + return endpoint; + }; + return API; + }()); + + var installedIntegrations = []; + /** Gets integration to install */ + function getIntegrationsToSetup(options) { + var defaultIntegrations = (options.defaultIntegrations && __spread(options.defaultIntegrations)) || []; + var userIntegrations = options.integrations; + var integrations = []; + if (Array.isArray(userIntegrations)) { + var userIntegrationsNames_1 = userIntegrations.map(function (i) { return i.name; }); + var pickedIntegrationsNames_1 = []; + // Leave only unique default integrations, that were not overridden with provided user integrations + defaultIntegrations.forEach(function (defaultIntegration) { + if (userIntegrationsNames_1.indexOf(defaultIntegration.name) === -1 && + pickedIntegrationsNames_1.indexOf(defaultIntegration.name) === -1) { + integrations.push(defaultIntegration); + pickedIntegrationsNames_1.push(defaultIntegration.name); + } + }); + // Don't add same user integration twice + userIntegrations.forEach(function (userIntegration) { + if (pickedIntegrationsNames_1.indexOf(userIntegration.name) === -1) { + integrations.push(userIntegration); + pickedIntegrationsNames_1.push(userIntegration.name); + } + }); + } + else if (typeof userIntegrations === 'function') { + integrations = userIntegrations(defaultIntegrations); + integrations = Array.isArray(integrations) ? integrations : [integrations]; + } + else { + integrations = __spread(defaultIntegrations); + } + // Make sure that if present, `Debug` integration will always run last + var integrationsNames = integrations.map(function (i) { return i.name; }); + var alwaysLastToRun = 'Debug'; + if (integrationsNames.indexOf(alwaysLastToRun) !== -1) { + integrations.push.apply(integrations, __spread(integrations.splice(integrationsNames.indexOf(alwaysLastToRun), 1))); + } + return integrations; + } + /** Setup given integration */ + function setupIntegration(integration) { + if (installedIntegrations.indexOf(integration.name) !== -1) { + return; + } + integration.setupOnce(addGlobalEventProcessor, getCurrentHub); + installedIntegrations.push(integration.name); + logger.log("Integration installed: " + integration.name); + } + /** + * Given a list of integration instances this installs them all. When `withDefaults` is set to `true` then all default + * integrations are added unless they were already provided before. + * @param integrations array of integration instances + * @param withDefault should enable default integrations + */ + function setupIntegrations(options) { + var integrations = {}; + getIntegrationsToSetup(options).forEach(function (integration) { + integrations[integration.name] = integration; + setupIntegration(integration); + }); + return integrations; + } + + /** + * Base implementation for all JavaScript SDK clients. + * + * Call the constructor with the corresponding backend constructor and options + * specific to the client subclass. To access these options later, use + * {@link Client.getOptions}. Also, the Backend instance is available via + * {@link Client.getBackend}. + * + * If a Dsn is specified in the options, it will be parsed and stored. Use + * {@link Client.getDsn} to retrieve the Dsn at any moment. In case the Dsn is + * invalid, the constructor will throw a {@link SentryException}. Note that + * without a valid Dsn, the SDK will not send any events to Sentry. + * + * Before sending an event via the backend, it is passed through + * {@link BaseClient.prepareEvent} to add SDK information and scope data + * (breadcrumbs and context). To add more custom information, override this + * method and extend the resulting prepared event. + * + * To issue automatically created events (e.g. via instrumentation), use + * {@link Client.captureEvent}. It will prepare the event and pass it through + * the callback lifecycle. To issue auto-breadcrumbs, use + * {@link Client.addBreadcrumb}. + * + * @example + * class NodeClient extends BaseClient { + * public constructor(options: NodeOptions) { + * super(NodeBackend, options); + * } + * + * // ... + * } + */ + var BaseClient = /** @class */ (function () { + /** + * Initializes this client instance. + * + * @param backendClass A constructor function to create the backend. + * @param options Options for the client. + */ + function BaseClient(backendClass, options) { + /** Array of used integrations. */ + this._integrations = {}; + /** Is the client still processing a call? */ + this._processing = false; + this._backend = new backendClass(options); + this._options = options; + if (options.dsn) { + this._dsn = new Dsn(options.dsn); + } + } + /** + * @inheritDoc + */ + BaseClient.prototype.captureException = function (exception, hint, scope) { + var _this = this; + var eventId = hint && hint.event_id; + this._processing = true; + this._getBackend() + .eventFromException(exception, hint) + .then(function (event) { return _this._processEvent(event, hint, scope); }) + .then(function (finalEvent) { + // We need to check for finalEvent in case beforeSend returned null + eventId = finalEvent && finalEvent.event_id; + _this._processing = false; + }) + .then(null, function (reason) { + logger.error(reason); + _this._processing = false; + }); + return eventId; + }; + /** + * @inheritDoc + */ + BaseClient.prototype.captureMessage = function (message, level, hint, scope) { + var _this = this; + var eventId = hint && hint.event_id; + this._processing = true; + var promisedEvent = isPrimitive(message) + ? this._getBackend().eventFromMessage("" + message, level, hint) + : this._getBackend().eventFromException(message, hint); + promisedEvent + .then(function (event) { return _this._processEvent(event, hint, scope); }) + .then(function (finalEvent) { + // We need to check for finalEvent in case beforeSend returned null + eventId = finalEvent && finalEvent.event_id; + _this._processing = false; + }) + .then(null, function (reason) { + logger.error(reason); + _this._processing = false; + }); + return eventId; + }; + /** + * @inheritDoc + */ + BaseClient.prototype.captureEvent = function (event, hint, scope) { + var _this = this; + var eventId = hint && hint.event_id; + this._processing = true; + this._processEvent(event, hint, scope) + .then(function (finalEvent) { + // We need to check for finalEvent in case beforeSend returned null + eventId = finalEvent && finalEvent.event_id; + _this._processing = false; + }) + .then(null, function (reason) { + logger.error(reason); + _this._processing = false; + }); + return eventId; + }; + /** + * @inheritDoc + */ + BaseClient.prototype.getDsn = function () { + return this._dsn; + }; + /** + * @inheritDoc + */ + BaseClient.prototype.getOptions = function () { + return this._options; + }; + /** + * @inheritDoc + */ + BaseClient.prototype.flush = function (timeout) { + var _this = this; + return this._isClientProcessing(timeout).then(function (status) { + clearInterval(status.interval); + return _this._getBackend() + .getTransport() + .close(timeout) + .then(function (transportFlushed) { return status.ready && transportFlushed; }); + }); + }; + /** + * @inheritDoc + */ + BaseClient.prototype.close = function (timeout) { + var _this = this; + return this.flush(timeout).then(function (result) { + _this.getOptions().enabled = false; + return result; + }); + }; + /** + * Sets up the integrations + */ + BaseClient.prototype.setupIntegrations = function () { + if (this._isEnabled()) { + this._integrations = setupIntegrations(this._options); + } + }; + /** + * @inheritDoc + */ + BaseClient.prototype.getIntegration = function (integration) { + try { + return this._integrations[integration.id] || null; + } + catch (_oO) { + logger.warn("Cannot retrieve integration " + integration.id + " from the current Client"); + return null; + } + }; + /** Waits for the client to be done with processing. */ + BaseClient.prototype._isClientProcessing = function (timeout) { + var _this = this; + return new SyncPromise(function (resolve) { + var ticked = 0; + var tick = 1; + var interval = 0; + clearInterval(interval); + interval = setInterval(function () { + if (!_this._processing) { + resolve({ + interval: interval, + ready: true, + }); + } + else { + ticked += tick; + if (timeout && ticked >= timeout) { + resolve({ + interval: interval, + ready: false, + }); + } + } + }, tick); + }); + }; + /** Returns the current backend. */ + BaseClient.prototype._getBackend = function () { + return this._backend; + }; + /** Determines whether this SDK is enabled and a valid Dsn is present. */ + BaseClient.prototype._isEnabled = function () { + return this.getOptions().enabled !== false && this._dsn !== undefined; + }; + /** + * Adds common information to events. + * + * The information includes release and environment from `options`, + * breadcrumbs and context (extra, tags and user) from the scope. + * + * Information that is already present in the event is never overwritten. For + * nested objects, such as the context, keys are merged. + * + * @param event The original event. + * @param hint May contain additional informartion about the original exception. + * @param scope A scope containing event metadata. + * @returns A new event with more information. + */ + BaseClient.prototype._prepareEvent = function (event, scope, hint) { + var _this = this; + var _a = this.getOptions(), environment = _a.environment, release = _a.release, dist = _a.dist, _b = _a.maxValueLength, maxValueLength = _b === void 0 ? 250 : _b, _c = _a.normalizeDepth, normalizeDepth = _c === void 0 ? 3 : _c; + var prepared = __assign({}, event); + if (prepared.environment === undefined && environment !== undefined) { + prepared.environment = environment; + } + if (prepared.release === undefined && release !== undefined) { + prepared.release = release; + } + if (prepared.dist === undefined && dist !== undefined) { + prepared.dist = dist; + } + if (prepared.message) { + prepared.message = truncate(prepared.message, maxValueLength); + } + var exception = prepared.exception && prepared.exception.values && prepared.exception.values[0]; + if (exception && exception.value) { + exception.value = truncate(exception.value, maxValueLength); + } + var request = prepared.request; + if (request && request.url) { + request.url = truncate(request.url, maxValueLength); + } + if (prepared.event_id === undefined) { + prepared.event_id = hint && hint.event_id ? hint.event_id : uuid4(); + } + this._addIntegrations(prepared.sdk); + // We prepare the result here with a resolved Event. + var result = SyncPromise.resolve(prepared); + // This should be the last thing called, since we want that + // {@link Hub.addEventProcessor} gets the finished prepared event. + if (scope) { + // In case we have a hub we reassign it. + result = scope.applyToEvent(prepared, hint); + } + return result.then(function (evt) { + // tslint:disable-next-line:strict-type-predicates + if (typeof normalizeDepth === 'number' && normalizeDepth > 0) { + return _this._normalizeEvent(evt, normalizeDepth); + } + return evt; + }); + }; + /** + * Applies `normalize` function on necessary `Event` attributes to make them safe for serialization. + * Normalized keys: + * - `breadcrumbs.data` + * - `user` + * - `contexts` + * - `extra` + * @param event Event + * @returns Normalized event + */ + BaseClient.prototype._normalizeEvent = function (event, depth) { + if (!event) { + return null; + } + // tslint:disable:no-unsafe-any + return __assign({}, event, (event.breadcrumbs && { + breadcrumbs: event.breadcrumbs.map(function (b) { return (__assign({}, b, (b.data && { + data: normalize(b.data, depth), + }))); }), + }), (event.user && { + user: normalize(event.user, depth), + }), (event.contexts && { + contexts: normalize(event.contexts, depth), + }), (event.extra && { + extra: normalize(event.extra, depth), + })); + }; + /** + * This function adds all used integrations to the SDK info in the event. + * @param sdkInfo The sdkInfo of the event that will be filled with all integrations. + */ + BaseClient.prototype._addIntegrations = function (sdkInfo) { + var integrationsArray = Object.keys(this._integrations); + if (sdkInfo && integrationsArray.length > 0) { + sdkInfo.integrations = integrationsArray; + } + }; + /** + * Processes an event (either error or message) and sends it to Sentry. + * + * This also adds breadcrumbs and context information to the event. However, + * platform specific meta data (such as the User's IP address) must be added + * by the SDK implementor. + * + * + * @param event The event to send to Sentry. + * @param hint May contain additional informartion about the original exception. + * @param scope A scope containing event metadata. + * @returns A SyncPromise that resolves with the event or rejects in case event was/will not be send. + */ + BaseClient.prototype._processEvent = function (event, hint, scope) { + var _this = this; + var _a = this.getOptions(), beforeSend = _a.beforeSend, sampleRate = _a.sampleRate; + if (!this._isEnabled()) { + return SyncPromise.reject('SDK not enabled, will not send event.'); + } + // 1.0 === 100% events are sent + // 0.0 === 0% events are sent + if (typeof sampleRate === 'number' && Math.random() > sampleRate) { + return SyncPromise.reject('This event has been sampled, will not send event.'); + } + return new SyncPromise(function (resolve, reject) { + _this._prepareEvent(event, scope, hint) + .then(function (prepared) { + if (prepared === null) { + reject('An event processor returned null, will not send event.'); + return; + } + var finalEvent = prepared; + var isInternalException = hint && hint.data && hint.data.__sentry__ === true; + if (isInternalException || !beforeSend) { + _this._getBackend().sendEvent(finalEvent); + resolve(finalEvent); + return; + } + var beforeSendResult = beforeSend(prepared, hint); + // tslint:disable-next-line:strict-type-predicates + if (typeof beforeSendResult === 'undefined') { + logger.error('`beforeSend` method has to return `null` or a valid event.'); + } + else if (isThenable(beforeSendResult)) { + _this._handleAsyncBeforeSend(beforeSendResult, resolve, reject); + } + else { + finalEvent = beforeSendResult; + if (finalEvent === null) { + logger.log('`beforeSend` returned `null`, will not send event.'); + resolve(null); + return; + } + // From here on we are really async + _this._getBackend().sendEvent(finalEvent); + resolve(finalEvent); + } + }) + .then(null, function (reason) { + _this.captureException(reason, { + data: { + __sentry__: true, + }, + originalException: reason, + }); + reject("Event processing pipeline threw an error, original event will not be sent. Details have been sent as a new event.\nReason: " + reason); + }); + }); + }; + /** + * Resolves before send Promise and calls resolve/reject on parent SyncPromise. + */ + BaseClient.prototype._handleAsyncBeforeSend = function (beforeSend, resolve, reject) { + var _this = this; + beforeSend + .then(function (processedEvent) { + if (processedEvent === null) { + reject('`beforeSend` returned `null`, will not send event.'); + return; + } + // From here on we are really async + _this._getBackend().sendEvent(processedEvent); + resolve(processedEvent); + }) + .then(null, function (e) { + reject("beforeSend rejected with " + e); + }); + }; + return BaseClient; + }()); + + /** Noop transport */ + var NoopTransport = /** @class */ (function () { + function NoopTransport() { + } + /** + * @inheritDoc + */ + NoopTransport.prototype.sendEvent = function (_) { + return SyncPromise.resolve({ + reason: "NoopTransport: Event has been skipped because no Dsn is configured.", + status: exports.Status.Skipped, + }); + }; + /** + * @inheritDoc + */ + NoopTransport.prototype.close = function (_) { + return SyncPromise.resolve(true); + }; + return NoopTransport; + }()); + + /** + * This is the base implemention of a Backend. + * @hidden + */ + var BaseBackend = /** @class */ (function () { + /** Creates a new backend instance. */ + function BaseBackend(options) { + this._options = options; + if (!this._options.dsn) { + logger.warn('No DSN provided, backend will not do anything.'); + } + this._transport = this._setupTransport(); + } + /** + * Sets up the transport so it can be used later to send requests. + */ + BaseBackend.prototype._setupTransport = function () { + return new NoopTransport(); + }; + /** + * @inheritDoc + */ + BaseBackend.prototype.eventFromException = function (_exception, _hint) { + throw new SentryError('Backend has to implement `eventFromException` method'); + }; + /** + * @inheritDoc + */ + BaseBackend.prototype.eventFromMessage = function (_message, _level, _hint) { + throw new SentryError('Backend has to implement `eventFromMessage` method'); + }; + /** + * @inheritDoc + */ + BaseBackend.prototype.sendEvent = function (event) { + this._transport.sendEvent(event).then(null, function (reason) { + logger.error("Error while sending event: " + reason); + }); + }; + /** + * @inheritDoc + */ + BaseBackend.prototype.getTransport = function () { + return this._transport; + }; + return BaseBackend; + }()); + + /** + * Internal function to create a new SDK client instance. The client is + * installed and then bound to the current scope. + * + * @param clientClass The client class to instanciate. + * @param options Options to pass to the client. + */ + function initAndBind(clientClass, options) { + if (options.debug === true) { + logger.enable(); + } + var hub = getCurrentHub(); + var client = new clientClass(options); + hub.bindClient(client); + } + + var originalFunctionToString; + /** Patch toString calls to return proper name for wrapped functions */ + var FunctionToString = /** @class */ (function () { + function FunctionToString() { + /** + * @inheritDoc + */ + this.name = FunctionToString.id; + } + /** + * @inheritDoc + */ + FunctionToString.prototype.setupOnce = function () { + originalFunctionToString = Function.prototype.toString; + Function.prototype.toString = function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + var context = this.__sentry_original__ || this; + // tslint:disable-next-line:no-unsafe-any + return originalFunctionToString.apply(context, args); + }; + }; + /** + * @inheritDoc + */ + FunctionToString.id = 'FunctionToString'; + return FunctionToString; + }()); + + // "Script error." is hard coded into browsers for errors that it can't read. + // this is the result of a script being pulled in from an external domain and CORS. + var DEFAULT_IGNORE_ERRORS = [/^Script error\.?$/, /^Javascript error: Script error\.? on line 0$/]; + /** Inbound filters configurable by the user */ + var InboundFilters = /** @class */ (function () { + function InboundFilters(_options) { + if (_options === void 0) { _options = {}; } + this._options = _options; + /** + * @inheritDoc + */ + this.name = InboundFilters.id; + } + /** + * @inheritDoc + */ + InboundFilters.prototype.setupOnce = function () { + addGlobalEventProcessor(function (event) { + var hub = getCurrentHub(); + if (!hub) { + return event; + } + var self = hub.getIntegration(InboundFilters); + if (self) { + var client = hub.getClient(); + var clientOptions = client ? client.getOptions() : {}; + var options = self._mergeOptions(clientOptions); + if (self._shouldDropEvent(event, options)) { + return null; + } + } + return event; + }); + }; + /** JSDoc */ + InboundFilters.prototype._shouldDropEvent = function (event, options) { + if (this._isSentryError(event, options)) { + logger.warn("Event dropped due to being internal Sentry Error.\nEvent: " + getEventDescription(event)); + return true; + } + if (this._isIgnoredError(event, options)) { + logger.warn("Event dropped due to being matched by `ignoreErrors` option.\nEvent: " + getEventDescription(event)); + return true; + } + if (this._isBlacklistedUrl(event, options)) { + logger.warn("Event dropped due to being matched by `blacklistUrls` option.\nEvent: " + getEventDescription(event) + ".\nUrl: " + this._getEventFilterUrl(event)); + return true; + } + if (!this._isWhitelistedUrl(event, options)) { + logger.warn("Event dropped due to not being matched by `whitelistUrls` option.\nEvent: " + getEventDescription(event) + ".\nUrl: " + this._getEventFilterUrl(event)); + return true; + } + return false; + }; + /** JSDoc */ + InboundFilters.prototype._isSentryError = function (event, options) { + if (options === void 0) { options = {}; } + if (!options.ignoreInternal) { + return false; + } + try { + return ((event && + event.exception && + event.exception.values && + event.exception.values[0] && + event.exception.values[0].type === 'SentryError') || + false); + } + catch (_oO) { + return false; + } + }; + /** JSDoc */ + InboundFilters.prototype._isIgnoredError = function (event, options) { + if (options === void 0) { options = {}; } + if (!options.ignoreErrors || !options.ignoreErrors.length) { + return false; + } + return this._getPossibleEventMessages(event).some(function (message) { + // Not sure why TypeScript complains here... + return options.ignoreErrors.some(function (pattern) { return isMatchingPattern(message, pattern); }); + }); + }; + /** JSDoc */ + InboundFilters.prototype._isBlacklistedUrl = function (event, options) { + if (options === void 0) { options = {}; } + // TODO: Use Glob instead? + if (!options.blacklistUrls || !options.blacklistUrls.length) { + return false; + } + var url = this._getEventFilterUrl(event); + return !url ? false : options.blacklistUrls.some(function (pattern) { return isMatchingPattern(url, pattern); }); + }; + /** JSDoc */ + InboundFilters.prototype._isWhitelistedUrl = function (event, options) { + if (options === void 0) { options = {}; } + // TODO: Use Glob instead? + if (!options.whitelistUrls || !options.whitelistUrls.length) { + return true; + } + var url = this._getEventFilterUrl(event); + return !url ? true : options.whitelistUrls.some(function (pattern) { return isMatchingPattern(url, pattern); }); + }; + /** JSDoc */ + InboundFilters.prototype._mergeOptions = function (clientOptions) { + if (clientOptions === void 0) { clientOptions = {}; } + return { + blacklistUrls: __spread((this._options.blacklistUrls || []), (clientOptions.blacklistUrls || [])), + ignoreErrors: __spread((this._options.ignoreErrors || []), (clientOptions.ignoreErrors || []), DEFAULT_IGNORE_ERRORS), + ignoreInternal: typeof this._options.ignoreInternal !== 'undefined' ? this._options.ignoreInternal : true, + whitelistUrls: __spread((this._options.whitelistUrls || []), (clientOptions.whitelistUrls || [])), + }; + }; + /** JSDoc */ + InboundFilters.prototype._getPossibleEventMessages = function (event) { + if (event.message) { + return [event.message]; + } + if (event.exception) { + try { + var _a = (event.exception.values && event.exception.values[0]) || {}, _b = _a.type, type = _b === void 0 ? '' : _b, _c = _a.value, value = _c === void 0 ? '' : _c; + return ["" + value, type + ": " + value]; + } + catch (oO) { + logger.error("Cannot extract message for event " + getEventDescription(event)); + return []; + } + } + return []; + }; + /** JSDoc */ + InboundFilters.prototype._getEventFilterUrl = function (event) { + try { + if (event.stacktrace) { + var frames_1 = event.stacktrace.frames; + return (frames_1 && frames_1[frames_1.length - 1].filename) || null; + } + if (event.exception) { + var frames_2 = event.exception.values && event.exception.values[0].stacktrace && event.exception.values[0].stacktrace.frames; + return (frames_2 && frames_2[frames_2.length - 1].filename) || null; + } + return null; + } + catch (oO) { + logger.error("Cannot extract url for event " + getEventDescription(event)); + return null; + } + }; + /** + * @inheritDoc + */ + InboundFilters.id = 'InboundFilters'; + return InboundFilters; + }()); + + + + var CoreIntegrations = /*#__PURE__*/Object.freeze({ + FunctionToString: FunctionToString, + InboundFilters: InboundFilters + }); + + // tslint:disable:object-literal-sort-keys + // global reference to slice + var UNKNOWN_FUNCTION = '?'; + // Chromium based browsers: Chrome, Brave, new Opera, new Edge + var chrome = /^\s*at (?:(.*?) ?\()?((?:file|https?|blob|chrome-extension|address|native|eval|webpack||[-a-z]+:|.*bundle|\/).*?)(?::(\d+))?(?::(\d+))?\)?\s*$/i; + // gecko regex: `(?:bundle|\d+\.js)`: `bundle` is for react native, `\d+\.js` also but specifically for ram bundles because it + // generates filenames without a prefix like `file://` the filenames in the stacktrace are just 42.js + // We need this specific case for now because we want no other regex to match. + var gecko = /^\s*(.*?)(?:\((.*?)\))?(?:^|@)?((?:file|https?|blob|chrome|webpack|resource|moz-extension).*?:\/.*?|\[native code\]|[^@]*(?:bundle|\d+\.js))(?::(\d+))?(?::(\d+))?\s*$/i; + var winjs = /^\s*at (?:((?:\[object object\])?.+) )?\(?((?:file|ms-appx|https?|webpack|blob):.*?):(\d+)(?::(\d+))?\)?\s*$/i; + var geckoEval = /(\S+) line (\d+)(?: > eval line \d+)* > eval/i; + var chromeEval = /\((\S*)(?::(\d+))(?::(\d+))\)/; + /** JSDoc */ + function computeStackTrace(ex) { + // tslint:disable:no-unsafe-any + var stack = null; + var popSize = ex && ex.framesToPop; + try { + // This must be tried first because Opera 10 *destroys* + // its stacktrace property if you try to access the stack + // property first!! + stack = computeStackTraceFromStacktraceProp(ex); + if (stack) { + return popFrames(stack, popSize); + } + } + catch (e) { + // no-empty + } + try { + stack = computeStackTraceFromStackProp(ex); + if (stack) { + return popFrames(stack, popSize); + } + } + catch (e) { + // no-empty + } + return { + message: extractMessage(ex), + name: ex && ex.name, + stack: [], + failed: true, + }; + } + /** JSDoc */ + // tslint:disable-next-line:cyclomatic-complexity + function computeStackTraceFromStackProp(ex) { + // tslint:disable:no-conditional-assignment + if (!ex || !ex.stack) { + return null; + } + var stack = []; + var lines = ex.stack.split('\n'); + var isEval; + var submatch; + var parts; + var element; + for (var i = 0; i < lines.length; ++i) { + if ((parts = chrome.exec(lines[i]))) { + var isNative = parts[2] && parts[2].indexOf('native') === 0; // start of line + isEval = parts[2] && parts[2].indexOf('eval') === 0; // start of line + if (isEval && (submatch = chromeEval.exec(parts[2]))) { + // throw out eval line/column and use top-most line/column number + parts[2] = submatch[1]; // url + parts[3] = submatch[2]; // line + parts[4] = submatch[3]; // column + } + element = { + // working with the regexp above is super painful. it is quite a hack, but just stripping the `address at ` + // prefix here seems like the quickest solution for now. + url: parts[2] && parts[2].indexOf('address at ') === 0 ? parts[2].substr('address at '.length) : parts[2], + func: parts[1] || UNKNOWN_FUNCTION, + args: isNative ? [parts[2]] : [], + line: parts[3] ? +parts[3] : null, + column: parts[4] ? +parts[4] : null, + }; + } + else if ((parts = winjs.exec(lines[i]))) { + element = { + url: parts[2], + func: parts[1] || UNKNOWN_FUNCTION, + args: [], + line: +parts[3], + column: parts[4] ? +parts[4] : null, + }; + } + else if ((parts = gecko.exec(lines[i]))) { + isEval = parts[3] && parts[3].indexOf(' > eval') > -1; + if (isEval && (submatch = geckoEval.exec(parts[3]))) { + // throw out eval line/column and use top-most line number + parts[1] = parts[1] || "eval"; + parts[3] = submatch[1]; + parts[4] = submatch[2]; + parts[5] = ''; // no column when eval + } + else if (i === 0 && !parts[5] && ex.columnNumber !== void 0) { + // FireFox uses this awesome columnNumber property for its top frame + // Also note, Firefox's column number is 0-based and everything else expects 1-based, + // so adding 1 + // NOTE: this hack doesn't work if top-most frame is eval + stack[0].column = ex.columnNumber + 1; + } + element = { + url: parts[3], + func: parts[1] || UNKNOWN_FUNCTION, + args: parts[2] ? parts[2].split(',') : [], + line: parts[4] ? +parts[4] : null, + column: parts[5] ? +parts[5] : null, + }; + } + else { + continue; + } + if (!element.func && element.line) { + element.func = UNKNOWN_FUNCTION; + } + stack.push(element); + } + if (!stack.length) { + return null; + } + return { + message: extractMessage(ex), + name: ex.name, + stack: stack, + }; + } + /** JSDoc */ + function computeStackTraceFromStacktraceProp(ex) { + if (!ex || !ex.stacktrace) { + return null; + } + // Access and store the stacktrace property before doing ANYTHING + // else to it because Opera is not very good at providing it + // reliably in other circumstances. + var stacktrace = ex.stacktrace; + var opera10Regex = / line (\d+).*script (?:in )?(\S+)(?:: in function (\S+))?$/i; + var opera11Regex = / line (\d+), column (\d+)\s*(?:in (?:]+)>|([^\)]+))\((.*)\))? in (.*):\s*$/i; + var lines = stacktrace.split('\n'); + var stack = []; + var parts; + for (var line = 0; line < lines.length; line += 2) { + // tslint:disable:no-conditional-assignment + var element = null; + if ((parts = opera10Regex.exec(lines[line]))) { + element = { + url: parts[2], + func: parts[3], + args: [], + line: +parts[1], + column: null, + }; + } + else if ((parts = opera11Regex.exec(lines[line]))) { + element = { + url: parts[6], + func: parts[3] || parts[4], + args: parts[5] ? parts[5].split(',') : [], + line: +parts[1], + column: +parts[2], + }; + } + if (element) { + if (!element.func && element.line) { + element.func = UNKNOWN_FUNCTION; + } + stack.push(element); + } + } + if (!stack.length) { + return null; + } + return { + message: extractMessage(ex), + name: ex.name, + stack: stack, + }; + } + /** Remove N number of frames from the stack */ + function popFrames(stacktrace, popSize) { + try { + return __assign({}, stacktrace, { stack: stacktrace.stack.slice(popSize) }); + } + catch (e) { + return stacktrace; + } + } + /** + * There are cases where stacktrace.message is an Event object + * https://github.com/getsentry/sentry-javascript/issues/1949 + * In this specific case we try to extract stacktrace.message.error.message + */ + function extractMessage(ex) { + var message = ex && ex.message; + if (!message) { + return 'No error message'; + } + if (message.error && typeof message.error.message === 'string') { + return message.error.message; + } + return message; + } + + var STACKTRACE_LIMIT = 50; + /** + * This function creates an exception from an TraceKitStackTrace + * @param stacktrace TraceKitStackTrace that will be converted to an exception + * @hidden + */ + function exceptionFromStacktrace(stacktrace) { + var frames = prepareFramesForEvent(stacktrace.stack); + var exception = { + type: stacktrace.name, + value: stacktrace.message, + }; + if (frames && frames.length) { + exception.stacktrace = { frames: frames }; + } + // tslint:disable-next-line:strict-type-predicates + if (exception.type === undefined && exception.value === '') { + exception.value = 'Unrecoverable error caught'; + } + return exception; + } + /** + * @hidden + */ + function eventFromPlainObject(exception, syntheticException, rejection) { + var event = { + exception: { + values: [ + { + type: isEvent(exception) ? exception.constructor.name : rejection ? 'UnhandledRejection' : 'Error', + value: "Non-Error " + (rejection ? 'promise rejection' : 'exception') + " captured with keys: " + extractExceptionKeysForMessage(exception), + }, + ], + }, + extra: { + __serialized__: normalizeToSize(exception), + }, + }; + if (syntheticException) { + var stacktrace = computeStackTrace(syntheticException); + var frames_1 = prepareFramesForEvent(stacktrace.stack); + event.stacktrace = { + frames: frames_1, + }; + } + return event; + } + /** + * @hidden + */ + function eventFromStacktrace(stacktrace) { + var exception = exceptionFromStacktrace(stacktrace); + return { + exception: { + values: [exception], + }, + }; + } + /** + * @hidden + */ + function prepareFramesForEvent(stack) { + if (!stack || !stack.length) { + return []; + } + var localStack = stack; + var firstFrameFunction = localStack[0].func || ''; + var lastFrameFunction = localStack[localStack.length - 1].func || ''; + // If stack starts with one of our API calls, remove it (starts, meaning it's the top of the stack - aka last call) + if (firstFrameFunction.indexOf('captureMessage') !== -1 || firstFrameFunction.indexOf('captureException') !== -1) { + localStack = localStack.slice(1); + } + // If stack ends with one of our internal API calls, remove it (ends, meaning it's the bottom of the stack - aka top-most call) + if (lastFrameFunction.indexOf('sentryWrapped') !== -1) { + localStack = localStack.slice(0, -1); + } + // The frame where the crash happened, should be the last entry in the array + return localStack + .map(function (frame) { return ({ + colno: frame.column === null ? undefined : frame.column, + filename: frame.url || localStack[0].url, + function: frame.func || '?', + in_app: true, + lineno: frame.line === null ? undefined : frame.line, + }); }) + .slice(0, STACKTRACE_LIMIT) + .reverse(); + } + + /** JSDoc */ + function eventFromUnknownInput(exception, syntheticException, options) { + if (options === void 0) { options = {}; } + var event; + if (isErrorEvent(exception) && exception.error) { + // If it is an ErrorEvent with `error` property, extract it to get actual Error + var errorEvent = exception; + exception = errorEvent.error; // tslint:disable-line:no-parameter-reassignment + event = eventFromStacktrace(computeStackTrace(exception)); + return event; + } + if (isDOMError(exception) || isDOMException(exception)) { + // If it is a DOMError or DOMException (which are legacy APIs, but still supported in some browsers) + // then we just extract the name and message, as they don't provide anything else + // https://developer.mozilla.org/en-US/docs/Web/API/DOMError + // https://developer.mozilla.org/en-US/docs/Web/API/DOMException + var domException = exception; + var name_1 = domException.name || (isDOMError(domException) ? 'DOMError' : 'DOMException'); + var message = domException.message ? name_1 + ": " + domException.message : name_1; + event = eventFromString(message, syntheticException, options); + addExceptionTypeValue(event, message); + return event; + } + if (isError(exception)) { + // we have a real Error object, do nothing + event = eventFromStacktrace(computeStackTrace(exception)); + return event; + } + if (isPlainObject(exception) || isEvent(exception)) { + // If it is plain Object or Event, serialize it manually and extract options + // This will allow us to group events based on top-level keys + // which is much better than creating new group when any key/value change + var objectException = exception; + event = eventFromPlainObject(objectException, syntheticException, options.rejection); + addExceptionMechanism(event, { + synthetic: true, + }); + return event; + } + // If none of previous checks were valid, then it means that it's not: + // - an instance of DOMError + // - an instance of DOMException + // - an instance of Event + // - an instance of Error + // - a valid ErrorEvent (one with an error property) + // - a plain Object + // + // So bail out and capture it as a simple message: + event = eventFromString(exception, syntheticException, options); + addExceptionTypeValue(event, "" + exception, undefined); + addExceptionMechanism(event, { + synthetic: true, + }); + return event; + } + // this._options.attachStacktrace + /** JSDoc */ + function eventFromString(input, syntheticException, options) { + if (options === void 0) { options = {}; } + var event = { + message: input, + }; + if (options.attachStacktrace && syntheticException) { + var stacktrace = computeStackTrace(syntheticException); + var frames_1 = prepareFramesForEvent(stacktrace.stack); + event.stacktrace = { + frames: frames_1, + }; + } + return event; + } + + /** Base Transport class implementation */ + var BaseTransport = /** @class */ (function () { + function BaseTransport(options) { + this.options = options; + /** A simple buffer holding all requests. */ + this._buffer = new PromiseBuffer(30); + this.url = new API(this.options.dsn).getStoreEndpointWithUrlEncodedAuth(); + } + /** + * @inheritDoc + */ + BaseTransport.prototype.sendEvent = function (_) { + throw new SentryError('Transport Class has to implement `sendEvent` method'); + }; + /** + * @inheritDoc + */ + BaseTransport.prototype.close = function (timeout) { + return this._buffer.drain(timeout); + }; + return BaseTransport; + }()); + + var global$3 = getGlobalObject(); + /** `fetch` based transport */ + var FetchTransport = /** @class */ (function (_super) { + __extends(FetchTransport, _super); + function FetchTransport() { + var _this = _super !== null && _super.apply(this, arguments) || this; + /** Locks transport after receiving 429 response */ + _this._disabledUntil = new Date(Date.now()); + return _this; + } + /** + * @inheritDoc + */ + FetchTransport.prototype.sendEvent = function (event) { + var _this = this; + if (new Date(Date.now()) < this._disabledUntil) { + return Promise.reject({ + event: event, + reason: "Transport locked till " + this._disabledUntil + " due to too many requests.", + status: 429, + }); + } + var defaultOptions = { + body: JSON.stringify(event), + method: 'POST', + // Despite all stars in the sky saying that Edge supports old draft syntax, aka 'never', 'always', 'origin' and 'default + // https://caniuse.com/#feat=referrer-policy + // It doesn't. And it throw exception instead of ignoring this parameter... + // REF: https://github.com/getsentry/raven-js/issues/1233 + referrerPolicy: (supportsReferrerPolicy() ? 'origin' : ''), + }; + if (this.options.headers !== undefined) { + defaultOptions.headers = this.options.headers; + } + return this._buffer.add(new SyncPromise(function (resolve, reject) { + global$3 + .fetch(_this.url, defaultOptions) + .then(function (response) { + var status = exports.Status.fromHttpCode(response.status); + if (status === exports.Status.Success) { + resolve({ status: status }); + return; + } + if (status === exports.Status.RateLimit) { + var now = Date.now(); + _this._disabledUntil = new Date(now + parseRetryAfterHeader(now, response.headers.get('Retry-After'))); + logger.warn("Too many requests, backing off till: " + _this._disabledUntil); + } + reject(response); + }) + .catch(reject); + })); + }; + return FetchTransport; + }(BaseTransport)); + + /** `XHR` based transport */ + var XHRTransport = /** @class */ (function (_super) { + __extends(XHRTransport, _super); + function XHRTransport() { + var _this = _super !== null && _super.apply(this, arguments) || this; + /** Locks transport after receiving 429 response */ + _this._disabledUntil = new Date(Date.now()); + return _this; + } + /** + * @inheritDoc + */ + XHRTransport.prototype.sendEvent = function (event) { + var _this = this; + if (new Date(Date.now()) < this._disabledUntil) { + return Promise.reject({ + event: event, + reason: "Transport locked till " + this._disabledUntil + " due to too many requests.", + status: 429, + }); + } + return this._buffer.add(new SyncPromise(function (resolve, reject) { + var request = new XMLHttpRequest(); + request.onreadystatechange = function () { + if (request.readyState !== 4) { + return; + } + var status = exports.Status.fromHttpCode(request.status); + if (status === exports.Status.Success) { + resolve({ status: status }); + return; + } + if (status === exports.Status.RateLimit) { + var now = Date.now(); + _this._disabledUntil = new Date(now + parseRetryAfterHeader(now, request.getResponseHeader('Retry-After'))); + logger.warn("Too many requests, backing off till: " + _this._disabledUntil); + } + reject(request); + }; + request.open('POST', _this.url); + for (var header in _this.options.headers) { + if (_this.options.headers.hasOwnProperty(header)) { + request.setRequestHeader(header, _this.options.headers[header]); + } + } + request.send(JSON.stringify(event)); + })); + }; + return XHRTransport; + }(BaseTransport)); + + + + var index = /*#__PURE__*/Object.freeze({ + BaseTransport: BaseTransport, + FetchTransport: FetchTransport, + XHRTransport: XHRTransport + }); + + /** + * The Sentry Browser SDK Backend. + * @hidden + */ + var BrowserBackend = /** @class */ (function (_super) { + __extends(BrowserBackend, _super); + function BrowserBackend() { + return _super !== null && _super.apply(this, arguments) || this; + } + /** + * @inheritDoc + */ + BrowserBackend.prototype._setupTransport = function () { + if (!this._options.dsn) { + // We return the noop transport here in case there is no Dsn. + return _super.prototype._setupTransport.call(this); + } + var transportOptions = __assign({}, this._options.transportOptions, { dsn: this._options.dsn }); + if (this._options.transport) { + return new this._options.transport(transportOptions); + } + if (supportsFetch()) { + return new FetchTransport(transportOptions); + } + return new XHRTransport(transportOptions); + }; + /** + * @inheritDoc + */ + BrowserBackend.prototype.eventFromException = function (exception, hint) { + var syntheticException = (hint && hint.syntheticException) || undefined; + var event = eventFromUnknownInput(exception, syntheticException, { + attachStacktrace: this._options.attachStacktrace, + }); + addExceptionMechanism(event, { + handled: true, + type: 'generic', + }); + event.level = exports.Severity.Error; + if (hint && hint.event_id) { + event.event_id = hint.event_id; + } + return SyncPromise.resolve(event); + }; + /** + * @inheritDoc + */ + BrowserBackend.prototype.eventFromMessage = function (message, level, hint) { + if (level === void 0) { level = exports.Severity.Info; } + var syntheticException = (hint && hint.syntheticException) || undefined; + var event = eventFromString(message, syntheticException, { + attachStacktrace: this._options.attachStacktrace, + }); + event.level = level; + if (hint && hint.event_id) { + event.event_id = hint.event_id; + } + return SyncPromise.resolve(event); + }; + return BrowserBackend; + }(BaseBackend)); + + var SDK_NAME = 'sentry.javascript.browser'; + var SDK_VERSION = '5.15.5'; + + /** + * The Sentry Browser SDK Client. + * + * @see BrowserOptions for documentation on configuration options. + * @see SentryClient for usage documentation. + */ + var BrowserClient = /** @class */ (function (_super) { + __extends(BrowserClient, _super); + /** + * Creates a new Browser SDK instance. + * + * @param options Configuration options for this SDK. + */ + function BrowserClient(options) { + if (options === void 0) { options = {}; } + return _super.call(this, BrowserBackend, options) || this; + } + /** + * @inheritDoc + */ + BrowserClient.prototype._prepareEvent = function (event, scope, hint) { + event.platform = event.platform || 'javascript'; + event.sdk = __assign({}, event.sdk, { name: SDK_NAME, packages: __spread(((event.sdk && event.sdk.packages) || []), [ + { + name: 'npm:@sentry/browser', + version: SDK_VERSION, + }, + ]), version: SDK_VERSION }); + return _super.prototype._prepareEvent.call(this, event, scope, hint); + }; + /** + * Show a report dialog to the user to send feedback to a specific event. + * + * @param options Set individual options for the dialog + */ + BrowserClient.prototype.showReportDialog = function (options) { + if (options === void 0) { options = {}; } + // doesn't work without a document (React Native) + var document = getGlobalObject().document; + if (!document) { + return; + } + if (!this._isEnabled()) { + logger.error('Trying to call showReportDialog with Sentry Client is disabled'); + return; + } + var dsn = options.dsn || this.getDsn(); + if (!options.eventId) { + logger.error('Missing `eventId` option in showReportDialog call'); + return; + } + if (!dsn) { + logger.error('Missing `Dsn` option in showReportDialog call'); + return; + } + var script = document.createElement('script'); + script.async = true; + script.src = new API(dsn).getReportDialogEndpoint(options); + if (options.onLoad) { + script.onload = options.onLoad; + } + (document.head || document.body).appendChild(script); + }; + return BrowserClient; + }(BaseClient)); + + var ignoreOnError = 0; + /** + * @hidden + */ + function shouldIgnoreOnError() { + return ignoreOnError > 0; + } + /** + * @hidden + */ + function ignoreNextOnError() { + // onerror should trigger before setTimeout + ignoreOnError += 1; + setTimeout(function () { + ignoreOnError -= 1; + }); + } + /** + * Instruments the given function and sends an event to Sentry every time the + * function throws an exception. + * + * @param fn A function to wrap. + * @returns The wrapped function. + * @hidden + */ + function wrap(fn, options, before) { + if (options === void 0) { options = {}; } + // tslint:disable-next-line:strict-type-predicates + if (typeof fn !== 'function') { + return fn; + } + try { + // We don't wanna wrap it twice + if (fn.__sentry__) { + return fn; + } + // If this has already been wrapped in the past, return that wrapped function + if (fn.__sentry_wrapped__) { + return fn.__sentry_wrapped__; + } + } + catch (e) { + // Just accessing custom props in some Selenium environments + // can cause a "Permission denied" exception (see raven-js#495). + // Bail on wrapping and return the function as-is (defers to window.onerror). + return fn; + } + var sentryWrapped = function () { + var args = Array.prototype.slice.call(arguments); + // tslint:disable:no-unsafe-any + try { + // tslint:disable-next-line:strict-type-predicates + if (before && typeof before === 'function') { + before.apply(this, arguments); + } + var wrappedArguments = args.map(function (arg) { return wrap(arg, options); }); + if (fn.handleEvent) { + // Attempt to invoke user-land function + // NOTE: If you are a Sentry user, and you are seeing this stack frame, it + // means the sentry.javascript SDK caught an error invoking your application code. This + // is expected behavior and NOT indicative of a bug with sentry.javascript. + return fn.handleEvent.apply(this, wrappedArguments); + } + // Attempt to invoke user-land function + // NOTE: If you are a Sentry user, and you are seeing this stack frame, it + // means the sentry.javascript SDK caught an error invoking your application code. This + // is expected behavior and NOT indicative of a bug with sentry.javascript. + return fn.apply(this, wrappedArguments); + // tslint:enable:no-unsafe-any + } + catch (ex) { + ignoreNextOnError(); + withScope(function (scope) { + scope.addEventProcessor(function (event) { + var processedEvent = __assign({}, event); + if (options.mechanism) { + addExceptionTypeValue(processedEvent, undefined, undefined); + addExceptionMechanism(processedEvent, options.mechanism); + } + processedEvent.extra = __assign({}, processedEvent.extra, { arguments: args }); + return processedEvent; + }); + captureException(ex); + }); + throw ex; + } + }; + // Accessing some objects may throw + // ref: https://github.com/getsentry/sentry-javascript/issues/1168 + try { + for (var property in fn) { + if (Object.prototype.hasOwnProperty.call(fn, property)) { + sentryWrapped[property] = fn[property]; + } + } + } + catch (_oO) { } // tslint:disable-line:no-empty + fn.prototype = fn.prototype || {}; + sentryWrapped.prototype = fn.prototype; + Object.defineProperty(fn, '__sentry_wrapped__', { + enumerable: false, + value: sentryWrapped, + }); + // Signal that this function has been wrapped/filled already + // for both debugging and to prevent it to being wrapped/filled twice + Object.defineProperties(sentryWrapped, { + __sentry__: { + enumerable: false, + value: true, + }, + __sentry_original__: { + enumerable: false, + value: fn, + }, + }); + // Restore original function name (not all browsers allow that) + try { + var descriptor = Object.getOwnPropertyDescriptor(sentryWrapped, 'name'); + if (descriptor.configurable) { + Object.defineProperty(sentryWrapped, 'name', { + get: function () { + return fn.name; + }, + }); + } + } + catch (_oO) { + /*no-empty*/ + } + return sentryWrapped; + } + + /** Global handlers */ + var GlobalHandlers = /** @class */ (function () { + /** JSDoc */ + function GlobalHandlers(options) { + /** + * @inheritDoc + */ + this.name = GlobalHandlers.id; + /** JSDoc */ + this._onErrorHandlerInstalled = false; + /** JSDoc */ + this._onUnhandledRejectionHandlerInstalled = false; + this._options = __assign({ onerror: true, onunhandledrejection: true }, options); + } + /** + * @inheritDoc + */ + GlobalHandlers.prototype.setupOnce = function () { + Error.stackTraceLimit = 50; + if (this._options.onerror) { + logger.log('Global Handler attached: onerror'); + this._installGlobalOnErrorHandler(); + } + if (this._options.onunhandledrejection) { + logger.log('Global Handler attached: onunhandledrejection'); + this._installGlobalOnUnhandledRejectionHandler(); + } + }; + /** JSDoc */ + GlobalHandlers.prototype._installGlobalOnErrorHandler = function () { + var _this = this; + if (this._onErrorHandlerInstalled) { + return; + } + addInstrumentationHandler({ + callback: function (data) { + var error = data.error; + var currentHub = getCurrentHub(); + var hasIntegration = currentHub.getIntegration(GlobalHandlers); + var isFailedOwnDelivery = error && error.__sentry_own_request__ === true; + if (!hasIntegration || shouldIgnoreOnError() || isFailedOwnDelivery) { + return; + } + var client = currentHub.getClient(); + var event = isPrimitive(error) + ? _this._eventFromIncompleteOnError(data.msg, data.url, data.line, data.column) + : _this._enhanceEventWithInitialFrame(eventFromUnknownInput(error, undefined, { + attachStacktrace: client && client.getOptions().attachStacktrace, + rejection: false, + }), data.url, data.line, data.column); + addExceptionMechanism(event, { + handled: false, + type: 'onerror', + }); + currentHub.captureEvent(event, { + originalException: error, + }); + }, + type: 'error', + }); + this._onErrorHandlerInstalled = true; + }; + /** JSDoc */ + GlobalHandlers.prototype._installGlobalOnUnhandledRejectionHandler = function () { + var _this = this; + if (this._onUnhandledRejectionHandlerInstalled) { + return; + } + addInstrumentationHandler({ + callback: function (e) { + var error = e; + // dig the object of the rejection out of known event types + try { + // PromiseRejectionEvents store the object of the rejection under 'reason' + // see https://developer.mozilla.org/en-US/docs/Web/API/PromiseRejectionEvent + if ('reason' in e) { + error = e.reason; + } + // something, somewhere, (likely a browser extension) effectively casts PromiseRejectionEvents + // to CustomEvents, moving the `promise` and `reason` attributes of the PRE into + // the CustomEvent's `detail` attribute, since they're not part of CustomEvent's spec + // see https://developer.mozilla.org/en-US/docs/Web/API/CustomEvent and + // https://github.com/getsentry/sentry-javascript/issues/2380 + else if ('detail' in e && 'reason' in e.detail) { + error = e.detail.reason; + } + } + catch (_oO) { + // no-empty + } + var currentHub = getCurrentHub(); + var hasIntegration = currentHub.getIntegration(GlobalHandlers); + var isFailedOwnDelivery = error && error.__sentry_own_request__ === true; + if (!hasIntegration || shouldIgnoreOnError() || isFailedOwnDelivery) { + return true; + } + var client = currentHub.getClient(); + var event = isPrimitive(error) + ? _this._eventFromIncompleteRejection(error) + : eventFromUnknownInput(error, undefined, { + attachStacktrace: client && client.getOptions().attachStacktrace, + rejection: true, + }); + event.level = exports.Severity.Error; + addExceptionMechanism(event, { + handled: false, + type: 'onunhandledrejection', + }); + currentHub.captureEvent(event, { + originalException: error, + }); + return; + }, + type: 'unhandledrejection', + }); + this._onUnhandledRejectionHandlerInstalled = true; + }; + /** + * This function creates a stack from an old, error-less onerror handler. + */ + GlobalHandlers.prototype._eventFromIncompleteOnError = function (msg, url, line, column) { + var ERROR_TYPES_RE = /^(?:[Uu]ncaught (?:exception: )?)?(?:((?:Eval|Internal|Range|Reference|Syntax|Type|URI|)Error): )?(.*)$/i; + // If 'message' is ErrorEvent, get real message from inside + var message = isErrorEvent(msg) ? msg.message : msg; + var name; + if (isString(message)) { + var groups = message.match(ERROR_TYPES_RE); + if (groups) { + name = groups[1]; + message = groups[2]; + } + } + var event = { + exception: { + values: [ + { + type: name || 'Error', + value: message, + }, + ], + }, + }; + return this._enhanceEventWithInitialFrame(event, url, line, column); + }; + /** + * This function creates an Event from an TraceKitStackTrace that has part of it missing. + */ + GlobalHandlers.prototype._eventFromIncompleteRejection = function (error) { + return { + exception: { + values: [ + { + type: 'UnhandledRejection', + value: "Non-Error promise rejection captured with value: " + error, + }, + ], + }, + }; + }; + /** JSDoc */ + GlobalHandlers.prototype._enhanceEventWithInitialFrame = function (event, url, line, column) { + event.exception = event.exception || {}; + event.exception.values = event.exception.values || []; + event.exception.values[0] = event.exception.values[0] || {}; + event.exception.values[0].stacktrace = event.exception.values[0].stacktrace || {}; + event.exception.values[0].stacktrace.frames = event.exception.values[0].stacktrace.frames || []; + var colno = isNaN(parseInt(column, 10)) ? undefined : column; + var lineno = isNaN(parseInt(line, 10)) ? undefined : line; + var filename = isString(url) && url.length > 0 ? url : getLocationHref(); + if (event.exception.values[0].stacktrace.frames.length === 0) { + event.exception.values[0].stacktrace.frames.push({ + colno: colno, + filename: filename, + function: '?', + in_app: true, + lineno: lineno, + }); + } + return event; + }; + /** + * @inheritDoc + */ + GlobalHandlers.id = 'GlobalHandlers'; + return GlobalHandlers; + }()); + + /** Wrap timer functions and event targets to catch errors and provide better meta data */ + var TryCatch = /** @class */ (function () { + function TryCatch() { + /** JSDoc */ + this._ignoreOnError = 0; + /** + * @inheritDoc + */ + this.name = TryCatch.id; + } + /** JSDoc */ + TryCatch.prototype._wrapTimeFunction = function (original) { + return function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + var originalCallback = args[0]; + args[0] = wrap(originalCallback, { + mechanism: { + data: { function: getFunctionName(original) }, + handled: true, + type: 'instrument', + }, + }); + return original.apply(this, args); + }; + }; + /** JSDoc */ + TryCatch.prototype._wrapRAF = function (original) { + return function (callback) { + return original(wrap(callback, { + mechanism: { + data: { + function: 'requestAnimationFrame', + handler: getFunctionName(original), + }, + handled: true, + type: 'instrument', + }, + })); + }; + }; + /** JSDoc */ + TryCatch.prototype._wrapEventTarget = function (target) { + var global = getGlobalObject(); + var proto = global[target] && global[target].prototype; + if (!proto || !proto.hasOwnProperty || !proto.hasOwnProperty('addEventListener')) { + return; + } + fill(proto, 'addEventListener', function (original) { + return function (eventName, fn, options) { + try { + // tslint:disable-next-line:no-unbound-method strict-type-predicates + if (typeof fn.handleEvent === 'function') { + fn.handleEvent = wrap(fn.handleEvent.bind(fn), { + mechanism: { + data: { + function: 'handleEvent', + handler: getFunctionName(fn), + target: target, + }, + handled: true, + type: 'instrument', + }, + }); + } + } + catch (err) { + // can sometimes get 'Permission denied to access property "handle Event' + } + return original.call(this, eventName, wrap(fn, { + mechanism: { + data: { + function: 'addEventListener', + handler: getFunctionName(fn), + target: target, + }, + handled: true, + type: 'instrument', + }, + }), options); + }; + }); + fill(proto, 'removeEventListener', function (original) { + return function (eventName, fn, options) { + var callback = fn; + try { + callback = callback && (callback.__sentry_wrapped__ || callback); + } + catch (e) { + // ignore, accessing __sentry_wrapped__ will throw in some Selenium environments + } + return original.call(this, eventName, callback, options); + }; + }); + }; + /** JSDoc */ + TryCatch.prototype._wrapXHR = function (originalSend) { + return function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + var xhr = this; // tslint:disable-line:no-this-assignment + var xmlHttpRequestProps = ['onload', 'onerror', 'onprogress', 'onreadystatechange']; + xmlHttpRequestProps.forEach(function (prop) { + if (prop in xhr && typeof xhr[prop] === 'function') { + fill(xhr, prop, function (original) { + var wrapOptions = { + mechanism: { + data: { + function: prop, + handler: getFunctionName(original), + }, + handled: true, + type: 'instrument', + }, + }; + // If Instrument integration has been called before TryCatch, get the name of original function + if (original.__sentry_original__) { + wrapOptions.mechanism.data.handler = getFunctionName(original.__sentry_original__); + } + // Otherwise wrap directly + return wrap(original, wrapOptions); + }); + } + }); + return originalSend.apply(this, args); + }; + }; + /** + * Wrap timer functions and event targets to catch errors + * and provide better metadata. + */ + TryCatch.prototype.setupOnce = function () { + this._ignoreOnError = this._ignoreOnError; + var global = getGlobalObject(); + fill(global, 'setTimeout', this._wrapTimeFunction.bind(this)); + fill(global, 'setInterval', this._wrapTimeFunction.bind(this)); + fill(global, 'requestAnimationFrame', this._wrapRAF.bind(this)); + if ('XMLHttpRequest' in global) { + fill(XMLHttpRequest.prototype, 'send', this._wrapXHR.bind(this)); + } + [ + 'EventTarget', + 'Window', + 'Node', + 'ApplicationCache', + 'AudioTrackList', + 'ChannelMergerNode', + 'CryptoOperation', + 'EventSource', + 'FileReader', + 'HTMLUnknownElement', + 'IDBDatabase', + 'IDBRequest', + 'IDBTransaction', + 'KeyOperation', + 'MediaController', + 'MessagePort', + 'ModalWindow', + 'Notification', + 'SVGElementInstance', + 'Screen', + 'TextTrack', + 'TextTrackCue', + 'TextTrackList', + 'WebSocket', + 'WebSocketWorker', + 'Worker', + 'XMLHttpRequest', + 'XMLHttpRequestEventTarget', + 'XMLHttpRequestUpload', + ].forEach(this._wrapEventTarget.bind(this)); + }; + /** + * @inheritDoc + */ + TryCatch.id = 'TryCatch'; + return TryCatch; + }()); + + /** + * Default Breadcrumbs instrumentations + * TODO: Deprecated - with v6, this will be renamed to `Instrument` + */ + var Breadcrumbs = /** @class */ (function () { + /** + * @inheritDoc + */ + function Breadcrumbs(options) { + /** + * @inheritDoc + */ + this.name = Breadcrumbs.id; + this._options = __assign({ console: true, dom: true, fetch: true, history: true, sentry: true, xhr: true }, options); + } + /** + * Creates breadcrumbs from console API calls + */ + Breadcrumbs.prototype._consoleBreadcrumb = function (handlerData) { + var breadcrumb = { + category: 'console', + data: { + arguments: handlerData.args, + logger: 'console', + }, + level: exports.Severity.fromString(handlerData.level), + message: safeJoin(handlerData.args, ' '), + }; + if (handlerData.level === 'assert') { + if (handlerData.args[0] === false) { + breadcrumb.message = "Assertion failed: " + (safeJoin(handlerData.args.slice(1), ' ') || 'console.assert'); + breadcrumb.data.arguments = handlerData.args.slice(1); + } + else { + // Don't capture a breadcrumb for passed assertions + return; + } + } + getCurrentHub().addBreadcrumb(breadcrumb, { + input: handlerData.args, + level: handlerData.level, + }); + }; + /** + * Creates breadcrumbs from DOM API calls + */ + Breadcrumbs.prototype._domBreadcrumb = function (handlerData) { + var target; + // Accessing event.target can throw (see getsentry/raven-js#838, #768) + try { + target = handlerData.event.target + ? htmlTreeAsString(handlerData.event.target) + : htmlTreeAsString(handlerData.event); + } + catch (e) { + target = ''; + } + if (target.length === 0) { + return; + } + getCurrentHub().addBreadcrumb({ + category: "ui." + handlerData.name, + message: target, + }, { + event: handlerData.event, + name: handlerData.name, + }); + }; + /** + * Creates breadcrumbs from XHR API calls + */ + Breadcrumbs.prototype._xhrBreadcrumb = function (handlerData) { + if (handlerData.endTimestamp) { + // We only capture complete, non-sentry requests + if (handlerData.xhr.__sentry_own_request__) { + return; + } + getCurrentHub().addBreadcrumb({ + category: 'xhr', + data: handlerData.xhr.__sentry_xhr__, + type: 'http', + }, { + xhr: handlerData.xhr, + }); + return; + } + // We only capture issued sentry requests + if (this._options.sentry && handlerData.xhr.__sentry_own_request__) { + addSentryBreadcrumb(handlerData.args[0]); + } + }; + /** + * Creates breadcrumbs from fetch API calls + */ + Breadcrumbs.prototype._fetchBreadcrumb = function (handlerData) { + // We only capture complete fetch requests + if (!handlerData.endTimestamp) { + return; + } + var client = getCurrentHub().getClient(); + var dsn = client && client.getDsn(); + if (this._options.sentry && dsn) { + var filterUrl = new API(dsn).getStoreEndpoint(); + // if Sentry key appears in URL, don't capture it as a request + // but rather as our own 'sentry' type breadcrumb + if (filterUrl && + handlerData.fetchData.url.indexOf(filterUrl) !== -1 && + handlerData.fetchData.method === 'POST' && + handlerData.args[1] && + handlerData.args[1].body) { + addSentryBreadcrumb(handlerData.args[1].body); + return; + } + } + if (handlerData.error) { + getCurrentHub().addBreadcrumb({ + category: 'fetch', + data: __assign({}, handlerData.fetchData, { status_code: handlerData.response.status }), + level: exports.Severity.Error, + type: 'http', + }, { + data: handlerData.error, + input: handlerData.args, + }); + } + else { + getCurrentHub().addBreadcrumb({ + category: 'fetch', + data: __assign({}, handlerData.fetchData, { status_code: handlerData.response.status }), + type: 'http', + }, { + input: handlerData.args, + response: handlerData.response, + }); + } + }; + /** + * Creates breadcrumbs from history API calls + */ + Breadcrumbs.prototype._historyBreadcrumb = function (handlerData) { + var global = getGlobalObject(); + var from = handlerData.from; + var to = handlerData.to; + var parsedLoc = parseUrl(global.location.href); + var parsedFrom = parseUrl(from); + var parsedTo = parseUrl(to); + // Initial pushState doesn't provide `from` information + if (!parsedFrom.path) { + parsedFrom = parsedLoc; + } + // Use only the path component of the URL if the URL matches the current + // document (almost all the time when using pushState) + if (parsedLoc.protocol === parsedTo.protocol && parsedLoc.host === parsedTo.host) { + // tslint:disable-next-line:no-parameter-reassignment + to = parsedTo.relative; + } + if (parsedLoc.protocol === parsedFrom.protocol && parsedLoc.host === parsedFrom.host) { + // tslint:disable-next-line:no-parameter-reassignment + from = parsedFrom.relative; + } + getCurrentHub().addBreadcrumb({ + category: 'navigation', + data: { + from: from, + to: to, + }, + }); + }; + /** + * Instrument browser built-ins w/ breadcrumb capturing + * - Console API + * - DOM API (click/typing) + * - XMLHttpRequest API + * - Fetch API + * - History API + */ + Breadcrumbs.prototype.setupOnce = function () { + var _this = this; + if (this._options.console) { + addInstrumentationHandler({ + callback: function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + _this._consoleBreadcrumb.apply(_this, __spread(args)); + }, + type: 'console', + }); + } + if (this._options.dom) { + addInstrumentationHandler({ + callback: function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + _this._domBreadcrumb.apply(_this, __spread(args)); + }, + type: 'dom', + }); + } + if (this._options.xhr) { + addInstrumentationHandler({ + callback: function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + _this._xhrBreadcrumb.apply(_this, __spread(args)); + }, + type: 'xhr', + }); + } + if (this._options.fetch) { + addInstrumentationHandler({ + callback: function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + _this._fetchBreadcrumb.apply(_this, __spread(args)); + }, + type: 'fetch', + }); + } + if (this._options.history) { + addInstrumentationHandler({ + callback: function () { + var args = []; + for (var _i = 0; _i < arguments.length; _i++) { + args[_i] = arguments[_i]; + } + _this._historyBreadcrumb.apply(_this, __spread(args)); + }, + type: 'history', + }); + } + }; + /** + * @inheritDoc + */ + Breadcrumbs.id = 'Breadcrumbs'; + return Breadcrumbs; + }()); + /** + * Create a breadcrumb of `sentry` from the events themselves + */ + function addSentryBreadcrumb(serializedData) { + // There's always something that can go wrong with deserialization... + try { + var event_1 = JSON.parse(serializedData); + getCurrentHub().addBreadcrumb({ + category: "sentry." + (event_1.type === 'transaction' ? 'transaction' : 'event'), + event_id: event_1.event_id, + level: event_1.level || exports.Severity.fromString('error'), + message: getEventDescription(event_1), + }, { + event: event_1, + }); + } + catch (_oO) { + logger.error('Error while adding sentry type breadcrumb'); + } + } + + var DEFAULT_KEY = 'cause'; + var DEFAULT_LIMIT = 5; + /** Adds SDK info to an event. */ + var LinkedErrors = /** @class */ (function () { + /** + * @inheritDoc + */ + function LinkedErrors(options) { + if (options === void 0) { options = {}; } + /** + * @inheritDoc + */ + this.name = LinkedErrors.id; + this._key = options.key || DEFAULT_KEY; + this._limit = options.limit || DEFAULT_LIMIT; + } + /** + * @inheritDoc + */ + LinkedErrors.prototype.setupOnce = function () { + addGlobalEventProcessor(function (event, hint) { + var self = getCurrentHub().getIntegration(LinkedErrors); + if (self) { + return self._handler(event, hint); + } + return event; + }); + }; + /** + * @inheritDoc + */ + LinkedErrors.prototype._handler = function (event, hint) { + if (!event.exception || !event.exception.values || !hint || !isInstanceOf(hint.originalException, Error)) { + return event; + } + var linkedErrors = this._walkErrorTree(hint.originalException, this._key); + event.exception.values = __spread(linkedErrors, event.exception.values); + return event; + }; + /** + * @inheritDoc + */ + LinkedErrors.prototype._walkErrorTree = function (error, key, stack) { + if (stack === void 0) { stack = []; } + if (!isInstanceOf(error[key], Error) || stack.length + 1 >= this._limit) { + return stack; + } + var stacktrace = computeStackTrace(error[key]); + var exception = exceptionFromStacktrace(stacktrace); + return this._walkErrorTree(error[key], key, __spread([exception], stack)); + }; + /** + * @inheritDoc + */ + LinkedErrors.id = 'LinkedErrors'; + return LinkedErrors; + }()); + + var global$4 = getGlobalObject(); + /** UserAgent */ + var UserAgent = /** @class */ (function () { + function UserAgent() { + /** + * @inheritDoc + */ + this.name = UserAgent.id; + } + /** + * @inheritDoc + */ + UserAgent.prototype.setupOnce = function () { + addGlobalEventProcessor(function (event) { + if (getCurrentHub().getIntegration(UserAgent)) { + if (!global$4.navigator || !global$4.location) { + return event; + } + // Request Interface: https://docs.sentry.io/development/sdk-dev/event-payloads/request/ + var request = event.request || {}; + request.url = request.url || global$4.location.href; + request.headers = request.headers || {}; + request.headers['User-Agent'] = global$4.navigator.userAgent; + return __assign({}, event, { request: request }); + } + return event; + }); + }; + /** + * @inheritDoc + */ + UserAgent.id = 'UserAgent'; + return UserAgent; + }()); + + + + var BrowserIntegrations = /*#__PURE__*/Object.freeze({ + GlobalHandlers: GlobalHandlers, + TryCatch: TryCatch, + Breadcrumbs: Breadcrumbs, + LinkedErrors: LinkedErrors, + UserAgent: UserAgent + }); + + var defaultIntegrations = [ + new InboundFilters(), + new FunctionToString(), + new TryCatch(), + new Breadcrumbs(), + new GlobalHandlers(), + new LinkedErrors(), + new UserAgent(), + ]; + /** + * The Sentry Browser SDK Client. + * + * To use this SDK, call the {@link init} function as early as possible when + * loading the web page. To set context information or send manual events, use + * the provided methods. + * + * @example + * + * ``` + * + * import { init } from '@sentry/browser'; + * + * init({ + * dsn: '__DSN__', + * // ... + * }); + * ``` + * + * @example + * ``` + * + * import { configureScope } from '@sentry/browser'; + * configureScope((scope: Scope) => { + * scope.setExtra({ battery: 0.7 }); + * scope.setTag({ user_mode: 'admin' }); + * scope.setUser({ id: '4711' }); + * }); + * ``` + * + * @example + * ``` + * + * import { addBreadcrumb } from '@sentry/browser'; + * addBreadcrumb({ + * message: 'My Breadcrumb', + * // ... + * }); + * ``` + * + * @example + * + * ``` + * + * import * as Sentry from '@sentry/browser'; + * Sentry.captureMessage('Hello, world!'); + * Sentry.captureException(new Error('Good bye')); + * Sentry.captureEvent({ + * message: 'Manual', + * stacktrace: [ + * // ... + * ], + * }); + * ``` + * + * @see {@link BrowserOptions} for documentation on configuration options. + */ + function init(options) { + if (options === void 0) { options = {}; } + if (options.defaultIntegrations === undefined) { + options.defaultIntegrations = defaultIntegrations; + } + if (options.release === undefined) { + var window_1 = getGlobalObject(); + // This supports the variable that sentry-webpack-plugin injects + if (window_1.SENTRY_RELEASE && window_1.SENTRY_RELEASE.id) { + options.release = window_1.SENTRY_RELEASE.id; + } + } + initAndBind(BrowserClient, options); + } + /** + * Present the user with a report dialog. + * + * @param options Everything is optional, we try to fetch all info need from the global scope. + */ + function showReportDialog(options) { + if (options === void 0) { options = {}; } + if (!options.eventId) { + options.eventId = getCurrentHub().lastEventId(); + } + var client = getCurrentHub().getClient(); + if (client) { + client.showReportDialog(options); + } + } + /** + * This is the getter for lastEventId. + * + * @returns The last event id of a captured event. + */ + function lastEventId() { + return getCurrentHub().lastEventId(); + } + /** + * This function is here to be API compatible with the loader. + * @hidden + */ + function forceLoad() { + // Noop + } + /** + * This function is here to be API compatible with the loader. + * @hidden + */ + function onLoad(callback) { + callback(); + } + /** + * A promise that resolves when all current events have been sent. + * If you provide a timeout and the queue takes longer to drain the promise returns false. + * + * @param timeout Maximum time in ms the client should wait. + */ + function flush(timeout) { + var client = getCurrentHub().getClient(); + if (client) { + return client.flush(timeout); + } + return SyncPromise.reject(false); + } + /** + * A promise that resolves when all current events have been sent. + * If you provide a timeout and the queue takes longer to drain the promise returns false. + * + * @param timeout Maximum time in ms the client should wait. + */ + function close(timeout) { + var client = getCurrentHub().getClient(); + if (client) { + return client.close(timeout); + } + return SyncPromise.reject(false); + } + /** + * Wrap code within a try/catch block so the SDK is able to capture errors. + * + * @param fn A function to wrap. + * + * @returns The result of wrapped function call. + */ + function wrap$1(fn) { + return wrap(fn)(); // tslint:disable-line:no-unsafe-any + } + + var windowIntegrations = {}; + // This block is needed to add compatibility with the integrations packages when used with a CDN + // tslint:disable: no-unsafe-any + var _window = getGlobalObject(); + if (_window.Sentry && _window.Sentry.Integrations) { + windowIntegrations = _window.Sentry.Integrations; + } + // tslint:enable: no-unsafe-any + var INTEGRATIONS = __assign({}, windowIntegrations, CoreIntegrations, BrowserIntegrations); + + exports.BrowserClient = BrowserClient; + exports.Hub = Hub; + exports.Integrations = INTEGRATIONS; + exports.SDK_NAME = SDK_NAME; + exports.SDK_VERSION = SDK_VERSION; + exports.Scope = Scope; + exports.Transports = index; + exports.addBreadcrumb = addBreadcrumb; + exports.addGlobalEventProcessor = addGlobalEventProcessor; + exports.captureEvent = captureEvent; + exports.captureException = captureException; + exports.captureMessage = captureMessage; + exports.close = close; + exports.configureScope = configureScope; + exports.defaultIntegrations = defaultIntegrations; + exports.flush = flush; + exports.forceLoad = forceLoad; + exports.getCurrentHub = getCurrentHub; + exports.getHubFromCarrier = getHubFromCarrier; + exports.init = init; + exports.lastEventId = lastEventId; + exports.onLoad = onLoad; + exports.setContext = setContext; + exports.setExtra = setExtra; + exports.setExtras = setExtras; + exports.setTag = setTag; + exports.setTags = setTags; + exports.setUser = setUser; + exports.showReportDialog = showReportDialog; + exports.withScope = withScope; + exports.wrap = wrap$1; + + return exports; + +}({})); +//# sourceMappingURL=bundle.js.map diff --git a/website/locale/en/LC_MESSAGES/messages.po b/website/locale/en/LC_MESSAGES/messages.po index 0caf8970259..ee9b7b2cc4f 100644 --- a/website/locale/en/LC_MESSAGES/messages.po +++ b/website/locale/en/LC_MESSAGES/messages.po @@ -8,7 +8,7 @@ msgid "" msgstr "" "Project-Id-Version: PROJECT VERSION\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2020-05-08 17:57+0300\n" +"POT-Creation-Date: 2020-05-19 11:11+0300\n" "PO-Revision-Date: YEAR-MO-DA HO:MI+ZONE\n" "Last-Translator: FULL NAME \n" "Language: en\n" @@ -37,27 +37,27 @@ msgstr "ClickHouse - fast open-source OLAP DBMS" msgid "ClickHouse DBMS" msgstr "ClickHouse DBMS" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "open-source" msgstr "open-source" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "relational" msgstr "relational" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytics" msgstr "analytics" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytical" msgstr "analytical" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "Big Data" msgstr "Big Data" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "web-analytics" msgstr "web-analytics" @@ -77,15 +77,51 @@ msgstr "" msgid "Yandex LLC" msgstr "Yandex LLC" +#: templates/docs/footer.html:3 +msgid "Rating" +msgstr "Rating" + +#: templates/docs/footer.html:3 +msgid "votes" +msgstr "votes" + #: templates/docs/footer.html:4 +msgid "Article Rating" +msgstr "Article Rating" + +#: templates/docs/footer.html:4 +msgid "Was this content helpful?" +msgstr "Was this content helpful?" + +#: templates/docs/footer.html:7 +msgid "Unusable" +msgstr "Unusable" + +#: templates/docs/footer.html:7 +msgid "Poor" +msgstr "Poor" + +#: templates/docs/footer.html:7 +msgid "Good" +msgstr "Good" + +#: templates/docs/footer.html:7 +msgid "Excellent" +msgstr "Excellent" + +#: templates/docs/footer.html:8 +msgid "documentation" +msgstr "documentation" + +#: templates/docs/footer.html:15 msgid "Built from" msgstr "Built from" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "published on" msgstr "published on" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "modified on" msgstr "modified on" diff --git a/website/locale/es/LC_MESSAGES/messages.mo b/website/locale/es/LC_MESSAGES/messages.mo index 8a44b54eaf5..b6bd03868f8 100644 Binary files a/website/locale/es/LC_MESSAGES/messages.mo and b/website/locale/es/LC_MESSAGES/messages.mo differ diff --git a/website/locale/es/LC_MESSAGES/messages.po b/website/locale/es/LC_MESSAGES/messages.po index 1a1aa85fb44..10fb0a88aa1 100644 --- a/website/locale/es/LC_MESSAGES/messages.po +++ b/website/locale/es/LC_MESSAGES/messages.po @@ -7,7 +7,7 @@ msgid "" msgstr "" "Project-Id-Version: PROJECT VERSION\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2020-05-08 17:57+0300\n" +"POT-Creation-Date: 2020-05-19 11:11+0300\n" "PO-Revision-Date: 2020-03-26 10:19+0300\n" "Last-Translator: FULL NAME \n" "Language: es\n" @@ -36,27 +36,27 @@ msgstr "ClickHouse - DBMS OLAP de código abierto rápido" msgid "ClickHouse DBMS" msgstr "Sistema abierto." -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "open-source" msgstr "de código abierto" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "relational" msgstr "relacional" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytics" msgstr "analítica" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytical" msgstr "analítico" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "Big Data" msgstr "Grandes Datos" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "web-analytics" msgstr "Sistema abierto." @@ -76,15 +76,51 @@ msgstr "" msgid "Yandex LLC" msgstr "Sistema abierto." +#: templates/docs/footer.html:3 +msgid "Rating" +msgstr "Clasificación" + +#: templates/docs/footer.html:3 +msgid "votes" +msgstr "voto" + #: templates/docs/footer.html:4 +msgid "Article Rating" +msgstr "Clasificación del artículo" + +#: templates/docs/footer.html:4 +msgid "Was this content helpful?" +msgstr "¿Este contenido fue útil?" + +#: templates/docs/footer.html:7 +msgid "Unusable" +msgstr "Inutilizable" + +#: templates/docs/footer.html:7 +msgid "Poor" +msgstr "Pobre" + +#: templates/docs/footer.html:7 +msgid "Good" +msgstr "Bien" + +#: templates/docs/footer.html:7 +msgid "Excellent" +msgstr "Excelente" + +#: templates/docs/footer.html:8 +msgid "documentation" +msgstr "documentación" + +#: templates/docs/footer.html:15 msgid "Built from" msgstr "Construido a partir de" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "published on" msgstr "publicado en" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "modified on" msgstr "modificado en" diff --git a/website/locale/fa/LC_MESSAGES/messages.mo b/website/locale/fa/LC_MESSAGES/messages.mo index 1b3bd3ff610..ef35154a13b 100644 Binary files a/website/locale/fa/LC_MESSAGES/messages.mo and b/website/locale/fa/LC_MESSAGES/messages.mo differ diff --git a/website/locale/fa/LC_MESSAGES/messages.po b/website/locale/fa/LC_MESSAGES/messages.po index 4290aba44b2..9340903800d 100644 --- a/website/locale/fa/LC_MESSAGES/messages.po +++ b/website/locale/fa/LC_MESSAGES/messages.po @@ -7,7 +7,7 @@ msgid "" msgstr "" "Project-Id-Version: PROJECT VERSION\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2020-05-08 17:57+0300\n" +"POT-Creation-Date: 2020-05-19 11:11+0300\n" "PO-Revision-Date: 2020-03-26 10:19+0300\n" "Last-Translator: FULL NAME \n" "Language: fa\n" @@ -36,27 +36,27 @@ msgstr "ClickHouse - سریع باز-منبع OLAP DBMS" msgid "ClickHouse DBMS" msgstr "خانه عروسکی" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "open-source" msgstr "متن باز" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "relational" msgstr "رابطه" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytics" msgstr "تجزیه و تحلیل" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytical" msgstr "تحلیلی" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "Big Data" msgstr "داده های بزرگ" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "web-analytics" msgstr "تجزیه و تحلیل وب سایت" @@ -76,15 +76,51 @@ msgstr "" msgid "Yandex LLC" msgstr "Yandex, LLC" +#: templates/docs/footer.html:3 +msgid "Rating" +msgstr "درجهبندی" + +#: templates/docs/footer.html:3 +msgid "votes" +msgstr "رای" + #: templates/docs/footer.html:4 +msgid "Article Rating" +msgstr "رتبه مقاله" + +#: templates/docs/footer.html:4 +msgid "Was this content helpful?" +msgstr "این مطالب مفید بود?" + +#: templates/docs/footer.html:7 +msgid "Unusable" +msgstr "غیرقابل استفاده" + +#: templates/docs/footer.html:7 +msgid "Poor" +msgstr "فقیر" + +#: templates/docs/footer.html:7 +msgid "Good" +msgstr "خوبه" + +#: templates/docs/footer.html:7 +msgid "Excellent" +msgstr "عالیه" + +#: templates/docs/footer.html:8 +msgid "documentation" +msgstr "مستندات" + +#: templates/docs/footer.html:15 msgid "Built from" msgstr "ساخته شده از" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "published on" msgstr "منتشر شده در" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "modified on" msgstr "تغییریافته" diff --git a/website/locale/fr/LC_MESSAGES/messages.mo b/website/locale/fr/LC_MESSAGES/messages.mo index 1b56afa4533..a5fe91d114e 100644 Binary files a/website/locale/fr/LC_MESSAGES/messages.mo and b/website/locale/fr/LC_MESSAGES/messages.mo differ diff --git a/website/locale/fr/LC_MESSAGES/messages.po b/website/locale/fr/LC_MESSAGES/messages.po index aa332939179..dcbe4b5ee8b 100644 --- a/website/locale/fr/LC_MESSAGES/messages.po +++ b/website/locale/fr/LC_MESSAGES/messages.po @@ -7,7 +7,7 @@ msgid "" msgstr "" "Project-Id-Version: PROJECT VERSION\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2020-05-08 17:57+0300\n" +"POT-Creation-Date: 2020-05-19 11:11+0300\n" "PO-Revision-Date: 2020-03-30 15:12+0300\n" "Last-Translator: FULL NAME \n" "Language: fr\n" @@ -36,27 +36,27 @@ msgstr "ClickHouse-SGBD OLAP open-source rapide" msgid "ClickHouse DBMS" msgstr "SGBD ClickHouse" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "open-source" msgstr "open-source" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "relational" msgstr "relationnel" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytics" msgstr "Analytics" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytical" msgstr "analytique" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "Big Data" msgstr "Big Data" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "web-analytics" msgstr "web-analytics" @@ -76,15 +76,51 @@ msgstr "" msgid "Yandex LLC" msgstr "Yandex LLC" +#: templates/docs/footer.html:3 +msgid "Rating" +msgstr "Évaluation" + +#: templates/docs/footer.html:3 +msgid "votes" +msgstr "vote" + #: templates/docs/footer.html:4 +msgid "Article Rating" +msgstr "L'Article De Notation" + +#: templates/docs/footer.html:4 +msgid "Was this content helpful?" +msgstr "Ce contenu a été utile?" + +#: templates/docs/footer.html:7 +msgid "Unusable" +msgstr "Inutilisable" + +#: templates/docs/footer.html:7 +msgid "Poor" +msgstr "Pauvre" + +#: templates/docs/footer.html:7 +msgid "Good" +msgstr "Bien" + +#: templates/docs/footer.html:7 +msgid "Excellent" +msgstr "Excellent" + +#: templates/docs/footer.html:8 +msgid "documentation" +msgstr "documentation" + +#: templates/docs/footer.html:15 msgid "Built from" msgstr "Construit à partir de" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "published on" msgstr "publié le" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "modified on" msgstr "modifié sur" diff --git a/website/locale/ja/LC_MESSAGES/messages.mo b/website/locale/ja/LC_MESSAGES/messages.mo index 879e68bd8eb..191276fb16d 100644 Binary files a/website/locale/ja/LC_MESSAGES/messages.mo and b/website/locale/ja/LC_MESSAGES/messages.mo differ diff --git a/website/locale/ja/LC_MESSAGES/messages.po b/website/locale/ja/LC_MESSAGES/messages.po index ba40c41edf1..54dc1d401b6 100644 --- a/website/locale/ja/LC_MESSAGES/messages.po +++ b/website/locale/ja/LC_MESSAGES/messages.po @@ -7,7 +7,7 @@ msgid "" msgstr "" "Project-Id-Version: PROJECT VERSION\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2020-05-08 17:57+0300\n" +"POT-Creation-Date: 2020-05-19 11:11+0300\n" "PO-Revision-Date: 2020-03-26 10:19+0300\n" "Last-Translator: FULL NAME \n" "Language: ja\n" @@ -33,27 +33,27 @@ msgstr "ツ環板篠ョツ嘉ッツ偲青エツδツ-ツエツスツ-ツシツ" msgid "ClickHouse DBMS" msgstr "クリックハウスDBMS" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "open-source" msgstr "オープンソース" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "relational" msgstr "関係" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytics" msgstr "analytics" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytical" msgstr "分析" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "Big Data" msgstr "ビッグデータ" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "web-analytics" msgstr "ウェブ分析" @@ -71,15 +71,51 @@ msgstr "ソフトウェアは、明示または黙示を問わず、いかなる msgid "Yandex LLC" msgstr "Yandex LLC" +#: templates/docs/footer.html:3 +msgid "Rating" +msgstr "評価" + +#: templates/docs/footer.html:3 +msgid "votes" +msgstr "投票" + #: templates/docs/footer.html:4 +msgid "Article Rating" +msgstr "記事の評価" + +#: templates/docs/footer.html:4 +msgid "Was this content helpful?" +msgstr "この内容は有用だったか。" + +#: templates/docs/footer.html:7 +msgid "Unusable" +msgstr "使用不可" + +#: templates/docs/footer.html:7 +msgid "Poor" +msgstr "貧しい" + +#: templates/docs/footer.html:7 +msgid "Good" +msgstr "よし" + +#: templates/docs/footer.html:7 +msgid "Excellent" +msgstr "優れた" + +#: templates/docs/footer.html:8 +msgid "documentation" +msgstr "文書" + +#: templates/docs/footer.html:15 msgid "Built from" msgstr "から構築" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "published on" msgstr "掲載され" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "modified on" msgstr "に変更" diff --git a/website/locale/messages.pot b/website/locale/messages.pot index b209f40175b..2e1259fc0c6 100644 --- a/website/locale/messages.pot +++ b/website/locale/messages.pot @@ -8,7 +8,7 @@ msgid "" msgstr "" "Project-Id-Version: PROJECT VERSION\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2020-05-08 17:57+0300\n" +"POT-Creation-Date: 2020-05-19 11:11+0300\n" "PO-Revision-Date: YEAR-MO-DA HO:MI+ZONE\n" "Last-Translator: FULL NAME \n" "Language-Team: LANGUAGE \n" @@ -32,27 +32,27 @@ msgstr "" msgid "ClickHouse DBMS" msgstr "" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "open-source" msgstr "" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "relational" msgstr "" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytics" msgstr "" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytical" msgstr "" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "Big Data" msgstr "" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "web-analytics" msgstr "" @@ -70,15 +70,51 @@ msgstr "" msgid "Yandex LLC" msgstr "" +#: templates/docs/footer.html:3 +msgid "Rating" +msgstr "" + +#: templates/docs/footer.html:3 +msgid "votes" +msgstr "" + #: templates/docs/footer.html:4 +msgid "Article Rating" +msgstr "" + +#: templates/docs/footer.html:4 +msgid "Was this content helpful?" +msgstr "" + +#: templates/docs/footer.html:7 +msgid "Unusable" +msgstr "" + +#: templates/docs/footer.html:7 +msgid "Poor" +msgstr "" + +#: templates/docs/footer.html:7 +msgid "Good" +msgstr "" + +#: templates/docs/footer.html:7 +msgid "Excellent" +msgstr "" + +#: templates/docs/footer.html:8 +msgid "documentation" +msgstr "" + +#: templates/docs/footer.html:15 msgid "Built from" msgstr "" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "published on" msgstr "" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "modified on" msgstr "" diff --git a/website/locale/ru/LC_MESSAGES/messages.mo b/website/locale/ru/LC_MESSAGES/messages.mo index 6d0187452dd..d6d6af10b60 100644 Binary files a/website/locale/ru/LC_MESSAGES/messages.mo and b/website/locale/ru/LC_MESSAGES/messages.mo differ diff --git a/website/locale/ru/LC_MESSAGES/messages.po b/website/locale/ru/LC_MESSAGES/messages.po index 5bb1fa8cef2..df9e1492c27 100644 --- a/website/locale/ru/LC_MESSAGES/messages.po +++ b/website/locale/ru/LC_MESSAGES/messages.po @@ -7,7 +7,7 @@ msgid "" msgstr "" "Project-Id-Version: PROJECT VERSION\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2020-05-08 17:57+0300\n" +"POT-Creation-Date: 2020-05-19 11:11+0300\n" "PO-Revision-Date: 2020-03-26 10:19+0300\n" "Last-Translator: FULL NAME \n" "Language: ru\n" @@ -38,27 +38,27 @@ msgstr "ClickHouse-быстрая СУБД OLAP с открытым исходн msgid "ClickHouse DBMS" msgstr "СУБД ClickHouse" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "open-source" msgstr "открытый исходный код" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "relational" msgstr "реляционный" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytics" msgstr "аналитика" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytical" msgstr "аналитический" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "Big Data" msgstr "большие данные" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "web-analytics" msgstr "веб-аналитика" @@ -78,15 +78,51 @@ msgstr "" msgid "Yandex LLC" msgstr "ООО «Яндекс»" +#: templates/docs/footer.html:3 +msgid "Rating" +msgstr "Рейтинг" + +#: templates/docs/footer.html:3 +msgid "votes" +msgstr "голоса" + #: templates/docs/footer.html:4 +msgid "Article Rating" +msgstr "Рейтинг Статей" + +#: templates/docs/footer.html:4 +msgid "Was this content helpful?" +msgstr "Был ли этот контент полезным?" + +#: templates/docs/footer.html:7 +msgid "Unusable" +msgstr "Непригодная" + +#: templates/docs/footer.html:7 +msgid "Poor" +msgstr "Плохая" + +#: templates/docs/footer.html:7 +msgid "Good" +msgstr "Хорошая" + +#: templates/docs/footer.html:7 +msgid "Excellent" +msgstr "Отличная" + +#: templates/docs/footer.html:8 +msgid "documentation" +msgstr "документация" + +#: templates/docs/footer.html:15 msgid "Built from" msgstr "Собрано из" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "published on" msgstr "опубликовано" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "modified on" msgstr "изменено" diff --git a/website/locale/tr/LC_MESSAGES/messages.mo b/website/locale/tr/LC_MESSAGES/messages.mo index 938a800f430..a8bb8901286 100644 Binary files a/website/locale/tr/LC_MESSAGES/messages.mo and b/website/locale/tr/LC_MESSAGES/messages.mo differ diff --git a/website/locale/tr/LC_MESSAGES/messages.po b/website/locale/tr/LC_MESSAGES/messages.po index 99fa010afd7..35cd573b0b3 100644 --- a/website/locale/tr/LC_MESSAGES/messages.po +++ b/website/locale/tr/LC_MESSAGES/messages.po @@ -7,7 +7,7 @@ msgid "" msgstr "" "Project-Id-Version: PROJECT VERSION\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2020-05-08 17:57+0300\n" +"POT-Creation-Date: 2020-05-19 11:11+0300\n" "PO-Revision-Date: 2020-04-15 13:17+0000\n" "Last-Translator: FULL NAME \n" "Language: tr\n" @@ -36,27 +36,27 @@ msgstr "ClickHouse - hızlı açık kaynak OLAP DBMS" msgid "ClickHouse DBMS" msgstr "ClickHouse DBMS" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "open-source" msgstr "açık kaynak" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "relational" msgstr "ilişkisel" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytics" msgstr "analiz" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytical" msgstr "analitik" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "Big Data" msgstr "Büyük Veri" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "web-analytics" msgstr "web-analyt -ics" @@ -76,15 +76,51 @@ msgstr "" msgid "Yandex LLC" msgstr "Yandex LLC" +#: templates/docs/footer.html:3 +msgid "Rating" +msgstr "Verim" + +#: templates/docs/footer.html:3 +msgid "votes" +msgstr "oylar" + #: templates/docs/footer.html:4 +msgid "Article Rating" +msgstr "Makale Değerlendirme" + +#: templates/docs/footer.html:4 +msgid "Was this content helpful?" +msgstr "Bu içerik yararlı mıydı?" + +#: templates/docs/footer.html:7 +msgid "Unusable" +msgstr "Kullanışsız" + +#: templates/docs/footer.html:7 +msgid "Poor" +msgstr "Zavallı" + +#: templates/docs/footer.html:7 +msgid "Good" +msgstr "İyi" + +#: templates/docs/footer.html:7 +msgid "Excellent" +msgstr "Mükemmel" + +#: templates/docs/footer.html:8 +msgid "documentation" +msgstr "belge" + +#: templates/docs/footer.html:15 msgid "Built from" msgstr "Dahili" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "published on" msgstr "yayınlanan" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "modified on" msgstr "modifiye on" diff --git a/website/locale/zh/LC_MESSAGES/messages.mo b/website/locale/zh/LC_MESSAGES/messages.mo index 9c36aa66bc4..0b2efedd38b 100644 Binary files a/website/locale/zh/LC_MESSAGES/messages.mo and b/website/locale/zh/LC_MESSAGES/messages.mo differ diff --git a/website/locale/zh/LC_MESSAGES/messages.po b/website/locale/zh/LC_MESSAGES/messages.po index f509557fed1..31195654358 100644 --- a/website/locale/zh/LC_MESSAGES/messages.po +++ b/website/locale/zh/LC_MESSAGES/messages.po @@ -7,7 +7,7 @@ msgid "" msgstr "" "Project-Id-Version: PROJECT VERSION\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2020-05-08 17:57+0300\n" +"POT-Creation-Date: 2020-05-19 11:11+0300\n" "PO-Revision-Date: 2020-03-26 10:19+0300\n" "Last-Translator: FULL NAME \n" "Language: zh\n" @@ -33,27 +33,27 @@ msgstr "ツ暗ェツ氾环催ツ団ツ法ツ人" msgid "ClickHouse DBMS" msgstr "ツ环板msョツ嘉ッツ偲" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "open-source" msgstr "开源" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "relational" msgstr "关系" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytics" msgstr "分析" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "analytical" msgstr "分析" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "Big Data" msgstr "大数据" -#: templates/common_meta.html:21 +#: templates/common_meta.html:24 msgid "web-analytics" msgstr "网络分析" @@ -71,15 +71,51 @@ msgstr "软件按\"原样\"分发,不附带任何明示或暗示的担保或 msgid "Yandex LLC" msgstr "Yandex LLC" +#: templates/docs/footer.html:3 +msgid "Rating" +msgstr "评分" + +#: templates/docs/footer.html:3 +msgid "votes" +msgstr "所得票数" + #: templates/docs/footer.html:4 +msgid "Article Rating" +msgstr "文章评级" + +#: templates/docs/footer.html:4 +msgid "Was this content helpful?" +msgstr "这个内容有帮助??" + +#: templates/docs/footer.html:7 +msgid "Unusable" +msgstr "无法使用" + +#: templates/docs/footer.html:7 +msgid "Poor" +msgstr "差" + +#: templates/docs/footer.html:7 +msgid "Good" +msgstr "很好" + +#: templates/docs/footer.html:7 +msgid "Excellent" +msgstr "善乎哉!" + +#: templates/docs/footer.html:8 +msgid "documentation" +msgstr "文件" + +#: templates/docs/footer.html:15 msgid "Built from" msgstr "建于" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "published on" msgstr "发表于" -#: templates/docs/footer.html:4 +#: templates/docs/footer.html:15 msgid "modified on" msgstr "修改于" diff --git a/website/main.html b/website/main.html index 18aa61d192e..07647c24a2d 100644 --- a/website/main.html +++ b/website/main.html @@ -11,6 +11,10 @@ {% set url = 'https://clickhouse.tech' + page.abs_url %} {% endif %} +{% if page and page.canonical_url %} + {% set canonical_url = page.canonical_url %} +{% endif %} + {% if page and page.meta and page.meta.title %} {% set title = page.meta.title %} {% elif page and page.title and not page.is_homepage and page.title != 'hidden' %} @@ -20,6 +24,7 @@ {% else %} {% set title = config.site_name %} {% endif %} +{% set title = title.strip() %} {% if page and page.content and not single_page %} {% set description = page.content|striptags %} {% set description = description.replace('¶','')[0:120] %} diff --git a/website/templates/common_meta.html b/website/templates/common_meta.html index adcae49f41b..09bef26d575 100644 --- a/website/templates/common_meta.html +++ b/website/templates/common_meta.html @@ -12,7 +12,7 @@ - + {% if page and not single_page %} {% endif %} diff --git a/website/templates/docs/amp.html b/website/templates/docs/amp.html index 7aac9ea6df1..a60270d23d0 100644 --- a/website/templates/docs/amp.html +++ b/website/templates/docs/amp.html @@ -4,19 +4,19 @@ {{ title }} - + {% include "templates/docs/ld_json.html" %} - +
@@ -31,7 +31,7 @@ {% include "templates/docs/footer.html" %}
-
+
diff --git a/website/templates/docs/footer.html b/website/templates/docs/footer.html index 8b23e602069..c9432a0b4dc 100644 --- a/website/templates/docs/footer.html +++ b/website/templates/docs/footer.html @@ -1,5 +1,16 @@ {% if not single_page %} -