Merge branch 'master' into fix_zero_copy_not_atomic

This commit is contained in:
alesapin 2023-04-29 21:26:26 +02:00
commit daa0b095ed
269 changed files with 7972 additions and 5009 deletions

3
.gitmodules vendored
View File

@ -335,3 +335,6 @@
[submodule "contrib/liburing"]
path = contrib/liburing
url = https://github.com/axboe/liburing
[submodule "contrib/isa-l"]
path = contrib/isa-l
url = https://github.com/ClickHouse/isa-l.git

View File

@ -1,4 +1,5 @@
### Table of Contents
**[ClickHouse release v23.4, 2023-04-26](#234)**<br/>
**[ClickHouse release v23.3 LTS, 2023-03-30](#233)**<br/>
**[ClickHouse release v23.2, 2023-02-23](#232)**<br/>
**[ClickHouse release v23.1, 2023-01-25](#231)**<br/>
@ -6,6 +7,153 @@
# 2023 Changelog
### <a id="234"></a> ClickHouse release 23.4, 2023-04-26
#### Backward Incompatible Change
* Formatter '%M' in function formatDateTime() now prints the month name instead of the minutes. This makes the behavior consistent with MySQL. The previous behavior can be restored using setting "formatdatetime_parsedatetime_m_is_month_name = 0". [#47246](https://github.com/ClickHouse/ClickHouse/pull/47246) ([Robert Schulze](https://github.com/rschu1ze)).
* This change makes sense only if you are using the virtual filesystem cache. If `path` in the virtual filesystem cache configuration is not empty and is not an absolute path, then it will be put in `<clickhouse server data directory>/caches/<path_from_cache_config>`. [#48784](https://github.com/ClickHouse/ClickHouse/pull/48784) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Primary/secondary indices and sorting keys with identical expressions are now rejected. This behavior can be disabled using setting `allow_suspicious_indices`. [#48536](https://github.com/ClickHouse/ClickHouse/pull/48536) ([凌涛](https://github.com/lingtaolf)).
#### New Feature
* Support new aggregate function `quantileGK`/`quantilesGK`, like [approx_percentile](https://spark.apache.org/docs/latest/api/sql/index.html#approx_percentile) in spark. Greenwald-Khanna algorithm refer to http://infolab.stanford.edu/~datar/courses/cs361a/papers/quantiles.pdf. [#46428](https://github.com/ClickHouse/ClickHouse/pull/46428) ([李扬](https://github.com/taiyang-li)).
* Add a statement `SHOW COLUMNS` which shows distilled information from system.columns. [#48017](https://github.com/ClickHouse/ClickHouse/pull/48017) ([Robert Schulze](https://github.com/rschu1ze)).
* Added `LIGHTWEIGHT` and `PULL` modifiers for `SYSTEM SYNC REPLICA` query. `LIGHTWEIGHT` version waits for fetches and drop-ranges only (merges and mutations are ignored). `PULL` version pulls new entries from ZooKeeper and does not wait for them. Fixes [#47794](https://github.com/ClickHouse/ClickHouse/issues/47794). [#48085](https://github.com/ClickHouse/ClickHouse/pull/48085) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Add `kafkaMurmurHash` function for compatibility with Kafka DefaultPartitioner. Closes [#47834](https://github.com/ClickHouse/ClickHouse/issues/47834). [#48185](https://github.com/ClickHouse/ClickHouse/pull/48185) ([Nikolay Degterinsky](https://github.com/evillique)).
* Allow to easily create a user with the same grants as the current user by using `GRANT CURRENT GRANTS`. [#48262](https://github.com/ClickHouse/ClickHouse/pull/48262) ([pufit](https://github.com/pufit)).
* Add statistical aggregate function `kolmogorovSmirnovTest`. Close [#48228](https://github.com/ClickHouse/ClickHouse/issues/48228). [#48325](https://github.com/ClickHouse/ClickHouse/pull/48325) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)).
* Added a `lost_part_count` column to the `system.replicas` table. The column value shows the total number of lost parts in the corresponding table. Value is stored in zookeeper and can be used instead of not persistent `ReplicatedDataLoss` profile event for monitoring. [#48526](https://github.com/ClickHouse/ClickHouse/pull/48526) ([Sergei Trifonov](https://github.com/serxa)).
* Add `soundex` function for compatibility. Closes [#39880](https://github.com/ClickHouse/ClickHouse/issues/39880). [#48567](https://github.com/ClickHouse/ClickHouse/pull/48567) ([FriendLey](https://github.com/FriendLey)).
* Support `Map` type for JSONExtract. [#48629](https://github.com/ClickHouse/ClickHouse/pull/48629) ([李扬](https://github.com/taiyang-li)).
* Add `PrettyJSONEachRow` format to output pretty JSON with new line delimiters and 4 space indents. [#48898](https://github.com/ClickHouse/ClickHouse/pull/48898) ([Kruglov Pavel](https://github.com/Avogar)).
* Add `ParquetMetadata` input format to read Parquet file metadata. [#48911](https://github.com/ClickHouse/ClickHouse/pull/48911) ([Kruglov Pavel](https://github.com/Avogar)).
* Add `extractKeyValuePairs` function to extract key value pairs from strings. Input strings might contain noise (i.e. log files / do not need to be 100% formatted in key-value-pair format), the algorithm will look for key value pairs matching the arguments passed to the function. As of now, function accepts the following arguments: `data_column` (mandatory), `key_value_pair_delimiter` (defaults to `:`), `pair_delimiters` (defaults to `\space \, \;`) and `quoting_character` (defaults to double quotes). [#43606](https://github.com/ClickHouse/ClickHouse/pull/43606) ([Arthur Passos](https://github.com/arthurpassos)).
* Functions replaceOne(), replaceAll(), replaceRegexpOne() and replaceRegexpAll() can now be called with non-const pattern and replacement arguments. [#46589](https://github.com/ClickHouse/ClickHouse/pull/46589) ([Robert Schulze](https://github.com/rschu1ze)).
* Added functions to work with columns of type `Map`: `mapConcat`, `mapSort`, `mapExists`. [#48071](https://github.com/ClickHouse/ClickHouse/pull/48071) ([Anton Popov](https://github.com/CurtizJ)).
#### Performance Improvement
* Reading files in `Parquet` format is now much faster. IO and decoding are parallelized (controlled by `max_threads` setting), and only required data ranges are read. [#47964](https://github.com/ClickHouse/ClickHouse/pull/47964) ([Michael Kolupaev](https://github.com/al13n321)).
* If we run a mutation with IN (subquery) like this: `ALTER TABLE t UPDATE col='new value' WHERE id IN (SELECT id FROM huge_table)` and the table `t` has multiple parts than for each part a set for subquery `SELECT id FROM huge_table` is built in memory. And if there are many parts then this might consume a lot of memory (and lead to an OOM) and CPU. The solution is to introduce a short-lived cache of sets that are currently being built by mutation tasks. If another task of the same mutation is executed concurrently it can look up the set in the cache, wait for it to be built and reuse it. [#46835](https://github.com/ClickHouse/ClickHouse/pull/46835) ([Alexander Gololobov](https://github.com/davenger)).
* Only check dependencies if necessary when applying `ALTER TABLE` queries. [#48062](https://github.com/ClickHouse/ClickHouse/pull/48062) ([Raúl Marín](https://github.com/Algunenano)).
* Optimize function `mapUpdate`. [#48118](https://github.com/ClickHouse/ClickHouse/pull/48118) ([Anton Popov](https://github.com/CurtizJ)).
* Now an internal query to local replica is sent explicitly and data from it received through loopback interface. Setting `prefer_localhost_replica` is not respected for parallel replicas. This is needed for better scheduling and makes the code cleaner: the initiator is only responsible for coordinating of the reading process and merging results, continuously answering for requests while all the secondary queries read the data. Note: Using loopback interface is not so performant, otherwise some replicas could starve for tasks which could lead to even slower query execution and not utilizing all possible resources. The initialization of the coordinator is now even more lazy. All incoming requests contain the information about the reading algorithm we initialize the coordinator with it when first request comes. If any replica decides to read with a different algorithman exception will be thrown and a query will be aborted. [#48246](https://github.com/ClickHouse/ClickHouse/pull/48246) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Do not build set for the right side of `IN` clause with subquery when it is used only for analysis of skip indexes, and they are disabled by setting (`use_skip_indexes=0`). Previously it might affect the performance of queries. [#48299](https://github.com/ClickHouse/ClickHouse/pull/48299) ([Anton Popov](https://github.com/CurtizJ)).
* Query processing is parallelized right after reading `FROM file(...)`. Related to [#38755](https://github.com/ClickHouse/ClickHouse/issues/38755). [#48525](https://github.com/ClickHouse/ClickHouse/pull/48525) ([Igor Nikonov](https://github.com/devcrafter)). Query processing is parallelized right after reading from any data source. Affected data sources are mostly simple or external storages like table functions `url`, `file`. [#48727](https://github.com/ClickHouse/ClickHouse/pull/48727) ([Igor Nikonov](https://github.com/devcrafter)). This is controlled by the setting `parallelize_output_from_storages` which is not enabled by default.
* Lowered contention of ThreadPool mutex (may increase performance for a huge amount of small jobs). [#48750](https://github.com/ClickHouse/ClickHouse/pull/48750) ([Sergei Trifonov](https://github.com/serxa)).
* Reduce memory usage for multiple `ALTER DELETE` mutations. [#48522](https://github.com/ClickHouse/ClickHouse/pull/48522) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Remove the excessive connection attempts if the `skip_unavailable_shards` setting is enabled. [#48771](https://github.com/ClickHouse/ClickHouse/pull/48771) ([Azat Khuzhin](https://github.com/azat)).
#### Experimental Feature
* Entries in the query cache are now squashed to max_block_size and compressed. [#45912](https://github.com/ClickHouse/ClickHouse/pull/45912) ([Robert Schulze](https://github.com/rschu1ze)).
* It is now possible to define per-user quotas in the query cache. [#48284](https://github.com/ClickHouse/ClickHouse/pull/48284) ([Robert Schulze](https://github.com/rschu1ze)).
* Some fixes for parallel replicas [#48433](https://github.com/ClickHouse/ClickHouse/pull/48433) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Implement zero-copy-replication (an experimental feature) on encrypted disks. [#48741](https://github.com/ClickHouse/ClickHouse/pull/48741) ([Vitaly Baranov](https://github.com/vitlibar)).
#### Improvement
* Increase default value for `connect_timeout_with_failover_ms` to 1000 ms (because of adding async connections in https://github.com/ClickHouse/ClickHouse/pull/47229) . Closes [#5188](https://github.com/ClickHouse/ClickHouse/issues/5188). [#49009](https://github.com/ClickHouse/ClickHouse/pull/49009) ([Kruglov Pavel](https://github.com/Avogar)).
* Several improvements around data lakes: - Make `Iceberg` work with non-partitioned data. - Support `Iceberg` format version v2 (previously only v1 was supported) - Support reading partitioned data for `DeltaLake`/`Hudi` - Faster reading of `DeltaLake` metadata by using Delta's checkpoint files - Fixed incorrect `Hudi` reads: previously it incorrectly chose which data to read and therefore was able to read correctly only small size tables - Made these engines to pickup updates of changed data (previously the state was set on table creation) - Make proper testing for `Iceberg`/`DeltaLake`/`Hudi` using spark. [#47307](https://github.com/ClickHouse/ClickHouse/pull/47307) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add async connection to socket and async writing to socket. Make creating connections and sending query/external tables async across shards. Refactor code with fibers. Closes [#46931](https://github.com/ClickHouse/ClickHouse/issues/46931). We will be able to increase `connect_timeout_with_failover_ms` by default after this PR (https://github.com/ClickHouse/ClickHouse/issues/5188). [#47229](https://github.com/ClickHouse/ClickHouse/pull/47229) ([Kruglov Pavel](https://github.com/Avogar)).
* Support config sections `keeper`/`keeper_server` as an alternative to `zookeeper`. Close [#34766](https://github.com/ClickHouse/ClickHouse/issues/34766) , [#34767](https://github.com/ClickHouse/ClickHouse/issues/34767). [#35113](https://github.com/ClickHouse/ClickHouse/pull/35113) ([李扬](https://github.com/taiyang-li)).
* It is possible to set _secure_ flag in named_collections for a dictionary with a ClickHouse table source. Addresses [#38450](https://github.com/ClickHouse/ClickHouse/issues/38450) . [#46323](https://github.com/ClickHouse/ClickHouse/pull/46323) ([Ilya Golshtein](https://github.com/ilejn)).
* `bitCount` function support `FixedString` and `String` data type. [#49044](https://github.com/ClickHouse/ClickHouse/pull/49044) ([flynn](https://github.com/ucasfl)).
* Added configurable retries for all operations with [Zoo]Keeper for Backup queries. [#47224](https://github.com/ClickHouse/ClickHouse/pull/47224) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Enable `use_environment_credentials` for S3 by default, so the entire provider chain is constructed by default. [#47397](https://github.com/ClickHouse/ClickHouse/pull/47397) ([Antonio Andelic](https://github.com/antonio2368)).
* Currently, the JSON_VALUE function is similar as spark's get_json_object function, which support to get value from JSON string by a path like '$.key'. But still has something different - 1. in spark's get_json_object will return null while the path is not exist, but in JSON_VALUE will return empty string; - 2. in spark's get_json_object will return a complex type value, such as a JSON object/array value, but in JSON_VALUE will return empty string. [#47494](https://github.com/ClickHouse/ClickHouse/pull/47494) ([KevinyhZou](https://github.com/KevinyhZou)).
* For `use_structure_from_insertion_table_in_table_functions` more flexible insert table structure propagation to table function. Fixed an issue with name mapping and using virtual columns. No more need for 'auto' setting. [#47962](https://github.com/ClickHouse/ClickHouse/pull/47962) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Do not continue retrying to connect to Keeper if the query is killed or over limits. [#47985](https://github.com/ClickHouse/ClickHouse/pull/47985) ([Raúl Marín](https://github.com/Algunenano)).
* Support Enum output/input in `BSONEachRow`, allow all map key types and avoid extra calculations on output. [#48122](https://github.com/ClickHouse/ClickHouse/pull/48122) ([Kruglov Pavel](https://github.com/Avogar)).
* Support more ClickHouse types in `ORC`/`Arrow`/`Parquet` formats: Enum(8|16), (U)Int(128|256), Decimal256 (for ORC), allow reading IPv4 from Int32 values (ORC outputs IPv4 as Int32, and we couldn't read it back), fix reading Nullable(IPv6) from binary data for `ORC`. [#48126](https://github.com/ClickHouse/ClickHouse/pull/48126) ([Kruglov Pavel](https://github.com/Avogar)).
* Add columns `perform_ttl_move_on_insert`, `load_balancing` for table `system.storage_policies`, modify column `volume_type` type to `Enum8`. [#48167](https://github.com/ClickHouse/ClickHouse/pull/48167) ([lizhuoyu5](https://github.com/lzydmxy)).
* Added support for `BACKUP ALL` command which backups all tables and databases, including temporary and system ones. [#48189](https://github.com/ClickHouse/ClickHouse/pull/48189) ([Vitaly Baranov](https://github.com/vitlibar)).
* Function mapFromArrays supports `Map` type as an input. [#48207](https://github.com/ClickHouse/ClickHouse/pull/48207) ([李扬](https://github.com/taiyang-li)).
* The output of some SHOW PROCESSLIST is now sorted. [#48241](https://github.com/ClickHouse/ClickHouse/pull/48241) ([Robert Schulze](https://github.com/rschu1ze)).
* Per-query/per-server throttling for remote IO/local IO/BACKUPs (server settings: `max_remote_read_network_bandwidth_for_server`, `max_remote_write_network_bandwidth_for_server`, `max_local_read_bandwidth_for_server`, `max_local_write_bandwidth_for_server`, `max_backup_bandwidth_for_server`, settings: `max_remote_read_network_bandwidth`, `max_remote_write_network_bandwidth`, `max_local_read_bandwidth`, `max_local_write_bandwidth`, `max_backup_bandwidth`). [#48242](https://github.com/ClickHouse/ClickHouse/pull/48242) ([Azat Khuzhin](https://github.com/azat)).
* Support more types in `CapnProto` format: Map, (U)Int(128|256), Decimal(128|256). Allow integer conversions during input/output. [#48257](https://github.com/ClickHouse/ClickHouse/pull/48257) ([Kruglov Pavel](https://github.com/Avogar)).
* Don't throw CURRENT_WRITE_BUFFER_IS_EXHAUSTED for normal behaviour. [#48288](https://github.com/ClickHouse/ClickHouse/pull/48288) ([Raúl Marín](https://github.com/Algunenano)).
* Add new setting `keeper_map_strict_mode` which enforces extra guarantees on operations made on top of `KeeperMap` tables. [#48293](https://github.com/ClickHouse/ClickHouse/pull/48293) ([Antonio Andelic](https://github.com/antonio2368)).
* Check primary key type for simple dictionary is native unsigned integer type Add setting `check_dictionary_primary_key ` for compatibility(set `check_dictionary_primary_key =false` to disable checking). [#48335](https://github.com/ClickHouse/ClickHouse/pull/48335) ([lizhuoyu5](https://github.com/lzydmxy)).
* Don't replicate mutations for `KeeperMap` because it's unnecessary. [#48354](https://github.com/ClickHouse/ClickHouse/pull/48354) ([Antonio Andelic](https://github.com/antonio2368)).
* Allow to write/read unnamed tuple as nested Message in Protobuf format. Tuple elements and Message fields are matched by position. [#48390](https://github.com/ClickHouse/ClickHouse/pull/48390) ([Kruglov Pavel](https://github.com/Avogar)).
* Support `additional_table_filters` and `additional_result_filter` settings in the new planner. Also, add a documentation entry for `additional_result_filter`. [#48405](https://github.com/ClickHouse/ClickHouse/pull/48405) ([Dmitry Novik](https://github.com/novikd)).
* `parseDateTime` now understands format string '%f' (fractional seconds). [#48420](https://github.com/ClickHouse/ClickHouse/pull/48420) ([Robert Schulze](https://github.com/rschu1ze)).
* Format string "%f" in formatDateTime() now prints "000000" if the formatted value has no fractional seconds, the previous behavior (single zero) can be restored using setting "formatdatetime_f_prints_single_zero = 1". [#48422](https://github.com/ClickHouse/ClickHouse/pull/48422) ([Robert Schulze](https://github.com/rschu1ze)).
* Don't replicate DELETE and TRUNCATE for KeeperMap. [#48434](https://github.com/ClickHouse/ClickHouse/pull/48434) ([Antonio Andelic](https://github.com/antonio2368)).
* Generate valid Decimals and Bools in generateRandom function. [#48436](https://github.com/ClickHouse/ClickHouse/pull/48436) ([Kruglov Pavel](https://github.com/Avogar)).
* Allow trailing commas in expression list of SELECT query, for example `SELECT a, b, c, FROM table`. Closes [#37802](https://github.com/ClickHouse/ClickHouse/issues/37802). [#48438](https://github.com/ClickHouse/ClickHouse/pull/48438) ([Nikolay Degterinsky](https://github.com/evillique)).
* Override `CLICKHOUSE_USER` and `CLICKHOUSE_PASSWORD` environment variables with `--user` and `--password` client parameters. Closes [#38909](https://github.com/ClickHouse/ClickHouse/issues/38909). [#48440](https://github.com/ClickHouse/ClickHouse/pull/48440) ([Nikolay Degterinsky](https://github.com/evillique)).
* Added retries to loading of data parts in `MergeTree` tables in case of retryable errors. [#48442](https://github.com/ClickHouse/ClickHouse/pull/48442) ([Anton Popov](https://github.com/CurtizJ)).
* Add support for `Date`, `Date32`, `DateTime`, `DateTime64` data types to `arrayMin`, `arrayMax`, `arrayDifference` functions. Closes [#21645](https://github.com/ClickHouse/ClickHouse/issues/21645). [#48445](https://github.com/ClickHouse/ClickHouse/pull/48445) ([Nikolay Degterinsky](https://github.com/evillique)).
* Add support for `{server_uuid}` macro. It is useful for identifying replicas in autoscaled clusters when new replicas are constantly added and removed in runtime. This closes [#48554](https://github.com/ClickHouse/ClickHouse/issues/48554). [#48563](https://github.com/ClickHouse/ClickHouse/pull/48563) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* The installation script will create a hard link instead of copying if it is possible. [#48578](https://github.com/ClickHouse/ClickHouse/pull/48578) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Support `SHOW TABLE` syntax meaning the same as `SHOW CREATE TABLE`. Closes [#48580](https://github.com/ClickHouse/ClickHouse/issues/48580). [#48591](https://github.com/ClickHouse/ClickHouse/pull/48591) ([flynn](https://github.com/ucasfl)).
* HTTP temporary buffers now support working by evicting data from the virtual filesystem cache. [#48664](https://github.com/ClickHouse/ClickHouse/pull/48664) ([Vladimir C](https://github.com/vdimir)).
* Make Schema inference works for `CREATE AS SELECT`. Closes [#47599](https://github.com/ClickHouse/ClickHouse/issues/47599). [#48679](https://github.com/ClickHouse/ClickHouse/pull/48679) ([flynn](https://github.com/ucasfl)).
* Added a `replicated_max_mutations_in_one_entry` setting for `ReplicatedMergeTree` that allows limiting the number of mutation commands per one `MUTATE_PART` entry (default is 10000). [#48731](https://github.com/ClickHouse/ClickHouse/pull/48731) ([Alexander Tokmakov](https://github.com/tavplubix)).
* In AggregateFunction types, don't count unused arena bytes as `read_bytes`. [#48745](https://github.com/ClickHouse/ClickHouse/pull/48745) ([Raúl Marín](https://github.com/Algunenano)).
* Fix some MySQL-related settings not being handled with the MySQL dictionary source + named collection. Closes [#48402](https://github.com/ClickHouse/ClickHouse/issues/48402). [#48759](https://github.com/ClickHouse/ClickHouse/pull/48759) ([Kseniia Sumarokova](https://github.com/kssenii)).
* If a user set `max_single_part_upload_size` to a very large value, it can lead to a crash due to a bug in the AWS S3 SDK. This fixes [#47679](https://github.com/ClickHouse/ClickHouse/issues/47679). [#48816](https://github.com/ClickHouse/ClickHouse/pull/48816) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix data race in `RabbitMQ` ([report](https://pastila.nl/?004f7100/de1505289ab5bb355e67ebe6c7cc8707)), refactor the code. [#48845](https://github.com/ClickHouse/ClickHouse/pull/48845) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add aliases `name` and `part_name` form `system.parts` and `system.part_log`. Closes [#48718](https://github.com/ClickHouse/ClickHouse/issues/48718). [#48850](https://github.com/ClickHouse/ClickHouse/pull/48850) ([sichenzhao](https://github.com/sichenzhao)).
* Functions "arrayDifferenceSupport()", "arrayCumSum()" and "arrayCumSumNonNegative()" now support input arrays of wide integer types (U)Int128/256. [#48866](https://github.com/ClickHouse/ClickHouse/pull/48866) ([cluster](https://github.com/infdahai)).
* Multi-line history in clickhouse-client is now no longer padded. This makes pasting more natural. [#48870](https://github.com/ClickHouse/ClickHouse/pull/48870) ([Joanna Hulboj](https://github.com/jh0x)).
* Implement a slight improvement for the rare case when ClickHouse is run inside LXC and LXCFS is used. The LXCFS has an issue: sometimes it returns an error "Transport endpoint is not connected" on reading from the file inside `/proc`. This error was correctly logged into ClickHouse's server log. We have additionally workaround this issue by reopening a file. This is a minuscule change. [#48922](https://github.com/ClickHouse/ClickHouse/pull/48922) ([Real](https://github.com/RunningXie)).
* Improve memory accounting for prefetches. Randomise prefetch settings In CI. [#48973](https://github.com/ClickHouse/ClickHouse/pull/48973) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Correctly set headers for native copy operations on GCS. [#48981](https://github.com/ClickHouse/ClickHouse/pull/48981) ([Antonio Andelic](https://github.com/antonio2368)).
* Add support for specifying setting names in the command line with dashes instead of underscores, for example, `--max-threads` instead of `--max_threads`. Additionally, support Unicode dash characters like `—` instead of `--` - this is useful when you communicate with a team in another company, and a manager from that team copy-pasted code from MS Word. [#48985](https://github.com/ClickHouse/ClickHouse/pull/48985) ([alekseygolub](https://github.com/alekseygolub)).
* Add fallback to password authentication when authentication with SSL user certificate has failed. Closes [#48974](https://github.com/ClickHouse/ClickHouse/issues/48974). [#48989](https://github.com/ClickHouse/ClickHouse/pull/48989) ([Nikolay Degterinsky](https://github.com/evillique)).
* Improve the embedded dashboard. Close [#46671](https://github.com/ClickHouse/ClickHouse/issues/46671). [#49036](https://github.com/ClickHouse/ClickHouse/pull/49036) ([Kevin Zhang](https://github.com/Kinzeng)).
* Add profile events for log messages, so you can easily see the count of log messages by severity. [#49042](https://github.com/ClickHouse/ClickHouse/pull/49042) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* In previous versions, the `LineAsString` format worked inconsistently when the parallel parsing was enabled or not, in presence of DOS or macOS Classic line breaks. This closes [#49039](https://github.com/ClickHouse/ClickHouse/issues/49039). [#49052](https://github.com/ClickHouse/ClickHouse/pull/49052) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* The exception message about the unparsed query parameter will also tell about the name of the parameter. Reimplement [#48878](https://github.com/ClickHouse/ClickHouse/issues/48878). Close [#48772](https://github.com/ClickHouse/ClickHouse/issues/48772). [#49061](https://github.com/ClickHouse/ClickHouse/pull/49061) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
#### Build/Testing/Packaging Improvement
* Update time zones. The following were updated: Africa/Cairo, Africa/Casablanca, Africa/El_Aaiun, America/Bogota, America/Cambridge_Bay, America/Ciudad_Juarez, America/Godthab, America/Inuvik, America/Iqaluit, America/Nuuk, America/Ojinaga, America/Pangnirtung, America/Rankin_Inlet, America/Resolute, America/Whitehorse, America/Yellowknife, Asia/Gaza, Asia/Hebron, Asia/Kuala_Lumpur, Asia/Singapore, Canada/Yukon, Egypt, Europe/Kirov, Europe/Volgograd, Singapore. [#48572](https://github.com/ClickHouse/ClickHouse/pull/48572) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Reduce the number of dependencies in the header files to speed up the build. [#47984](https://github.com/ClickHouse/ClickHouse/pull/47984) ([Dmitry Novik](https://github.com/novikd)).
* Randomize compression of marks and indices in tests. [#48286](https://github.com/ClickHouse/ClickHouse/pull/48286) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Bump internal ZSTD from 1.5.4 to 1.5.5. [#46797](https://github.com/ClickHouse/ClickHouse/pull/46797) ([Robert Schulze](https://github.com/rschu1ze)).
* Randomize vertical merges from compact to wide parts in tests. [#48287](https://github.com/ClickHouse/ClickHouse/pull/48287) ([Raúl Marín](https://github.com/Algunenano)).
* Support for CRC32 checksum in HDFS. Fix performance issues. [#48614](https://github.com/ClickHouse/ClickHouse/pull/48614) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Remove remainders of GCC support. [#48671](https://github.com/ClickHouse/ClickHouse/pull/48671) ([Robert Schulze](https://github.com/rschu1ze)).
* Add CI run with new analyzer infrastructure enabled. [#48719](https://github.com/ClickHouse/ClickHouse/pull/48719) ([Dmitry Novik](https://github.com/novikd)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Fix system.query_views_log for MVs that are pushed from background threads [#46668](https://github.com/ClickHouse/ClickHouse/pull/46668) ([Azat Khuzhin](https://github.com/azat)).
* Fix several `RENAME COLUMN` bugs [#46946](https://github.com/ClickHouse/ClickHouse/pull/46946) ([alesapin](https://github.com/alesapin)).
* Fix minor hiliting issues in clickhouse-format [#47610](https://github.com/ClickHouse/ClickHouse/pull/47610) ([Natasha Murashkina](https://github.com/murfel)).
* Fix a bug in LLVM's libc++ leading to a crash for uploading parts to S3 which size is greater than INT_MAX [#47693](https://github.com/ClickHouse/ClickHouse/pull/47693) ([Azat Khuzhin](https://github.com/azat)).
* Fix overflow in the `sparkbar` function [#48121](https://github.com/ClickHouse/ClickHouse/pull/48121) ([Vladimir C](https://github.com/vdimir)).
* Fix race in S3 [#48190](https://github.com/ClickHouse/ClickHouse/pull/48190) ([Anton Popov](https://github.com/CurtizJ)).
* Disable JIT for aggregate functions due to inconsistent behavior [#48195](https://github.com/ClickHouse/ClickHouse/pull/48195) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix alter formatting (minor) [#48289](https://github.com/ClickHouse/ClickHouse/pull/48289) ([Natasha Murashkina](https://github.com/murfel)).
* Fix CPU usage in RabbitMQ (was worsened in 23.2 after [#44404](https://github.com/ClickHouse/ClickHouse/issues/44404)) [#48311](https://github.com/ClickHouse/ClickHouse/pull/48311) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix crash in EXPLAIN PIPELINE for Merge over Distributed [#48320](https://github.com/ClickHouse/ClickHouse/pull/48320) ([Azat Khuzhin](https://github.com/azat)).
* Fix serializing LowCardinality as Arrow dictionary [#48361](https://github.com/ClickHouse/ClickHouse/pull/48361) ([Kruglov Pavel](https://github.com/Avogar)).
* Reset downloader for cache file segment in TemporaryFileStream [#48386](https://github.com/ClickHouse/ClickHouse/pull/48386) ([Vladimir C](https://github.com/vdimir)).
* Fix possible SYSTEM SYNC REPLICA stuck in case of DROP/REPLACE PARTITION [#48391](https://github.com/ClickHouse/ClickHouse/pull/48391) ([Azat Khuzhin](https://github.com/azat)).
* Fix a startup error when loading a distributed table that depends on a dictionary [#48419](https://github.com/ClickHouse/ClickHouse/pull/48419) ([MikhailBurdukov](https://github.com/MikhailBurdukov)).
* Don't check dependencies when renaming system tables automatically [#48431](https://github.com/ClickHouse/ClickHouse/pull/48431) ([Raúl Marín](https://github.com/Algunenano)).
* Update only affected rows in KeeperMap storage [#48435](https://github.com/ClickHouse/ClickHouse/pull/48435) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix possible segfault in the VFS cache [#48469](https://github.com/ClickHouse/ClickHouse/pull/48469) ([Kseniia Sumarokova](https://github.com/kssenii)).
* `toTimeZone` function throws an error when no constant string is provided [#48471](https://github.com/ClickHouse/ClickHouse/pull/48471) ([Jordi Villar](https://github.com/jrdi)).
* Fix logical error with IPv4 in Protobuf, add support for Date32 [#48486](https://github.com/ClickHouse/ClickHouse/pull/48486) ([Kruglov Pavel](https://github.com/Avogar)).
* "changed" flag in system.settings was calculated incorrectly for settings with multiple values [#48516](https://github.com/ClickHouse/ClickHouse/pull/48516) ([MikhailBurdukov](https://github.com/MikhailBurdukov)).
* Fix storage `Memory` with enabled compression [#48517](https://github.com/ClickHouse/ClickHouse/pull/48517) ([Anton Popov](https://github.com/CurtizJ)).
* Fix bracketed-paste mode messing up password input in the event of client reconnection [#48528](https://github.com/ClickHouse/ClickHouse/pull/48528) ([Michael Kolupaev](https://github.com/al13n321)).
* Fix nested map for keys of IP and UUID types [#48556](https://github.com/ClickHouse/ClickHouse/pull/48556) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Fix an uncaught exception in case of parallel loader for hashed dictionaries [#48571](https://github.com/ClickHouse/ClickHouse/pull/48571) ([Azat Khuzhin](https://github.com/azat)).
* The `groupArray` aggregate function correctly works for empty result over nullable types [#48593](https://github.com/ClickHouse/ClickHouse/pull/48593) ([lgbo](https://github.com/lgbo-ustc)).
* Fix bug in Keeper when a node is not created with scheme `auth` in ACL sometimes. [#48595](https://github.com/ClickHouse/ClickHouse/pull/48595) ([Aleksei Filatov](https://github.com/aalexfvk)).
* Allow IPv4 comparison operators with UInt [#48611](https://github.com/ClickHouse/ClickHouse/pull/48611) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Fix possible error from cache [#48636](https://github.com/ClickHouse/ClickHouse/pull/48636) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Async inserts with empty data will no longer throw exception. [#48663](https://github.com/ClickHouse/ClickHouse/pull/48663) ([Anton Popov](https://github.com/CurtizJ)).
* Fix table dependencies in case of failed RENAME TABLE [#48683](https://github.com/ClickHouse/ClickHouse/pull/48683) ([Azat Khuzhin](https://github.com/azat)).
* If the primary key has duplicate columns (which is only possible for projections), in previous versions it might lead to a bug [#48838](https://github.com/ClickHouse/ClickHouse/pull/48838) ([Amos Bird](https://github.com/amosbird)).
* Fix for a race condition in ZooKeeper when joining send_thread/receive_thread [#48849](https://github.com/ClickHouse/ClickHouse/pull/48849) ([Alexander Gololobov](https://github.com/davenger)).
* Fix unexpected part name error when trying to drop a ignored detached part with zero copy replication [#48862](https://github.com/ClickHouse/ClickHouse/pull/48862) ([Michael Lex](https://github.com/mlex)).
* Fix reading `Date32` Parquet/Arrow column into not a `Date32` column [#48864](https://github.com/ClickHouse/ClickHouse/pull/48864) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix `UNKNOWN_IDENTIFIER` error while selecting from table with row policy and column with dots [#48976](https://github.com/ClickHouse/ClickHouse/pull/48976) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix aggregation by empty nullable strings [#48999](https://github.com/ClickHouse/ClickHouse/pull/48999) ([LiuNeng](https://github.com/liuneng1994)).
### <a id="233"></a> ClickHouse release 23.3 LTS, 2023-03-30
#### Upgrade Notes

View File

@ -176,6 +176,12 @@ if (OS_DARWIN)
set (ENABLE_CURL_BUILD OFF)
endif ()
option(ENABLE_ISAL_LIBRARY "Enable ISA-L library ON by default except on aarch64." ON)
if (ARCH_AARCH64)
# Disable ISA-L libray on aarch64.
set (ENABLE_ISAL_LIBRARY OFF)
endif ()
if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE")
# Can be lld or ld-lld or lld-13 or /path/to/lld.
if (LINKER_NAME MATCHES "lld")

View File

@ -13,9 +13,10 @@ The following versions of ClickHouse server are currently being supported with s
| Version | Supported |
|:-|:-|
| 23.4 | ✔️ |
| 23.3 | ✔️ |
| 23.2 | ✔️ |
| 23.1 | ✔️ |
| 23.1 | |
| 22.12 | ❌ |
| 22.11 | ❌ |
| 22.10 | ❌ |

View File

@ -235,6 +235,17 @@ ssize_t getrandom(void *buf, size_t buflen, unsigned flags)
return syscall(SYS_getrandom, buf, buflen, flags);
}
/* Structure for scatter/gather I/O. */
struct iovec
{
void *iov_base; /* Pointer to data. */
size_t iov_len; /* Length of data. */
};
ssize_t preadv(int __fd, const struct iovec *__iovec, int __count, __off_t __offset)
{
return syscall(SYS_preadv, __fd, __iovec, __count, (long)(__offset), (long)(__offset>>32));
}
#include <errno.h>
#include <limits.h>

View File

@ -2,11 +2,11 @@
# NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION,
# only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes.
SET(VERSION_REVISION 54473)
SET(VERSION_REVISION 54474)
SET(VERSION_MAJOR 23)
SET(VERSION_MINOR 4)
SET(VERSION_MINOR 5)
SET(VERSION_PATCH 1)
SET(VERSION_GITHASH 46e85357ce2da2a99f56ee83a079e892d7ec3726)
SET(VERSION_DESCRIBE v23.4.1.1-testing)
SET(VERSION_STRING 23.4.1.1)
SET(VERSION_GITHASH 3920eb987f7ed837ada5de8907284adf123f0583)
SET(VERSION_DESCRIBE v23.5.1.1-testing)
SET(VERSION_STRING 23.5.1.1)
# end of autochange

View File

@ -7,10 +7,6 @@ if (FUZZER)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=fuzzer-no-link")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=fuzzer-no-link")
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=fuzzer-no-link")
endif()
# NOTE: oss-fuzz can change LIB_FUZZING_ENGINE variable
if (NOT LIB_FUZZING_ENGINE)
set (LIB_FUZZING_ENGINE "-fsanitize=fuzzer")

View File

@ -16,49 +16,24 @@ if (SANITIZE)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${ASAN_FLAGS}")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${ASAN_FLAGS}")
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${ASAN_FLAGS}")
endif()
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libasan")
endif ()
elseif (SANITIZE STREQUAL "memory")
# MemorySanitizer flags are set according to the official documentation:
# https://clang.llvm.org/docs/MemorySanitizer.html#usage
#
# For now, it compiles with `cmake -DSANITIZE=memory -DCMAKE_BUILD_TYPE=RelWithDebInfo -DCMAKE_CXX_FLAGS_ADD="-O1" -DCMAKE_C_FLAGS_ADD="-O1"`
# Compiling with -DCMAKE_BUILD_TYPE=Debug leads to ld.lld failures because
# of large files (was not tested with ld.gold). This is why we compile with
# RelWithDebInfo, and downgrade optimizations to -O1 but not to -Og, to
# keep the binary size down.
# TODO: try compiling with -Og and with ld.gold.
# Linking can fail due to relocation overflows (see #49145), caused by too big object files / libraries.
# Work around this with position-independent builds (-fPIC and -fpie), this is slightly slower than non-PIC/PIE but that's okay.
set (MSAN_FLAGS "-fsanitize=memory -fsanitize-memory-use-after-dtor -fsanitize-memory-track-origins -fno-optimize-sibling-calls -fPIC -fpie -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/msan_suppressions.txt")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${MSAN_FLAGS}")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${MSAN_FLAGS}")
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=memory")
endif()
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libmsan")
endif ()
elseif (SANITIZE STREQUAL "thread")
set (TSAN_FLAGS "-fsanitize=thread")
if (COMPILER_CLANG)
set (TSAN_FLAGS "${TSAN_FLAGS} -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/tsan_suppressions.txt")
endif()
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${TSAN_FLAGS}")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${TSAN_FLAGS}")
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=thread")
endif()
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libtsan")
endif ()
elseif (SANITIZE STREQUAL "undefined")
set (UBSAN_FLAGS "-fsanitize=undefined -fno-sanitize-recover=all -fno-sanitize=float-divide-by-zero")
@ -77,12 +52,6 @@ if (SANITIZE)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${UBSAN_FLAGS}")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${UBSAN_FLAGS}")
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=undefined")
endif()
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libubsan")
endif ()
# llvm-tblgen, that is used during LLVM build, doesn't work with UBSan.
set (ENABLE_EMBEDDED_COMPILER 0 CACHE BOOL "")

View File

@ -191,6 +191,10 @@ add_contrib (google-benchmark-cmake google-benchmark)
add_contrib (ulid-c-cmake ulid-c)
if (ENABLE_ISAL_LIBRARY)
add_contrib (isa-l-cmake isa-l)
endif()
# Put all targets defined here and in subdirectories under "contrib/<immediate-subdir>" folders in GUI-based IDEs.
# Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear
# in "contrib/..." as originally planned, so we workaround this by fixing FOLDER properties of all targets manually,

2
contrib/curl vendored

@ -1 +1 @@
Subproject commit c12fb3ddaf48e709a7a4deaa55ec485e4df163ee
Subproject commit b0edf0b7dae44d9e66f270a257cf654b35d5263d

View File

@ -12,6 +12,9 @@ set (SRCS
"${LIBRARY_DIR}/lib/noproxy.c"
"${LIBRARY_DIR}/lib/idn.c"
"${LIBRARY_DIR}/lib/cfilters.c"
"${LIBRARY_DIR}/lib/cf-socket.c"
"${LIBRARY_DIR}/lib/cf-haproxy.c"
"${LIBRARY_DIR}/lib/cf-https-connect.c"
"${LIBRARY_DIR}/lib/file.c"
"${LIBRARY_DIR}/lib/timeval.c"
"${LIBRARY_DIR}/lib/base64.c"
@ -37,8 +40,8 @@ set (SRCS
"${LIBRARY_DIR}/lib/strcase.c"
"${LIBRARY_DIR}/lib/easy.c"
"${LIBRARY_DIR}/lib/curl_fnmatch.c"
"${LIBRARY_DIR}/lib/curl_log.c"
"${LIBRARY_DIR}/lib/fileinfo.c"
"${LIBRARY_DIR}/lib/wildcard.c"
"${LIBRARY_DIR}/lib/krb5.c"
"${LIBRARY_DIR}/lib/memdebug.c"
"${LIBRARY_DIR}/lib/http_chunks.c"
@ -96,6 +99,7 @@ set (SRCS
"${LIBRARY_DIR}/lib/rand.c"
"${LIBRARY_DIR}/lib/curl_multibyte.c"
"${LIBRARY_DIR}/lib/conncache.c"
"${LIBRARY_DIR}/lib/cf-h1-proxy.c"
"${LIBRARY_DIR}/lib/http2.c"
"${LIBRARY_DIR}/lib/smb.c"
"${LIBRARY_DIR}/lib/curl_endian.c"
@ -113,12 +117,13 @@ set (SRCS
"${LIBRARY_DIR}/lib/altsvc.c"
"${LIBRARY_DIR}/lib/socketpair.c"
"${LIBRARY_DIR}/lib/bufref.c"
"${LIBRARY_DIR}/lib/bufq.c"
"${LIBRARY_DIR}/lib/dynbuf.c"
"${LIBRARY_DIR}/lib/dynhds.c"
"${LIBRARY_DIR}/lib/hsts.c"
"${LIBRARY_DIR}/lib/http_aws_sigv4.c"
"${LIBRARY_DIR}/lib/mqtt.c"
"${LIBRARY_DIR}/lib/rename.c"
"${LIBRARY_DIR}/lib/h2h3.c"
"${LIBRARY_DIR}/lib/headers.c"
"${LIBRARY_DIR}/lib/timediff.c"
"${LIBRARY_DIR}/lib/vauth/vauth.c"
@ -133,6 +138,7 @@ set (SRCS
"${LIBRARY_DIR}/lib/vauth/oauth2.c"
"${LIBRARY_DIR}/lib/vauth/spnego_gssapi.c"
"${LIBRARY_DIR}/lib/vauth/spnego_sspi.c"
"${LIBRARY_DIR}/lib/vquic/vquic.c"
"${LIBRARY_DIR}/lib/vtls/openssl.c"
"${LIBRARY_DIR}/lib/vtls/gtls.c"
"${LIBRARY_DIR}/lib/vtls/vtls.c"
@ -147,9 +153,6 @@ set (SRCS
"${LIBRARY_DIR}/lib/vtls/keylog.c"
"${LIBRARY_DIR}/lib/vtls/x509asn1.c"
"${LIBRARY_DIR}/lib/vtls/hostcheck.c"
"${LIBRARY_DIR}/lib/vquic/ngtcp2.c"
"${LIBRARY_DIR}/lib/vquic/quiche.c"
"${LIBRARY_DIR}/lib/vquic/msh3.c"
"${LIBRARY_DIR}/lib/vssh/libssh2.c"
"${LIBRARY_DIR}/lib/vssh/libssh.c"
)

1
contrib/isa-l vendored Submodule

@ -0,0 +1 @@
Subproject commit 9f2b68f05752097f0f16632fc4a9a86950831efd

View File

@ -0,0 +1,192 @@
set(ISAL_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/isa-l")
# The YASM and NASM assembers are somewhat mutually compatible. ISAL specifically needs NASM. If only YASM is installed, then check_language(ASM_NASM)
# below happily finds YASM, leading to weird errors at build time. Therefore, do an explicit check for NASM here.
find_program(NASM_PATH NAMES nasm)
if (NOT NASM_PATH)
message(FATAL_ERROR "Please install NASM from 'https://www.nasm.us/' because NASM compiler can not be found!")
endif ()
include(CheckLanguage)
check_language(ASM_NASM)
if(NOT CMAKE_ASM_NASM_COMPILER)
message(FATAL_ERROR "Please install NASM from 'https://www.nasm.us/' because NASM compiler can not be found!")
endif()
enable_language(ASM_NASM)
set(ISAL_C_SRC
${ISAL_SOURCE_DIR}/crc/crc_base_aliases.c
${ISAL_SOURCE_DIR}/crc/crc_base.c
${ISAL_SOURCE_DIR}/crc/crc64_base.c
${ISAL_SOURCE_DIR}/erasure_code/ec_base.c
${ISAL_SOURCE_DIR}/erasure_code/ec_base_aliases.c
${ISAL_SOURCE_DIR}/erasure_code/ec_highlevel_func.c
${ISAL_SOURCE_DIR}/erasure_code/gen_rs_matrix_limits.c
${ISAL_SOURCE_DIR}/erasure_code/gf_vect_dot_prod_1tbl.c
${ISAL_SOURCE_DIR}/igzip/adler32_base.c
${ISAL_SOURCE_DIR}/igzip/encode_df.c
${ISAL_SOURCE_DIR}/igzip/flatten_ll.c
${ISAL_SOURCE_DIR}/igzip/generate_custom_hufftables.c
${ISAL_SOURCE_DIR}/igzip/generate_static_inflate.c
${ISAL_SOURCE_DIR}/igzip/huff_codes.c
${ISAL_SOURCE_DIR}/igzip/hufftables_c.c
${ISAL_SOURCE_DIR}/igzip/igzip_base_aliases.c
${ISAL_SOURCE_DIR}/igzip/igzip_base.c
${ISAL_SOURCE_DIR}/igzip/igzip_icf_base.c
${ISAL_SOURCE_DIR}/igzip/igzip_icf_body.c
${ISAL_SOURCE_DIR}/igzip/igzip_inflate.c
${ISAL_SOURCE_DIR}/igzip/igzip.c
${ISAL_SOURCE_DIR}/mem/mem_zero_detect_base_aliases.c
${ISAL_SOURCE_DIR}/mem/mem_zero_detect_base.c
${ISAL_SOURCE_DIR}/programs/igzip_cli.c
${ISAL_SOURCE_DIR}/raid/raid_base_aliases.c
${ISAL_SOURCE_DIR}/raid/raid_base.c
)
set(ISAL_ASM_SRC
${ISAL_SOURCE_DIR}/crc/crc_multibinary.asm
${ISAL_SOURCE_DIR}/crc/crc16_t10dif_01.asm
${ISAL_SOURCE_DIR}/crc/crc16_t10dif_02.asm
${ISAL_SOURCE_DIR}/crc/crc16_t10dif_by4.asm
${ISAL_SOURCE_DIR}/crc/crc16_t10dif_by16_10.asm
${ISAL_SOURCE_DIR}/crc/crc16_t10dif_copy_by4_02.asm
${ISAL_SOURCE_DIR}/crc/crc16_t10dif_copy_by4.asm
${ISAL_SOURCE_DIR}/crc/crc32_gzip_refl_by8_02.asm
${ISAL_SOURCE_DIR}/crc/crc32_gzip_refl_by8.asm
${ISAL_SOURCE_DIR}/crc/crc32_gzip_refl_by16_10.asm
${ISAL_SOURCE_DIR}/crc/crc32_ieee_01.asm
${ISAL_SOURCE_DIR}/crc/crc32_ieee_02.asm
${ISAL_SOURCE_DIR}/crc/crc32_ieee_by4.asm
${ISAL_SOURCE_DIR}/crc/crc32_ieee_by16_10.asm
${ISAL_SOURCE_DIR}/crc/crc32_iscsi_00.asm
${ISAL_SOURCE_DIR}/crc/crc32_iscsi_01.asm
${ISAL_SOURCE_DIR}/crc/crc32_iscsi_by16_10.asm
${ISAL_SOURCE_DIR}/crc/crc64_ecma_norm_by8.asm
${ISAL_SOURCE_DIR}/crc/crc64_ecma_norm_by16_10.asm
${ISAL_SOURCE_DIR}/crc/crc64_ecma_refl_by8.asm
${ISAL_SOURCE_DIR}/crc/crc64_ecma_refl_by16_10.asm
${ISAL_SOURCE_DIR}/crc/crc64_iso_norm_by8.asm
${ISAL_SOURCE_DIR}/crc/crc64_iso_norm_by16_10.asm
${ISAL_SOURCE_DIR}/crc/crc64_iso_refl_by8.asm
${ISAL_SOURCE_DIR}/crc/crc64_iso_refl_by16_10.asm
${ISAL_SOURCE_DIR}/crc/crc64_jones_norm_by8.asm
${ISAL_SOURCE_DIR}/crc/crc64_jones_norm_by16_10.asm
${ISAL_SOURCE_DIR}/crc/crc64_jones_refl_by8.asm
${ISAL_SOURCE_DIR}/crc/crc64_jones_refl_by16_10.asm
${ISAL_SOURCE_DIR}/crc/crc64_multibinary.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_2vect_dot_prod_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_2vect_dot_prod_avx2.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_2vect_dot_prod_avx512.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_2vect_dot_prod_sse.asm
${ISAL_SOURCE_DIR}/erasure_code/ec_multibinary.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_2vect_mad_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_2vect_mad_avx2.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_2vect_mad_avx512.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_2vect_mad_sse.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_3vect_dot_prod_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_3vect_dot_prod_avx2.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_3vect_dot_prod_avx512.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_3vect_dot_prod_sse.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_3vect_mad_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_3vect_mad_avx2.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_3vect_mad_avx512.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_3vect_mad_sse.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_4vect_dot_prod_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_4vect_dot_prod_avx2.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_4vect_dot_prod_avx512.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_4vect_dot_prod_sse.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_4vect_mad_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_4vect_mad_avx2.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_4vect_mad_avx512.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_4vect_mad_sse.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_5vect_dot_prod_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_5vect_dot_prod_avx2.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_5vect_dot_prod_avx512.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_5vect_dot_prod_sse.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_5vect_mad_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_5vect_mad_avx2.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_5vect_mad_avx512.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_5vect_mad_sse.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_6vect_dot_prod_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_6vect_dot_prod_avx2.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_6vect_dot_prod_avx512.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_6vect_dot_prod_sse.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_6vect_mad_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_6vect_mad_avx2.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_6vect_mad_avx512.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_6vect_mad_sse.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_vect_dot_prod_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_vect_dot_prod_avx2.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_vect_dot_prod_avx512.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_vect_dot_prod_sse.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_vect_mad_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_vect_mad_avx2.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_vect_mad_avx512.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_vect_mad_sse.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_vect_mul_avx.asm
${ISAL_SOURCE_DIR}/erasure_code/gf_vect_mul_sse.asm
${ISAL_SOURCE_DIR}/igzip/adler32_avx2_4.asm
${ISAL_SOURCE_DIR}/igzip/adler32_sse.asm
${ISAL_SOURCE_DIR}/igzip/bitbuf2.asm
${ISAL_SOURCE_DIR}/igzip/encode_df_04.asm
${ISAL_SOURCE_DIR}/igzip/encode_df_06.asm
${ISAL_SOURCE_DIR}/igzip/heap_macros.asm
${ISAL_SOURCE_DIR}/igzip/huffman.asm
${ISAL_SOURCE_DIR}/igzip/igzip_body.asm
${ISAL_SOURCE_DIR}/igzip/igzip_compare_types.asm
${ISAL_SOURCE_DIR}/igzip/igzip_decode_block_stateless_01.asm
${ISAL_SOURCE_DIR}/igzip/igzip_decode_block_stateless_04.asm
${ISAL_SOURCE_DIR}/igzip/igzip_deflate_hash.asm
${ISAL_SOURCE_DIR}/igzip/igzip_finish.asm
${ISAL_SOURCE_DIR}/igzip/igzip_gen_icf_map_lh1_04.asm
${ISAL_SOURCE_DIR}/igzip/igzip_gen_icf_map_lh1_06.asm
${ISAL_SOURCE_DIR}/igzip/igzip_icf_body_h1_gr_bt.asm
${ISAL_SOURCE_DIR}/igzip/igzip_icf_finish.asm
${ISAL_SOURCE_DIR}/igzip/igzip_inflate_multibinary.asm
${ISAL_SOURCE_DIR}/igzip/igzip_multibinary.asm
${ISAL_SOURCE_DIR}/igzip/igzip_set_long_icf_fg_04.asm
${ISAL_SOURCE_DIR}/igzip/igzip_set_long_icf_fg_06.asm
${ISAL_SOURCE_DIR}/igzip/igzip_update_histogram_01.asm
${ISAL_SOURCE_DIR}/igzip/igzip_update_histogram_04.asm
${ISAL_SOURCE_DIR}/igzip/lz0a_const.asm
${ISAL_SOURCE_DIR}/igzip/options.asm
${ISAL_SOURCE_DIR}/igzip/proc_heap.asm
${ISAL_SOURCE_DIR}/igzip/rfc1951_lookup.asm
${ISAL_SOURCE_DIR}/igzip/stdmac.asm
${ISAL_SOURCE_DIR}/mem/mem_multibinary.asm
${ISAL_SOURCE_DIR}/mem/mem_zero_detect_avx.asm
${ISAL_SOURCE_DIR}/mem/mem_zero_detect_avx2.asm
${ISAL_SOURCE_DIR}/mem/mem_zero_detect_avx512.asm
${ISAL_SOURCE_DIR}/mem/mem_zero_detect_sse.asm
${ISAL_SOURCE_DIR}/raid/pq_check_sse.asm
${ISAL_SOURCE_DIR}/raid/pq_gen_avx.asm
${ISAL_SOURCE_DIR}/raid/pq_gen_avx2.asm
${ISAL_SOURCE_DIR}/raid/pq_gen_avx512.asm
${ISAL_SOURCE_DIR}/raid/pq_gen_sse.asm
${ISAL_SOURCE_DIR}/raid/raid_multibinary.asm
${ISAL_SOURCE_DIR}/raid/xor_check_sse.asm
${ISAL_SOURCE_DIR}/raid/xor_gen_avx.asm
${ISAL_SOURCE_DIR}/raid/xor_gen_avx512.asm
${ISAL_SOURCE_DIR}/raid/xor_gen_sse.asm
)
# Adding ISA-L library target
add_library(_isal ${ISAL_C_SRC} ${ISAL_ASM_SRC})
# Setting external and internal interfaces for ISA-L library
target_include_directories(_isal
PUBLIC ${ISAL_SOURCE_DIR}/include
PUBLIC ${ISAL_SOURCE_DIR}/igzip
PUBLIC ${ISAL_SOURCE_DIR}/crc
PUBLIC ${ISAL_SOURCE_DIR}/erasure_code)
# Here must remove "-fno-sanitize=undefined" from COMPILE_OPTIONS.
# Otherwise nasm compiler would fail to proceed due to unrecognition of "-fno-sanitize=undefined"
if (SANITIZE STREQUAL "undefined")
get_target_property(target_options _isal COMPILE_OPTIONS)
list(REMOVE_ITEM target_options "-fno-sanitize=undefined")
set_property(TARGET _isal PROPERTY COMPILE_OPTIONS ${target_options})
endif()
add_library(ch_contrib::isal ALIAS _isal)

2
contrib/libhdfs3 vendored

@ -1 +1 @@
Subproject commit 3c91d96ff29fe5928f055519c6d979c4b104db9e
Subproject commit 164b89253fad7991bce77882f01b51ab81d19f3d

View File

@ -70,6 +70,30 @@ set(SRCS
"${HDFS3_SOURCE_DIR}/client/Token.cpp"
"${HDFS3_SOURCE_DIR}/client/PacketPool.cpp"
"${HDFS3_SOURCE_DIR}/client/OutputStream.cpp"
"${HDFS3_SOURCE_DIR}/client/AbstractNativeRawDecoder.cpp"
"${HDFS3_SOURCE_DIR}/client/AbstractNativeRawEncoder.cpp"
"${HDFS3_SOURCE_DIR}/client/ByteBufferDecodingState.cpp"
"${HDFS3_SOURCE_DIR}/client/ByteBufferEncodingState.cpp"
"${HDFS3_SOURCE_DIR}/client/CoderUtil.cpp"
"${HDFS3_SOURCE_DIR}/client/ECChunk.cpp"
"${HDFS3_SOURCE_DIR}/client/ErasureCoderOptions.cpp"
"${HDFS3_SOURCE_DIR}/client/GF256.cpp"
"${HDFS3_SOURCE_DIR}/client/GaloisField.cpp"
"${HDFS3_SOURCE_DIR}/client/NativeRSRawDecoder.cpp"
"${HDFS3_SOURCE_DIR}/client/NativeRSRawEncoder.cpp"
"${HDFS3_SOURCE_DIR}/client/Preconditions.cpp"
"${HDFS3_SOURCE_DIR}/client/RSUtil.cpp"
"${HDFS3_SOURCE_DIR}/client/RawErasureCoderFactory.cpp"
"${HDFS3_SOURCE_DIR}/client/RawErasureDecoder.cpp"
"${HDFS3_SOURCE_DIR}/client/RawErasureEncoder.cpp"
"${HDFS3_SOURCE_DIR}/client/StatefulStripeReader.cpp"
"${HDFS3_SOURCE_DIR}/client/StripeReader.cpp"
"${HDFS3_SOURCE_DIR}/client/StripedBlockUtil.cpp"
"${HDFS3_SOURCE_DIR}/client/StripedInputStreamImpl.cpp"
"${HDFS3_SOURCE_DIR}/client/StripedOutputStreamImpl.cpp"
"${HDFS3_SOURCE_DIR}/client/SystemECPolicies.cpp"
"${HDFS3_SOURCE_DIR}/client/dump.cpp"
"${HDFS3_SOURCE_DIR}/client/erasure_coder.cpp"
"${HDFS3_SOURCE_DIR}/rpc/RpcChannelKey.cpp"
"${HDFS3_SOURCE_DIR}/rpc/RpcProtocolInfo.cpp"
"${HDFS3_SOURCE_DIR}/rpc/RpcClient.cpp"
@ -148,6 +172,11 @@ if (TARGET OpenSSL::SSL)
target_link_libraries(_hdfs3 PRIVATE OpenSSL::Crypto OpenSSL::SSL)
endif()
if (ENABLE_ISAL_LIBRARY)
target_link_libraries(_hdfs3 PRIVATE ch_contrib::isal)
add_definitions(-DHADOOP_ISAL_LIBRARY)
endif()
add_library(ch_contrib::hdfs ALIAS _hdfs3)
if (ENABLE_CLICKHOUSE_BENCHMARK)

2
contrib/qpl vendored

@ -1 +1 @@
Subproject commit d75a29d95d8a548297fce3549d21020005364dc8
Subproject commit 0bce2b03423f6fbeb8bce66cc8be0bf558058848

View File

@ -40,9 +40,10 @@ set (LOG_HW_INIT OFF)
set (SANITIZE_MEMORY OFF)
set (SANITIZE_THREADS OFF)
set (LIB_FUZZING_ENGINE OFF)
set (DYNAMIC_LOADING_LIBACCEL_CONFIG OFF)
function(GetLibraryVersion _content _outputVar)
string(REGEX MATCHALL "Qpl VERSION (.+) LANGUAGES" VERSION_REGEX "${_content}")
string(REGEX MATCHALL "QPL VERSION (.+) LANGUAGES" VERSION_REGEX "${_content}")
SET(${_outputVar} ${CMAKE_MATCH_1} PARENT_SCOPE)
endfunction()
@ -240,7 +241,9 @@ add_library(core_iaa OBJECT ${HW_PATH_SRC})
target_include_directories(core_iaa
PRIVATE ${UUID_DIR}
PUBLIC $<BUILD_INTERFACE:${QPL_SRC_DIR}/core-iaa/include>
PRIVATE $<BUILD_INTERFACE:${QPL_SRC_DIR}/core-iaa/sources/include>
PUBLIC $<BUILD_INTERFACE:${QPL_SRC_DIR}/core-iaa/sources/include>
PRIVATE $<BUILD_INTERFACE:${QPL_PROJECT_DIR}/include> # status.h in own_checkers.h
PRIVATE $<BUILD_INTERFACE:${QPL_PROJECT_DIR}/sources/c_api> # own_checkers.h
PRIVATE $<TARGET_PROPERTY:qplcore_avx512,INTERFACE_INCLUDE_DIRECTORIES>)
target_compile_options(core_iaa
@ -339,4 +342,7 @@ target_link_libraries(_qpl
PRIVATE ${CMAKE_DL_LIBS})
add_library (ch_contrib::qpl ALIAS _qpl)
target_include_directories(_qpl SYSTEM BEFORE PUBLIC "${QPL_PROJECT_DIR}/include")
target_include_directories(_qpl SYSTEM BEFORE
PUBLIC "${QPL_PROJECT_DIR}/include"
PUBLIC "${LIBACCEL_SOURCE_DIR}/accfg"
PUBLIC ${UUID_DIR})

View File

@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \
esac
ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release"
ARG VERSION="23.3.2.37"
ARG VERSION="23.4.1.1943"
ARG PACKAGES="clickhouse-keeper"
# user/group precreated explicitly with fixed uid/gid on purpose.

View File

@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
ARG VERSION="23.3.2.37"
ARG VERSION="23.4.1.1943"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# user/group precreated explicitly with fixed uid/gid on purpose.

View File

@ -22,7 +22,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
ARG VERSION="23.3.2.37"
ARG VERSION="23.4.1.1943"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# set non-empty deb_location_url url to create a docker image

View File

@ -80,11 +80,9 @@ def process_test_log(log_path, broken_tests):
test_results.append(
(
test_name,
"FAIL",
"SKIPPED",
test_time,
[
"Test is expected to fail! Please, update broken_tests.txt!\n"
],
["This test passed. Update broken_tests.txt.\n"],
)
)
else:

View File

@ -0,0 +1,375 @@
---
sidebar_position: 1
sidebar_label: 2023
---
# 2023 Changelog
### ClickHouse release v23.4.1.1943-stable (3920eb987f7) FIXME as compared to v23.3.1.2823-lts (46e85357ce2)
#### Backward Incompatible Change
* If `path` in cache configuration is not empty and is not absolute path, then it will be put in `<clickhouse server data directory>/caches/<path_from_cache_config>`. [#48784](https://github.com/ClickHouse/ClickHouse/pull/48784) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Compatibility setting `parallelize_output_from_storages` to enable behavior before [#48727](https://github.com/ClickHouse/ClickHouse/issues/48727). [#49101](https://github.com/ClickHouse/ClickHouse/pull/49101) ([Igor Nikonov](https://github.com/devcrafter)).
#### New Feature
* Add `extractKeyValuePairs` function to extract key value pairs from strings. Input strings might contain noise (i.e log files / do not need to be 100% formatted in key-value-pair format), the algorithm will look for key value pairs matching the arguments passed to the function. As of now, function accepts the following arguments: `data_column` (mandatory), `key_value_pair_delimiter` (defaults to `:`), `pair_delimiters` (defaults to `\space \, \;`) and `quoting_character` (defaults to double quotes). [#43606](https://github.com/ClickHouse/ClickHouse/pull/43606) ([Arthur Passos](https://github.com/arthurpassos)).
* Add MemoryTracker for the background tasks (merges and mutation). Introduces `merges_mutations_memory_usage_soft_limit` and `merges_mutations_memory_usage_to_ram_ratio` settings that represent the soft memory limit for merges and mutations. If this limit is reached ClickHouse won't schedule new merge or mutation tasks. Also `MergesMutationsMemoryTracking` metric is introduced to allow observing current memory usage of background tasks. Closes [#45710](https://github.com/ClickHouse/ClickHouse/issues/45710). [#46089](https://github.com/ClickHouse/ClickHouse/pull/46089) ([Dmitry Novik](https://github.com/novikd)).
* Support new aggregate function quantileGK/quantilesGK, like [approx_percentile](https://spark.apache.org/docs/latest/api/sql/index.html#approx_percentile) in spark. Greenwald-Khanna algorithm refer to http://infolab.stanford.edu/~datar/courses/cs361a/papers/quantiles.pdf. [#46428](https://github.com/ClickHouse/ClickHouse/pull/46428) ([李扬](https://github.com/taiyang-li)).
* Add statement `SHOW COLUMNS` which shows distilled information from system.columns. [#48017](https://github.com/ClickHouse/ClickHouse/pull/48017) ([Robert Schulze](https://github.com/rschu1ze)).
* Added `LIGHTWEIGHT` and `PULL` modifiers for `SYSTEM SYNC REPLICA` query. `LIGHTWEIGHT` version waits for fetches and drop-ranges only (merges and mutations are ignored). `PULL` version pulls new entries from ZooKeeper and does not wait for them. Fixes [#47794](https://github.com/ClickHouse/ClickHouse/issues/47794). [#48085](https://github.com/ClickHouse/ClickHouse/pull/48085) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Add kafkaMurmurHash function for compatibility with Kafka DefaultPartitioner. Closes [#47834](https://github.com/ClickHouse/ClickHouse/issues/47834). [#48185](https://github.com/ClickHouse/ClickHouse/pull/48185) ([Nikolay Degterinsky](https://github.com/evillique)).
* Allow to easily create a user with the same grants as the current user by using `GRANT CURRENT GRANTS`. [#48262](https://github.com/ClickHouse/ClickHouse/pull/48262) ([pufit](https://github.com/pufit)).
* Add statistical aggregate function `kolmogorovSmirnovTest`. close [#48228](https://github.com/ClickHouse/ClickHouse/issues/48228). [#48325](https://github.com/ClickHouse/ClickHouse/pull/48325) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)).
* Added a `lost_part_count` column to the `system.replicas` table. The column value shows the total number of lost parts in the corresponding table. Value is stored in zookeeper and can be used instead of not persistent `ReplicatedDataLoss` profile event for monitoring. [#48526](https://github.com/ClickHouse/ClickHouse/pull/48526) ([Sergei Trifonov](https://github.com/serxa)).
* Add soundex function. Closes [#39880](https://github.com/ClickHouse/ClickHouse/issues/39880). [#48567](https://github.com/ClickHouse/ClickHouse/pull/48567) ([FriendLey](https://github.com/FriendLey)).
* Support map type for JSONExtract. [#48629](https://github.com/ClickHouse/ClickHouse/pull/48629) ([李扬](https://github.com/taiyang-li)).
* Add PrettyJSONEachRow format to output pretty JSON with new line delimieters and 4 space indents. [#48898](https://github.com/ClickHouse/ClickHouse/pull/48898) ([Kruglov Pavel](https://github.com/Avogar)).
* Add ParquetMetadata input format to read Parquet file metadata. [#48911](https://github.com/ClickHouse/ClickHouse/pull/48911) ([Kruglov Pavel](https://github.com/Avogar)).
#### Performance Improvement
* Reading files in Parquet format is now much faster. IO and decoding are parallelized (controlled by `max_threads` setting), and only required data ranges are read. [#47964](https://github.com/ClickHouse/ClickHouse/pull/47964) ([Michael Kolupaev](https://github.com/al13n321)).
* Only check dependencies if necessary when applying `ALTER TABLE` queries. [#48062](https://github.com/ClickHouse/ClickHouse/pull/48062) ([Raúl Marín](https://github.com/Algunenano)).
* Optimize function `mapUpdate`. [#48118](https://github.com/ClickHouse/ClickHouse/pull/48118) ([Anton Popov](https://github.com/CurtizJ)).
* Now an internal query to local replica is sent explicitly and data from it received through loopback interface. Setting `prefer_localhost_replica` is not respected for parallel replicas. This is needed for better scheduling and makes the code cleaner: the initiator is only responsible for coordinating of the reading process and merging results, continiously answering for requests while all the secondary queries read the data. Note: Using loopback interface is not so performant, otherwise some replicas could starve for tasks which could lead to even slower query execution and not utilizing all possible resources. The initialization of the coordinator is now even more lazy. All incoming requests contain the information about the reading algorithm we initialize the coordinator with it when first request comes. If any replica will decide to read with different algorithm - an exception will be thrown and a query will be aborted. [#48246](https://github.com/ClickHouse/ClickHouse/pull/48246) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Do not build set for the right side of `IN` clause with subquery when it is used only for analysis of skip indexes and they are disabled by setting (`use_skip_indexes=0`). Previously it might affect the performance of queries. [#48299](https://github.com/ClickHouse/ClickHouse/pull/48299) ([Anton Popov](https://github.com/CurtizJ)).
* Query processing is parallelized right after reading `FROM file(...)`. Related to [#38755](https://github.com/ClickHouse/ClickHouse/issues/38755). [#48525](https://github.com/ClickHouse/ClickHouse/pull/48525) ([Igor Nikonov](https://github.com/devcrafter)).
* Query processing is parallelized right after reading from a data source. Affected data sources are mostly simple or external storages like table functions `url`, `file`. [#48727](https://github.com/ClickHouse/ClickHouse/pull/48727) ([Igor Nikonov](https://github.com/devcrafter)).
* Using correct memory order for counter in `numebers_mt()`. [#48729](https://github.com/ClickHouse/ClickHouse/pull/48729) ([Igor Nikonov](https://github.com/devcrafter)).
* Lowered contention of ThreadPool mutex (may increase performance for a huge amount of small jobs). [#48750](https://github.com/ClickHouse/ClickHouse/pull/48750) ([Sergei Trifonov](https://github.com/serxa)).
* Simplify accounting of approximate size of granule in prefetched read pool. [#49051](https://github.com/ClickHouse/ClickHouse/pull/49051) ([Nikita Taranov](https://github.com/nickitat)).
#### Improvement
* Support config sections `keeper`/`keeper_server` as an alternative to `zookeeper`. Close [#34766](https://github.com/ClickHouse/ClickHouse/issues/34766) , [#34767](https://github.com/ClickHouse/ClickHouse/issues/34767). [#35113](https://github.com/ClickHouse/ClickHouse/pull/35113) ([李扬](https://github.com/taiyang-li)).
* Many issues in ClickHouse applications's help were fixed. Help is now written to stdout from all tools. Status code for `clickhouse help` invocation is now 0. Updated help for `clickhouse-local`, `clickhouse-benchmark`, `clickhouse-client`, `clickhouse hash`, `clickhouse su`, `clickhouse-install`. [#45819](https://github.com/ClickHouse/ClickHouse/pull/45819) ([Ilya Yatsishin](https://github.com/qoega)).
* Entries in the query cache are now squashed to max_block_size and compressed. [#45912](https://github.com/ClickHouse/ClickHouse/pull/45912) ([Robert Schulze](https://github.com/rschu1ze)).
* It is possible to set _secure_ flag in named_collections for a dictionary with a ClickHouse table source. Addresses [#38450](https://github.com/ClickHouse/ClickHouse/issues/38450) . [#46323](https://github.com/ClickHouse/ClickHouse/pull/46323) ([Ilya Golshtein](https://github.com/ilejn)).
* Functions replaceOne(), replaceAll(), replaceRegexpOne() and replaceRegexpAll() can now be called with non-const pattern and replacement arguments. [#46589](https://github.com/ClickHouse/ClickHouse/pull/46589) ([Robert Schulze](https://github.com/rschu1ze)).
* Bump internal ZSTD from 1.5.4 to 1.5.5. [#46797](https://github.com/ClickHouse/ClickHouse/pull/46797) ([Robert Schulze](https://github.com/rschu1ze)).
* If we run a mutation with IN (subquery) like this: `ALTER TABLE t UPDATE col='new value' WHERE id IN (SELECT id FROM huge_table)` and the table `t` has multiple parts than for each part a set for subquery `SELECT id FROM huge_table` is built in memory. And if there are many parts then this might consume a lot of memory (and lead to an OOM) and CPU. The solution is to introduce a short-lived cache of sets that are currently being built by mutation tasks. If another task of the same mutation is executed concurrently it can lookup the set in the cache, wait for it be be built and reuse it. [#46835](https://github.com/ClickHouse/ClickHouse/pull/46835) ([Alexander Gololobov](https://github.com/davenger)).
* Added configurable retries for all operations with [Zoo]Keeper for Backup queries. [#47224](https://github.com/ClickHouse/ClickHouse/pull/47224) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Add async connection to socket and async writing to socket. Make creating connections and sending query/external tables async across shards. Refactor code with fibers. Closes [#46931](https://github.com/ClickHouse/ClickHouse/issues/46931). We will be able to increase `connect_timeout_with_failover_ms` by default after this PR (https://github.com/ClickHouse/ClickHouse/issues/5188). [#47229](https://github.com/ClickHouse/ClickHouse/pull/47229) ([Kruglov Pavel](https://github.com/Avogar)).
* Formatter '%M' in function formatDateTime() now prints the month name instead of the minutes. This makes the behavior consistent with MySQL. The previous behavior can be restored using setting "formatdatetime_parsedatetime_m_is_month_name = 0". [#47246](https://github.com/ClickHouse/ClickHouse/pull/47246) ([Robert Schulze](https://github.com/rschu1ze)).
* Several improvements around data lakes: - Make StorageIceberg work with non-partitioned data. - Support Iceberg format version V2 (previously only V1 was supported) - Support reading partitioned data for DeltaLake/Hudi - Faster reading of DeltaLake metadata by using Delta's checkpoint files - Fixed incorrect Hudi reads: previously it incorrectly chose which data to read and therefore was able to read correctly only small size tables - Made these engines to pickup updates of changed data (previously the state was set on table creation) - Make proper testing for Iceberg/DeltaLake/Hudi using spark. [#47307](https://github.com/ClickHouse/ClickHouse/pull/47307) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Enable `use_environment_credentials` for S3 by default, so the entire provider chain is constructed by default. [#47397](https://github.com/ClickHouse/ClickHouse/pull/47397) ([Antonio Andelic](https://github.com/antonio2368)).
* Currently, the JSON_VALUE function is similar as spark's get_json_object function, which support to get value from json string by a path like '$.key'. But still has something different - 1. in spark's get_json_object will return null while the path is not exist, but in JSON_VALUE will return empty string; - 2. in spark's get_json_object will return a complext type value, such as a json object/array value, but in JSON_VALUE will return empty string. [#47494](https://github.com/ClickHouse/ClickHouse/pull/47494) ([KevinyhZou](https://github.com/KevinyhZou)).
* Add CNF/constraint optimizer in new analyzer. [#47617](https://github.com/ClickHouse/ClickHouse/pull/47617) ([Antonio Andelic](https://github.com/antonio2368)).
* For use_structure_from_insertion_table_in_table_functions more flexible insert table structure propagation to table function. Fixed bug with name mapping and using virtual columns. No more need for 'auto' setting. [#47962](https://github.com/ClickHouse/ClickHouse/pull/47962) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Do not continue retrying to connect to ZK if the query is killed or over limits. [#47985](https://github.com/ClickHouse/ClickHouse/pull/47985) ([Raúl Marín](https://github.com/Algunenano)).
* Added functions to work with columns of type `Map`: `mapConcat`, `mapSort`, `mapExists`. [#48071](https://github.com/ClickHouse/ClickHouse/pull/48071) ([Anton Popov](https://github.com/CurtizJ)).
* Support Enum output/input in BSONEachRow, allow all map key types and avoid extra calculations on output. [#48122](https://github.com/ClickHouse/ClickHouse/pull/48122) ([Kruglov Pavel](https://github.com/Avogar)).
* Support more ClickHouse types in ORC/Arrow/Parquet formats: Enum(8|16), (U)Int(128|256), Decimal256 (for ORC), allow reading IPv4 from Int32 values (ORC outputs IPv4 as Int32 and we couldn't read it back), fix reading Nullable(IPv6) from binary data for ORC. [#48126](https://github.com/ClickHouse/ClickHouse/pull/48126) ([Kruglov Pavel](https://github.com/Avogar)).
* Add columns `perform_ttl_move_on_insert`, `load_balancing` for table `system.storage_policies`, modify column `volume_type` type to `enum8`. [#48167](https://github.com/ClickHouse/ClickHouse/pull/48167) ([lizhuoyu5](https://github.com/lzydmxy)).
* Added support for `BACKUP ALL` command which backups all tables and databases, including temporary and system ones. [#48189](https://github.com/ClickHouse/ClickHouse/pull/48189) ([Vitaly Baranov](https://github.com/vitlibar)).
* Function mapFromArrays support map type as input. [#48207](https://github.com/ClickHouse/ClickHouse/pull/48207) ([李扬](https://github.com/taiyang-li)).
* The output of some SHOW PROCESSLIST is now sorted. [#48241](https://github.com/ClickHouse/ClickHouse/pull/48241) ([Robert Schulze](https://github.com/rschu1ze)).
* Per-query/per-server throttling for remote IO/local IO/BACKUPs (server settings: `max_remote_read_network_bandwidth_for_server`, `max_remote_write_network_bandwidth_for_server`, `max_local_read_bandwidth_for_server`, `max_local_write_bandwidth_for_server`, `max_backup_bandwidth_for_server`, settings: `max_remote_read_network_bandwidth`, `max_remote_write_network_bandwidth`, `max_local_read_bandwidth`, `max_local_write_bandwidth`, `max_backup_bandwidth`). [#48242](https://github.com/ClickHouse/ClickHouse/pull/48242) ([Azat Khuzhin](https://github.com/azat)).
* Support more types in CapnProto format: Map, (U)Int(128|256), Decimal(128|256). Allow integer conversions during input/output. [#48257](https://github.com/ClickHouse/ClickHouse/pull/48257) ([Kruglov Pavel](https://github.com/Avogar)).
* It is now possible to define per-user quotas in the query cache. [#48284](https://github.com/ClickHouse/ClickHouse/pull/48284) ([Robert Schulze](https://github.com/rschu1ze)).
* Don't throw CURRENT_WRITE_BUFFER_IS_EXHAUSTED for normal behaviour. [#48288](https://github.com/ClickHouse/ClickHouse/pull/48288) ([Raúl Marín](https://github.com/Algunenano)).
* Add new setting `keeper_map_strict_mode` which enforces extra guarantees on operations made on top of `KeeperMap` tables. [#48293](https://github.com/ClickHouse/ClickHouse/pull/48293) ([Antonio Andelic](https://github.com/antonio2368)).
* Check primary key type for simple dictionary is native unsigned integer type Add setting `check_dictionary_primary_key ` for compatibility(set `check_dictionary_primary_key =false` to disable checking). [#48335](https://github.com/ClickHouse/ClickHouse/pull/48335) ([lizhuoyu5](https://github.com/lzydmxy)).
* Don't replicate mutations for `KeeperMap` because it's unnecessary. [#48354](https://github.com/ClickHouse/ClickHouse/pull/48354) ([Antonio Andelic](https://github.com/antonio2368)).
* Allow write/read unnamed tuple as nested Message in Protobuf format. Tuple elements and Message fields are mathced by position. [#48390](https://github.com/ClickHouse/ClickHouse/pull/48390) ([Kruglov Pavel](https://github.com/Avogar)).
* Support `additional_table_filters` and `additional_result_filter` settings in the new planner. Also, add a documentation entry for `additional_result_filter`. [#48405](https://github.com/ClickHouse/ClickHouse/pull/48405) ([Dmitry Novik](https://github.com/novikd)).
* Parsedatetime now understands format string '%f' (fractional seconds). [#48420](https://github.com/ClickHouse/ClickHouse/pull/48420) ([Robert Schulze](https://github.com/rschu1ze)).
* Format string "%f" in formatDateTime() now prints "000000" if the formatted value has no fractional seconds, the previous behavior (single zero) can be restored using setting "formatdatetime_f_prints_single_zero = 1". [#48422](https://github.com/ClickHouse/ClickHouse/pull/48422) ([Robert Schulze](https://github.com/rschu1ze)).
* Don't replicate DELETE and TRUNCATE for KeeperMap. [#48434](https://github.com/ClickHouse/ClickHouse/pull/48434) ([Antonio Andelic](https://github.com/antonio2368)).
* Generate valid Decimals and Bools in generateRandom function. [#48436](https://github.com/ClickHouse/ClickHouse/pull/48436) ([Kruglov Pavel](https://github.com/Avogar)).
* Allow trailing commas in expression list of SELECT query, for example `SELECT a, b, c, FROM table`. Closes [#37802](https://github.com/ClickHouse/ClickHouse/issues/37802). [#48438](https://github.com/ClickHouse/ClickHouse/pull/48438) ([Nikolay Degterinsky](https://github.com/evillique)).
* Override `CLICKHOUSE_USER` and `CLICKHOUSE_PASSWORD` environment variables with `--user` and `--password` client parameters. Closes [#38909](https://github.com/ClickHouse/ClickHouse/issues/38909). [#48440](https://github.com/ClickHouse/ClickHouse/pull/48440) ([Nikolay Degterinsky](https://github.com/evillique)).
* Added retries to loading of data parts in `MergeTree` tables in case of retryable errors. [#48442](https://github.com/ClickHouse/ClickHouse/pull/48442) ([Anton Popov](https://github.com/CurtizJ)).
* Add support for `Date`, `Date32`, `DateTime`, `DateTime64` data types to `arrayMin`, `arrayMax`, `arrayDifference` functions. Closes [#21645](https://github.com/ClickHouse/ClickHouse/issues/21645). [#48445](https://github.com/ClickHouse/ClickHouse/pull/48445) ([Nikolay Degterinsky](https://github.com/evillique)).
* Reduce memory usage for multiple `ALTER DELETE` mutations. [#48522](https://github.com/ClickHouse/ClickHouse/pull/48522) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Primary/secondary indices and sorting keys with identical expressions are now rejected. This behavior can be disabled using setting `allow_suspicious_indices`. [#48536](https://github.com/ClickHouse/ClickHouse/pull/48536) ([凌涛](https://github.com/lingtaolf)).
* Just fix small typo in comment around `lockForAlter` method in `IStorage.h`. [#48559](https://github.com/ClickHouse/ClickHouse/pull/48559) ([artem-pershin](https://github.com/artem-pershin)).
* Add support for `{server_uuid}` macro. It is useful for identifying replicas in autoscaled clusters when new replicas are constantly added and removed in runtime. This closes [#48554](https://github.com/ClickHouse/ClickHouse/issues/48554). [#48563](https://github.com/ClickHouse/ClickHouse/pull/48563) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* The installation script will create a hard link instead of copying if it is possible. [#48578](https://github.com/ClickHouse/ClickHouse/pull/48578) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Support `SHOW TABLE` syntax meaning the same as `SHOW CREATE TABLE`. Closes [#48580](https://github.com/ClickHouse/ClickHouse/issues/48580). [#48591](https://github.com/ClickHouse/ClickHouse/pull/48591) ([flynn](https://github.com/ucasfl)).
* HTTP temporary buffer support working with fs cache. [#48664](https://github.com/ClickHouse/ClickHouse/pull/48664) ([Vladimir C](https://github.com/vdimir)).
* Make Schema inference works for `CREATE AS SELECT`. Closes [#47599](https://github.com/ClickHouse/ClickHouse/issues/47599). [#48679](https://github.com/ClickHouse/ClickHouse/pull/48679) ([flynn](https://github.com/ucasfl)).
* Added a `replicated_max_mutations_in_one_entry` setting for `ReplicatedMergeTree` that allows limiting the number of mutation commands per one `MUTATE_PART` entry (default is 10000). [#48731](https://github.com/ClickHouse/ClickHouse/pull/48731) ([Alexander Tokmakov](https://github.com/tavplubix)).
* In AggregateFunction types, don't count unused arena bytes as `read_bytes`. [#48745](https://github.com/ClickHouse/ClickHouse/pull/48745) ([Raúl Marín](https://github.com/Algunenano)).
* Fix some mysql related settings not being handled with mysql dictionary source + named collection. Closes [#48402](https://github.com/ClickHouse/ClickHouse/issues/48402). [#48759](https://github.com/ClickHouse/ClickHouse/pull/48759) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix squashing in query cache. [#48763](https://github.com/ClickHouse/ClickHouse/pull/48763) ([Robert Schulze](https://github.com/rschu1ze)).
* Support following new jsonpath format - '$.1key', path element begins with number - '$[key]', '$[“key”]', '$[\\\'key\\\']', '$["key 123"]', path element embraced in []. [#48768](https://github.com/ClickHouse/ClickHouse/pull/48768) ([lgbo](https://github.com/lgbo-ustc)).
* If a user set `max_single_part_upload_size` to a very large value, it can lead to a crash due to a bug in the AWS S3 SDK. This fixes [#47679](https://github.com/ClickHouse/ClickHouse/issues/47679). [#48816](https://github.com/ClickHouse/ClickHouse/pull/48816) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Not for changelog. [#48824](https://github.com/ClickHouse/ClickHouse/pull/48824) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Fix data race in `StorageRabbitMQ` ([report](https://pastila.nl/?004f7100/de1505289ab5bb355e67ebe6c7cc8707)), refactor the code. [#48845](https://github.com/ClickHouse/ClickHouse/pull/48845) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add aliases `name` and `part_name` form `system.parts` and `system.part_log`. Closes [#48718](https://github.com/ClickHouse/ClickHouse/issues/48718). [#48850](https://github.com/ClickHouse/ClickHouse/pull/48850) ([sichenzhao](https://github.com/sichenzhao)).
* Functions "arrayDifferenceSupport()", "arrayCumSum()" and "arrayCumSumNonNegative()" now support input arrays of wide integer types (U)Int128/256. [#48866](https://github.com/ClickHouse/ClickHouse/pull/48866) ([cluster](https://github.com/infdahai)).
* Multi-line history in clickhouse-client is now no longer padded. This makes pasting more natural. [#48870](https://github.com/ClickHouse/ClickHouse/pull/48870) ([Joanna Hulboj](https://github.com/jh0x)).
* Not for changelog. [#48873](https://github.com/ClickHouse/ClickHouse/pull/48873) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Implement a slight improvement for the rare case when ClickHouse is run inside LXC and LXCFS is used. The LXCFS has an issue: sometimes it returns an error "Transport endpoint is not connected" on reading from the file inside `/proc`. This error was correctly logged into ClickHouse's server log. We have additionally workaround this issue by reopening a file. This is a minuscule change. [#48922](https://github.com/ClickHouse/ClickHouse/pull/48922) ([Real](https://github.com/RunningXie)).
* Improve memory accounting for prefetches. Randomise prefetch settings In CI. [#48973](https://github.com/ClickHouse/ClickHouse/pull/48973) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Correctly set headers for native copy operations on GCS. [#48981](https://github.com/ClickHouse/ClickHouse/pull/48981) ([Antonio Andelic](https://github.com/antonio2368)).
* Add support for specifying setting names in the command line with dashes instead of underscores, for example, `--max-threads` instead of `--max_threads`. Additionally, support Unicode dash characters like `—` instead of `--` - this is useful when you communicate with a team in another company, and a manager from that team copy-pasted code from MS Word. [#48985](https://github.com/ClickHouse/ClickHouse/pull/48985) ([alekseygolub](https://github.com/alekseygolub)).
* Add fallback to password authentication when authentication with SSL user certificate has failed. Closes [#48974](https://github.com/ClickHouse/ClickHouse/issues/48974). [#48989](https://github.com/ClickHouse/ClickHouse/pull/48989) ([Nikolay Degterinsky](https://github.com/evillique)).
* Increase default value for `connect_timeout_with_failover_ms` to 1000 ms (because of adding async connections in https://github.com/ClickHouse/ClickHouse/pull/47229) . Closes [#5188](https://github.com/ClickHouse/ClickHouse/issues/5188). [#49009](https://github.com/ClickHouse/ClickHouse/pull/49009) ([Kruglov Pavel](https://github.com/Avogar)).
* Improve the embedded dashboard. Close [#46671](https://github.com/ClickHouse/ClickHouse/issues/46671). [#49036](https://github.com/ClickHouse/ClickHouse/pull/49036) ([Kevin Zhang](https://github.com/Kinzeng)).
* Add profile events for log messages, so you can easily see the count of log messages by severity. [#49042](https://github.com/ClickHouse/ClickHouse/pull/49042) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* `bitCount` function support `FixedString` and `String` data type. [#49044](https://github.com/ClickHouse/ClickHouse/pull/49044) ([flynn](https://github.com/ucasfl)).
* In previous versions, the `LineAsString` format worked inconsistently when the parallel parsing was enabled or not, in presence of DOS or MacOS Classic line breaks. This closes [#49039](https://github.com/ClickHouse/ClickHouse/issues/49039). [#49052](https://github.com/ClickHouse/ClickHouse/pull/49052) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* The exception message about the unparsed query parameter will also tell about the name of the parameter. Reimplement [#48878](https://github.com/ClickHouse/ClickHouse/issues/48878). Close [#48772](https://github.com/ClickHouse/ClickHouse/issues/48772). [#49061](https://github.com/ClickHouse/ClickHouse/pull/49061) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Added field `rows` with number of rows parsed from asynchronous insert to `system.asynchronous_insert_log`. [#49120](https://github.com/ClickHouse/ClickHouse/pull/49120) ([Anton Popov](https://github.com/CurtizJ)).
* 1. Bump Intel QPL from v1.0.0 to v1.1.0 (fixes build issue [#47877](https://github.com/ClickHouse/ClickHouse/issues/47877)) 2. the DEFLATE_QPL codec now respects the maximum hardware jobs returned by libaccel_config. [#49126](https://github.com/ClickHouse/ClickHouse/pull/49126) ([jasperzhu](https://github.com/jinjunzh)).
#### Build/Testing/Packaging Improvement
* Reduce the number of dependencies in the header files to speed up the build. [#47984](https://github.com/ClickHouse/ClickHouse/pull/47984) ([Dmitry Novik](https://github.com/novikd)).
* Randomize compression of marks and indices in tests. [#48286](https://github.com/ClickHouse/ClickHouse/pull/48286) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Randomize vertical merges from compact to wide parts in tests. [#48287](https://github.com/ClickHouse/ClickHouse/pull/48287) ([Raúl Marín](https://github.com/Algunenano)).
* With the current approach, all ports are calculated at the beginning and could overlap or even be highjacked, see [the report](https://s3.amazonaws.com/clickhouse-test-reports/46793/02928ae50c52f31ce8e5bfa99eb1b5db046f4a4f/integration_tests__release__[1/2]/integration_run_parallel8_0.log) for `port is already allocated`. It's possibly the reason for [#45368](https://github.com/ClickHouse/ClickHouse/issues/45368). [#48393](https://github.com/ClickHouse/ClickHouse/pull/48393) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Update time zones. The following were updated: Africa/Cairo, Africa/Casablanca, Africa/El_Aaiun, America/Bogota, America/Cambridge_Bay, America/Ciudad_Juarez, America/Godthab, America/Inuvik, America/Iqaluit, America/Nuuk, America/Ojinaga, America/Pangnirtung, America/Rankin_Inlet, America/Resolute, America/Whitehorse, America/Yellowknife, Asia/Gaza, Asia/Hebron, Asia/Kuala_Lumpur, Asia/Singapore, Canada/Yukon, Egypt, Europe/Kirov, Europe/Volgograd, Singapore. [#48572](https://github.com/ClickHouse/ClickHouse/pull/48572) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Support for CRC32 checksum in HDFS. Fix performance issues. [#48614](https://github.com/ClickHouse/ClickHouse/pull/48614) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Remove remainders of GCC support. [#48671](https://github.com/ClickHouse/ClickHouse/pull/48671) ([Robert Schulze](https://github.com/rschu1ze)).
* Add CI run with new analyzer infrastructure enabled. [#48719](https://github.com/ClickHouse/ClickHouse/pull/48719) ([Dmitry Novik](https://github.com/novikd)).
* Not for changelog. [#48879](https://github.com/ClickHouse/ClickHouse/pull/48879) ([larryluogit](https://github.com/larryluogit)).
* After the recent update, the `dockerd` requires `--tlsverify=false` together with the http port explicitly. [#48924](https://github.com/ClickHouse/ClickHouse/pull/48924) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Run more functional tests concurrently. [#48970](https://github.com/ClickHouse/ClickHouse/pull/48970) ([alesapin](https://github.com/alesapin)).
* Fix glibc compatibility check: replace `preadv` from musl. [#49144](https://github.com/ClickHouse/ClickHouse/pull/49144) ([alesapin](https://github.com/alesapin)).
* Use position independent encoding/code for sanitizers (at least msan :D) build to avoid issues with maximum relocation size. [#49145](https://github.com/ClickHouse/ClickHouse/pull/49145) ([alesapin](https://github.com/alesapin)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Fix system.query_views_log for MVs that are pushed from background threads [#46668](https://github.com/ClickHouse/ClickHouse/pull/46668) ([Azat Khuzhin](https://github.com/azat)).
* Fix several `RENAME COLUMN` bugs [#46946](https://github.com/ClickHouse/ClickHouse/pull/46946) ([alesapin](https://github.com/alesapin)).
* Fix minor hiliting issues in clickhouse-format [#47610](https://github.com/ClickHouse/ClickHouse/pull/47610) ([Natasha Murashkina](https://github.com/murfel)).
* Fix crash for uploading parts which size is greater then INT_MAX to S3 [#47693](https://github.com/ClickHouse/ClickHouse/pull/47693) ([Azat Khuzhin](https://github.com/azat)).
* Fix overflow in sparkbar function [#48121](https://github.com/ClickHouse/ClickHouse/pull/48121) ([Vladimir C](https://github.com/vdimir)).
* Fix race in StorageS3 [#48190](https://github.com/ClickHouse/ClickHouse/pull/48190) ([Anton Popov](https://github.com/CurtizJ)).
* Remove a feature [#48195](https://github.com/ClickHouse/ClickHouse/pull/48195) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix alter formatting (minor) [#48289](https://github.com/ClickHouse/ClickHouse/pull/48289) ([Natasha Murashkina](https://github.com/murfel)).
* Fix cpu usage in rabbitmq (was worsened in 23.2 after [#44404](https://github.com/ClickHouse/ClickHouse/issues/44404)) [#48311](https://github.com/ClickHouse/ClickHouse/pull/48311) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix ThreadPool for DistributedSink and use StrongTypedef for CurrentMetrics/ProfileEvents/StatusInfo to avoid further errors [#48314](https://github.com/ClickHouse/ClickHouse/pull/48314) ([Azat Khuzhin](https://github.com/azat)).
* Fix crash in EXPLAIN PIPELINE for Merge over Distributed [#48320](https://github.com/ClickHouse/ClickHouse/pull/48320) ([Azat Khuzhin](https://github.com/azat)).
* Check node for Backup Restore concurrency [#48342](https://github.com/ClickHouse/ClickHouse/pull/48342) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
* close client [#48347](https://github.com/ClickHouse/ClickHouse/pull/48347) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Fix serializing LowCardinality as Arrow dictionary [#48361](https://github.com/ClickHouse/ClickHouse/pull/48361) ([Kruglov Pavel](https://github.com/Avogar)).
* Reset downloader for cache file segment in TemporaryFileStream [#48386](https://github.com/ClickHouse/ClickHouse/pull/48386) ([Vladimir C](https://github.com/vdimir)).
* Fix possible SYSTEM SYNC REPLICA stuck in case of DROP/REPLACE PARTITION [#48391](https://github.com/ClickHouse/ClickHouse/pull/48391) ([Azat Khuzhin](https://github.com/azat)).
* ClickHouse startup error when loading a distributed table that depends on a dictionary [#48419](https://github.com/ClickHouse/ClickHouse/pull/48419) ([MikhailBurdukov](https://github.com/MikhailBurdukov)).
* Don't check dependencies when renaming system tables automatically [#48431](https://github.com/ClickHouse/ClickHouse/pull/48431) ([Raúl Marín](https://github.com/Algunenano)).
* Some fixes for parallel replicas [#48433](https://github.com/ClickHouse/ClickHouse/pull/48433) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Update only affected rows in KV storage [#48435](https://github.com/ClickHouse/ClickHouse/pull/48435) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix possible segfault in cache [#48469](https://github.com/ClickHouse/ClickHouse/pull/48469) ([Kseniia Sumarokova](https://github.com/kssenii)).
* toTimeZone function throw an error when no constant string is provided [#48471](https://github.com/ClickHouse/ClickHouse/pull/48471) ([Jordi Villar](https://github.com/jrdi)).
* Fix logical error with IPv4 in Protobuf, add support for Date32 [#48486](https://github.com/ClickHouse/ClickHouse/pull/48486) ([Kruglov Pavel](https://github.com/Avogar)).
* "changed" flag in system.settings is calculated incorrectly for settings with multiple values [#48516](https://github.com/ClickHouse/ClickHouse/pull/48516) ([MikhailBurdukov](https://github.com/MikhailBurdukov)).
* Fix storage `Memory` with enabled compression [#48517](https://github.com/ClickHouse/ClickHouse/pull/48517) ([Anton Popov](https://github.com/CurtizJ)).
* Fix bracketed-paste mode messing up password input in client reconnect [#48528](https://github.com/ClickHouse/ClickHouse/pull/48528) ([Michael Kolupaev](https://github.com/al13n321)).
* Avoid sending `nullptr` to `memcpy` in `copyStringInArena` [#48532](https://github.com/ClickHouse/ClickHouse/pull/48532) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix nested map for keys of IP and UUID types [#48556](https://github.com/ClickHouse/ClickHouse/pull/48556) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Fix uncaught exception in case of parallel loader for hashed dictionaries [#48571](https://github.com/ClickHouse/ClickHouse/pull/48571) ([Azat Khuzhin](https://github.com/azat)).
* `groupArray` returns cannot be nullable [#48593](https://github.com/ClickHouse/ClickHouse/pull/48593) ([lgbo](https://github.com/lgbo-ustc)).
* Fix bug in Keeper when a node is not created with scheme `auth` in ACL sometimes. [#48595](https://github.com/ClickHouse/ClickHouse/pull/48595) ([Aleksei Filatov](https://github.com/aalexfvk)).
* Fix IPv4 comparable with UInt [#48611](https://github.com/ClickHouse/ClickHouse/pull/48611) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
* Fix possible error from cache [#48636](https://github.com/ClickHouse/ClickHouse/pull/48636) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix async inserts with empty data [#48663](https://github.com/ClickHouse/ClickHouse/pull/48663) ([Anton Popov](https://github.com/CurtizJ)).
* Fix table dependencies in case of failed RENAME TABLE [#48683](https://github.com/ClickHouse/ClickHouse/pull/48683) ([Azat Khuzhin](https://github.com/azat)).
* Fix zero-copy-replication on encrypted disks. [#48741](https://github.com/ClickHouse/ClickHouse/pull/48741) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fix skip_unavailable_shards in case of unavailable hosts [#48771](https://github.com/ClickHouse/ClickHouse/pull/48771) ([Azat Khuzhin](https://github.com/azat)).
* Fix key condition on duplicate primary keys [#48838](https://github.com/ClickHouse/ClickHouse/pull/48838) ([Amos Bird](https://github.com/amosbird)).
* Fix for race in ZooKeeper when joining send_thread/receive_thread [#48849](https://github.com/ClickHouse/ClickHouse/pull/48849) ([Alexander Gololobov](https://github.com/davenger)).
* Fix unexpected part name error when trying to drop a ignored detached part with zero copy replication [#48862](https://github.com/ClickHouse/ClickHouse/pull/48862) ([Michael Lex](https://github.com/mlex)).
* Fix reading Date32 Parquet/Arrow column into not Date32 column [#48864](https://github.com/ClickHouse/ClickHouse/pull/48864) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix UNKNOWN_IDENTIFIER error while select from table with row policy and column with dots [#48976](https://github.com/ClickHouse/ClickHouse/pull/48976) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix aggregate empty string error [#48999](https://github.com/ClickHouse/ClickHouse/pull/48999) ([LiuNeng](https://github.com/liuneng1994)).
* Fix postgres database setting [#49100](https://github.com/ClickHouse/ClickHouse/pull/49100) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix flaky test_cache_with_full_disk_space [#49110](https://github.com/ClickHouse/ClickHouse/pull/49110) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix "prepared statement insert already exists" [#49154](https://github.com/ClickHouse/ClickHouse/pull/49154) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix replace[Regexp]{One,All}() with const haystacks [#49220](https://github.com/ClickHouse/ClickHouse/pull/49220) ([Robert Schulze](https://github.com/rschu1ze)).
#### Build Improvement
* Fixed hashing issue in creating partition IDs for s390x. [#48134](https://github.com/ClickHouse/ClickHouse/pull/48134) ([Harry Lee](https://github.com/HarryLeeIBM)).
#### NO CL ENTRY
* NO CL ENTRY: 'Revert "Randomize JIT settings in tests"'. [#48277](https://github.com/ClickHouse/ClickHouse/pull/48277) ([Alexander Tokmakov](https://github.com/tavplubix)).
* NO CL ENTRY: 'Fix test "02494_query_cache_drop.sql"'. [#48358](https://github.com/ClickHouse/ClickHouse/pull/48358) ([Anton Popov](https://github.com/CurtizJ)).
* NO CL ENTRY: 'Revert "Check simple dictionary key is native unsigned integer"'. [#48732](https://github.com/ClickHouse/ClickHouse/pull/48732) ([Alexander Tokmakov](https://github.com/tavplubix)).
* NO CL ENTRY: 'Revert "Make Schema inference works for CREATE AS SELECT"'. [#48758](https://github.com/ClickHouse/ClickHouse/pull/48758) ([pufit](https://github.com/pufit)).
* NO CL ENTRY: 'Revert "Add MemoryTracker for the background tasks"'. [#48760](https://github.com/ClickHouse/ClickHouse/pull/48760) ([Alexander Tokmakov](https://github.com/tavplubix)).
* NO CL ENTRY: 'Revert "Added tests for ClickHouse apps help and fixed help issues"'. [#48991](https://github.com/ClickHouse/ClickHouse/pull/48991) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* NO CL ENTRY: 'Revert "Adapt marks count for prefetch read pool"'. [#49068](https://github.com/ClickHouse/ClickHouse/pull/49068) ([Nikita Taranov](https://github.com/nickitat)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* merge and mutation make thread group for setting memory trackers right [#47104](https://github.com/ClickHouse/ClickHouse/pull/47104) ([Sema Checherinda](https://github.com/CheSema)).
* Query plan: update sort description [#47319](https://github.com/ClickHouse/ClickHouse/pull/47319) ([Igor Nikonov](https://github.com/devcrafter)).
* Sqllogic [#47784](https://github.com/ClickHouse/ClickHouse/pull/47784) ([Sema Checherinda](https://github.com/CheSema)).
* Fix race between DROP MatView and RESTART REPLICAS [#47863](https://github.com/ClickHouse/ClickHouse/pull/47863) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Merge [#35113](https://github.com/ClickHouse/ClickHouse/issues/35113) [#47934](https://github.com/ClickHouse/ClickHouse/pull/47934) ([Antonio Andelic](https://github.com/antonio2368)).
* Add a test for ClientInfo initial_query_start_time in inter-server mode [#48036](https://github.com/ClickHouse/ClickHouse/pull/48036) ([Azat Khuzhin](https://github.com/azat)).
* Make custom key for parallel replicas work in new analyzer [#48054](https://github.com/ClickHouse/ClickHouse/pull/48054) ([Antonio Andelic](https://github.com/antonio2368)).
* throw exception while non-parametric functions having parameters [#48115](https://github.com/ClickHouse/ClickHouse/pull/48115) ([save-my-heart](https://github.com/save-my-heart)).
* Move FunctionsJSON implementation to header file [#48142](https://github.com/ClickHouse/ClickHouse/pull/48142) ([DimasKovas](https://github.com/DimasKovas)).
* Use ThreadPool in PipelineExecutor [#48146](https://github.com/ClickHouse/ClickHouse/pull/48146) ([Azat Khuzhin](https://github.com/azat)).
* Add sanity checks for writing number in variable length format (resubmit) [#48154](https://github.com/ClickHouse/ClickHouse/pull/48154) ([Azat Khuzhin](https://github.com/azat)).
* Try fix 02151_hash_table_sizes_stats.sh test [#48178](https://github.com/ClickHouse/ClickHouse/pull/48178) ([Nikita Taranov](https://github.com/nickitat)).
* Add scripts for sparse checkout of some contribs [#48183](https://github.com/ClickHouse/ClickHouse/pull/48183) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Do not take lock for shared context in setTempDataOnDisk [#48219](https://github.com/ClickHouse/ClickHouse/pull/48219) ([Vladimir C](https://github.com/vdimir)).
* parseDateTime[InJodaSyntax](): Require format argument [#48222](https://github.com/ClickHouse/ClickHouse/pull/48222) ([Robert Schulze](https://github.com/rschu1ze)).
* Do not partially cancel processors added from expand pipeline. [#48231](https://github.com/ClickHouse/ClickHouse/pull/48231) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix some tests [#48267](https://github.com/ClickHouse/ClickHouse/pull/48267) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix compiling examples without Hive [#48269](https://github.com/ClickHouse/ClickHouse/pull/48269) ([Azat Khuzhin](https://github.com/azat)).
* In messages, put values into quotes [#48271](https://github.com/ClickHouse/ClickHouse/pull/48271) ([Vadim Chekan](https://github.com/vchekan)).
* Fix 01710_projection_optimize_materialize flakiness [#48276](https://github.com/ClickHouse/ClickHouse/pull/48276) ([Azat Khuzhin](https://github.com/azat)).
* Fix UB (signed integer overflow) in StorageMergeTree::backupData() [#48278](https://github.com/ClickHouse/ClickHouse/pull/48278) ([Azat Khuzhin](https://github.com/azat)).
* Update version after release [#48279](https://github.com/ClickHouse/ClickHouse/pull/48279) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Update version_date.tsv and changelogs after v23.3.1.2823-lts [#48281](https://github.com/ClickHouse/ClickHouse/pull/48281) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Small follow-up to [#48017](https://github.com/ClickHouse/ClickHouse/issues/48017) [#48292](https://github.com/ClickHouse/ClickHouse/pull/48292) ([Robert Schulze](https://github.com/rschu1ze)).
* Try to update arrow library to release 11.0.0 [#48294](https://github.com/ClickHouse/ClickHouse/pull/48294) ([Kruglov Pavel](https://github.com/Avogar)).
* fix test numbers again 2 [#48295](https://github.com/ClickHouse/ClickHouse/pull/48295) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix: copy forgotten show_secrets in FormatSettings semi-copy-ctor [#48297](https://github.com/ClickHouse/ClickHouse/pull/48297) ([Natasha Murashkina](https://github.com/murfel)).
* Do not remove inputs from maybe compiled DAG. [#48303](https://github.com/ClickHouse/ClickHouse/pull/48303) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Update version_date.tsv and changelogs after v22.3.20.29-lts [#48304](https://github.com/ClickHouse/ClickHouse/pull/48304) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Update version_date.tsv and changelogs after v22.12.6.22-stable, v22.3.20.29-lts [#48305](https://github.com/ClickHouse/ClickHouse/pull/48305) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Merging [#46323](https://github.com/ClickHouse/ClickHouse/issues/46323) [#48312](https://github.com/ClickHouse/ClickHouse/pull/48312) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Follow-up to [#47863](https://github.com/ClickHouse/ClickHouse/issues/47863) [#48315](https://github.com/ClickHouse/ClickHouse/pull/48315) ([Alexander Tokmakov](https://github.com/tavplubix)).
* test / some complex query (it fails with analyzer enabled) [#48324](https://github.com/ClickHouse/ClickHouse/pull/48324) ([Denny Crane](https://github.com/den-crane)).
* Fix constraints after merge [#48328](https://github.com/ClickHouse/ClickHouse/pull/48328) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add logging for concurrency checks for backups [#48337](https://github.com/ClickHouse/ClickHouse/pull/48337) ([Vitaly Baranov](https://github.com/vitlibar)).
* Update version_date.tsv and changelogs after v23.1.6.42-stable [#48345](https://github.com/ClickHouse/ClickHouse/pull/48345) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Update version_date.tsv and changelogs after v23.2.5.46-stable [#48346](https://github.com/ClickHouse/ClickHouse/pull/48346) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Fix lambda type resolution [#48355](https://github.com/ClickHouse/ClickHouse/pull/48355) ([Dmitry Novik](https://github.com/novikd)).
* Avoid abort in protobuf library in debug build [#48356](https://github.com/ClickHouse/ClickHouse/pull/48356) ([Kruglov Pavel](https://github.com/Avogar)).
* Batch fix for projections analysis with analyzer. [#48357](https://github.com/ClickHouse/ClickHouse/pull/48357) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix tests with explain and analyzer where names changed. [#48360](https://github.com/ClickHouse/ClickHouse/pull/48360) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Small follow-up to [#45912](https://github.com/ClickHouse/ClickHouse/issues/45912) [#48373](https://github.com/ClickHouse/ClickHouse/pull/48373) ([Robert Schulze](https://github.com/rschu1ze)).
* Update version_date.tsv and changelogs after v22.8.16.32-lts [#48376](https://github.com/ClickHouse/ClickHouse/pull/48376) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Add script for a slack bot that reports broken tests [#48382](https://github.com/ClickHouse/ClickHouse/pull/48382) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix flaky `test_keeper_mntr_data_size` [#48384](https://github.com/ClickHouse/ClickHouse/pull/48384) ([Antonio Andelic](https://github.com/antonio2368)).
* WITH FILL clarification and cleanup [#48395](https://github.com/ClickHouse/ClickHouse/pull/48395) ([Igor Nikonov](https://github.com/devcrafter)).
* Cleanup mess in .clang-tidy [#48396](https://github.com/ClickHouse/ClickHouse/pull/48396) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix test_backup_all [#48400](https://github.com/ClickHouse/ClickHouse/pull/48400) ([Vitaly Baranov](https://github.com/vitlibar)).
* Find big allocations without memory limits checks [#48401](https://github.com/ClickHouse/ClickHouse/pull/48401) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix issue with krb5 and building w/ OpenSSL [#48407](https://github.com/ClickHouse/ClickHouse/pull/48407) ([Boris Kuschel](https://github.com/bkuschel)).
* Make CI slack bot less noisy [#48409](https://github.com/ClickHouse/ClickHouse/pull/48409) ([Alexander Tokmakov](https://github.com/tavplubix)).
* AST fuzzer: Fix assertion in TopK serialization [#48412](https://github.com/ClickHouse/ClickHouse/pull/48412) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix possible flakiness of lightweight delete tests (due to index granularity randomization) [#48413](https://github.com/ClickHouse/ClickHouse/pull/48413) ([Azat Khuzhin](https://github.com/azat)).
* Fix flaky `test_keeper_snapshots` [#48417](https://github.com/ClickHouse/ClickHouse/pull/48417) ([Antonio Andelic](https://github.com/antonio2368)).
* Update sort desc: more efficient original node search in ActionsDAG [#48427](https://github.com/ClickHouse/ClickHouse/pull/48427) ([Igor Nikonov](https://github.com/devcrafter)).
* test for [#16399](https://github.com/ClickHouse/ClickHouse/issues/16399) [#48439](https://github.com/ClickHouse/ClickHouse/pull/48439) ([Denny Crane](https://github.com/den-crane)).
* Better exception messages from Keeper client [#48444](https://github.com/ClickHouse/ClickHouse/pull/48444) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Small documentation follow-up to [#47246](https://github.com/ClickHouse/ClickHouse/issues/47246) [#48463](https://github.com/ClickHouse/ClickHouse/pull/48463) ([Robert Schulze](https://github.com/rschu1ze)).
* Update 00002_log_and_exception_messages_formatting.sql [#48467](https://github.com/ClickHouse/ClickHouse/pull/48467) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Avoid operation on uninitialised data in readDateTimeTextImpl [#48472](https://github.com/ClickHouse/ClickHouse/pull/48472) ([Kruglov Pavel](https://github.com/Avogar)).
* Add reading step for system zookeeper. Analyze path from filter DAG. [#48485](https://github.com/ClickHouse/ClickHouse/pull/48485) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Fix deadlock due to debug tracking of memory allocations [#48487](https://github.com/ClickHouse/ClickHouse/pull/48487) ([Azat Khuzhin](https://github.com/azat)).
* Register datediff and trim aliases in system.functions [#48489](https://github.com/ClickHouse/ClickHouse/pull/48489) ([Robert Schulze](https://github.com/rschu1ze)).
* Change error code [#48490](https://github.com/ClickHouse/ClickHouse/pull/48490) ([Anton Popov](https://github.com/CurtizJ)).
* Update 00002_log_and_exception_messages_formatting.sql [#48499](https://github.com/ClickHouse/ClickHouse/pull/48499) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix query cache with sparse columns [#48500](https://github.com/ClickHouse/ClickHouse/pull/48500) ([Anton Popov](https://github.com/CurtizJ)).
* Use std::string_view to get rid of strlen [#48509](https://github.com/ClickHouse/ClickHouse/pull/48509) ([ltrk2](https://github.com/ltrk2)).
* Fix bytesSize() of zk SetRequest [#48512](https://github.com/ClickHouse/ClickHouse/pull/48512) ([Sergei Trifonov](https://github.com/serxa)).
* Remove dead code and unused dependencies [#48518](https://github.com/ClickHouse/ClickHouse/pull/48518) ([ltrk2](https://github.com/ltrk2)).
* Use forward declaration of ThreadPool [#48519](https://github.com/ClickHouse/ClickHouse/pull/48519) ([Azat Khuzhin](https://github.com/azat)).
* Use std::string_view instead of strlen [#48520](https://github.com/ClickHouse/ClickHouse/pull/48520) ([ltrk2](https://github.com/ltrk2)).
* Use std::string::starts_with instead of a roll your own variant [#48521](https://github.com/ClickHouse/ClickHouse/pull/48521) ([ltrk2](https://github.com/ltrk2)).
* Fix flaky `test_alternative_keeper_config` [#48533](https://github.com/ClickHouse/ClickHouse/pull/48533) ([Antonio Andelic](https://github.com/antonio2368)).
* Use one ThreadGroup while pushing to materialized views (and some refactoring for ThreadGroup) [#48543](https://github.com/ClickHouse/ClickHouse/pull/48543) ([Azat Khuzhin](https://github.com/azat)).
* Fix some tests [#48550](https://github.com/ClickHouse/ClickHouse/pull/48550) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix 02477_projection_materialize_and_zero_copy flakiness (due to index granularity randomization) [#48551](https://github.com/ClickHouse/ClickHouse/pull/48551) ([Azat Khuzhin](https://github.com/azat)).
* Better exception message for ZSTD [#48552](https://github.com/ClickHouse/ClickHouse/pull/48552) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Remove misleading comment and block [#48562](https://github.com/ClickHouse/ClickHouse/pull/48562) ([Sergei Trifonov](https://github.com/serxa)).
* Update 02207_allow_plaintext_and_no_password.sh [#48566](https://github.com/ClickHouse/ClickHouse/pull/48566) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* bugfix: compare Bits and sizeof(Arithmetic) * 8 [#48569](https://github.com/ClickHouse/ClickHouse/pull/48569) ([caipengxiang](https://github.com/awfeequdng)).
* Remove superfluous includes of logger_userful.h from headers [#48570](https://github.com/ClickHouse/ClickHouse/pull/48570) ([Azat Khuzhin](https://github.com/azat)).
* Remove slow test from debug builds [#48574](https://github.com/ClickHouse/ClickHouse/pull/48574) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Don't use type conversion with String query parameters [#48577](https://github.com/ClickHouse/ClickHouse/pull/48577) ([Nikolay Degterinsky](https://github.com/evillique)).
* Fix TSan report in Kerberos [#48579](https://github.com/ClickHouse/ClickHouse/pull/48579) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add second_deadlock_stack=1 for TSan on CI and fix some lock-order-inversion problems [#48596](https://github.com/ClickHouse/ClickHouse/pull/48596) ([Azat Khuzhin](https://github.com/azat)).
* Fix LOGICAL_ERROR in executable table function [#48605](https://github.com/ClickHouse/ClickHouse/pull/48605) ([Nikolay Degterinsky](https://github.com/evillique)).
* Fix flakiness of test_store_cleanup in case of image rebuild [#48610](https://github.com/ClickHouse/ClickHouse/pull/48610) ([Azat Khuzhin](https://github.com/azat)).
* Remove strange code [#48612](https://github.com/ClickHouse/ClickHouse/pull/48612) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Minor refactoring of formatDateTime() [#48627](https://github.com/ClickHouse/ClickHouse/pull/48627) ([Robert Schulze](https://github.com/rschu1ze)).
* Better handling of values too large for VarInt encoding [#48628](https://github.com/ClickHouse/ClickHouse/pull/48628) ([Robert Schulze](https://github.com/rschu1ze)).
* refine some messages of exception in regexp tree [#48632](https://github.com/ClickHouse/ClickHouse/pull/48632) ([Han Fei](https://github.com/hanfei1991)).
* Partially revert e0252db8d and fix pr-bugfix labeling [#48637](https://github.com/ClickHouse/ClickHouse/pull/48637) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Fix build src/Interpreters/InterpreterInsertQuery.h [#48638](https://github.com/ClickHouse/ClickHouse/pull/48638) ([Vladimir C](https://github.com/vdimir)).
* Fix build ThreadGroupPtr [#48641](https://github.com/ClickHouse/ClickHouse/pull/48641) ([Vladimir C](https://github.com/vdimir)).
* Fix flaky test test_drop_replica_and_achieve_quorum [#48642](https://github.com/ClickHouse/ClickHouse/pull/48642) ([Kruglov Pavel](https://github.com/Avogar)).
* fix 02504_regexp_dictionary_table_source [#48662](https://github.com/ClickHouse/ClickHouse/pull/48662) ([Han Fei](https://github.com/hanfei1991)).
* Remove strange code from MutateTask [#48666](https://github.com/ClickHouse/ClickHouse/pull/48666) ([alesapin](https://github.com/alesapin)).
* SonarCloud: C++ Reporting Standards [#48668](https://github.com/ClickHouse/ClickHouse/pull/48668) ([Julio Jimenez](https://github.com/juliojimenez)).
* Remove lock for duplicated parts UUIDs (allow_experimental_query_deduplication=1) [#48670](https://github.com/ClickHouse/ClickHouse/pull/48670) ([Azat Khuzhin](https://github.com/azat)).
* show result of minio listings for test test_attach_detach_partition [#48674](https://github.com/ClickHouse/ClickHouse/pull/48674) ([Sema Checherinda](https://github.com/CheSema)).
* Fix tests for analyzer [#48675](https://github.com/ClickHouse/ClickHouse/pull/48675) ([Igor Nikonov](https://github.com/devcrafter)).
* Call IProcessor::onCancel() once [#48687](https://github.com/ClickHouse/ClickHouse/pull/48687) ([Igor Nikonov](https://github.com/devcrafter)).
* Update MergeTree syntax for optional index granularity argument [#48692](https://github.com/ClickHouse/ClickHouse/pull/48692) ([Robert Schulze](https://github.com/rschu1ze)).
* Add test for old bug [#7826](https://github.com/ClickHouse/ClickHouse/issues/7826) [#48697](https://github.com/ClickHouse/ClickHouse/pull/48697) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix flaky `test_keeper_session` [#48699](https://github.com/ClickHouse/ClickHouse/pull/48699) ([Antonio Andelic](https://github.com/antonio2368)).
* Better messages formatting in the CI Slack bot [#48712](https://github.com/ClickHouse/ClickHouse/pull/48712) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Add trusted contributors [#48715](https://github.com/ClickHouse/ClickHouse/pull/48715) ([Aleksei Filatov](https://github.com/aalexfvk)).
* Do not remove broken detached parts on startup [#48730](https://github.com/ClickHouse/ClickHouse/pull/48730) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Remove `-Wshadow` suppression which leaked into global namespace [#48737](https://github.com/ClickHouse/ClickHouse/pull/48737) ([Robert Schulze](https://github.com/rschu1ze)).
* VarInt coding: Always perform sanity check [#48740](https://github.com/ClickHouse/ClickHouse/pull/48740) ([Robert Schulze](https://github.com/rschu1ze)).
* Try to fix flaky 02455_one_row_from_csv_memory_usage [#48756](https://github.com/ClickHouse/ClickHouse/pull/48756) ([Dmitry Novik](https://github.com/novikd)).
* insert UInt32 Hashvalue in reverse order on big endian machine [#48764](https://github.com/ClickHouse/ClickHouse/pull/48764) ([Suzy Wang](https://github.com/SuzyWangIBMer)).
* Limit size of messages from the CI slack bot [#48766](https://github.com/ClickHouse/ClickHouse/pull/48766) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Update README.md [#48776](https://github.com/ClickHouse/ClickHouse/pull/48776) ([Tyler Hannan](https://github.com/tylerhannan)).
* Remove duplicate definition of SingleEndpointHTTPSessionPool [#48779](https://github.com/ClickHouse/ClickHouse/pull/48779) ([JaySon](https://github.com/JaySon-Huang)).
* Fix flaky test_version_update_after_mutation/test.py::test_upgrade_while_mutation [#48783](https://github.com/ClickHouse/ClickHouse/pull/48783) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix flaky test test_backup_all [#48789](https://github.com/ClickHouse/ClickHouse/pull/48789) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fix a confusing warning about interserver mode [#48793](https://github.com/ClickHouse/ClickHouse/pull/48793) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Store clusters from ClusterDiscovery in separate map [#48795](https://github.com/ClickHouse/ClickHouse/pull/48795) ([Vladimir C](https://github.com/vdimir)).
* Reimplement [#48790](https://github.com/ClickHouse/ClickHouse/issues/48790) [#48797](https://github.com/ClickHouse/ClickHouse/pull/48797) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Allow running integration tests without spark [#48803](https://github.com/ClickHouse/ClickHouse/pull/48803) ([Vitaly Baranov](https://github.com/vitlibar)).
* forbid gwpsan in debug mode to rescue stress tests [#48804](https://github.com/ClickHouse/ClickHouse/pull/48804) ([Han Fei](https://github.com/hanfei1991)).
* Simplify FileCacheFactory [#48805](https://github.com/ClickHouse/ClickHouse/pull/48805) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix double whitespace in exception message [#48815](https://github.com/ClickHouse/ClickHouse/pull/48815) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add a test for [#38128](https://github.com/ClickHouse/ClickHouse/issues/38128) [#48817](https://github.com/ClickHouse/ClickHouse/pull/48817) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Remove excessive logging [#48826](https://github.com/ClickHouse/ClickHouse/pull/48826) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* remove duplicate indentwith in clang-format [#48834](https://github.com/ClickHouse/ClickHouse/pull/48834) ([cluster](https://github.com/infdahai)).
* Try fix flacky test_concurrent_alter_move_and_drop [#48843](https://github.com/ClickHouse/ClickHouse/pull/48843) ([Sergei Trifonov](https://github.com/serxa)).
* fix the race wait loading parts [#48844](https://github.com/ClickHouse/ClickHouse/pull/48844) ([Sema Checherinda](https://github.com/CheSema)).
* suppress assert of progress for test_system_replicated_fetches [#48856](https://github.com/ClickHouse/ClickHouse/pull/48856) ([Han Fei](https://github.com/hanfei1991)).
* Fix: do not run test_store_cleanup_disk_s3 in parallel [#48863](https://github.com/ClickHouse/ClickHouse/pull/48863) ([Igor Nikonov](https://github.com/devcrafter)).
* Update README.md [#48883](https://github.com/ClickHouse/ClickHouse/pull/48883) ([Tyler Hannan](https://github.com/tylerhannan)).
* Fix test reference files for join using nullable column [#48893](https://github.com/ClickHouse/ClickHouse/pull/48893) ([Vladimir C](https://github.com/vdimir)).
* bitNot marked as NO_SANITIZE_UNDEFINED [#48899](https://github.com/ClickHouse/ClickHouse/pull/48899) ([Vladimir C](https://github.com/vdimir)).
* Fix order by in test_storage_delta [#48903](https://github.com/ClickHouse/ClickHouse/pull/48903) ([Vladimir C](https://github.com/vdimir)).
* Fix segfault when set is not built yet [#48904](https://github.com/ClickHouse/ClickHouse/pull/48904) ([Alexander Gololobov](https://github.com/davenger)).
* A non significant change (does not affect anything): add support for signed integers in the maskBits function [#48920](https://github.com/ClickHouse/ClickHouse/pull/48920) ([caipengxiang](https://github.com/awfeequdng)).
* Follow-up to [#48866](https://github.com/ClickHouse/ClickHouse/issues/48866) [#48929](https://github.com/ClickHouse/ClickHouse/pull/48929) ([Robert Schulze](https://github.com/rschu1ze)).
* Un-flake 01079_new_range_reader_segfault [#48934](https://github.com/ClickHouse/ClickHouse/pull/48934) ([Robert Schulze](https://github.com/rschu1ze)).
* Add building stage to the fasttests report, respect existing status on rerun [#48935](https://github.com/ClickHouse/ClickHouse/pull/48935) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Update Settings.h [#48948](https://github.com/ClickHouse/ClickHouse/pull/48948) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Update cluster.py [#48949](https://github.com/ClickHouse/ClickHouse/pull/48949) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Docs: Replace annoying three spaces in enumerations by a single space [#48951](https://github.com/ClickHouse/ClickHouse/pull/48951) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix flaky 02706_arrow_different_dictionaries [#48952](https://github.com/ClickHouse/ClickHouse/pull/48952) ([Kruglov Pavel](https://github.com/Avogar)).
* Use default `{replica}`, `{shard}` arguments in Replicated engine [#48961](https://github.com/ClickHouse/ClickHouse/pull/48961) ([Nikolay Degterinsky](https://github.com/evillique)).
* Rename quantileApprox -> quantileGK [#48969](https://github.com/ClickHouse/ClickHouse/pull/48969) ([Vladimir C](https://github.com/vdimir)).
* Don't throw logical error when column is not found in Parquet/Arrow schema [#48987](https://github.com/ClickHouse/ClickHouse/pull/48987) ([Kruglov Pavel](https://github.com/Avogar)).
* Reimplement [#48986](https://github.com/ClickHouse/ClickHouse/issues/48986) [#49005](https://github.com/ClickHouse/ClickHouse/pull/49005) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Dont allow bad changelogs [#49006](https://github.com/ClickHouse/ClickHouse/pull/49006) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Update README.md [#49007](https://github.com/ClickHouse/ClickHouse/pull/49007) ([Nick-71](https://github.com/Nick-71)).
* Remove outdated test [#49014](https://github.com/ClickHouse/ClickHouse/pull/49014) ([alesapin](https://github.com/alesapin)).
* Fix typo [#49027](https://github.com/ClickHouse/ClickHouse/pull/49027) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix assertion after [#48636](https://github.com/ClickHouse/ClickHouse/issues/48636) [#49029](https://github.com/ClickHouse/ClickHouse/pull/49029) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix build error for big-endian platforms [#49037](https://github.com/ClickHouse/ClickHouse/pull/49037) ([ltrk2](https://github.com/ltrk2)).
* Update version_date.tsv and changelogs after v22.8.17.17-lts [#49046](https://github.com/ClickHouse/ClickHouse/pull/49046) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Update version_date.tsv and changelogs after v23.1.7.30-stable [#49047](https://github.com/ClickHouse/ClickHouse/pull/49047) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Update version_date.tsv and changelogs after v23.3.2.37-lts [#49048](https://github.com/ClickHouse/ClickHouse/pull/49048) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Remove some code [#49054](https://github.com/ClickHouse/ClickHouse/pull/49054) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Remove some dead code in poco [#49075](https://github.com/ClickHouse/ClickHouse/pull/49075) ([Robert Schulze](https://github.com/rschu1ze)).
* Prevent false positive report by static analyzer [#49078](https://github.com/ClickHouse/ClickHouse/pull/49078) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Update version_date.tsv and changelogs after v23.2.6.34-stable [#49080](https://github.com/ClickHouse/ClickHouse/pull/49080) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Enforce documentation change for a new-feature PR [#49090](https://github.com/ClickHouse/ClickHouse/pull/49090) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Update clickhouse-test [#49094](https://github.com/ClickHouse/ClickHouse/pull/49094) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Disable long 02581 in debug, enable with sanitizers [#49105](https://github.com/ClickHouse/ClickHouse/pull/49105) ([Alexander Gololobov](https://github.com/davenger)).
* Fix flaky integration test test_async_query_sending [#49107](https://github.com/ClickHouse/ClickHouse/pull/49107) ([Kruglov Pavel](https://github.com/Avogar)).
* Correct functional test to reflect interoperability [#49108](https://github.com/ClickHouse/ClickHouse/pull/49108) ([ltrk2](https://github.com/ltrk2)).
* Cleanup build guide [#49119](https://github.com/ClickHouse/ClickHouse/pull/49119) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix building iceberg without avro [#49125](https://github.com/ClickHouse/ClickHouse/pull/49125) ([Azat Khuzhin](https://github.com/azat)).
* Add slash for close tag of user_defined_zookeeper_path [#49131](https://github.com/ClickHouse/ClickHouse/pull/49131) ([Hollin](https://github.com/Hooollin)).
* Improve some lambdas [#49133](https://github.com/ClickHouse/ClickHouse/pull/49133) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Do not randomize prefetch settings for debug build [#49134](https://github.com/ClickHouse/ClickHouse/pull/49134) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Don't throw LOGICAL_ERROR when reading from remote if there is no local replica [#49136](https://github.com/ClickHouse/ClickHouse/pull/49136) ([Raúl Marín](https://github.com/Algunenano)).
* Docs: Make caption of processors_profile_log page consistent with other pages [#49138](https://github.com/ClickHouse/ClickHouse/pull/49138) ([Robert Schulze](https://github.com/rschu1ze)).
* Improve test reports [#49151](https://github.com/ClickHouse/ClickHouse/pull/49151) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Add a note regarding private/public repo to logs [#49152](https://github.com/ClickHouse/ClickHouse/pull/49152) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* suppress two timeout tests [#49175](https://github.com/ClickHouse/ClickHouse/pull/49175) ([Han Fei](https://github.com/hanfei1991)).
* Document makeDateTime() and its variants [#49183](https://github.com/ClickHouse/ClickHouse/pull/49183) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix after [#49110](https://github.com/ClickHouse/ClickHouse/issues/49110) [#49206](https://github.com/ClickHouse/ClickHouse/pull/49206) ([Kseniia Sumarokova](https://github.com/kssenii)).

View File

@ -22,7 +22,7 @@ The minimum recommended Ubuntu version for development is 22.04 LTS.
### Install Prerequisites {#install-prerequisites}
``` bash
sudo apt-get install git cmake ccache python3 ninja-build yasm gawk
sudo apt-get install git cmake ccache python3 ninja-build nasm yasm gawk
```
### Install and Use the Clang compiler
@ -72,7 +72,7 @@ cmake -S . -B build
cmake --build build # or: `cd build; ninja`
```
To create an executable, run `cmake --build --target clickhouse` (or: `cd build; ninja clickhouse`).
To create an executable, run `cmake --build build --target clickhouse` (or: `cd build; ninja clickhouse`).
This will create executable `build/programs/clickhouse` which can be used with `client` or `server` arguments.
## Building on Any Linux {#how-to-build-clickhouse-on-any-linux}
@ -92,7 +92,7 @@ If all the components are installed, you may build in the same way as the steps
Example for OpenSUSE Tumbleweed:
``` bash
sudo zypper install git cmake ninja clang-c++ python lld yasm gawk
sudo zypper install git cmake ninja clang-c++ python lld nasm yasm gawk
git clone --recursive https://github.com/ClickHouse/ClickHouse.git
mkdir build
cmake -S . -B build
@ -103,7 +103,7 @@ Example for Fedora Rawhide:
``` bash
sudo yum update
sudo yum --nogpg install git cmake make clang python3 ccache yasm gawk
sudo yum --nogpg install git cmake make clang python3 ccache nasm yasm gawk
git clone --recursive https://github.com/ClickHouse/ClickHouse.git
mkdir build
cmake -S . -B build

View File

@ -40,7 +40,7 @@ If the user tries to violate the constraints an exception is thrown and the sett
There are supported few types of constraints: `min`, `max`, `readonly` (with alias `const`) and `changeable_in_readonly`. The `min` and `max` constraints specify upper and lower boundaries for a numeric setting and can be used in combination. The `readonly` or `const` constraint specifies that the user cannot change the corresponding setting at all. The `changeable_in_readonly` constraint type allows user to change the setting within `min`/`max` range even if `readonly` setting is set to 1, otherwise settings are not allow to be changed in `readonly=1` mode. Note that `changeable_in_readonly` is supported only if `settings_constraints_replace_previous` is enabled:
``` xml
<access_control_improvements>
<settings_constraints_replace_previous>true<settings_constraints_replace_previous>
<settings_constraints_replace_previous>true</settings_constraints_replace_previous>
</access_control_improvements>
```

View File

@ -890,7 +890,7 @@ Write time that processor spent during execution/waiting for data to `system.pro
See also:
- [`system.processors_profile_log`](../../operations/system-tables/processors_profile_log.md#system-processors_profile_log)
- [`system.processors_profile_log`](../../operations/system-tables/processors_profile_log.md)
- [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline)
## max_insert_block_size {#settings-max_insert_block_size}

View File

@ -1,4 +1,4 @@
# system.processors_profile_log {#system-processors_profile_log}
# processors_profile_log
This table contains profiling on processors level (that you can find in [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline)).
@ -73,4 +73,4 @@ Here you can see:
**See Also**
- [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline)
- [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline)

View File

@ -0,0 +1,118 @@
---
slug: /en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest
sidebar_position: 300
sidebar_label: kolmogorovSmirnovTest
---
# kolmogorovSmirnovTest
Applies Kolmogorov-Smirnov's test to samples from two populations.
**Syntax**
``` sql
kolmogorovSmirnovTest([alternative, computation_method])(sample_data, sample_index)
```
Values of both samples are in the `sample_data` column. If `sample_index` equals to 0 then the value in that row belongs to the sample from the first population. Otherwise it belongs to the sample from the second population.
Samples must belong to continuous, one-dimensional probability distributions.
**Arguments**
- `sample_data` — Sample data. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md).
- `sample_index` — Sample index. [Integer](../../../sql-reference/data-types/int-uint.md).
**Parameters**
- `alternative` — alternative hypothesis. (Optional, default: `'two-sided'`.) [String](../../../sql-reference/data-types/string.md).
Let F(x) and G(x) be the CDFs of the first and second distributions respectively.
- `'two-sided'`
The null hypothesis is that samples come from the same distribution, e.g. F(x) = G(x) for all x.
And the alternative is that the distributions are not identical.
- `'greater'`
The null hypothesis is that values in the first sample are *stohastically smaller* than those in the second one,
e.g. the CDF of first distribution lies above and hence to the left of that for the second one.
Which in fact means that F(x) >= G(x) for all x. And the alternative in this case is that F(x) < G(x) for at least one x.
- `'less'`.
The null hypothesis is that values in the first sample are *stohastically greater* than those in the second one,
e.g. the CDF of first distribution lies below and hence to the right of that for the second one.
Which in fact means that F(x) <= G(x) for all x. And the alternative in this case is that F(x) > G(x) for at least one x.
- `computation_method` — the method used to compute p-value. (Optional, default: `'auto'`.) [String](../../../sql-reference/data-types/string.md).
- `'exact'` - calculation is performed using precise probability distribution of the test statistics. Compute intensive and wasteful except for small samples.
- `'asymp'` (`'asymptotic'`) - calculation is performed using an approximation. For large sample sizes, the exact and asymptotic p-values are very similar.
- `'auto'` - the `'exact'` method is used when a maximum number of samples is less than 10'000.
**Returned values**
[Tuple](../../../sql-reference/data-types/tuple.md) with two elements:
- calculated statistic. [Float64](../../../sql-reference/data-types/float.md).
- calculated p-value. [Float64](../../../sql-reference/data-types/float.md).
**Example**
Query:
``` sql
SELECT kolmogorovSmirnovTest('less', 'exact')(value, num)
FROM
(
SELECT
randNormal(0, 10) AS value,
0 AS num
FROM numbers(10000)
UNION ALL
SELECT
randNormal(0, 10) AS value,
1 AS num
FROM numbers(10000)
)
```
Result:
``` text
┌─kolmogorovSmirnovTest('less', 'exact')(value, num)─┐
│ (0.009899999999999996,0.37528595205132287) │
└────────────────────────────────────────────────────┘
```
Note:
P-value is bigger than 0.05 (for confidence level of 95%), so null hypothesis is not rejected.
Query:
``` sql
SELECT kolmogorovSmirnovTest('two-sided', 'exact')(value, num)
FROM
(
SELECT
randStudentT(10) AS value,
0 AS num
FROM numbers(100)
UNION ALL
SELECT
randNormal(0, 10) AS value,
1 AS num
FROM numbers(100)
)
```
Result:
``` text
┌─kolmogorovSmirnovTest('two-sided', 'exact')(value, num)─┐
│ (0.4100000000000002,6.61735760482795e-8) │
└─────────────────────────────────────────────────────────┘
```
Note:
P-value is less than 0.05 (for confidence level of 95%), so null hypothesis is rejected.
**See Also**
- [Kolmogorov-Smirnov'test](https://en.wikipedia.org/wiki/Kolmogorov%E2%80%93Smirnov_test)

View File

@ -1658,6 +1658,7 @@ Example of settings:
<password></password>
<db>test</db>
<collection>dictionary_source</collection>
<options>ssl=true</options>
</mongodb>
</source>
```
@ -1672,6 +1673,7 @@ SOURCE(MONGODB(
password ''
db 'test'
collection 'dictionary_source'
options 'ssl=true'
))
```
@ -1683,6 +1685,8 @@ Setting fields:
- `password` Password of the MongoDB user.
- `db` Name of the database.
- `collection` Name of the collection.
- `options` - MongoDB connection string options (optional parameter).
### Redis

View File

@ -24,6 +24,90 @@ SELECT
└─────────────────────┴────────────┴────────────┴─────────────────────┘
```
## makeDate
Creates a [Date](../../sql-reference/data-types/date.md) from a year, month and day argument.
**Syntax**
``` sql
makeDate(year, month, day)
```
**Arguments**
- `year` — Year. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md).
- `month` — Month. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md).
- `day` — Day. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md).
**Returned value**
- A date created from the arguments.
Type: [Date](../../sql-reference/data-types/date.md).
**Example**
``` sql
SELECT makeDate(2023, 2, 28) AS Date;
```
Result:
``` text
┌───────date─┐
│ 2023-02-28 │
└────────────┘
```
## makeDate32
Like [makeDate](#makeDate) but produces a [Date32](../../sql-reference/data-types/date32.md).
## makeDateTime
Creates a [DateTime](../../sql-reference/data-types/datetime.md) from a year, month, day, hour, minute and second argument.
**Syntax**
``` sql
makeDateTime(year, month, day, hour, minute, second[, timezone])
```
**Arguments**
- `year` — Year. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md).
- `month` — Month. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md).
- `day` — Day. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md).
- `hour` — Hour. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md).
- `minute` — Minute. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md).
- `second` — Second. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md).
- `timezone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional).
**Returned value**
- A date with time created from the arguments.
Type: [DateTime](../../sql-reference/data-types/datetime.md).
**Example**
``` sql
SELECT makeDateTime(2023, 2, 28, 17, 12, 33) AS DateTime;
```
Result:
``` text
┌────────────DateTime─┐
│ 2023-02-28 17:12:33 │
└─────────────────────┘
```
## makeDateTime64
Like [makeDateTime](#makedatetime) but produces a [DateTime64](../../sql-reference/data-types/datetime64.md).
## timeZone
Returns the timezone of the server.

View File

@ -109,7 +109,7 @@ For the query to run successfully, the following conditions must be met:
- Both tables must have the same structure.
- Both tables must have the same partition key, the same order by key and the same primary key.
- Both tables must have the same storage policy (a disk where the partition is stored should be available for both tables).
- Both tables must have the same storage policy.
## REPLACE PARTITION
@ -123,7 +123,7 @@ For the query to run successfully, the following conditions must be met:
- Both tables must have the same structure.
- Both tables must have the same partition key, the same order by key and the same primary key.
- Both tables must have the same storage policy (a disk where the partition is stored should be available for both tables).
- Both tables must have the same storage policy.
## MOVE PARTITION TO TABLE
@ -137,7 +137,7 @@ For the query to run successfully, the following conditions must be met:
- Both tables must have the same structure.
- Both tables must have the same partition key, the same order by key and the same primary key.
- Both tables must have the same storage policy (a disk where the partition is stored should be available for both tables).
- Both tables must have the same storage policy.
- Both tables must be the same engine family (replicated or non-replicated).
## CLEAR COLUMN IN PARTITION

View File

@ -92,6 +92,20 @@ There are multiple ways of user identification:
CREATE USER name4 IDENTIFIED WITH double_sha1_hash BY 'CCD3A959D6A004B9C3807B728BC2E55B67E10518'
```
5. The type of the password can also be omitted:
```sql
CREATE USER name4 IDENTIFIED BY 'my_password'
```
In this case, ClickHouse will use the default password type specified in the server configuration:
```xml
<default_password_type>sha256_password</default_password_type>
```
The available password types are: `plaintext_password`, `sha256_password`, `double_sha1_password`.
## User Host
User host is a host from which a connection to ClickHouse server could be established. The host can be specified in the `HOST` query section in the following ways:

View File

@ -0,0 +1,117 @@
---
slug: /ru/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest
sidebar_position: 300
sidebar_label: kolmogorovSmirnovTest
---
# kolmogorovSmirnovTest {#kolmogorovSmirnovTest}
Проводит статистический тест Колмогорова-Смирнова для двух независимых выборок.
**Синтаксис**
``` sql
kolmogorovSmirnovTest([alternative, computation_method])(sample_data, sample_index)
```
Значения выборок берутся из столбца `sample_data`. Если `sample_index` равно 0, то значение из этой строки принадлежит первой выборке. Во всех остальных случаях значение принадлежит второй выборке.
Выборки должны принадлежать непрерывным одномерным распределениям.
**Аргументы**
- `sample_data` — данные выборок. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md).
- `sample_index` — индексы выборок. [Integer](../../../sql-reference/data-types/int-uint.md).
**Параметры**
- `alternative` — альтернативная гипотеза (Необязательный параметр, по умолчанию: `'two-sided'`.) [String](../../../sql-reference/data-types/string.md).
Пусть F(x) и G(x) - функции распределения первой и второй выборки соотвественно.
- `'two-sided'`
Нулевая гипотеза состоит в том, что выборки происходит из одного и того же распределение, то есть F(x) = G(x) для любого x.
Альтернатива - выборки принадлежат разным распределениям.
- `'greater'`
Нулевая гипотеза состоит в том, что элементы первой выборки в асимптотически почти наверное меньше элементов из второй выборки,
то есть функция распределения первой выборки лежит выше и соотвественно левее, чем функция распределения второй выборки.
Таким образом это означает, что F(x) >= G(x) for любого x, а альтернатива в этом случае состоит в том, что F(x) < G(x) хотя бы для одного x.
- `'less'`.
Нулевая гипотеза состоит в том, что элементы первой выборки в асимптотически почти наверное больше элементов из второй выборки,
то есть функция распределения первой выборки лежит ниже и соотвественно правее, чем функция распределения второй выборки.
Таким образом это означает, что F(x) <= G(x) for любого x, а альтернатива в этом случае состоит в том, что F(x) > G(x) хотя бы для одного x.
- `computation_method` — метод, используемый для вычисления p-value. (Необязательный параметр, по умолчанию: `'auto'`.) [String](../../../sql-reference/data-types/string.md).
- `'exact'` - вычисление производится с помощью вычисления точного распределения статистики. Требует большого количества вычислительных ресурсов и расточительно для больших выборок.
- `'asymp'`(`'asymptotic'`) - используется приближенное вычисление. Для больших выборок приближенный результат и точный почти идентичны.
- `'auto'` - значение вычисляется точно (с помощью метода `'exact'`), если максимальный размер двух выборок не превышает 10'000.
**Возвращаемые значения**
[Кортеж](../../../sql-reference/data-types/tuple.md) с двумя элементами:
- вычисленное статистики. [Float64](../../../sql-reference/data-types/float.md).
- вычисленное p-value. [Float64](../../../sql-reference/data-types/float.md).
**Пример**
Запрос:
``` sql
SELECT kolmogorovSmirnovTest('less', 'exact')(value, num)
FROM
(
SELECT
randNormal(0, 10) AS value,
0 AS num
FROM numbers(10000)
UNION ALL
SELECT
randNormal(0, 10) AS value,
1 AS num
FROM numbers(10000)
)
```
Результат:
``` text
┌─kolmogorovSmirnovTest('less', 'exact')(value, num)─┐
│ (0.009899999999999996,0.37528595205132287) │
└────────────────────────────────────────────────────┘
```
Заметки:
P-value больше чем 0.05 (для уровня значимости 95%), то есть нулевая гипотеза не отвергается.
Запрос:
``` sql
SELECT kolmogorovSmirnovTest('two-sided', 'exact')(value, num)
FROM
(
SELECT
randStudentT(10) AS value,
0 AS num
FROM numbers(100)
UNION ALL
SELECT
randNormal(0, 10) AS value,
1 AS num
FROM numbers(100)
)
```
Результат:
``` text
┌─kolmogorovSmirnovTest('two-sided', 'exact')(value, num)─┐
│ (0.4100000000000002,6.61735760482795e-8) │
└─────────────────────────────────────────────────────────┘
```
Заметки:
P-value меньше чем 0.05 (для уровня значимости 95%), то есть нулевая гипотеза отвергается.
**Смотрите также**
- [Критерий согласия Колмогорова-Смирнова](https://ru.wikipedia.org/wiki/%D0%9A%D1%80%D0%B8%D1%82%D0%B5%D1%80%D0%B8%D0%B9_%D1%81%D0%BE%D0%B3%D0%BB%D0%B0%D1%81%D0%B8%D1%8F_%D0%9A%D0%BE%D0%BB%D0%BC%D0%BE%D0%B3%D0%BE%D1%80%D0%BE%D0%B2%D0%B0)

View File

@ -26,6 +26,7 @@
#include <Common/TLDListsHolder.h>
#include <Common/quoteString.h>
#include <Common/randomSeed.h>
#include <Common/ThreadPool.h>
#include <Loggers/Loggers.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadBufferFromString.h>

View File

@ -476,6 +476,11 @@
<allow_no_password>1</allow_no_password>
<allow_implicit_no_password>1</allow_implicit_no_password>
<!-- When a user does not specify a password type in the CREATE USER query, the default password type is used.
Accepted values are: 'plaintext_password', 'sha256_password', and 'double_sha1_password'.
-->
<default_password_type>sha256_password</default_password_type>
<!-- Complexity requirements for user passwords. -->
<!-- <password_complexity>
<rule>
@ -1293,7 +1298,7 @@
<!-- Path in ZooKeeper to store user-defined SQL functions created by the command CREATE FUNCTION.
If not specified they will be stored locally. -->
<!-- <user_defined_zookeeper_path>/clickhouse/user_defined<user_defined_zookeeper_path> -->
<!-- <user_defined_zookeeper_path>/clickhouse/user_defined</user_defined_zookeeper_path> -->
<!-- Uncomment if you want data to be compressed 30-100% better.
Don't do that if you just started using ClickHouse.

View File

@ -271,6 +271,7 @@ void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration
setImplicitNoPasswordAllowed(config_.getBool("allow_implicit_no_password", true));
setNoPasswordAllowed(config_.getBool("allow_no_password", true));
setPlaintextPasswordAllowed(config_.getBool("allow_plaintext_password", true));
setDefaultPasswordTypeFromConfig(config_.getString("default_password_type", "sha256_password"));
setPasswordComplexityRulesFromConfig(config_);
/// Optional improvements in access control system.
@ -653,6 +654,27 @@ bool AccessControl::isPlaintextPasswordAllowed() const
return allow_plaintext_password;
}
void AccessControl::setDefaultPasswordTypeFromConfig(const String & type_)
{
for (auto check_type : collections::range(AuthenticationType::MAX))
{
const auto & info = AuthenticationTypeInfo::get(check_type);
if (type_ == info.name && info.is_password)
{
default_password_type = check_type;
return;
}
}
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown password type in 'default_password_type' in config");
}
AuthenticationType AccessControl::getDefaultPasswordType() const
{
return default_password_type;
}
void AccessControl::setPasswordComplexityRulesFromConfig(const Poco::Util::AbstractConfiguration & config_)
{
password_rules->setPasswordComplexityRulesFromConfig(config_);

View File

@ -1,6 +1,7 @@
#pragma once
#include <Access/MultipleAccessStorage.h>
#include <Access/Common/AuthenticationType.h>
#include <Common/SettingsChanges.h>
#include <Common/ZooKeeper/Common.h>
#include <base/scope_guard.h>
@ -147,8 +148,11 @@ public:
void setPlaintextPasswordAllowed(const bool allow_plaintext_password_);
bool isPlaintextPasswordAllowed() const;
/// Check complexity requirements for plaintext passwords
/// Default password type when the user does not specify it.
void setDefaultPasswordTypeFromConfig(const String & type_);
AuthenticationType getDefaultPasswordType() const;
/// Check complexity requirements for passwords
void setPasswordComplexityRulesFromConfig(const Poco::Util::AbstractConfiguration & config_);
void setPasswordComplexityRules(const std::vector<std::pair<String, String>> & rules_);
void checkPasswordComplexityRules(const String & password_) const;
@ -242,6 +246,7 @@ private:
std::atomic_bool select_from_system_db_requires_grant = false;
std::atomic_bool select_from_information_schema_requires_grant = false;
std::atomic_bool settings_constraints_replace_previous = false;
std::atomic<AuthenticationType> default_password_type = AuthenticationType::SHA256_PASSWORD;
};
}

View File

@ -1,5 +1,5 @@
#include <Access/Authentication.h>
#include <Access/Common/AuthenticationData.h>
#include <Access/AuthenticationData.h>
#include <Access/Credentials.h>
#include <Access/ExternalAuthenticators.h>
#include <Access/LDAPClient.h>

View File

@ -1,6 +1,6 @@
#pragma once
#include <Access/Common/AuthenticationData.h>
#include <Access/AuthenticationData.h>
#include <Common/Exception.h>
#include <base/types.h>

View File

@ -1,11 +1,26 @@
#include <Access/Common/AuthenticationData.h>
#include <Access/AccessControl.h>
#include <Access/AuthenticationData.h>
#include <Common/Exception.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTLiteral.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include <Common/OpenSSLHelpers.h>
#include <Poco/SHA1Engine.h>
#include <base/types.h>
#include <boost/algorithm/hex.hpp>
#include <boost/algorithm/string/case_conv.hpp>
#include "config.h"
#if USE_SSL
# include <openssl/crypto.h>
# include <openssl/rand.h>
# include <openssl/err.h>
#endif
namespace DB
{
@ -15,62 +30,9 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int OPENSSL_ERROR;
}
const AuthenticationTypeInfo & AuthenticationTypeInfo::get(AuthenticationType type_)
{
static constexpr auto make_info = [](const char * raw_name_)
{
String init_name = raw_name_;
boost::to_lower(init_name);
return AuthenticationTypeInfo{raw_name_, std::move(init_name)};
};
switch (type_)
{
case AuthenticationType::NO_PASSWORD:
{
static const auto info = make_info("NO_PASSWORD");
return info;
}
case AuthenticationType::PLAINTEXT_PASSWORD:
{
static const auto info = make_info("PLAINTEXT_PASSWORD");
return info;
}
case AuthenticationType::SHA256_PASSWORD:
{
static const auto info = make_info("SHA256_PASSWORD");
return info;
}
case AuthenticationType::DOUBLE_SHA1_PASSWORD:
{
static const auto info = make_info("DOUBLE_SHA1_PASSWORD");
return info;
}
case AuthenticationType::LDAP:
{
static const auto info = make_info("LDAP");
return info;
}
case AuthenticationType::KERBEROS:
{
static const auto info = make_info("KERBEROS");
return info;
}
case AuthenticationType::SSL_CERTIFICATE:
{
static const auto info = make_info("SSL_CERTIFICATE");
return info;
}
case AuthenticationType::MAX:
break;
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown authentication type: {}", static_cast<int>(type_));
}
AuthenticationData::Digest AuthenticationData::Util::encodeSHA256(std::string_view text [[maybe_unused]])
{
#if USE_SSL
@ -225,4 +187,172 @@ void AuthenticationData::setSSLCertificateCommonNames(boost::container::flat_set
ssl_certificate_common_names = std::move(common_names_);
}
std::shared_ptr<ASTAuthenticationData> AuthenticationData::toAST() const
{
auto node = std::make_shared<ASTAuthenticationData>();
auto auth_type = getType();
node->type = auth_type;
switch (auth_type)
{
case AuthenticationType::PLAINTEXT_PASSWORD:
{
node->contains_password = true;
node->children.push_back(std::make_shared<ASTLiteral>(getPassword()));
break;
}
case AuthenticationType::SHA256_PASSWORD:
{
node->contains_hash = true;
node->children.push_back(std::make_shared<ASTLiteral>(getPasswordHashHex()));
if (!getSalt().empty())
node->children.push_back(std::make_shared<ASTLiteral>(getSalt()));
break;
}
case AuthenticationType::DOUBLE_SHA1_PASSWORD:
{
node->contains_hash = true;
node->children.push_back(std::make_shared<ASTLiteral>(getPasswordHashHex()));
break;
}
case AuthenticationType::LDAP:
{
node->children.push_back(std::make_shared<ASTLiteral>(getLDAPServerName()));
break;
}
case AuthenticationType::KERBEROS:
{
const auto & realm = getKerberosRealm();
if (!realm.empty())
node->children.push_back(std::make_shared<ASTLiteral>(realm));
break;
}
case AuthenticationType::SSL_CERTIFICATE:
{
for (const auto & name : getSSLCertificateCommonNames())
node->children.push_back(std::make_shared<ASTLiteral>(name));
break;
}
case AuthenticationType::NO_PASSWORD: [[fallthrough]];
case AuthenticationType::MAX:
throw Exception(ErrorCodes::LOGICAL_ERROR, "AST: Unexpected authentication type {}", toString(auth_type));
}
return node;
}
AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & query, ContextPtr context, bool check_password_rules)
{
if (query.type && *query.type == AuthenticationType::NO_PASSWORD)
return AuthenticationData();
size_t args_size = query.children.size();
ASTs args(args_size);
for (size_t i = 0; i < args_size; ++i)
args[i] = evaluateConstantExpressionAsLiteral(query.children[i], context);
if (query.contains_password)
{
if (!query.type && !context)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get default password type without context");
if (check_password_rules && !context)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot check password complexity rules without context");
/// NOTE: We will also extract bcrypt workfactor from context
String value = checkAndGetLiteralArgument<String>(args[0], "password");
AuthenticationType current_type;
if (query.type)
current_type = *query.type;
else
current_type = context->getAccessControl().getDefaultPasswordType();
AuthenticationData auth_data(current_type);
if (check_password_rules)
context->getAccessControl().checkPasswordComplexityRules(value);
if (query.type == AuthenticationType::SHA256_PASSWORD)
{
#if USE_SSL
///random generator FIPS complaint
uint8_t key[32];
if (RAND_bytes(key, sizeof(key)) != 1)
{
char buf[512] = {0};
ERR_error_string_n(ERR_get_error(), buf, sizeof(buf));
throw Exception(ErrorCodes::OPENSSL_ERROR, "Cannot generate salt for password. OpenSSL {}", buf);
}
String salt;
salt.resize(sizeof(key) * 2);
char * buf_pos = salt.data();
for (uint8_t k : key)
{
writeHexByteUppercase(k, buf_pos);
buf_pos += 2;
}
value.append(salt);
auth_data.setSalt(salt);
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"SHA256 passwords support is disabled, because ClickHouse was built without SSL library");
#endif
}
auth_data.setPassword(value);
return auth_data;
}
AuthenticationData auth_data(*query.type);
if (query.contains_hash)
{
String value = checkAndGetLiteralArgument<String>(args[0], "hash");
auth_data.setPasswordHashHex(value);
if (query.type == AuthenticationType::SHA256_PASSWORD && args_size == 2)
{
String parsed_salt = checkAndGetLiteralArgument<String>(args[1], "salt");
auth_data.setSalt(parsed_salt);
}
}
else if (query.type == AuthenticationType::LDAP)
{
String value = checkAndGetLiteralArgument<String>(args[0], "ldap_server_name");
auth_data.setLDAPServerName(value);
}
else if (query.type == AuthenticationType::KERBEROS)
{
if (!args.empty())
{
String value = checkAndGetLiteralArgument<String>(args[0], "kerberos_realm");
auth_data.setKerberosRealm(value);
}
}
else if (query.type == AuthenticationType::SSL_CERTIFICATE)
{
boost::container::flat_set<String> common_names;
for (const auto & arg : args)
common_names.insert(checkAndGetLiteralArgument<String>(arg, "common_name"));
auth_data.setSSLCertificateCommonNames(std::move(common_names));
}
else
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected ASTAuthenticationData structure");
}
return auth_data;
}
}

View File

@ -1,5 +1,9 @@
#pragma once
#include <Access/Common/AuthenticationType.h>
#include <Parsers/Access/ASTAuthenticationData.h>
#include <Interpreters/Context_fwd.h>
#include <base/types.h>
#include <boost/container/flat_set.hpp>
#include <vector>
@ -7,47 +11,6 @@
namespace DB
{
enum class AuthenticationType
{
/// User doesn't have to enter password.
NO_PASSWORD,
/// Password is stored as is.
PLAINTEXT_PASSWORD,
/// Password is encrypted in SHA256 hash.
SHA256_PASSWORD,
/// SHA1(SHA1(password)).
/// This kind of hash is used by the `mysql_native_password` authentication plugin.
DOUBLE_SHA1_PASSWORD,
/// Password is checked by a [remote] LDAP server. Connection will be made at each authentication attempt.
LDAP,
/// Kerberos authentication performed through GSS-API negotiation loop.
KERBEROS,
/// Authentication is done in SSL by checking user certificate.
/// Certificates may only be trusted if 'strict' SSL mode is enabled.
SSL_CERTIFICATE,
MAX,
};
struct AuthenticationTypeInfo
{
const char * const raw_name;
const String name; /// Lowercased with underscores, e.g. "sha256_password".
static const AuthenticationTypeInfo & get(AuthenticationType type_);
};
inline String toString(AuthenticationType type_)
{
return AuthenticationTypeInfo::get(type_).raw_name;
}
/// Stores data for checking password when a user logins.
class AuthenticationData
{
@ -94,6 +57,9 @@ public:
friend bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs);
friend bool operator !=(const AuthenticationData & lhs, const AuthenticationData & rhs) { return !(lhs == rhs); }
static AuthenticationData fromAST(const ASTAuthenticationData & query, ContextPtr context, bool check_password_rules);
std::shared_ptr<ASTAuthenticationData> toAST() const;
struct Util
{
static Digest stringToDigest(std::string_view text) { return Digest(text.data(), text.data() + text.size()); }

View File

@ -0,0 +1,66 @@
#include <Access/Common/AuthenticationType.h>
#include <Common/Exception.h>
#include <boost/algorithm/string/case_conv.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
const AuthenticationTypeInfo & AuthenticationTypeInfo::get(AuthenticationType type_)
{
static constexpr auto make_info = [](const char * raw_name_, bool is_password_ = false)
{
String init_name = raw_name_;
boost::to_lower(init_name);
return AuthenticationTypeInfo{raw_name_, std::move(init_name), is_password_};
};
switch (type_)
{
case AuthenticationType::NO_PASSWORD:
{
static const auto info = make_info("NO_PASSWORD");
return info;
}
case AuthenticationType::PLAINTEXT_PASSWORD:
{
static const auto info = make_info("PLAINTEXT_PASSWORD", true);
return info;
}
case AuthenticationType::SHA256_PASSWORD:
{
static const auto info = make_info("SHA256_PASSWORD", true);
return info;
}
case AuthenticationType::DOUBLE_SHA1_PASSWORD:
{
static const auto info = make_info("DOUBLE_SHA1_PASSWORD", true);
return info;
}
case AuthenticationType::LDAP:
{
static const auto info = make_info("LDAP");
return info;
}
case AuthenticationType::KERBEROS:
{
static const auto info = make_info("KERBEROS");
return info;
}
case AuthenticationType::SSL_CERTIFICATE:
{
static const auto info = make_info("SSL_CERTIFICATE");
return info;
}
case AuthenticationType::MAX:
break;
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown authentication type: {}", static_cast<int>(type_));
}
}

View File

@ -0,0 +1,49 @@
#pragma once
#include <base/types.h>
namespace DB
{
enum class AuthenticationType
{
/// User doesn't have to enter password.
NO_PASSWORD,
/// Password is stored as is.
PLAINTEXT_PASSWORD,
/// Password is encrypted in SHA256 hash.
SHA256_PASSWORD,
/// SHA1(SHA1(password)).
/// This kind of hash is used by the `mysql_native_password` authentication plugin.
DOUBLE_SHA1_PASSWORD,
/// Password is checked by a [remote] LDAP server. Connection will be made at each authentication attempt.
LDAP,
/// Kerberos authentication performed through GSS-API negotiation loop.
KERBEROS,
/// Authentication is done in SSL by checking user certificate.
/// Certificates may only be trusted if 'strict' SSL mode is enabled.
SSL_CERTIFICATE,
MAX,
};
struct AuthenticationTypeInfo
{
const char * const raw_name;
const String name; /// Lowercased with underscores, e.g. "sha256_password".
bool is_password;
static const AuthenticationTypeInfo & get(AuthenticationType type_);
};
inline String toString(AuthenticationType type_)
{
return AuthenticationTypeInfo::get(type_).raw_name;
}
}

View File

@ -2,7 +2,7 @@
#include <Access/IAccessEntity.h>
#include <Access/AccessRights.h>
#include <Access/Common/AuthenticationData.h>
#include <Access/AuthenticationData.h>
#include <Access/Common/AllowedClientHosts.h>
#include <Access/GrantedRoles.h>
#include <Access/RolesOrUsersSet.h>

View File

@ -91,9 +91,9 @@ struct KolmogorovSmirnov : public StatisticalSample<Float64, Float64>
UInt64 ny_g = n2 / g;
if (method == "auto")
method = std::max(n1, n2) <= 10000 ? "exact" : "asymp";
method = std::max(n1, n2) <= 10000 ? "exact" : "asymptotic";
else if (method == "exact" && nx_g >= std::numeric_limits<Int32>::max() / ny_g)
method = "asymp";
method = "asymptotic";
Float64 p_value = std::numeric_limits<Float64>::infinity();
@ -143,7 +143,7 @@ struct KolmogorovSmirnov : public StatisticalSample<Float64, Float64>
}
p_value = c[n1];
}
else if (method == "asymp")
else if (method == "asymp" || method == "asymptotic")
{
Float64 n = std::min(n1, n2);
Float64 m = std::max(n1, n2);
@ -242,9 +242,9 @@ public:
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a String", getName());
method = params[1].get<String>();
if (method != "auto" && method != "exact" && method != "asymp")
if (method != "auto" && method != "exact" && method != "asymp" && method != "asymptotic")
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown method in aggregate function {}. "
"It must be one of: 'auto', 'exact', 'asymp'", getName());
"It must be one of: 'auto', 'exact', 'asymp' (or 'asymptotic')", getName());
}
String getName() const override

View File

@ -47,7 +47,7 @@ private:
using ColVecType = ColumnVectorOrDecimal<T>;
bool has_value = false; /// We need to remember if at least one value has been passed. This is necessary for AggregateFunctionIf.
T value;
T value = T{};
public:
static constexpr bool is_nullable = false;

View File

@ -163,15 +163,11 @@ public:
if constexpr (std::is_same_v<Data, QuantileTiming<Value>>)
{
/// QuantileTiming only supports unsigned integers. Too large values are also meaningless.
#ifdef OS_DARWIN
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wimplicit-const-int-float-conversion"
#endif
if (isNaN(value) || value > std::numeric_limits<Int64>::max() || value < 0)
return;
#ifdef OS_DARWIN
# pragma clang diagnostic pop
#endif
}
if constexpr (has_second_arg)

View File

@ -13,6 +13,7 @@
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <base/scope_guard.h>
extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size)
try

View File

@ -4081,12 +4081,12 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I
if (apply_transformer_was_used || replace_transformer_was_used)
continue;
replace_transformer_was_used = true;
auto replace_expression = replace_transformer->findReplacementExpression(column_name);
if (!replace_expression)
continue;
replace_transformer_was_used = true;
if (replace_transformer->isStrict())
strict_transformer_to_used_column_names[replace_transformer].insert(column_name);
@ -6679,7 +6679,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
bool is_rollup_or_cube = query_node_typed.isGroupByWithRollup() || query_node_typed.isGroupByWithCube();
if (query_node_typed.isGroupByWithGroupingSets() && query_node_typed.isGroupByWithTotals())
if (query_node_typed.isGroupByWithGroupingSets()
&& query_node_typed.isGroupByWithTotals()
&& query_node_typed.getGroupBy().getNodes().size() != 1)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS and GROUPING SETS are not supported together");
if (query_node_typed.isGroupByWithGroupingSets() && is_rollup_or_cube)

View File

@ -34,6 +34,7 @@
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTCreateFunctionQuery.h>
#include <Parsers/Access/ASTCreateUserQuery.h>
#include <Parsers/Access/ASTAuthenticationData.h>
#include <Parsers/ASTDropQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSetQuery.h>
@ -1612,10 +1613,15 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
if (const auto * create_user_query = parsed_query->as<ASTCreateUserQuery>())
{
if (!create_user_query->attach && create_user_query->temporary_password_for_checks)
if (!create_user_query->attach && create_user_query->auth_data)
{
global_context->getAccessControl().checkPasswordComplexityRules(create_user_query->temporary_password_for_checks.value());
create_user_query->temporary_password_for_checks.reset();
if (const auto * auth_data = create_user_query->auth_data->as<ASTAuthenticationData>())
{
auto password = auth_data->getPassword();
if (password)
global_context->getAccessControl().checkPasswordComplexityRules(*password);
}
}
}

View File

@ -67,8 +67,15 @@ AsynchronousMetrics::AsynchronousMetrics(
openFileIfExists("/proc/uptime", uptime);
openFileIfExists("/proc/net/dev", net_dev);
openFileIfExists("/sys/fs/cgroup/memory/memory.limit_in_bytes", cgroupmem_limit_in_bytes);
openFileIfExists("/sys/fs/cgroup/memory/memory.usage_in_bytes", cgroupmem_usage_in_bytes);
/// CGroups v2
openFileIfExists("/sys/fs/cgroup/memory.max", cgroupmem_limit_in_bytes);
openFileIfExists("/sys/fs/cgroup/memory.current", cgroupmem_usage_in_bytes);
/// CGroups v1
if (!cgroupmem_limit_in_bytes)
openFileIfExists("/sys/fs/cgroup/memory/memory.limit_in_bytes", cgroupmem_limit_in_bytes);
if (!cgroupmem_usage_in_bytes)
openFileIfExists("/sys/fs/cgroup/memory/memory.usage_in_bytes", cgroupmem_usage_in_bytes);
openSensors();
openBlockDevices();
@ -900,33 +907,25 @@ void AsynchronousMetrics::update(TimePoint update_time)
if (cgroupmem_limit_in_bytes && cgroupmem_usage_in_bytes)
{
try {
try
{
cgroupmem_limit_in_bytes->rewind();
cgroupmem_usage_in_bytes->rewind();
uint64_t cgroup_mem_limit_in_bytes = 0;
uint64_t cgroup_mem_usage_in_bytes = 0;
uint64_t limit = 0;
uint64_t usage = 0;
readText(cgroup_mem_limit_in_bytes, *cgroupmem_limit_in_bytes);
readText(cgroup_mem_usage_in_bytes, *cgroupmem_usage_in_bytes);
tryReadText(limit, *cgroupmem_limit_in_bytes);
tryReadText(usage, *cgroupmem_usage_in_bytes);
if (cgroup_mem_limit_in_bytes && cgroup_mem_usage_in_bytes)
{
new_values["CgroupMemoryTotal"] = { cgroup_mem_limit_in_bytes, "The total amount of memory in cgroup, in bytes." };
new_values["CgroupMemoryUsed"] = { cgroup_mem_usage_in_bytes, "The amount of memory used in cgroup, in bytes." };
}
else
{
LOG_DEBUG(log, "Cannot read statistics about the cgroup memory total and used. Total got '{}', Used got '{}'.",
cgroup_mem_limit_in_bytes, cgroup_mem_usage_in_bytes);
}
new_values["CGroupMemoryTotal"] = { limit, "The total amount of memory in cgroup, in bytes. If stated zero, the limit is the same as OSMemoryTotal." };
new_values["CGroupMemoryUsed"] = { usage, "The amount of memory used in cgroup, in bytes." };
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
if (meminfo)
{
try

View File

@ -0,0 +1,30 @@
#include <Common/Documentation.h>
namespace DB
{
std::string Documentation::examplesAsString() const
{
std::string res;
for (const auto & [example_name, example_query] : examples)
{
res += example_name + ":\n\n";
res += "```sql\n";
res += example_query + "\n";
res += "```\n";
}
return res;
}
std::string Documentation::categoriesAsString() const
{
if (categories.empty())
return "";
std::string res = categories[0];
for (size_t i = 1; i < categories.size(); ++i)
res += ", " + categories[i];
return res;
}
}

View File

@ -42,27 +42,44 @@ namespace DB
*
* Documentation does not support multiple languages.
* The only available language is English.
*
* TODO: Allow to specify Syntax, Argument(s) and a Returned Value.
* TODO: Organize Examples as a struct of ExampleName, ExampleQuery and ExampleResult.
*/
struct Documentation
{
using Description = std::string;
using Syntax = std::string;
using Argument = std::string;
using Arguments = std::vector<Argument>;
using ReturnedValue = std::string;
using ExampleName = std::string;
using ExampleQuery = std::string;
using Examples = std::map<ExampleName, ExampleQuery>;
using Category = std::string;
using Categories = std::vector<Category>;
using Related = std::string;
Description description;
Examples examples;
Categories categories;
Documentation(Description description_) : description(std::move(description_)) {}
Documentation(Description description_) : description(std::move(description_)) {} /// NOLINT
Documentation(Description description_, Examples examples_) : description(std::move(description_)), examples(std::move(examples_)) {}
Documentation(Description description_, Examples examples_, Categories categories_)
: description(std::move(description_)), examples(std::move(examples_)), categories(std::move(categories_)) {}
/// TODO: Please remove this constructor. Documentation should always be non-empty.
Documentation() {}
Documentation() = default;
std::string examplesAsString() const;
std::string categoriesAsString() const;
};
}

View File

@ -10,6 +10,7 @@
M(InsertQuery, "Same as Query, but only for INSERT queries.") \
M(AsyncInsertQuery, "Same as InsertQuery, but only for asynchronous INSERT queries.") \
M(AsyncInsertBytes, "Data size in bytes of asynchronous INSERT queries.") \
M(AsyncInsertRows, "Number of rows inserted by asynchronous INSERT queries.") \
M(AsyncInsertCacheHits, "Number of times a duplicate hash id has been found in asynchronous INSERT hash id cache.") \
M(FailedQuery, "Number of failed queries.") \
M(FailedSelectQuery, "Same as FailedQuery, but only for SELECT queries.") \

View File

@ -7,6 +7,7 @@
#include <Parsers/ASTIdentifier.h>
#include <Poco/Logger.h>
#include <Common/logger_useful.h>
#include "libaccel_config.h"
namespace DB
{
@ -16,11 +17,6 @@ namespace ErrorCodes
extern const int CANNOT_DECOMPRESS;
}
std::array<qpl_job *, DeflateQplJobHWPool::MAX_HW_JOB_NUMBER> DeflateQplJobHWPool::hw_job_ptr_pool;
std::array<std::atomic_bool, DeflateQplJobHWPool::MAX_HW_JOB_NUMBER> DeflateQplJobHWPool::hw_job_ptr_locks;
bool DeflateQplJobHWPool::job_pool_ready = false;
std::unique_ptr<uint8_t[]> DeflateQplJobHWPool::hw_jobs_buffer;
DeflateQplJobHWPool & DeflateQplJobHWPool::instance()
{
static DeflateQplJobHWPool pool;
@ -28,47 +24,69 @@ DeflateQplJobHWPool & DeflateQplJobHWPool::instance()
}
DeflateQplJobHWPool::DeflateQplJobHWPool()
: random_engine(std::random_device()())
, distribution(0, MAX_HW_JOB_NUMBER - 1)
: max_hw_jobs(0)
, random_engine(std::random_device()())
{
Poco::Logger * log = &Poco::Logger::get("DeflateQplJobHWPool");
UInt32 job_size = 0;
const char * qpl_version = qpl_get_library_version();
/// Get size required for saving a single qpl job object
qpl_get_job_size(qpl_path_hardware, &job_size);
/// Allocate entire buffer for storing all job objects
hw_jobs_buffer = std::make_unique<uint8_t[]>(job_size * MAX_HW_JOB_NUMBER);
/// Initialize pool for storing all job object pointers
/// Reallocate buffer by shifting address offset for each job object.
for (UInt32 index = 0; index < MAX_HW_JOB_NUMBER; ++index)
// loop all configured workqueue size to get maximum job number.
accfg_ctx * ctx_ptr = nullptr;
auto ctx_status = accfg_new(&ctx_ptr);
if (ctx_status == 0)
{
qpl_job * qpl_job_ptr = reinterpret_cast<qpl_job *>(hw_jobs_buffer.get() + index * job_size);
if (auto status = qpl_init_job(qpl_path_hardware, qpl_job_ptr); status != QPL_STS_OK)
auto * dev_ptr = accfg_device_get_first(ctx_ptr);
while (dev_ptr != nullptr)
{
for (auto * wq_ptr = accfg_wq_get_first(dev_ptr); wq_ptr != nullptr; wq_ptr = accfg_wq_get_next(wq_ptr))
max_hw_jobs += accfg_wq_get_size(wq_ptr);
dev_ptr = accfg_device_get_next(dev_ptr);
}
}
else
{
job_pool_ready = false;
LOG_WARNING(log, "Initialization of hardware-assisted DeflateQpl codec failed, falling back to software DeflateQpl codec. Failed to create new libaccel_config context -> status: {}, QPL Version: {}.", ctx_status, qpl_version);
return;
}
if (max_hw_jobs == 0)
{
job_pool_ready = false;
LOG_WARNING(log, "Initialization of hardware-assisted DeflateQpl codec failed, falling back to software DeflateQpl codec. Failed to get available workqueue size -> total_wq_size: {}, QPL Version: {}.", max_hw_jobs, qpl_version);
return;
}
distribution = std::uniform_int_distribution<int>(0, max_hw_jobs - 1);
/// Get size required for saving a single qpl job object
qpl_get_job_size(qpl_path_hardware, &per_job_size);
/// Allocate job buffer pool for storing all job objects
hw_jobs_buffer = std::make_unique<uint8_t[]>(per_job_size * max_hw_jobs);
hw_job_ptr_locks = std::make_unique<std::atomic_bool[]>(max_hw_jobs);
/// Initialize all job objects in job buffer pool
for (UInt32 index = 0; index < max_hw_jobs; ++index)
{
qpl_job * job_ptr = reinterpret_cast<qpl_job *>(hw_jobs_buffer.get() + index * per_job_size);
if (auto status = qpl_init_job(qpl_path_hardware, job_ptr); status != QPL_STS_OK)
{
job_pool_ready = false;
LOG_WARNING(log, "Initialization of hardware-assisted DeflateQpl codec failed: {} , falling back to software DeflateQpl codec. Please check if Intel In-Memory Analytics Accelerator (IAA) is properly set up. QPL Version: {}.", static_cast<UInt32>(status), qpl_version);
LOG_WARNING(log, "Initialization of hardware-assisted DeflateQpl codec failed, falling back to software DeflateQpl codec. Failed to Initialize qpl job -> status: {}, QPL Version: {}.", static_cast<UInt32>(status), qpl_version);
return;
}
hw_job_ptr_pool[index] = qpl_job_ptr;
unLockJob(index);
}
job_pool_ready = true;
LOG_DEBUG(log, "Hardware-assisted DeflateQpl codec is ready! QPL Version: {}",qpl_version);
LOG_DEBUG(log, "Hardware-assisted DeflateQpl codec is ready! QPL Version: {}, max_hw_jobs: {}",qpl_version, max_hw_jobs);
}
DeflateQplJobHWPool::~DeflateQplJobHWPool()
{
for (UInt32 i = 0; i < MAX_HW_JOB_NUMBER; ++i)
for (UInt32 i = 0; i < max_hw_jobs; ++i)
{
if (hw_job_ptr_pool[i])
{
while (!tryLockJob(i));
qpl_fini_job(hw_job_ptr_pool[i]);
unLockJob(i);
hw_job_ptr_pool[i] = nullptr;
}
qpl_job * job_ptr = reinterpret_cast<qpl_job *>(hw_jobs_buffer.get() + i * per_job_size);
while (!tryLockJob(i));
qpl_fini_job(job_ptr);
unLockJob(i);
}
job_pool_ready = false;
}
@ -83,14 +101,14 @@ qpl_job * DeflateQplJobHWPool::acquireJob(UInt32 & job_id)
{
index = distribution(random_engine);
retry++;
if (retry > MAX_HW_JOB_NUMBER)
if (retry > max_hw_jobs)
{
return nullptr;
}
}
job_id = MAX_HW_JOB_NUMBER - index;
assert(index < MAX_HW_JOB_NUMBER);
return hw_job_ptr_pool[index];
job_id = max_hw_jobs - index;
assert(index < max_hw_jobs);
return reinterpret_cast<qpl_job *>(hw_jobs_buffer.get() + index * per_job_size);
}
else
return nullptr;
@ -99,19 +117,19 @@ qpl_job * DeflateQplJobHWPool::acquireJob(UInt32 & job_id)
void DeflateQplJobHWPool::releaseJob(UInt32 job_id)
{
if (isJobPoolReady())
unLockJob(MAX_HW_JOB_NUMBER - job_id);
unLockJob(max_hw_jobs - job_id);
}
bool DeflateQplJobHWPool::tryLockJob(UInt32 index)
{
bool expected = false;
assert(index < MAX_HW_JOB_NUMBER);
assert(index < max_hw_jobs);
return hw_job_ptr_locks[index].compare_exchange_strong(expected, true);
}
void DeflateQplJobHWPool::unLockJob(UInt32 index)
{
assert(index < MAX_HW_JOB_NUMBER);
assert(index < max_hw_jobs);
hw_job_ptr_locks[index].store(false);
}

View File

@ -24,22 +24,23 @@ public:
static DeflateQplJobHWPool & instance();
qpl_job * acquireJob(UInt32 & job_id);
static void releaseJob(UInt32 job_id);
static const bool & isJobPoolReady() { return job_pool_ready; }
void releaseJob(UInt32 job_id);
const bool & isJobPoolReady() { return job_pool_ready; }
private:
static bool tryLockJob(UInt32 index);
static void unLockJob(UInt32 index);
bool tryLockJob(UInt32 index);
void unLockJob(UInt32 index);
/// size of each job objects
UInt32 per_job_size;
/// Maximum jobs running in parallel supported by IAA hardware
static constexpr auto MAX_HW_JOB_NUMBER = 1024;
UInt32 max_hw_jobs;
/// Entire buffer for storing all job objects
static std::unique_ptr<uint8_t[]> hw_jobs_buffer;
/// Job pool for storing all job object pointers
static std::array<qpl_job *, MAX_HW_JOB_NUMBER> hw_job_ptr_pool;
std::unique_ptr<uint8_t[]> hw_jobs_buffer;
/// Locks for accessing each job object pointers
static std::array<std::atomic_bool, MAX_HW_JOB_NUMBER> hw_job_ptr_locks;
static bool job_pool_ready;
std::unique_ptr<std::atomic_bool[]> hw_job_ptr_locks;
bool job_pool_ready;
std::mt19937 random_engine;
std::uniform_int_distribution<int> distribution;
};

View File

@ -71,6 +71,7 @@ class IColumn;
M(UInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.", 0) \
M(UInt64, distributed_connections_pool_size, 1024, "Maximum number of connections with one remote server in the pool.", 0) \
M(UInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \
M(UInt64, s3_strict_upload_part_size, 0, "The exact size of part to upload during multipart upload to S3 (some implementations does not supports variable size parts).", 0) \
M(UInt64, s3_min_upload_part_size, 16*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \
M(UInt64, s3_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to S3.", 0) \
M(UInt64, s3_upload_part_size_multiply_factor, 2, "Multiply s3_min_upload_part_size by this factor each time s3_multiply_parts_count_threshold parts were uploaded from a single write to S3.", 0) \
@ -709,6 +710,8 @@ class IColumn;
\
M(String, workload, "default", "Name of workload to be used to access resources", 0) \
\
M(Bool, parallelize_output_from_storages, false, "Parallelize output for reading step from storage. It allows parallelizing query processing right after reading from storage if possible", 0) \
\
/** Experimental functions */ \
M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \
M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \

View File

@ -17,6 +17,7 @@
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
#include <Common/quoteString.h>
#include <Common/filesystemHelpers.h>
#include <Common/logger_useful.h>
#include <filesystem>
namespace fs = std::filesystem;
@ -51,6 +52,7 @@ DatabasePostgreSQL::DatabasePostgreSQL(
, configuration(configuration_)
, pool(std::move(pool_))
, cache_tables(cache_tables_)
, log(&Poco::Logger::get("DatabasePostgreSQL(" + dbname_ + ")"))
{
cleaner_task = getContext()->getSchedulePool().createTask("PostgreSQLCleanerTask", [this]{ removeOutdatedTables(); });
cleaner_task->deactivate();
@ -192,7 +194,10 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr,
ColumnsDescription{columns_info->columns}, ConstraintsDescription{}, String{}, configuration.schema, configuration.on_conflict);
if (cache_tables)
{
LOG_TEST(log, "Cached table `{}`", table_name);
cached_tables[table_name] = storage;
}
return storage;
}

View File

@ -73,6 +73,7 @@ private:
mutable Tables cached_tables;
std::unordered_set<std::string> detached_or_dropped;
BackgroundSchedulePool::TaskHolder cleaner_task;
Poco::Logger * log;
String getTableNameForLogs(const String & table_name) const;

View File

@ -3,13 +3,13 @@
#include "DictionaryStructure.h"
#include "registerDictionaries.h"
#include <Storages/ExternalDataSourceConfiguration.h>
#include <Storages/StorageMongoDBSocketFactory.h>
namespace DB
{
static const std::unordered_set<std::string_view> dictionary_allowed_keys = {
"host", "port", "user", "password", "db", "database", "uri", "collection", "name", "method"};
"host", "port", "user", "password", "db", "database", "uri", "collection", "name", "method", "options"};
void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
{
@ -51,6 +51,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
config.getString(config_prefix + ".method", ""),
configuration.database,
config.getString(config_prefix + ".collection"),
config.getString(config_prefix + ".options", ""),
sample_block);
};
@ -98,6 +99,7 @@ MongoDBDictionarySource::MongoDBDictionarySource(
const std::string & method_,
const std::string & db_,
const std::string & collection_,
const std::string & options_,
const Block & sample_block_)
: dict_struct{dict_struct_}
, uri{uri_}
@ -108,13 +110,15 @@ MongoDBDictionarySource::MongoDBDictionarySource(
, method{method_}
, db{db_}
, collection{collection_}
, options(options_)
, sample_block{sample_block_}
, connection{std::make_shared<Poco::MongoDB::Connection>()}
{
StorageMongoDBSocketFactory socket_factory;
if (!uri.empty())
{
// Connect with URI.
Poco::MongoDB::Connection::SocketFactory socket_factory;
connection->connect(uri, socket_factory);
Poco::URI poco_uri(connection->uri());
@ -140,8 +144,10 @@ MongoDBDictionarySource::MongoDBDictionarySource(
}
else
{
// Connect with host/port/user/etc.
connection->connect(host, port);
// Connect with host/port/user/etc through constructing the uri
std::string uri_constructed("mongodb://" + host + ":" + std::to_string(port) + "/" + db + (options.empty() ? "" : "?" + options));
connection->connect(uri_constructed, socket_factory);
if (!user.empty())
{
Poco::MongoDB::Database poco_db(db);
@ -154,7 +160,9 @@ MongoDBDictionarySource::MongoDBDictionarySource(
MongoDBDictionarySource::MongoDBDictionarySource(const MongoDBDictionarySource & other)
: MongoDBDictionarySource{
other.dict_struct, other.uri, other.host, other.port, other.user, other.password, other.method, other.db, other.collection, other.sample_block}
other.dict_struct, other.uri, other.host, other.port, other.user, other.password, other.method, other.db,
other.collection, other.options, other.sample_block
}
{
}

View File

@ -41,6 +41,7 @@ public:
const std::string & method_,
const std::string & db_,
const std::string & collection_,
const std::string & options,
const Block & sample_block_);
MongoDBDictionarySource(const MongoDBDictionarySource & other);
@ -80,6 +81,7 @@ private:
const std::string method;
std::string db;
const std::string collection;
const std::string options;
Block sample_block;
std::shared_ptr<Poco::MongoDB::Connection> connection;

View File

@ -1,6 +1,7 @@
#include "CachedOnDiskReadBufferFromFile.h"
#include <Disks/IO/createReadBufferFromFileBase.h>
#include <Disks/ObjectStorages/Cached/CachedObjectStorage.h>
#include <IO/ReadBufferFromFile.h>
#include <base/scope_guard.h>
#include <Common/assert_cast.h>
@ -115,27 +116,25 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size)
if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache)
{
file_segments_holder.emplace(cache->get(cache_key, offset, size));
file_segments = cache->get(cache_key, offset, size);
}
else
{
CreateFileSegmentSettings create_settings(is_persistent ? FileSegmentKind::Persistent : FileSegmentKind::Regular);
file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, create_settings));
file_segments = cache->getOrSet(cache_key, offset, size, create_settings);
}
/**
* Segments in returned list are ordered in ascending order and represent a full contiguous
* interval (no holes). Each segment in returned list has state: DOWNLOADED, DOWNLOADING or EMPTY.
*/
if (file_segments_holder->file_segments.empty())
if (file_segments->empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "List of file segments cannot be empty");
LOG_TEST(
log,
"Having {} file segments to read: {}, current offset: {}",
file_segments_holder->file_segments.size(), file_segments_holder->toString(), file_offset_of_buffer_end);
current_file_segment_it = file_segments_holder->file_segments.begin();
file_segments->size(), file_segments->toString(), file_offset_of_buffer_end);
initialized = true;
}
@ -165,7 +164,7 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm
}
CachedOnDiskReadBufferFromFile::ImplementationBufferPtr
CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegment & file_segment, ReadType read_type_)
CachedOnDiskReadBufferFromFile::getRemoteReadBuffer(FileSegment & file_segment, ReadType read_type_)
{
switch (read_type_)
{
@ -202,7 +201,7 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegment & file_segment
}
else
{
chassert(remote_fs_segment_reader->getFileOffsetOfBufferEnd() == file_segment.getCurrentWriteOffset());
chassert(remote_fs_segment_reader->getFileOffsetOfBufferEnd() == file_segment.getCurrentWriteOffset(false));
}
return remote_fs_segment_reader;
@ -239,27 +238,27 @@ bool CachedOnDiskReadBufferFromFile::canStartFromCache(size_t current_offset, co
/// requested_range: [__________]
/// ^
/// current_offset
size_t first_non_downloaded_offset = file_segment.getFirstNonDownloadedOffset();
size_t first_non_downloaded_offset = file_segment.getFirstNonDownloadedOffset(true);
return first_non_downloaded_offset > current_offset;
}
CachedOnDiskReadBufferFromFile::ImplementationBufferPtr
CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & file_segment)
CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegment & file_segment)
{
auto download_state = file_segment->state();
auto download_state = file_segment.state();
if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache)
{
if (download_state == FileSegment::State::DOWNLOADED)
{
read_type = ReadType::CACHED;
return getCacheReadBuffer(*file_segment);
return getCacheReadBuffer(file_segment);
}
else
{
LOG_TEST(log, "Bypassing cache because `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` option is used");
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
return getRemoteFSReadBuffer(*file_segment, read_type);
return getRemoteReadBuffer(file_segment, read_type);
}
}
@ -267,15 +266,15 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
{
switch (download_state)
{
case FileSegment::State::SKIP_CACHE:
case FileSegment::State::DETACHED:
{
LOG_TRACE(log, "Bypassing cache because file segment state is `SKIP_CACHE`");
LOG_TRACE(log, "Bypassing cache because file segment state is `DETACHED`");
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
return getRemoteFSReadBuffer(*file_segment, read_type);
return getRemoteReadBuffer(file_segment, read_type);
}
case FileSegment::State::DOWNLOADING:
{
if (canStartFromCache(file_offset_of_buffer_end, *file_segment))
if (canStartFromCache(file_offset_of_buffer_end, file_segment))
{
/// segment{k} state: DOWNLOADING
/// cache: [______|___________
@ -286,21 +285,21 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
/// file_offset_of_buffer_end
read_type = ReadType::CACHED;
return getCacheReadBuffer(*file_segment);
return getCacheReadBuffer(file_segment);
}
download_state = file_segment->wait();
download_state = file_segment.wait(file_offset_of_buffer_end);
continue;
}
case FileSegment::State::DOWNLOADED:
{
read_type = ReadType::CACHED;
return getCacheReadBuffer(*file_segment);
return getCacheReadBuffer(file_segment);
}
case FileSegment::State::EMPTY:
case FileSegment::State::PARTIALLY_DOWNLOADED:
{
if (canStartFromCache(file_offset_of_buffer_end, *file_segment))
if (canStartFromCache(file_offset_of_buffer_end, file_segment))
{
/// segment{k} state: PARTIALLY_DOWNLOADED
/// cache: [______|___________
@ -311,13 +310,13 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
/// file_offset_of_buffer_end
read_type = ReadType::CACHED;
return getCacheReadBuffer(*file_segment);
return getCacheReadBuffer(file_segment);
}
auto downloader_id = file_segment->getOrSetDownloader();
if (downloader_id == file_segment->getCallerId())
auto downloader_id = file_segment.getOrSetDownloader();
if (downloader_id == file_segment.getCallerId())
{
if (canStartFromCache(file_offset_of_buffer_end, *file_segment))
if (canStartFromCache(file_offset_of_buffer_end, file_segment))
{
/// segment{k}
/// cache: [______|___________
@ -328,11 +327,12 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
/// file_offset_of_buffer_end
read_type = ReadType::CACHED;
file_segment->resetDownloader();
return getCacheReadBuffer(*file_segment);
file_segment.resetDownloader();
return getCacheReadBuffer(file_segment);
}
if (file_segment->getCurrentWriteOffset() < file_offset_of_buffer_end)
auto current_write_offset = file_segment.getCurrentWriteOffset(false);
if (current_write_offset < file_offset_of_buffer_end)
{
/// segment{1}
/// cache: [_____|___________
@ -342,25 +342,25 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
/// ^
/// file_offset_of_buffer_end
LOG_TEST(log, "Predownload. File segment info: {}", file_segment->getInfoForLog());
chassert(file_offset_of_buffer_end > file_segment->getCurrentWriteOffset());
bytes_to_predownload = file_offset_of_buffer_end - file_segment->getCurrentWriteOffset();
chassert(bytes_to_predownload < file_segment->range().size());
LOG_TEST(log, "Predownload. File segment info: {}", file_segment.getInfoForLog());
chassert(file_offset_of_buffer_end > current_write_offset);
bytes_to_predownload = file_offset_of_buffer_end - current_write_offset;
chassert(bytes_to_predownload < file_segment.range().size());
}
read_type = ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE;
return getRemoteFSReadBuffer(*file_segment, read_type);
return getRemoteReadBuffer(file_segment, read_type);
}
download_state = file_segment->state();
download_state = file_segment.state();
continue;
}
case FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION:
{
if (canStartFromCache(file_offset_of_buffer_end, *file_segment))
if (canStartFromCache(file_offset_of_buffer_end, file_segment))
{
read_type = ReadType::CACHED;
return getCacheReadBuffer(*file_segment);
return getCacheReadBuffer(file_segment);
}
else
{
@ -368,7 +368,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
log,
"Bypassing cache because file segment state is `PARTIALLY_DOWNLOADED_NO_CONTINUATION` and downloaded part already used");
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
return getRemoteFSReadBuffer(*file_segment, read_type);
return getRemoteReadBuffer(file_segment, read_type);
}
}
}
@ -376,12 +376,12 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
}
CachedOnDiskReadBufferFromFile::ImplementationBufferPtr
CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_segment)
CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegment & file_segment)
{
chassert(!file_segment->isDownloader());
chassert(file_offset_of_buffer_end >= file_segment->range().left);
chassert(!file_segment.isDownloader());
chassert(file_offset_of_buffer_end >= file_segment.range().left);
auto range = file_segment->range();
auto range = file_segment.range();
bytes_to_predownload = 0;
Stopwatch watch(CLOCK_MONOTONIC);
@ -393,17 +393,18 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se
ProfileEvents::FileSegmentWaitReadBufferMicroseconds, watch.elapsedMicroseconds());
[[maybe_unused]] auto download_current_segment = read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE;
chassert(download_current_segment == file_segment->isDownloader());
chassert(download_current_segment == file_segment.isDownloader());
chassert(file_segment->range() == range);
chassert(file_segment.range() == range);
chassert(file_offset_of_buffer_end >= range.left && file_offset_of_buffer_end <= range.right);
LOG_TEST(
log,
"Current file segment: {}, read type: {}, current file offset: {}",
range.toString(),
"Current read type: {}, read offset: {}, impl read range: {}, file segment: {}",
toString(read_type),
file_offset_of_buffer_end);
file_offset_of_buffer_end,
read_buffer_for_file_segment->getFileOffsetOfBufferEnd(),
file_segment.getInfoForLog());
read_buffer_for_file_segment->setReadUntilPosition(range.right + 1); /// [..., range.right]
@ -445,11 +446,11 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se
}
case ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE:
{
chassert(file_segment->isDownloader());
chassert(file_segment.isDownloader());
if (bytes_to_predownload)
{
size_t current_write_offset = file_segment->getCurrentWriteOffset();
const size_t current_write_offset = file_segment.getCurrentWriteOffset(false);
read_buffer_for_file_segment->seek(current_write_offset, SEEK_SET);
}
else
@ -459,7 +460,7 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se
assert(read_buffer_for_file_segment->getFileOffsetOfBufferEnd() == file_offset_of_buffer_end);
}
auto current_write_offset = file_segment->getCurrentWriteOffset();
const auto current_write_offset = file_segment.getCurrentWriteOffset(false);
if (current_write_offset != static_cast<size_t>(read_buffer_for_file_segment->getPosition()))
{
throw Exception(
@ -470,7 +471,7 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se
current_write_offset,
read_buffer_for_file_segment->getPosition(),
read_buffer_for_file_segment->getFileOffsetOfBufferEnd(),
file_segment->getInfoForLog());
file_segment.getInfoForLog());
}
break;
@ -484,52 +485,46 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se
bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext()
{
LOG_TEST(log, "Completed segment: {}", (*current_file_segment_it)->range().toString());
auto * current_file_segment = &file_segments->front();
auto completed_range = current_file_segment->range();
if (enable_logging)
appendFilesystemCacheLog((*current_file_segment_it)->range(), read_type);
appendFilesystemCacheLog(completed_range, read_type);
auto file_segment_it = current_file_segment_it++;
auto & file_segment = *file_segment_it;
[[maybe_unused]] const auto & range = file_segment->range();
chassert(file_offset_of_buffer_end > range.right);
LOG_TEST(
log,
"Removing file segment: {}, downloader: {}, state: {}",
file_segment->range().toString(),
file_segment->getDownloader(),
file_segment->state());
/// Do not hold pointer to file segment if it is not needed anymore
/// so can become releasable and can be evicted from cache.
file_segment->completeWithoutState();
file_segments_holder->file_segments.erase(file_segment_it);
if (current_file_segment_it == file_segments_holder->file_segments.end())
return false;
implementation_buffer = getImplementationBuffer(*current_file_segment_it);
chassert(file_offset_of_buffer_end > completed_range.right);
if (read_type == ReadType::CACHED)
(*current_file_segment_it)->incrementHitsCount();
{
chassert(current_file_segment->getDownloadedSize(true) == current_file_segment->range().size());
}
file_segments->popFront();
if (file_segments->empty())
return false;
current_file_segment = &file_segments->front();
current_file_segment->use();
implementation_buffer = getImplementationBuffer(*current_file_segment);
if (read_type == ReadType::CACHED)
current_file_segment->incrementHitsCount();
LOG_TEST(
log, "New segment range: {}, old range: {}",
current_file_segment->range().toString(), completed_range.toString());
LOG_TEST(log, "New segment: {}", (*current_file_segment_it)->range().toString());
return true;
}
CachedOnDiskReadBufferFromFile::~CachedOnDiskReadBufferFromFile()
{
if (enable_logging
&& file_segments_holder
&& current_file_segment_it != file_segments_holder->file_segments.end())
if (enable_logging && file_segments && !file_segments->empty())
{
appendFilesystemCacheLog((*current_file_segment_it)->range(), read_type);
appendFilesystemCacheLog(file_segments->front().range(), read_type);
}
}
void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment)
{
Stopwatch predownload_watch(CLOCK_MONOTONIC);
SCOPE_EXIT({
@ -548,9 +543,10 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
/// download from offset a'' < a', but return buffer from offset a'.
LOG_TEST(log, "Bytes to predownload: {}, caller_id: {}", bytes_to_predownload, FileSegment::getCallerId());
chassert(static_cast<size_t>(implementation_buffer->getPosition()) == file_segment->getCurrentWriteOffset());
size_t current_offset = file_segment->getCurrentWriteOffset();
const auto & current_range = file_segment->range();
/// chassert(implementation_buffer->getFileOffsetOfBufferEnd() == file_segment.getCurrentWriteOffset(false));
chassert(static_cast<size_t>(implementation_buffer->getPosition()) == file_segment.getCurrentWriteOffset(false));
size_t current_offset = file_segment.getCurrentWriteOffset(false);
const auto & current_range = file_segment.range();
while (true)
{
@ -575,7 +571,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
"current download offset: {}, expected: {}, eof: {}",
bytes_to_predownload,
current_range.toString(),
file_segment->getCurrentWriteOffset(),
file_segment.getCurrentWriteOffset(false),
file_offset_of_buffer_end,
implementation_buffer->eof());
@ -585,7 +581,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
{
nextimpl_working_buffer_offset = implementation_buffer->offset();
auto current_write_offset = file_segment->getCurrentWriteOffset();
auto current_write_offset = file_segment.getCurrentWriteOffset(false);
if (current_write_offset != static_cast<size_t>(implementation_buffer->getPosition())
|| current_write_offset != file_offset_of_buffer_end)
{
@ -597,7 +593,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
current_write_offset,
file_offset_of_buffer_end,
implementation_buffer->getPosition(),
file_segment->getInfoForLog());
file_segment.getInfoForLog());
}
}
@ -609,15 +605,15 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromSourceBytes, current_impl_buffer_size);
bool continue_predownload = file_segment->reserve(current_predownload_size);
bool continue_predownload = file_segment.reserve(current_predownload_size);
if (continue_predownload)
{
LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size);
chassert(file_segment->getCurrentWriteOffset() == static_cast<size_t>(implementation_buffer->getPosition()));
chassert(file_segment.getCurrentWriteOffset(false) == static_cast<size_t>(implementation_buffer->getPosition()));
bool success = writeCache(implementation_buffer->buffer().begin(), current_predownload_size, current_offset, *file_segment);
if (success)
continue_predownload = writeCache(implementation_buffer->buffer().begin(), current_predownload_size, current_offset, file_segment);
if (continue_predownload)
{
current_offset += current_predownload_size;
@ -627,13 +623,8 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
else
{
LOG_TEST(log, "Bypassing cache because writeCache (in predownload) method failed");
continue_predownload = false;
}
}
else
{
file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
}
if (!continue_predownload)
{
@ -653,21 +644,21 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
/// TODO: allow seek more than once with seek avoiding.
bytes_to_predownload = 0;
file_segment.completePartAndResetDownloader();
chassert(file_segment.state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
chassert(file_segment->state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION
|| file_segment->state() == FileSegment::State::SKIP_CACHE);
LOG_TEST(log, "Bypassing cache because for {}", file_segment->getInfoForLog());
LOG_TEST(log, "Bypassing cache because for {}", file_segment.getInfoForLog());
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
swap(*implementation_buffer);
resetWorkingBuffer();
implementation_buffer = getRemoteFSReadBuffer(*file_segment, read_type);
implementation_buffer = getRemoteReadBuffer(file_segment, read_type);
swap(*implementation_buffer);
implementation_buffer->setReadUntilPosition(file_segment->range().right + 1); /// [..., range.right]
implementation_buffer->setReadUntilPosition(file_segment.range().right + 1); /// [..., range.right]
implementation_buffer->seek(file_offset_of_buffer_end, SEEK_SET);
LOG_TRACE(
@ -684,12 +675,12 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded()
{
auto & file_segment = *current_file_segment_it;
auto current_read_range = file_segment->range();
auto current_state = file_segment->state();
auto & file_segment = file_segments->front();
const auto & current_read_range = file_segment.range();
auto current_state = file_segment.state();
chassert(current_read_range.left <= file_offset_of_buffer_end);
chassert(!file_segment->isDownloader());
chassert(!file_segment.isDownloader());
if (file_offset_of_buffer_end > current_read_range.right)
{
@ -708,7 +699,7 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded()
/// ^
/// file_offset_of_buffer_end
auto current_write_offset = file_segment->getCurrentWriteOffset();
auto current_write_offset = file_segment.getCurrentWriteOffset(true);
bool cached_part_is_finished = current_write_offset == file_offset_of_buffer_end;
LOG_TEST(log, "Current write offset: {}, file offset of buffer end: {}", current_write_offset, file_offset_of_buffer_end);
@ -716,7 +707,7 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded()
if (cached_part_is_finished)
{
/// TODO: makes sense to reuse local file reader if we return here with CACHED read type again?
implementation_buffer = getImplementationBuffer(*current_file_segment_it);
implementation_buffer = getImplementationBuffer(file_segment);
return true;
}
@ -744,7 +735,7 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded()
* to read by marks range given to him. Therefore, each nextImpl() call, in case of
* READ_AND_PUT_IN_CACHE, starts with getOrSetDownloader().
*/
implementation_buffer = getImplementationBuffer(*current_file_segment_it);
implementation_buffer = getImplementationBuffer(file_segment);
}
return true;
@ -795,15 +786,13 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
{
last_caller_id = FileSegment::getCallerId();
assertCorrectness();
if (file_offset_of_buffer_end == read_until_position)
return false;
if (!initialized)
initialize(file_offset_of_buffer_end, getTotalSizeToRead());
if (current_file_segment_it == file_segments_holder->file_segments.end())
if (file_segments->empty())
return false;
bool implementation_buffer_can_be_reused = false;
@ -813,25 +802,25 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
/// Save state of current file segment before it is completed.
nextimpl_step_log_info = getInfoForLog();
if (current_file_segment_it == file_segments_holder->file_segments.end())
if (file_segments->empty())
return;
auto & file_segment = *current_file_segment_it;
auto & file_segment = file_segments->front();
bool download_current_segment = read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE;
if (download_current_segment)
{
bool need_complete_file_segment = file_segment->isDownloader();
bool need_complete_file_segment = file_segment.isDownloader();
if (need_complete_file_segment)
{
if (!implementation_buffer_can_be_reused)
file_segment->resetRemoteFileReader();
file_segment.resetRemoteFileReader();
file_segment->completePartAndResetDownloader();
file_segment.completePartAndResetDownloader();
}
}
chassert(!file_segment->isDownloader());
chassert(!file_segment.isDownloader());
}
catch (...)
{
@ -849,10 +838,10 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
}
else
{
implementation_buffer = getImplementationBuffer(*current_file_segment_it);
implementation_buffer = getImplementationBuffer(file_segments->front());
if (read_type == ReadType::CACHED)
(*current_file_segment_it)->incrementHitsCount();
file_segments->front().incrementHitsCount();
}
chassert(!internal_buffer.empty());
@ -863,16 +852,16 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
// the caller doesn't try to use this CachedOnDiskReadBufferFromFile after it threw an exception.)
swap(*implementation_buffer);
auto & file_segment = *current_file_segment_it;
auto current_read_range = file_segment->range();
auto & file_segment = file_segments->front();
const auto & current_read_range = file_segment.range();
LOG_TEST(
log,
"Current count: {}, position: {}, buffer end: {}, file segment: {}",
implementation_buffer->count(),
implementation_buffer->getPosition(),
"Current read type: {}, read offset: {}, impl offset: {}, file segment: {}",
toString(read_type),
file_offset_of_buffer_end,
implementation_buffer->getFileOffsetOfBufferEnd(),
file_segment->getInfoForLog());
file_segment.getInfoForLog());
chassert(current_read_range.left <= file_offset_of_buffer_end);
chassert(current_read_range.right >= file_offset_of_buffer_end);
@ -890,12 +879,12 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
}
auto download_current_segment = read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE;
if (download_current_segment != file_segment->isDownloader())
if (download_current_segment != file_segment.isDownloader())
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Incorrect segment state. Having read type: {}, file segment info: {}",
toString(read_type), file_segment->getInfoForLog());
toString(read_type), file_segment.getInfoForLog());
}
if (!result)
@ -937,7 +926,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
log,
"Read {} bytes, read type {}, position: {}, offset: {}, segment end: {}",
size, toString(read_type), implementation_buffer->getPosition(),
implementation_buffer->getFileOffsetOfBufferEnd(), file_segment->range().right);
implementation_buffer->getFileOffsetOfBufferEnd(), file_segment.range().right);
if (read_type == ReadType::CACHED)
{
@ -955,20 +944,20 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
{
if (download_current_segment)
{
chassert(file_offset_of_buffer_end + size - 1 <= file_segment->range().right);
chassert(file_offset_of_buffer_end + size - 1 <= file_segment.range().right);
bool success = file_segment->reserve(size);
bool success = file_segment.reserve(size);
if (success)
{
chassert(file_segment->getCurrentWriteOffset() == static_cast<size_t>(implementation_buffer->getPosition()));
chassert(file_segment.getCurrentWriteOffset(false) == static_cast<size_t>(implementation_buffer->getPosition()));
success = writeCache(implementation_buffer->position(), size, file_offset_of_buffer_end, *file_segment);
success = writeCache(implementation_buffer->position(), size, file_offset_of_buffer_end, file_segment);
if (success)
{
chassert(file_segment->getCurrentWriteOffset() <= file_segment->range().right + 1);
chassert(file_segment.getCurrentWriteOffset(false) <= file_segment.range().right + 1);
chassert(
std::next(current_file_segment_it) == file_segments_holder->file_segments.end()
|| file_segment->getCurrentWriteOffset() == implementation_buffer->getFileOffsetOfBufferEnd());
/* last_file_segment */file_segments->size() == 1
|| file_segment.getCurrentWriteOffset(false) == implementation_buffer->getFileOffsetOfBufferEnd());
LOG_TEST(log, "Successfully written {} bytes", size);
@ -980,20 +969,13 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
}
else
{
chassert(file_segment->state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
chassert(file_segment.state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
LOG_TRACE(log, "Bypassing cache because writeCache method failed");
}
}
else
{
LOG_TRACE(log, "No space left in cache to reserve {} bytes, will continue without cache download", size);
file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
}
if (!success)
{
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
download_current_segment = false;
}
}
@ -1003,7 +985,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
/// Therefore need to resize to a smaller size. And resize must be done after write into cache.
/// - If last file segment was read from local fs, then we could read more than
/// file_segemnt->range().right, so resize is also needed.
if (std::next(current_file_segment_it) == file_segments_holder->file_segments.end())
if (file_segments->size() == 1)
{
size_t remaining_size_to_read
= std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1;
@ -1023,17 +1005,17 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
// No necessary because of the SCOPE_EXIT above, but useful for logging below.
if (download_current_segment)
file_segment->completePartAndResetDownloader();
file_segment.completePartAndResetDownloader();
chassert(!file_segment->isDownloader());
chassert(!file_segment.isDownloader());
LOG_TEST(
log,
"Key: {}. Returning with {} bytes, buffer position: {} (offset: {}, predownloaded: {}), "
"buffer available: {}, current range: {}, current offset: {}, file segment state: {}, "
"buffer available: {}, current range: {}, file offset of buffer end: {}, impl offset: {}, file segment state: {}, "
"current write offset: {}, read_type: {}, reading until position: {}, started with offset: {}, "
"remaining ranges: {}",
getHexUIntLowercase(cache_key),
cache_key.toString(),
working_buffer.size(),
getPosition(),
offset(),
@ -1041,12 +1023,13 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
available(),
current_read_range.toString(),
file_offset_of_buffer_end,
FileSegment::stateToString(file_segment->state()),
file_segment->getCurrentWriteOffset(),
implementation_buffer->getFileOffsetOfBufferEnd(),
FileSegment::stateToString(file_segment.state()),
file_segment.getCurrentWriteOffset(false),
toString(read_type),
read_until_position,
first_offset,
file_segments_holder->toString());
file_segments->toString());
if (size == 0 && file_offset_of_buffer_end < read_until_position)
{
@ -1065,7 +1048,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
cache_file_size ? std::to_string(cache_file_size) : "None",
cache_file_path,
implementation_buffer->getFileOffsetOfBufferEnd(),
file_segment->getInfoForLog());
file_segment.getInfoForLog());
}
return result;
@ -1113,13 +1096,13 @@ off_t CachedOnDiskReadBufferFromFile::seek(off_t offset, int whence)
first_offset = file_offset_of_buffer_end = new_pos;
resetWorkingBuffer();
// if (file_segments_holder && current_file_segment_it != file_segments_holder->file_segments.end())
// if (file_segments && current_file_segment_it != file_segments->file_segments.end())
// {
// auto & file_segments = file_segments_holder->file_segments;
// auto & file_segments = file_segments->file_segments;
// LOG_TRACE(
// log,
// "Having {} file segments to read: {}, current offset: {}",
// file_segments_holder->file_segments.size(), file_segments_holder->toString(), file_offset_of_buffer_end);
// file_segments->file_segments.size(), file_segments->toString(), file_offset_of_buffer_end);
// auto it = std::upper_bound(
// file_segments.begin(),
@ -1150,7 +1133,7 @@ off_t CachedOnDiskReadBufferFromFile::seek(off_t offset, int whence)
// }
// }
file_segments_holder.reset();
file_segments.reset();
implementation_buffer.reset();
initialized = false;
@ -1185,7 +1168,7 @@ void CachedOnDiskReadBufferFromFile::setReadUntilPosition(size_t position)
file_offset_of_buffer_end = getPosition();
resetWorkingBuffer();
file_segments_holder.reset();
file_segments.reset();
implementation_buffer.reset();
initialized = false;
@ -1204,25 +1187,9 @@ off_t CachedOnDiskReadBufferFromFile::getPosition()
return file_offset_of_buffer_end - available();
}
std::optional<size_t> CachedOnDiskReadBufferFromFile::getLastNonDownloadedOffset() const
{
if (!file_segments_holder)
throw Exception(ErrorCodes::LOGICAL_ERROR, "File segments holder not initialized");
const auto & file_segments = file_segments_holder->file_segments;
for (auto it = file_segments.rbegin(); it != file_segments.rend(); ++it)
{
const auto & file_segment = *it;
if (file_segment->state() != FileSegment::State::DOWNLOADED)
return file_segment->range().right;
}
return std::nullopt;
}
void CachedOnDiskReadBufferFromFile::assertCorrectness() const
{
if (FileCache::isReadOnly()
if (!CachedObjectStorage::canUseReadThroughCache()
&& !settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache usage is not allowed (query_id: {})", query_id);
}
@ -1230,16 +1197,16 @@ void CachedOnDiskReadBufferFromFile::assertCorrectness() const
String CachedOnDiskReadBufferFromFile::getInfoForLog()
{
String current_file_segment_info;
if (current_file_segment_it != file_segments_holder->file_segments.end())
current_file_segment_info = (*current_file_segment_it)->getInfoForLog();
else
if (file_segments->empty())
current_file_segment_info = "None";
else
current_file_segment_info = file_segments->front().getInfoForLog();
return fmt::format(
"Buffer path: {}, hash key: {}, file_offset_of_buffer_end: {}, read_until_position: {}, "
"internal buffer end: {}, read_type: {}, last caller: {}, file segment info: {}",
source_file_path,
getHexUIntLowercase(cache_key),
cache_key.toString(),
file_offset_of_buffer_end,
read_until_position,
implementation_buffer ? std::to_string(implementation_buffer->getFileOffsetOfBufferEnd()) : "None",

View File

@ -62,25 +62,28 @@ public:
private:
void initialize(size_t offset, size_t size);
void assertCorrectness() const;
ImplementationBufferPtr getImplementationBuffer(FileSegmentPtr & file_segment);
/**
* Return a list of file segments ordered in ascending order. This list represents
* a full contiguous interval (without holes).
*/
FileSegmentsHolderPtr getFileSegments(size_t offset, size_t size) const;
ImplementationBufferPtr getReadBufferForFileSegment(FileSegmentPtr & file_segment);
ImplementationBufferPtr getImplementationBuffer(FileSegment & file_segment);
ImplementationBufferPtr getReadBufferForFileSegment(FileSegment & file_segment);
ImplementationBufferPtr getCacheReadBuffer(const FileSegment & file_segment) const;
std::optional<size_t> getLastNonDownloadedOffset() const;
ImplementationBufferPtr getRemoteReadBuffer(FileSegment & file_segment, ReadType read_type_);
bool updateImplementationBufferIfNeeded();
void predownload(FileSegmentPtr & file_segment);
void predownload(FileSegment & file_segment);
bool nextImplStep();
void assertCorrectness() const;
std::shared_ptr<ReadBufferFromFileBase> getRemoteFSReadBuffer(FileSegment & file_segment, ReadType read_type_);
size_t getTotalSizeToRead();
bool completeFileSegmentAndGetNext();
@ -107,8 +110,7 @@ private:
/// Remote read buffer, which can only be owned by current buffer.
FileSegment::RemoteFileReaderPtr remote_file_reader;
std::optional<FileSegmentsHolder> file_segments_holder;
FileSegments::iterator current_file_segment_it;
FileSegmentsHolderPtr file_segments;
ImplementationBufferPtr implementation_buffer;
bool initialized = false;
@ -142,7 +144,7 @@ private:
CurrentMetrics::Increment metric_increment{CurrentMetrics::FilesystemCacheReadBuffers};
ProfileEvents::Counters current_file_segment_counters;
FileCache::QueryContextHolder query_context_holder;
FileCache::QueryContextHolderPtr query_context_holder;
bool is_persistent;
};

View File

@ -50,27 +50,29 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset
offset, expected_write_offset);
}
auto & file_segments = file_segments_holder.file_segments;
FileSegment * file_segment;
if (file_segments.empty() || file_segments.back()->isDownloaded())
if (file_segments.empty() || file_segments.back().isDownloaded())
{
allocateFileSegment(expected_write_offset, segment_kind);
file_segment = &allocateFileSegment(expected_write_offset, segment_kind);
}
else
{
file_segment = &file_segments.back();
}
auto & file_segment = file_segments.back();
SCOPE_EXIT({
if (file_segments.back()->isDownloader())
file_segments.back()->completePartAndResetDownloader();
if (file_segments.back().isDownloader())
file_segments.back().completePartAndResetDownloader();
});
while (size > 0)
{
size_t available_size = file_segment->range().size() - file_segment->getDownloadedSize();
size_t available_size = file_segment->range().size() - file_segment->getDownloadedSize(false);
if (available_size == 0)
{
completeFileSegment(*file_segment);
file_segment = allocateFileSegment(expected_write_offset, segment_kind);
file_segment = &allocateFileSegment(expected_write_offset, segment_kind);
continue;
}
@ -86,7 +88,6 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset
bool reserved = file_segment->reserve(size_to_write);
if (!reserved)
{
file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
appendFilesystemCacheLog(*file_segment);
LOG_DEBUG(
@ -113,11 +114,10 @@ void FileSegmentRangeWriter::finalize()
if (finalized)
return;
auto & file_segments = file_segments_holder.file_segments;
if (file_segments.empty())
return;
completeFileSegment(*file_segments.back());
completeFileSegment(file_segments.back());
finalized = true;
}
@ -134,24 +134,21 @@ FileSegmentRangeWriter::~FileSegmentRangeWriter()
}
}
FileSegmentPtr & FileSegmentRangeWriter::allocateFileSegment(size_t offset, FileSegmentKind segment_kind)
FileSegment & FileSegmentRangeWriter::allocateFileSegment(size_t offset, FileSegmentKind segment_kind)
{
/**
* Allocate a new file segment starting `offset`.
* File segment capacity will equal `max_file_segment_size`, but actual size is 0.
*/
std::lock_guard cache_lock(cache->mutex);
CreateFileSegmentSettings create_settings(segment_kind);
CreateFileSegmentSettings create_settings(segment_kind, false);
/// We set max_file_segment_size to be downloaded,
/// if we have less size to write, file segment will be resized in complete() method.
auto file_segment = cache->createFileSegmentForDownload(
key, offset, cache->max_file_segment_size, create_settings, cache_lock);
auto & file_segments = file_segments_holder.file_segments;
return *file_segments.insert(file_segments.end(), file_segment);
auto holder = cache->set(key, offset, cache->getMaxFileSegmentSize(), create_settings);
chassert(holder->size() == 1);
holder->moveTo(file_segments);
return file_segments.back();
}
void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_segment)
@ -159,7 +156,7 @@ void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_s
if (cache_log)
{
auto file_segment_range = file_segment.range();
size_t file_segment_right_bound = file_segment_range.left + file_segment.getDownloadedSize() - 1;
size_t file_segment_right_bound = file_segment_range.left + file_segment.getDownloadedSize(false) - 1;
FilesystemCacheLogElement elem
{
@ -185,7 +182,7 @@ void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment)
if (file_segment.isDetached() || file_segment.isCompleted())
return;
file_segment.completeWithoutState();
file_segment.complete();
appendFilesystemCacheLog(file_segment);
}
@ -224,7 +221,7 @@ void CachedOnDiskWriteBufferFromFile::nextImpl()
{
/// If something was already written to cache, remove it.
cache_writer.reset();
cache->removeIfExists(key);
cache->removeKeyIfExists(key);
throw;
}

View File

@ -39,7 +39,7 @@ public:
~FileSegmentRangeWriter();
private:
FileSegmentPtr & allocateFileSegment(size_t offset, FileSegmentKind segment_kind);
FileSegment & allocateFileSegment(size_t offset, FileSegmentKind segment_kind);
void appendFilesystemCacheLog(const FileSegment & file_segment);
@ -53,7 +53,7 @@ private:
String query_id;
String source_path;
FileSegmentsHolder file_segments_holder{};
FileSegmentsHolder file_segments{};
size_t expected_write_offset = 0;

View File

@ -3,6 +3,7 @@
#include <IO/SeekableReadBuffer.h>
#include <Disks/IO/CachedOnDiskReadBufferFromFile.h>
#include <Disks/ObjectStorages/Cached/CachedObjectStorage.h>
#include <Common/logger_useful.h>
#include <iostream>
#include <base/hex.h>
@ -56,7 +57,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
if (with_cache)
{
auto cache_key = settings.remote_fs_cache->hash(object_path);
auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path);
return std::make_shared<CachedOnDiskReadBufferFromFile>(
object_path,
cache_key,

View File

@ -43,13 +43,7 @@ DataSourceDescription CachedObjectStorage::getDataSourceDescription() const
FileCache::Key CachedObjectStorage::getCacheKey(const std::string & path) const
{
return cache->hash(path);
}
String CachedObjectStorage::getCachePath(const std::string & path) const
{
FileCache::Key cache_key = getCacheKey(path);
return cache->getPathInLocalCache(cache_key);
return cache->createKeyForPath(path);
}
std::string CachedObjectStorage::generateBlobNameForPath(const std::string & path)
@ -62,7 +56,7 @@ ReadSettings CachedObjectStorage::patchSettings(const ReadSettings & read_settin
ReadSettings modified_settings{read_settings};
modified_settings.remote_fs_cache = cache;
if (FileCache::isReadOnly())
if (!canUseReadThroughCache())
modified_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true;
return IObjectStorage::patchSettings(modified_settings);
@ -192,7 +186,6 @@ std::unique_ptr<WriteBufferFromFileBase> CachedObjectStorage::writeObject( /// N
if (cache_on_write)
{
auto key = getCacheKey(path_key_for_cache);
LOG_TEST(log, "Caching file `{}` to `{}` with key {}", object.absolute_path, getCachePath(path_key_for_cache), key.toString());
return std::make_unique<CachedOnDiskWriteBufferFromFile>(
std::move(implementation_buffer),
@ -213,7 +206,7 @@ void CachedObjectStorage::removeCacheIfExists(const std::string & path_key_for_c
return;
/// Add try catch?
cache->removeIfExists(getCacheKey(path_key_for_cache));
cache->removeKeyIfExists(getCacheKey(path_key_for_cache));
}
void CachedObjectStorage::removeObject(const StoredObject & object)
@ -308,4 +301,11 @@ String CachedObjectStorage::getObjectsNamespace() const
return object_storage->getObjectsNamespace();
}
bool CachedObjectStorage::canUseReadThroughCache()
{
return CurrentThread::isInitialized()
&& CurrentThread::get().getQueryContext()
&& !CurrentThread::getQueryId().empty();
}
}

View File

@ -113,11 +113,11 @@ public:
WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const override;
static bool canUseReadThroughCache();
private:
FileCache::Key getCacheKey(const std::string & path) const;
String getCachePath(const std::string & path) const;
ReadSettings patchSettings(const ReadSettings & read_settings) const override;
ObjectStoragePtr object_storage;

View File

@ -15,6 +15,7 @@
#include <Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h>
#include <Disks/ObjectStorages/DiskObjectStorageTransaction.h>
#include <Disks/FakeDiskTransaction.h>
#include <Common/ThreadPool.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Interpreters/Context.h>

View File

@ -128,7 +128,7 @@ std::unique_ptr<S3::Client> getClient(
if (uri.key.back() != '/')
throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 path must ends with '/', but '{}' doesn't.", uri.key);
client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", 10000);
client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", 1000);
client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 30000);
client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", 100);
client_configuration.endpointOverride = uri.endpoint;

View File

@ -47,7 +47,9 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
{
const ColumnPtr column_haystack = arguments[0].column;
ColumnPtr column_haystack = arguments[0].column;
column_haystack = column_haystack->convertToFullColumnIfConst();
const ColumnPtr column_needle = arguments[1].column;
const ColumnPtr column_replacement = arguments[2].column;

View File

@ -3301,7 +3301,7 @@ private:
};
}
WrapperType createMapToMapWrrapper(const DataTypes & from_kv_types, const DataTypes & to_kv_types) const
WrapperType createMapToMapWrapper(const DataTypes & from_kv_types, const DataTypes & to_kv_types) const
{
return [element_wrappers = getElementWrappers(from_kv_types, to_kv_types), from_kv_types, to_kv_types]
(ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t /*input_rows_count*/) -> ColumnPtr
@ -3322,7 +3322,7 @@ private:
}
/// The case of: [(key1, value1), (key2, value2), ...]
WrapperType createArrayToMapWrrapper(const DataTypes & from_kv_types, const DataTypes & to_kv_types) const
WrapperType createArrayToMapWrapper(const DataTypes & from_kv_types, const DataTypes & to_kv_types) const
{
return [element_wrappers = getElementWrappers(from_kv_types, to_kv_types), from_kv_types, to_kv_types]
(ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t /*input_rows_count*/) -> ColumnPtr
@ -3348,8 +3348,12 @@ private:
if (const auto * from_tuple = checkAndGetDataType<DataTypeTuple>(from_type_untyped.get()))
{
if (from_tuple->getElements().size() != 2)
throw Exception(ErrorCodes::TYPE_MISMATCH, "CAST AS Map from tuple requeires 2 elements. "
"Left type: {}, right type: {}", from_tuple->getName(), to_type->getName());
throw Exception(
ErrorCodes::TYPE_MISMATCH,
"CAST AS Map from tuple requires 2 elements. "
"Left type: {}, right type: {}",
from_tuple->getName(),
to_type->getName());
DataTypes from_kv_types;
const auto & to_kv_types = to_type->getKeyValueTypes();
@ -3370,14 +3374,18 @@ private:
{
const auto * nested_tuple = typeid_cast<const DataTypeTuple *>(from_array->getNestedType().get());
if (!nested_tuple || nested_tuple->getElements().size() != 2)
throw Exception(ErrorCodes::TYPE_MISMATCH, "CAST AS Map from array requeires nested tuple of 2 elements. "
"Left type: {}, right type: {}", from_array->getName(), to_type->getName());
throw Exception(
ErrorCodes::TYPE_MISMATCH,
"CAST AS Map from array requires nested tuple of 2 elements. "
"Left type: {}, right type: {}",
from_array->getName(),
to_type->getName());
return createArrayToMapWrrapper(nested_tuple->getElements(), to_type->getKeyValueTypes());
return createArrayToMapWrapper(nested_tuple->getElements(), to_type->getKeyValueTypes());
}
else if (const auto * from_type = checkAndGetDataType<DataTypeMap>(from_type_untyped.get()))
{
return createMapToMapWrrapper(from_type->getKeyValueTypes(), to_type->getKeyValueTypes());
return createMapToMapWrapper(from_type->getKeyValueTypes(), to_type->getKeyValueTypes());
}
else
{

View File

@ -1,9 +1,11 @@
#include <Functions/JSONPath/ASTs/ASTJSONPathMemberAccess.h>
#include <Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTIdentifier_fwd.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/Lexer.h>
#include <Common/StringUtils/StringUtils.h>
namespace DB
{
@ -16,18 +18,60 @@ namespace DB
*/
bool ParserJSONPathMemberAccess::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (pos->type != TokenType::Dot)
// There's a special case, that a path member can begin with number
// some invalid cases as following
// - ".123" is parsed as a number, not a dot and a number
// - ".123abc" is parsed as two parts, a number ".123" and a token "abc"
// - ".abc" is parsed as two parts. a dot and a token "abc"
// "$..123abc" is parsed into three parts, ".", ".123" and "abc"
if (pos->type != TokenType::Dot && pos->type != TokenType::Number)
return false;
if (pos->type != TokenType::Number)
{
++pos;
// Check the case "$..123abc"
if (pos->type == TokenType::Number)
{
return false;
}
}
++pos;
if (pos->type != TokenType::BareWord && pos->type !=TokenType::QuotedIdentifier)
return false;
ParserIdentifier name_p;
ASTPtr member_name;
if (!name_p.parse(pos, member_name, expected))
return false;
if (pos->type == TokenType::Number)[[unlikely]]
{
for (const auto * c = pos->begin; c != pos->end; ++c)
{
if (*c == '.' && c == pos->begin)
continue;
if (!isNumericASCII(*c))
{
return false;
}
}
const auto * last_begin = *pos->begin == '.' ? pos->begin + 1 : pos->begin;
const auto * last_end = pos->end;
++pos;
if (pos.isValid() && pos->type == TokenType::BareWord && pos->begin == last_end)
{
member_name = std::make_shared<ASTIdentifier>(String(last_begin, pos->end));
++pos;
}
else
{
return false;
}
}
else
{
if (pos->type != TokenType::BareWord && pos->type != TokenType::QuotedIdentifier)
return false;
ParserIdentifier name_p;
if (!name_p.parse(pos, member_name, expected))
return false;
}
auto member_access = std::make_shared<ASTJSONPathMemberAccess>();
node = member_access;

View File

@ -0,0 +1,44 @@
#include "ParserJSONPathMemberSquareBracketAccess.h"
#include <memory>
#include <Functions/JSONPath/ASTs/ASTJSONPathMemberAccess.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/ReadHelpers.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ExpressionElementParsers.h>
namespace DB
{
bool ParserJSONPathMemberSquareBracketAccess::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (pos->type != TokenType::OpeningSquareBracket)
return false;
++pos;
ASTPtr member_name;
if (pos->type == TokenType::BareWord || pos->type == TokenType::QuotedIdentifier)
{
ParserIdentifier name_p;
if (!name_p.parse(pos, member_name, expected))
return false;
}
else if (pos->type == TokenType::StringLiteral)
{
ReadBufferFromMemory in(pos->begin, pos->size());
String name;
readQuotedStringWithSQLStyle(name, in);
member_name = std::make_shared<ASTIdentifier>(name);
++pos;
}
else
{
return false;
}
if (pos->type != TokenType::ClosingSquareBracket)
{
return false;
}
++pos;
auto member_access = std::make_shared<ASTJSONPathMemberAccess>();
node = member_access;
return tryGetIdentifierNameInto(member_name, member_access->member_name);
}
}

View File

@ -0,0 +1,17 @@
#pragma once
#include <Parsers/IParserBase.h>
// cases
// - [ident]
// - ['ident']
// - ["ident"]
namespace DB
{
class ParserJSONPathMemberSquareBracketAccess : public IParserBase
{
private:
const char * getName() const override { return "ParserJSONPathMemberSquareBracketAccess"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
public:
explicit ParserJSONPathMemberSquareBracketAccess() = default;
};
}

View File

@ -2,6 +2,7 @@
#include <Functions/JSONPath/Parsers/ParserJSONPathQuery.h>
#include <Functions/JSONPath/Parsers/ParserJSONPathRoot.h>
#include <Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.h>
#include <Functions/JSONPath/Parsers/ParserJSONPathMemberSquareBracketAccess.h>
#include <Functions/JSONPath/Parsers/ParserJSONPathRange.h>
#include <Functions/JSONPath/Parsers/ParserJSONPathStar.h>
@ -19,6 +20,7 @@ bool ParserJSONPathQuery::parseImpl(Pos & pos, ASTPtr & query, Expected & expect
{
query = std::make_shared<ASTJSONPathQuery>();
ParserJSONPathMemberAccess parser_jsonpath_member_access;
ParserJSONPathMemberSquareBracketAccess parser_jsonpath_member_square_bracket_access;
ParserJSONPathRange parser_jsonpath_range;
ParserJSONPathStar parser_jsonpath_star;
ParserJSONPathRoot parser_jsonpath_root;
@ -32,6 +34,7 @@ bool ParserJSONPathQuery::parseImpl(Pos & pos, ASTPtr & query, Expected & expect
ASTPtr accessor;
while (parser_jsonpath_member_access.parse(pos, accessor, expected)
|| parser_jsonpath_member_square_bracket_access.parse(pos, accessor, expected)
|| parser_jsonpath_range.parse(pos, accessor, expected)
|| parser_jsonpath_star.parse(pos, accessor, expected))
{

View File

@ -24,6 +24,9 @@ public:
explicit FunctionCaseWithExpression(ContextPtr context_) : context(context_) {}
bool isVariadic() const override { return true; }
bool useDefaultImplementationForConstants() const override { return false; }
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForNothing() const override { return false; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
size_t getNumberOfArguments() const override { return 0; }
String getName() const override { return name; }

View File

@ -205,6 +205,10 @@ public:
{
return FunctionFactory::instance().getImpl("arrayConcat", context)->build(arguments);
}
else if (isMap(arguments.at(0).type))
{
return FunctionFactory::instance().getImpl("mapConcat", context)->build(arguments);
}
else
return std::make_unique<FunctionToFunctionBaseAdaptor>(
FunctionConcat::create(context), collections::map<DataTypes>(arguments, [](const auto & elem) { return elem.type; }), return_type);

View File

@ -50,8 +50,8 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
protected:
template <class AgrumentNames>
void checkRequiredArguments(const ColumnsWithTypeAndName & arguments, const AgrumentNames & argument_names, const size_t optional_argument_count) const
template <class ArgumentNames>
void checkRequiredArguments(const ColumnsWithTypeAndName & arguments, const ArgumentNames & argument_names, const size_t optional_argument_count) const
{
if (arguments.size() < argument_names.size() || arguments.size() > argument_names.size() + optional_argument_count)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
@ -67,8 +67,8 @@ protected:
}
}
template <class AgrumentNames>
void convertRequiredArguments(const ColumnsWithTypeAndName & arguments, const AgrumentNames & argument_names, Columns & converted_arguments) const
template <class ArgumentNames>
void convertRequiredArguments(const ColumnsWithTypeAndName & arguments, const ArgumentNames & argument_names, Columns & converted_arguments) const
{
const DataTypePtr converted_argument_type = std::make_shared<DataTypeFloat32>();
converted_arguments.clear();
@ -87,7 +87,7 @@ template <typename Traits>
class FunctionMakeDate : public FunctionWithNumericParamsBase
{
private:
static constexpr std::array<const char*, 3> argument_names = {"year", "month", "day"};
static constexpr std::array argument_names = {"year", "month", "day"};
public:
static constexpr auto name = Traits::name;
@ -112,7 +112,7 @@ public:
Columns converted_arguments;
convertRequiredArguments(arguments, argument_names, converted_arguments);
auto res_column = Traits::ReturnColumnType::create(input_rows_count);
auto res_column = Traits::ReturnDataType::ColumnType::create(input_rows_count);
auto & result_data = res_column->getData();
const auto & year_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[0]).getData();
@ -150,7 +150,6 @@ struct MakeDateTraits
{
static constexpr auto name = "makeDate";
using ReturnDataType = DataTypeDate;
using ReturnColumnType = ColumnDate;
static constexpr auto MIN_YEAR = 1970;
static constexpr auto MAX_YEAR = 2149;
@ -163,7 +162,6 @@ struct MakeDate32Traits
{
static constexpr auto name = "makeDate32";
using ReturnDataType = DataTypeDate32;
using ReturnColumnType = ColumnDate32;
static constexpr auto MIN_YEAR = 1900;
static constexpr auto MAX_YEAR = 2299;
@ -174,7 +172,7 @@ struct MakeDate32Traits
class FunctionMakeDateTimeBase : public FunctionWithNumericParamsBase
{
protected:
static constexpr std::array<const char*, 6> argument_names = {"year", "month", "day", "hour", "minute", "second"};
static constexpr std::array argument_names = {"year", "month", "day", "hour", "minute", "second"};
public:
bool isVariadic() const override { return true; }
@ -197,13 +195,13 @@ protected:
{
/// Note that hour, minute and second are checked against 99 to behave consistently with parsing DateTime from String
/// E.g. "select cast('1984-01-01 99:99:99' as DateTime);" returns "1984-01-05 04:40:39"
if (unlikely(std::isnan(year) || std::isnan(month) || std::isnan(day_of_month) ||
if (std::isnan(year) || std::isnan(month) || std::isnan(day_of_month) ||
std::isnan(hour) || std::isnan(minute) || std::isnan(second) ||
year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31 ||
hour < 0 || hour > 99 || minute < 0 || minute > 99 || second < 0 || second > 99))
hour < 0 || hour > 99 || minute < 0 || minute > 99 || second < 0 || second > 99) [[unlikely]]
return minDateTime(lut);
if (unlikely(year > DATE_LUT_MAX_YEAR))
if (year > DATE_LUT_MAX_YEAR) [[unlikely]]
return maxDateTime(lut);
return lut.makeDateTime(
@ -290,9 +288,9 @@ public:
const auto second = second_data[i];
auto date_time = dateTime(year, month, day, hour, minute, second, date_lut);
if (unlikely(date_time < 0))
if (date_time < 0) [[unlikely]]
date_time = 0;
else if (unlikely(date_time > 0x0ffffffffll))
else if (date_time > 0x0ffffffffll) [[unlikely]]
date_time = 0x0ffffffffll;
result_data[i] = static_cast<UInt32>(date_time);
@ -394,21 +392,21 @@ public:
auto date_time = dateTime(year, month, day, hour, minute, second, date_lut);
double fraction = 0;
if (unlikely(date_time == min_date_time))
if (date_time == min_date_time) [[unlikely]]
fraction = 0;
else if (unlikely(date_time == max_date_time))
else if (date_time == max_date_time) [[unlikely]]
fraction = 999999999;
else
{
fraction = fraction_data ? (*fraction_data)[i] : 0;
if (unlikely(std::isnan(fraction)))
if (std::isnan(fraction)) [[unlikely]]
{
date_time = min_date_time;
fraction = 0;
}
else if (unlikely(fraction < 0))
else if (fraction < 0) [[unlikely]]
fraction = 0;
else if (unlikely(fraction > max_fraction))
else if (fraction > max_fraction) [[unlikely]]
fraction = max_fraction;
}

File diff suppressed because it is too large Load Diff

View File

@ -203,11 +203,13 @@ off_t ReadBufferFromS3::seek(off_t offset_, int whence)
return offset_;
if (impl && restricted_seek)
{
throw Exception(
ErrorCodes::CANNOT_SEEK_THROUGH_FILE,
"Seek is allowed only before first read attempt from the buffer (current offset: "
"{}, new offset: {}, reading until position: {}, available: {})",
getPosition(), offset_, read_until_position, available());
ErrorCodes::CANNOT_SEEK_THROUGH_FILE,
"Seek is allowed only before first read attempt from the buffer (current offset: "
"{}, new offset: {}, reading until position: {}, available: {})",
getPosition(), offset_, read_until_position, available());
}
if (whence != SEEK_SET)
throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET mode is allowed.");

View File

@ -260,6 +260,7 @@ void PocoHTTPClient::makeRequestInternal(
Poco::Logger * log = &Poco::Logger::get("AWSClient");
auto uri = request.GetUri().GetURIString();
#if 0
auto provider_type = getProviderTypeFromURL(uri);
if (provider_type == ProviderType::GCS)
@ -269,6 +270,7 @@ void PocoHTTPClient::makeRequestInternal(
request.DeleteHeader("amz-sdk-invocation-id");
request.DeleteHeader("amz-sdk-request");
}
#endif
if (enable_s3_requests_logging)
LOG_TEST(log, "Make request to: {}", uri);

View File

@ -85,7 +85,8 @@ WriteBufferFromS3::WriteBufferFromS3(
, upload_settings(request_settings.getUploadSettings())
, client_ptr(std::move(client_ptr_))
, object_metadata(std::move(object_metadata_))
, upload_part_size(upload_settings.min_upload_part_size)
, strict_upload_part_size(upload_settings.strict_upload_part_size)
, current_upload_part_size(upload_settings.min_upload_part_size)
, schedule(std::move(schedule_))
, write_settings(write_settings_)
{
@ -100,28 +101,79 @@ void WriteBufferFromS3::nextImpl()
/// Buffer in a bad state after exception
if (temporary_buffer->tellp() == -1)
allocateBuffer();
else
chassert(temporary_buffer->tellp() == static_cast<std::streamoff>(last_part_size));
if (strict_upload_part_size)
processWithStrictParts();
else
processWithDynamicParts();
waitForReadyBackGroundTasks();
}
void WriteBufferFromS3::processWithStrictParts()
{
chassert(strict_upload_part_size > 0);
size_t buffer_size = offset();
size_t left_in_buffer = buffer_size;
size_t new_size = last_part_size + buffer_size;
size_t buffer_offset = 0;
if (new_size > strict_upload_part_size)
{
/// Data size will exceed fixed part size threshold for multipart upload, need to use multipart upload.
if (multipart_upload_id.empty())
createMultipartUpload();
while (new_size > strict_upload_part_size)
{
size_t to_write = strict_upload_part_size - last_part_size;
temporary_buffer->write(working_buffer.begin() + buffer_offset, to_write);
buffer_offset += to_write;
writePart();
allocateBuffer();
new_size -= strict_upload_part_size;
left_in_buffer -= to_write;
}
}
if (left_in_buffer)
{
temporary_buffer->write(working_buffer.begin() + buffer_offset, left_in_buffer);
last_part_size += left_in_buffer;
}
ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Bytes, buffer_size);
if (write_settings.remote_throttler)
write_settings.remote_throttler->add(buffer_size, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds);
}
void WriteBufferFromS3::processWithDynamicParts()
{
chassert(current_upload_part_size > 0);
size_t size = offset();
temporary_buffer->write(working_buffer.begin(), size);
ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Bytes, size);
last_part_size += size;
ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Bytes, offset());
last_part_size += offset();
if (write_settings.remote_throttler)
write_settings.remote_throttler->add(offset(), ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds);
write_settings.remote_throttler->add(size, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds);
/// Data size exceeds singlepart upload threshold, need to use multipart upload.
if (multipart_upload_id.empty() && last_part_size > upload_settings.max_single_part_upload_size)
createMultipartUpload();
chassert(upload_part_size > 0);
if (!multipart_upload_id.empty() && last_part_size > upload_part_size)
if (!multipart_upload_id.empty() && last_part_size > current_upload_part_size)
{
writePart();
allocateBuffer();
}
waitForReadyBackGroundTasks();
}
void WriteBufferFromS3::allocateBuffer()
@ -335,14 +387,17 @@ void WriteBufferFromS3::fillUploadRequest(S3::UploadPartRequest & req)
/// If we don't do it, AWS SDK can mistakenly set it to application/xml, see https://github.com/aws/aws-sdk-cpp/issues/1840
req.SetContentType("binary/octet-stream");
/// Maybe increase `upload_part_size` (we need to increase it sometimes to keep `part_number` less or equal than `max_part_number`).
auto threshold = upload_settings.upload_part_size_multiply_parts_count_threshold;
if (!multipart_upload_id.empty() && (part_number % threshold == 0))
if (!strict_upload_part_size)
{
auto max_upload_part_size = upload_settings.max_upload_part_size;
auto upload_part_size_multiply_factor = upload_settings.upload_part_size_multiply_factor;
upload_part_size *= upload_part_size_multiply_factor;
upload_part_size = std::min(upload_part_size, max_upload_part_size);
/// Maybe increase `current_upload_part_size` (we need to increase it sometimes to keep `part_number` less or equal than `max_part_number`).
auto threshold = upload_settings.upload_part_size_multiply_parts_count_threshold;
if (!multipart_upload_id.empty() && (part_number % threshold == 0))
{
auto max_upload_part_size = upload_settings.max_upload_part_size;
auto upload_part_size_multiply_factor = upload_settings.upload_part_size_multiply_factor;
current_upload_part_size *= upload_part_size_multiply_factor;
current_upload_part_size = std::min(current_upload_part_size, max_upload_part_size);
}
}
}

View File

@ -58,6 +58,9 @@ public:
private:
void allocateBuffer();
void processWithStrictParts();
void processWithDynamicParts();
void createMultipartUpload();
void writePart();
void completeMultipartUpload();
@ -86,7 +89,10 @@ private:
const std::shared_ptr<const S3::Client> client_ptr;
const std::optional<std::map<String, String>> object_metadata;
size_t upload_part_size = 0;
/// Strict/static Part size, no adjustments will be done on fly.
size_t strict_upload_part_size = 0;
/// Part size will be adjusted on fly (for bigger uploads)
size_t current_upload_part_size = 0;
std::shared_ptr<Aws::StringStream> temporary_buffer; /// Buffer to accumulate data.
size_t last_part_size = 0;
size_t part_number = 0;

View File

@ -23,10 +23,12 @@ namespace
void updateUserFromQueryImpl(
User & user,
const ASTCreateUserQuery & query,
const std::optional<AuthenticationData> auth_data,
const std::shared_ptr<ASTUserNameWithHost> & override_name,
const std::optional<RolesOrUsersSet> & override_default_roles,
const std::optional<SettingsProfileElements> & override_settings,
const std::optional<RolesOrUsersSet> & override_grantees,
bool allow_implicit_no_password,
bool allow_no_password,
bool allow_plaintext_password)
{
@ -37,10 +39,16 @@ namespace
else if (query.names->size() == 1)
user.setName(query.names->front()->toString());
if (query.auth_data)
user.auth_data = *query.auth_data;
if (!query.attach && !query.alter && !auth_data && !allow_implicit_no_password)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Authentication type NO_PASSWORD must "
"be explicitly specified, check the setting allow_implicit_no_password "
"in the server configuration");
if (query.auth_data || !query.alter)
if (auth_data)
user.auth_data = *auth_data;
if (auth_data || !query.alter)
{
auto auth_type = user.auth_data.getType();
if (((auth_type == AuthenticationType::NO_PASSWORD) && !allow_no_password) ||
@ -104,17 +112,9 @@ BlockIO InterpreterCreateUserQuery::execute()
bool no_password_allowed = access_control.isNoPasswordAllowed();
bool plaintext_password_allowed = access_control.isPlaintextPasswordAllowed();
if (!query.attach && !query.alter && !query.auth_data && !implicit_no_password_allowed)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Authentication type NO_PASSWORD must "
"be explicitly specified, check the setting allow_implicit_no_password "
"in the server configuration");
if (!query.attach && query.temporary_password_for_checks)
{
access_control.checkPasswordComplexityRules(query.temporary_password_for_checks.value());
query.temporary_password_for_checks.reset();
}
std::optional<AuthenticationData> auth_data;
if (query.auth_data)
auth_data = AuthenticationData::fromAST(*query.auth_data, getContext(), !query.attach);
std::optional<RolesOrUsersSet> default_roles_from_query;
if (query.default_roles)
@ -148,7 +148,7 @@ BlockIO InterpreterCreateUserQuery::execute()
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
{
auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
updateUserFromQueryImpl(*updated_user, query, {}, default_roles_from_query, settings_from_query, grantees_from_query, no_password_allowed, plaintext_password_allowed);
updateUserFromQueryImpl(*updated_user, query, auth_data, {}, default_roles_from_query, settings_from_query, grantees_from_query, implicit_no_password_allowed, no_password_allowed, plaintext_password_allowed);
return updated_user;
};
@ -167,7 +167,7 @@ BlockIO InterpreterCreateUserQuery::execute()
for (const auto & name : *query.names)
{
auto new_user = std::make_shared<User>();
updateUserFromQueryImpl(*new_user, query, name, default_roles_from_query, settings_from_query, RolesOrUsersSet::AllTag{}, no_password_allowed, plaintext_password_allowed);
updateUserFromQueryImpl(*new_user, query, auth_data, name, default_roles_from_query, settings_from_query, RolesOrUsersSet::AllTag{}, implicit_no_password_allowed, no_password_allowed, plaintext_password_allowed);
new_users.emplace_back(std::move(new_user));
}
@ -197,7 +197,11 @@ BlockIO InterpreterCreateUserQuery::execute()
void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreateUserQuery & query, bool allow_no_password, bool allow_plaintext_password)
{
updateUserFromQueryImpl(user, query, {}, {}, {}, {}, allow_no_password, allow_plaintext_password);
std::optional<AuthenticationData> auth_data;
if (query.auth_data)
auth_data = AuthenticationData::fromAST(*query.auth_data, {}, !query.attach);
updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, allow_no_password, allow_plaintext_password, true);
}
}

View File

@ -62,7 +62,7 @@ namespace
}
if (user.auth_data.getType() != AuthenticationType::NO_PASSWORD)
query->auth_data = user.auth_data;
query->auth_data = user.auth_data.toAST();
if (!user.settings.empty())
{

View File

@ -36,6 +36,7 @@ NamesAndTypesList AsynchronousInsertLogElement::getNamesAndTypes()
{"format", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
{"query_id", std::make_shared<DataTypeString>()},
{"bytes", std::make_shared<DataTypeUInt64>()},
{"rows", std::make_shared<DataTypeUInt64>()},
{"exception", std::make_shared<DataTypeString>()},
{"status", type_status},
@ -71,6 +72,7 @@ void AsynchronousInsertLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(insert_query.format);
columns[i++]->insert(query_id);
columns[i++]->insert(bytes);
columns[i++]->insert(rows);
columns[i++]->insert(exception);
columns[i++]->insert(status);

View File

@ -24,6 +24,7 @@ struct AsynchronousInsertLogElement
ASTPtr query;
String query_id;
UInt64 bytes{};
UInt64 rows{};
String exception;
Status status{};

View File

@ -40,6 +40,7 @@ namespace ProfileEvents
{
extern const Event AsyncInsertQuery;
extern const Event AsyncInsertBytes;
extern const Event AsyncInsertRows;
extern const Event FailedAsyncInsertQuery;
}
@ -444,7 +445,8 @@ try
{
auto buffer = std::make_unique<ReadBufferFromString>(entry->bytes);
current_entry = entry;
total_rows += executor.execute(*buffer);
size_t num_rows = executor.execute(*buffer);
total_rows += num_rows;
chunk_info->offsets.push_back(total_rows);
/// Keep buffer, because it still can be used
@ -459,6 +461,7 @@ try
elem.query = key.query;
elem.query_id = entry->query_id;
elem.bytes = entry->bytes.size();
elem.rows = num_rows;
elem.exception = current_exception;
current_exception.clear();
@ -479,6 +482,7 @@ try
format->addBuffer(std::move(last_buffer));
auto insert_query_id = insert_context->getCurrentQueryId();
ProfileEvents::increment(ProfileEvents::AsyncInsertRows, total_rows);
auto finish_entries = [&]
{

File diff suppressed because it is too large Load Diff

View File

@ -9,43 +9,52 @@
#include <unordered_map>
#include <unordered_set>
#include <boost/functional/hash.hpp>
#include <boost/noncopyable.hpp>
#include <Core/Types.h>
#include <Common/ThreadPool.h>
#include <IO/ReadSettings.h>
#include <Interpreters/Cache/IFileCachePriority.h>
#include <Interpreters/Cache/FileCacheKey.h>
#include <Core/BackgroundSchedulePool.h>
#include <Interpreters/Cache/LRUFileCachePriority.h>
#include <Interpreters/Cache/FileCache_fwd.h>
#include <Interpreters/Cache/FileSegment.h>
#include <Interpreters/Cache/Metadata.h>
#include <Interpreters/Cache/QueryLimit.h>
#include <Interpreters/Cache/FileCache_fwd_internal.h>
#include <filesystem>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
/// Local cache for remote filesystem files, represented as a set of non-overlapping non-empty file segments.
/// Different caching algorithms are implemented using IFileCachePriority.
class FileCache : private boost::noncopyable
{
friend class FileSegment;
friend class IFileCachePriority;
friend struct FileSegmentsHolder;
friend class FileSegmentRangeWriter;
struct QueryContext;
using QueryContextPtr = std::shared_ptr<QueryContext>;
public:
using Key = DB::FileCacheKey;
using QueryLimit = DB::FileCacheQueryLimit;
using Priority = IFileCachePriority;
using PriorityEntry = IFileCachePriority::Entry;
using PriorityIterator = IFileCachePriority::Iterator;
using PriorityIterationResult = IFileCachePriority::IterationResult;
explicit FileCache(const FileCacheSettings & settings);
~FileCache() = default;
~FileCache();
void initialize();
const String & getBasePath() const { return cache_base_path; }
const String & getBasePath() const;
static Key createKeyForPath(const String & path);
String getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const;
String getPathInLocalCache(const Key & key) const;
/**
* Given an `offset` and `size` representing [offset, offset + size) bytes interval,
@ -58,8 +67,7 @@ public:
* As long as pointers to returned file segments are held
* it is guaranteed that these file segments are not removed from cache.
*/
FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings);
FileSegmentsHolder set(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings);
FileSegmentsHolderPtr getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings);
/**
* Segments in returned list are ordered in ascending order and represent a full contiguous
@ -70,53 +78,40 @@ public:
* with the destruction of the holder, while in getOrSet() EMPTY file segments can eventually change
* it's state (and become DOWNLOADED).
*/
FileSegmentsHolder get(const Key & key, size_t offset, size_t size);
FileSegmentsHolderPtr get(const Key & key, size_t offset, size_t size);
FileSegmentsHolderPtr set(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings);
/// Remove files by `key`. Removes files which might be used at the moment.
void removeIfExists(const Key & key);
void removeKeyIfExists(const Key & key);
/// Remove files by `key`. Will not remove files which are used at the moment.
void removeIfReleasable();
static Key hash(const String & path);
String getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const;
String getPathInLocalCache(const Key & key) const;
void removeAllReleasable();
std::vector<String> tryGetCachePaths(const Key & key);
size_t capacity() const { return max_size; }
size_t getUsedCacheSize() const;
size_t getFileSegmentsNum() const;
static bool isReadOnly();
size_t getMaxFileSegmentSize() const { return max_file_segment_size; }
/**
* Create a file segment of exactly requested size with EMPTY state.
* Throw exception if requested size exceeds max allowed file segment size.
* This method is for protected usage: file segment range writer uses it
* to dynamically allocate file segments.
*/
FileSegmentPtr createFileSegmentForDownload(
const Key & key,
size_t offset,
size_t size,
const CreateFileSegmentSettings & create_settings,
std::lock_guard<std::mutex> & cache_lock);
bool tryReserve(FileSegment & file_segment, size_t size);
FileSegments getSnapshot() const;
FileSegmentsHolderPtr getSnapshot();
/// For debug.
String dumpStructure(const Key & key);
FileSegmentsHolderPtr getSnapshot(const Key & key);
/// Save a query context information, and adopt different cache policies
/// for different queries through the context cache layer.
FileSegmentsHolderPtr dumpQueue();
void cleanup();
void deactivateBackgroundOperations();
/// For per query cache limit.
struct QueryContextHolder : private boost::noncopyable
{
QueryContextHolder(const String & query_id_, FileCache * cache_, QueryContextPtr context_);
QueryContextHolder(const String & query_id_, FileCache * cache_, QueryLimit::QueryContextPtr context_);
QueryContextHolder() = default;
@ -124,198 +119,95 @@ public:
String query_id;
FileCache * cache = nullptr;
QueryContextPtr context;
QueryLimit::QueryContextPtr context;
};
using QueryContextHolderPtr = std::unique_ptr<QueryContextHolder>;
QueryContextHolderPtr getQueryContextHolder(const String & query_id, const ReadSettings & settings);
QueryContextHolder getQueryContextHolder(const String & query_id, const ReadSettings & settings);
CacheGuard::Lock lockCache() { return cache_guard.lock(); }
private:
String cache_base_path;
using KeyAndOffset = FileCacheKeyAndOffset;
const size_t max_size;
const size_t max_element_size;
const size_t max_file_segment_size;
const bool allow_persistent_files;
const size_t enable_cache_hits_threshold;
const bool enable_filesystem_query_cache_limit;
const size_t bypass_cache_threshold = 0;
const size_t delayed_cleanup_interval_ms;
const bool enable_bypass_cache_with_threashold;
const size_t bypass_cache_threashold;
mutable std::mutex mutex;
Poco::Logger * log;
bool is_initialized = false;
std::exception_ptr initialization_exception;
std::exception_ptr init_exception;
std::atomic<bool> is_initialized = false;
mutable std::mutex init_mutex;
void assertInitialized(std::lock_guard<std::mutex> & cache_lock) const;
CacheMetadata metadata;
bool tryReserve(const Key & key, size_t offset, size_t size, std::lock_guard<std::mutex> & cache_lock);
FileCachePriorityPtr main_priority;
mutable CacheGuard cache_guard;
void remove(
Key key,
size_t offset,
std::lock_guard<std::mutex> & cache_lock,
std::unique_lock<std::mutex> & segment_lock);
void remove(
FileSegmentPtr file_segment,
std::lock_guard<std::mutex> & cache_lock);
bool isLastFileSegmentHolder(
const Key & key,
size_t offset,
std::lock_guard<std::mutex> & cache_lock,
std::unique_lock<std::mutex> & segment_lock);
void reduceSizeToDownloaded(
const Key & key,
size_t offset,
std::lock_guard<std::mutex> & cache_lock,
std::unique_lock<std::mutex> & segment_lock);
struct FileSegmentCell : private boost::noncopyable
struct HitsCountStash
{
FileSegmentPtr file_segment;
/// Iterator is put here on first reservation attempt, if successful.
IFileCachePriority::WriteIterator queue_iterator;
/// Pointer to file segment is always hold by the cache itself.
/// Apart from pointer in cache, it can be hold by cache users, when they call
/// getorSet(), but cache users always hold it via FileSegmentsHolder.
bool releasable() const { return file_segment.unique(); }
size_t size() const { return file_segment->reserved_size; }
FileSegmentCell(FileSegmentPtr file_segment_, FileCache * cache, std::lock_guard<std::mutex> & cache_lock);
FileSegmentCell(FileSegmentCell && other) noexcept
: file_segment(std::move(other.file_segment)), queue_iterator(std::move(other.queue_iterator)) {}
};
using AccessKeyAndOffset = std::pair<Key, size_t>;
struct KeyAndOffsetHash
{
std::size_t operator()(const AccessKeyAndOffset & key) const
HitsCountStash(size_t hits_threashold_, size_t queue_size_)
: hits_threshold(hits_threashold_), queue(std::make_unique<LRUFileCachePriority>(0, queue_size_))
{
return std::hash<UInt128>()(key.first.key) ^ std::hash<UInt64>()(key.second);
if (!queue_size_)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Queue size for hits queue must be non-zero");
}
const size_t hits_threshold;
FileCachePriorityPtr queue;
using Records = std::unordered_map<KeyAndOffset, PriorityIterator, FileCacheKeyAndOffsetHash>;
Records records;
};
using FileSegmentsByOffset = std::map<size_t, FileSegmentCell>;
using CachedFiles = std::unordered_map<Key, FileSegmentsByOffset>;
using FileCacheRecords = std::unordered_map<AccessKeyAndOffset, IFileCachePriority::WriteIterator, KeyAndOffsetHash>;
/**
* A HitsCountStash allows to cache certain data only after it reached
* a certain hit rate, e.g. if hit rate it 5, then data is cached on 6th cache hit.
*/
mutable std::unique_ptr<HitsCountStash> stash;
/**
* A QueryLimit allows to control cache write limit per query.
* E.g. if a query needs n bytes from cache, but it has only k bytes, where 0 <= k <= n
* then allowed loaded cache size is std::min(n - k, max_query_cache_size).
*/
FileCacheQueryLimitPtr query_limit;
/**
* A background cleanup task.
* Clears removed cache entries from metadata.
*/
BackgroundSchedulePool::TaskHolder cleanup_task;
CachedFiles files;
std::unique_ptr<IFileCachePriority> main_priority;
void assertInitialized() const;
FileCacheRecords stash_records;
std::unique_ptr<IFileCachePriority> stash_priority;
size_t max_stash_element_size;
void assertCacheCorrectness();
void loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock);
void loadMetadata();
FileSegments getImpl(const Key & key, const FileSegment::Range & range, std::lock_guard<std::mutex> & cache_lock);
FileSegments getImpl(const LockedKey & locked_key, const FileSegment::Range & range) const;
FileSegmentCell * getCell(const Key & key, size_t offset, std::lock_guard<std::mutex> & cache_lock);
/// Returns non-owned pointer to the cell stored in the `files` map.
/// Doesn't reserve any space.
FileSegmentCell * addCell(
const Key & key,
FileSegments splitRangeIntoFileSegments(
LockedKey & locked_key,
size_t offset,
size_t size,
FileSegment::State state,
const CreateFileSegmentSettings & create_settings,
std::lock_guard<std::mutex> & cache_lock);
static void useCell(const FileSegmentCell & cell, FileSegments & result, std::lock_guard<std::mutex> & cache_lock);
bool tryReserveForMainList(
const Key & key,
size_t offset,
size_t size,
QueryContextPtr query_context,
std::lock_guard<std::mutex> & cache_lock);
FileSegments splitRangeIntoCells(
const Key & key,
size_t offset,
size_t size,
FileSegment::State state,
const CreateFileSegmentSettings & create_settings,
std::lock_guard<std::mutex> & cache_lock);
String dumpStructureUnlocked(const Key & key_, std::lock_guard<std::mutex> & cache_lock);
const CreateFileSegmentSettings & create_settings);
void fillHolesWithEmptyFileSegments(
LockedKey & locked_key,
FileSegments & file_segments,
const Key & key,
const FileSegment::Range & range,
bool fill_with_detached_file_segments,
const CreateFileSegmentSettings & settings,
std::lock_guard<std::mutex> & cache_lock);
const CreateFileSegmentSettings & settings);
size_t getUsedCacheSizeUnlocked(std::lock_guard<std::mutex> & cache_lock) const;
KeyMetadata::iterator addFileSegment(
LockedKey & locked_key,
size_t offset,
size_t size,
FileSegment::State state,
const CreateFileSegmentSettings & create_settings,
const CacheGuard::Lock *);
size_t getAvailableCacheSizeUnlocked(std::lock_guard<std::mutex> & cache_lock) const;
size_t getFileSegmentsNumUnlocked(std::lock_guard<std::mutex> & cache_lock) const;
void assertCacheCellsCorrectness(const FileSegmentsByOffset & cells_by_offset, std::lock_guard<std::mutex> & cache_lock);
void removeKeyDirectoryIfExists(const Key & key, std::lock_guard<std::mutex> & cache_lock) const;
/// Used to track and control the cache access of each query.
/// Through it, we can realize the processing of different queries by the cache layer.
struct QueryContext
{
FileCacheRecords records;
FileCachePriorityPtr priority;
size_t cache_size = 0;
size_t max_cache_size;
bool skip_download_if_exceeds_query_cache;
QueryContext(size_t max_cache_size_, bool skip_download_if_exceeds_query_cache_)
: max_cache_size(max_cache_size_)
, skip_download_if_exceeds_query_cache(skip_download_if_exceeds_query_cache_) {}
size_t getMaxCacheSize() const { return max_cache_size; }
size_t getCacheSize() const { return cache_size; }
FileCachePriorityPtr getPriority() const { return priority; }
bool isSkipDownloadIfExceed() const { return skip_download_if_exceeds_query_cache; }
void remove(const Key & key, size_t offset, size_t size, std::lock_guard<std::mutex> & cache_lock);
void reserve(const Key & key, size_t offset, size_t size, std::lock_guard<std::mutex> & cache_lock);
void use(const Key & key, size_t offset, std::lock_guard<std::mutex> & cache_lock);
};
using QueryContextMap = std::unordered_map<String, QueryContextPtr>;
QueryContextMap query_map;
QueryContextPtr getCurrentQueryContext(std::lock_guard<std::mutex> & cache_lock);
QueryContextPtr getQueryContext(const String & query_id, std::lock_guard<std::mutex> & cache_lock);
void removeQueryContext(const String & query_id);
QueryContextPtr getOrSetQueryContext(const String & query_id, const ReadSettings & settings, std::lock_guard<std::mutex> &);
public:
void assertCacheCorrectness(const Key & key, std::lock_guard<std::mutex> & cache_lock);
void assertCacheCorrectness(std::lock_guard<std::mutex> & cache_lock);
void assertPriorityCorrectness(std::lock_guard<std::mutex> & cache_lock);
void cleanupThreadFunc();
};
}

View File

@ -0,0 +1,31 @@
#include "FileCacheKey.h"
#include <base/hex.h>
#include <Common/SipHash.h>
#include <Core/UUID.h>
namespace DB
{
FileCacheKey::FileCacheKey(const std::string & path)
: key(sipHash128(path.data(), path.size()))
{
}
FileCacheKey::FileCacheKey(const UInt128 & key_)
: key(key_)
{
}
std::string FileCacheKey::toString() const
{
return getHexUIntLowercase(key);
}
FileCacheKey FileCacheKey::random()
{
return FileCacheKey(UUIDHelpers::generateV4().toUnderType());
}
}

View File

@ -1,26 +1,37 @@
#pragma once
#include <Core/Types.h>
#include <base/hex.h>
#include <Core/UUID.h>
#include <fmt/format.h>
namespace DB
{
struct FileCacheKey
{
UInt128 key;
using KeyHash = UInt128;
KeyHash key;
String toString() const { return getHexUIntLowercase(key); }
std::string toString() const;
FileCacheKey() = default;
explicit FileCacheKey(const UInt128 & key_) : key(key_) { }
explicit FileCacheKey(const std::string & path);
static FileCacheKey random() { return FileCacheKey(UUIDHelpers::generateV4().toUnderType()); }
explicit FileCacheKey(const UInt128 & key_);
static FileCacheKey random();
bool operator==(const FileCacheKey & other) const { return key == other.key; }
};
using FileCacheKeyAndOffset = std::pair<FileCacheKey, size_t>;
struct FileCacheKeyAndOffsetHash
{
std::size_t operator()(const FileCacheKeyAndOffset & key) const
{
return std::hash<UInt128>()(key.first.key) ^ std::hash<UInt64>()(key.second);
}
};
}
namespace std
@ -32,3 +43,13 @@ struct hash<DB::FileCacheKey>
};
}
template <>
struct fmt::formatter<DB::FileCacheKey> : fmt::formatter<std::string>
{
template <typename FormatCtx>
auto format(const DB::FileCacheKey & key, FormatCtx & ctx) const
{
return fmt::formatter<std::string>::format(key.toString(), ctx);
}
};

View File

@ -30,24 +30,26 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration &
if (path.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk Cache requires non-empty `path` field (cache base path) in config");
max_elements = config.getUInt64(config_prefix + ".max_elements", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS);
max_elements = config.getUInt64(config_prefix + ".max_elements", FILECACHE_DEFAULT_MAX_ELEMENTS);
if (config.has(config_prefix + ".max_file_segment_size"))
max_file_segment_size = parseWithSizeSuffix<uint64_t>(config.getString(config_prefix + ".max_file_segment_size"));
else
max_file_segment_size = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE;
max_file_segment_size = FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE;
cache_on_write_operations = config.getUInt64(config_prefix + ".cache_on_write_operations", false);
enable_filesystem_query_cache_limit = config.getUInt64(config_prefix + ".enable_filesystem_query_cache_limit", false);
enable_cache_hits_threshold = config.getUInt64(config_prefix + ".enable_cache_hits_threshold", REMOTE_FS_OBJECTS_CACHE_ENABLE_HITS_THRESHOLD);
cache_hits_threshold = config.getUInt64(config_prefix + ".cache_hits_threshold", FILECACHE_DEFAULT_HITS_THRESHOLD);
enable_bypass_cache_with_threashold = config.getUInt64(config_prefix + ".enable_bypass_cache_with_threashold", false);
if (config.has(config_prefix + ".bypass_cache_threashold"))
bypass_cache_threashold = parseWithSizeSuffix<uint64_t>(config.getString(config_prefix + ".bypass_cache_threashold"));
else
bypass_cache_threashold = REMOTE_FS_OBJECTS_CACHE_BYPASS_THRESHOLD;
bypass_cache_threashold = FILECACHE_BYPASS_THRESHOLD;
do_not_evict_index_and_mark_files = config.getUInt64(config_prefix + ".do_not_evict_index_and_mark_files", false);
delayed_cleanup_interval_ms = config.getUInt64(config_prefix + ".delayed_cleanup_interval_ms", FILECACHE_DELAYED_CLEANUP_INTERVAL_MS);
}
}

View File

@ -13,18 +13,19 @@ struct FileCacheSettings
std::string base_path;
size_t max_size = 0;
size_t max_elements = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS;
size_t max_file_segment_size = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE;
size_t max_elements = FILECACHE_DEFAULT_MAX_ELEMENTS;
size_t max_file_segment_size = FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE;
bool cache_on_write_operations = false;
size_t enable_cache_hits_threshold = REMOTE_FS_OBJECTS_CACHE_ENABLE_HITS_THRESHOLD;
size_t cache_hits_threshold = FILECACHE_DEFAULT_HITS_THRESHOLD;
bool enable_filesystem_query_cache_limit = false;
bool do_not_evict_index_and_mark_files = true;
bool enable_bypass_cache_with_threashold = false;
size_t bypass_cache_threashold = REMOTE_FS_OBJECTS_CACHE_BYPASS_THRESHOLD;
size_t bypass_cache_threashold = FILECACHE_BYPASS_THRESHOLD;
size_t delayed_cleanup_interval_ms = FILECACHE_DELAYED_CLEANUP_INTERVAL_MS;
void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
};

View File

@ -4,10 +4,11 @@
namespace DB
{
static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 100 * 1024 * 1024;
static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS = 1024 * 1024;
static constexpr int REMOTE_FS_OBJECTS_CACHE_ENABLE_HITS_THRESHOLD = 0;
static constexpr size_t REMOTE_FS_OBJECTS_CACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024;;
static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 100 * 1024 * 1024;
static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 1024 * 1024;
static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0;
static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024;
static constexpr size_t FILECACHE_DELAYED_CLEANUP_INTERVAL_MS = 1000 * 60; /// 1 min
class FileCache;
using FileCachePtr = std::shared_ptr<FileCache>;

View File

@ -0,0 +1,26 @@
#pragma once
#include <list>
namespace DB
{
class FileCache;
using FileCachePtr = std::shared_ptr<FileCache>;
class IFileCachePriority;
using FileCachePriorityPtr = std::unique_ptr<IFileCachePriority>;
class FileSegment;
using FileSegmentPtr = std::shared_ptr<FileSegment>;
using FileSegments = std::list<FileSegmentPtr>;
struct FileSegmentMetadata;
using FileSegmentMetadataPtr = std::shared_ptr<FileSegmentMetadata>;
struct LockedKey;
using LockedKeyPtr = std::shared_ptr<LockedKey>;
struct KeyMetadata;
using KeyMetadataPtr = std::shared_ptr<KeyMetadata>;
}

File diff suppressed because it is too large Load Diff

View File

@ -2,13 +2,16 @@
#include <boost/noncopyable.hpp>
#include <Interpreters/Cache/FileCacheKey.h>
#include <Interpreters/Cache/Guards.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromFileBase.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <IO/OpenedFileCache.h>
#include <base/getThreadId.h>
#include <list>
#include <Interpreters/Cache/IFileCachePriority.h>
#include <Interpreters/Cache/FileCache_fwd_internal.h>
#include <queue>
@ -22,14 +25,8 @@ extern const Metric CacheFileSegments;
namespace DB
{
class FileCache;
class ReadBufferFromFileBase;
class FileSegment;
using FileSegmentPtr = std::shared_ptr<FileSegment>;
using FileSegments = std::list<FileSegmentPtr>;
/*
* FileSegmentKind is used to specify the eviction policy for file segments.
*/
@ -61,17 +58,13 @@ struct CreateFileSegmentSettings
CreateFileSegmentSettings() = default;
explicit CreateFileSegmentSettings(FileSegmentKind kind_, bool unbounded_ = false)
: kind(kind_), unbounded(unbounded_)
{}
: kind(kind_), unbounded(unbounded_) {}
};
class FileSegment : private boost::noncopyable, public std::enable_shared_from_this<FileSegment>
{
friend class FileCache;
friend struct FileSegmentsHolder;
friend class FileSegmentRangeWriter;
friend class StorageSystemFilesystemCache;
friend struct LockedKey;
friend class FileCache; /// Because of reserved_size in tryReserve().
public:
using Key = FileCacheKey;
@ -79,6 +72,7 @@ public:
using LocalCacheWriterPtr = std::unique_ptr<WriteBufferFromFile>;
using Downloader = std::string;
using DownloaderId = std::string;
using Priority = IFileCachePriority;
enum class State
{
@ -111,18 +105,20 @@ public:
* If file segment cannot possibly be downloaded (first space reservation attempt failed), mark
* this file segment as out of cache scope.
*/
SKIP_CACHE,
DETACHED,
};
FileSegment(
const Key & key_,
size_t offset_,
size_t size_,
const Key & key_,
FileCache * cache_,
State download_state_,
const CreateFileSegmentSettings & create_settings);
const CreateFileSegmentSettings & create_settings = {},
FileCache * cache_ = nullptr,
std::weak_ptr<KeyMetadata> key_metadata_ = std::weak_ptr<KeyMetadata>(),
Priority::Iterator queue_iterator_ = Priority::Iterator{});
~FileSegment();
~FileSegment() = default;
State state() const;
@ -158,11 +154,10 @@ public:
size_t offset() const { return range().left; }
FileSegmentKind getKind() const { return segment_kind; }
bool isPersistent() const { return segment_kind == FileSegmentKind::Persistent; }
bool isUnbound() const { return is_unbound; }
using UniqueId = std::pair<FileCacheKey, size_t>;
UniqueId getUniqueId() const { return std::pair(key(), offset()); }
bool isPersistent() const { return segment_kind == FileSegmentKind::Persistent; }
bool isUnbound() const { return is_unbound; }
String getPathInLocalCache() const;
@ -177,7 +172,7 @@ public:
DownloaderId getDownloader() const;
/// Wait for the change of state from DOWNLOADING to any other.
State wait();
State wait(size_t offset);
bool isDownloaded() const;
@ -187,11 +182,13 @@ public:
void incrementHitsCount() { ++hits_count; }
size_t getCurrentWriteOffset() const;
size_t getCurrentWriteOffset(bool sync) const;
size_t getFirstNonDownloadedOffset() const;
size_t getFirstNonDownloadedOffset(bool sync) const;
size_t getDownloadedSize() const;
size_t getDownloadedSize(bool sync) const;
size_t getReservedSize() const;
/// Now detached status can be used in the following cases:
/// 1. there is only 1 remaining file segment holder
@ -207,15 +204,43 @@ public:
/// 2. Detached file segment can still be hold by some cache users, but it's state became
/// immutable at the point it was detached, any non-const / stateful method will throw an
/// exception.
void detach(std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & segment_lock);
void detach(const FileSegmentGuard::Lock &, const LockedKey &);
static FileSegmentPtr getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard<std::mutex> & cache_lock);
static FileSegmentPtr getSnapshot(const FileSegmentPtr & file_segment);
bool isDetached() const;
bool isCompleted() const;
/// File segment has a completed state, if this state is final and
/// is not going to be changed. Completed states: DOWNALODED, DETACHED.
bool isCompleted(bool sync = false) const;
void assertCorrectness() const;
void use();
/**
* ========== Methods used by `cache` ========================
*/
FileSegmentGuard::Lock lock() const { return segment_guard.lock(); }
Priority::Iterator getQueueIterator() const;
void setQueueIterator(Priority::Iterator iterator);
KeyMetadataPtr tryGetKeyMetadata() const;
KeyMetadataPtr getKeyMetadata() const;
bool assertCorrectness() const;
/**
* ========== Methods that must do cv.notify() ==================
*/
void complete();
void completePartAndResetDownloader();
void resetDownloader();
/**
* ========== Methods for _only_ file segment's `downloader` ==================
@ -233,16 +258,6 @@ public:
/// Write data into reserved space.
void write(const char * from, size_t size, size_t offset);
/// Complete file segment with a certain state.
void completeWithState(State state);
void completeWithoutState();
/// Complete file segment's part which was last written.
void completePartAndResetDownloader();
void resetDownloader();
// Invariant: if state() != DOWNLOADING and remote file reader is present, the reader's
// available() == 0, and getFileOffsetOfBufferEnd() == our getCurrentWriteOffset().
//
@ -252,125 +267,112 @@ public:
RemoteFileReaderPtr extractRemoteFileReader();
void setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_);
void resetRemoteFileReader();
void setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_);
void setDownloadedSize(size_t delta);
LocalCacheWriterPtr detachWriter();
private:
size_t getFirstNonDownloadedOffsetUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
size_t getCurrentWriteOffsetUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
size_t getDownloadedSizeUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
String getDownloaderUnlocked(const FileSegmentGuard::Lock &) const;
bool isDownloaderUnlocked(const FileSegmentGuard::Lock & segment_lock) const;
void resetDownloaderUnlocked(const FileSegmentGuard::Lock &);
String getInfoForLogUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
void setDownloadState(State state, const FileSegmentGuard::Lock &);
void resetDownloadingStateUnlocked(const FileSegmentGuard::Lock &);
void setDetachedState(const FileSegmentGuard::Lock &);
String getDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
void resetDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock);
void resetDownloadingStateUnlocked(std::unique_lock<std::mutex> & segment_lock);
String getInfoForLogUnlocked(const FileSegmentGuard::Lock &) const;
void setDownloadState(State state);
void setDownloadedUnlocked(const FileSegmentGuard::Lock &);
void setDownloadFailedUnlocked(const FileSegmentGuard::Lock &);
void setDownloadedUnlocked(std::unique_lock<std::mutex> & segment_lock);
void setDownloadFailedUnlocked(std::unique_lock<std::mutex> & segment_lock);
void setDownloadedSizeUnlocked(std::unique_lock<std::mutex> & /* download_lock */, size_t delta);
bool hasFinalizedStateUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
bool isDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
bool isDetached(std::unique_lock<std::mutex> & /* segment_lock */) const { return is_detached; }
void detachAssumeStateFinalized(std::unique_lock<std::mutex> & segment_lock);
[[noreturn]] void throwIfDetachedUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
void assertDetachedStatus(std::unique_lock<std::mutex> & segment_lock) const;
void assertNotDetached() const;
void assertNotDetachedUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
void assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock<std::mutex> & segment_lock) const;
void assertCorrectnessUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
void assertNotDetachedUnlocked(const FileSegmentGuard::Lock &) const;
void assertIsDownloaderUnlocked(const std::string & operation, const FileSegmentGuard::Lock &) const;
bool assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) const;
/// completeWithoutStateUnlocked() is called from destructor of FileSegmentsHolder.
/// Function might check if the caller of the method
/// is the last alive holder of the segment. Therefore, completion and destruction
/// of the file segment pointer must be done under the same cache mutex.
void completeWithoutStateUnlocked(std::lock_guard<std::mutex> & cache_lock);
void completeBasedOnCurrentState(std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & segment_lock);
void completePartAndResetDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock);
void wrapWithCacheInfo(Exception & e, const String & message, std::unique_lock<std::mutex> & segment_lock) const;
LockedKeyPtr lockKeyMetadata(bool assert_exists = true) const;
Key file_key;
Range segment_range;
const FileSegmentKind segment_kind;
/// Size of the segment is not known until it is downloaded and
/// can be bigger than max_file_segment_size.
const bool is_unbound = false;
State download_state;
/// The one who prepares the download
DownloaderId downloader_id;
std::atomic<State> download_state;
DownloaderId downloader_id; /// The one who prepares the download
RemoteFileReaderPtr remote_file_reader;
LocalCacheWriterPtr cache_writer;
bool detached_writer = false;
/// downloaded_size should always be less or equal to reserved_size
size_t downloaded_size = 0;
size_t reserved_size = 0;
/// global locking order rule:
/// 1. cache lock
/// 2. segment lock
mutable std::mutex mutex;
std::condition_variable cv;
/// Protects downloaded_size access with actual write into fs.
/// downloaded_size is not protected by download_mutex in methods which
/// can never be run in parallel to FileSegment::write() method
/// as downloaded_size is updated only in FileSegment::write() method.
/// Such methods are identified by isDownloader() check at their start,
/// e.g. they are executed strictly by the same thread, sequentially.
std::atomic<size_t> downloaded_size = 0;
std::atomic<size_t> reserved_size = 0;
mutable std::mutex download_mutex;
Key file_key;
mutable FileSegmentGuard segment_guard;
std::weak_ptr<KeyMetadata> key_metadata;
mutable Priority::Iterator queue_iterator; /// Iterator is put here on first reservation attempt, if successful.
FileCache * cache;
std::condition_variable cv;
Poco::Logger * log;
/// "detached" file segment means that it is not owned by cache ("detached" from cache).
/// In general case, all file segments are owned by cache.
bool is_detached = false;
bool is_completed = false;
bool is_downloaded = false;
std::atomic<size_t> hits_count = 0; /// cache hits.
std::atomic<size_t> ref_count = 0; /// Used for getting snapshot state
FileSegmentKind segment_kind;
/// Size of the segment is not known until it is downloaded and can be bigger than max_file_segment_size.
bool is_unbound = false;
CurrentMetrics::Increment metric_increment{CurrentMetrics::CacheFileSegments};
};
struct FileSegmentsHolder : private boost::noncopyable
{
FileSegmentsHolder() = default;
explicit FileSegmentsHolder(FileSegments && file_segments_) : file_segments(std::move(file_segments_)) {}
FileSegmentsHolder(FileSegmentsHolder && other) noexcept : file_segments(std::move(other.file_segments)) {}
void reset();
bool empty() const { return file_segments.empty(); }
explicit FileSegmentsHolder(FileSegments && file_segments_, bool complete_on_dtor_ = true)
: file_segments(std::move(file_segments_)), complete_on_dtor(complete_on_dtor_) {}
~FileSegmentsHolder();
bool empty() const { return file_segments.empty(); }
size_t size() const { return file_segments.size(); }
String toString();
void popFront() { completeAndPopFrontImpl(); }
FileSegment & front() { return *file_segments.front(); }
FileSegment & back() { return *file_segments.back(); }
FileSegment & add(FileSegmentPtr && file_segment)
{
file_segments.push_back(file_segment);
return *file_segments.back();
}
FileSegments::iterator begin() { return file_segments.begin(); }
FileSegments::iterator end() { return file_segments.end(); }
FileSegments::const_iterator begin() const { return file_segments.begin(); }
FileSegments::const_iterator end() const { return file_segments.end(); }
void moveTo(FileSegmentsHolder & holder)
{
holder.file_segments.insert(holder.file_segments.end(), file_segments.begin(), file_segments.end());
file_segments.clear();
}
private:
FileSegments file_segments{};
const bool complete_on_dtor = true;
FileSegments::iterator completeAndPopFrontImpl();
};
using FileSegmentsHolderPtr = std::unique_ptr<FileSegmentsHolder>;
}

View File

@ -0,0 +1,117 @@
#pragma once
#include <mutex>
#include <Interpreters/Cache/FileCache_fwd.h>
#include <boost/noncopyable.hpp>
#include <map>
namespace DB
{
/**
* FileCache::get/getOrSet/set
* 1. CacheMetadataGuard::Lock (take key lock and release metadata lock)
* 2. KeyGuard::Lock (hold till the end of the method)
*
* FileCache::tryReserve
* 1. CacheGuard::Lock
* 2. KeyGuard::Lock (taken without metadata lock)
* 3. any number of KeyGuard::Lock's for files which are going to be evicted (taken via metadata lock)
*
* FileCache::removeIfExists
* 1. CacheGuard::Lock
* 2. KeyGuard::Lock (taken via metadata lock)
* 3. FileSegmentGuard::Lock
*
* FileCache::removeAllReleasable
* 1. CacheGuard::Lock
* 2. any number of KeyGuard::Lock's locks (takken via metadata lock), but at a moment of time only one key lock can be hold
* 3. FileSegmentGuard::Lock
*
* FileCache::getSnapshot (for all cache)
* 1. metadata lock
* 2. any number of KeyGuard::Lock's locks (takken via metadata lock), but at a moment of time only one key lock can be hold
* 3. FileSegmentGuard::Lock
*
* FileCache::getSnapshot(key)
* 1. KeyGuard::Lock (taken via metadata lock)
* 2. FileSegmentGuard::Lock
*
* FileSegment::complete
* 1. CacheGuard::Lock
* 2. KeyGuard::Lock (taken without metadata lock)
* 3. FileSegmentGuard::Lock
*
* Rules:
* 1. Priority of locking: CacheGuard::Lock > CacheMetadataGuard::Lock > KeyGuard::Lock > FileSegmentGuard::Lock
* 2. If we take more than one key lock at a moment of time, we need to take CacheGuard::Lock (example: tryReserve())
*
*
* _CacheGuard_
* 1. FileCache::tryReserve
* 2. FileCache::removeIfExists(key)
* 3. FileCache::removeAllReleasable
* 4. FileSegment::complete
*
* _KeyGuard_ _CacheMetadataGuard_
* 1. all from CacheGuard 1. getOrSet/get/set
* 2. getOrSet/get/Set
*
* *This table does not include locks taken for introspection and system tables.
*/
/**
* Cache priority queue guard.
*/
struct CacheGuard : private boost::noncopyable
{
struct Lock : public std::unique_lock<std::mutex>
{
explicit Lock(std::mutex & mutex_) : std::unique_lock<std::mutex>(mutex_) {}
};
Lock lock() { return Lock(mutex); }
std::mutex mutex;
};
/**
* Guard for cache metadata.
*/
struct CacheMetadataGuard : private boost::noncopyable
{
struct Lock : public std::unique_lock<std::mutex>
{
explicit Lock(std::mutex & mutex_) : std::unique_lock<std::mutex>(mutex_) {}
};
Lock lock() { return Lock(mutex); }
std::mutex mutex;
};
/**
* Key guard. A separate guard for each cache key.
*/
struct KeyGuard : private boost::noncopyable
{
struct Lock : public std::unique_lock<std::mutex>
{
explicit Lock(std::mutex & mutex_) : std::unique_lock<std::mutex>(mutex_) {}
};
Lock lock() { return Lock(mutex); }
std::mutex mutex;
};
/**
* Guard for a file segment.
*/
struct FileSegmentGuard : private boost::noncopyable
{
struct Lock : public std::unique_lock<std::mutex>
{
explicit Lock(std::mutex & mutex_) : std::unique_lock<std::mutex>(mutex_) {}
};
Lock lock() { return Lock(mutex); }
std::mutex mutex;
};
}

View File

@ -5,33 +5,35 @@
#include <Core/Types.h>
#include <Common/Exception.h>
#include <Interpreters/Cache/FileCacheKey.h>
#include <Interpreters/Cache/Guards.h>
#include <Interpreters/Cache/FileCache_fwd_internal.h>
namespace DB
{
class IFileCachePriority;
using FileCachePriorityPtr = std::shared_ptr<IFileCachePriority>;
/// IFileCachePriority is used to maintain the priority of cached data.
class IFileCachePriority
class IFileCachePriority : private boost::noncopyable
{
public:
class IIterator;
using Key = FileCacheKey;
using ReadIterator = std::unique_ptr<const IIterator>;
using WriteIterator = std::shared_ptr<IIterator>;
using KeyAndOffset = FileCacheKeyAndOffset;
struct FileCacheRecord
struct Entry
{
Key key;
size_t offset;
size_t size;
size_t hits = 0;
Entry(const Key & key_, size_t offset_, size_t size_, KeyMetadataPtr key_metadata_)
: key(key_), offset(offset_), size(size_), key_metadata(key_metadata_) {}
FileCacheRecord(const Key & key_, size_t offset_, size_t size_) : key(key_), offset(offset_), size(size_) { }
Entry(const Entry & other)
: key(other.key), offset(other.offset), size(other.size.load()), hits(other.hits), key_metadata(other.key_metadata) {}
const Key key;
const size_t offset;
std::atomic<size_t> size;
size_t hits = 0;
const KeyMetadataPtr key_metadata;
};
/// It provides an iterator to traverse the cache priority. Under normal circumstances,
/// Provides an iterator to traverse the cache priority. Under normal circumstances,
/// the iterator can only return the records that have been directly swapped out.
/// For example, in the LRU algorithm, it can traverse all records, but in the LRU-K, it
/// can only traverse the records in the low priority queue.
@ -40,56 +42,54 @@ public:
public:
virtual ~IIterator() = default;
virtual const Key & key() const = 0;
virtual size_t use(const CacheGuard::Lock &) = 0;
virtual size_t offset() const = 0;
virtual std::shared_ptr<IIterator> remove(const CacheGuard::Lock &) = 0;
virtual size_t size() const = 0;
virtual const Entry & getEntry() const = 0;
virtual size_t hits() const = 0;
virtual Entry & getEntry() = 0;
/// Point the iterator to the next higher priority cache record.
virtual void next() const = 0;
virtual void annul() = 0;
virtual bool valid() const = 0;
/// Mark a cache record as recently used, it will update the priority
/// of the cache record according to different cache algorithms.
virtual void use(std::lock_guard<std::mutex> &) = 0;
/// Deletes an existing cached record. And to avoid pointer suspension
/// the iterator should automatically point to the next record.
virtual void removeAndGetNext(std::lock_guard<std::mutex> &) = 0;
virtual void updateSize(ssize_t, std::lock_guard<std::mutex> &) = 0;
virtual void updateSize(int64_t size) = 0;
};
public:
using Iterator = std::shared_ptr<IIterator>;
using ConstIterator = std::shared_ptr<const IIterator>;
enum class IterationResult
{
BREAK,
CONTINUE,
REMOVE_AND_CONTINUE,
};
using IterateFunc = std::function<IterationResult(LockedKey &, FileSegmentMetadataPtr)>;
IFileCachePriority(size_t max_size_, size_t max_elements_) : max_size(max_size_), max_elements(max_elements_) {}
virtual ~IFileCachePriority() = default;
/// Add a cache record that did not exist before, and throw a
/// logical exception if the cache block already exists.
virtual WriteIterator add(const Key & key, size_t offset, size_t size, std::lock_guard<std::mutex> & cache_lock) = 0;
size_t getElementsLimit() const { return max_elements; }
/// This method is used for assertions in debug mode. So we do not care about complexity here.
/// Query whether a cache record exists. If it exists, return true. If not, return false.
virtual bool contains(const Key & key, size_t offset, std::lock_guard<std::mutex> & cache_lock) = 0;
size_t getSizeLimit() const { return max_size; }
virtual void removeAll(std::lock_guard<std::mutex> & cache_lock) = 0;
virtual size_t getSize(const CacheGuard::Lock &) const = 0;
/// Returns an iterator pointing to the lowest priority cached record.
/// We can traverse all cached records through the iterator's next().
virtual ReadIterator getLowestPriorityReadIterator(std::lock_guard<std::mutex> & cache_lock) = 0;
virtual size_t getElementsCount(const CacheGuard::Lock &) const = 0;
/// The same as getLowestPriorityReadIterator(), but it is writeable.
virtual WriteIterator getLowestPriorityWriteIterator(std::lock_guard<std::mutex> & cache_lock) = 0;
virtual Iterator add(
KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) = 0;
virtual size_t getElementsNum(std::lock_guard<std::mutex> & cache_lock) const = 0;
virtual void pop(const CacheGuard::Lock &) = 0;
size_t getCacheSize(std::lock_guard<std::mutex> &) const { return cache_size; }
virtual void removeAll(const CacheGuard::Lock &) = 0;
protected:
size_t max_cache_size = 0;
size_t cache_size = 0;
virtual void iterate(IterateFunc && func, const CacheGuard::Lock &) = 0;
private:
const size_t max_size = 0;
const size_t max_elements = 0;
};
};

Some files were not shown because too many files have changed in this diff Show More