Merge branch 'master' into decimal_rounding_fix

This commit is contained in:
mergify[bot] 2022-06-16 10:53:49 +00:00 committed by GitHub
commit 2557e8ad51
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
84 changed files with 2627 additions and 870 deletions

View File

@ -9,6 +9,8 @@ concurrency:
on: # yamllint disable-line rule:truthy
schedule:
- cron: '0 */3 * * *'
workflow_dispatch:
jobs:
CherryPick:
runs-on: [self-hosted, style-checker]

View File

@ -1,4 +1,5 @@
### Table of Contents
**[ClickHouse release v22.6, 2022-06-16](#226)**<br>
**[ClickHouse release v22.5, 2022-05-19](#225)**<br>
**[ClickHouse release v22.4, 2022-04-20](#224)**<br>
**[ClickHouse release v22.3-lts, 2022-03-17](#223)**<br>
@ -6,6 +7,172 @@
**[ClickHouse release v22.1, 2022-01-18](#221)**<br>
**[Changelog for 2021](https://clickhouse.com/docs/en/whats-new/changelog/2021/)**<br>
### <a id="226"></a> ClickHouse release 22.6, 2022-06-16
#### Backward Incompatible Change
* Remove support for octal number literals in SQL. In previous versions they were parsed as Float64. [#37765](https://github.com/ClickHouse/ClickHouse/pull/37765) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Changes how settings using `seconds` as type are parsed to support floating point values (for example: `max_execution_time=0.5`). Infinity or NaN values will throw an exception. [#37187](https://github.com/ClickHouse/ClickHouse/pull/37187) ([Raúl Marín](https://github.com/Algunenano)).
* Changed format of binary serialization of columns of experimental type `Object`. New format is more convenient to implement by third-party clients. [#37482](https://github.com/ClickHouse/ClickHouse/pull/37482) ([Anton Popov](https://github.com/CurtizJ)).
* Turn on setting `output_format_json_named_tuples_as_objects` by default. It allows to serialize named tuples as JSON objects in JSON formats. [#37756](https://github.com/ClickHouse/ClickHouse/pull/37756) ([Anton Popov](https://github.com/CurtizJ)).
* LIKE patterns with trailing escape symbol ('\\') are now disallowed (as mandated by the SQL standard). [#37764](https://github.com/ClickHouse/ClickHouse/pull/37764) ([Robert Schulze](https://github.com/rschu1ze)).
* If you run different ClickHouse versions on a cluster with AArch64 CPU or mix AArch64 and amd64 on a cluster, and use distributed queries with GROUP BY multiple keys of fixed-size type that fit in 256 bits but don't fit in 64 bits, and the size of the result is huge, the data will not be fully aggregated in the result of these queries during upgrade. Workaround: upgrade with downtime instead of a rolling upgrade.
#### New Feature
* A new codec [FPC](https://userweb.cs.txstate.edu/~burtscher/papers/dcc07a.pdf) algorithm for floating point data compression. [#37553](https://github.com/ClickHouse/ClickHouse/pull/37553) ([Mikhail Guzov](https://github.com/koloshmet)).
* Add new columnar JSON formats: `JSONColumns`, `JSONCompactColumns`, `JSONColumnsWithMetadata`. Closes [#36338](https://github.com/ClickHouse/ClickHouse/issues/36338) Closes [#34509](https://github.com/ClickHouse/ClickHouse/issues/34509). [#36975](https://github.com/ClickHouse/ClickHouse/pull/36975) ([Kruglov Pavel](https://github.com/Avogar)).
* Added open telemetry traces visualizing tool based on d3js. [#37810](https://github.com/ClickHouse/ClickHouse/pull/37810) ([Sergei Trifonov](https://github.com/serxa)).
* Support INSERTs into `system.zookeeper` table. Closes [#22130](https://github.com/ClickHouse/ClickHouse/issues/22130). [#37596](https://github.com/ClickHouse/ClickHouse/pull/37596) ([Han Fei](https://github.com/hanfei1991)).
* Support non-constant pattern argument for `LIKE`, `ILIKE` and `match` functions. [#37251](https://github.com/ClickHouse/ClickHouse/pull/37251) ([Robert Schulze](https://github.com/rschu1ze)).
* Executable user defined functions now support parameters. Example: `SELECT test_function(parameters)(arguments)`. Closes [#37578](https://github.com/ClickHouse/ClickHouse/issues/37578). [#37720](https://github.com/ClickHouse/ClickHouse/pull/37720) ([Maksim Kita](https://github.com/kitaisreal)).
* Add `merge_reason` column to system.part_log table. [#36912](https://github.com/ClickHouse/ClickHouse/pull/36912) ([Sema Checherinda](https://github.com/CheSema)).
* Add support for Maps and Records in Avro format. Add new setting `input_format_avro_null_as_default ` that allow to insert null as default in Avro format. Closes [#18925](https://github.com/ClickHouse/ClickHouse/issues/18925) Closes [#37378](https://github.com/ClickHouse/ClickHouse/issues/37378) Closes [#32899](https://github.com/ClickHouse/ClickHouse/issues/32899). [#37525](https://github.com/ClickHouse/ClickHouse/pull/37525) ([Kruglov Pavel](https://github.com/Avogar)).
* Add `clickhouse-disks` tool to introspect and operate on virtual filesystems configured for ClickHouse. [#36060](https://github.com/ClickHouse/ClickHouse/pull/36060) ([Artyom Yurkov](https://github.com/Varinara)).
* Adds H3 unidirectional edge functions. [#36843](https://github.com/ClickHouse/ClickHouse/pull/36843) ([Bharat Nallan](https://github.com/bharatnc)).
* Add support for calculating [hashids](https://hashids.org/) from unsigned integers. [#37013](https://github.com/ClickHouse/ClickHouse/pull/37013) ([Michael Nutt](https://github.com/mnutt)).
* Explicit `SALT` specification is allowed for `CREATE USER <user> IDENTIFIED WITH sha256_hash`. [#37377](https://github.com/ClickHouse/ClickHouse/pull/37377) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Add two new settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines` to allow skipping specified number of lines in the beginning of the file in CSV/TSV formats. [#37537](https://github.com/ClickHouse/ClickHouse/pull/37537) ([Kruglov Pavel](https://github.com/Avogar)).
* `showCertificate` function shows current server's SSL certificate. [#37540](https://github.com/ClickHouse/ClickHouse/pull/37540) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* HTTP source for Data Dictionaries in Named Collections is supported. [#37581](https://github.com/ClickHouse/ClickHouse/pull/37581) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Added a new window function `nonNegativeDerivative(metric_column, timestamp_column[, INTERVAL x SECOND])`. [#37628](https://github.com/ClickHouse/ClickHouse/pull/37628) ([Andrey Zvonov](https://github.com/zvonand)).
* Implemented changing the comment for `ReplicatedMergeTree` tables. [#37416](https://github.com/ClickHouse/ClickHouse/pull/37416) ([Vasily Nemkov](https://github.com/Enmk)).
* Added `SYSTEM UNFREEZE` query that deletes the whole backup regardless if the corresponding table is deleted or not. [#36424](https://github.com/ClickHouse/ClickHouse/pull/36424) ([Vadim Volodin](https://github.com/PolyProgrammist)).
#### Experimental Feature
* Add `GROUPING` function. Closes [#19426](https://github.com/ClickHouse/ClickHouse/issues/19426). [#37163](https://github.com/ClickHouse/ClickHouse/pull/37163) ([Dmitry Novik](https://github.com/novikd)). The behavior of this function will be changed in subsequent releases.
* Enables `POPULATE` for `WINDOW VIEW`. [#36945](https://github.com/ClickHouse/ClickHouse/pull/36945) ([vxider](https://github.com/Vxider)).
* `ALTER TABLE ... MODIFY QUERY` support for `WINDOW VIEW`. [#37188](https://github.com/ClickHouse/ClickHouse/pull/37188) ([vxider](https://github.com/Vxider)).
* This PR changes the behavior of the `ENGINE` syntax in `WINDOW VIEW`, to make it like in `MATERIALIZED VIEW`. [#37214](https://github.com/ClickHouse/ClickHouse/pull/37214) ([vxider](https://github.com/Vxider)).
#### Performance Improvement
* Added numerous optimizations for ARM NEON [#38093](https://github.com/ClickHouse/ClickHouse/pull/38093)([Daniel Kutenin](https://github.com/danlark1)), ([Alexandra Pilipyuk](https://github.com/chalice19)) Note: if you run different ClickHouse versions on a cluster with ARM CPU and use distributed queries with GROUP BY multiple keys of fixed-size type that fit in 256 bits but don't fit in 64 bits, the result of the aggregation query will be wrong during upgrade. Workaround: upgrade with downtime instead of a rolling upgrade.
* Improve performance and memory usage for select of subset of columns for formats Native, Protobuf, CapnProto, JSONEachRow, TSKV, all formats with suffixes WithNames/WithNamesAndTypes. Previously while selecting only subset of columns from files in these formats all columns were read and stored in memory. Now only required columns are read. This PR enables setting `input_format_skip_unknown_fields` by default, because otherwise in case of select of subset of columns exception will be thrown. [#37192](https://github.com/ClickHouse/ClickHouse/pull/37192) ([Kruglov Pavel](https://github.com/Avogar)).
* Now more filters can be pushed down for join. [#37472](https://github.com/ClickHouse/ClickHouse/pull/37472) ([Amos Bird](https://github.com/amosbird)).
* Load marks for only necessary columns when reading wide parts. [#36879](https://github.com/ClickHouse/ClickHouse/pull/36879) ([Anton Kozlov](https://github.com/tonickkozlov)).
* Improved performance of aggregation in case, when sparse columns (can be enabled by experimental setting `ratio_of_defaults_for_sparse_serialization` in `MergeTree` tables) are used as arguments in aggregate functions. [#37617](https://github.com/ClickHouse/ClickHouse/pull/37617) ([Anton Popov](https://github.com/CurtizJ)).
* Optimize function `COALESCE` with two arguments. [#37666](https://github.com/ClickHouse/ClickHouse/pull/37666) ([Anton Popov](https://github.com/CurtizJ)).
* Replace `multiIf` to `if` in case when `multiIf` has only one condition, because function `if` is more performant. [#37695](https://github.com/ClickHouse/ClickHouse/pull/37695) ([Anton Popov](https://github.com/CurtizJ)).
* Improve performance of `dictGetDescendants`, `dictGetChildren` functions, create temporary parent to children hierarchical index per query, not per function call during query. Allow to specify `BIDIRECTIONAL` for `HIERARHICAL` attributes, dictionary will maintain parent to children index in memory, that way functions `dictGetDescendants`, `dictGetChildren` will not create temporary index per query. Closes [#32481](https://github.com/ClickHouse/ClickHouse/issues/32481). [#37148](https://github.com/ClickHouse/ClickHouse/pull/37148) ([Maksim Kita](https://github.com/kitaisreal)).
* Aggregates state destruction now may be posted on a thread pool. For queries with LIMIT and big state it provides significant speedup, e.g. `select uniq(number) from numbers_mt(1e7) group by number limit 100` became around 2.5x faster. [#37855](https://github.com/ClickHouse/ClickHouse/pull/37855) ([Nikita Taranov](https://github.com/nickitat)).
* Improve sort performance by single column. [#37195](https://github.com/ClickHouse/ClickHouse/pull/37195) ([Maksim Kita](https://github.com/kitaisreal)).
* Improve performance of single column sorting using sorting queue specializations. [#37990](https://github.com/ClickHouse/ClickHouse/pull/37990) ([Maksim Kita](https://github.com/kitaisreal)).
* Improved performance on array norm and distance functions 2x-4x times. [#37394](https://github.com/ClickHouse/ClickHouse/pull/37394) ([Alexander Gololobov](https://github.com/davenger)).
* Improve performance of number comparison functions using dynamic dispatch. [#37399](https://github.com/ClickHouse/ClickHouse/pull/37399) ([Maksim Kita](https://github.com/kitaisreal)).
* Improve performance of ORDER BY with LIMIT. [#37481](https://github.com/ClickHouse/ClickHouse/pull/37481) ([Maksim Kita](https://github.com/kitaisreal)).
* Improve performance of `hasAll` function using dynamic dispatch infrastructure. [#37484](https://github.com/ClickHouse/ClickHouse/pull/37484) ([Maksim Kita](https://github.com/kitaisreal)).
* Improve performance of `greatCircleAngle`, `greatCircleDistance`, `geoDistance` functions. [#37524](https://github.com/ClickHouse/ClickHouse/pull/37524) ([Maksim Kita](https://github.com/kitaisreal)).
* Improve performance of insert into MergeTree if there are multiple columns in ORDER BY. [#35762](https://github.com/ClickHouse/ClickHouse/pull/35762) ([Maksim Kita](https://github.com/kitaisreal)).
* Fix excessive CPU usage in background when there are a lot of tables. [#38028](https://github.com/ClickHouse/ClickHouse/pull/38028) ([Maksim Kita](https://github.com/kitaisreal)).
* Improve performance of `not` function using dynamic dispatch. [#38058](https://github.com/ClickHouse/ClickHouse/pull/38058) ([Maksim Kita](https://github.com/kitaisreal)).
* Optimized the internal caching of re2 patterns which occur e.g. in LIKE and MATCH functions. [#37544](https://github.com/ClickHouse/ClickHouse/pull/37544) ([Robert Schulze](https://github.com/rschu1ze)).
* Improve filter bitmask generator function all in one with AVX-512 instructions. [#37588](https://github.com/ClickHouse/ClickHouse/pull/37588) ([yaqi-zhao](https://github.com/yaqi-zhao)).
* Apply read method `threadpool` for Hive integration engine. This will significantly speed up reading. [#36328](https://github.com/ClickHouse/ClickHouse/pull/36328) ([李扬](https://github.com/taiyang-li)).
* When all the columns to read are partition keys, construct columns by the file's row number without real reading the Hive file. [#37103](https://github.com/ClickHouse/ClickHouse/pull/37103) ([lgbo](https://github.com/lgbo-ustc)).
* Support multi disks for caching hive files. [#37279](https://github.com/ClickHouse/ClickHouse/pull/37279) ([lgbo](https://github.com/lgbo-ustc)).
* Limiting the maximum cache usage per query can effectively prevent cache pool contamination. [Related Issues](https://github.com/ClickHouse/ClickHouse/issues/28961). [#37859](https://github.com/ClickHouse/ClickHouse/pull/37859) ([Han Shukai](https://github.com/KinderRiven)).
* Currently clickhouse directly downloads all remote files to the local cache (even if they are only read once), which will frequently cause IO of the local hard disk. In some scenarios, these IOs may not be necessary and may easily cause negative optimization. As shown in the figure below, when we run SSB Q1-Q4, the performance of the cache has caused negative optimization. [#37516](https://github.com/ClickHouse/ClickHouse/pull/37516) ([Han Shukai](https://github.com/KinderRiven)).
* Allow to prune the list of files via virtual columns such as `_file` and `_path` when reading from S3. This is for [#37174](https://github.com/ClickHouse/ClickHouse/issues/37174) , [#23494](https://github.com/ClickHouse/ClickHouse/issues/23494). [#37356](https://github.com/ClickHouse/ClickHouse/pull/37356) ([Amos Bird](https://github.com/amosbird)).
* In function: CompressedWriteBuffer::nextImpl(), there is an unnecessary write-copy step that would happen frequently during inserting data. Below shows the differentiation with this patch: - Before: 1. Compress "working_buffer" into "compressed_buffer" 2. write-copy into "out" - After: Directly Compress "working_buffer" into "out". [#37242](https://github.com/ClickHouse/ClickHouse/pull/37242) ([jasperzhu](https://github.com/jinjunzh)).
#### Improvement
* Support types with non-standard defaults in ROLLUP, CUBE, GROUPING SETS. Closes [#37360](https://github.com/ClickHouse/ClickHouse/issues/37360). [#37667](https://github.com/ClickHouse/ClickHouse/pull/37667) ([Dmitry Novik](https://github.com/novikd)).
* Fix stack traces collection on ARM. Closes [#37044](https://github.com/ClickHouse/ClickHouse/issues/37044). Closes [#15638](https://github.com/ClickHouse/ClickHouse/issues/15638). [#37797](https://github.com/ClickHouse/ClickHouse/pull/37797) ([Maksim Kita](https://github.com/kitaisreal)).
* Client will try every IP address returned by DNS resolution until successful connection. [#37273](https://github.com/ClickHouse/ClickHouse/pull/37273) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Allow to use String type instead of Binary in Arrow/Parquet/ORC formats. This PR introduces 3 new settings for it: `output_format_arrow_string_as_string`, `output_format_parquet_string_as_string`, `output_format_orc_string_as_string`. Default value for all settings is `false`. [#37327](https://github.com/ClickHouse/ClickHouse/pull/37327) ([Kruglov Pavel](https://github.com/Avogar)).
* Apply setting `input_format_max_rows_to_read_for_schema_inference` for all read rows in total from all files in globs. Previously setting `input_format_max_rows_to_read_for_schema_inference` was applied for each file in glob separately and in case of huge number of nulls we could read first `input_format_max_rows_to_read_for_schema_inference` rows from each file and get nothing. Also increase default value for this setting to 25000. [#37332](https://github.com/ClickHouse/ClickHouse/pull/37332) ([Kruglov Pavel](https://github.com/Avogar)).
* Add separate `CLUSTER` grant (and `access_control_improvements.on_cluster_queries_require_cluster_grant` configuration directive, for backward compatibility, default to `false`). [#35767](https://github.com/ClickHouse/ClickHouse/pull/35767) ([Azat Khuzhin](https://github.com/azat)).
* Added support for schema inference for `hdfsCluster`. [#35812](https://github.com/ClickHouse/ClickHouse/pull/35812) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Implement `least_used` load balancing algorithm for disks inside volume (multi disk configuration). [#36686](https://github.com/ClickHouse/ClickHouse/pull/36686) ([Azat Khuzhin](https://github.com/azat)).
* Modify the HTTP Endpoint to return the full stats under the `X-ClickHouse-Summary` header when `send_progress_in_http_headers=0` (before it would return all zeros). - Modify the HTTP Endpoint to return `X-ClickHouse-Exception-Code` header when progress has been sent before (`send_progress_in_http_headers=1`) - Modify the HTTP Endpoint to return `HTTP_REQUEST_TIMEOUT` (408) instead of `HTTP_INTERNAL_SERVER_ERROR` (500) on `TIMEOUT_EXCEEDED` errors. [#36884](https://github.com/ClickHouse/ClickHouse/pull/36884) ([Raúl Marín](https://github.com/Algunenano)).
* Allow a user to inspect grants from granted roles. [#36941](https://github.com/ClickHouse/ClickHouse/pull/36941) ([nvartolomei](https://github.com/nvartolomei)).
* Do not calculate an integral numerically but use CDF functions instead. This will speed up execution and will increase the precision. This fixes [#36714](https://github.com/ClickHouse/ClickHouse/issues/36714). [#36953](https://github.com/ClickHouse/ClickHouse/pull/36953) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Add default implementation for Nothing in functions. Now most of the functions will return column with type Nothing in case one of it's arguments is Nothing. It also solves problem with functions like arrayMap/arrayFilter and similar when they have empty array as an argument. Previously queries like `select arrayMap(x -> 2 * x, []);` failed because function inside lambda cannot work with type `Nothing`, now such queries return empty array with type `Array(Nothing)`. Also add support for arrays of nullable types in functions like arrayFilter/arrayFill. Previously, queries like `select arrayFilter(x -> x % 2, [1, NULL])` failed, now they work (if the result of lambda is NULL, then this value won't be included in the result). Closes [#37000](https://github.com/ClickHouse/ClickHouse/issues/37000). [#37048](https://github.com/ClickHouse/ClickHouse/pull/37048) ([Kruglov Pavel](https://github.com/Avogar)).
* Now if a shard has local replica we create a local plan and a plan to read from all remote replicas. They have shared initiator which coordinates reading. [#37204](https://github.com/ClickHouse/ClickHouse/pull/37204) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Do no longer abort server startup if configuration option "mark_cache_size" is not explicitly set. [#37326](https://github.com/ClickHouse/ClickHouse/pull/37326) ([Robert Schulze](https://github.com/rschu1ze)).
* Allows providing `NULL`/`NOT NULL` right after type in column declaration. [#37337](https://github.com/ClickHouse/ClickHouse/pull/37337) ([Igor Nikonov](https://github.com/devcrafter)).
* optimize file segment PARTIALLY_DOWNLOADED get read buffer. [#37338](https://github.com/ClickHouse/ClickHouse/pull/37338) ([xiedeyantu](https://github.com/xiedeyantu)).
* Try to improve short circuit functions processing to fix problems with stress tests. [#37384](https://github.com/ClickHouse/ClickHouse/pull/37384) ([Kruglov Pavel](https://github.com/Avogar)).
* Closes [#37395](https://github.com/ClickHouse/ClickHouse/issues/37395). [#37415](https://github.com/ClickHouse/ClickHouse/pull/37415) ([Memo](https://github.com/Joeywzr)).
* Fix extremely rare deadlock during part fetch in zero-copy replication. Fixes [#37423](https://github.com/ClickHouse/ClickHouse/issues/37423). [#37424](https://github.com/ClickHouse/ClickHouse/pull/37424) ([metahys](https://github.com/metahys)).
* Don't allow to create storage with unknown data format. [#37450](https://github.com/ClickHouse/ClickHouse/pull/37450) ([Kruglov Pavel](https://github.com/Avogar)).
* Set `global_memory_usage_overcommit_max_wait_microseconds` default value to 5 seconds. Add info about `OvercommitTracker` to OOM exception message. Add `MemoryOvercommitWaitTimeMicroseconds` profile event. [#37460](https://github.com/ClickHouse/ClickHouse/pull/37460) ([Dmitry Novik](https://github.com/novikd)).
* Do not display `-0.0` CPU time in clickhouse-client. It can appear due to rounding errors. This closes [#38003](https://github.com/ClickHouse/ClickHouse/issues/38003). This closes [#38038](https://github.com/ClickHouse/ClickHouse/issues/38038). [#38064](https://github.com/ClickHouse/ClickHouse/pull/38064) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Play UI: Keep controls in place when the page is scrolled horizontally. This makes edits comfortable even if the table is wide and it was scrolled far to the right. The feature proposed by Maksym Tereshchenko from CaspianDB. [#37470](https://github.com/ClickHouse/ClickHouse/pull/37470) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Modify query div in play.html to be extendable beyond 20% height. In case of very long queries it is helpful to extend the textarea element, only today, since the div is fixed height, the extended textarea hides the data div underneath. With this fix, extending the textarea element will push the data div down/up such the extended textarea won't hide it. Also, keeps query box width 100% even when the user adjusting the size of the query textarea. [#37488](https://github.com/ClickHouse/ClickHouse/pull/37488) ([guyco87](https://github.com/guyco87)).
* Added `ProfileEvents` for introspection of type of written (inserted or merged) parts (`Inserted{Wide/Compact/InMemory}Parts`, `MergedInto{Wide/Compact/InMemory}Parts`. Added column `part_type` to `system.part_log`. Resolves [#37495](https://github.com/ClickHouse/ClickHouse/issues/37495). [#37536](https://github.com/ClickHouse/ClickHouse/pull/37536) ([Anton Popov](https://github.com/CurtizJ)).
* clickhouse-keeper improvement: move broken logs to a timestamped folder. [#37565](https://github.com/ClickHouse/ClickHouse/pull/37565) ([Antonio Andelic](https://github.com/antonio2368)).
* Do not write expired columns by TTL after subsequent merges (before only first merge/optimize of the part will not write expired by TTL columns, all other will do). [#37570](https://github.com/ClickHouse/ClickHouse/pull/37570) ([Azat Khuzhin](https://github.com/azat)).
* More precise result of the `dumpColumnStructure` miscellaneous function in presence of LowCardinality or Sparse columns. In previous versions, these functions were converting the argument to a full column before returning the result. This is needed to provide an answer in [#6935](https://github.com/ClickHouse/ClickHouse/issues/6935). [#37633](https://github.com/ClickHouse/ClickHouse/pull/37633) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* clickhouse-keeper: store only unique session IDs for watches. [#37641](https://github.com/ClickHouse/ClickHouse/pull/37641) ([Azat Khuzhin](https://github.com/azat)).
* Fix possible "Cannot write to finalized buffer". [#37645](https://github.com/ClickHouse/ClickHouse/pull/37645) ([Azat Khuzhin](https://github.com/azat)).
* Add setting `support_batch_delete` for `DiskS3` to disable multiobject delete calls, which Google Cloud Storage doesn't support. [#37659](https://github.com/ClickHouse/ClickHouse/pull/37659) ([Fred Wulff](https://github.com/frew)).
* Add an option to disable connection pooling in ODBC bridge. [#37705](https://github.com/ClickHouse/ClickHouse/pull/37705) ([Anton Kozlov](https://github.com/tonickkozlov)).
* Functions `dictGetHierarchy`, `dictIsIn`, `dictGetChildren`, `dictGetDescendants` added support nullable `HIERARCHICAL` attribute in dictionaries. Closes [#35521](https://github.com/ClickHouse/ClickHouse/issues/35521). [#37805](https://github.com/ClickHouse/ClickHouse/pull/37805) ([Maksim Kita](https://github.com/kitaisreal)).
* Expose BoringSSL version related info in the `system.build_options` table. [#37850](https://github.com/ClickHouse/ClickHouse/pull/37850) ([Bharat Nallan](https://github.com/bharatnc)).
* Now clickhouse-server removes `delete_tmp` directories on server start. Fixes [#26503](https://github.com/ClickHouse/ClickHouse/issues/26503). [#37906](https://github.com/ClickHouse/ClickHouse/pull/37906) ([alesapin](https://github.com/alesapin)).
* Clean up broken detached parts after timeout. Closes [#25195](https://github.com/ClickHouse/ClickHouse/issues/25195). [#37975](https://github.com/ClickHouse/ClickHouse/pull/37975) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Now in MergeTree table engines family failed-to-move parts will be removed instantly. [#37994](https://github.com/ClickHouse/ClickHouse/pull/37994) ([alesapin](https://github.com/alesapin)).
* Now if setting `always_fetch_merged_part` is enabled for ReplicatedMergeTree merges will try to find parts on other replicas rarely with smaller load for [Zoo]Keeper. [#37995](https://github.com/ClickHouse/ClickHouse/pull/37995) ([alesapin](https://github.com/alesapin)).
* Add implicit grants with grant option too. For example `GRANT CREATE TABLE ON test.* TO A WITH GRANT OPTION` now allows `A` to execute `GRANT CREATE VIEW ON test.* TO B`. [#38017](https://github.com/ClickHouse/ClickHouse/pull/38017) ([Vitaly Baranov](https://github.com/vitlibar)).
#### Build/Testing/Packaging Improvement
* Use `clang-14` and LLVM infrastructure version 14 for builds. This closes [#34681](https://github.com/ClickHouse/ClickHouse/issues/34681). [#34754](https://github.com/ClickHouse/ClickHouse/pull/34754) ([Alexey Milovidov](https://github.com/alexey-milovidov)). Note: `clang-14` has [a bug](https://github.com/google/sanitizers/issues/1540) in ThreadSanitizer that makes our CI work worse.
* Allow to drop privileges at startup. This simplifies Docker images. Closes [#36293](https://github.com/ClickHouse/ClickHouse/issues/36293). [#36341](https://github.com/ClickHouse/ClickHouse/pull/36341) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add docs spellcheck to CI. [#37790](https://github.com/ClickHouse/ClickHouse/pull/37790) ([Vladimir C](https://github.com/vdimir)).
* Fix overly aggressive stripping which removed the embedded hash required for checking the consistency of the executable. [#37993](https://github.com/ClickHouse/ClickHouse/pull/37993) ([Robert Schulze](https://github.com/rschu1ze)).
#### Bug Fix
* Fix `SELECT ... INTERSECT` and `EXCEPT SELECT` statements with constant string types. [#37738](https://github.com/ClickHouse/ClickHouse/pull/37738) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix `GROUP BY` `AggregateFunction` (i.e. you `GROUP BY` by the column that has `AggregateFunction` type). [#37093](https://github.com/ClickHouse/ClickHouse/pull/37093) ([Azat Khuzhin](https://github.com/azat)).
* (experimental WINDOW VIEW) Fix `addDependency` in WindowView. This bug can be reproduced like [#37237](https://github.com/ClickHouse/ClickHouse/issues/37237). [#37224](https://github.com/ClickHouse/ClickHouse/pull/37224) ([vxider](https://github.com/Vxider)).
* Fix inconsistency in ORDER BY ... WITH FILL feature. Query, containing ORDER BY ... WITH FILL, can generate extra rows when multiple WITH FILL columns are present. [#38074](https://github.com/ClickHouse/ClickHouse/pull/38074) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* This PR moving `addDependency` from constructor to `startup()` to avoid adding dependency to a *dropped* table, fix [#37237](https://github.com/ClickHouse/ClickHouse/issues/37237). [#37243](https://github.com/ClickHouse/ClickHouse/pull/37243) ([vxider](https://github.com/Vxider)).
* Fix inserting defaults for missing values in columnar formats. Previously missing columns were filled with defaults for types, not for columns. [#37253](https://github.com/ClickHouse/ClickHouse/pull/37253) ([Kruglov Pavel](https://github.com/Avogar)).
* (experimental Object type) Fix some cases of insertion nested arrays to columns of type `Object`. [#37305](https://github.com/ClickHouse/ClickHouse/pull/37305) ([Anton Popov](https://github.com/CurtizJ)).
* Fix unexpected errors with a clash of constant strings in aggregate function, prewhere and join. Close [#36891](https://github.com/ClickHouse/ClickHouse/issues/36891). [#37336](https://github.com/ClickHouse/ClickHouse/pull/37336) ([Vladimir C](https://github.com/vdimir)).
* Fix projections with GROUP/ORDER BY in query and optimize_aggregation_in_order (before the result was incorrect since only finish sorting was performed). [#37342](https://github.com/ClickHouse/ClickHouse/pull/37342) ([Azat Khuzhin](https://github.com/azat)).
* Fixed error with symbols in key name in S3. Fixes [#33009](https://github.com/ClickHouse/ClickHouse/issues/33009). [#37344](https://github.com/ClickHouse/ClickHouse/pull/37344) ([Vladimir Chebotarev](https://github.com/excitoon)).
* Throw an exception when GROUPING SETS used with ROLLUP or CUBE. [#37367](https://github.com/ClickHouse/ClickHouse/pull/37367) ([Dmitry Novik](https://github.com/novikd)).
* Fix LOGICAL_ERROR in getMaxSourcePartsSizeForMerge during merges (in case of non standard, greater, values of `background_pool_size`/`background_merges_mutations_concurrency_ratio` has been specified in `config.xml` (new way) not in `users.xml` (deprecated way)). [#37413](https://github.com/ClickHouse/ClickHouse/pull/37413) ([Azat Khuzhin](https://github.com/azat)).
* Stop removing UTF-8 BOM in RowBinary format. [#37428](https://github.com/ClickHouse/ClickHouse/pull/37428) ([Paul Loyd](https://github.com/loyd)). [#37428](https://github.com/ClickHouse/ClickHouse/pull/37428) ([Paul Loyd](https://github.com/loyd)).
* clickhouse-keeper bugfix: fix force recovery for single node cluster. [#37440](https://github.com/ClickHouse/ClickHouse/pull/37440) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix logical error in normalizeUTF8 functions. Closes [#37298](https://github.com/ClickHouse/ClickHouse/issues/37298). [#37443](https://github.com/ClickHouse/ClickHouse/pull/37443) ([Maksim Kita](https://github.com/kitaisreal)).
* Fix cast lowcard of nullable in JoinSwitcher, close [#37385](https://github.com/ClickHouse/ClickHouse/issues/37385). [#37453](https://github.com/ClickHouse/ClickHouse/pull/37453) ([Vladimir C](https://github.com/vdimir)).
* Fix named tuples output in ORC/Arrow/Parquet formats. [#37458](https://github.com/ClickHouse/ClickHouse/pull/37458) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix optimization of monotonous functions in ORDER BY clause in presence of GROUPING SETS. Fixes [#37401](https://github.com/ClickHouse/ClickHouse/issues/37401). [#37493](https://github.com/ClickHouse/ClickHouse/pull/37493) ([Dmitry Novik](https://github.com/novikd)).
* Fix error on joining with dictionary on some conditions. Close [#37386](https://github.com/ClickHouse/ClickHouse/issues/37386). [#37530](https://github.com/ClickHouse/ClickHouse/pull/37530) ([Vladimir C](https://github.com/vdimir)).
* Prohibit `optimize_aggregation_in_order` with `GROUPING SETS` (fixes `LOGICAL_ERROR`). [#37542](https://github.com/ClickHouse/ClickHouse/pull/37542) ([Azat Khuzhin](https://github.com/azat)).
* Fix wrong dump information of ActionsDAG. [#37587](https://github.com/ClickHouse/ClickHouse/pull/37587) ([zhanglistar](https://github.com/zhanglistar)).
* Fix converting types for UNION queries (may produce LOGICAL_ERROR). [#37593](https://github.com/ClickHouse/ClickHouse/pull/37593) ([Azat Khuzhin](https://github.com/azat)).
* Fix `WITH FILL` modifier with negative intervals in `STEP` clause. Fixes [#37514](https://github.com/ClickHouse/ClickHouse/issues/37514). [#37600](https://github.com/ClickHouse/ClickHouse/pull/37600) ([Anton Popov](https://github.com/CurtizJ)).
* Fix illegal joinGet array usage when ` join_use_nulls = 1`. This fixes [#37562](https://github.com/ClickHouse/ClickHouse/issues/37562) . [#37650](https://github.com/ClickHouse/ClickHouse/pull/37650) ([Amos Bird](https://github.com/amosbird)).
* Fix columns number mismatch in cross join, close [#37561](https://github.com/ClickHouse/ClickHouse/issues/37561). [#37653](https://github.com/ClickHouse/ClickHouse/pull/37653) ([Vladimir C](https://github.com/vdimir)).
* Fix segmentation fault in `show create table` from mysql database when it is configured with named collections. Closes [#37683](https://github.com/ClickHouse/ClickHouse/issues/37683). [#37690](https://github.com/ClickHouse/ClickHouse/pull/37690) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix RabbitMQ Storage not being able to startup on server restart if storage was create without SETTINGS clause. Closes [#37463](https://github.com/ClickHouse/ClickHouse/issues/37463). [#37691](https://github.com/ClickHouse/ClickHouse/pull/37691) ([Kseniia Sumarokova](https://github.com/kssenii)).
* SQL user defined functions disable CREATE/DROP in readonly mode. Closes [#37280](https://github.com/ClickHouse/ClickHouse/issues/37280). [#37699](https://github.com/ClickHouse/ClickHouse/pull/37699) ([Maksim Kita](https://github.com/kitaisreal)).
* Fix formatting of Nullable arguments for executable user defined functions. Closes [#35897](https://github.com/ClickHouse/ClickHouse/issues/35897). [#37711](https://github.com/ClickHouse/ClickHouse/pull/37711) ([Maksim Kita](https://github.com/kitaisreal)).
* Fix optimization enabled by setting `optimize_monotonous_functions_in_order_by` in distributed queries. Fixes [#36037](https://github.com/ClickHouse/ClickHouse/issues/36037). [#37724](https://github.com/ClickHouse/ClickHouse/pull/37724) ([Anton Popov](https://github.com/CurtizJ)).
* Fix possible logical error: `Invalid Field get from type UInt64 to type Float64` in `values` table function. Closes [#37602](https://github.com/ClickHouse/ClickHouse/issues/37602). [#37754](https://github.com/ClickHouse/ClickHouse/pull/37754) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix possible segfault in schema inference in case of exception in SchemaReader constructor. Closes [#37680](https://github.com/ClickHouse/ClickHouse/issues/37680). [#37760](https://github.com/ClickHouse/ClickHouse/pull/37760) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix setting cast_ipv4_ipv6_default_on_conversion_error for internal cast function. Closes [#35156](https://github.com/ClickHouse/ClickHouse/issues/35156). [#37761](https://github.com/ClickHouse/ClickHouse/pull/37761) ([Maksim Kita](https://github.com/kitaisreal)).
* Fix toString error on DatatypeDate32. [#37775](https://github.com/ClickHouse/ClickHouse/pull/37775) ([LiuNeng](https://github.com/liuneng1994)).
* The clickhouse-keeper setting `dead_session_check_period_ms` was transformed into microseconds (multiplied by 1000), which lead to dead sessions only being cleaned up after several minutes (instead of 500ms). [#37824](https://github.com/ClickHouse/ClickHouse/pull/37824) ([Michael Lex](https://github.com/mlex)).
* Fix possible "No more packets are available" for distributed queries (in case of `async_socket_for_remote`/`use_hedged_requests` is disabled). [#37826](https://github.com/ClickHouse/ClickHouse/pull/37826) ([Azat Khuzhin](https://github.com/azat)).
* (experimental WINDOW VIEW) Do not drop the inner target table when executing `ALTER TABLE … MODIFY QUERY` in WindowView. [#37879](https://github.com/ClickHouse/ClickHouse/pull/37879) ([vxider](https://github.com/Vxider)).
* Fix directory ownership of coordination dir in clickhouse-keeper Docker image. Fixes [#37914](https://github.com/ClickHouse/ClickHouse/issues/37914). [#37915](https://github.com/ClickHouse/ClickHouse/pull/37915) ([James Maidment](https://github.com/jamesmaidment)).
* Dictionaries fix custom query with update field and `{condition}`. Closes [#33746](https://github.com/ClickHouse/ClickHouse/issues/33746). [#37947](https://github.com/ClickHouse/ClickHouse/pull/37947) ([Maksim Kita](https://github.com/kitaisreal)).
* Fix possible incorrect result of `SELECT ... WITH FILL` in the case when `ORDER BY` should be applied after `WITH FILL` result (e.g. for outer query). Incorrect result was caused by optimization for `ORDER BY` expressions ([#35623](https://github.com/ClickHouse/ClickHouse/issues/35623)). Closes [#37904](https://github.com/ClickHouse/ClickHouse/issues/37904). [#37959](https://github.com/ClickHouse/ClickHouse/pull/37959) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* (experimental WINDOW VIEW) Add missing default columns when pushing to the target table in WindowView, fix [#37815](https://github.com/ClickHouse/ClickHouse/issues/37815). [#37965](https://github.com/ClickHouse/ClickHouse/pull/37965) ([vxider](https://github.com/Vxider)).
* Fixed too large stack frame that would cause compilation to fail. [#37996](https://github.com/ClickHouse/ClickHouse/pull/37996) ([Han Shukai](https://github.com/KinderRiven)).
* When open enable_filesystem_query_cache_limit, throw Reserved cache size exceeds the remaining cache size. [#38004](https://github.com/ClickHouse/ClickHouse/pull/38004) ([xiedeyantu](https://github.com/xiedeyantu)).
* Fix converting types for UNION queries (may produce LOGICAL_ERROR). [#34775](https://github.com/ClickHouse/ClickHouse/pull/34775) ([Azat Khuzhin](https://github.com/azat)).
* TTL merge may not be scheduled again if BackgroundExecutor is busy. --merges_with_ttl_counter is increased in selectPartsToMerge() --merge task will be ignored if BackgroundExecutor is busy --merges_with_ttl_counter will not be decrease. [#36387](https://github.com/ClickHouse/ClickHouse/pull/36387) ([lthaooo](https://github.com/lthaooo)).
* Fix overridden settings value of `normalize_function_names`. [#36937](https://github.com/ClickHouse/ClickHouse/pull/36937) ([李扬](https://github.com/taiyang-li)).
* Fix for exponential time decaying window functions. Now respecting boundaries of the window. [#36944](https://github.com/ClickHouse/ClickHouse/pull/36944) ([Vladimir Chebotarev](https://github.com/excitoon)).
* Fix possible heap-use-after-free error when reading system.projection_parts and system.projection_parts_columns . This fixes [#37184](https://github.com/ClickHouse/ClickHouse/issues/37184). [#37185](https://github.com/ClickHouse/ClickHouse/pull/37185) ([Amos Bird](https://github.com/amosbird)).
* Fixed `DateTime64` fractional seconds behavior prior to Unix epoch. [#37697](https://github.com/ClickHouse/ClickHouse/pull/37697) ([Andrey Zvonov](https://github.com/zvonand)). [#37039](https://github.com/ClickHouse/ClickHouse/pull/37039) ([李扬](https://github.com/taiyang-li)).
### <a id="225"></a> ClickHouse release 22.5, 2022-05-19
#### Upgrade Notes

View File

@ -244,16 +244,18 @@ endif ()
# Add a section with the hash of the compiled machine code for integrity checks.
# Only for official builds, because adding a section can be time consuming (rewrite of several GB).
# And cross compiled binaries are not supported (since you cannot execute clickhouse hash-binary)
if (OBJCOPY_PATH AND CLICKHOUSE_OFFICIAL_BUILD AND (NOT CMAKE_TOOLCHAIN_FILE OR CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-x86_64.cmake$"))
if (CLICKHOUSE_OFFICIAL_BUILD AND (NOT CMAKE_TOOLCHAIN_FILE OR CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-x86_64.cmake$"))
message(STATUS "Official build: A checksum hash will be added to the clickhouse executable")
set (USE_BINARY_HASH 1 CACHE STRING "Calculate binary hash and store it in the separate section")
else ()
message(STATUS "No official build: A checksum hash will not be added to the clickhouse executable")
endif ()
# Allows to build stripped binary in a separate directory
if (OBJCOPY_PATH AND STRIP_PATH)
option(INSTALL_STRIPPED_BINARIES "Build stripped binaries with debug info in separate directory" OFF)
if (INSTALL_STRIPPED_BINARIES)
set(STRIPPED_BINARIES_OUTPUT "stripped" CACHE STRING "A separate directory for stripped information")
endif()
# Optionally split binaries and debug symbols.
option(INSTALL_STRIPPED_BINARIES "Split binaries and debug symbols" OFF)
if (INSTALL_STRIPPED_BINARIES)
message(STATUS "Will split binaries and debug symbols")
set(STRIPPED_BINARIES_OUTPUT "stripped" CACHE STRING "A separate directory for stripped information")
endif()
cmake_host_system_information(RESULT AVAILABLE_PHYSICAL_MEMORY QUERY AVAILABLE_PHYSICAL_MEMORY) # Not available under freebsd

View File

@ -49,7 +49,7 @@ struct Decimal
using NativeType = T;
constexpr Decimal() = default;
constexpr Decimal(Decimal<T> &&) = default;
constexpr Decimal(Decimal<T> &&) noexcept = default;
constexpr Decimal(const Decimal<T> &) = default;
constexpr Decimal(const T & value_): value(value_) {}
@ -57,7 +57,7 @@ struct Decimal
template <typename U>
constexpr Decimal(const Decimal<U> & x): value(x.value) {}
constexpr Decimal<T> & operator = (Decimal<T> &&) = default;
constexpr Decimal<T> & operator=(Decimal<T> &&) noexcept = default;
constexpr Decimal<T> & operator = (const Decimal<T> &) = default;
constexpr operator T () const { return value; }

View File

@ -29,7 +29,7 @@ if (ARCH_NATIVE)
set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native")
elseif (ARCH_AARCH64)
set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=armv8-a+crc")
set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=armv8-a+crc+simd+crypto+dotprod+ssbs")
elseif (ARCH_PPC64LE)
# Note that gcc and clang have support for x86 SSE2 intrinsics when building for PowerPC

View File

@ -19,9 +19,12 @@ macro(clickhouse_strip_binary)
COMMAND mkdir -p "${STRIP_DESTINATION_DIR}/lib/debug/bin"
COMMAND mkdir -p "${STRIP_DESTINATION_DIR}/bin"
COMMAND cp "${STRIP_BINARY_PATH}" "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}"
# Splits debug symbols into separate file, leaves the binary untouched:
COMMAND "${OBJCOPY_PATH}" --only-keep-debug --compress-debug-sections "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug"
COMMAND chmod 0644 "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug"
COMMAND "${STRIP_PATH}" --remove-section=.comment --remove-section=.note "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}"
# Strips binary, sections '.note' & '.comment' are removed in line with Debian's stripping policy: www.debian.org/doc/debian-policy/ch-files.html, section '.clickhouse.hash' is needed for integrity check:
COMMAND "${STRIP_PATH}" --remove-section=.comment --remove-section=.note --keep-section=.clickhouse.hash "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}"
# Associate stripped binary with debug symbols:
COMMAND "${OBJCOPY_PATH}" --add-gnu-debuglink "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}"
COMMENT "Stripping clickhouse binary" VERBATIM
)

2
contrib/grpc vendored

@ -1 +1 @@
Subproject commit 7eac189a6badddac593580ec2ad1478bd2656fc7
Subproject commit 5e23e96c0c02e451dbb291cf9f66231d02b6cdb6

2
contrib/libunwind vendored

@ -1 +1 @@
Subproject commit c4ea9848a697747dfa35325af9b3452f30841685
Subproject commit 5022f30f3e092a54a7c101c335ce5e08769db366

2
contrib/simdjson vendored

@ -1 +1 @@
Subproject commit 8df32cea3359cb30120795da6020b3b73da01d38
Subproject commit de196dd7a3a16e4056b0551ffa3b85c2f52581e1

2
contrib/zstd vendored

@ -1 +1 @@
Subproject commit a488ba114ec17ea1054b9057c26a046fc122b3b6
Subproject commit b944db0c451ba1bc6bbd8e201d5f88f9041bf1f9

View File

@ -50,7 +50,7 @@ GetLibraryVersion("${HEADER_CONTENT}" LIBVER_MAJOR LIBVER_MINOR LIBVER_RELEASE)
MESSAGE(STATUS "ZSTD VERSION ${LIBVER_MAJOR}.${LIBVER_MINOR}.${LIBVER_RELEASE}")
# cd contrib/zstd/lib
# find . -name '*.c' | grep -vP 'deprecated|legacy' | sort | sed 's/^\./ "${LIBRARY_DIR}/"'
# find . -name '*.c' -or -name '*.S' | grep -vP 'deprecated|legacy' | sort | sed 's/^\./ "${LIBRARY_DIR}/"'
SET(Sources
"${LIBRARY_DIR}/common/debug.c"
"${LIBRARY_DIR}/common/entropy_common.c"
@ -73,6 +73,7 @@ SET(Sources
"${LIBRARY_DIR}/compress/zstd_ldm.c"
"${LIBRARY_DIR}/compress/zstdmt_compress.c"
"${LIBRARY_DIR}/compress/zstd_opt.c"
"${LIBRARY_DIR}/decompress/huf_decompress_amd64.S"
"${LIBRARY_DIR}/decompress/huf_decompress.c"
"${LIBRARY_DIR}/decompress/zstd_ddict.c"
"${LIBRARY_DIR}/decompress/zstd_decompress_block.c"
@ -85,6 +86,7 @@ SET(Sources
# cd contrib/zstd/lib
# find . -name '*.h' | grep -vP 'deprecated|legacy' | sort | sed 's/^\./ "${LIBRARY_DIR}/"'
SET(Headers
"${LIBRARY_DIR}/common/bits.h"
"${LIBRARY_DIR}/common/bitstream.h"
"${LIBRARY_DIR}/common/compiler.h"
"${LIBRARY_DIR}/common/cpu.h"
@ -94,11 +96,13 @@ SET(Headers
"${LIBRARY_DIR}/common/huf.h"
"${LIBRARY_DIR}/common/mem.h"
"${LIBRARY_DIR}/common/pool.h"
"${LIBRARY_DIR}/common/portability_macros.h"
"${LIBRARY_DIR}/common/threading.h"
"${LIBRARY_DIR}/common/xxhash.h"
"${LIBRARY_DIR}/common/zstd_deps.h"
"${LIBRARY_DIR}/common/zstd_internal.h"
"${LIBRARY_DIR}/common/zstd_trace.h"
"${LIBRARY_DIR}/compress/clevels.h"
"${LIBRARY_DIR}/compress/hist.h"
"${LIBRARY_DIR}/compress/zstd_compress_internal.h"
"${LIBRARY_DIR}/compress/zstd_compress_literals.h"

View File

@ -11,6 +11,7 @@ TIMEOUT_SIGN = "[ Timeout! "
UNKNOWN_SIGN = "[ UNKNOWN "
SKIPPED_SIGN = "[ SKIPPED "
HUNG_SIGN = "Found hung queries in processlist"
DATABASE_SIGN = "Database: "
SUCCESS_FINISH_SIGNS = ["All tests have finished", "No tests were run"]
@ -27,14 +28,19 @@ def process_test_log(log_path):
retries = False
success_finish = False
test_results = []
test_end = True
with open(log_path, "r") as test_file:
for line in test_file:
original_line = line
line = line.strip()
if any(s in line for s in SUCCESS_FINISH_SIGNS):
success_finish = True
# Ignore hung check report, since it may be quite large.
# (and may break python parser which has limit of 128KiB for each row).
if HUNG_SIGN in line:
hung = True
break
if RETRIES_SIGN in line:
retries = True
if any(
@ -67,8 +73,17 @@ def process_test_log(log_path):
else:
success += int(OK_SIGN in line)
test_results.append((test_name, "OK", test_time, []))
elif len(test_results) > 0 and test_results[-1][1] == "FAIL":
test_end = False
elif (
len(test_results) > 0 and test_results[-1][1] == "FAIL" and not test_end
):
test_results[-1][3].append(original_line)
# Database printed after everything else in case of failures,
# so this is a stop marker for capturing test output.
#
# And it is handled after everything else to include line with database into the report.
if DATABASE_SIGN in line:
test_end = True
test_results = [
(test[0], test[1], test[2], "".join(test[3])) for test in test_results
@ -113,7 +128,7 @@ def process_result(result_path):
test_results,
) = process_test_log(result_path)
is_flacky_check = 1 < int(os.environ.get("NUM_TRIES", 1))
logging.info("Is flacky check: %s", is_flacky_check)
logging.info("Is flaky check: %s", is_flacky_check)
# If no tests were run (success == 0) it indicates an error (e.g. server did not start or crashed immediately)
# But it's Ok for "flaky checks" - they can contain just one test for check which is marked as skipped.
if failed != 0 or unknown != 0 or (success == 0 and (not is_flacky_check)):

View File

@ -2,6 +2,18 @@
The values of `merge_tree` settings (for all MergeTree tables) can be viewed in the table `system.merge_tree_settings`, they can be overridden in `config.xml` in the `merge_tree` section, or set in the `SETTINGS` section of each table.
These are example overrides for `max_suspicious_broken_parts`:
## max_suspicious_broken_parts
If the number of broken parts in a single partition exceeds the `max_suspicious_broken_parts` value, automatic deletion is denied.
Possible values:
- Any positive integer.
Default value: 10.
Override example in `config.xml`:
``` text
@ -152,6 +164,187 @@ Default value: 604800 (1 week).
Similar to [replicated_deduplication_window](#replicated-deduplication-window), `replicated_deduplication_window_seconds` specifies how long to store hash sums of blocks for insert deduplication. Hash sums older than `replicated_deduplication_window_seconds` are removed from ClickHouse Keeper, even if they are less than ` replicated_deduplication_window`.
## max_replicated_logs_to_keep
How many records may be in the ClickHouse Keeper log if there is inactive replica. An inactive replica becomes lost when when this number exceed.
Possible values:
- Any positive integer.
Default value: 1000
## min_replicated_logs_to_keep
Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.
Possible values:
- Any positive integer.
Default value: 10
## prefer_fetch_merged_part_time_threshold
If the time passed since a replication log (ClickHouse Keeper or ZooKeeper) entry creation exceeds this threshold, and the sum of the size of parts is greater than `prefer_fetch_merged_part_size_threshold`, then prefer fetching merged part from a replica instead of doing merge locally. This is to speed up very long merges.
Possible values:
- Any positive integer.
Default value: 3600
## prefer_fetch_merged_part_size_threshold
If the sum of the size of parts exceeds this threshold and the time since a replication log entry creation is greater than `prefer_fetch_merged_part_time_threshold`, then prefer fetching merged part from a replica instead of doing merge locally. This is to speed up very long merges.
Possible values:
- Any positive integer.
Default value: 10,737,418,240
## execute_merges_on_single_replica_time_threshold
When this setting has a value greater than zero, only a single replica starts the merge immediately, and other replicas wait up to that amount of time to download the result instead of doing merges locally. If the chosen replica doesn't finish the merge during that amount of time, fallback to standard behavior happens.
Possible values:
- Any positive integer.
Default value: 0 (seconds)
## remote_fs_execute_merges_on_single_replica_time_threshold
When this setting has a value greater than than zero only a single replica starts the merge immediately if merged part on shared storage and `allow_remote_fs_zero_copy_replication` is enabled.
Possible values:
- Any positive integer.
Default value: 1800
## try_fetch_recompressed_part_timeout
Recompression works slow in most cases, so we don't start merge with recompression until this timeout and trying to fetch recompressed part from replica which assigned this merge with recompression.
Possible values:
- Any positive integer.
Default value: 7200
## always_fetch_merged_part
If true, this replica never merges parts and always downloads merged parts from other replicas.
Possible values:
- true, false
Default value: false
## max_suspicious_broken_parts
Max broken parts, if more - deny automatic deletion.
Possible values:
- Any positive integer.
Default value: 10
## max_suspicious_broken_parts_bytes
Max size of all broken parts, if more - deny automatic deletion.
Possible values:
- Any positive integer.
Default value: 1,073,741,824
## max_files_to_modify_in_alter_columns
Do not apply ALTER if number of files for modification(deletion, addition) is greater than this setting.
Possible values:
- Any positive integer.
Default value: 75
## max_files_to_remove_in_alter_columns
Do not apply ALTER, if the number of files for deletion is greater than this setting.
Possible values:
- Any positive integer.
Default value: 50
## replicated_max_ratio_of_wrong_parts
If the ratio of wrong parts to total number of parts is less than this - allow to start.
Possible values:
- Float, 0.0 - 1.0
Default value: 0.5
## replicated_max_parallel_fetches_for_host
Limit parallel fetches from endpoint (actually pool size).
Possible values:
- Any positive integer.
Default value: 15
## replicated_fetches_http_connection_timeout
HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.
Possible values:
- Any positive integer.
Default value: Inherited from default profile `http_connection_timeout` if not set explicitly.
## replicated_can_become_leader
If true, replicated tables replicas on this node will try to acquire leadership.
Possible values:
- true, false
Default value: true
## zookeeper_session_expiration_check_period
ZooKeeper session expiration check period, in seconds.
Possible values:
- Any positive integer.
Default value: 60
## detach_old_local_parts_when_cloning_replica
Do not remove old local parts when repairing lost replica.
Possible values:
- true, false
Default value: true
## replicated_fetches_http_connection_timeout {#replicated_fetches_http_connection_timeout}
HTTP connection timeout (in seconds) for part fetch requests. Inherited from default profile [http_connection_timeout](./settings.md#http_connection_timeout) if not set explicitly.

View File

@ -248,6 +248,7 @@ Specialized codecs:
- `Delta(delta_bytes)` — Compression approach in which raw values are replaced by the difference of two neighboring values, except for the first value that stays unchanged. Up to `delta_bytes` are used for storing delta values, so `delta_bytes` is the maximum size of raw values. Possible `delta_bytes` values: 1, 2, 4, 8. The default value for `delta_bytes` is `sizeof(type)` if equal to 1, 2, 4, or 8. In all other cases, its 1.
- `DoubleDelta` — Calculates delta of deltas and writes it in compact binary form. Optimal compression rates are achieved for monotonic sequences with a constant stride, such as time series data. Can be used with any fixed-width type. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. Uses 1 extra bit for 32-byte deltas: 5-bit prefixes instead of 4-bit prefixes. For additional information, see Compressing Time Stamps in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf).
- `Gorilla` — Calculates XOR between current and previous value and writes it in compact binary form. Efficient when storing a series of floating point values that change slowly, because the best compression rate is achieved when neighboring values are binary equal. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. For additional information, see Compressing Values in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf).
- `FPC` - Repeatedly predicts the next floating point value in the sequence using the better of two predictors, then XORs the actual with the predicted value, and leading-zero compresses the result. Similar to Gorilla, this is efficient when storing a series of floating point values that change slowly. For 64-bit values (double), FPC is faster than Gorilla, for 32-bit values your mileage may vary. For a detailed description of the algorithm see [High Throughput Compression of Double-Precision Floating-Point Data](https://userweb.cs.txstate.edu/~burtscher/papers/dcc07a.pdf).
- `T64` — Compression approach that crops unused high bits of values in integer data types (including `Enum`, `Date` and `DateTime`). At each step of its algorithm, codec takes a block of 64 values, puts them into 64x64 bit matrix, transposes it, crops the unused bits of values and returns the rest as a sequence. Unused bits are the bits, that do not differ between maximum and minimum values in the whole data part for which the compression is used.
`DoubleDelta` and `Gorilla` codecs are used in Gorilla TSDB as the components of its compressing algorithm. Gorilla approach is effective in scenarios when there is a sequence of slowly changing values with their timestamps. Timestamps are effectively compressed by the `DoubleDelta` codec, and values are effectively compressed by the `Gorilla` codec. For example, to get an effectively stored table, you can create it in the following configuration:

View File

@ -508,7 +508,7 @@ else ()
endif()
if (USE_BINARY_HASH)
add_custom_command(TARGET clickhouse POST_BUILD COMMAND ./clickhouse hash-binary > hash && ${OBJCOPY_PATH} --add-section .note.ClickHouse.hash=hash clickhouse COMMENT "Adding .note.ClickHouse.hash to clickhouse" VERBATIM)
add_custom_command(TARGET clickhouse POST_BUILD COMMAND ./clickhouse hash-binary > hash && ${OBJCOPY_PATH} --add-section .clickhouse.hash=hash clickhouse COMMENT "Adding section '.clickhouse.hash' to clickhouse binary" VERBATIM)
endif()
if (INSTALL_STRIPPED_BINARIES)

View File

@ -82,7 +82,7 @@ int mainEntryClickHouseDisks(int argc, char ** argv);
int mainEntryClickHouseHashBinary(int, char **)
{
/// Intentionally without newline. So you can run:
/// objcopy --add-section .note.ClickHouse.hash=<(./clickhouse hash-binary) clickhouse
/// objcopy --add-section .clickhouse.hash=<(./clickhouse hash-binary) clickhouse
std::cout << getHashOfLoadedBinaryHex();
return 0;
}

View File

@ -744,16 +744,18 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// But there are other sections of the binary (e.g. exception handling tables)
/// that are interpreted (not executed) but can alter the behaviour of the program as well.
/// Please keep the below log messages in-sync with the ones in daemon/BaseDaemon.cpp
String calculated_binary_hash = getHashOfLoadedBinaryHex();
if (stored_binary_hash.empty())
{
LOG_WARNING(log, "Calculated checksum of the binary: {}."
" There is no information about the reference checksum.", calculated_binary_hash);
LOG_WARNING(log, "Integrity check of the executable skipped because the reference checksum could not be read."
" (calculated checksum: {})", calculated_binary_hash);
}
else if (calculated_binary_hash == stored_binary_hash)
{
LOG_INFO(log, "Calculated checksum of the binary: {}, integrity check passed.", calculated_binary_hash);
LOG_INFO(log, "Integrity check of the executable successfully passed (checksum: {})", calculated_binary_hash);
}
else
{
@ -769,14 +771,14 @@ int Server::main(const std::vector<std::string> & /*args*/)
else
{
throw Exception(ErrorCodes::CORRUPTED_DATA,
"Calculated checksum of the ClickHouse binary ({0}) does not correspond"
" to the reference checksum stored in the binary ({1})."
" It may indicate one of the following:"
" - the file {2} was changed just after startup;"
" - the file {2} is damaged on disk due to faulty hardware;"
" - the loaded executable is damaged in memory due to faulty hardware;"
"Calculated checksum of the executable ({0}) does not correspond"
" to the reference checksum stored in the executable ({1})."
" This may indicate one of the following:"
" - the executable {2} was changed just after startup;"
" - the executable {2} was corrupted on disk due to faulty hardware;"
" - the loaded executable was corrupted in memory due to faulty hardware;"
" - the file {2} was intentionally modified;"
" - logical error in code."
" - a logical error in the code."
, calculated_binary_hash, stored_binary_hash, executable_path);
}
}

View File

@ -337,7 +337,7 @@ void ColumnVector<T>::updatePermutation(IColumn::PermutationSortDirection direct
PaddedPODArray<ValueWithIndex<T>> pairs(size);
size_t index = 0;
for (auto it = begin; it != end; ++it)
for (auto * it = begin; it != end; ++it)
{
pairs[index] = {data[*it], static_cast<UInt32>(*it)};
++index;

View File

@ -8,6 +8,9 @@
#if defined(__AVX512F__) || defined(__AVX512BW__) || defined(__AVX__) || defined(__AVX2__)
#include <immintrin.h>
#endif
#if defined(__aarch64__) && defined(__ARM_NEON)
# include <arm_neon.h>
#endif
/// Common helper methods for implementation of different columns.
@ -44,6 +47,22 @@ inline UInt64 bytes64MaskToBits64Mask(const UInt8 * bytes64)
_mm_loadu_si128(reinterpret_cast<const __m128i *>(bytes64 + 32)), zero16))) << 32) & 0xffff00000000)
| ((static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(bytes64 + 48)), zero16))) << 48) & 0xffff000000000000);
#elif defined(__aarch64__) && defined(__ARM_NEON)
const uint8x16_t bitmask = {0x01, 0x02, 0x4, 0x8, 0x10, 0x20, 0x40, 0x80, 0x01, 0x02, 0x4, 0x8, 0x10, 0x20, 0x40, 0x80};
const auto * src = reinterpret_cast<const unsigned char *>(bytes64);
const uint8x16_t p0 = vceqzq_u8(vld1q_u8(src));
const uint8x16_t p1 = vceqzq_u8(vld1q_u8(src + 16));
const uint8x16_t p2 = vceqzq_u8(vld1q_u8(src + 32));
const uint8x16_t p3 = vceqzq_u8(vld1q_u8(src + 48));
uint8x16_t t0 = vandq_u8(p0, bitmask);
uint8x16_t t1 = vandq_u8(p1, bitmask);
uint8x16_t t2 = vandq_u8(p2, bitmask);
uint8x16_t t3 = vandq_u8(p3, bitmask);
uint8x16_t sum0 = vpaddq_u8(t0, t1);
uint8x16_t sum1 = vpaddq_u8(t2, t3);
sum0 = vpaddq_u8(sum0, sum1);
sum0 = vpaddq_u8(sum0, sum0);
UInt64 res = vgetq_lane_u64(vreinterpretq_u64_u8(sum0), 0);
#else
UInt64 res = 0;
for (size_t i = 0; i < 64; ++i)

View File

@ -176,9 +176,9 @@ String Elf::getBuildID(const char * nhdr_pos, size_t size)
#endif // OS_SUNOS
String Elf::getBinaryHash() const
String Elf::getStoredBinaryHash() const
{
if (auto section = findSectionByName(".note.ClickHouse.hash"))
if (auto section = findSectionByName(".clickhouse.hash"))
return {section->begin(), section->end()};
else
return {};

View File

@ -61,7 +61,7 @@ public:
static String getBuildID(const char * nhdr_pos, size_t size);
/// Hash of the binary for integrity checks.
String getBinaryHash() const;
String getStoredBinaryHash() const;
private:
MMapReadBufferFromFile in;

View File

@ -296,6 +296,19 @@ struct UInt128HashCRC32
}
};
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
struct UInt128HashCRC32
{
size_t operator()(UInt128 x) const
{
UInt64 crc = -1ULL;
crc = __crc32cd(crc, x.items[0]);
crc = __crc32cd(crc, x.items[1]);
return crc;
}
};
#else
/// On other platforms we do not use CRC32. NOTE This can be confusing.
@ -339,6 +352,21 @@ struct UInt256HashCRC32
}
};
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
struct UInt256HashCRC32
{
size_t operator()(UInt256 x) const
{
UInt64 crc = -1ULL;
crc = __crc32cd(crc, x.items[0]);
crc = __crc32cd(crc, x.items[1]);
crc = __crc32cd(crc, x.items[2]);
crc = __crc32cd(crc, x.items[3]);
return crc;
}
};
#else
/// We do not need to use CRC32 on other platforms. NOTE This can be confusing.

View File

@ -9,6 +9,13 @@
#include <emmintrin.h>
#endif
#if defined(__aarch64__) && defined(__ARM_NEON)
# include <arm_neon.h>
# ifdef HAS_RESERVED_IDENTIFIER
# pragma clang diagnostic ignored "-Wreserved-identifier"
# endif
#endif
namespace DB
{
@ -66,6 +73,18 @@ inline size_t countCodePoints(const UInt8 * data, size_t size)
for (; data < src_end_sse; data += bytes_sse)
res += __builtin_popcount(_mm_movemask_epi8(
_mm_cmpgt_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i *>(data)), threshold)));
#elif defined(__aarch64__) && defined(__ARM_NEON)
/// Returns a 64 bit mask of nibbles (4 bits for each byte).
auto get_nibble_mask
= [](uint8x16_t input) -> uint64_t { return vget_lane_u64(vreinterpret_u64_u8(vshrn_n_u16(vreinterpretq_u16_u8(input), 4)), 0); };
constexpr auto bytes_sse = 16;
const auto * src_end_sse = data + size / bytes_sse * bytes_sse;
const auto threshold = vdupq_n_s8(0xBF);
for (; data < src_end_sse; data += bytes_sse)
res += __builtin_popcountll(get_nibble_mask(vcgtq_s8(vld1q_s8(reinterpret_cast<const int8_t *>(data)), threshold)));
res >>= 2;
#endif
for (; data < end; ++data) /// Skip UTF-8 continuation bytes.

View File

@ -1,7 +1,7 @@
#pragma once
#include <cstdint>
#include <algorithm>
#include <cstdint>
#include <Core/Defines.h>
@ -26,7 +26,7 @@ inline int cmp(T a, T b)
/// Results don't depend on the values inside uninitialized memory but Memory Sanitizer cannot see it.
/// Disable optimized functions if compile with Memory Sanitizer.
#if defined(__AVX512BW__) && defined(__AVX512VL__) && !defined(MEMORY_SANITIZER)
#include <immintrin.h>
# include <immintrin.h>
/** All functions works under the following assumptions:
@ -45,7 +45,8 @@ inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char
{
uint16_t mask = _mm_cmp_epi8_mask(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a + offset)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset)), _MM_CMPINT_NE);
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset)),
_MM_CMPINT_NE);
if (mask)
{
@ -76,7 +77,8 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz
{
uint16_t mask = _mm_cmp_epi8_mask(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a + offset)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset)), _MM_CMPINT_NE);
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset)),
_MM_CMPINT_NE);
if (mask)
{
@ -117,9 +119,7 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz
for (size_t offset = min_size; offset < max_size; offset += 16)
{
uint16_t mask = _mm_cmpneq_epi8_mask(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(longest + offset)),
zero16);
uint16_t mask = _mm_cmpneq_epi8_mask(_mm_loadu_si128(reinterpret_cast<const __m128i *>(longest + offset)), zero16);
if (mask)
{
@ -145,7 +145,8 @@ inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t siz
{
uint16_t mask = _mm_cmp_epi8_mask(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a + offset)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset)), _MM_CMPINT_NE);
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset)),
_MM_CMPINT_NE);
if (mask)
{
@ -174,7 +175,8 @@ inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Ch
{
uint16_t mask = _mm_cmp_epi8_mask(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a + offset)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset)), _MM_CMPINT_NE);
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset)),
_MM_CMPINT_NE);
if (mask)
{
@ -196,7 +198,8 @@ inline int memcmpSmallMultipleOf16(const Char * a, const Char * b, size_t size)
{
uint16_t mask = _mm_cmp_epi8_mask(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a + offset)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset)), _MM_CMPINT_NE);
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b + offset)),
_MM_CMPINT_NE);
if (mask)
{
@ -215,8 +218,7 @@ template <typename Char>
inline int memcmp16(const Char * a, const Char * b)
{
uint16_t mask = _mm_cmp_epi8_mask(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b)), _MM_CMPINT_NE);
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a)), _mm_loadu_si128(reinterpret_cast<const __m128i *>(b)), _MM_CMPINT_NE);
if (mask)
{
@ -232,9 +234,9 @@ inline int memcmp16(const Char * a, const Char * b)
*/
inline bool memequal16(const void * a, const void * b)
{
return 0xFFFF == _mm_cmp_epi8_mask(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b)), _MM_CMPINT_EQ);
return 0xFFFF
== _mm_cmp_epi8_mask(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a)), _mm_loadu_si128(reinterpret_cast<const __m128i *>(b)), _MM_CMPINT_EQ);
}
@ -245,8 +247,8 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
for (size_t offset = 0; offset < size; offset += 16)
{
uint16_t mask = _mm_cmp_epi8_mask(zero16,
_mm_loadu_si128(reinterpret_cast<const __m128i *>(reinterpret_cast<const char *>(data) + offset)), _MM_CMPINT_NE);
uint16_t mask = _mm_cmp_epi8_mask(
zero16, _mm_loadu_si128(reinterpret_cast<const __m128i *>(reinterpret_cast<const char *>(data) + offset)), _MM_CMPINT_NE);
if (mask)
{
@ -259,7 +261,7 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
}
#elif defined(__SSE2__) && !defined(MEMORY_SANITIZER)
#include <emmintrin.h>
# include <emmintrin.h>
/** All functions works under the following assumptions:
@ -352,9 +354,7 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz
for (size_t offset = min_size; offset < max_size; offset += 16)
{
uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(longest + offset)),
zero16));
uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i *>(longest + offset)), zero16));
mask = ~mask;
if (mask)
@ -453,9 +453,8 @@ inline int memcmpSmallMultipleOf16(const Char * a, const Char * b, size_t size)
template <typename Char>
inline int memcmp16(const Char * a, const Char * b)
{
uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b))));
uint16_t mask = _mm_movemask_epi8(
_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i *>(a)), _mm_loadu_si128(reinterpret_cast<const __m128i *>(b))));
mask = ~mask;
if (mask)
@ -472,9 +471,9 @@ inline int memcmp16(const Char * a, const Char * b)
*/
inline bool memequal16(const void * a, const void * b)
{
return 0xFFFF == _mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(b))));
return 0xFFFF
== _mm_movemask_epi8(_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a)), _mm_loadu_si128(reinterpret_cast<const __m128i *>(b))));
}
@ -485,8 +484,8 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
for (size_t offset = 0; offset < size; offset += 16)
{
uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(zero16,
_mm_loadu_si128(reinterpret_cast<const __m128i *>(reinterpret_cast<const char *>(data) + offset))));
uint16_t mask = _mm_movemask_epi8(
_mm_cmpeq_epi8(zero16, _mm_loadu_si128(reinterpret_cast<const __m128i *>(reinterpret_cast<const char *>(data) + offset))));
mask = ~mask;
if (mask)
@ -499,6 +498,209 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
return true;
}
#elif defined(__aarch64__) && defined(__ARM_NEON)
# include <arm_neon.h>
# ifdef HAS_RESERVED_IDENTIFIER
# pragma clang diagnostic ignored "-Wreserved-identifier"
# endif
inline uint64_t getNibbleMask(uint8x16_t res)
{
return vget_lane_u64(vreinterpret_u64_u8(vshrn_n_u16(vreinterpretq_u16_u8(res), 4)), 0);
}
template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
size_t min_size = std::min(a_size, b_size);
for (size_t offset = 0; offset < min_size; offset += 16)
{
uint64_t mask = getNibbleMask(vceqq_u8(
vld1q_u8(reinterpret_cast<const unsigned char *>(a + offset)), vld1q_u8(reinterpret_cast<const unsigned char *>(b + offset))));
mask = ~mask;
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
if (offset >= min_size)
break;
return detail::cmp(a[offset], b[offset]);
}
}
return detail::cmp(a_size, b_size);
}
template <typename Char>
inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
size_t min_size = std::min(a_size, b_size);
for (size_t offset = 0; offset < min_size; offset += 16)
{
uint64_t mask = getNibbleMask(vceqq_u8(
vld1q_u8(reinterpret_cast<const unsigned char *>(a + offset)), vld1q_u8(reinterpret_cast<const unsigned char *>(b + offset))));
mask = ~mask;
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
if (offset >= min_size)
break;
return detail::cmp(a[offset], b[offset]);
}
}
/// The strings are equal up to min_size.
/// If the rest of the larger string is zero bytes then the strings are
/// considered equal.
size_t max_size;
const Char * longest;
int cmp;
if (a_size == b_size)
{
return 0;
}
else if (a_size > b_size)
{
max_size = a_size;
longest = a;
cmp = 1;
}
else
{
max_size = b_size;
longest = b;
cmp = -1;
}
for (size_t offset = min_size; offset < max_size; offset += 16)
{
uint64_t mask = getNibbleMask(vceqzq_u8(vld1q_u8(reinterpret_cast<const unsigned char *>(longest + offset))));
mask = ~mask;
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
if (offset >= max_size)
return 0;
return cmp;
}
}
return 0;
}
template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t size)
{
for (size_t offset = 0; offset < size; offset += 16)
{
uint64_t mask = getNibbleMask(vceqq_u8(
vld1q_u8(reinterpret_cast<const unsigned char *>(a + offset)), vld1q_u8(reinterpret_cast<const unsigned char *>(b + offset))));
mask = ~mask;
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
if (offset >= size)
return 0;
return detail::cmp(a[offset], b[offset]);
}
}
return 0;
}
template <typename Char>
inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
if (a_size != b_size)
return false;
for (size_t offset = 0; offset < a_size; offset += 16)
{
uint64_t mask = getNibbleMask(vceqq_u8(
vld1q_u8(reinterpret_cast<const unsigned char *>(a + offset)), vld1q_u8(reinterpret_cast<const unsigned char *>(b + offset))));
mask = ~mask;
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
return offset >= a_size;
}
}
return true;
}
template <typename Char>
inline int memcmpSmallMultipleOf16(const Char * a, const Char * b, size_t size)
{
for (size_t offset = 0; offset < size; offset += 16)
{
uint64_t mask = getNibbleMask(vceqq_u8(
vld1q_u8(reinterpret_cast<const unsigned char *>(a + offset)), vld1q_u8(reinterpret_cast<const unsigned char *>(b + offset))));
mask = ~mask;
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
return detail::cmp(a[offset], b[offset]);
}
}
return 0;
}
template <typename Char>
inline int memcmp16(const Char * a, const Char * b)
{
uint64_t mask = getNibbleMask(
vceqq_u8(vld1q_u8(reinterpret_cast<const unsigned char *>(a)), vld1q_u8(reinterpret_cast<const unsigned char *>(b))));
mask = ~mask;
if (mask)
{
auto offset = __builtin_ctzll(mask) >> 2;
return detail::cmp(a[offset], b[offset]);
}
return 0;
}
inline bool memequal16(const void * a, const void * b)
{
return 0xFFFFFFFFFFFFFFFFull
== getNibbleMask(
vceqq_u8(vld1q_u8(reinterpret_cast<const unsigned char *>(a)), vld1q_u8(reinterpret_cast<const unsigned char *>(b))));
}
inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
{
for (size_t offset = 0; offset < size; offset += 16)
{
uint64_t mask = getNibbleMask(vceqzq_u8(vld1q_u8(reinterpret_cast<const unsigned char *>(data) + offset)));
mask = ~mask;
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
return offset >= size;
}
}
return true;
}
#else

View File

@ -3,8 +3,15 @@
#include <string.h>
#ifdef __SSE2__
#include <emmintrin.h>
# include <emmintrin.h>
#endif
#if defined(__aarch64__) && defined(__ARM_NEON)
# include <arm_neon.h>
# ifdef HAS_RESERVED_IDENTIFIER
# pragma clang diagnostic ignored "-Wreserved-identifier"
# endif
#endif
/** memcpy function could work suboptimal if all the following conditions are met:
* 1. Size of memory region is relatively small (approximately, under 50 bytes).
@ -27,6 +34,7 @@
* Use with caution.
*/
#ifdef __SSE2__ /// Implementation for x86 platform
namespace detail
{
inline void memcpySmallAllowReadWriteOverflow15Impl(char * __restrict dst, const char * __restrict src, ssize_t n)
@ -51,11 +59,33 @@ inline void memcpySmallAllowReadWriteOverflow15(void * __restrict dst, const voi
detail::memcpySmallAllowReadWriteOverflow15Impl(reinterpret_cast<char *>(dst), reinterpret_cast<const char *>(src), n);
}
#elif defined(__aarch64__) && defined(__ARM_NEON) /// Implementation for arm platform, similar to x86
namespace detail
{
inline void memcpySmallAllowReadWriteOverflow15Impl(char * __restrict dst, const char * __restrict src, ssize_t n)
{
while (n > 0)
{
vst1q_s8(reinterpret_cast<signed char *>(dst), vld1q_s8(reinterpret_cast<const signed char *>(src)));
dst += 16;
src += 16;
n -= 16;
}
}
}
inline void memcpySmallAllowReadWriteOverflow15(void * __restrict dst, const void * __restrict src, size_t n)
{
detail::memcpySmallAllowReadWriteOverflow15Impl(reinterpret_cast<char *>(dst), reinterpret_cast<const char *>(src), n);
}
/** NOTE There was also a function, that assumes, that you could read any bytes inside same memory page of src.
* This function was unused, and also it requires special handling for Valgrind and ASan.
*/
#else /// Implementation for other platforms.
#else /// Implementation for other platforms.
inline void memcpySmallAllowReadWriteOverflow15(void * __restrict dst, const void * __restrict src, size_t n)
{

View File

@ -14,6 +14,10 @@
namespace DB
{
/// An implementation of the FPC codec for floating-point values described in the paper
/// M. Burtscher, P. Ratanaworabhan: "FPC: A high-speed compressor for double-precision floating-point data" (2008).
/// Note: The paper only describes compression of 64-bit doubles and leaves 32-bit floats to future work. The code
/// implements them anyways. Your mileage with respect to performance and compression may vary.
class CompressionCodecFPC : public ICompressionCodec
{
public:
@ -23,8 +27,8 @@ public:
void updateHash(SipHash & hash) const override;
static constexpr UInt8 MAX_COMPRESSION_LEVEL{28};
static constexpr UInt8 DEFAULT_COMPRESSION_LEVEL{12};
static constexpr UInt8 MAX_COMPRESSION_LEVEL = 28;
static constexpr UInt8 DEFAULT_COMPRESSION_LEVEL = 12;
protected:
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
@ -37,10 +41,11 @@ protected:
bool isGenericCompression() const override { return false; }
private:
static constexpr UInt32 HEADER_SIZE{3};
static constexpr UInt32 HEADER_SIZE = 2;
UInt8 float_width; // size of uncompressed float in bytes
UInt8 level; // compression level, 2^level * float_width is the size of predictors table in bytes
// below members are used by compression, decompression ignores them:
const UInt8 float_width; // size of uncompressed float in bytes
const UInt8 level; // compression level, 2^level * float_width is the size of predictors table in bytes
};
@ -96,30 +101,6 @@ UInt8 getFloatBytesSize(const IDataType & column_type)
column_type.getName());
}
std::byte encodeEndianness(std::endian endian)
{
switch (endian)
{
case std::endian::little:
return std::byte{0};
case std::endian::big:
return std::byte{1};
}
throw Exception("Unsupported endianness", ErrorCodes::BAD_ARGUMENTS);
}
std::endian decodeEndianness(std::byte endian)
{
switch (std::to_integer<unsigned char>(endian))
{
case 0:
return std::endian::little;
case 1:
return std::endian::big;
}
throw Exception("Unsupported endianness", ErrorCodes::BAD_ARGUMENTS);
}
}
void registerCodecFPC(CompressionCodecFactory & factory)
@ -127,7 +108,7 @@ void registerCodecFPC(CompressionCodecFactory & factory)
auto method_code = static_cast<UInt8>(CompressionMethodByte::FPC);
auto codec_builder = [&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr
{
UInt8 float_width{0};
UInt8 float_width = 0;
if (column_type != nullptr)
float_width = getFloatBytesSize(*column_type);
@ -145,10 +126,8 @@ void registerCodecFPC(CompressionCodecFactory & factory)
throw Exception("FPC codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER);
level = literal->value.safeGet<UInt8>();
if (level == 0)
throw Exception("FPC codec level must be at least 1", ErrorCodes::ILLEGAL_CODEC_PARAMETER);
if (level > CompressionCodecFPC::MAX_COMPRESSION_LEVEL)
throw Exception("FPC codec level must be at most 28", ErrorCodes::ILLEGAL_CODEC_PARAMETER);
if (level < 1 || level > CompressionCodecFPC::MAX_COMPRESSION_LEVEL)
throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "FPC codec level must be between {} and {}", 1, static_cast<int>(CompressionCodecFPC::MAX_COMPRESSION_LEVEL));
}
return std::make_shared<CompressionCodecFPC>(float_width, level);
};
@ -159,11 +138,12 @@ namespace
{
template <std::unsigned_integral TUint>
requires (sizeof(TUint) >= 4)
requires (sizeof(TUint) >= 4)
class DfcmPredictor
{
public:
explicit DfcmPredictor(std::size_t table_size): table(table_size, 0), prev_value{0}, hash{0}
explicit DfcmPredictor(std::size_t table_size)
: table(table_size, 0), prev_value{0}, hash{0}
{
}
@ -200,11 +180,12 @@ private:
};
template <std::unsigned_integral TUint>
requires (sizeof(TUint) >= 4)
requires (sizeof(TUint) >= 4)
class FcmPredictor
{
public:
explicit FcmPredictor(std::size_t table_size): table(table_size, 0), hash{0}
explicit FcmPredictor(std::size_t table_size)
: table(table_size, 0), hash{0}
{
}
@ -238,18 +219,17 @@ private:
std::size_t hash;
};
template <std::unsigned_integral TUint, std::endian Endian = std::endian::native>
requires (Endian == std::endian::little || Endian == std::endian::big)
template <std::unsigned_integral TUint>
class FPCOperation
{
static constexpr std::size_t CHUNK_SIZE{64};
static constexpr auto VALUE_SIZE = sizeof(TUint);
static constexpr std::byte FCM_BIT{0};
static constexpr std::byte DFCM_BIT{1u << 3};
static constexpr auto DFCM_BIT_1 = DFCM_BIT << 4;
static constexpr auto DFCM_BIT_2 = DFCM_BIT;
static constexpr unsigned MAX_ZERO_BYTE_COUNT{0b111u};
static constexpr unsigned MAX_ZERO_BYTE_COUNT = 0b111u;
static constexpr std::endian ENDIAN = std::endian::little;
static constexpr std::size_t CHUNK_SIZE = 64;
public:
FPCOperation(std::span<std::byte> destination, UInt8 compression_level)
@ -264,8 +244,8 @@ public:
std::span chunk_view(chunk);
for (std::size_t i = 0; i < data.size(); i += chunk_view.size_bytes())
{
auto written_values = importChunk(data.subspan(i), chunk_view);
encodeChunk(chunk_view.subspan(0, written_values));
auto written_values_count = importChunk(data.subspan(i), chunk_view);
encodeChunk(chunk_view.subspan(0, written_values_count));
}
return initial_size - result.size();
@ -273,7 +253,7 @@ public:
void decode(std::span<const std::byte> values, std::size_t decoded_size) &&
{
std::size_t read_bytes{0};
std::size_t read_bytes = 0;
std::span<TUint> chunk_view(chunk);
for (std::size_t i = 0; i < decoded_size; i += chunk_view.size_bytes())
@ -329,14 +309,14 @@ private:
std::byte predictor;
};
unsigned encodeCompressedZeroByteCount(int compressed)
unsigned encodeCompressedZeroByteCount(unsigned compressed)
{
if constexpr (VALUE_SIZE == MAX_ZERO_BYTE_COUNT + 1)
{
if (compressed >= 4)
--compressed;
}
return std::min(static_cast<unsigned>(compressed), MAX_ZERO_BYTE_COUNT);
return std::min(compressed, MAX_ZERO_BYTE_COUNT);
}
unsigned decodeCompressedZeroByteCount(unsigned encoded_size)
@ -360,14 +340,14 @@ private:
auto zeroes_dfcm = std::countl_zero(compressed_dfcm);
auto zeroes_fcm = std::countl_zero(compressed_fcm);
if (zeroes_dfcm > zeroes_fcm)
return {compressed_dfcm, encodeCompressedZeroByteCount(zeroes_dfcm / BITS_PER_BYTE), DFCM_BIT};
return {compressed_fcm, encodeCompressedZeroByteCount(zeroes_fcm / BITS_PER_BYTE), FCM_BIT};
return {compressed_dfcm, encodeCompressedZeroByteCount(static_cast<unsigned>(zeroes_dfcm) / BITS_PER_BYTE), DFCM_BIT};
return {compressed_fcm, encodeCompressedZeroByteCount(static_cast<unsigned>(zeroes_fcm) / BITS_PER_BYTE), FCM_BIT};
}
void encodePair(TUint first, TUint second)
{
auto [value1, zero_byte_count1, predictor1] = compressValue(first);
auto [value2, zero_byte_count2, predictor2] = compressValue(second);
auto [compressed_value1, zero_byte_count1, predictor1] = compressValue(first);
auto [compressed_value2, zero_byte_count2, predictor2] = compressValue(second);
std::byte header{0x0};
header |= (predictor1 << 4) | predictor2;
header |= static_cast<std::byte>((zero_byte_count1 << 4) | zero_byte_count2);
@ -378,14 +358,14 @@ private:
auto tail_size1 = VALUE_SIZE - zero_byte_count1;
auto tail_size2 = VALUE_SIZE - zero_byte_count2;
std::memcpy(result.data() + 1, valueTail(value1, zero_byte_count1), tail_size1);
std::memcpy(result.data() + 1 + tail_size1, valueTail(value2, zero_byte_count2), tail_size2);
std::memcpy(result.data() + 1, valueTail(compressed_value1, zero_byte_count1), tail_size1);
std::memcpy(result.data() + 1 + tail_size1, valueTail(compressed_value2, zero_byte_count2), tail_size2);
result = result.subspan(1 + tail_size1 + tail_size2);
}
std::size_t decodeChunk(std::span<const std::byte> values, std::span<TUint> seq)
{
std::size_t read_bytes{0};
std::size_t read_bytes = 0;
for (std::size_t i = 0; i < seq.size(); i += 2)
{
read_bytes += decodePair(values.subspan(read_bytes), seq[i], seq[i + 1]);
@ -411,7 +391,7 @@ private:
std::size_t decodePair(std::span<const std::byte> bytes, TUint& first, TUint& second)
{
if (bytes.empty())
if (bytes.empty()) [[unlikely]]
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Unexpected end of encoded sequence");
auto zero_byte_count1 = decodeCompressedZeroByteCount(
@ -422,11 +402,11 @@ private:
auto tail_size1 = VALUE_SIZE - zero_byte_count1;
auto tail_size2 = VALUE_SIZE - zero_byte_count2;
if (bytes.size() < 1 + tail_size1 + tail_size2)
if (bytes.size() < 1 + tail_size1 + tail_size2) [[unlikely]]
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Unexpected end of encoded sequence");
TUint value1{0};
TUint value2{0};
TUint value1 = 0;
TUint value2 = 0;
std::memcpy(valueTail(value1, zero_byte_count1), bytes.data() + 1, tail_size1);
std::memcpy(valueTail(value2, zero_byte_count2), bytes.data() + 1 + tail_size1, tail_size2);
@ -441,7 +421,7 @@ private:
static void* valueTail(TUint& value, unsigned compressed_size)
{
if constexpr (Endian == std::endian::little)
if constexpr (ENDIAN == std::endian::little)
{
return &value;
}
@ -453,7 +433,10 @@ private:
DfcmPredictor<TUint> dfcm_predictor;
FcmPredictor<TUint> fcm_predictor;
// memcpy the input into this buffer to align reads, this improves performance compared to unaligned reads (bit_cast) by ~10%
std::array<TUint, CHUNK_SIZE> chunk{};
std::span<std::byte> result{};
};
@ -463,7 +446,6 @@ UInt32 CompressionCodecFPC::doCompressData(const char * source, UInt32 source_si
{
dest[0] = static_cast<char>(float_width);
dest[1] = static_cast<char>(level);
dest[2] = std::to_integer<char>(encodeEndianness(std::endian::native));
auto dest_size = getMaxCompressedDataSize(source_size);
auto destination = std::as_writable_bytes(std::span(dest, dest_size).subspan(HEADER_SIZE));
@ -490,8 +472,6 @@ void CompressionCodecFPC::doDecompressData(const char * source, UInt32 source_si
auto compressed_level = std::to_integer<UInt8>(compressed_data[1]);
if (compressed_level == 0 || compressed_level > MAX_COMPRESSION_LEVEL)
throw Exception("Cannot decompress. File has incorrect level", ErrorCodes::CANNOT_DECOMPRESS);
if (decodeEndianness(compressed_data[2]) != std::endian::native)
throw Exception("Cannot decompress. File has incorrect endianness", ErrorCodes::CANNOT_DECOMPRESS);
auto destination = std::as_writable_bytes(std::span(dest, uncompressed_size));
auto src = compressed_data.subspan(HEADER_SIZE);

View File

@ -10,18 +10,6 @@
namespace DB
{
class TaskNotification final : public Poco::Notification
{
public:
explicit TaskNotification(const BackgroundSchedulePoolTaskInfoPtr & task_) : task(task_) {}
void execute() { task->execute(); }
private:
BackgroundSchedulePoolTaskInfoPtr task;
};
BackgroundSchedulePoolTaskInfo::BackgroundSchedulePoolTaskInfo(
BackgroundSchedulePool & pool_, const std::string & log_name_, const BackgroundSchedulePool::TaskFunc & function_)
: pool(pool_), log_name(log_name_), function(function_)
@ -39,7 +27,7 @@ bool BackgroundSchedulePoolTaskInfo::schedule()
return true;
}
bool BackgroundSchedulePoolTaskInfo::scheduleAfter(size_t ms, bool overwrite)
bool BackgroundSchedulePoolTaskInfo::scheduleAfter(size_t milliseconds, bool overwrite)
{
std::lock_guard lock(schedule_mutex);
@ -48,7 +36,7 @@ bool BackgroundSchedulePoolTaskInfo::scheduleAfter(size_t ms, bool overwrite)
if (delayed && !overwrite)
return false;
pool.scheduleDelayedTask(shared_from_this(), ms, lock);
pool.scheduleDelayedTask(shared_from_this(), milliseconds, lock);
return true;
}
@ -106,7 +94,7 @@ void BackgroundSchedulePoolTaskInfo::execute()
UInt64 milliseconds = watch.elapsedMilliseconds();
/// If the task is executed longer than specified time, it will be logged.
static const int32_t slow_execution_threshold_ms = 200;
static constexpr UInt64 slow_execution_threshold_ms = 200;
if (milliseconds >= slow_execution_threshold_ms)
LOG_TRACE(&Poco::Logger::get(log_name), "Execution took {} ms.", milliseconds);
@ -121,7 +109,7 @@ void BackgroundSchedulePoolTaskInfo::execute()
/// will have their chance to execute
if (scheduled)
pool.queue.enqueueNotification(new TaskNotification(shared_from_this()));
pool.scheduleTask(shared_from_this());
}
}
@ -136,14 +124,14 @@ void BackgroundSchedulePoolTaskInfo::scheduleImpl(std::lock_guard<std::mutex> &
/// But if it is currently executing, do nothing because it will be enqueued
/// at the end of the execute() method.
if (!executing)
pool.queue.enqueueNotification(new TaskNotification(shared_from_this()));
pool.scheduleTask(shared_from_this());
}
Coordination::WatchCallback BackgroundSchedulePoolTaskInfo::getWatchCallback()
{
return [t = shared_from_this()](const Coordination::WatchResponse &)
return [task = shared_from_this()](const Coordination::WatchResponse &)
{
t->schedule();
task->schedule();
};
}
@ -184,15 +172,18 @@ BackgroundSchedulePool::~BackgroundSchedulePool()
try
{
{
std::unique_lock lock(delayed_tasks_mutex);
std::lock_guard lock_tasks(tasks_mutex);
std::lock_guard lock_delayed_tasks(delayed_tasks_mutex);
shutdown = true;
wakeup_cond.notify_all();
}
queue.wakeUpAll();
delayed_thread.join();
tasks_cond_var.notify_all();
delayed_tasks_cond_var.notify_all();
LOG_TRACE(&Poco::Logger::get("BackgroundSchedulePool/" + thread_name), "Waiting for threads to finish.");
delayed_thread.join();
for (auto & thread : threads)
thread.join();
}
@ -208,6 +199,15 @@ BackgroundSchedulePool::TaskHolder BackgroundSchedulePool::createTask(const std:
return TaskHolder(std::make_shared<TaskInfo>(*this, name, function));
}
void BackgroundSchedulePool::scheduleTask(TaskInfoPtr task_info)
{
{
std::lock_guard tasks_lock(tasks_mutex);
tasks.push_back(std::move(task_info));
}
tasks_cond_var.notify_one();
}
void BackgroundSchedulePool::scheduleDelayedTask(const TaskInfoPtr & task, size_t ms, std::lock_guard<std::mutex> & /* task_schedule_mutex_lock */)
{
@ -223,7 +223,7 @@ void BackgroundSchedulePool::scheduleDelayedTask(const TaskInfoPtr & task, size_
task->delayed = true;
}
wakeup_cond.notify_all();
delayed_tasks_cond_var.notify_all();
}
@ -235,7 +235,7 @@ void BackgroundSchedulePool::cancelDelayedTask(const TaskInfoPtr & task, std::lo
task->delayed = false;
}
wakeup_cond.notify_all();
delayed_tasks_cond_var.notify_all();
}
@ -264,20 +264,25 @@ void BackgroundSchedulePool::threadFunction()
while (!shutdown)
{
/// We have to wait with timeout to prevent very rare deadlock, caused by the following race condition:
/// 1. Background thread N: threadFunction(): checks for shutdown (it's false)
/// 2. Main thread: ~BackgroundSchedulePool(): sets shutdown to true, calls queue.wakeUpAll(), it triggers
/// all existing Poco::Events inside Poco::NotificationQueue which background threads are waiting on.
/// 3. Background thread N: threadFunction(): calls queue.waitDequeueNotification(), it creates
/// new Poco::Event inside Poco::NotificationQueue and starts to wait on it
/// Background thread N will never be woken up.
/// TODO Do we really need Poco::NotificationQueue? Why not to use std::queue + mutex + condvar or maybe even DB::ThreadPool?
constexpr size_t wait_timeout_ms = 500;
if (Poco::AutoPtr<Poco::Notification> notification = queue.waitDequeueNotification(wait_timeout_ms))
TaskInfoPtr task;
{
TaskNotification & task_notification = static_cast<TaskNotification &>(*notification);
task_notification.execute();
std::unique_lock<std::mutex> tasks_lock(tasks_mutex);
tasks_cond_var.wait(tasks_lock, [&]()
{
return shutdown || !tasks.empty();
});
if (!tasks.empty())
{
task = tasks.front();
tasks.pop_front();
}
}
if (task)
task->execute();
}
}
@ -309,7 +314,7 @@ void BackgroundSchedulePool::delayExecutionThreadFunction()
if (!task)
{
wakeup_cond.wait(lock);
delayed_tasks_cond_var.wait(lock);
continue;
}
@ -317,7 +322,7 @@ void BackgroundSchedulePool::delayExecutionThreadFunction()
if (min_time > current_time)
{
wakeup_cond.wait_for(lock, std::chrono::microseconds(min_time - current_time));
delayed_tasks_cond_var.wait_for(lock, std::chrono::microseconds(min_time - current_time));
continue;
}
else

View File

@ -62,6 +62,8 @@ private:
void threadFunction();
void delayExecutionThreadFunction();
void scheduleTask(TaskInfoPtr task_info);
/// Schedule task for execution after specified delay from now.
void scheduleDelayedTask(const TaskInfoPtr & task_info, size_t ms, std::lock_guard<std::mutex> & task_schedule_mutex_lock);
@ -69,12 +71,16 @@ private:
void cancelDelayedTask(const TaskInfoPtr & task_info, std::lock_guard<std::mutex> & task_schedule_mutex_lock);
std::atomic<bool> shutdown {false};
/// Tasks.
std::condition_variable tasks_cond_var;
std::mutex tasks_mutex;
std::deque<TaskInfoPtr> tasks;
Threads threads;
Poco::NotificationQueue queue;
/// Delayed notifications.
/// Delayed tasks.
std::condition_variable wakeup_cond;
std::condition_variable delayed_tasks_cond_var;
std::mutex delayed_tasks_mutex;
/// Thread waiting for next delayed task.
ThreadFromGlobalPool delayed_thread;
@ -102,7 +108,7 @@ public:
/// Schedule for execution after specified delay.
/// If overwrite is set then the task will be re-scheduled (if it was already scheduled, i.e. delayed == true).
bool scheduleAfter(size_t ms, bool overwrite = true);
bool scheduleAfter(size_t milliseconds, bool overwrite = true);
/// Further attempts to schedule become no-op. Will wait till the end of the current execution of the task.
void deactivate();

View File

@ -352,26 +352,27 @@ private:
#if defined(OS_LINUX)
/// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace.
/// Please keep the below log messages in-sync with the ones in programs/server/Server.cpp
String calculated_binary_hash = getHashOfLoadedBinaryHex();
if (daemon.stored_binary_hash.empty())
{
LOG_FATAL(log, "Calculated checksum of the binary: {}."
" There is no information about the reference checksum.", calculated_binary_hash);
LOG_FATAL(log, "Integrity check of the executable skipped because the reference checksum could not be read."
" (calculated checksum: {})", calculated_binary_hash);
}
else if (calculated_binary_hash == daemon.stored_binary_hash)
{
LOG_FATAL(log, "Checksum of the binary: {}, integrity check passed.", calculated_binary_hash);
LOG_FATAL(log, "Integrity check of the executable successfully passed (checksum: {})", calculated_binary_hash);
}
else
{
LOG_FATAL(log, "Calculated checksum of the ClickHouse binary ({0}) does not correspond"
" to the reference checksum stored in the binary ({1})."
" It may indicate one of the following:"
" - the file was changed just after startup;"
" - the file is damaged on disk due to faulty hardware;"
" - the loaded executable is damaged in memory due to faulty hardware;"
LOG_FATAL(log, "Calculated checksum of the executable ({0}) does not correspond"
" to the reference checksum stored in the executable ({1})."
" This may indicate one of the following:"
" - the executable was changed just after startup;"
" - the executable was corrupted on disk due to faulty hardware;"
" - the loaded executable was corrupted in memory due to faulty hardware;"
" - the file was intentionally modified;"
" - logical error in code."
" - a logical error in the code."
, calculated_binary_hash, daemon.stored_binary_hash);
}
#endif
@ -872,7 +873,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing()
std::string executable_path = getExecutablePath();
if (!executable_path.empty())
stored_binary_hash = DB::Elf(executable_path).getBinaryHash();
stored_binary_hash = DB::Elf(executable_path).getStoredBinaryHash();
#endif
}

View File

@ -20,6 +20,10 @@
# include <nmmintrin.h>
#endif
#if defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
# include <arm_acle.h>
#endif
namespace DB
{
/** Distance function implementation.
@ -64,6 +68,8 @@ struct NgramDistanceImpl
UInt64 combined = (static_cast<UInt64>(code_points[0]) << 32) | code_points[1];
#ifdef __SSE4_2__
return _mm_crc32_u64(code_points[2], combined) & 0xFFFFu;
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
return __crc32cd(code_points[2], combined) & 0xFFFFu;
#else
return (intHashCRC32(combined) ^ intHashCRC32(code_points[2])) & 0xFFFFu;
#endif

View File

@ -2,7 +2,13 @@
#include <Functions/FunctionBinaryArithmetic.h>
#if defined(__SSE2__)
# define LIBDIVIDE_SSE2 1
# define LIBDIVIDE_SSE2
#elif defined(__AVX512F__) || defined(__AVX512BW__) || defined(__AVX512VL__)
# define LIBDIVIDE_AVX512
#elif defined(__AVX2__)
# define LIBDIVIDE_AVX2
#elif defined(__aarch64__) && defined(__ARM_NEON)
# define LIBDIVIDE_NEON
#endif
#include <libdivide.h>

View File

@ -11,8 +11,16 @@
# include <emmintrin.h>
#endif
#if defined(__aarch64__) && defined(__ARM_NEON)
# include <arm_neon.h>
# ifdef HAS_RESERVED_IDENTIFIER
# pragma clang diagnostic ignored "-Wreserved-identifier"
# endif
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
@ -61,6 +69,24 @@ struct ToValidUTF8Impl
while (p < simd_end && !_mm_movemask_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i *>(p))))
p += SIMD_BYTES;
if (!(p < end))
break;
#elif defined(__aarch64__) && defined(__ARM_NEON)
/// Fast skip of ASCII for aarch64.
static constexpr size_t SIMD_BYTES = 16;
const char * simd_end = p + (end - p) / SIMD_BYTES * SIMD_BYTES;
/// Returns a 64 bit mask of nibbles (4 bits for each byte).
auto get_nibble_mask = [](uint8x16_t input) -> uint64_t
{ return vget_lane_u64(vreinterpret_u64_u8(vshrn_n_u16(vreinterpretq_u16_u8(input), 4)), 0); };
/// Other options include
/// vmaxvq_u8(input) < 0b10000000;
/// Used by SIMDJSON, has latency 3 for M1, 6 for everything else
/// SIMDJSON uses it for 64 byte masks, so it's a little different.
/// vmaxvq_u32(vandq_u32(input, vdupq_n_u32(0x80808080))) // u32 version has latency 3
/// shrn version has universally <=3 cycles, on servers 2 cycles.
while (p < simd_end && get_nibble_mask(vcgeq_u8(vld1q_u8(reinterpret_cast<const uint8_t *>(p)), vdupq_n_u8(0x80))) == 0)
p += SIMD_BYTES;
if (!(p < end))
break;
#endif

View File

@ -15,6 +15,13 @@
#include <emmintrin.h>
#endif
#if defined(__aarch64__) && defined(__ARM_NEON)
# include <arm_neon.h>
# ifdef HAS_RESERVED_IDENTIFIER
# pragma clang diagnostic ignored "-Wreserved-identifier"
# endif
#endif
namespace DB
{
@ -695,6 +702,24 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV &
return;
}
}
#elif defined(__aarch64__) && defined(__ARM_NEON)
auto rc = vdupq_n_u8('\r');
auto nc = vdupq_n_u8('\n');
auto dc = vdupq_n_u8(delimiter);
/// Returns a 64 bit mask of nibbles (4 bits for each byte).
auto get_nibble_mask = [](uint8x16_t input) -> uint64_t
{ return vget_lane_u64(vreinterpret_u64_u8(vshrn_n_u16(vreinterpretq_u16_u8(input), 4)), 0); };
for (; next_pos + 15 < buf.buffer().end(); next_pos += 16)
{
uint8x16_t bytes = vld1q_u8(reinterpret_cast<const uint8_t *>(next_pos));
auto eq = vorrq_u8(vorrq_u8(vceqq_u8(bytes, rc), vceqq_u8(bytes, nc)), vceqq_u8(bytes, dc));
uint64_t bit_mask = get_nibble_mask(eq);
if (bit_mask)
{
next_pos += __builtin_ctzll(bit_mask) >> 2;
return;
}
}
#endif
while (next_pos < buf.buffer().end()
&& *next_pos != delimiter && *next_pos != '\r' && *next_pos != '\n')

View File

@ -908,6 +908,8 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re
return ReturnType(false);
}
int negative_multiplier = 1;
DB::DecimalUtils::DecimalComponents<DateTime64> components{static_cast<DateTime64::NativeType>(whole), 0};
if (!buf.eof() && *buf.position() == '.')
@ -934,29 +936,18 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re
while (!buf.eof() && isNumericASCII(*buf.position()))
++buf.position();
/// Keep sign of fractional part the same with whole part if datetime64 is negative
/// Case1:
/// 1965-12-12 12:12:12.123
/// => whole = -127914468, fractional = 123(coefficient>0)
/// => new whole = -127914467, new fractional = 877(coefficient<0)
///
/// Case2:
/// 1969-12-31 23:59:59.123
/// => whole = -1, fractional = 123(coefficient>0)
/// => new whole = 0, new fractional = -877(coefficient>0)
/// Fractional part (subseconds) is treated as positive by users
/// (as DateTime64 itself is a positive, although underlying decimal is negative)
/// setting fractional part to be negative when whole is 0 results in wrong value,
/// so we multiply result by -1.
if (components.whole < 0 && components.fractional != 0)
{
const auto scale_multiplier = DecimalUtils::scaleMultiplier<DateTime64::NativeType>(scale);
++components.whole;
if (components.whole)
components.fractional = scale_multiplier - components.fractional;
if (!components.whole)
{
/// whole keep the sign, fractional should be non-negative
components.fractional = scale_multiplier - components.fractional;
}
else
{
/// when whole is zero, fractional should keep the sign
components.fractional = components.fractional - scale_multiplier;
negative_multiplier = -1;
}
}
}
@ -969,7 +960,7 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re
components.whole = components.whole / common::exp10_i32(scale);
}
datetime64 = DecimalUtils::decimalFromComponents<DateTime64>(components, scale);
datetime64 = negative_multiplier * DecimalUtils::decimalFromComponents<DateTime64>(components, scale);
return ReturnType(true);
}

View File

@ -6,6 +6,12 @@
#include <emmintrin.h>
#endif
#if defined(__aarch64__) && defined(__ARM_NEON)
# include <arm_neon.h>
# ifdef HAS_RESERVED_IDENTIFIER
# pragma clang diagnostic ignored "-Wreserved-identifier"
# endif
#endif
namespace DB
{
@ -67,13 +73,31 @@ void WriteBufferValidUTF8::nextImpl()
while (p < pos)
{
#ifdef __SSE2__
/// Fast skip of ASCII
/// Fast skip of ASCII for x86.
static constexpr size_t SIMD_BYTES = 16;
const char * simd_end = p + (pos - p) / SIMD_BYTES * SIMD_BYTES;
while (p < simd_end && !_mm_movemask_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p))))
p += SIMD_BYTES;
if (!(p < pos))
break;
#elif defined(__aarch64__) && defined(__ARM_NEON)
/// Fast skip of ASCII for aarch64.
static constexpr size_t SIMD_BYTES = 16;
const char * simd_end = p + (pos - p) / SIMD_BYTES * SIMD_BYTES;
/// Returns a 64 bit mask of nibbles (4 bits for each byte).
auto get_nibble_mask = [](uint8x16_t input) -> uint64_t
{ return vget_lane_u64(vreinterpret_u64_u8(vshrn_n_u16(vreinterpretq_u16_u8(input), 4)), 0); };
/// Other options include
/// vmaxvq_u8(input) < 0b10000000;
/// Used by SIMDJSON, has latency 3 for M1, 6 for everything else
/// SIMDJSON uses it for 64 byte masks, so it's a little different.
/// vmaxvq_u32(vandq_u32(input, vdupq_n_u32(0x80808080))) // u32 version has latency 3
/// shrn version has universally <=3 cycles, on servers 2 cycles.
while (p < simd_end && get_nibble_mask(vcgeq_u8(vld1q_u8(reinterpret_cast<const uint8_t *>(p)), vdupq_n_u8(0x80))) == 0)
p += SIMD_BYTES;
if (!(p < pos))
break;
#endif

View File

@ -257,7 +257,6 @@ SelectStreamFactory::ShardPlans SelectStreamFactory::createForShardWithParallelR
std::move(scalars),
std::move(external_tables),
&Poco::Logger::get("ReadFromParallelRemoteReplicasStep"),
shard_count,
storage_limits);
remote_plan->addStep(std::move(read_from_remote));

View File

@ -76,7 +76,7 @@ bool FillingRow::next(const FillingRow & to_row)
auto next_value = row[pos];
getFillDescription(pos).step_func(next_value);
if (less(to_row.row[pos], next_value, getDirection(pos)))
if (less(to_row.row[pos], next_value, getDirection(pos)) || equals(next_value, getFillDescription(pos).fill_to))
return false;
row[pos] = next_value;

View File

@ -6,7 +6,13 @@
#include <type_traits>
#if defined(__SSE2__)
# define LIBDIVIDE_SSE2 1
# define LIBDIVIDE_SSE2
#elif defined(__AVX512F__) || defined(__AVX512BW__) || defined(__AVX512VL__)
# define LIBDIVIDE_AVX512
#elif defined(__AVX2__)
# define LIBDIVIDE_AVX2
#elif defined(__aarch64__) && defined(__ARM_NEON)
# define LIBDIVIDE_NEON
#endif
#include <libdivide.h>

View File

@ -1,4 +1,5 @@
#include <Processors/Merges/IMergingTransform.h>
#include <Processors/Transforms/SelectorInfo.h>
namespace DB
{
@ -180,4 +181,68 @@ IProcessor::Status IMergingTransformBase::prepare()
return Status::Ready;
}
static void filterChunk(IMergingAlgorithm::Input & input, size_t selector_position)
{
if (!input.chunk.getChunkInfo())
throw Exception("IMergingTransformBase expected ChunkInfo for input chunk", ErrorCodes::LOGICAL_ERROR);
const auto * chunk_info = typeid_cast<const SelectorInfo *>(input.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 = input.chunk.getNumRows();
auto columns = input.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;
}
}
if (!filter.empty() && filter.back() == 0)
{
filter.back() = 1;
++num_result_rows;
input.skip_last_row = true;
}
for (auto & column : columns)
column = column->filter(filter, num_result_rows);
input.chunk.clear();
input.chunk.setColumns(std::move(columns), num_result_rows);
}
void IMergingTransformBase::filterChunks()
{
if (state.selector_position < 0)
return;
if (!state.init_chunks.empty())
{
for (size_t i = 0; i < input_states.size(); ++i)
{
auto & input = state.init_chunks[i];
if (!input.chunk)
continue;
filterChunk(input, state.selector_position);
}
}
if (state.has_input)
filterChunk(state.input_chunk, state.selector_position);
}
}

View File

@ -28,10 +28,17 @@ 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.
void filterChunks(); /// Filter chunks if selector position was set. For parallel final.
/// Processor state.
struct State
{
@ -43,6 +50,7 @@ protected:
size_t next_input_to_read = 0;
IMergingAlgorithm::Inputs init_chunks;
ssize_t selector_position = -1;
};
State state;
@ -84,6 +92,8 @@ public:
void work() override
{
filterChunks();
if (!state.init_chunks.empty())
algorithm.initialize(std::move(state.init_chunks));

View File

@ -86,9 +86,6 @@ static void doDescribeProcessor(const IProcessor & processor, size_t count, IQue
doDescribeHeader(*last_header, num_equal_headers, settings);
}
if (!processor.getDescription().empty())
settings.out << String(settings.offset, settings.indent_char) << "Description: " << processor.getDescription() << '\n';
settings.offset += settings.indent;
}

View File

@ -1,274 +0,0 @@
#include <algorithm>
#include <memory>
#include <numeric>
#include <queue>
#include <unordered_map>
#include <vector>
#include <Core/Field.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Processors/QueryPlan/PartsSplitter.h>
#include <Processors/Transforms/FilterSortedStreamByRange.h>
#include <Storages/MergeTree/RangesInDataPart.h>
using namespace DB;
namespace
{
using Value = std::vector<Field>;
std::string toString(const Value & value)
{
return fmt::format("({})", fmt::join(value, ", "));
}
/// Adaptor to access PK values from index.
class IndexAccess
{
public:
explicit IndexAccess(const RangesInDataParts & parts_) : parts(parts_) { }
Value getValue(size_t part_idx, size_t mark) const
{
const auto & index = parts[part_idx].data_part->index;
Value value(index.size());
for (size_t i = 0; i < value.size(); ++i)
index[i]->get(mark, value[i]);
return value;
}
size_t getMarkRows(size_t part_idx, size_t mark) const { return parts[part_idx].data_part->index_granularity.getMarkRows(mark); }
size_t getTotalRowCount() const
{
size_t total = 0;
for (const auto & part : parts)
total += part.getRowsCount();
return total;
}
private:
const RangesInDataParts & parts;
};
/// Splits parts into layers, each layer will contain parts subranges with PK values from its own range.
/// Will try to produce exactly max_layer layers but may return less if data is distributed in not a very parallelizable way.
std::pair<std::vector<Value>, std::vector<RangesInDataParts>> split(RangesInDataParts parts, size_t max_layers)
{
// We will advance the iterator pointing to the mark with the smallest PK value until there will be not less than rows_per_layer rows in the current layer (roughly speaking).
// Then we choose the last observed value as the new border, so the current layer will consists of granules with values greater than the previous mark and less or equal
// than the new border.
struct PartsRangesIterator
{
struct RangeInDataPart : MarkRange
{
size_t part_idx;
};
enum class EventType
{
RangeBeginning,
RangeEnding,
};
bool operator<(const PartsRangesIterator & other) const { return std::tie(value, event) > std::tie(other.value, other.event); }
Value value;
RangeInDataPart range;
EventType event;
};
const auto index_access = std::make_unique<IndexAccess>(parts);
std::priority_queue<PartsRangesIterator> parts_ranges_queue;
for (size_t part_idx = 0; part_idx < parts.size(); ++part_idx)
{
for (const auto & range : parts[part_idx].ranges)
{
parts_ranges_queue.push(
{index_access->getValue(part_idx, range.begin), {range, part_idx}, PartsRangesIterator::EventType::RangeBeginning});
const auto & index_granularity = parts[part_idx].data_part->index_granularity;
if (index_granularity.hasFinalMark() && range.end + 1 == index_granularity.getMarksCount())
parts_ranges_queue.push(
{index_access->getValue(part_idx, range.end), {range, part_idx}, PartsRangesIterator::EventType::RangeEnding});
}
}
/// The beginning of currently started (but not yet finished) range of marks of a part in the current layer.
std::unordered_map<size_t, size_t> current_part_range_begin;
/// The current ending of a range of marks of a part in the current layer.
std::unordered_map<size_t, size_t> current_part_range_end;
/// Determine borders between layers.
std::vector<Value> borders;
std::vector<RangesInDataParts> result_layers;
const size_t rows_per_layer = std::max<size_t>(index_access->getTotalRowCount() / max_layers, 1);
while (!parts_ranges_queue.empty())
{
// New layer should include last granules of still open ranges from the previous layer,
// because they may already contain values greater than the last border.
size_t rows_in_current_layer = 0;
size_t marks_in_current_layer = 0;
// Intersection between the current and next layers is just the last observed marks of each still open part range. Ratio is empirical.
auto layers_intersection_is_too_big = [&]()
{
const auto intersected_parts = current_part_range_end.size();
return marks_in_current_layer < intersected_parts * 2;
};
result_layers.emplace_back();
while (rows_in_current_layer < rows_per_layer || layers_intersection_is_too_big() || result_layers.size() == max_layers)
{
// We're advancing iterators until a new value showed up.
Value last_value;
while (!parts_ranges_queue.empty() && (last_value.empty() || last_value == parts_ranges_queue.top().value))
{
auto current = parts_ranges_queue.top();
parts_ranges_queue.pop();
const auto part_idx = current.range.part_idx;
if (current.event == PartsRangesIterator::EventType::RangeEnding)
{
result_layers.back().emplace_back(
parts[part_idx].data_part,
parts[part_idx].part_index_in_query,
MarkRanges{{current_part_range_begin[part_idx], current.range.end}});
current_part_range_begin.erase(part_idx);
current_part_range_end.erase(part_idx);
continue;
}
last_value = std::move(current.value);
rows_in_current_layer += index_access->getMarkRows(part_idx, current.range.begin);
marks_in_current_layer++;
current_part_range_begin.try_emplace(part_idx, current.range.begin);
current_part_range_end[part_idx] = current.range.begin;
if (current.range.begin + 1 < current.range.end)
{
current.range.begin++;
current.value = index_access->getValue(part_idx, current.range.begin);
parts_ranges_queue.push(std::move(current));
}
}
if (parts_ranges_queue.empty())
break;
if (rows_in_current_layer >= rows_per_layer && !layers_intersection_is_too_big() && result_layers.size() < max_layers)
borders.push_back(last_value);
}
for (const auto & [part_idx, last_mark] : current_part_range_end)
{
result_layers.back().emplace_back(
parts[part_idx].data_part,
parts[part_idx].part_index_in_query,
MarkRanges{{current_part_range_begin[part_idx], last_mark + 1}});
current_part_range_begin[part_idx] = current_part_range_end[part_idx];
}
}
for (auto & layer : result_layers)
{
std::stable_sort(
layer.begin(),
layer.end(),
[](const auto & lhs, const auto & rhs) { return lhs.part_index_in_query < rhs.part_index_in_query; });
}
return std::make_pair(std::move(borders), std::move(result_layers));
}
/// Will return borders.size()+1 filters in total, i-th filter will accept rows with PK values within the range [borders[i-1], borders[i]).
std::vector<ASTPtr> buildFilters(const KeyDescription & primary_key, const std::vector<Value> & borders)
{
auto add_and_condition = [&](ASTPtr & result, const ASTPtr & foo) { result = !result ? foo : makeASTFunction("and", result, foo); };
/// Produces ASTPtr to predicate (pk_col0, pk_col1, ... , pk_colN) > (value[0], value[1], ... , value[N])
auto lexicographically_greater = [&](const Value & value)
{
// PK may contain functions of the table columns, so we need the actual PK AST with all expressions it contains.
ASTPtr pk_columns_as_tuple = makeASTFunction("tuple", primary_key.expression_list_ast->children);
ASTPtr value_ast = std::make_shared<ASTExpressionList>();
for (size_t i = 0; i < value.size(); ++i)
{
const auto & types = primary_key.data_types;
ASTPtr component_ast = std::make_shared<ASTLiteral>(value[i]);
// Values of some types (e.g. Date, DateTime) are stored in columns as numbers and we get them as just numbers from the index.
// So we need an explicit Cast for them.
if (isColumnedAsNumber(types.at(i)->getTypeId()) && !isNumber(types.at(i)->getTypeId()))
component_ast = makeASTFunction("cast", std::move(component_ast), std::make_shared<ASTLiteral>(types.at(i)->getName()));
value_ast->children.push_back(std::move(component_ast));
}
ASTPtr value_as_tuple = makeASTFunction("tuple", value_ast->children);
return makeASTFunction("greater", pk_columns_as_tuple, value_as_tuple);
};
std::vector<ASTPtr> filters(borders.size() + 1);
for (size_t layer = 0; layer <= borders.size(); ++layer)
{
if (layer > 0)
add_and_condition(filters[layer], lexicographically_greater(borders[layer - 1]));
if (layer < borders.size())
add_and_condition(filters[layer], makeASTFunction("not", lexicographically_greater(borders[layer])));
}
return filters;
}
}
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
Pipes buildPipesForReadingByPKRanges(
const KeyDescription & primary_key,
RangesInDataParts parts,
size_t max_layers,
ContextPtr context,
ReadingInOrderStepGetter && reading_step_getter)
{
if (max_layers <= 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "max_layer should be greater than 1.");
auto && [borders, result_layers] = split(std::move(parts), max_layers);
auto filters = buildFilters(primary_key, borders);
Pipes pipes(result_layers.size());
for (size_t i = 0; i < result_layers.size(); ++i)
{
pipes[i] = reading_step_getter(std::move(result_layers[i]));
auto & filter_function = filters[i];
if (!filter_function)
continue;
auto syntax_result = TreeRewriter(context).analyze(filter_function, primary_key.expression->getRequiredColumnsWithTypes());
auto actions = ExpressionAnalyzer(filter_function, syntax_result, context).getActionsDAG(false);
ExpressionActionsPtr expression_actions = std::make_shared<ExpressionActions>(std::move(actions));
auto description = fmt::format(
"filter values in [{}, {})", i ? ::toString(borders[i - 1]) : "-inf", i < borders.size() ? ::toString(borders[i]) : "+inf");
auto pk_expression = std::make_shared<ExpressionActions>(primary_key.expression->getActionsDAG().clone());
pipes[i].addSimpleTransform([pk_expression](const Block & header)
{ return std::make_shared<ExpressionTransform>(header, pk_expression); });
pipes[i].addSimpleTransform(
[&](const Block & header)
{
auto step = std::make_shared<FilterSortedStreamByRange>(header, expression_actions, filter_function->getColumnName(), true);
step->setDescription(description);
return step;
});
}
return pipes;
}
}

View File

@ -1,25 +0,0 @@
#pragma once
#include <functional>
#include <Interpreters/Context_fwd.h>
#include <QueryPipeline/Pipe.h>
#include <Storages/KeyDescription.h>
#include <Storages/MergeTree/RangesInDataPart.h>
namespace DB
{
using ReadingInOrderStepGetter = std::function<Pipe(RangesInDataParts)>;
/// Splits parts into layers, each layer will contain parts subranges with PK values from its own range.
/// A separate pipe will be constructed for each layer with a reading step (provided by the reading_step_getter) and a filter for this layer's range of PK values.
/// Will try to produce exactly max_layer pipes but may return less if data is distributed in not a very parallelizable way.
Pipes buildPipesForReadingByPKRanges(
const KeyDescription & primary_key,
RangesInDataParts parts,
size_t max_layers,
ContextPtr context,
ReadingInOrderStepGetter && reading_step_getter);
}

View File

@ -1,16 +1,14 @@
#include <algorithm>
#include <functional>
#include <memory>
#include <numeric>
#include <queue>
#include <stdexcept>
#include <IO/Operators.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Processors/ConcatProcessor.h>
#include <Processors/Transforms/ReverseTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/FilterTransform.h>
#include <Processors/Transforms/AddingSelectorTransform.h>
#include <Processors/Transforms/CopyTransform.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/Merges/AggregatingSortedTransform.h>
#include <Processors/Merges/CollapsingSortedTransform.h>
#include <Processors/Merges/GraphiteRollupSortedTransform.h>
@ -18,22 +16,17 @@
#include <Processors/Merges/ReplacingSortedTransform.h>
#include <Processors/Merges/SummingSortedTransform.h>
#include <Processors/Merges/VersionedCollapsingTransform.h>
#include <Processors/QueryPlan/PartsSplitter.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/FilterTransform.h>
#include <Processors/Transforms/ReverseTransform.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <Storages/MergeTree/MergeTreeInOrderSelectProcessor.h>
#include <Storages/MergeTree/MergeTreeReadPool.h>
#include <Storages/MergeTree/MergeTreeReverseSelectProcessor.h>
#include <Storages/MergeTree/MergeTreeThreadSelectProcessor.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <Storages/MergeTree/MergeTreeReadPool.h>
#include <Storages/VirtualColumnUtils.h>
#include <IO/Operators.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <Common/logger_useful.h>
#include <base/sort.h>
#include <Poco/Logger.h>
#include <Common/JSONBuilder.h>
namespace ProfileEvents
@ -567,6 +560,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
static void addMergingFinal(
Pipe & pipe,
size_t num_output_streams,
const SortDescription & sort_description,
MergeTreeData::MergingParams merging_params,
Names partition_key_columns,
@ -613,7 +607,56 @@ static void addMergingFinal(
__builtin_unreachable();
};
pipe.addTransform(get_merging_processor());
if (num_output_streams <= 1 || sort_description.empty())
{
pipe.addTransform(get_merging_processor());
return;
}
ColumnNumbers key_columns;
key_columns.reserve(sort_description.size());
for (const auto & desc : sort_description)
key_columns.push_back(header.getPositionByName(desc.column_name));
pipe.addSimpleTransform([&](const Block & stream_header)
{
return std::make_shared<AddingSelectorTransform>(stream_header, num_output_streams, key_columns);
});
pipe.transform([&](OutputPortRawPtrs ports)
{
Processors transforms;
std::vector<OutputPorts::iterator> output_ports;
transforms.reserve(ports.size() + num_output_streams);
output_ports.reserve(ports.size());
for (auto & port : ports)
{
auto copier = std::make_shared<CopyTransform>(header, num_output_streams);
connect(*port, copier->getInputPort());
output_ports.emplace_back(copier->getOutputs().begin());
transforms.emplace_back(std::move(copier));
}
for (size_t i = 0; i < num_output_streams; ++i)
{
auto merge = get_merging_processor();
merge->setSelectorPosition(i);
auto input = merge->getInputs().begin();
/// Connect i-th merge with i-th input port of every copier.
for (size_t j = 0; j < ports.size(); ++j)
{
connect(*output_ports[j], *input);
++output_ports[j];
++input;
}
transforms.emplace_back(std::move(merge));
}
return transforms;
});
}
@ -667,7 +710,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index)
{
Pipes pipes;
Pipe pipe;
{
RangesInDataParts new_parts;
@ -694,39 +738,21 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
if (new_parts.empty())
continue;
if (num_streams > 1 && metadata_for_reading->hasPrimaryKey())
{
// Let's split parts into layers to ensure data parallelism of final.
auto reading_step_getter = [this, &column_names, &info](auto parts)
{
return read(
std::move(parts),
column_names,
ReadFromMergeTree::ReadType::InOrder,
1 /* num_streams */,
0 /* min_marks_for_concurrent_read */,
info.use_uncompressed_cache);
};
pipes = buildPipesForReadingByPKRanges(
metadata_for_reading->getPrimaryKey(), std::move(new_parts), num_streams, context, std::move(reading_step_getter));
}
else
{
pipes.emplace_back(read(
std::move(new_parts), column_names, ReadFromMergeTree::ReadType::InOrder, num_streams, 0, info.use_uncompressed_cache));
}
pipe = read(std::move(new_parts), column_names, ReadFromMergeTree::ReadType::InOrder,
num_streams, 0, info.use_uncompressed_cache);
/// Drop temporary columns, added by 'sorting_key_expr'
if (!out_projection)
out_projection = createProjection(pipes.front().getHeader());
out_projection = createProjection(pipe.getHeader());
}
auto sorting_expr = std::make_shared<ExpressionActions>(
metadata_for_reading->getSortingKey().expression->getActionsDAG().clone());
for (auto & pipe : pipes)
pipe.addSimpleTransform([sorting_expr](const Block & header)
{ return std::make_shared<ExpressionTransform>(header, sorting_expr); });
pipe.addSimpleTransform([sorting_expr](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, sorting_expr);
});
/// If do_not_merge_across_partitions_select_final is true and there is only one part in partition
/// with level > 0 then we won't postprocess this part
@ -734,7 +760,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 &&
parts_to_merge_ranges[range_index]->data_part->info.level > 0)
{
partition_pipes.emplace_back(Pipe::unitePipes(std::move(pipes)));
partition_pipes.emplace_back(std::move(pipe));
continue;
}
@ -751,21 +777,21 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
for (size_t i = 0; i < sort_columns_size; ++i)
sort_description.emplace_back(sort_columns[i], 1, 1);
for (auto & pipe : pipes)
addMergingFinal(
pipe,
sort_description,
data.merging_params,
partition_key_columns,
max_block_size);
addMergingFinal(
pipe,
std::min<size_t>(num_streams, settings.max_final_threads),
sort_description, data.merging_params, partition_key_columns, max_block_size);
partition_pipes.emplace_back(Pipe::unitePipes(std::move(pipes)));
partition_pipes.emplace_back(std::move(pipe));
}
if (!lonely_parts.empty())
{
RangesInDataParts new_parts;
size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size();
const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead(
settings.merge_tree_min_rows_for_concurrent_read,
settings.merge_tree_min_bytes_for_concurrent_read,

View File

@ -239,7 +239,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
Scalars scalars_,
Tables external_tables_,
Poco::Logger * log_,
UInt32 shard_count_,
std::shared_ptr<const StorageLimitsList> storage_limits_)
: ISourceStep(DataStream{.header = std::move(header_)})
, coordinator(std::move(coordinator_))
@ -253,7 +252,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
, external_tables{external_tables_}
, storage_limits(std::move(storage_limits_))
, log(log_)
, shard_count(shard_count_)
{
std::vector<String> description;

View File

@ -83,7 +83,6 @@ public:
Scalars scalars_,
Tables external_tables_,
Poco::Logger * log_,
UInt32 shard_count_,
std::shared_ptr<const StorageLimitsList> storage_limits_);
String getName() const override { return "ReadFromRemoteParallelReplicas"; }
@ -110,8 +109,6 @@ private:
std::shared_ptr<const StorageLimitsList> storage_limits;
Poco::Logger * log;
UInt32 shard_count{0};
};
}

View File

@ -0,0 +1,76 @@
#include <Processors/Transforms/AddingSelectorTransform.h>
#include <Processors/Transforms/SelectorInfo.h>
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<SelectorInfo>();
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));
}
}

View File

@ -0,0 +1,26 @@
#pragma once
#include <Processors/IProcessor.h>
#include <Processors/ISimpleTransform.h>
#include <Core/ColumnNumbers.h>
#include <Common/WeakHash.h>
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;
};
}

View File

@ -1,66 +0,0 @@
#pragma once
#include <Interpreters/ExpressionActions.h>
#include <Processors/ISimpleTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/FilterTransform.h>
namespace DB
{
/// Could be used when the predicate given by expression_ is true only on one continuous range of values and input is monotonous by that value.
/// The following optimization applies: when a new chunk of data comes in we firstly execute the expression_ only on the first and the last row.
/// If it evaluates to true on both rows then the whole chunk is immediately passed to further steps.
/// Otherwise, we apply the expression_ to all rows.
class FilterSortedStreamByRange : public ISimpleTransform
{
public:
FilterSortedStreamByRange(
const Block & header_,
ExpressionActionsPtr expression_,
String filter_column_name_,
bool remove_filter_column_,
bool on_totals_ = false)
: ISimpleTransform(
header_,
FilterTransform::transformHeader(header_, expression_->getActionsDAG(), filter_column_name_, remove_filter_column_),
true)
, filter_transform(header_, expression_, filter_column_name_, remove_filter_column_, on_totals_)
{
}
String getName() const override { return "FilterSortedStreamByRange"; }
void transform(Chunk & chunk) override
{
int rows_before_filtration = chunk.getNumRows();
if (rows_before_filtration < 2)
{
filter_transform.transform(chunk);
return;
}
// Evaluate expression on just the first and the last row.
// If both of them satisfies conditions, than skip calculation for all the rows in between.
auto quick_check_columns = chunk.cloneEmptyColumns();
auto src_columns = chunk.detachColumns();
for (auto row : {0, rows_before_filtration - 1})
for (size_t col = 0; col < quick_check_columns.size(); ++col)
quick_check_columns[col]->insertFrom(*src_columns[col].get(), row);
chunk.setColumns(std::move(quick_check_columns), 2);
filter_transform.transform(chunk);
const bool all_rows_will_pass_filter = chunk.getNumRows() == 2;
chunk.setColumns(std::move(src_columns), rows_before_filtration);
// Not all rows satisfy conditions.
if (!all_rows_will_pass_filter)
filter_transform.transform(chunk);
}
private:
FilterTransform filter_transform;
};
}

View File

@ -32,6 +32,7 @@ public:
Status prepare() override;
protected:
void transform(Chunk & chunk) override;
private:

View File

@ -1,50 +1,26 @@
#include <Processors/Transforms/PartialSortingTransform.h>
#include <Interpreters/sortBlock.h>
#include <Core/SortCursor.h>
#include <Common/PODArray.h>
namespace DB
{
PartialSortingTransform::PartialSortingTransform(
const Block & header_, SortDescription & description_, UInt64 limit_)
: ISimpleTransform(header_, header_, false)
, description(description_), limit(limit_)
namespace
{
// Sorting by no columns doesn't make sense.
assert(!description.empty());
}
static ColumnRawPtrs extractColumns(const Block & block, const SortDescription & description)
ColumnRawPtrs extractRawColumns(const Block & block, const SortDescriptionWithPositions & description)
{
size_t size = description.size();
ColumnRawPtrs res;
res.reserve(size);
ColumnRawPtrs result(size);
for (size_t i = 0; i < size; ++i)
{
const IColumn * column = block.getByName(description[i].column_name).column.get();
res.emplace_back(column);
}
result[i] = block.safeGetByPosition(description[i].column_number).column.get();
return res;
return result;
}
bool less(const ColumnRawPtrs & lhs, UInt64 lhs_row_num,
const ColumnRawPtrs & rhs, UInt64 rhs_row_num, const SortDescription & description)
{
size_t size = description.size();
for (size_t i = 0; i < size; ++i)
{
int res = description[i].direction * lhs[i]->compareAt(lhs_row_num, rhs_row_num, *rhs[i], description[i].nulls_direction);
if (res < 0)
return true;
else if (res > 0)
return false;
}
return false;
}
size_t getFilterMask(const ColumnRawPtrs & lhs, const ColumnRawPtrs & rhs, size_t rhs_row_num,
size_t getFilterMask(const ColumnRawPtrs & raw_block_columns, const Columns & threshold_columns,
const SortDescription & description, size_t num_rows, IColumn::Filter & filter,
PaddedPODArray<UInt64> & rows_to_compare, PaddedPODArray<Int8> & compare_results)
{
@ -54,7 +30,7 @@ size_t getFilterMask(const ColumnRawPtrs & lhs, const ColumnRawPtrs & rhs, size_
if (description.size() == 1)
{
/// Fast path for single column
lhs[0]->compareColumn(*rhs[0], rhs_row_num, nullptr, compare_results,
raw_block_columns[0]->compareColumn(*threshold_columns[0], 0, nullptr, compare_results,
description[0].direction, description[0].nulls_direction);
}
else
@ -67,7 +43,7 @@ size_t getFilterMask(const ColumnRawPtrs & lhs, const ColumnRawPtrs & rhs, size_
size_t size = description.size();
for (size_t i = 0; i < size; ++i)
{
lhs[i]->compareColumn(*rhs[i], rhs_row_num, &rows_to_compare, compare_results,
raw_block_columns[i]->compareColumn(*threshold_columns[i], 0, &rows_to_compare, compare_results,
description[i].direction, description[i].nulls_direction);
if (rows_to_compare.empty())
@ -87,6 +63,41 @@ size_t getFilterMask(const ColumnRawPtrs & lhs, const ColumnRawPtrs & rhs, size_
return result_size_hint;
}
bool compareWithThreshold(const ColumnRawPtrs & raw_block_columns, size_t min_block_index, const Columns & threshold_columns, const SortDescription & sort_description)
{
assert(raw_block_columns.size() == threshold_columns.size());
assert(raw_block_columns.size() == sort_description.size());
size_t raw_block_columns_size = raw_block_columns.size();
for (size_t i = 0; i < raw_block_columns_size; ++i)
{
int res = sort_description[i].direction * raw_block_columns[i]->compareAt(min_block_index, 0, *threshold_columns[0], sort_description[i].nulls_direction);
if (res < 0)
return true;
else if (res > 0)
return false;
}
return false;
}
}
PartialSortingTransform::PartialSortingTransform(
const Block & header_, SortDescription & description_, UInt64 limit_)
: ISimpleTransform(header_, header_, false)
, description(description_)
, limit(limit_)
{
// Sorting by no columns doesn't make sense.
assert(!description_.empty());
size_t description_size = description.size();
for (size_t i = 0; i < description_size; ++i)
description_with_positions.emplace_back(description[i], header_.getPositionByName(description[i].column_name));
}
void PartialSortingTransform::transform(Chunk & chunk)
{
if (chunk.getNumRows())
@ -101,17 +112,18 @@ void PartialSortingTransform::transform(Chunk & chunk)
read_rows->add(chunk.getNumRows());
auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns());
size_t block_rows_before_filter = block.rows();
/** If we've saved columns from previously blocks we could filter all rows from current block
* which are unnecessary for sortBlock(...) because they obviously won't be in the top LIMIT rows.
*/
if (!threshold_block_columns.empty())
if (!sort_description_threshold_columns.empty())
{
UInt64 rows_num = block.rows();
auto block_columns = extractColumns(block, description);
auto block_columns = extractRawColumns(block, description_with_positions);
size_t result_size_hint = getFilterMask(
block_columns, threshold_block_columns, limit - 1,
block_columns, sort_description_threshold_columns,
description, rows_num, filter, rows_to_compare, compare_results);
/// Everything was filtered. Skip whole chunk.
@ -127,16 +139,34 @@ void PartialSortingTransform::transform(Chunk & chunk)
sortBlock(block, description, limit);
/// Check if we can use this block for optimization.
if (min_limit_for_partial_sort_optimization <= limit && limit <= block.rows())
{
auto block_columns = extractColumns(block, description);
size_t block_rows_after_filter = block.rows();
if (threshold_block_columns.empty() ||
less(block_columns, limit - 1, threshold_block_columns, limit - 1, description))
/// Check if we can use this block for optimization.
if (min_limit_for_partial_sort_optimization <= limit && block_rows_after_filter > 0 && limit <= block_rows_before_filter)
{
/** If we filtered more than limit rows from block take block last row.
* Otherwise take last limit row.
*
* If current threshold value is empty, update current threshold value.
* If min block value is less than current threshold value, update current threshold value.
*/
size_t min_row_to_compare = limit <= block_rows_after_filter ? (limit - 1) : (block_rows_after_filter - 1);
auto raw_block_columns = extractRawColumns(block, description_with_positions);
if (sort_description_threshold_columns.empty() ||
compareWithThreshold(raw_block_columns, min_row_to_compare, sort_description_threshold_columns, description))
{
threshold_block = block;
threshold_block_columns.swap(block_columns);
size_t raw_block_columns_size = raw_block_columns.size();
Columns sort_description_threshold_columns_updated(raw_block_columns_size);
for (size_t i = 0; i < raw_block_columns_size; ++i)
{
MutableColumnPtr sort_description_threshold_column_updated = raw_block_columns[i]->cloneEmpty();
sort_description_threshold_column_updated->insertFrom(*raw_block_columns[i], min_row_to_compare);
sort_description_threshold_columns_updated[i] = std::move(sort_description_threshold_column_updated);
}
sort_description_threshold_columns = std::move(sort_description_threshold_columns_updated);
}
}

View File

@ -8,7 +8,6 @@ namespace DB
{
/** Sorts each block individually by the values of the specified columns.
* At the moment, not very optimal algorithm is used.
*/
class PartialSortingTransform : public ISimpleTransform
{
@ -28,15 +27,11 @@ protected:
private:
SortDescription description;
SortDescriptionWithPositions description_with_positions;
UInt64 limit;
RowsBeforeLimitCounterPtr read_rows;
/** threshold_block is using for saving columns from previously processed block.
* threshold_block_columns contains pointers to columns from threshold_block which used for comparison.
* That's all for PartialSort optimization
*/
Block threshold_block;
ColumnRawPtrs threshold_block_columns;
Columns sort_description_threshold_columns;
/// This are just buffers which reserve memory to reduce the number of allocations.
PaddedPODArray<UInt64> rows_to_compare;

View File

@ -0,0 +1,14 @@
#pragma once
#include <Processors/Chunk.h>
#include <Common/PODArray.h>
namespace DB
{
/// ChunkInfo with IColumn::Selector. It is added by AddingSelectorTransform.
struct SelectorInfo : public ChunkInfo
{
IColumn::Selector selector;
};
}

View File

@ -28,10 +28,7 @@ void printPipeline(const Processors & processors, const Statuses & statuses, Wri
/// Nodes // TODO quoting and escaping
for (const auto & processor : processors)
{
auto description = processor->getDescription();
if (!description.empty())
description = ": " + description;
out << " n" << get_proc_id(*processor) << "[label=\"" << processor->getName() << description;
out << " n" << get_proc_id(*processor) << "[label=\"" << processor->getName() << processor->getDescription();
if (statuses_iter != statuses.end())
{

View File

@ -4438,10 +4438,7 @@ std::set<String> MergeTreeData::getPartitionIdsAffectedByCommands(
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorForInternalUsage(
const DataPartStates & affordable_states,
const DataPartsLock & /*lock*/,
DataPartStateVector * out_states,
bool require_projection_parts) const
const DataPartStates & affordable_states, const DataPartsLock & /*lock*/, DataPartStateVector * out_states) const
{
DataPartsVector res;
DataPartsVector buf;
@ -4449,86 +4446,89 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorForInternalUsage
for (auto state : affordable_states)
{
auto range = getDataPartsStateRange(state);
if (require_projection_parts)
{
for (const auto & part : range)
{
for (const auto & [_, projection_part] : part->getProjectionParts())
res.push_back(projection_part);
}
}
else
{
std::swap(buf, res);
res.clear();
std::merge(range.begin(), range.end(), buf.begin(), buf.end(), std::back_inserter(res), LessDataPart()); //-V783
}
std::swap(buf, res);
res.clear();
std::merge(range.begin(), range.end(), buf.begin(), buf.end(), std::back_inserter(res), LessDataPart()); //-V783
}
if (out_states != nullptr)
{
out_states->resize(res.size());
if (require_projection_parts)
{
for (size_t i = 0; i < res.size(); ++i)
(*out_states)[i] = res[i]->getParentPart()->getState();
}
else
{
for (size_t i = 0; i < res.size(); ++i)
(*out_states)[i] = res[i]->getState();
}
for (size_t i = 0; i < res.size(); ++i)
(*out_states)[i] = res[i]->getState();
}
return res;
}
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorForInternalUsage(
const DataPartStates & affordable_states,
DataPartStateVector * out_states,
bool require_projection_parts) const
MergeTreeData::DataPartsVector
MergeTreeData::getDataPartsVectorForInternalUsage(const DataPartStates & affordable_states, DataPartStateVector * out_states) const
{
auto lock = lockParts();
return getDataPartsVectorForInternalUsage(affordable_states, lock, out_states, require_projection_parts);
return getDataPartsVectorForInternalUsage(affordable_states, lock, out_states);
}
MergeTreeData::DataPartsVector
MergeTreeData::getAllDataPartsVector(MergeTreeData::DataPartStateVector * out_states, bool require_projection_parts) const
MergeTreeData::ProjectionPartsVector
MergeTreeData::getProjectionPartsVectorForInternalUsage(const DataPartStates & affordable_states, DataPartStateVector * out_states) const
{
auto lock = lockParts();
ProjectionPartsVector res;
for (auto state : affordable_states)
{
auto range = getDataPartsStateRange(state);
for (const auto & part : range)
{
res.data_parts.push_back(part);
for (const auto & [_, projection_part] : part->getProjectionParts())
res.projection_parts.push_back(projection_part);
}
}
if (out_states != nullptr)
{
out_states->resize(res.projection_parts.size());
for (size_t i = 0; i < res.projection_parts.size(); ++i)
(*out_states)[i] = res.projection_parts[i]->getParentPart()->getState();
}
return res;
}
MergeTreeData::DataPartsVector MergeTreeData::getAllDataPartsVector(MergeTreeData::DataPartStateVector * out_states) const
{
DataPartsVector res;
if (require_projection_parts)
auto lock = lockParts();
res.assign(data_parts_by_info.begin(), data_parts_by_info.end());
if (out_states != nullptr)
{
auto lock = lockParts();
for (const auto & part : data_parts_by_info)
{
for (const auto & [p_name, projection_part] : part->getProjectionParts())
res.push_back(projection_part);
}
if (out_states != nullptr)
{
out_states->resize(res.size());
for (size_t i = 0; i < res.size(); ++i)
(*out_states)[i] = res[i]->getParentPart()->getState();
}
}
else
{
auto lock = lockParts();
res.assign(data_parts_by_info.begin(), data_parts_by_info.end());
if (out_states != nullptr)
{
out_states->resize(res.size());
for (size_t i = 0; i < res.size(); ++i)
(*out_states)[i] = res[i]->getState();
}
out_states->resize(res.size());
for (size_t i = 0; i < res.size(); ++i)
(*out_states)[i] = res[i]->getState();
}
return res;
}
MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states) const
{
ProjectionPartsVector res;
auto lock = lockParts();
for (const auto & part : data_parts_by_info)
{
res.data_parts.push_back(part);
for (const auto & [p_name, projection_part] : part->getProjectionParts())
res.projection_parts.push_back(projection_part);
}
if (out_states != nullptr)
{
out_states->resize(res.projection_parts.size());
for (size_t i = 0; i < res.projection_parts.size(); ++i)
(*out_states)[i] = res.projection_parts[i]->getParentPart()->getState();
}
return res;
}
DetachedPartsInfo MergeTreeData::getDetachedParts() const
{
DetachedPartsInfo res;

View File

@ -449,26 +449,31 @@ public:
Int64 getMaxBlockNumber() const;
struct ProjectionPartsVector
{
DataPartsVector projection_parts;
DataPartsVector data_parts;
};
/// Returns a copy of the list so that the caller shouldn't worry about locks.
DataParts getDataParts(const DataPartStates & affordable_states) const;
DataPartsVector getDataPartsVectorForInternalUsage(
const DataPartStates & affordable_states,
const DataPartsLock & lock,
DataPartStateVector * out_states = nullptr,
bool require_projection_parts = false) const;
const DataPartStates & affordable_states, const DataPartsLock & lock, DataPartStateVector * out_states = nullptr) const;
/// Returns sorted list of the parts with specified states
/// out_states will contain snapshot of each part state
DataPartsVector getDataPartsVectorForInternalUsage(
const DataPartStates & affordable_states, DataPartStateVector * out_states = nullptr, bool require_projection_parts = false) const;
const DataPartStates & affordable_states, DataPartStateVector * out_states = nullptr) const;
/// Same as above but only returns projection parts
ProjectionPartsVector getProjectionPartsVectorForInternalUsage(
const DataPartStates & affordable_states, DataPartStateVector * out_states = nullptr) const;
/// Returns absolutely all parts (and snapshot of their states)
DataPartsVector getAllDataPartsVector(
DataPartStateVector * out_states = nullptr,
bool require_projection_parts = false) const;
DataPartsVector getAllDataPartsVector(DataPartStateVector * out_states = nullptr) const;
/// Same as above but only returns projection parts
ProjectionPartsVector getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states = nullptr) const;
/// Returns parts in Active state
DataParts getDataPartsForInternalUsage() const;

View File

@ -11,6 +11,12 @@
#include <emmintrin.h>
#endif
#if defined(__aarch64__) && defined(__ARM_NEON)
# include <arm_neon.h>
# ifdef HAS_RESERVED_IDENTIFIER
# pragma clang diagnostic ignored "-Wreserved-identifier"
# endif
#endif
namespace DB
{
@ -551,6 +557,34 @@ size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, con
return count;
}
}
#elif defined(__aarch64__) && defined(__ARM_NEON)
const uint8x16_t bitmask = {0x01, 0x02, 0x4, 0x8, 0x10, 0x20, 0x40, 0x80, 0x01, 0x02, 0x4, 0x8, 0x10, 0x20, 0x40, 0x80};
while (end - begin >= 64)
{
end -= 64;
const auto * src = reinterpret_cast<const unsigned char *>(end);
const uint8x16_t p0 = vceqzq_u8(vld1q_u8(src));
const uint8x16_t p1 = vceqzq_u8(vld1q_u8(src + 16));
const uint8x16_t p2 = vceqzq_u8(vld1q_u8(src + 32));
const uint8x16_t p3 = vceqzq_u8(vld1q_u8(src + 48));
uint8x16_t t0 = vandq_u8(p0, bitmask);
uint8x16_t t1 = vandq_u8(p1, bitmask);
uint8x16_t t2 = vandq_u8(p2, bitmask);
uint8x16_t t3 = vandq_u8(p3, bitmask);
uint8x16_t sum0 = vpaddq_u8(t0, t1);
uint8x16_t sum1 = vpaddq_u8(t2, t3);
sum0 = vpaddq_u8(sum0, sum1);
sum0 = vpaddq_u8(sum0, sum0);
UInt64 val = vgetq_lane_u64(vreinterpretq_u64_u8(sum0), 0);
val = ~val;
if (val == 0)
count += 64;
else
{
count += __builtin_clzll(val);
return count;
}
}
#endif
while (end > begin && *(--end) == 0)

View File

@ -48,9 +48,25 @@ bool StorageSystemPartsBase::hasStateColumn(const Names & column_names, const St
}
MergeTreeData::DataPartsVector
StoragesInfo::getParts(MergeTreeData::DataPartStateVector & state, bool has_state_column, bool require_projection_parts) const
StoragesInfo::getParts(MergeTreeData::DataPartStateVector & state, bool has_state_column) const
{
if (require_projection_parts && data->getInMemoryMetadataPtr()->projections.empty())
using State = MergeTreeData::DataPartState;
if (need_inactive_parts)
{
/// If has_state_column is requested, return all states.
if (!has_state_column)
return data->getDataPartsVectorForInternalUsage({State::Active, State::Outdated}, &state);
return data->getAllDataPartsVector(&state);
}
return data->getDataPartsVectorForInternalUsage({State::Active}, &state);
}
MergeTreeData::ProjectionPartsVector
StoragesInfo::getProjectionParts(MergeTreeData::DataPartStateVector & state, bool has_state_column) const
{
if (data->getInMemoryMetadataPtr()->projections.empty())
return {};
using State = MergeTreeData::DataPartState;
@ -58,12 +74,12 @@ StoragesInfo::getParts(MergeTreeData::DataPartStateVector & state, bool has_stat
{
/// If has_state_column is requested, return all states.
if (!has_state_column)
return data->getDataPartsVectorForInternalUsage({State::Active, State::Outdated}, &state, require_projection_parts);
return data->getProjectionPartsVectorForInternalUsage({State::Active, State::Outdated}, &state);
return data->getAllDataPartsVector(&state, require_projection_parts);
return data->getAllProjectionPartsVector(&state);
}
return data->getDataPartsVectorForInternalUsage({State::Active}, &state, require_projection_parts);
return data->getProjectionPartsVectorForInternalUsage({State::Active}, &state);
}
StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context)

View File

@ -23,8 +23,9 @@ struct StoragesInfo
MergeTreeData * data = nullptr;
explicit operator bool() const { return storage != nullptr; }
MergeTreeData::DataPartsVector
getParts(MergeTreeData::DataPartStateVector & state, bool has_state_column, bool require_projection_parts = false) const;
MergeTreeData::DataPartsVector getParts(MergeTreeData::DataPartStateVector & state, bool has_state_column) const;
MergeTreeData::ProjectionPartsVector getProjectionParts(MergeTreeData::DataPartStateVector & state, bool has_state_column) const;
};
/** A helper class that enumerates the storages that match given query. */

View File

@ -94,14 +94,13 @@ void StorageSystemProjectionParts::processNextStorage(
{
using State = IMergeTreeDataPart::State;
MergeTreeData::DataPartStateVector all_parts_state;
MergeTreeData::DataPartsVector all_parts;
all_parts = info.getParts(all_parts_state, has_state_column, true /* require_projection_parts */);
for (size_t part_number = 0; part_number < all_parts.size(); ++part_number)
MergeTreeData::ProjectionPartsVector all_parts = info.getProjectionParts(all_parts_state, has_state_column);
for (size_t part_number = 0; part_number < all_parts.projection_parts.size(); ++part_number)
{
const auto & part = all_parts[part_number];
const auto & part = all_parts.projection_parts[part_number];
const auto * parent_part = part->getParentPart();
chassert(parent_part);
auto part_state = all_parts_state[part_number];
ColumnSize columns_size = part->getTotalColumnsSize();

View File

@ -100,14 +100,15 @@ void StorageSystemProjectionPartsColumns::processNextStorage(
}
}
/// Go through the list of parts.
/// Go through the list of projection parts.
MergeTreeData::DataPartStateVector all_parts_state;
MergeTreeData::DataPartsVector all_parts;
all_parts = info.getParts(all_parts_state, has_state_column, true /* require_projection_parts */);
for (size_t part_number = 0; part_number < all_parts.size(); ++part_number)
MergeTreeData::ProjectionPartsVector all_parts = info.getProjectionParts(all_parts_state, has_state_column);
for (size_t part_number = 0; part_number < all_parts.projection_parts.size(); ++part_number)
{
const auto & part = all_parts[part_number];
const auto & part = all_parts.projection_parts[part_number];
const auto * parent_part = part->getParentPart();
chassert(parent_part);
auto part_state = all_parts_state[part_number];
auto columns_size = part->getTotalColumnsSize();
auto parent_columns_size = parent_part->getTotalColumnsSize();

View File

@ -37,7 +37,11 @@
</substitution>
</substitutions>
<create_query>CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple();</create_query>
<create_query>
CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec}))
ENGINE = MergeTree PARTITION BY tuple()
ORDER BY tuple();
</create_query>
<fill_query>INSERT INTO codec_seq_{type}_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</fill_query>
<fill_query>INSERT INTO codec_mon_{type}_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1</fill_query>

View File

@ -18,7 +18,6 @@
<value>collapsing_final_16p_str_keys_rnd</value>
<value>collapsing_final_1024p_ord</value>
<value>collapsing_final_1024p_rnd</value>
<value>collapsing_final_1p_ord</value>
</values>
</substitution>
</substitutions>
@ -31,7 +30,6 @@
<create_query>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_query>
<create_query>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_query>
<create_query>create table collapsing_final_1024p_rnd (key1 UInt32, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1) partition by key1 % 1024 </create_query>
<create_query>create table collapsing_final_1p_ord (key1 UInt64, key2 UInt64, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1, key2)</create_query>
<!-- 16 parts, 8192 * 1024 rows each -->
<fill_query>insert into collapsing_final_16p_ord select number, number, 1, number from numbers_mt(8388608) </fill_query>
@ -45,9 +43,6 @@
<fill_query>insert into collapsing_final_1024p_ord select number, 1, number from numbers_mt(16777216) </fill_query>
<fill_query>insert into collapsing_final_1024p_rnd select number, 1, number from numbers_mt(16777216) </fill_query>
<!-- 1 big part of 5e7 rows -->
<fill_query>insert into collapsing_final_1p_ord select number, number + 1, 1, number from numbers_mt(5e7)</fill_query>
<fill_query>optimize table {collapsing} final</fill_query>
<query>SELECT count() FROM {collapsing} final</query>

View File

@ -376,11 +376,6 @@
2019-05-03 4
2019-05-03 1
2019-05-03 -2
2019-05-01 10
2019-05-01 7
2019-05-01 4
2019-05-01 1
2019-05-01 -2
*** date WITH FILL TO 2019-06-23 STEP 3, val WITH FILL FROM -10 STEP 2
2019-05-07 -10
2019-05-07 -8

View File

@ -4,6 +4,6 @@
11965 11890
499500 499500 999
499500 499500 999
5858 11890
5857 11890
499500 499500 999
499500 499500 999

View File

@ -16,15 +16,8 @@ ExpressionTransform
ExpressionTransform × 2
(ReadFromMergeTree)
ExpressionTransform × 2
ReplacingSorted
ExpressionTransform
FilterSortedStreamByRange
Description: filter values in [(5), +inf)
ExpressionTransform
MergeTreeInOrder 0 → 1
ReplacingSorted 2 → 1
ExpressionTransform × 2
FilterSortedStreamByRange × 2
Description: filter values in [-inf, (5))
ExpressionTransform × 2
MergeTreeInOrder × 2 0 → 1
ReplacingSorted × 2 2 → 1
Copy × 2 1 → 2
AddingSelector × 2
ExpressionTransform × 2
MergeTreeInOrder × 2 0 → 1

View File

@ -34,12 +34,3 @@
6 -4
6 -3
6 -2
7 -10
7 -9
7 -8
7 -7
7 -6
7 -5
7 -4
7 -3
7 -2

View File

@ -103,10 +103,6 @@
2020-04-01 2 0
2020-04-01 3 0
2020-04-01 4 0
2020-05-01 1 0
2020-05-01 2 0
2020-05-01 3 0
2020-05-01 4 0
1970-01-04
1970-01-03
1970-01-02

View File

@ -1,9 +0,0 @@
2
2
3
5
8
8
8
8
8

View File

@ -1,31 +0,0 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
test_random_values() {
layers=$1
$CLICKHOUSE_CLIENT -n -q "
create table tbl_8parts_${layers}granules_rnd (key1 UInt32, sign Int8) engine = CollapsingMergeTree(sign) order by (key1) partition by (key1 % 8);
insert into tbl_8parts_${layers}granules_rnd select number, 1 from numbers_mt($((layers * 8 * 8192)));
explain pipeline select * from tbl_8parts_${layers}granules_rnd final settings max_threads = 16;" 2>&1 |
grep -c "CollapsingSortedTransform"
}
for layers in 2 3 5 8; do
test_random_values $layers
done;
test_sequential_values() {
layers=$1
$CLICKHOUSE_CLIENT -n -q "
create table tbl_8parts_${layers}granules_seq (key1 UInt32, sign Int8) engine = CollapsingMergeTree(sign) order by (key1) partition by (key1 / $((layers * 8192)))::UInt64;
insert into tbl_8parts_${layers}granules_seq select number, 1 from numbers_mt($((layers * 8 * 8192)));
explain pipeline select * from tbl_8parts_${layers}granules_seq final settings max_threads = 8;" 2>&1 |
grep -c "CollapsingSortedTransform"
}
for layers in 2 3 5 8 16; do
test_sequential_values $layers
done;

View File

@ -0,0 +1,4 @@
1 1 1 1
1 1 1 1
1 1 1 1
1 1 1 1

View File

@ -0,0 +1,39 @@
-- Before UNIX epoch
WITH
toDateTime64('1959-09-16 19:20:12.999999998', 9, 'UTC') AS dt1,
toDateTime64('1959-09-16 19:20:12.999999999', 9, 'UTC') AS dt2
SELECT
dt1 < dt2,
(dt1 + INTERVAL 1 NANOSECOND) = dt2,
(dt1 + INTERVAL 2 NANOSECOND) > dt2,
(dt1 + INTERVAL 3 NANOSECOND) > dt2;
-- At UNIX epoch border
WITH
toDateTime64('1969-12-31 23:59:59.999999998', 9, 'UTC') AS dt1,
toDateTime64('1969-12-31 23:59:59.999999999', 9, 'UTC') AS dt2
SELECT
dt1 < dt2,
(dt1 + INTERVAL 1 NANOSECOND) = dt2,
(dt1 + INTERVAL 2 NANOSECOND) > dt2,
(dt1 + INTERVAL 3 NANOSECOND) > dt2;
-- After UNIX epoch
WITH
toDateTime64('2001-12-31 23:59:59.999999998', 9, 'UTC') AS dt1,
toDateTime64('2001-12-31 23:59:59.999999999', 9, 'UTC') AS dt2
SELECT
dt1 < dt2,
(dt1 + INTERVAL 1 NANOSECOND) = dt2,
(dt1 + INTERVAL 2 NANOSECOND) > dt2,
(dt1 + INTERVAL 3 NANOSECOND) > dt2;
-- At upper DT64 bound (DT64 precision is lower here by design)
WITH
toDateTime64('2282-12-31 23:59:59.999998', 6, 'UTC') AS dt1,
toDateTime64('2282-12-31 23:59:59.999999', 6, 'UTC') AS dt2
SELECT
dt1 < dt2,
(dt1 + INTERVAL 1 MICROSECOND) = dt2,
(dt1 + INTERVAL 2 MICROSECOND) > dt2,
(dt1 + INTERVAL 3 MICROSECOND) > dt2;

View File

@ -5,7 +5,7 @@
ROOT_PATH=$(git rev-parse --show-toplevel)
codespell \
--skip "*generated*,*gperf*,*.bin,*.mrk*,*.idx,checksums.txt,*.dat,*.pyc,*.kate-swp,*obfuscateQueries.cpp,${ROOT_PATH}/utils/check-style/aspell-ignore" \
--skip "*generated*,*gperf*,*.bin,*.mrk*,*.idx,checksums.txt,*.dat,*.pyc,*.kate-swp,*obfuscateQueries.cpp,d3-*.js,*.min.js,${ROOT_PATH}/utils/check-style/aspell-ignore" \
--ignore-words "${ROOT_PATH}/utils/check-style/codespell-ignore-words.list" \
--exclude-file "${ROOT_PATH}/utils/check-style/codespell-ignore-lines.list" \
--quiet-level 2 \

View File

@ -8,6 +8,7 @@
#include <string.h>
#include <errno.h>
#include <memory>
#include <iostream>
#include "types.h"
@ -54,7 +55,7 @@ int doCompress(char * input, char * output, off_t & in_offset, off_t & out_offse
size_t compressed_size = ZSTD_compress2(cctx, output + out_offset, output_size, input + in_offset, input_size);
if (ZSTD_isError(compressed_size))
{
fprintf(stderr, "Error (ZSTD): %zu %s\n", compressed_size, ZSTD_getErrorName(compressed_size));
std::cerr << "Error (ZSTD): " << compressed_size << " " << ZSTD_getErrorName(compressed_size) << std::endl;
return 1;
}
in_offset += input_size;
@ -79,7 +80,7 @@ int compress(int in_fd, int out_fd, int level, off_t & pointer, const struct sta
ZSTD_CCtx * cctx = ZSTD_createCCtx();
if (cctx == nullptr)
{
fprintf(stderr, "Error (ZSTD): failed to create compression context\n");
std::cerr << "Error (ZSTD): failed to create compression context" << std::endl;
return 1;
}
@ -89,13 +90,13 @@ int compress(int in_fd, int out_fd, int level, off_t & pointer, const struct sta
check_result = ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, level);
if (ZSTD_isError(check_result))
{
fprintf(stderr, "Error (ZSTD): %zu %s\n", check_result, ZSTD_getErrorName(check_result));
std::cerr << "Error (ZSTD): " << check_result << " " << ZSTD_getErrorName(check_result) << std::endl;
return 1;
}
check_result = ZSTD_CCtx_setParameter(cctx, ZSTD_c_checksumFlag, 1);
if (ZSTD_isError(check_result))
{
fprintf(stderr, "Error (ZSTD): %zu %s\n", check_result, ZSTD_getErrorName(check_result));
std::cerr << "Error (ZSTD): " << check_result << " " << ZSTD_getErrorName(check_result) << std::endl;
return 1;
}
@ -254,7 +255,7 @@ int compressFiles(char* filenames[], int count, int output_fd, int level, const
continue;
}
printf("Size: %td\n", info_in.st_size);
std::cout << "Size: " << info_in.st_size << std::endl;
/// Save umask
files_data[i].umask = info_in.st_mode;
@ -317,7 +318,7 @@ int copy_decompressor(const char *self, int output_fd)
if (sz < 0)
perror(nullptr);
else
fprintf(stderr, "Error: unable to extract decompressor.\n");
std::cerr << "Error: unable to extract decompressor" << std::endl;
close(input_fd);
return 1;
}
@ -398,7 +399,7 @@ int main(int argc, char* argv[])
struct stat info_out;
if (stat(argv[start_of_files], &info_out) != -1 || errno != ENOENT)
{
fprintf(stderr, "Error: output file [%s] already exists.\n", argv[start_of_files]);
std::cerr << "Error: output file [" << argv[start_of_files] << "] already exists" << std::endl;
return 1;
}
@ -419,7 +420,7 @@ int main(int argc, char* argv[])
return 1;
}
printf("Compression with level %d\n", level);
std::cout << "Compression with level: " << level << std::endl;
if (0 != compressFiles(&argv[start_of_files], argc - start_of_files, output_fd, level, info_out))
{
printf("Compression failed.\n");

View File

@ -11,6 +11,7 @@
#include <stdlib.h>
#include <stdio.h>
#include <string.h>
#include <iostream>
#include "types.h"
@ -21,7 +22,7 @@ int doDecompress(char * input, char * output, off_t & in_offset, off_t & out_off
size_t decompressed_size = ZSTD_decompressDCtx(dctx, output + out_offset, output_size, input + in_offset, input_size);
if (ZSTD_isError(decompressed_size))
{
fprintf(stderr, "Error (ZSTD): %zu %s\n", decompressed_size, ZSTD_getErrorName(decompressed_size));
std::cerr << "Error (ZSTD):" << decompressed_size << " " << ZSTD_getErrorName(decompressed_size) << std::endl;
return 1;
}
return 0;
@ -40,7 +41,7 @@ int decompress(char * input, char * output, off_t start, off_t end, size_t max_n
ZSTD_DCtx * dctx = ZSTD_createDCtx();
if (dctx == nullptr)
{
fprintf(stderr, "Error (ZSTD): failed to create decompression context\n");
std::cerr << "Error (ZSTD): failed to create decompression context" << std::endl;
return 1;
}
pid_t pid;
@ -52,7 +53,7 @@ int decompress(char * input, char * output, off_t start, off_t end, size_t max_n
size = ZSTD_findFrameCompressedSize(input + in_pointer, max_block_size);
if (ZSTD_isError(size))
{
fprintf(stderr, "Error (ZSTD): %td %s\n", size, ZSTD_getErrorName(size));
std::cerr << "Error (ZSTD): " << size << " " << ZSTD_getErrorName(size) << std::endl;
error_happened = true;
break;
}
@ -60,7 +61,7 @@ int decompress(char * input, char * output, off_t start, off_t end, size_t max_n
decompressed_size = ZSTD_getFrameContentSize(input + in_pointer, max_block_size);
if (ZSTD_isError(decompressed_size))
{
fprintf(stderr, "Error (ZSTD): %td %s\n", decompressed_size, ZSTD_getErrorName(decompressed_size));
std::cerr << "Error (ZSTD): " << decompressed_size << " " << ZSTD_getErrorName(decompressed_size) << std::endl;
error_happened = true;
break;
}
@ -171,8 +172,7 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress
}
if (fs_info.f_blocks * info_in.st_blksize < decompressed_full_size)
{
fprintf(stderr, "Not enough space for decompression. Have %tu, need %zu.",
fs_info.f_blocks * info_in.st_blksize, decompressed_full_size);
std::cerr << "Not enough space for decompression. Have " << fs_info.f_blocks * info_in.st_blksize << ", need " << decompressed_full_size << std::endl;
return 1;
}

View File

@ -0,0 +1,36 @@
Trace visualizer is a tool for representation of a tracing data as a Gantt diagram.
# Quick start
For now this tool is not integrated into ClickHouse and requires a lot of manual adjustments.
```bash
cd utils/trace-visualizer
python3 -m http.server
```
Open [localhost](http://localhost:8000). It will show an example of data. To show your tracing data you have to put it in JSON format near `index.html` and change call to `fetchData()` function at the bottom of `index.html`. (Do not forget to disable browser caching while changing it).
# Visualizing query trace
First of all [opentelemetry_span_log](https://clickhouse.com/docs/en/operations/opentelemetry/) system table must be enabled to save query traces. Then run a query you want to trace with a setting:
```sql
set opentelemetry_start_trace_probability=1;
SELECT 1;
```
To find out `trace_id` of a query run the following command:
```sql
SELECT DISTINCT trace_id FROM system.opentelemetry_span_log ORDER BY query_start_time DESC;
```
To obtain JSON data suitable for visualizing run:
```sql
SELECT tuple (parent_span_id, attribute['clickhouse.thread_id'] || attribute['thread_number'] as thread_id)::Tuple(parent_span_id UInt64, thread_id String) as group, operation_name, start_time_us, finish_time_us, sipHash64(operation_name) as color, attribute
from system.opentelemetry_span_log
WHERE trace_id = 'your-trace-id'
ORDER BY group ASC
FORMAT JSON SETTINGS output_format_json_named_tuples_as_objects = 1;
```
# Dependencies
1. [D3js](https://github.com/d3/d3) (v4).
2. [Tooltips for D3](https://github.com/caged/d3-tip).
3. [jquery](https://github.com/jquery/jquery).
4. [Bootstrap](https://github.com/twbs/bootstrap).

File diff suppressed because one or more lines are too long

View File

@ -0,0 +1,78 @@
/*
* d3-gantt.css by @serxa
*/
.chart {
font-family: Arial, sans-serif;
font-size: 12px;
}
rect.zoom-panel {
/*cursor: ew-resize;*/
fill: none;
pointer-events: all;
}
.axis path,.axis line {
fill: none;
stroke: #000;
shape-rendering: crispEdges;
}
.axis.y {
font-size: 16px;
cursor: ns-resize;
}
.axis.x {
font-size: 16px;
}
#ruler {
text-anchor: middle;
alignment-baseline: before-edge;
font-size: 16px;
font-family: sans-serif;
pointer-events: none;
}
.d3-tip {
line-height: 1;
font-weight: bold;
padding: 12px;
background: rgba(0, 0, 0, 0.8);
color: #fff;
border-radius: 2px;
}
.d3-tip pre {
font-weight: bold;
padding: 12px;
background: rgba(0, 0, 0, 0);
color: #fff;
border: 0px;
}
/* Style northward tooltips differently */
.d3-tip.n:after {
margin: -1px 0 0 0;
top: 100%;
left: 0;
}
/* for arrowhead marker */
#arrow {
stroke-width:1;
stroke-dasharray:0;
}
.bar:hover {
stroke-width: 1px;
stroke: black;
}
#errmsg {
width: 95vw;
margin: 0 auto;
padding: 10px;
}

View File

@ -0,0 +1,118 @@
<!DOCTYPE html>
<html lang="en">
<head>
<meta charset="UTF-8">
<meta http-equiv="X-UA-Compatible" content="IE=edge">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>Trace Gantt</title>
<link rel="stylesheet" href="css/bootstrap.min.css">
<link rel="stylesheet" href="css/d3-gantt.css">
</head>
<body>
<script language="javascript" type="text/javascript" src="js/jquery.min.js"></script>
<script language="javascript" type="text/javascript" src="js/bootstrap.min.js"></script>
<script language="javascript" type="text/javascript" src="js/d3.v4.min.js"></script>
<script language="javascript" type="text/javascript" src="js/d3-tip-0.8.0-alpha.1.js"></script>
<script language="javascript" type="text/javascript" src="js/d3-gantt.js"></script>
<div id="placeholder" class="chart-placeholder"></div>
</body>
</html>
<script language="javascript">
var example_json = [
{ t1: 100, t2: 200, band: "band1", color: "#888", text: "text1" },
{ t1: 300, t2: 400, band: "band2", color: "#ff8", text: "text2" },
{ t1: 100, t2: 400, band: "band3", color: "#888", text: "some very long text with a lot of letters in it" },
{ t1: 300, t2: 400, band: "band1", color: "#8ff", text: "some_very_long_identifier_with_a_lot_of_letters_in_it" },
{ t1: 500, t2: 800, band: "band2", color: "#f8f", text: "test\nif\nnew\nline\nworks\nhere?" }
];
var chart = d3.gantt()
.height(window.innerHeight - $("#placeholder")[0].getBoundingClientRect().y - window.scrollY)
.selector("#placeholder");
var data = null;
// Error message popup
$("<div id='errmsg'></div>").css({
position: "absolute",
display: "none",
border: "1px solid #faa",
padding: "2px",
"background-color": "#fcc",
opacity: 0.80
}).appendTo("body");
function fetchData(dataurl, parser = x => x) {
function onDataReceived(json, textStatus, xhr) {
$("#errmsg").hide();
data = parser(json);
chart(data);
}
function onDataError(xhr, error) {
console.log(arguments);
$("#errmsg").text("Fetch data error: " + error + (xhr.status == 200? xhr.responseText: ""))
.css({bottom: "5px", left: "25%", width: "50%"})
.fadeIn(200);
}
if (dataurl) {
$.ajax({
url: dataurl,
type: "GET",
dataType: "json",
success: function (json, textStatus, xhr) { onDataReceived(json, textStatus, xhr); },
error: onDataError
});
} else {
onDataReceived(example_json, "textStatus", "xhr");
}
}
function parseClickHouseTrace(json) {
let min_time_us = Number.MAX_VALUE;
for (let i = 0; i < json.data.length; i++) {
let span = json.data[i];
min_time_us = Math.min(min_time_us, +span.start_time_us);
}
let max_time_ms = 0;
function convertTime(us) {
let value = (us - min_time_us) / 1000;
max_time_ms = Math.max(max_time_ms, value);
return value;
}
function strHash(str) {
var hash = 0;
if (str.length === 0)
return hash;
for (let i = 0; i < str.length; i++) {
hash = ((hash << 5) - hash) + str.charCodeAt(i);
hash |= 0; // Convert to 32bit integer
}
if (hash < 0)
hash = -hash;
return hash;
}
let result = [];
for (let i = 0; i < json.data.length; i++) {
let span = json.data[i];
result.push({
t1: convertTime(+span.start_time_us),
t2: convertTime(+span.finish_time_us),
band: Object.values(span.group).join(' '),
color: d3.interpolateRainbow((strHash(span.color) % 256) / 256),
text: span.operation_name
});
}
chart.timeDomain([0, max_time_ms]);
return result;
}
fetchData(); // do not fetch, just draw example_json w/o parsing
//fetchData("your-traces.json" , parseClickHouseTrace);
</script>

File diff suppressed because one or more lines are too long

485
utils/trace-visualizer/js/d3-gantt.js vendored Normal file
View File

@ -0,0 +1,485 @@
/*
* d3-gantt.js by @serxa
* Based on https://github.com/ydb-platform/ydb/blob/stable-22-2/library/cpp/lwtrace/mon/static/js/d3-gantt.js
*/
d3.gantt = function() {
function gantt(input_data) {
data = input_data;
initAxis();
// create svg element
svg = d3.select(selector)
.append("svg")
.attr("class", "chart")
.attr("width", width + margin.left + margin.right)
.attr("height", height + margin.top + margin.bottom)
;
// create arrowhead marker
defs = svg.append("defs");
defs.append("marker")
.attr("id", "arrow")
.attr("viewBox", "0 -5 10 10")
.attr("refX", 5)
.attr("refY", 0)
.attr("markerWidth", 4)
.attr("markerHeight", 4)
.attr("orient", "auto")
.append("path")
.attr("d", "M0,-5L10,0L0,5")
.attr("class","arrowHead")
;
zoom = d3.zoom()
.scaleExtent([0.1, 1000])
//.translateExtent([0, 0], [1000,0])
.on("zoom", function() {
if (tipShown != null) {
tip.hide(tipShown);
}
var tr = d3.event.transform;
xZoomed = tr.rescaleX(x);
svg.select("g.x.axis").call(xAxis.scale(xZoomed));
var dy = d3.event.sourceEvent.screenY - zoom.startScreenY;
var newScrollTop = documentBodyScrollTop() - dy;
window.scrollTo(documentBodyScrollLeft(), newScrollTop);
documentBodyScrollTop(newScrollTop);
zoom.startScreenY = d3.event.sourceEvent.screenY;
zoomContainer1.attr("transform", "translate(" + tr.x + ",0) scale(" + tr.k + ",1)");
zoomContainer2.attr("transform", "translate(" + tr.x + ",0) scale(" + tr.k + ",1)");
render();
})
.on("start", function() {
zoom.startScreenY = d3.event.sourceEvent.screenY;
})
.on("end", function() {
})
;
svgChartContainer = svg.append('g')
.attr("transform", "translate(" + margin.left + ", " + margin.top + ")")
;
svgChart = svgChartContainer.append("svg")
.attr("top", 0)
.attr("left", 0)
.attr("width", width)
.attr("height", height)
.attr("viewBox", "0 0 " + width + " " + height)
;
zoomContainer1 = svgChart.append("g");
zoomPanel = svgChart.append("rect")
.attr("class", "zoom-panel")
.attr("width", width)
.attr("height", height)
.call(zoom)
;
zoomContainer2 = svgChart.append("g");
bandsSvg = zoomContainer2.append("g");
// tooltips for bands
var maxTipHeight = 130;
const tipDirection = d => y(d.band) - maxTipHeight < documentBodyScrollTop()? 's': 'n';
tip = d3.tip()
.attr("class", "d3-tip")
.offset(function(d) {
// compute x to return tip in chart region
var t0 = (d.t1 + d.t2) / 2;
var t1 = Math.min(Math.max(t0, xZoomed.invert(0)), xZoomed.invert(width));
var dir = tipDirection(d);
return [dir === 'n'? -10 : 10, xZoomed(t1) - xZoomed(t0)];
})
.direction(tipDirection)
.html(d => "<pre>" + d.text + "</pre>")
;
bandsSvg.call(tip);
render();
// container for non-zoomable elements
fixedContainer = svg.append("g")
.attr("transform", "translate(" + margin.left + ", " + margin.top + ")")
;
// create x axis
fixedContainer.append("g")
.attr("class", "x axis")
.attr("transform", "translate(0, " + (height - margin.top - margin.bottom) + ")")
.transition()
.call(xAxis)
;
// create y axis
fixedContainer.append("g")
.attr("class", "y axis")
.transition()
.call(yAxis)
;
// make y axis ticks draggable
var ytickdrag = d3.drag()
.on("drag", function(d) {
var ypos = d3.event.y - margin.top;
var index = Math.floor((ypos / y.step()));
index = Math.min(Math.max(index, 0), this.initDomain.length - 1);
if (index != this.curIndex) {
var newDomain = [];
for (var i = 0; i < this.initDomain.length; ++i) {
newDomain.push(this.initDomain[i]);
}
newDomain.splice(this.initIndex, 1);
newDomain.splice(index, 0, this.initDomain[this.initIndex]);
this.curIndex = index;
this.curDomain = newDomain;
y.domain(newDomain);
// rearange y scale and axis
svg.select("g.y.axis").transition().call(yAxis);
// rearange other stuff
render(-1, true);
}
})
.on("start", function(d) {
var ypos = d3.event.y - margin.top;
this.initIndex = Math.floor((ypos / y.step()));
this.initDomain = y.domain();
})
.on("end", function(d) {
svg.select("g.y.axis").call(yAxis);
})
;
svg.selectAll("g.y.axis .tick")
.call(ytickdrag)
;
// right margin
var rmargin = fixedContainer.append("g")
.attr("id", "right-margin")
.attr("transform", "translate(" + width + ", 0)")
;
rmargin.append("rect")
.attr("x", 0)
.attr("y", 0)
.attr("width", 1)
.attr("height", height - margin.top - margin.bottom)
;
// top margin
var tmargin = fixedContainer.append("g")
.attr("id", "top-margin")
.attr("transform", "translate(0, 0)")
;
tmargin.append("rect")
.attr("x", 0)
.attr("y", 0)
.attr("width", width)
.attr("height", 1)
;
// ruler
ruler = fixedContainer.append("g")
.attr("id", "ruler")
.attr("transform", "translate(0, 0)")
;
ruler.append("rect")
.attr("id", "ruler-line")
.attr("x", 0)
.attr("y", 0)
.attr("width", "1")
.attr("height", height - margin.top - margin.bottom + 8)
;
ruler.append("rect")
.attr("id", "bgrect")
.attr("x", 0)
.attr("y", 0)
.attr("width", 0)
.attr("height", 0)
.style("fill", "white")
;
ruler.append("text")
.attr("x", 0)
.attr("y", height - margin.top - margin.bottom + 16)
.attr("dy", "0.71em")
.text("0")
;
svg.on('mousemove', function() {
positionRuler(d3.event.pageX);
});
// scroll handling
window.onscroll = function myFunction() {
documentBodyScrollLeft(document.body.scrollLeft);
documentBodyScrollTop(document.body.scrollTop);
var scroll = scrollParams();
svgChartContainer
.attr("transform", "translate(" + margin.left
+ ", " + (margin.top + scroll.y1) + ")");
svgChart
.attr("viewBox", "0 " + scroll.y1 + " " + width + " " + scroll.h)
.attr("height", scroll.h);
tmargin
.attr("transform", "translate(0," + scroll.y1 + ")");
fixedContainer.select(".x.axis")
.attr("transform", "translate(0," + scroll.y2 + ")");
rmargin.select("rect")
.attr("y", scroll.y1)
.attr("height", scroll.h);
ruler.select("#ruler-line")
.attr("y", scroll.y1)
.attr("height", scroll.h);
positionRuler();
}
// render axis
svg.select("g.x.axis").call(xAxis);
svg.select("g.y.axis").call(yAxis);
// update to initiale state
window.onscroll(0);
return gantt;
}
// private:
var keyFunction = function(d) {
return d.t1.toString() + d.t2.toString() + d.band.toString();
}
var bandTransform = function(d) {
return "translate(" + x(d.t1) + "," + y(d.band) + ")";
}
var xPixel = function(d) {
return xZoomed.invert(1) - xZoomed.invert(0);
}
var render = function(t0, smooth) {
// Save/restore last t0 value
if (!arguments.length || t0 == -1) {
t0 = render.t0;
}
render.t0 = t0;
smooth = smooth || false;
// Create rectangles for bands
bands = bandsSvg.selectAll("rect.bar")
.data(data, keyFunction);
bands.exit().remove();
bands.enter().append("rect")
.attr("class", "bar")
.attr("vector-effect", "non-scaling-stroke")
.style("fill", d => d.color)
.on('click', function(d) {
if (tipShown != d) {
tipShown = d;
tip.show(d);
} else {
tipShown = null;
tip.hide(d);
}
})
.merge(bands)
.transition().duration(smooth? 250: 0)
.attr("y", 0)
.attr("transform", bandTransform)
.attr("height", y.bandwidth())
.attr("width", d => Math.max(1*xPixel(), x(d.t2) - x(d.t1)))
;
var emptyMarker = bandsSvg.selectAll("text")
.data(data.length == 0? ["no data to show"]: []);
emptyMarker.exit().remove();
emptyMarker.enter().append("text")
.text(d => d)
;
}
function initAxis() {
x = d3.scaleLinear()
.domain([timeDomainStart, timeDomainEnd])
.range([0, width])
//.clamp(true); // dosn't work with zoom/pan
xZoomed = x;
y = d3.scaleBand()
.domain(Object.values(data).map(d => d.band).sort())
.rangeRound([0, height - margin.top - margin.bottom])
.padding(0.5);
xAxis = d3.axisBottom()
.scale(x)
//.tickSubdivide(true)
.tickSize(8)
.tickPadding(8);
yAxis = d3.axisLeft()
.scale(y)
.tickSize(0);
}
// slow function wrapper
var documentBodyScrollLeft = function(value) {
if (!arguments.length) {
if (documentBodyScrollLeft.value === undefined) {
documentBodyScrollLeft.value = document.body.scrollLeft;
}
return documentBodyScrollLeft.value;
} else {
documentBodyScrollLeft.value = value;
}
}
// slow function wrapper
var documentBodyScrollTop = function(value) {
if (!arguments.length) {
if (!documentBodyScrollTop.value === undefined) {
documentBodyScrollTop.value = document.body.scrollTop;
}
return documentBodyScrollTop.value;
} else {
documentBodyScrollTop.value = value;
}
}
var scrollParams = function() {
var y1 = documentBodyScrollTop();
var y2 = y1 + window.innerHeight - margin.footer;
y2 = Math.min(y2, height - margin.top - margin.bottom);
var h = y2 - y1;
return {
y1: y1,
y2: y2,
h: h
};
}
var posTextFormat = d3.format(".1f");
var positionRuler = function(pageX) {
if (!arguments.length) {
pageX = positionRuler.pageX || 0;
} else {
positionRuler.pageX = pageX;
}
// x-coordinate
if (!positionRuler.svgLeft) {
positionRuler.svgLeft = svg.node().getBoundingClientRect().x;
}
var xpos = pageX - margin.left + 1 - positionRuler.svgLeft;
var tpos = xZoomed.invert(xpos);
tpos = Math.min(Math.max(tpos, xZoomed.invert(0)), xZoomed.invert(width));
ruler.attr("transform", "translate(" + xZoomed(tpos) + ", 0)");
var posText = posTextFormat(tpos);
// scroll-related
var scroll = scrollParams();
var text = ruler.select("text")
.attr("y", scroll.y2 + 16)
;
// getBBox() is very slow, so compute symbol width once
var xpadding = 5;
var ypadding = 5;
if (!positionRuler.bbox) {
positionRuler.bbox = text.node().getBBox();
}
text.text(posText);
var textWidth = 10 * posText.length;
ruler.select("#bgrect")
.attr("x", -textWidth/2 - xpadding)
.attr("y", positionRuler.bbox.y - ypadding)
.attr("width", textWidth + (xpadding*2))
.attr("height", positionRuler.bbox.height + (ypadding*2))
;
render(tpos);
}
// public:
gantt.width = function(value) {
if (!arguments.length)
return width;
width = +value;
return gantt;
}
gantt.height = function(value) {
if (!arguments.length)
return height;
height = +value;
return gantt;
}
gantt.selector = function(value) {
if (!arguments.length)
return selector;
selector = value;
return gantt;
}
gantt.timeDomain = function(value) {
if (!arguments.length)
return [timeDomainStart, timeDomainEnd];
timeDomainStart = value[0];
timeDomainEnd = value[1];
return gantt;
}
gantt.data = function() {
return data;
}
// constructor
// Config
var margin = { top: 20, right: 40, bottom: 20, left: 200, footer: 100 },
height = document.body.clientHeight - margin.top - margin.bottom - 5,
width = document.body.clientWidth - margin.right - margin.left - 5,
selector = 'body',
timeDomainStart = 0,
timeDomainEnd = 1000,
scales = {};
;
// View
var x = null,
xZoomed = null,
y = null,
xAxis = null,
yAxis = null,
svg = null,
defs = null,
svgChartContainer = null,
svgChart = null,
zoomPanel = null,
zoomContainer1 = null,
zoomContainer2 = null,
fixedContainer = null,
zoom = null,
bandsSvg = null,
bands = null,
tip = null,
tipShown = null,
ruler = null
;
// Model
var data = null;
return gantt;
}

View File

@ -0,0 +1,352 @@
/**
* d3.tip
* Copyright (c) 2013 Justin Palmer
*
* Tooltips for d3.js SVG visualizations
*/
// eslint-disable-next-line no-extra-semi
;(function(root, factory) {
if (typeof define === 'function' && define.amd) {
// AMD. Register as an anonymous module with d3 as a dependency.
define([
'd3-collection',
'd3-selection'
], factory)
} else if (typeof module === 'object' && module.exports) {
/* eslint-disable global-require */
// CommonJS
var d3Collection = require('d3-collection'),
d3Selection = require('d3-selection')
module.exports = factory(d3Collection, d3Selection)
/* eslint-enable global-require */
} else {
// Browser global.
var d3 = root.d3
// eslint-disable-next-line no-param-reassign
root.d3.tip = factory(d3, d3)
}
}(this, function(d3Collection, d3Selection) {
// Public - contructs a new tooltip
//
// Returns a tip
return function() {
var direction = d3TipDirection,
offset = d3TipOffset,
html = d3TipHTML,
rootElement = document.body,
node = initNode(),
svg = null,
point = null,
target = null
function tip(vis) {
svg = getSVGNode(vis)
if (!svg) return
point = svg.createSVGPoint()
rootElement.appendChild(node)
}
// Public - show the tooltip on the screen
//
// Returns a tip
tip.show = function() {
var args = Array.prototype.slice.call(arguments)
if (args[args.length - 1] instanceof SVGElement) target = args.pop()
var content = html.apply(this, args),
poffset = offset.apply(this, args),
dir = direction.apply(this, args),
nodel = getNodeEl(),
i = directions.length,
coords,
scrollTop = document.documentElement.scrollTop ||
rootElement.scrollTop,
scrollLeft = document.documentElement.scrollLeft ||
rootElement.scrollLeft
nodel.html(content)
.style('opacity', 1).style('pointer-events', 'all')
while (i--) nodel.classed(directions[i], false)
coords = directionCallbacks.get(dir).apply(this)
nodel.classed(dir, true)
.style('top', (coords.top + poffset[0]) + scrollTop + 'px')
.style('left', (coords.left + poffset[1]) + scrollLeft + 'px')
return tip
}
// Public - hide the tooltip
//
// Returns a tip
tip.hide = function() {
var nodel = getNodeEl()
nodel.style('opacity', 0).style('pointer-events', 'none')
return tip
}
// Public: Proxy attr calls to the d3 tip container.
// Sets or gets attribute value.
//
// n - name of the attribute
// v - value of the attribute
//
// Returns tip or attribute value
// eslint-disable-next-line no-unused-vars
tip.attr = function(n, v) {
if (arguments.length < 2 && typeof n === 'string') {
return getNodeEl().attr(n)
}
var args = Array.prototype.slice.call(arguments)
d3Selection.selection.prototype.attr.apply(getNodeEl(), args)
return tip
}
// Public: Proxy style calls to the d3 tip container.
// Sets or gets a style value.
//
// n - name of the property
// v - value of the property
//
// Returns tip or style property value
// eslint-disable-next-line no-unused-vars
tip.style = function(n, v) {
if (arguments.length < 2 && typeof n === 'string') {
return getNodeEl().style(n)
}
var args = Array.prototype.slice.call(arguments)
d3Selection.selection.prototype.style.apply(getNodeEl(), args)
return tip
}
// Public: Set or get the direction of the tooltip
//
// v - One of n(north), s(south), e(east), or w(west), nw(northwest),
// sw(southwest), ne(northeast) or se(southeast)
//
// Returns tip or direction
tip.direction = function(v) {
if (!arguments.length) return direction
direction = v == null ? v : functor(v)
return tip
}
// Public: Sets or gets the offset of the tip
//
// v - Array of [x, y] offset
//
// Returns offset or
tip.offset = function(v) {
if (!arguments.length) return offset
offset = v == null ? v : functor(v)
return tip
}
// Public: sets or gets the html value of the tooltip
//
// v - String value of the tip
//
// Returns html value or tip
tip.html = function(v) {
if (!arguments.length) return html
html = v == null ? v : functor(v)
return tip
}
// Public: sets or gets the root element anchor of the tooltip
//
// v - root element of the tooltip
//
// Returns root node of tip
tip.rootElement = function(v) {
if (!arguments.length) return rootElement
rootElement = v == null ? v : functor(v)
return tip
}
// Public: destroys the tooltip and removes it from the DOM
//
// Returns a tip
tip.destroy = function() {
if (node) {
getNodeEl().remove()
node = null
}
return tip
}
function d3TipDirection() { return 'n' }
function d3TipOffset() { return [0, 0] }
function d3TipHTML() { return ' ' }
var directionCallbacks = d3Collection.map({
n: directionNorth,
s: directionSouth,
e: directionEast,
w: directionWest,
nw: directionNorthWest,
ne: directionNorthEast,
sw: directionSouthWest,
se: directionSouthEast
}),
directions = directionCallbacks.keys()
function directionNorth() {
var bbox = getScreenBBox()
return {
top: bbox.n.y - node.offsetHeight,
left: bbox.n.x - node.offsetWidth / 2
}
}
function directionSouth() {
var bbox = getScreenBBox()
return {
top: bbox.s.y,
left: bbox.s.x - node.offsetWidth / 2
}
}
function directionEast() {
var bbox = getScreenBBox()
return {
top: bbox.e.y - node.offsetHeight / 2,
left: bbox.e.x
}
}
function directionWest() {
var bbox = getScreenBBox()
return {
top: bbox.w.y - node.offsetHeight / 2,
left: bbox.w.x - node.offsetWidth
}
}
function directionNorthWest() {
var bbox = getScreenBBox()
return {
top: bbox.nw.y - node.offsetHeight,
left: bbox.nw.x - node.offsetWidth
}
}
function directionNorthEast() {
var bbox = getScreenBBox()
return {
top: bbox.ne.y - node.offsetHeight,
left: bbox.ne.x
}
}
function directionSouthWest() {
var bbox = getScreenBBox()
return {
top: bbox.sw.y,
left: bbox.sw.x - node.offsetWidth
}
}
function directionSouthEast() {
var bbox = getScreenBBox()
return {
top: bbox.se.y,
left: bbox.se.x
}
}
function initNode() {
var div = d3Selection.select(document.createElement('div'))
div
.style('position', 'absolute')
.style('top', 0)
.style('opacity', 0)
.style('pointer-events', 'none')
.style('box-sizing', 'border-box')
return div.node()
}
function getSVGNode(element) {
var svgNode = element.node()
if (!svgNode) return null
if (svgNode.tagName.toLowerCase() === 'svg') return svgNode
return svgNode.ownerSVGElement
}
function getNodeEl() {
if (node == null) {
node = initNode()
// re-add node to DOM
rootElement.appendChild(node)
}
return d3Selection.select(node)
}
// Private - gets the screen coordinates of a shape
//
// Given a shape on the screen, will return an SVGPoint for the directions
// n(north), s(south), e(east), w(west), ne(northeast), se(southeast),
// nw(northwest), sw(southwest).
//
// +-+-+
// | |
// + +
// | |
// +-+-+
//
// Returns an Object {n, s, e, w, nw, sw, ne, se}
function getScreenBBox() {
var targetel = target || d3Selection.event.target
while (targetel.getScreenCTM == null && targetel.parentNode == null) {
targetel = targetel.parentNode
}
var bbox = {},
matrix = targetel.getScreenCTM(),
tbbox = targetel.getBBox(),
width = tbbox.width,
height = tbbox.height,
x = tbbox.x,
y = tbbox.y
point.x = x
point.y = y
bbox.nw = point.matrixTransform(matrix)
point.x += width
bbox.ne = point.matrixTransform(matrix)
point.y += height
bbox.se = point.matrixTransform(matrix)
point.x -= width
bbox.sw = point.matrixTransform(matrix)
point.y -= height / 2
bbox.w = point.matrixTransform(matrix)
point.x += width
bbox.e = point.matrixTransform(matrix)
point.x -= width / 2
point.y -= height / 2
bbox.n = point.matrixTransform(matrix)
point.y += height
bbox.s = point.matrixTransform(matrix)
return bbox
}
// Private - replace D3JS 3.X d3.functor() function
function functor(v) {
return typeof v === 'function' ? v : function() {
return v
}
}
return tip
}
// eslint-disable-next-line semi
}));

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long