Merge branch 'master' into remove-merging-streams

This commit is contained in:
mergify[bot] 2021-10-05 14:16:34 +00:00 committed by GitHub
commit ab55225c8f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2426 changed files with 19417 additions and 10703 deletions

View File

@ -1,3 +1,105 @@
### ClickHouse release v21.10, 2021-10-08
#### Backward Incompatible Change
* Now the following MergeTree table-level settings: `replicated_max_parallel_sends`, `replicated_max_parallel_sends_for_table`, `replicated_max_parallel_fetches`, `replicated_max_parallel_fetches_for_table` do nothing. They never worked well and were replaced with `max_replicated_fetches_network_bandwidth`, `max_replicated_sends_network_bandwidth` and `background_fetches_pool_size`. [#28404](https://github.com/ClickHouse/ClickHouse/pull/28404) ([alesapin](https://github.com/alesapin)).
#### New Feature
* Add feature for creating user-defined functions (UDF) as lambda expressions. Syntax `CREATE FUNCTION {function_name} as ({parameters}) -> {function core}`. Example `CREATE FUNCTION plus_one as (a) -> a + 1`. Authors @Realist007. [#27796](https://github.com/ClickHouse/ClickHouse/pull/27796) ([Maksim Kita](https://github.com/kitaisreal)) [#23978](https://github.com/ClickHouse/ClickHouse/pull/23978) ([Realist007](https://github.com/Realist007)).
* Added `Executable` storage engine and `executable` table function. It enables data processing with external scripts in streaming fashion. [#28102](https://github.com/ClickHouse/ClickHouse/pull/28102) ([Maksim Kita](https://github.com/kitaisreal)) ([ruct](https://github.com/ruct)).
* Added `ExecutablePool` storage engine. Similar to `Executable` but it's using a pool of long running processes. [#28518](https://github.com/ClickHouse/ClickHouse/pull/28518) ([Maksim Kita](https://github.com/kitaisreal)).
* Add `ALTER TABLE ... MATERIALIZE COLUMN` query. [#27038](https://github.com/ClickHouse/ClickHouse/pull/27038) ([Vladimir Chebotarev](https://github.com/excitoon)).
* Support for partitioned write into `s3` table function. [#23051](https://github.com/ClickHouse/ClickHouse/pull/23051) ([Vladimir Chebotarev](https://github.com/excitoon)).
* Support `lz4` compression format (in addition to `gz`, `bz2`, `xz`, `zstd`) for data import / export. [#25310](https://github.com/ClickHouse/ClickHouse/pull/25310) ([Bharat Nallan](https://github.com/bharatnc)).
* Allow positional arguments under setting `enable_positional_arguments`. Closes [#2592](https://github.com/ClickHouse/ClickHouse/issues/2592). [#27530](https://github.com/ClickHouse/ClickHouse/pull/27530) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Accept user settings related to file formats in `SETTINGS` clause in `CREATE` query for s3 tables. This closes [#27580](https://github.com/ClickHouse/ClickHouse/issues/27580). [#28037](https://github.com/ClickHouse/ClickHouse/pull/28037) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Allow SSL connection for `RabbitMQ` engine. [#28365](https://github.com/ClickHouse/ClickHouse/pull/28365) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Add `getServerPort` function to allow getting server port. When the port is not used by the server, throw an exception. [#27900](https://github.com/ClickHouse/ClickHouse/pull/27900) ([Amos Bird](https://github.com/amosbird)).
* Add conversion functions between "snowflake id" and `DateTime`, `DateTime64`. See [#27058](https://github.com/ClickHouse/ClickHouse/issues/27058). [#27704](https://github.com/ClickHouse/ClickHouse/pull/27704) ([jasine](https://github.com/jasine)).
* Add function `SHA512`. [#27830](https://github.com/ClickHouse/ClickHouse/pull/27830) ([zhanglistar](https://github.com/zhanglistar)).
* Add `log_queries_probability` setting that allows user to write to query_log only a sample of queries. Closes [#16609](https://github.com/ClickHouse/ClickHouse/issues/16609). [#27527](https://github.com/ClickHouse/ClickHouse/pull/27527) ([Nikolay Degterinsky](https://github.com/evillique)).
#### Experimental Feature
* `web` type of disks to store readonly tables on web server in form of static files. See [#23982](https://github.com/ClickHouse/ClickHouse/issues/23982). [#25251](https://github.com/ClickHouse/ClickHouse/pull/25251) ([Kseniia Sumarokova](https://github.com/kssenii)). This is mostly needed to faciliate testing of operation on shared storage and for easy importing of datasets. Not recommended to use before release 21.11.
* Added new commands `BACKUP` and `RESTORE`. [#21945](https://github.com/ClickHouse/ClickHouse/pull/21945) ([Vitaly Baranov](https://github.com/vitlibar)). This is under development and not intended to be used in current version.
#### Performance Improvement
* Speed up `sumIf` and `countIf` aggregation functions. [#28272](https://github.com/ClickHouse/ClickHouse/pull/28272) ([Raúl Marín](https://github.com/Algunenano)).
* Create virtual projection for `minmax` indices. Now, when `allow_experimental_projection_optimization` is enabled, queries will use minmax index instead of reading the data when possible. [#26286](https://github.com/ClickHouse/ClickHouse/pull/26286) ([Amos Bird](https://github.com/amosbird)).
* Introducing two checks in `sequenceMatch` and `sequenceCount` that allow for early exit when some deterministic part of the sequence pattern is missing from the events list. This change unlocks many queries that would previously fail due to reaching operations cap, and generally speeds up the pipeline. [#27729](https://github.com/ClickHouse/ClickHouse/pull/27729) ([Jakub Kuklis](https://github.com/jkuklis)).
* Enhance primary key analysis with always monotonic information of binary functions, notably non-zero constant division. [#28302](https://github.com/ClickHouse/ClickHouse/pull/28302) ([Amos Bird](https://github.com/amosbird)).
* Make `hasAll` filter condition leverage bloom filter data-skipping indexes. [#27984](https://github.com/ClickHouse/ClickHouse/pull/27984) ([Braulio Valdivielso Martínez](https://github.com/BraulioVM)).
* Speed up data parts loading by delaying table startup process. [#28313](https://github.com/ClickHouse/ClickHouse/pull/28313) ([Amos Bird](https://github.com/amosbird)).
* Fixed possible excessive number of conditions moved from `WHERE` to `PREWHERE` (optimization controlled by settings `optimize_move_to_prewhere`). [#28139](https://github.com/ClickHouse/ClickHouse/pull/28139) ([lthaooo](https://github.com/lthaooo)).
* Enable `optimize_distributed_group_by_sharding_key` by default. [#28105](https://github.com/ClickHouse/ClickHouse/pull/28105) ([Azat Khuzhin](https://github.com/azat)).
#### Improvement
* Check cluster name before creating `Distributed` table, do not allow to create a table with incorrect cluster name. Fixes [#27832](https://github.com/ClickHouse/ClickHouse/issues/27832). [#27927](https://github.com/ClickHouse/ClickHouse/pull/27927) ([tavplubix](https://github.com/tavplubix)).
* Add aggregate function `quantileBFloat16Weighted` similarly to other quantile...Weighted functions. This closes [#27745](https://github.com/ClickHouse/ClickHouse/issues/27745). [#27758](https://github.com/ClickHouse/ClickHouse/pull/27758) ([Ivan Novitskiy](https://github.com/RedClusive)).
* Allow to create dictionaries with empty attributes list. [#27905](https://github.com/ClickHouse/ClickHouse/pull/27905) ([Maksim Kita](https://github.com/kitaisreal)).
* Add interactive documentation in `clickhouse-client` about how to reset the password. This is useful in scenario when user has installed ClickHouse, set up the password and instantly forget it. See [#27750](https://github.com/ClickHouse/ClickHouse/issues/27750). [#27903](https://github.com/ClickHouse/ClickHouse/pull/27903) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Support the case when the data is enclosed in array in `JSONAsString` input format. Closes [#25517](https://github.com/ClickHouse/ClickHouse/issues/25517). [#25633](https://github.com/ClickHouse/ClickHouse/pull/25633) ([Kruglov Pavel](https://github.com/Avogar)).
* Add new column `last_queue_update_exception` to `system.replicas` table. [#26843](https://github.com/ClickHouse/ClickHouse/pull/26843) ([nvartolomei](https://github.com/nvartolomei)).
* Support reconnections on failover for `MaterializedPostgreSQL` tables. Closes [#28529](https://github.com/ClickHouse/ClickHouse/issues/28529). [#28614](https://github.com/ClickHouse/ClickHouse/pull/28614) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Generate a unique server UUID on first server start. [#20089](https://github.com/ClickHouse/ClickHouse/pull/20089) ([Bharat Nallan](https://github.com/bharatnc)).
* Introduce `connection_wait_timeout` (default to 5 seconds, 0 - do not wait) setting for `MySQL` engine. [#28474](https://github.com/ClickHouse/ClickHouse/pull/28474) ([Azat Khuzhin](https://github.com/azat)).
* Do not allow creating `MaterializedPostgreSQL` with bad arguments. Closes [#28423](https://github.com/ClickHouse/ClickHouse/issues/28423). [#28430](https://github.com/ClickHouse/ClickHouse/pull/28430) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Use real tmp file instead of predefined "rows_sources" for vertical merges. This avoids generating garbage directories in tmp disks. [#28299](https://github.com/ClickHouse/ClickHouse/pull/28299) ([Amos Bird](https://github.com/amosbird)).
* Added `libhdfs3_conf` in server config instead of export env `LIBHDFS3_CONF` in clickhouse-server.service. This is for configuration of interaction with HDFS. [#28268](https://github.com/ClickHouse/ClickHouse/pull/28268) ([Zhichang Yu](https://github.com/yuzhichang)).
* Fix removing of parts in a Temporary state which can lead to an unexpected exception (`Part %name% doesn't exist`). Fixes [#23661](https://github.com/ClickHouse/ClickHouse/issues/23661). [#28221](https://github.com/ClickHouse/ClickHouse/pull/28221) [#28221](https://github.com/ClickHouse/ClickHouse/issues/28221)) ([Azat Khuzhin](https://github.com/azat)).
* Fix `zookeeper_log.address` (before the first patch in this PR the address was always `::`) and reduce number of calls `getpeername(2)` for this column (since each time entry for `zookeeper_log` is added `getpeername()` is called, cache this address in the zookeeper client to avoid this). [#28212](https://github.com/ClickHouse/ClickHouse/pull/28212) ([Azat Khuzhin](https://github.com/azat)).
* Support implicit conversions between index in operator `[]` and key of type `Map` (e.g. different `Int` types, `String` and `FixedString`). [#28096](https://github.com/ClickHouse/ClickHouse/pull/28096) ([Anton Popov](https://github.com/CurtizJ)).
* Support `ON CONFLICT` clause when inserting into PostgreSQL table engine or table function. Closes [#27727](https://github.com/ClickHouse/ClickHouse/issues/27727). [#28081](https://github.com/ClickHouse/ClickHouse/pull/28081) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Lower restrictions for `Enum` data type to allow attaching compatible data. Closes [#26672](https://github.com/ClickHouse/ClickHouse/issues/26672). [#28028](https://github.com/ClickHouse/ClickHouse/pull/28028) ([Dmitry Novik](https://github.com/novikd)).
* Add a setting `empty_result_for_aggregation_by_constant_keys_on_empty_set` to control the behavior of grouping by constant keys on empty set. This is to bring back the old baviour of [#6842](https://github.com/ClickHouse/ClickHouse/issues/6842). [#27932](https://github.com/ClickHouse/ClickHouse/pull/27932) ([Amos Bird](https://github.com/amosbird)).
* Added `replication_wait_for_inactive_replica_timeout` setting. It allows to specify how long to wait for inactive replicas to execute `ALTER`/`OPTIMZE`/`TRUNCATE` query (default is 120 seconds). If `replication_alter_partitions_sync` is 2 and some replicas are not active for more than `replication_wait_for_inactive_replica_timeout` seconds, then `UNFINISHED` will be thrown. [#27931](https://github.com/ClickHouse/ClickHouse/pull/27931) ([tavplubix](https://github.com/tavplubix)).
* Support lambda argument for `APPLY` column transformer which allows applying functions with more than one argument. This is for [#27877](https://github.com/ClickHouse/ClickHouse/issues/27877). [#27901](https://github.com/ClickHouse/ClickHouse/pull/27901) ([Amos Bird](https://github.com/amosbird)).
* Enable `tcp_keep_alive_timeout` by default. [#27882](https://github.com/ClickHouse/ClickHouse/pull/27882) ([Azat Khuzhin](https://github.com/azat)).
* Improve remote query cancelation (in case of remote server abnormaly terminated). [#27881](https://github.com/ClickHouse/ClickHouse/pull/27881) ([Azat Khuzhin](https://github.com/azat)).
* Use Multipart copy upload for large S3 objects. [#27858](https://github.com/ClickHouse/ClickHouse/pull/27858) ([ianton-ru](https://github.com/ianton-ru)).
* Allow symlink traversal for library dictionaty path. [#27815](https://github.com/ClickHouse/ClickHouse/pull/27815) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Now `ALTER MODIFY COLUM` `T` to `Nullable(T)` doesn't require mutation. [#27787](https://github.com/ClickHouse/ClickHouse/pull/27787) ([victorgao](https://github.com/kafka1991)).
* Don't silently ignore errors and don't count delays in `ReadBufferFromS3`. [#27484](https://github.com/ClickHouse/ClickHouse/pull/27484) ([Vladimir Chebotarev](https://github.com/excitoon)).
* Improve `ALTER ... MATERIALIZE TTL` by recalculating metadata only without actual TTL action. [#27019](https://github.com/ClickHouse/ClickHouse/pull/27019) ([lthaooo](https://github.com/lthaooo)).
* Allow reading the list of custom top level domains without a new line at EOF. [#28213](https://github.com/ClickHouse/ClickHouse/pull/28213) ([Azat Khuzhin](https://github.com/azat)).
#### Bug Fix
* Fix cases, when reading compressed data from `carbon-clickhouse` fails with 'attempt to read after end of file'. Closes [#26149](https://github.com/ClickHouse/ClickHouse/issues/26149). [#28150](https://github.com/ClickHouse/ClickHouse/pull/28150) ([FArthur-cmd](https://github.com/FArthur-cmd)).
* Fix checking access grants when executing `GRANT WITH REPLACE` statement with `ON CLUSTER` clause. This PR improves fix [#27001](https://github.com/ClickHouse/ClickHouse/pull/27701). [#27983](https://github.com/ClickHouse/ClickHouse/pull/27983) ([Vitaly Baranov](https://github.com/vitlibar)).
* Allow selecting with `extremes = 1` from a column of the type `LowCardinality(UUID)`. [#27918](https://github.com/ClickHouse/ClickHouse/pull/27918) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fix PostgreSQL-style cast (`::` operator) with negative numbers. [#27876](https://github.com/ClickHouse/ClickHouse/pull/27876) ([Anton Popov](https://github.com/CurtizJ)).
* After [#26864](https://github.com/ClickHouse/ClickHouse/pull/26864). Fix shutdown of `NamedSessionStorage`: session contexts stored in `NamedSessionStorage` are now destroyed before destroying the global context. [#27875](https://github.com/ClickHouse/ClickHouse/pull/27875) ([Vitaly Baranov](https://github.com/vitlibar)).
* Bugfix for `windowFunnel` "strict" mode. This fixes [#27469](https://github.com/ClickHouse/ClickHouse/issues/27469). [#27563](https://github.com/ClickHouse/ClickHouse/pull/27563) ([achimbab](https://github.com/achimbab)).
* Fix infinite loop while reading truncated `bzip2` archive. [#28543](https://github.com/ClickHouse/ClickHouse/pull/28543) ([Azat Khuzhin](https://github.com/azat)).
* Fix UUID overlap in `DROP TABLE` for internal DDL from `MaterializedMySQL`. MaterializedMySQL is an experimental feature. [#28533](https://github.com/ClickHouse/ClickHouse/pull/28533) ([Azat Khuzhin](https://github.com/azat)).
* Fix `There is no subcolumn` error, while select from tables, which have `Nested` columns and scalar columns with dot in name and the same prefix as `Nested` (e.g. `n.id UInt32, n.arr1 Array(UInt64), n.arr2 Array(UInt64)`). [#28531](https://github.com/ClickHouse/ClickHouse/pull/28531) ([Anton Popov](https://github.com/CurtizJ)).
* Fix bug which can lead to error `Existing table metadata in ZooKeeper differs in sorting key expression.` after ALTER of `ReplicatedVersionedCollapsingMergeTree`. Fixes [#28515](https://github.com/ClickHouse/ClickHouse/issues/28515). [#28528](https://github.com/ClickHouse/ClickHouse/pull/28528) ([alesapin](https://github.com/alesapin)).
* Fixed possible ZooKeeper watches leak (minor issue) on background processing of distributed DDL queue. Closes [#26036](https://github.com/ClickHouse/ClickHouse/issues/26036). [#28446](https://github.com/ClickHouse/ClickHouse/pull/28446) ([tavplubix](https://github.com/tavplubix)).
* Fix missing quoting of table names in `MaterializedPostgreSQL` engine. Closes [#28316](https://github.com/ClickHouse/ClickHouse/issues/28316). [#28433](https://github.com/ClickHouse/ClickHouse/pull/28433) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix the wrong behaviour of non joined rows from nullable column. Close [#27691](https://github.com/ClickHouse/ClickHouse/issues/27691). [#28349](https://github.com/ClickHouse/ClickHouse/pull/28349) ([vdimir](https://github.com/vdimir)).
* Fix NOT-IN index optimization when not all key columns are used. This fixes [#28120](https://github.com/ClickHouse/ClickHouse/issues/28120). [#28315](https://github.com/ClickHouse/ClickHouse/pull/28315) ([Amos Bird](https://github.com/amosbird)).
* Fix intersecting parts due to new part had been replaced with an empty part. [#28310](https://github.com/ClickHouse/ClickHouse/pull/28310) ([Azat Khuzhin](https://github.com/azat)).
* Fix inconsistent result in queries with `ORDER BY` and `Merge` tables with enabled setting `optimize_read_in_order`. [#28266](https://github.com/ClickHouse/ClickHouse/pull/28266) ([Anton Popov](https://github.com/CurtizJ)).
* Fix possible read of uninitialized memory for queries with `Nullable(LowCardinality)` type and the setting `extremes` set to 1. Fixes [#28165](https://github.com/ClickHouse/ClickHouse/issues/28165). [#28205](https://github.com/ClickHouse/ClickHouse/pull/28205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Multiple small fixes for projections. See detailed description in the PR. [#28178](https://github.com/ClickHouse/ClickHouse/pull/28178) ([Amos Bird](https://github.com/amosbird)).
* Fix extremely rare segfaults on shutdown due to incorrect order of context/config reloader shutdown. [#28088](https://github.com/ClickHouse/ClickHouse/pull/28088) ([nvartolomei](https://github.com/nvartolomei)).
* Fix handling null value with type of `Nullable(String)` in function `JSONExtract`. This fixes [#27929](https://github.com/ClickHouse/ClickHouse/issues/27929) and [#27930](https://github.com/ClickHouse/ClickHouse/issues/27930). This was introduced in https://github.com/ClickHouse/ClickHouse/pull/25452 . [#27939](https://github.com/ClickHouse/ClickHouse/pull/27939) ([Amos Bird](https://github.com/amosbird)).
* Multiple fixes for the new `clickhouse-keeper` tool. Fix a rare bug in `clickhouse-keeper` when the client can receive a watch response before request-response. [#28197](https://github.com/ClickHouse/ClickHouse/pull/28197) ([alesapin](https://github.com/alesapin)). Fix incorrect behavior in `clickhouse-keeper` when list watches (`getChildren`) triggered with `set` requests for children. [#28190](https://github.com/ClickHouse/ClickHouse/pull/28190) ([alesapin](https://github.com/alesapin)). Fix rare case when changes of `clickhouse-keeper` settings may lead to lost logs and server hung. [#28360](https://github.com/ClickHouse/ClickHouse/pull/28360) ([alesapin](https://github.com/alesapin)). Fix bug in `clickhouse-keeper` which can lead to endless logs when `rotate_logs_interval` decreased. [#28152](https://github.com/ClickHouse/ClickHouse/pull/28152) ([alesapin](https://github.com/alesapin)).
#### Build/Testing/Packaging Improvement
* Enable Thread Fuzzer in Stress Test. Thread Fuzzer is ClickHouse feature that allows to test more permutations of thread scheduling and discover more potential issues. This closes [#9813](https://github.com/ClickHouse/ClickHouse/issues/9813). This closes [#9814](https://github.com/ClickHouse/ClickHouse/issues/9814). This closes [#9515](https://github.com/ClickHouse/ClickHouse/issues/9515). This closes [#9516](https://github.com/ClickHouse/ClickHouse/issues/9516). [#27538](https://github.com/ClickHouse/ClickHouse/pull/27538) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Add new log level `test` for testing environments. It is even more verbose than the default `trace`. [#28559](https://github.com/ClickHouse/ClickHouse/pull/28559) ([alesapin](https://github.com/alesapin)).
* Print out git status information at CMake configure stage. [#28047](https://github.com/ClickHouse/ClickHouse/pull/28047) ([Braulio Valdivielso Martínez](https://github.com/BraulioVM)).
* Temporarily switched ubuntu apt repository to mirror ru.archive.ubuntu.com as the default one (archive.ubuntu.com) is not responding from our CI. [#28016](https://github.com/ClickHouse/ClickHouse/pull/28016) ([Ilya Yatsishin](https://github.com/qoega)).
### ClickHouse release v21.9, 2021-09-09
#### Backward Incompatible Change
@ -7,6 +109,7 @@
* Under clickhouse-local, always treat local addresses with a port as remote. [#26736](https://github.com/ClickHouse/ClickHouse/pull/26736) ([Raúl Marín](https://github.com/Algunenano)).
* Fix the issue that in case of some sophisticated query with column aliases identical to the names of expressions, bad cast may happen. This fixes [#25447](https://github.com/ClickHouse/ClickHouse/issues/25447). This fixes [#26914](https://github.com/ClickHouse/ClickHouse/issues/26914). This fix may introduce backward incompatibility: if there are different expressions with identical names, exception will be thrown. It may break some rare cases when `enable_optimize_predicate_expression` is set. [#26639](https://github.com/ClickHouse/ClickHouse/pull/26639) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Now, scalar subquery always returns `Nullable` result if it's type can be `Nullable`. It is needed because in case of empty subquery it's result should be `Null`. Previously, it was possible to get error about incompatible types (type deduction does not execute scalar subquery, and it could use not-nullable type). Scalar subquery with empty result which can't be converted to `Nullable` (like `Array` or `Tuple`) now throws error. Fixes [#25411](https://github.com/ClickHouse/ClickHouse/issues/25411). [#26423](https://github.com/ClickHouse/ClickHouse/pull/26423) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Introduce syntax for here documents. Example `SELECT $doc$ VALUE $doc$`. [#26671](https://github.com/ClickHouse/ClickHouse/pull/26671) ([Maksim Kita](https://github.com/kitaisreal)). This change is backward incompatible if in query there are identifiers that contain `$` [#28768](https://github.com/ClickHouse/ClickHouse/issues/28768).
#### New Feature
@ -17,7 +120,6 @@
* Added integration with S2 geometry library. [#24980](https://github.com/ClickHouse/ClickHouse/pull/24980) ([Andr0901](https://github.com/Andr0901)). ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Add SQLite table engine, table function, database engine. [#24194](https://github.com/ClickHouse/ClickHouse/pull/24194) ([Arslan Gumerov](https://github.com/g-arslan)). ([Kseniia Sumarokova](https://github.com/kssenii)).
* Added support for custom query for `MySQL`, `PostgreSQL`, `ClickHouse`, `JDBC`, `Cassandra` dictionary source. Closes [#1270](https://github.com/ClickHouse/ClickHouse/issues/1270). [#26995](https://github.com/ClickHouse/ClickHouse/pull/26995) ([Maksim Kita](https://github.com/kitaisreal)).
* Introduce syntax for here documents. Example `SELECT $doc$ VALUE $doc$`. [#26671](https://github.com/ClickHouse/ClickHouse/pull/26671) ([Maksim Kita](https://github.com/kitaisreal)).
* Add shared (replicated) storage of user, roles, row policies, quotas and settings profiles through ZooKeeper. [#27426](https://github.com/ClickHouse/ClickHouse/pull/27426) ([Kevin Michel](https://github.com/kmichel-aiven)).
* Add compression for `INTO OUTFILE` that automatically choose compression algorithm. Closes [#3473](https://github.com/ClickHouse/ClickHouse/issues/3473). [#27134](https://github.com/ClickHouse/ClickHouse/pull/27134) ([Filatenkov Artur](https://github.com/FArthur-cmd)).
* Add `INSERT ... FROM INFILE` similarly to `SELECT ... INTO OUTFILE`. [#27655](https://github.com/ClickHouse/ClickHouse/pull/27655) ([Filatenkov Artur](https://github.com/FArthur-cmd)).
@ -1355,7 +1457,7 @@
* Export current max ddl entry executed by DDLWorker via server metric. It's useful to check if DDLWorker hangs somewhere. [#17464](https://github.com/ClickHouse/ClickHouse/pull/17464) ([Amos Bird](https://github.com/amosbird)).
* Export asynchronous metrics of all servers current threads. It's useful to track down issues like [this](https://github.com/ClickHouse-Extras/poco/pull/28). [#17463](https://github.com/ClickHouse/ClickHouse/pull/17463) ([Amos Bird](https://github.com/amosbird)).
* Include dynamic columns like MATERIALIZED / ALIAS for wildcard query when settings `asterisk_include_materialized_columns` and `asterisk_include_alias_columns` are turned on. [#17462](https://github.com/ClickHouse/ClickHouse/pull/17462) ([Ken Chen](https://github.com/chenziliang)).
* Allow specifying [TTL](https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/mergetree/#mergetree-table-ttl) to remove old entries from [system log tables](https://clickhouse.tech/docs/en/operations/system-tables/), using the `<ttl>` attribute in `config.xml`. [#17438](https://github.com/ClickHouse/ClickHouse/pull/17438) ([Du Chuan](https://github.com/spongedu)).
* Allow specifying [TTL](https://clickhouse.com/docs/en/engines/table-engines/mergetree-family/mergetree/#mergetree-table-ttl) to remove old entries from [system log tables](https://clickhouse.com/docs/en/operations/system-tables/), using the `<ttl>` attribute in `config.xml`. [#17438](https://github.com/ClickHouse/ClickHouse/pull/17438) ([Du Chuan](https://github.com/spongedu)).
* Now queries coming to the server via MySQL and PostgreSQL protocols have distinctive interface types (which can be seen in the `interface` column of the table`system.query_log`): `4` for MySQL, and `5` for PostgreSQL, instead of formerly used `1` which is now used for the native protocol only. [#17437](https://github.com/ClickHouse/ClickHouse/pull/17437) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fix parsing of SETTINGS clause of the `INSERT ... SELECT ... SETTINGS` query. [#17414](https://github.com/ClickHouse/ClickHouse/pull/17414) ([Azat Khuzhin](https://github.com/azat)).
* Correctly account memory in RadixSort. [#17412](https://github.com/ClickHouse/ClickHouse/pull/17412) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).

View File

@ -1,4 +1,4 @@
cmake_minimum_required(VERSION 3.3)
cmake_minimum_required(VERSION 3.14)
foreach(policy
CMP0023
@ -176,6 +176,13 @@ if (COMPILER_CLANG)
endif()
endif ()
# If compiler has support for -Wreserved-identifier. It is difficult to detect by clang version,
# because there are two different branches of clang: clang and AppleClang.
# (AppleClang is not supported by ClickHouse, but some developers have misfortune to use it).
if (HAS_RESERVED_IDENTIFIER)
add_compile_definitions (HAS_RESERVED_IDENTIFIER)
endif ()
# If turned `ON`, assumes the user has either the system GTest library or the bundled one.
option(ENABLE_TESTS "Provide unit_test_dbms target with Google.Test unit tests" ON)
option(ENABLE_EXAMPLES "Build all example programs in 'examples' subdirectories" OFF)
@ -187,6 +194,14 @@ elseif(GLIBC_COMPATIBILITY)
message (${RECONFIGURE_MESSAGE_LEVEL} "Glibc compatibility cannot be enabled in current configuration")
endif ()
if (GLIBC_COMPATIBILITY)
# NOTE: we may also want to check glibc version and add -include only for 2.32+
# however this is extra complexity, especially for cross compiling.
# And anyway it should not break anything for <2.32.
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -include ${CMAKE_CURRENT_SOURCE_DIR}/base/glibc-compatibility/glibc-compat-2.32.h")
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -include ${CMAKE_CURRENT_SOURCE_DIR}/base/glibc-compatibility/glibc-compat-2.32.h")
endif()
if (NOT CMAKE_VERSION VERSION_GREATER "3.9.0")
message (WARNING "CMake version must be greater than 3.9.0 for production builds.")
endif ()

View File

@ -6,38 +6,6 @@ Thank you.
## Technical Info
We have a [developer's guide](https://clickhouse.yandex/docs/en/development/developer_instruction/) for writing code for ClickHouse. Besides this guide, you can find [Overview of ClickHouse Architecture](https://clickhouse.yandex/docs/en/development/architecture/) and instructions on how to build ClickHouse in different environments.
We have a [developer's guide](https://clickhouse.com/docs/en/development/developer_instruction/) for writing code for ClickHouse. Besides this guide, you can find [Overview of ClickHouse Architecture](https://clickhouse.com/docs/en/development/architecture/) and instructions on how to build ClickHouse in different environments.
If you want to contribute to documentation, read the [Contributing to ClickHouse Documentation](docs/README.md) guide.
## Legal Info
In order for us (YANDEX LLC) to accept patches and other contributions from you, you may adopt our Yandex Contributor License Agreement (the "**CLA**"). The current version of the CLA you may find here:
1) https://yandex.ru/legal/cla/?lang=en (in English) and
2) https://yandex.ru/legal/cla/?lang=ru (in Russian).
By adopting the CLA, you state the following:
* You obviously wish and are willingly licensing your contributions to us for our open source projects under the terms of the CLA,
* You have read the terms and conditions of the CLA and agree with them in full,
* You are legally able to provide and license your contributions as stated,
* We may use your contributions for our open source projects and for any other our project too,
* We rely on your assurances concerning the rights of third parties in relation to your contributions.
If you agree with these principles, please read and adopt our CLA. By providing us your contributions, you hereby declare that you have already read and adopt our CLA, and we may freely merge your contributions with our corresponding open source project and use it in further in accordance with terms and conditions of the CLA.
If you have already adopted terms and conditions of the CLA, you are able to provide your contributes. When you submit your pull request, please add the following information into it:
```
I hereby agree to the terms of the CLA available at: [link].
```
Replace the bracketed text as follows:
* [link] is the link at the current version of the CLA (you may add here a link https://yandex.ru/legal/cla/?lang=en (in English) or a link https://yandex.ru/legal/cla/?lang=ru (in Russian).
It is enough to provide us such notification once.
As an alternative, you can provide DCO instead of CLA. You can find the text of DCO here: https://developercertificate.org/
It is enough to read and copy it verbatim to your pull request.
If you don't agree with the CLA and don't want to provide DCO, you still can open a pull request to provide your contributions.

View File

@ -188,7 +188,7 @@ Copyright 2016-2021 ClickHouse, Inc.
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright 2016-2021 Yandex LLC
Copyright 2016-2021 ClickHouse, Inc.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.

View File

@ -1,14 +1,14 @@
[![ClickHouse — open source distributed column-oriented DBMS](https://github.com/ClickHouse/ClickHouse/raw/master/website/images/logo-400x240.png)](https://clickhouse.tech)
[![ClickHouse — open source distributed column-oriented DBMS](https://github.com/ClickHouse/ClickHouse/raw/master/website/images/logo-400x240.png)](https://clickhouse.com)
ClickHouse® is an open-source column-oriented database management system that allows generating analytical data reports in real time.
## Useful Links
* [Official website](https://clickhouse.tech/) has quick high-level overview of ClickHouse on main page.
* [Tutorial](https://clickhouse.tech/docs/en/getting_started/tutorial/) shows how to set up and query small ClickHouse cluster.
* [Documentation](https://clickhouse.tech/docs/en/) provides more in-depth information.
* [Official website](https://clickhouse.com/) has quick high-level overview of ClickHouse on main page.
* [Tutorial](https://clickhouse.com/docs/en/getting_started/tutorial/) shows how to set up and query small ClickHouse cluster.
* [Documentation](https://clickhouse.com/docs/en/) provides more in-depth information.
* [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format.
* [Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-rxm3rdrk-lIUmhLC3V8WTaL0TGxsOmg) and [Telegram](https://telegram.me/clickhouse_en) allow to chat with ClickHouse users in real-time.
* [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announcements and reports about events.
* [Code Browser](https://clickhouse.tech/codebrowser/html_report/ClickHouse/index.html) with syntax highlight and navigation.
* [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any.
* [Blog](https://clickhouse.com/blog/en/) contains various ClickHouse-related articles, as well as announcements and reports about events.
* [Code Browser](https://clickhouse.com/codebrowser/html_report/ClickHouse/index.html) with syntax highlight and navigation.
* [Contacts](https://clickhouse.com/company/#contact) can help to get your questions answered if there are any.

View File

@ -1,7 +1,7 @@
# Security Policy
## Security Announcements
Security fixes will be announced by posting them in the [security changelog](https://clickhouse.tech/docs/en/whats-new/security-changelog/)
Security fixes will be announced by posting them in the [security changelog](https://clickhouse.com/docs/en/whats-new/security-changelog/)
## Scope and Supported Versions

View File

@ -2,7 +2,7 @@ if (USE_CLANG_TIDY)
set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}")
endif ()
add_subdirectory (common)
add_subdirectory (base)
add_subdirectory (daemon)
add_subdirectory (loggers)
add_subdirectory (pcg-random)

View File

@ -6,8 +6,8 @@
#include <mutex>
#include <condition_variable>
#include <common/defines.h>
#include <common/MoveOrCopyIfThrow.h>
#include <base/defines.h>
#include <base/MoveOrCopyIfThrow.h>
/** Pool for limited size objects that cannot be used from different threads simultaneously.
* The main use case is to have fixed size of objects that can be reused in difference threads during their lifetime
@ -88,10 +88,12 @@ public:
/// Return object into pool. Client must return same object that was borrowed.
inline void returnObject(T && object_to_return)
{
std::unique_lock<std::mutex> lock(objects_mutex);
{
std::lock_guard<std::mutex> lock(objects_mutex);
objects.emplace_back(std::move(object_to_return));
--borrowed_objects_size;
objects.emplace_back(std::move(object_to_return));
--borrowed_objects_size;
}
condition_variable.notify_one();
}

View File

@ -29,7 +29,7 @@ elseif (ENABLE_READLINE)
endif ()
if (USE_DEBUG_HELPERS)
set (INCLUDE_DEBUG_HELPERS "-include \"${ClickHouse_SOURCE_DIR}/base/common/iostream_debug_helpers.h\"")
set (INCLUDE_DEBUG_HELPERS "-include \"${ClickHouse_SOURCE_DIR}/base/base/iostream_debug_helpers.h\"")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${INCLUDE_DEBUG_HELPERS}")
endif ()

73
base/base/CachedFn.h Normal file
View File

@ -0,0 +1,73 @@
#pragma once
#include <map>
#include <mutex>
#include "FnTraits.h"
/**
* Caching proxy for a functor that decays to a pointer-to-function.
* Saves pairs (func args, func result on args).
* Cache size is unlimited. Cache items are evicted only on manual drop.
* Invocation/update is O(log(saved cache values)).
*
* See Common/tests/cached_fn.cpp for examples.
*/
template <auto * Func>
struct CachedFn
{
private:
using Traits = FnTraits<decltype(Func)>;
using Key = typename Traits::DecayedArgs;
using Result = typename Traits::Ret;
std::map<Key, Result> cache; // Can't use hashmap as tuples are unhashable by default
mutable std::mutex mutex;
public:
template <class ...Args>
Result operator()(Args && ...args)
{
Key key{std::forward<Args>(args)...};
{
std::lock_guard lock(mutex);
if (auto it = cache.find(key); it != cache.end())
return it->second;
}
Result res = std::apply(Func, key);
{
std::lock_guard lock(mutex);
cache.emplace(std::move(key), res);
}
return res;
}
template <class ...Args>
void update(Args && ...args)
{
Key key{std::forward<Args>(args)...};
Result res = std::apply(Func, key);
{
std::lock_guard lock(mutex);
// TODO Can't use emplace(std::move(key), ..), causes test_host_ip_change errors.
cache[key] = std::move(res);
}
}
size_t size() const
{
std::lock_guard lock(mutex);
return cache.size();
}
void drop()
{
std::lock_guard lock(mutex);
cache.clear();
}
};

View File

@ -3,7 +3,7 @@
#include <cctz/civil_time.h>
#include <cctz/time_zone.h>
#include <cctz/zone_info_source.h>
#include <common/getResource.h>
#include <base/getResource.h>
#include <Poco/Exception.h>
#include <algorithm>

View File

@ -1,7 +1,7 @@
#pragma once
#include <common/types.h>
#include <common/strong_typedef.h>
#include <base/types.h>
#include <base/strong_typedef.h>
/** Represents number of days since 1970-01-01.
* See DateLUTImpl for usage examples.

View File

@ -1,5 +1,5 @@
#pragma once
#include "common/extended_types.h"
#include <base/extended_types.h>
#if !defined(NO_SANITIZE_UNDEFINED)
#if defined(__clang__)

View File

@ -3,7 +3,7 @@
#include <cstdint>
#include <cstddef>
#include <cstring>
#include <common/extended_types.h>
#include <base/extended_types.h>
/// Allows to check the internals of IEEE-754 floating point number.

View File

@ -1,7 +1,7 @@
#pragma once
#include <Poco/ErrorHandler.h>
#include <common/logger_useful.h>
#include <base/logger_useful.h>
#include <Common/Exception.h>

37
base/base/FnTraits.h Normal file
View File

@ -0,0 +1,37 @@
#pragma once
#include <tuple>
#include <concepts>
#include <type_traits>
namespace detail
{
template <class T>
struct FnTraits { template <class> static constexpr bool value = false; };
template <class R, class ...A>
struct FnTraits<R(A...)>
{
template <class F>
static constexpr bool value = std::is_invocable_r_v<R, F, A...>;
using Ret = R;
using Args = std::tuple<A...>;
using DecayedArgs = std::tuple<typename std::decay<A>::type...>;
};
template <class R, class ...A>
struct FnTraits<R(*)(A...)> : FnTraits<R(A...)> {};
}
template <class T> using FnTraits = detail::FnTraits<T>;
/**
* A less-typing alias for std::is_invokable_r_v.
* @example void foo(Fn<bool(int, char)> auto && functor)
*/
template <class F, class FS>
concept Fn = FnTraits<FS>::template value<F>;
template <auto Value>
using Constant = std::integral_constant<decltype(Value), Value>;

View File

@ -3,9 +3,9 @@
#include <Poco/UTF8Encoding.h>
#include <Poco/NumberParser.h>
#include <common/JSON.h>
#include <common/find_symbols.h>
#include <common/preciseExp10.h>
#include <base/JSON.h>
#include <base/find_symbols.h>
#include <base/preciseExp10.h>
#include <iostream>

View File

@ -2,8 +2,8 @@
#include <typeinfo>
#include <Poco/Exception.h>
#include <common/StringRef.h>
#include <common/types.h>
#include <base/StringRef.h>
#include <base/types.h>
/** Очень простой класс для чтения JSON (или его кусочков).

View File

@ -1,4 +1,4 @@
#include <common/LineReader.h>
#include <base/LineReader.h>
#include <iostream>
#include <string_view>
@ -16,7 +16,7 @@ extern "C"
}
#endif
#if defined(__clang__) && __clang_major__ >= 13
#ifdef HAS_RESERVED_IDENTIFIER
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif

View File

@ -1,6 +1,6 @@
#pragma once
#include <common/types.h>
#include <base/types.h>
#include <atomic>
#include <vector>

View File

@ -4,7 +4,7 @@
#include <string>
#include <sstream>
#include <exception>
#include <common/DateLUT.h>
#include <base/DateLUT.h>
/** Stores a calendar date in broken-down form (year, month, day-in-month).

View File

@ -3,8 +3,8 @@
#include <string>
#include <iomanip>
#include <exception>
#include <common/DateLUT.h>
#include <common/LocalDate.h>
#include <base/DateLUT.h>
#include <base/LocalDate.h>
/** Stores calendar date and time in broken-down form.

View File

@ -1,6 +1,6 @@
#pragma once
#include <common/types.h>
#include <base/types.h>
namespace detail
{

View File

@ -1,6 +1,6 @@
#include <common/ReadlineLineReader.h>
#include <common/errnoToString.h>
#include <common/scope_guard.h>
#include <base/ReadlineLineReader.h>
#include <base/errnoToString.h>
#include <base/scope_guard.h>
#include <errno.h>
#include <signal.h>

View File

@ -1,5 +1,5 @@
#include <common/ReplxxLineReader.h>
#include <common/errnoToString.h>
#include <base/ReplxxLineReader.h>
#include <base/errnoToString.h>
#include <chrono>
#include <cerrno>

View File

@ -7,8 +7,8 @@
#include <functional>
#include <iosfwd>
#include <common/types.h>
#include <common/unaligned.h>
#include <base/types.h>
#include <base/unaligned.h>
#include <city.h>

View File

@ -1,7 +1,7 @@
#pragma once
#include <common/extended_types.h>
#include <common/defines.h>
#include <base/extended_types.h>
#include <base/defines.h>
namespace common

View File

@ -8,38 +8,25 @@
/// NOTE:
/// - __has_feature cannot be simply undefined,
/// since this will be broken if some C++ header will be included after
/// including <common/defines.h>
/// including <base/defines.h>
/// - it should not have fallback to 0,
/// since this may create false-positive detection (common problem)
#if defined(__clang__) && defined(__has_feature)
# define ch_has_feature __has_feature
#endif
#if defined(_MSC_VER)
# if !defined(likely)
# define likely(x) (x)
# endif
# if !defined(unlikely)
# define unlikely(x) (x)
# endif
#else
# if !defined(likely)
# define likely(x) (__builtin_expect(!!(x), 1))
# endif
# if !defined(unlikely)
# define unlikely(x) (__builtin_expect(!!(x), 0))
# endif
#if !defined(likely)
# define likely(x) (__builtin_expect(!!(x), 1))
#endif
#if !defined(unlikely)
# define unlikely(x) (__builtin_expect(!!(x), 0))
#endif
#if defined(_MSC_VER)
# define ALWAYS_INLINE __forceinline
# define NO_INLINE static __declspec(noinline)
# define MAY_ALIAS
#else
# define ALWAYS_INLINE __attribute__((__always_inline__))
# define NO_INLINE __attribute__((__noinline__))
# define MAY_ALIAS __attribute__((__may_alias__))
#endif
// more aliases: https://mailman.videolan.org/pipermail/x264-devel/2014-May/010660.html
#define ALWAYS_INLINE __attribute__((__always_inline__))
#define NO_INLINE __attribute__((__noinline__))
#define MAY_ALIAS __attribute__((__may_alias__))
#if !defined(__x86_64__) && !defined(__aarch64__) && !defined(__PPC__)
# error "The only supported platforms are x86_64 and AArch64, PowerPC (work in progress)"
@ -117,6 +104,11 @@
# define ALWAYS_INLINE_NO_SANITIZE_UNDEFINED ALWAYS_INLINE
#endif
#if !__has_include(<sanitizer/asan_interface.h>) || !defined(ADDRESS_SANITIZER)
# define ASAN_UNPOISON_MEMORY_REGION(a, b)
# define ASAN_POISON_MEMORY_REGION(a, b)
#endif
/// A template function for suppressing warnings about unused variables or function results.
template <typename... Args>
constexpr void UNUSED(Args &&... args [[maybe_unused]])

View File

@ -1,19 +1,4 @@
#include <common/demangle.h>
#if defined(_MSC_VER)
DemangleResult tryDemangle(const char *)
{
return DemangleResult{};
}
std::string demangle(const char * name, int & status)
{
status = 0;
return name;
}
#else
#include <base/demangle.h>
#include <stdlib.h>
#include <cxxabi.h>
@ -39,6 +24,3 @@ std::string demangle(const char * name, int & status)
return name;
}
#endif

View File

@ -2,8 +2,8 @@
#include <type_traits>
#include <common/types.h>
#include <common/wide_integer.h>
#include <base/types.h>
#include <base/wide_integer.h>
using Int128 = wide::integer<128, signed>;

View File

@ -1,5 +1,5 @@
#include <Poco/Net/DNS.h>
#include <common/getFQDNOrHostName.h>
#include <base/getFQDNOrHostName.h>
namespace

View File

@ -1,6 +1,6 @@
#include <stdexcept>
#include "common/getMemoryAmount.h"
#include "common/getPageSize.h"
#include <base/getMemoryAmount.h>
#include <base/getPageSize.h>
#include <unistd.h>
#include <sys/types.h>

View File

@ -1,7 +1,7 @@
#include "common/getPageSize.h"
#include <base/getPageSize.h>
#include <unistd.h>
Int64 getPageSize()
{
return sysconf(_SC_PAGESIZE);

View File

@ -1,6 +1,7 @@
#pragma once
#include "common/types.h"
#include <base/types.h>
/// Get memory page size
Int64 getPageSize();

View File

@ -1,4 +1,4 @@
#include <common/getThreadId.h>
#include <base/getThreadId.h>
#if defined(OS_ANDROID)
#include <sys/types.h>

View File

@ -30,7 +30,7 @@
#include <cstddef>
#include <cstring>
#include <type_traits>
#include <common/extended_types.h>
#include <base/extended_types.h>
namespace impl

View File

@ -1,4 +1,4 @@
#include <common/mremap.h>
#include <base/mremap.h>
#include <cstddef>
#include <cstdlib>
@ -19,22 +19,14 @@ void * mremap_fallback(
return MAP_FAILED;
}
#if defined(_MSC_VER)
void * new_address = ::operator new(new_size);
#else
void * new_address = mmap(nullptr, new_size, mmap_prot, mmap_flags, mmap_fd, mmap_offset);
if (MAP_FAILED == new_address)
return MAP_FAILED;
#endif
memcpy(new_address, old_address, old_size);
#if defined(_MSC_VER)
delete old_address;
#else
if (munmap(old_address, old_size))
abort();
#endif
return new_address;
}

View File

@ -2,9 +2,7 @@
#include <cstddef>
#include <sys/types.h>
#if !defined(_MSC_VER)
#include <sys/mman.h>
#endif
#ifdef MREMAP_MAYMOVE

View File

@ -1,10 +1,10 @@
#if defined(__clang__) && __clang_major__ >= 13
#ifdef HAS_RESERVED_IDENTIFIER
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
/// This code was based on the code by Fedor Korotkiy (prime@yandex-team.ru) for YT product in Yandex.
#include <common/defines.h>
#include <base/defines.h>
#if defined(__linux__) && !defined(THREAD_SANITIZER)
#define USE_PHDR_CACHE 1

View File

@ -1,7 +1,7 @@
#pragma once
#include <common/scope_guard.h>
#include <common/logger_useful.h>
#include <base/scope_guard.h>
#include <base/logger_useful.h>
#include <Common/MemoryTracker.h>
/// Same as SCOPE_EXIT() but block the MEMORY_LIMIT_EXCEEDED errors.

View File

@ -1,5 +1,5 @@
#include <common/setTerminalEcho.h>
#include <common/errnoToString.h>
#include <base/setTerminalEcho.h>
#include <base/errnoToString.h>
#include <stdexcept>
#include <cstring>
#include <string>

View File

@ -1,4 +1,4 @@
#include <common/shift10.h>
#include <base/shift10.h>
#include "defines.h"

View File

@ -1,6 +1,6 @@
#pragma once
#include <common/types.h>
#include <base/types.h>
/** Almost the same as x = x * exp10(exponent), but gives more accurate result.
* Example:

View File

@ -1,4 +1,4 @@
#include "common/sleep.h"
#include <base/sleep.h>
#include <time.h>
#include <errno.h>

View File

@ -1,5 +1,8 @@
#pragma once
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wold-style-cast"
#if !defined(ARCADIA_BUILD)
# include <miniselect/floyd_rivest_select.h> // Y_IGNORE
#else
@ -34,4 +37,7 @@ void partial_sort(RandomIt first, RandomIt middle, RandomIt last, Compare compar
#else
::std::partial_sort(first, middle, last, compare);
#endif
#pragma GCC diagnostic pop
}

View File

@ -1,5 +1,5 @@
#include <string>
#include <common/terminalColors.h>
#include <base/terminalColors.h>
std::string setColor(UInt64 hash)

View File

@ -1,7 +1,7 @@
#pragma once
#include <string>
#include <common/types.h>
#include <base/types.h>
/** Set color in terminal based on 64-bit hash value.

View File

@ -1,4 +1,4 @@
#include <common/iostream_debug_helpers.h>
#include <base/iostream_debug_helpers.h>
#include <iostream>
#include <memory>

View File

@ -1,7 +1,7 @@
#pragma once
#include <cstddef>
#if defined(__clang__) && __clang_major__ >= 13
#ifdef HAS_RESERVED_IDENTIFIER
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif

View File

@ -4,12 +4,12 @@
#include <Poco/Net/NetException.h>
#include <Poco/Util/HelpFormatter.h>
#include <common/logger_useful.h>
#include <common/range.h>
#include <base/logger_useful.h>
#include <base/range.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/SensitiveDataMasker.h>
#include <common/errnoToString.h>
#include <base/errnoToString.h>
#include <IO/ReadHelpers.h>
#include <Formats/registerFormats.h>
#include <Server/HTTP/HTTPServer.h>

View File

@ -1,82 +0,0 @@
#pragma once
#include <map>
#include <tuple>
#include <mutex>
#include <common/function_traits.h>
/** The simplest cache for a free function.
* You can also pass a static class method or lambda without captures.
* The size is unlimited. Values are stored permanently and never evicted.
* But single record or all cache can be manually dropped.
* Mutex is used for synchronization.
* Suitable only for the simplest cases.
*
* Usage
*
* SimpleCache<decltype(func), &func> func_cached;
* std::cerr << func_cached(args...);
*/
template <typename F, F* f>
class SimpleCache
{
private:
using Key = typename function_traits<F>::arguments_decay;
using Result = typename function_traits<F>::result;
std::map<Key, Result> cache;
mutable std::mutex mutex;
public:
template <typename... Args>
Result operator() (Args &&... args)
{
Key key{std::forward<Args>(args)...};
{
std::lock_guard lock(mutex);
auto it = cache.find(key);
if (cache.end() != it)
return it->second;
}
/// The calculations themselves are not done under mutex.
Result res = std::apply(f, key);
{
std::lock_guard lock(mutex);
cache.emplace(std::forward_as_tuple(args...), res);
}
return res;
}
template <typename... Args>
void update(Args &&... args)
{
Key key{std::forward<Args>(args)...};
Result res = std::apply(f, key);
{
std::lock_guard lock(mutex);
cache[key] = std::move(res);
}
}
size_t size() const
{
std::lock_guard lock(mutex);
return cache.size();
}
void drop()
{
std::lock_guard lock(mutex);
cache.clear();
}
};

View File

@ -1,16 +0,0 @@
#pragma once
#include <tuple>
#include <type_traits>
template <typename T>
struct function_traits;
template <typename ReturnType, typename... Args>
struct function_traits<ReturnType(Args...)>
{
using result = ReturnType;
using arguments = std::tuple<Args...>;
using arguments_decay = std::tuple<typename std::decay<Args>::type...>;
};

View File

@ -1,4 +1,4 @@
#if defined(__clang__) && __clang_major__ >= 13
#ifdef HAS_RESERVED_IDENTIFIER
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
@ -25,7 +25,7 @@
#include <fstream>
#include <sstream>
#include <memory>
#include <common/scope_guard.h>
#include <base/scope_guard.h>
#include <Poco/Observer.h>
#include <Poco/AutoPtr.h>
@ -38,12 +38,12 @@
#include <Poco/SyslogChannel.h>
#include <Poco/DirectoryIterator.h>
#include <common/logger_useful.h>
#include <common/ErrorHandlers.h>
#include <common/argsToConfig.h>
#include <common/getThreadId.h>
#include <common/coverage.h>
#include <common/sleep.h>
#include <base/logger_useful.h>
#include <base/ErrorHandlers.h>
#include <base/argsToConfig.h>
#include <base/getThreadId.h>
#include <base/coverage.h>
#include <base/sleep.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>

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