Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into nanodbc

This commit is contained in:
kssenii 2021-04-09 11:13:01 +00:00
commit 72d89bc933
224 changed files with 4193 additions and 1523 deletions

View File

@ -1,3 +1,154 @@
## ClickHouse release 21.4
### ClickHouse release 21.4.1 2021-04-08
#### Backward Incompatible Change
* The `toStartOfIntervalFunction` will align hour intervals to the midnight (in previous versions they were aligned to the start of unix epoch). For example, `toStartOfInterval(x, INTERVAL 11 HOUR)` will split every day into three intervals: `00:00:00..10:59:59`, `11:00:00..21:59:59` and `22:00:00..23:59:59`. This behaviour is more suited for practical needs. This closes [#9510](https://github.com/ClickHouse/ClickHouse/issues/9510). [#22060](https://github.com/ClickHouse/ClickHouse/pull/22060) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* `Age` and `Precision` in graphite rollup configs should increase from retention to retention. Now it's checked and the wrong config raises an exception. [#21496](https://github.com/ClickHouse/ClickHouse/pull/21496) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Fix `cutToFirstSignificantSubdomainCustom()`/`firstSignificantSubdomainCustom()` returning wrong result for 3+ level domains present in custom top-level domain list. For input domains matching these custom top-level domains, the third-level domain was considered to be the first significant one. This is now fixed. This change may introduce incompatibility if the function is used in e.g. the sharding key. [#21946](https://github.com/ClickHouse/ClickHouse/pull/21946) ([Azat Khuzhin](https://github.com/azat)).
* Column `keys` in table `system.dictionaries` was replaced to columns `key.names` and `key.types`. Columns `key.names`, `key.types`, `attribute.names`, `attribute.types` from `system.dictionaries` table does not require dictionary to be loaded. [#21884](https://github.com/ClickHouse/ClickHouse/pull/21884) ([Maksim Kita](https://github.com/kitaisreal)).
* Now replicas that are processing the `ALTER TABLE ATTACH PART[ITION]` command search in their `detached/` folders before fetching the data from other replicas. As an implementation detail, a new command `ATTACH_PART` is introduced in the replicated log. Parts are searched and compared by their checksums. [#18978](https://github.com/ClickHouse/ClickHouse/pull/18978) ([Mike Kot](https://github.com/myrrc)). **Note**:
* `ATTACH PART[ITION]` queries may not work during cluster upgrade.
* It's not possible to rollback to older ClickHouse version after executing `ALTER ... ATTACH` query in new version as the old servers would fail to pass the `ATTACH_PART` entry in the replicated log.
#### New Feature
* Extended range of `DateTime64` to support dates from year 1925 to 2283. Improved support of `DateTime` around zero date (`1970-01-01`). [#9404](https://github.com/ClickHouse/ClickHouse/pull/9404) ([alexey-milovidov](https://github.com/alexey-milovidov), [Vasily Nemkov](https://github.com/Enmk)).
* Add `prefer_column_name_to_alias` setting to use original column names instead of aliases. it is needed to be more compatible with common databases' aliasing rules. This is for [#9715](https://github.com/ClickHouse/ClickHouse/issues/9715) and [#9887](https://github.com/ClickHouse/ClickHouse/issues/9887). [#22044](https://github.com/ClickHouse/ClickHouse/pull/22044) ([Amos Bird](https://github.com/amosbird)).
* Added functions `dictGetChildren(dictionary, key)`, `dictGetDescendants(dictionary, key, level)`. Function `dictGetChildren` return all children as an array if indexes. It is a inverse transformation for `dictGetHierarchy`. Function `dictGetDescendants` return all descendants as if `dictGetChildren` was applied `level` times recursively. Zero `level` value is equivalent to infinity. Closes [#14656](https://github.com/ClickHouse/ClickHouse/issues/14656). [#22096](https://github.com/ClickHouse/ClickHouse/pull/22096) ([Maksim Kita](https://github.com/kitaisreal)).
* Added `executable_pool` dictionary source. Close [#14528](https://github.com/ClickHouse/ClickHouse/issues/14528). [#21321](https://github.com/ClickHouse/ClickHouse/pull/21321) ([Maksim Kita](https://github.com/kitaisreal)).
* Added table function `dictionary`. It works the same way as `Dictionary` engine. Closes [#21560](https://github.com/ClickHouse/ClickHouse/issues/21560). [#21910](https://github.com/ClickHouse/ClickHouse/pull/21910) ([Maksim Kita](https://github.com/kitaisreal)).
* Support `Nullable` type for `PolygonDictionary` attribute. [#21890](https://github.com/ClickHouse/ClickHouse/pull/21890) ([Maksim Kita](https://github.com/kitaisreal)).
* Functions `dictGet`, `dictHas` use current database name if it is not specified for dictionaries created with DDL. Closes [#21632](https://github.com/ClickHouse/ClickHouse/issues/21632). [#21859](https://github.com/ClickHouse/ClickHouse/pull/21859) ([Maksim Kita](https://github.com/kitaisreal)).
* Added function `dictGetOrNull`. It works like `dictGet`, but return `Null` in case key was not found in dictionary. Closes [#22375](https://github.com/ClickHouse/ClickHouse/issues/22375). [#22413](https://github.com/ClickHouse/ClickHouse/pull/22413) ([Maksim Kita](https://github.com/kitaisreal)).
* Added async update in `ComplexKeyCache`, `SSDCache`, `SSDComplexKeyCache` dictionaries. Added support for `Nullable` type in `Cache`, `ComplexKeyCache`, `SSDCache`, `SSDComplexKeyCache` dictionaries. Added support for multiple attributes fetch with `dictGet`, `dictGetOrDefault` functions. Fixes [#21517](https://github.com/ClickHouse/ClickHouse/issues/21517). [#20595](https://github.com/ClickHouse/ClickHouse/pull/20595) ([Maksim Kita](https://github.com/kitaisreal)).
* Support `dictHas` function for `RangeHashedDictionary`. Fixes [#6680](https://github.com/ClickHouse/ClickHouse/issues/6680). [#19816](https://github.com/ClickHouse/ClickHouse/pull/19816) ([Maksim Kita](https://github.com/kitaisreal)).
* Add function `timezoneOf` that returns the timezone name of `DateTime` or `DateTime64` data types. This does not close [#9959](https://github.com/ClickHouse/ClickHouse/issues/9959). Fix inconsistencies in function names: add aliases `timezone` and `timeZone` as well as `toTimezone` and `toTimeZone` and `timezoneOf` and `timeZoneOf`. [#22001](https://github.com/ClickHouse/ClickHouse/pull/22001) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Add new optional clause `GRANTEES` for `CREATE/ALTER USER` commands. It specifies users or roles which are allowed to receive grants from this user on condition this user has also all required access granted with grant option. By default `GRANTEES ANY` is used which means a user with grant option can grant to anyone. Syntax: `CREATE USER ... GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]`. [#21641](https://github.com/ClickHouse/ClickHouse/pull/21641) ([Vitaly Baranov](https://github.com/vitlibar)).
* Add new column `slowdowns_count` to `system.clusters`. When using hedged requests, it shows how many times we switched to another replica because this replica was responding slowly. Also show actual value of `errors_count` in `system.clusters`. [#21480](https://github.com/ClickHouse/ClickHouse/pull/21480) ([Kruglov Pavel](https://github.com/Avogar)).
* Add `_partition_id` virtual column for `MergeTree*` engines. Allow to prune partitions by `_partition_id`. Add `partitionID()` function to calculate partition id string. [#21401](https://github.com/ClickHouse/ClickHouse/pull/21401) ([Amos Bird](https://github.com/amosbird)).
* Add function `isIPAddressInRange` to test if an IPv4 or IPv6 address is contained in a given CIDR network prefix. [#21329](https://github.com/ClickHouse/ClickHouse/pull/21329) ([PHO](https://github.com/depressed-pho)).
* Added new SQL command `ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name'`. This command is needed to properly remove 'freezed' partitions from all disks. [#21142](https://github.com/ClickHouse/ClickHouse/pull/21142) ([Pavel Kovalenko](https://github.com/Jokser)).
* Supports implicit key type conversion for JOIN. [#19885](https://github.com/ClickHouse/ClickHouse/pull/19885) ([Vladimir](https://github.com/vdimir)).
#### Experimental Feature
* Support `RANGE OFFSET` frame (for window functions) for floating point types. Implement `lagInFrame`/`leadInFrame` window functions, which are analogous to `lag`/`lead`, but respect the window frame. They are identical when the frame is `between unbounded preceding and unbounded following`. This closes [#5485](https://github.com/ClickHouse/ClickHouse/issues/5485). [#21895](https://github.com/ClickHouse/ClickHouse/pull/21895) ([Alexander Kuzmenkov](https://github.com/akuzm)).
* Zero-copy replication for `ReplicatedMergeTree` over S3 storage. [#16240](https://github.com/ClickHouse/ClickHouse/pull/16240) ([ianton-ru](https://github.com/ianton-ru)).
* Added possibility to migrate existing S3 disk to the schema with backup-restore capabilities. [#22070](https://github.com/ClickHouse/ClickHouse/pull/22070) ([Pavel Kovalenko](https://github.com/Jokser)).
#### Performance Improvement
* Supported parallel formatting in `clickhouse-local` and everywhere else. [#21630](https://github.com/ClickHouse/ClickHouse/pull/21630) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Support parallel parsing for `CSVWithNames` and `TSVWithNames` formats. This closes [#21085](https://github.com/ClickHouse/ClickHouse/issues/21085). [#21149](https://github.com/ClickHouse/ClickHouse/pull/21149) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Enable read with mmap IO for file ranges from 64 MiB (the settings `min_bytes_to_use_mmap_io`). It may lead to moderate performance improvement. [#22326](https://github.com/ClickHouse/ClickHouse/pull/22326) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Add cache for files read with `min_bytes_to_use_mmap_io` setting. It makes significant (2x and more) performance improvement when the value of the setting is small by avoiding frequent mmap/munmap calls and the consequent page faults. Note that mmap IO has major drawbacks that makes it less reliable in production (e.g. hung or SIGBUS on faulty disks; less controllable memory usage). Nevertheless it is good in benchmarks. [#22206](https://github.com/ClickHouse/ClickHouse/pull/22206) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Avoid unnecessary data copy when using codec `NONE`. Please note that codec `NONE` is mostly useless - it's recommended to always use compression (`LZ4` is by default). Despite the common belief, disabling compression may not improve performance (the opposite effect is possible). The `NONE` codec is useful in some cases: - when data is uncompressable; - for synthetic benchmarks. [#22145](https://github.com/ClickHouse/ClickHouse/pull/22145) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Faster `GROUP BY` with small `max_rows_to_group_by` and `group_by_overflow_mode='any'`. [#21856](https://github.com/ClickHouse/ClickHouse/pull/21856) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Optimize performance of queries like `SELECT ... FINAL ... WHERE`. Now in queries with `FINAL` it's allowed to move to `PREWHERE` columns, which are in sorting key. [#21830](https://github.com/ClickHouse/ClickHouse/pull/21830) ([foolchi](https://github.com/foolchi)).
* Improved performance by replacing `memcpy` to another implementation. This closes [#18583](https://github.com/ClickHouse/ClickHouse/issues/18583). [#21520](https://github.com/ClickHouse/ClickHouse/pull/21520) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Improve performance of aggregation in order of sorting key (with enabled setting `optimize_aggregation_in_order`). [#19401](https://github.com/ClickHouse/ClickHouse/pull/19401) ([Anton Popov](https://github.com/CurtizJ)).
#### Improvement
* Add connection pool for PostgreSQL table/database engine and dictionary source. Should fix [#21444](https://github.com/ClickHouse/ClickHouse/issues/21444). [#21839](https://github.com/ClickHouse/ClickHouse/pull/21839) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Support non-default table schema for postgres storage/table-function. Closes [#21701](https://github.com/ClickHouse/ClickHouse/issues/21701). [#21711](https://github.com/ClickHouse/ClickHouse/pull/21711) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Support replicas priority for postgres dictionary source. [#21710](https://github.com/ClickHouse/ClickHouse/pull/21710) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Introduce a new merge tree setting `min_bytes_to_rebalance_partition_over_jbod` which allows assigning new parts to different disks of a JBOD volume in a balanced way. [#16481](https://github.com/ClickHouse/ClickHouse/pull/16481) ([Amos Bird](https://github.com/amosbird)).
* Added `Grant`, `Revoke` and `System` values of `query_kind` column for corresponding queries in `system.query_log`. [#21102](https://github.com/ClickHouse/ClickHouse/pull/21102) ([Vasily Nemkov](https://github.com/Enmk)).
* Allow customizing timeouts for HTTP connections used for replication independently from other HTTP timeouts. [#20088](https://github.com/ClickHouse/ClickHouse/pull/20088) ([nvartolomei](https://github.com/nvartolomei)).
* Better exception message in client in case of exception while server is writing blocks. In previous versions client may get misleading message like `Data compressed with different methods`. [#22427](https://github.com/ClickHouse/ClickHouse/pull/22427) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Fix error `Directory tmp_fetch_XXX already exists` which could happen after failed fetch part. Delete temporary fetch directory if it already exists. Fixes [#14197](https://github.com/ClickHouse/ClickHouse/issues/14197). [#22411](https://github.com/ClickHouse/ClickHouse/pull/22411) ([nvartolomei](https://github.com/nvartolomei)).
* Fix MSan report for function `range` with `UInt256` argument (support for large integers is experimental). This closes [#22157](https://github.com/ClickHouse/ClickHouse/issues/22157). [#22387](https://github.com/ClickHouse/ClickHouse/pull/22387) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Add `current_database` column to `system.processes` table. It contains the current database of the query. [#22365](https://github.com/ClickHouse/ClickHouse/pull/22365) ([Alexander Kuzmenkov](https://github.com/akuzm)).
* Add case-insensitive history search/navigation and subword movement features to `clickhouse-client`. [#22105](https://github.com/ClickHouse/ClickHouse/pull/22105) ([Amos Bird](https://github.com/amosbird)).
* If tuple of NULLs, e.g. `(NULL, NULL)` is on the left hand side of `IN` operator with tuples of non-NULLs on the right hand side, e.g. `SELECT (NULL, NULL) IN ((0, 0), (3, 1))` return 0 instead of throwing an exception about incompatible types. The expression may also appear due to optimization of something like `SELECT (NULL, NULL) = (8, 0) OR (NULL, NULL) = (3, 2) OR (NULL, NULL) = (0, 0) OR (NULL, NULL) = (3, 1)`. This closes [#22017](https://github.com/ClickHouse/ClickHouse/issues/22017). [#22063](https://github.com/ClickHouse/ClickHouse/pull/22063) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Update used version of simdjson to 0.9.1. This fixes [#21984](https://github.com/ClickHouse/ClickHouse/issues/21984). [#22057](https://github.com/ClickHouse/ClickHouse/pull/22057) ([Vitaly Baranov](https://github.com/vitlibar)).
* Added case insensitive aliases for `CONNECTION_ID()` and `VERSION()` functions. This fixes [#22028](https://github.com/ClickHouse/ClickHouse/issues/22028). [#22042](https://github.com/ClickHouse/ClickHouse/pull/22042) ([Eugene Klimov](https://github.com/Slach)).
* Add option `strict_increase` to `windowFunnel` function to calculate each event once (resolve [#21835](https://github.com/ClickHouse/ClickHouse/issues/21835)). [#22025](https://github.com/ClickHouse/ClickHouse/pull/22025) ([Vladimir](https://github.com/vdimir)).
* If partition key of a `MergeTree` table does not include `Date` or `DateTime` columns but includes exactly one `DateTime64` column, expose its values in the `min_time` and `max_time` columns in `system.parts` and `system.parts_columns` tables. Add `min_time` and `max_time` columns to `system.parts_columns` table (these was inconsistency to the `system.parts` table). This closes [#18244](https://github.com/ClickHouse/ClickHouse/issues/18244). [#22011](https://github.com/ClickHouse/ClickHouse/pull/22011) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Supported `replication_alter_partitions_sync=1` setting in `clickhouse-copier` for moving partitions from helping table to destination. Decreased default timeouts. Fixes [#21911](https://github.com/ClickHouse/ClickHouse/issues/21911). [#21912](https://github.com/ClickHouse/ClickHouse/pull/21912) ([turbo jason](https://github.com/songenjie)).
* Show path to data directory of `EmbeddedRocksDB` tables in system tables. [#21903](https://github.com/ClickHouse/ClickHouse/pull/21903) ([tavplubix](https://github.com/tavplubix)).
* Add profile event `HedgedRequestsChangeReplica`, change read data timeout from sec to ms. [#21886](https://github.com/ClickHouse/ClickHouse/pull/21886) ([Kruglov Pavel](https://github.com/Avogar)).
* DiskS3 (experimental feature under development). Fixed bug with the impossibility to move directory if the destination is not empty and cache disk is used. [#21837](https://github.com/ClickHouse/ClickHouse/pull/21837) ([Pavel Kovalenko](https://github.com/Jokser)).
* Better formatting for `Array` and `Map` data types in Web UI. [#21798](https://github.com/ClickHouse/ClickHouse/pull/21798) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Update clusters only if their configurations were updated. [#21685](https://github.com/ClickHouse/ClickHouse/pull/21685) ([Kruglov Pavel](https://github.com/Avogar)).
* Propagate query and session settings for distributed DDL queries. Set `distributed_ddl_entry_format_version` to 2 to enable this. Added `distributed_ddl_output_mode` setting. Supported modes: `none`, `throw` (default), `null_status_on_timeout` and `never_throw`. Miscellaneous fixes and improvements for `Replicated` database engine. [#21535](https://github.com/ClickHouse/ClickHouse/pull/21535) ([tavplubix](https://github.com/tavplubix)).
* If `PODArray` was instantiated with element size that is neither a fraction or a multiple of 16, buffer overflow was possible. No bugs in current releases exist. [#21533](https://github.com/ClickHouse/ClickHouse/pull/21533) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Add `last_error_time`/`last_error_message`/`last_error_stacktrace`/`remote` columns for `system.errors`. [#21529](https://github.com/ClickHouse/ClickHouse/pull/21529) ([Azat Khuzhin](https://github.com/azat)).
* Add aliases `simpleJSONExtract/simpleJSONHas` to `visitParam/visitParamExtract{UInt, Int, Bool, Float, Raw, String}`. Fixes #21383. [#21519](https://github.com/ClickHouse/ClickHouse/pull/21519) ([fastio](https://github.com/fastio)).
* Add setting `optimize_skip_unused_shards_limit` to limit the number of sharding key values for `optimize_skip_unused_shards`. [#21512](https://github.com/ClickHouse/ClickHouse/pull/21512) ([Azat Khuzhin](https://github.com/azat)).
* Improve `clickhouse-format` to not throw exception when there are extra spaces or comment after the last query, and throw exception early with readable message when format `ASTInsertQuery` with data . [#21311](https://github.com/ClickHouse/ClickHouse/pull/21311) ([flynn](https://github.com/ucasFL)).
* Improve support of integer keys in data type `Map`. [#21157](https://github.com/ClickHouse/ClickHouse/pull/21157) ([Anton Popov](https://github.com/CurtizJ)).
* MaterializeMySQL: attempt to reconnect to MySQL if the connection is lost. [#20961](https://github.com/ClickHouse/ClickHouse/pull/20961) ([Håvard Kvålen](https://github.com/havardk)).
* Support more cases to rewrite `CROSS JOIN` to `INNER JOIN`. [#20392](https://github.com/ClickHouse/ClickHouse/pull/20392) ([Vladimir](https://github.com/vdimir)).
* Do not create empty parts on INSERT when `optimize_on_insert` setting enabled. Fixes [#20304](https://github.com/ClickHouse/ClickHouse/issues/20304). [#20387](https://github.com/ClickHouse/ClickHouse/pull/20387) ([Kruglov Pavel](https://github.com/Avogar)).
* `MaterializeMySQL`: add minmax skipping index for `_version` column. [#20382](https://github.com/ClickHouse/ClickHouse/pull/20382) ([Stig Bakken](https://github.com/stigsb)).
* Add option `--backslash` for `clickhouse-format`, which can add a backslash at the end of each line of the formatted query. [#21494](https://github.com/ClickHouse/ClickHouse/pull/21494) ([flynn](https://github.com/ucasFL)).
* Now clickhouse will not throw `LOGICAL_ERROR` exception when we try to mutate the already covered part. Fixes [#22013](https://github.com/ClickHouse/ClickHouse/issues/22013). [#22291](https://github.com/ClickHouse/ClickHouse/pull/22291) ([alesapin](https://github.com/alesapin)).
#### Bug Fix
* Remove socket from epoll before cancelling packet receiver in `HedgedConnections` to prevent possible race. Fixes [#22161](https://github.com/ClickHouse/ClickHouse/issues/22161). [#22443](https://github.com/ClickHouse/ClickHouse/pull/22443) ([Kruglov Pavel](https://github.com/Avogar)).
* Add (missing) memory accounting in parallel parsing routines. In previous versions OOM was possible when the resultset contains very large blocks of data. This closes [#22008](https://github.com/ClickHouse/ClickHouse/issues/22008). [#22425](https://github.com/ClickHouse/ClickHouse/pull/22425) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Fix exception which may happen when `SELECT` has constant `WHERE` condition and source table has columns which names are digits. [#22270](https://github.com/ClickHouse/ClickHouse/pull/22270) ([LiuNeng](https://github.com/liuneng1994)).
* Fix query cancellation with `use_hedged_requests=0` and `async_socket_for_remote=1`. [#22183](https://github.com/ClickHouse/ClickHouse/pull/22183) ([Azat Khuzhin](https://github.com/azat)).
* Fix uncaught exception in `InterserverIOHTTPHandler`. [#22146](https://github.com/ClickHouse/ClickHouse/pull/22146) ([Azat Khuzhin](https://github.com/azat)).
* Fix docker entrypoint in case `http_port` is not in the config. [#22132](https://github.com/ClickHouse/ClickHouse/pull/22132) ([Ewout](https://github.com/devwout)).
* Fix error `Invalid number of rows in Chunk` in `JOIN` with `TOTALS` and `arrayJoin`. Closes [#19303](https://github.com/ClickHouse/ClickHouse/issues/19303). [#22129](https://github.com/ClickHouse/ClickHouse/pull/22129) ([Vladimir](https://github.com/vdimir)).
* Fix the background thread pool name which used to poll message from Kafka. The Kafka engine with the broken thread pool will not consume the message from message queue. [#22122](https://github.com/ClickHouse/ClickHouse/pull/22122) ([fastio](https://github.com/fastio)).
* Fix waiting for `OPTIMIZE` and `ALTER` queries for `ReplicatedMergeTree` table engines. Now the query will not hang when the table was detached or restarted. [#22118](https://github.com/ClickHouse/ClickHouse/pull/22118) ([alesapin](https://github.com/alesapin)).
* Disable `async_socket_for_remote`/`use_hedged_requests` for buggy Linux kernels. [#22109](https://github.com/ClickHouse/ClickHouse/pull/22109) ([Azat Khuzhin](https://github.com/azat)).
* Docker entrypoint: avoid chown of `.` in case when `LOG_PATH` is empty. Closes [#22100](https://github.com/ClickHouse/ClickHouse/issues/22100). [#22102](https://github.com/ClickHouse/ClickHouse/pull/22102) ([filimonov](https://github.com/filimonov)).
* The function `decrypt` was lacking a check for the minimal size of data encrypted in `AEAD` mode. This closes [#21897](https://github.com/ClickHouse/ClickHouse/issues/21897). [#22064](https://github.com/ClickHouse/ClickHouse/pull/22064) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* In rare case, merge for `CollapsingMergeTree` may create granule with `index_granularity + 1` rows. Because of this, internal check, added in [#18928](https://github.com/ClickHouse/ClickHouse/issues/18928) (affects 21.2 and 21.3), may fail with error `Incomplete granules are not allowed while blocks are granules size`. This error did not allow parts to merge. [#21976](https://github.com/ClickHouse/ClickHouse/pull/21976) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Reverted [#15454](https://github.com/ClickHouse/ClickHouse/issues/15454) that may cause significant increase in memory usage while loading external dictionaries of hashed type. This closes [#21935](https://github.com/ClickHouse/ClickHouse/issues/21935). [#21948](https://github.com/ClickHouse/ClickHouse/pull/21948) ([Maksim Kita](https://github.com/kitaisreal)).
* Prevent hedged connections overlaps (`Unknown packet 9 from server` error). [#21941](https://github.com/ClickHouse/ClickHouse/pull/21941) ([Azat Khuzhin](https://github.com/azat)).
* Fix reading the HTTP POST request with "multipart/form-data" content type in some cases. [#21936](https://github.com/ClickHouse/ClickHouse/pull/21936) ([Ivan](https://github.com/abyss7)).
* Fix wrong `ORDER BY` results when a query contains window functions, and optimization for reading in primary key order is applied. Fixes [#21828](https://github.com/ClickHouse/ClickHouse/issues/21828). [#21915](https://github.com/ClickHouse/ClickHouse/pull/21915) ([Alexander Kuzmenkov](https://github.com/akuzm)).
* Fix deadlock in first catboost model execution. Closes [#13832](https://github.com/ClickHouse/ClickHouse/issues/13832). [#21844](https://github.com/ClickHouse/ClickHouse/pull/21844) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix incorrect query result (and possible crash) which could happen when `WHERE` or `HAVING` condition is pushed before `GROUP BY`. Fixes [#21773](https://github.com/ClickHouse/ClickHouse/issues/21773). [#21841](https://github.com/ClickHouse/ClickHouse/pull/21841) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Better error handling and logging in `WriteBufferFromS3`. [#21836](https://github.com/ClickHouse/ClickHouse/pull/21836) ([Pavel Kovalenko](https://github.com/Jokser)).
* Fix possible crashes in aggregate functions with combinator `Distinct`, while using two-level aggregation. This is a follow-up fix of [#18365](https://github.com/ClickHouse/ClickHouse/pull/18365) . Can only reproduced in production env. [#21818](https://github.com/ClickHouse/ClickHouse/pull/21818) ([Amos Bird](https://github.com/amosbird)).
* Fix scalar subquery index analysis. This fixes [#21717](https://github.com/ClickHouse/ClickHouse/issues/21717) , which was introduced in [#18896](https://github.com/ClickHouse/ClickHouse/pull/18896). [#21766](https://github.com/ClickHouse/ClickHouse/pull/21766) ([Amos Bird](https://github.com/amosbird)).
* Fix bug for `ReplicatedMerge` table engines when `ALTER MODIFY COLUMN` query doesn't change the type of `Decimal` column if its size (32 bit or 64 bit) doesn't change. [#21728](https://github.com/ClickHouse/ClickHouse/pull/21728) ([alesapin](https://github.com/alesapin)).
* Fix possible infinite waiting when concurrent `OPTIMIZE` and `DROP` are run for `ReplicatedMergeTree`. [#21716](https://github.com/ClickHouse/ClickHouse/pull/21716) ([Azat Khuzhin](https://github.com/azat)).
* Fix function `arrayElement` with type `Map` for constant integer arguments. [#21699](https://github.com/ClickHouse/ClickHouse/pull/21699) ([Anton Popov](https://github.com/CurtizJ)).
* Fix SIGSEGV on not existing attributes from `ip_trie` with `access_to_key_from_attributes`. [#21692](https://github.com/ClickHouse/ClickHouse/pull/21692) ([Azat Khuzhin](https://github.com/azat)).
* Server now start accepting connections only after `DDLWorker` and dictionaries initialization. [#21676](https://github.com/ClickHouse/ClickHouse/pull/21676) ([Azat Khuzhin](https://github.com/azat)).
* Add type conversion for keys of tables of type `Join` (previously led to SIGSEGV). [#21646](https://github.com/ClickHouse/ClickHouse/pull/21646) ([Azat Khuzhin](https://github.com/azat)).
* Fix distributed requests cancellation (for example simple select from multiple shards with limit, i.e. `select * from remote('127.{2,3}', system.numbers) limit 100`) with `async_socket_for_remote=1`. [#21643](https://github.com/ClickHouse/ClickHouse/pull/21643) ([Azat Khuzhin](https://github.com/azat)).
* Fix `fsync_part_directory` for horizontal merge. [#21642](https://github.com/ClickHouse/ClickHouse/pull/21642) ([Azat Khuzhin](https://github.com/azat)).
* Remove unknown columns from joined table in `WHERE` for queries to external database engines (MySQL, PostgreSQL). close [#14614](https://github.com/ClickHouse/ClickHouse/issues/14614), close [#19288](https://github.com/ClickHouse/ClickHouse/issues/19288) (dup), close [#19645](https://github.com/ClickHouse/ClickHouse/issues/19645) (dup). [#21640](https://github.com/ClickHouse/ClickHouse/pull/21640) ([Vladimir](https://github.com/vdimir)).
* `std::terminate` was called if there is an error writing data into s3. [#21624](https://github.com/ClickHouse/ClickHouse/pull/21624) ([Vladimir](https://github.com/vdimir)).
* Fix possible error `Cannot find column` when `optimize_skip_unused_shards` is enabled and zero shards are used. [#21579](https://github.com/ClickHouse/ClickHouse/pull/21579) ([Azat Khuzhin](https://github.com/azat)).
* In case if query has constant `WHERE` condition, and setting `optimize_skip_unused_shards` enabled, all shards may be skipped and query could return incorrect empty result. [#21550](https://github.com/ClickHouse/ClickHouse/pull/21550) ([Amos Bird](https://github.com/amosbird)).
* Fix table function `clusterAllReplicas` returns wrong `_shard_num`. close [#21481](https://github.com/ClickHouse/ClickHouse/issues/21481). [#21498](https://github.com/ClickHouse/ClickHouse/pull/21498) ([flynn](https://github.com/ucasFL)).
* Fix that S3 table holds old credentials after config update. [#21457](https://github.com/ClickHouse/ClickHouse/pull/21457) ([Grigory Pervakov](https://github.com/GrigoryPervakov)).
* Fixed race on SSL object inside `SecureSocket` in Poco. [#21456](https://github.com/ClickHouse/ClickHouse/pull/21456) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Fix `Avro` format parsing for `Kafka`. Fixes [#21437](https://github.com/ClickHouse/ClickHouse/issues/21437). [#21438](https://github.com/ClickHouse/ClickHouse/pull/21438) ([Ilya Golshtein](https://github.com/ilejn)).
* Fix receive and send timeouts and non-blocking read in secure socket. [#21429](https://github.com/ClickHouse/ClickHouse/pull/21429) ([Kruglov Pavel](https://github.com/Avogar)).
* `force_drop_table` flag didn't work for `MATERIALIZED VIEW`, it's fixed. Fixes [#18943](https://github.com/ClickHouse/ClickHouse/issues/18943). [#20626](https://github.com/ClickHouse/ClickHouse/pull/20626) ([tavplubix](https://github.com/tavplubix)).
* Fix name clashes in `PredicateRewriteVisitor`. It caused incorrect `WHERE` filtration after full join. Close [#20497](https://github.com/ClickHouse/ClickHouse/issues/20497). [#20622](https://github.com/ClickHouse/ClickHouse/pull/20622) ([Vladimir](https://github.com/vdimir)).
* Fixed open behavior of remote host filter in case when there is `remote_url_allow_hosts` section in configuration but no entries there. [#20058](https://github.com/ClickHouse/ClickHouse/pull/20058) ([Vladimir Chebotarev](https://github.com/excitoon)).
#### Build/Testing/Packaging Improvement
* Add [Jepsen](https://github.com/jepsen-io/jepsen) tests for ClickHouse Keeper. [#21677](https://github.com/ClickHouse/ClickHouse/pull/21677) ([alesapin](https://github.com/alesapin)).
* Run stateless tests in parallel in CI. Depends on [#22181](https://github.com/ClickHouse/ClickHouse/issues/22181). [#22300](https://github.com/ClickHouse/ClickHouse/pull/22300) ([alesapin](https://github.com/alesapin)).
* Enable status check for [SQLancer](https://github.com/sqlancer/sqlancer) CI run. [#22015](https://github.com/ClickHouse/ClickHouse/pull/22015) ([Ilya Yatsishin](https://github.com/qoega)).
* Multiple preparations for PowerPC builds: Enable the bundled openldap on `ppc64le`. [#22487](https://github.com/ClickHouse/ClickHouse/pull/22487) ([Kfir Itzhak](https://github.com/mastertheknife)). Enable compiling on `ppc64le` with Clang. [#22476](https://github.com/ClickHouse/ClickHouse/pull/22476) ([Kfir Itzhak](https://github.com/mastertheknife)). Fix compiling boost on `ppc64le`. [#22474](https://github.com/ClickHouse/ClickHouse/pull/22474) ([Kfir Itzhak](https://github.com/mastertheknife)). Fix CMake error about internal CMake variable `CMAKE_ASM_COMPILE_OBJECT` not set on `ppc64le`. [#22469](https://github.com/ClickHouse/ClickHouse/pull/22469) ([Kfir Itzhak](https://github.com/mastertheknife)). Fix Fedora/RHEL/CentOS not finding `libclang_rt.builtins` on `ppc64le`. [#22458](https://github.com/ClickHouse/ClickHouse/pull/22458) ([Kfir Itzhak](https://github.com/mastertheknife)). Enable building with `jemalloc` on `ppc64le`. [#22447](https://github.com/ClickHouse/ClickHouse/pull/22447) ([Kfir Itzhak](https://github.com/mastertheknife)). Fix ClickHouse's config embedding and cctz's timezone embedding on `ppc64le`. [#22445](https://github.com/ClickHouse/ClickHouse/pull/22445) ([Kfir Itzhak](https://github.com/mastertheknife)). Fixed compiling on `ppc64le` and use the correct instruction pointer register on `ppc64le`. [#22430](https://github.com/ClickHouse/ClickHouse/pull/22430) ([Kfir Itzhak](https://github.com/mastertheknife)).
* Re-enable the S3 (AWS) library on `aarch64`. [#22484](https://github.com/ClickHouse/ClickHouse/pull/22484) ([Kfir Itzhak](https://github.com/mastertheknife)).
* Add `tzdata` to Docker containers because reading `ORC` formats requires it. This closes [#14156](https://github.com/ClickHouse/ClickHouse/issues/14156). [#22000](https://github.com/ClickHouse/ClickHouse/pull/22000) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Introduce 2 arguments for `clickhouse-server` image Dockerfile: `deb_location` & `single_binary_location`. [#21977](https://github.com/ClickHouse/ClickHouse/pull/21977) ([filimonov](https://github.com/filimonov)).
* Allow to use clang-tidy with release builds by enabling assertions if it is used. [#21914](https://github.com/ClickHouse/ClickHouse/pull/21914) ([alexey-milovidov](https://github.com/alexey-milovidov)).
* Add llvm-12 binaries name to search in cmake scripts. Implicit constants conversions to mute clang warnings. Updated submodules to build with CMake 3.19. Mute recursion in macro expansion in `readpassphrase` library. Deprecated `-fuse-ld` changed to `--ld-path` for clang. [#21597](https://github.com/ClickHouse/ClickHouse/pull/21597) ([Ilya Yatsishin](https://github.com/qoega)).
* Updating `docker/test/testflows/runner/dockerd-entrypoint.sh` to use Yandex dockerhub-proxy, because Docker Hub has enabled very restrictive rate limits [#21551](https://github.com/ClickHouse/ClickHouse/pull/21551) ([vzakaznikov](https://github.com/vzakaznikov)).
* Fix macOS shared lib build. [#20184](https://github.com/ClickHouse/ClickHouse/pull/20184) ([nvartolomei](https://github.com/nvartolomei)).
* Add `ctime` option to `zookeeper-dump-tree`. It allows to dump node creation time. [#21842](https://github.com/ClickHouse/ClickHouse/pull/21842) ([Ilya](https://github.com/HumanUser)).
## ClickHouse release 21.3 (LTS)
### ClickHouse release v21.3, 2021-03-12

View File

@ -68,17 +68,30 @@ endif ()
include (cmake/find/ccache.cmake)
option(ENABLE_CHECK_HEAVY_BUILDS "Don't allow C++ translation units to compile too long or to take too much memory while compiling" OFF)
# Take care to add prlimit in command line before ccache, or else ccache thinks that
# prlimit is compiler, and clang++ is its input file, and refuses to work with
# multiple inputs, e.g in ccache log:
# [2021-03-31T18:06:32.655327 36900] Command line: /usr/bin/ccache prlimit --as=10000000000 --data=5000000000 --cpu=600 /usr/bin/clang++-11 - ...... std=gnu++2a -MD -MT src/CMakeFiles/dbms.dir/Storages/MergeTree/IMergeTreeDataPart.cpp.o -MF src/CMakeFiles/dbms.dir/Storages/MergeTree/IMergeTreeDataPart.cpp.o.d -o src/CMakeFiles/dbms.dir/Storages/MergeTree/IMergeTreeDataPart.cpp.o -c ../src/Storages/MergeTree/IMergeTreeDataPart.cpp
#
# [2021-03-31T18:06:32.656704 36900] Multiple input files: /usr/bin/clang++-11 and ../src/Storages/MergeTree/IMergeTreeDataPart.cpp
#
# Another way would be to use --ccache-skip option before clang++-11 to make
# ccache ignore it.
option(ENABLE_CHECK_HEAVY_BUILDS "Don't allow C++ translation units to compile too long or to take too much memory while compiling." OFF)
if (ENABLE_CHECK_HEAVY_BUILDS)
# set DATA (since RSS does not work since 2.6.x+) to 2G
set (RLIMIT_DATA 5000000000)
# set VIRT (RLIMIT_AS) to 10G (DATA*10)
set (RLIMIT_AS 10000000000)
# set CPU time limit to 600 seconds
set (RLIMIT_CPU 600)
# gcc10/gcc10/clang -fsanitize=memory is too heavy
if (SANITIZE STREQUAL "memory" OR COMPILER_GCC)
set (RLIMIT_DATA 10000000000)
endif()
set (CMAKE_CXX_COMPILER_LAUNCHER prlimit --as=${RLIMIT_AS} --data=${RLIMIT_DATA} --cpu=600)
set (CMAKE_CXX_COMPILER_LAUNCHER prlimit --as=${RLIMIT_AS} --data=${RLIMIT_DATA} --cpu=${RLIMIT_CPU} ${CMAKE_CXX_COMPILER_LAUNCHER})
endif ()
if (NOT CMAKE_BUILD_TYPE OR CMAKE_BUILD_TYPE STREQUAL "None")

View File

@ -0,0 +1,66 @@
#pragma once
#include <ext/scope_guard.h>
#include <common/logger_useful.h>
#include <Common/MemoryTracker.h>
/// Same as SCOPE_EXIT() but block the MEMORY_LIMIT_EXCEEDED errors.
///
/// Typical example of SCOPE_EXIT_MEMORY() usage is when code under it may do
/// some tiny allocations, that may fail under high memory pressure or/and low
/// max_memory_usage (and related limits).
///
/// NOTE: it should be used with caution.
#define SCOPE_EXIT_MEMORY(...) SCOPE_EXIT( \
MemoryTracker::LockExceptionInThread lock_memory_tracker; \
__VA_ARGS__; \
)
/// Same as SCOPE_EXIT() but try/catch/tryLogCurrentException any exceptions.
///
/// SCOPE_EXIT_SAFE() should be used in case the exception during the code
/// under SCOPE_EXIT() is not "that fatal" and error message in log is enough.
///
/// Good example is calling CurrentThread::detachQueryIfNotDetached().
///
/// Anti-pattern is calling WriteBuffer::finalize() under SCOPE_EXIT_SAFE()
/// (since finalize() can do final write and it is better to fail abnormally
/// instead of ignoring write error).
///
/// NOTE: it should be used with double caution.
#define SCOPE_EXIT_SAFE(...) SCOPE_EXIT( \
try \
{ \
__VA_ARGS__; \
} \
catch (...) \
{ \
tryLogCurrentException(__PRETTY_FUNCTION__); \
} \
)
/// Same as SCOPE_EXIT() but:
/// - block the MEMORY_LIMIT_EXCEEDED errors,
/// - try/catch/tryLogCurrentException any exceptions.
///
/// SCOPE_EXIT_MEMORY_SAFE() can be used when the error can be ignored, and in
/// addition to SCOPE_EXIT_SAFE() it will also lock MEMORY_LIMIT_EXCEEDED to
/// avoid such exceptions.
///
/// It does exists as a separate helper, since you do not need to lock
/// MEMORY_LIMIT_EXCEEDED always (there are cases when code under SCOPE_EXIT does
/// not do any allocations, while LockExceptionInThread increment atomic
/// variable).
///
/// NOTE: it should be used with triple caution.
#define SCOPE_EXIT_MEMORY_SAFE(...) SCOPE_EXIT( \
try \
{ \
MemoryTracker::LockExceptionInThread lock_memory_tracker; \
__VA_ARGS__; \
} \
catch (...) \
{ \
tryLogCurrentException(__PRETTY_FUNCTION__); \
} \
)

View File

@ -32,7 +32,9 @@ if (CCACHE_FOUND AND NOT COMPILER_MATCHES_CCACHE)
if (CCACHE_VERSION VERSION_GREATER "3.2.0" OR NOT CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
message(STATUS "Using ${CCACHE_FOUND} ${CCACHE_VERSION}")
set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE ${CCACHE_FOUND})
set (CMAKE_CXX_COMPILER_LAUNCHER ${CCACHE_FOUND} ${CMAKE_CXX_COMPILER_LAUNCHER})
set (CMAKE_C_COMPILER_LAUNCHER ${CCACHE_FOUND} ${CMAKE_C_COMPILER_LAUNCHER})
set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK ${CCACHE_FOUND})
# debian (debhelpers) set SOURCE_DATE_EPOCH environment variable, that is

View File

@ -11,7 +11,7 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/NuRaft/CMakeLists.txt")
return()
endif ()
if (NOT OS_FREEBSD AND NOT OS_DARWIN)
if (NOT OS_FREEBSD)
set (USE_NURAFT 1)
set (NURAFT_LIBRARY nuraft)

2
contrib/NuRaft vendored

@ -1 +1 @@
Subproject commit 241fd3754a1eb4d82ab68a9a875dc99391ec9f02
Subproject commit c35819f2c8a378d4ba88cc930c17bc20aeb875eb

View File

@ -75,6 +75,8 @@
#define HAVE_STRNDUP 1
// strerror_r
#define HAVE_STRERROR_R 1
// rand_r
#define HAVE_RAND_R 1
#ifdef __APPLE__
// pthread_setname_np

@ -1 +1 @@
Subproject commit f4476ee7311b35b593750f6ae2cbdb62a4006374
Subproject commit 5f4034a3a6376416504f17186c55fe401c6d8e5e

View File

@ -35,35 +35,38 @@ RUN apt-get update \
RUN apt-get update \
&& apt-get install \
bash \
cmake \
build-essential \
ccache \
curl \
gcc-9 \
g++-9 \
clang-10 \
clang-tidy-10 \
lld-10 \
llvm-10 \
llvm-10-dev \
clang-11 \
clang-tidy-10 \
clang-tidy-11 \
lld-11 \
llvm-11 \
llvm-11-dev \
cmake \
cmake \
curl \
g++-9 \
gcc-9 \
gdb \
git \
gperf \
gperf \
intel-opencl-icd \
libicu-dev \
libreadline-dev \
lld-10 \
lld-11 \
llvm-10 \
llvm-10-dev \
llvm-11 \
llvm-11-dev \
moreutils \
ninja-build \
gperf \
git \
opencl-headers \
ocl-icd-libopencl1 \
intel-opencl-icd \
tzdata \
gperf \
cmake \
gdb \
opencl-headers \
pigz \
pixz \
rename \
build-essential \
tzdata \
--yes --no-install-recommends
# This symlink required by gcc to find lld compiler
@ -111,4 +114,4 @@ RUN rm /etc/apt/sources.list.d/proposed-repositories.list && apt-get update
COPY build.sh /
CMD ["/bin/bash", "/build.sh"]
CMD ["bash", "-c", "/build.sh 2>&1 | ts"]

View File

@ -11,16 +11,28 @@ tar xJf gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz -C build/cmake/toolc
mkdir -p build/cmake/toolchain/freebsd-x86_64
tar xJf freebsd-11.3-toolchain.tar.xz -C build/cmake/toolchain/freebsd-x86_64 --strip-components=1
# Uncomment to debug ccache. Don't put ccache log in /output right away, or it
# will be confusingly packed into the "performance" package.
# export CCACHE_LOGFILE=/build/ccache.log
# export CCACHE_DEBUG=1
mkdir -p build/build_docker
cd build/build_docker
ccache --show-stats ||:
ccache --zero-stats ||:
rm -f CMakeCache.txt
# Read cmake arguments into array (possibly empty)
read -ra CMAKE_FLAGS <<< "${CMAKE_FLAGS:-}"
cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" ..
ccache --show-config ||:
ccache --show-stats ||:
ccache --zero-stats ||:
# shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty.
ninja $NINJA_FLAGS clickhouse-bundle
ccache --show-config ||:
ccache --show-stats ||:
mv ./programs/clickhouse* /output
mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds
find . -name '*.so' -print -exec mv '{}' /output \;
@ -64,8 +76,21 @@ then
cp ../programs/server/config.xml /output/config
cp ../programs/server/users.xml /output/config
cp -r --dereference ../programs/server/config.d /output/config
tar -czvf "$COMBINED_OUTPUT.tgz" /output
tar -cv -I pigz -f "$COMBINED_OUTPUT.tgz" /output
rm -r /output/*
mv "$COMBINED_OUTPUT.tgz" /output
fi
ccache --show-stats ||:
if [ "${CCACHE_DEBUG:-}" == "1" ]
then
find . -name '*.ccache-*' -print0 \
| tar -c -I pixz -f /output/ccache-debug.txz --null -T -
fi
if [ -n "$CCACHE_LOGFILE" ]
then
# Compress the log as well, or else the CI will try to compress all log
# files in place, and will fail because this directory is not writable.
tar -cv -I pixz -f /output/ccache.log.txz "$CCACHE_LOGFILE"
fi

View File

@ -34,31 +34,32 @@ RUN curl -O https://clickhouse-builds.s3.yandex.net/utils/1/dpkg-deb \
# Libraries from OS are only needed to test the "unbundled" build (this is not used in production).
RUN apt-get update \
&& apt-get install \
gcc-9 \
g++-9 \
clang-11 \
clang-tidy-11 \
lld-11 \
llvm-11 \
llvm-11-dev \
alien \
clang-10 \
clang-11 \
clang-tidy-10 \
clang-tidy-11 \
cmake \
debhelper \
devscripts \
g++-9 \
gcc-9 \
gdb \
git \
gperf \
lld-10 \
lld-11 \
llvm-10 \
llvm-10-dev \
llvm-11 \
llvm-11-dev \
moreutils \
ninja-build \
perl \
pkg-config \
devscripts \
debhelper \
git \
tzdata \
gperf \
alien \
cmake \
gdb \
moreutils \
pigz \
pixz \
pkg-config \
tzdata \
--yes --no-install-recommends
# NOTE: For some reason we have outdated version of gcc-10 in ubuntu 20.04 stable.

View File

@ -2,8 +2,14 @@
set -x -e
# Uncomment to debug ccache.
# export CCACHE_LOGFILE=/build/ccache.log
# export CCACHE_DEBUG=1
ccache --show-config ||:
ccache --show-stats ||:
ccache --zero-stats ||:
read -ra ALIEN_PKGS <<< "${ALIEN_PKGS:-}"
build/release --no-pbuilder "${ALIEN_PKGS[@]}" | ts '%Y-%m-%d %H:%M:%S'
mv /*.deb /output
@ -22,4 +28,19 @@ then
mv /build/obj-*/src/unit_tests_dbms /output/binary
fi
fi
ccache --show-config ||:
ccache --show-stats ||:
if [ "${CCACHE_DEBUG:-}" == "1" ]
then
find /build -name '*.ccache-*' -print0 \
| tar -c -I pixz -f /output/ccache-debug.txz --null -T -
fi
if [ -n "$CCACHE_LOGFILE" ]
then
# Compress the log as well, or else the CI will try to compress all log
# files in place, and will fail because this directory is not writable.
tar -cv -I pixz -f /output/ccache.log.txz "$CCACHE_LOGFILE"
fi

View File

@ -198,7 +198,7 @@ case "$stage" in
# Lost connection to the server. This probably means that the server died
# with abort.
echo "failure" > status.txt
if ! grep -ao "Received signal.*\|Logical error.*\|Assertion.*failed\|Failed assertion.*\|.*runtime error: .*\|.*is located.*\|SUMMARY: MemorySanitizer:.*\|SUMMARY: ThreadSanitizer:.*\|.*_LIBCPP_ASSERT.*" server.log > description.txt
if ! grep -ao "Received signal.*\|Logical error.*\|Assertion.*failed\|Failed assertion.*\|.*runtime error: .*\|.*is located.*\|SUMMARY: AddressSanitizer:.*\|SUMMARY: MemorySanitizer:.*\|SUMMARY: ThreadSanitizer:.*\|.*_LIBCPP_ASSERT.*" server.log > description.txt
then
echo "Lost connection to server. See the logs." > description.txt
fi

View File

@ -108,6 +108,11 @@ zgrep -Fav "ASan doesn't fully support makecontext/swapcontext functions" > /dev
|| echo -e 'No sanitizer asserts\tOK' >> /test_output/test_results.tsv
rm -f /test_output/tmp
# OOM
zgrep -Fa " <Fatal> Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
# Logical errors
zgrep -Fa "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'Logical error thrown (see clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \
@ -118,7 +123,7 @@ zgrep -Fa "########################################" /var/log/clickhouse-server/
&& echo -e 'Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Not crashed\tOK' >> /test_output/test_results.tsv
# It also checks for OOM or crash without stacktrace (printed by watchdog)
# It also checks for crash without stacktrace (printed by watchdog)
zgrep -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'Fatal message in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv

View File

@ -5,36 +5,87 @@ toc_title: Third-Party Libraries Used
# Third-Party Libraries Used {#third-party-libraries-used}
| Library | License |
|---------------------|----------------------------------------------------------------------------------------------------------------------------------------------|
| base64 | [BSD 2-Clause License](https://github.com/aklomp/base64/blob/a27c565d1b6c676beaf297fe503c4518185666f7/LICENSE) |
| boost | [Boost Software License 1.0](https://github.com/ClickHouse-Extras/boost-extra/blob/6883b40449f378019aec792f9983ce3afc7ff16e/LICENSE_1_0.txt) |
| brotli | [MIT](https://github.com/google/brotli/blob/master/LICENSE) |
| capnproto | [MIT](https://github.com/capnproto/capnproto/blob/master/LICENSE) |
| cctz | [Apache License 2.0](https://github.com/google/cctz/blob/4f9776a310f4952454636363def82c2bf6641d5f/LICENSE.txt) |
| double-conversion | [BSD 3-Clause License](https://github.com/google/double-conversion/blob/cf2f0f3d547dc73b4612028a155b80536902ba02/LICENSE) |
| FastMemcpy | [MIT](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libmemcpy/impl/LICENSE) |
| googletest | [BSD 3-Clause License](https://github.com/google/googletest/blob/master/LICENSE) |
| h3 | [Apache License 2.0](https://github.com/uber/h3/blob/master/LICENSE) |
| hyperscan | [BSD 3-Clause License](https://github.com/intel/hyperscan/blob/master/LICENSE) |
| libcxxabi | [BSD + MIT](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libglibc-compatibility/libcxxabi/LICENSE.TXT) |
| libdivide | [Zlib License](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libdivide/LICENSE.txt) |
| libgsasl | [LGPL v2.1](https://github.com/ClickHouse-Extras/libgsasl/blob/3b8948a4042e34fb00b4fb987535dc9e02e39040/LICENSE) |
| libhdfs3 | [Apache License 2.0](https://github.com/ClickHouse-Extras/libhdfs3/blob/bd6505cbb0c130b0db695305b9a38546fa880e5a/LICENSE.txt) |
| libmetrohash | [Apache License 2.0](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libmetrohash/LICENSE) |
| libpcg-random | [Apache License 2.0](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libpcg-random/LICENSE-APACHE.txt) |
| libressl | [OpenSSL License](https://github.com/ClickHouse-Extras/ssl/blob/master/COPYING) |
| librdkafka | [BSD 2-Clause License](https://github.com/edenhill/librdkafka/blob/363dcad5a23dc29381cc626620e68ae418b3af19/LICENSE) |
| libwidechar_width | [CC0 1.0 Universal](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) |
| llvm | [BSD 3-Clause License](https://github.com/ClickHouse-Extras/llvm/blob/163def217817c90fb982a6daf384744d8472b92b/llvm/LICENSE.TXT) |
| lz4 | [BSD 2-Clause License](https://github.com/lz4/lz4/blob/c10863b98e1503af90616ae99725ecd120265dfb/LICENSE) |
| mariadb-connector-c | [LGPL v2.1](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/3.1/COPYING.LIB) |
| murmurhash | [Public Domain](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/murmurhash/LICENSE) |
| pdqsort | [Zlib License](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/pdqsort/license.txt) |
| poco | [Boost Software License - Version 1.0](https://github.com/ClickHouse-Extras/poco/blob/fe5505e56c27b6ecb0dcbc40c49dc2caf4e9637f/LICENSE) |
| protobuf | [BSD 3-Clause License](https://github.com/ClickHouse-Extras/protobuf/blob/12735370922a35f03999afff478e1c6d7aa917a4/LICENSE) |
| re2 | [BSD 3-Clause License](https://github.com/google/re2/blob/7cf8b88e8f70f97fd4926b56aa87e7f53b2717e0/LICENSE) |
| sentry-native | [MIT License](https://github.com/getsentry/sentry-native/blob/master/LICENSE) |
| UnixODBC | [LGPL v2.1](https://github.com/ClickHouse-Extras/UnixODBC/tree/b0ad30f7f6289c12b76f04bfb9d466374bb32168) |
| zlib-ng | [Zlib License](https://github.com/ClickHouse-Extras/zlib-ng/blob/develop/LICENSE.md) |
| zstd | [BSD 3-Clause License](https://github.com/facebook/zstd/blob/dev/LICENSE) |
The list of third-party libraries can be obtained by the following query:
```
SELECT library_name, license_type, license_path FROM system.licenses ORDER BY library_name COLLATE 'en'
```
[Example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIGxpYnJhcnlfbmFtZSwgbGljZW5zZV90eXBlLCBsaWNlbnNlX3BhdGggRlJPTSBzeXN0ZW0ubGljZW5zZXMgT1JERVIgQlkgbGlicmFyeV9uYW1lIENPTExBVEUgJ2VuJw==)
| library_name | license_type | license_path |
|:-|:-|:-|
| abseil-cpp | Apache | /contrib/abseil-cpp/LICENSE |
| AMQP-CPP | Apache | /contrib/AMQP-CPP/LICENSE |
| arrow | Apache | /contrib/arrow/LICENSE.txt |
| avro | Apache | /contrib/avro/LICENSE.txt |
| aws | Apache | /contrib/aws/LICENSE.txt |
| aws-c-common | Apache | /contrib/aws-c-common/LICENSE |
| aws-c-event-stream | Apache | /contrib/aws-c-event-stream/LICENSE |
| aws-checksums | Apache | /contrib/aws-checksums/LICENSE |
| base64 | BSD 2-clause | /contrib/base64/LICENSE |
| boost | Boost | /contrib/boost/LICENSE_1_0.txt |
| boringssl | BSD | /contrib/boringssl/LICENSE |
| brotli | MIT | /contrib/brotli/LICENSE |
| capnproto | MIT | /contrib/capnproto/LICENSE |
| cassandra | Apache | /contrib/cassandra/LICENSE.txt |
| cctz | Apache | /contrib/cctz/LICENSE.txt |
| cityhash102 | MIT | /contrib/cityhash102/COPYING |
| cppkafka | BSD 2-clause | /contrib/cppkafka/LICENSE |
| croaring | Apache | /contrib/croaring/LICENSE |
| curl | Apache | /contrib/curl/docs/LICENSE-MIXING.md |
| cyrus-sasl | BSD 2-clause | /contrib/cyrus-sasl/COPYING |
| double-conversion | BSD 3-clause | /contrib/double-conversion/LICENSE |
| dragonbox | Apache | /contrib/dragonbox/LICENSE-Apache2-LLVM |
| fast_float | Apache | /contrib/fast_float/LICENSE |
| fastops | MIT | /contrib/fastops/LICENSE |
| flatbuffers | Apache | /contrib/flatbuffers/LICENSE.txt |
| fmtlib | Unknown | /contrib/fmtlib/LICENSE.rst |
| gcem | Apache | /contrib/gcem/LICENSE |
| googletest | BSD 3-clause | /contrib/googletest/LICENSE |
| grpc | Apache | /contrib/grpc/LICENSE |
| h3 | Apache | /contrib/h3/LICENSE |
| hyperscan | Boost | /contrib/hyperscan/LICENSE |
| icu | Public Domain | /contrib/icu/icu4c/LICENSE |
| icudata | Public Domain | /contrib/icudata/LICENSE |
| jemalloc | BSD 2-clause | /contrib/jemalloc/COPYING |
| krb5 | MIT | /contrib/krb5/src/lib/gssapi/LICENSE |
| libc-headers | LGPL | /contrib/libc-headers/LICENSE |
| libcpuid | BSD 2-clause | /contrib/libcpuid/COPYING |
| libcxx | Apache | /contrib/libcxx/LICENSE.TXT |
| libcxxabi | Apache | /contrib/libcxxabi/LICENSE.TXT |
| libdivide | zLib | /contrib/libdivide/LICENSE.txt |
| libfarmhash | MIT | /contrib/libfarmhash/COPYING |
| libgsasl | LGPL | /contrib/libgsasl/LICENSE |
| libhdfs3 | Apache | /contrib/libhdfs3/LICENSE.txt |
| libmetrohash | Apache | /contrib/libmetrohash/LICENSE |
| libpq | Unknown | /contrib/libpq/COPYRIGHT |
| libpqxx | BSD 3-clause | /contrib/libpqxx/COPYING |
| librdkafka | MIT | /contrib/librdkafka/LICENSE.murmur2 |
| libunwind | Apache | /contrib/libunwind/LICENSE.TXT |
| libuv | BSD | /contrib/libuv/LICENSE |
| llvm | Apache | /contrib/llvm/llvm/LICENSE.TXT |
| lz4 | BSD | /contrib/lz4/LICENSE |
| mariadb-connector-c | LGPL | /contrib/mariadb-connector-c/COPYING.LIB |
| miniselect | Boost | /contrib/miniselect/LICENSE_1_0.txt |
| msgpack-c | Boost | /contrib/msgpack-c/LICENSE_1_0.txt |
| murmurhash | Public Domain | /contrib/murmurhash/LICENSE |
| NuRaft | Apache | /contrib/NuRaft/LICENSE |
| openldap | Unknown | /contrib/openldap/LICENSE |
| orc | Apache | /contrib/orc/LICENSE |
| poco | Boost | /contrib/poco/LICENSE |
| protobuf | BSD 3-clause | /contrib/protobuf/LICENSE |
| rapidjson | MIT | /contrib/rapidjson/bin/jsonschema/LICENSE |
| re2 | BSD 3-clause | /contrib/re2/LICENSE |
| replxx | BSD 3-clause | /contrib/replxx/LICENSE.md |
| rocksdb | BSD 3-clause | /contrib/rocksdb/LICENSE.leveldb |
| sentry-native | MIT | /contrib/sentry-native/LICENSE |
| simdjson | Apache | /contrib/simdjson/LICENSE |
| snappy | Public Domain | /contrib/snappy/COPYING |
| sparsehash-c11 | BSD 3-clause | /contrib/sparsehash-c11/LICENSE |
| stats | Apache | /contrib/stats/LICENSE |
| thrift | Apache | /contrib/thrift/LICENSE |
| unixodbc | LGPL | /contrib/unixodbc/COPYING |
| xz | Public Domain | /contrib/xz/COPYING |
| zlib-ng | zLib | /contrib/zlib-ng/LICENSE.md |
| zstd | BSD | /contrib/zstd/LICENSE |

File diff suppressed because one or more lines are too long

View File

@ -48,7 +48,8 @@ toc_title: Adopters
| <a href="https://www.diva-e.com" class="favicon">Diva-e</a> | Digital consulting | Main Product | — | — | [Slides in English, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup29/ClickHouse-MeetUp-Unusual-Applications-sd-2019-09-17.pdf) |
| <a href="https://www.ecwid.com/" class="favicon">Ecwid</a> | E-commerce SaaS | Metrics, Logging | — | — | [Slides in Russian, April 2019](https://nastachku.ru/var/files/1/presentation/backend/2_Backend_6.pdf) |
| <a href="https://www.ebay.com/" class="favicon">eBay</a> | E-commerce | Logs, Metrics and Events | — | — | [Official website, Sep 2020](https://tech.ebayinc.com/engineering/ou-online-analytical-processing/) |
| <a href="https://www.exness.com" class="favicon">Exness</a> | Trading | Metrics, Logging | — | — | [Talk in Russian, May 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) |
| <a href="https://www.exness.com/" class="favicon">Exness</a> | Trading | Metrics, Logging | — | — | [Talk in Russian, May 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) |
| <a href="https://www.eventbunker.io/" class="favicon">EventBunker.io</a> | Serverless Data Processing | — | — | — | [Tweet, April 2021](https://twitter.com/Halil_D_/status/1379839133472985091) |
| <a href="https://fastnetmon.com/" class="favicon">FastNetMon</a> | DDoS Protection | Main Product | | — | [Official website](https://fastnetmon.com/docs-fnm-advanced/fastnetmon-advanced-traffic-persistency/) |
| <a href="https://www.flipkart.com/" class="favicon">Flipkart</a> | e-Commerce | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=239) |
| <a href="https://fun.co/rp" class="favicon">FunCorp</a> | Games | | — | 14 bn records/day as of Jan 2021 | [Article](https://www.altinity.com/blog/migrating-from-redshift-to-clickhouse) |

View File

@ -345,7 +345,8 @@ Similar to `interserver_http_host`, except that this hostname can be used by oth
The username and password used to authenticate during [replication](../../engines/table-engines/mergetree-family/replication.md) with the Replicated\* engines. These credentials are used only for communication between replicas and are unrelated to credentials for ClickHouse clients. The server is checking these credentials for connecting replicas and use the same credentials when connecting to other replicas. So, these credentials should be set the same for all replicas in a cluster.
By default, the authentication is not used.
**Note:** These credentials are common for replication through `HTTP` and `HTTPS`.
!!! note "Note"
These credentials are common for replication through `HTTP` and `HTTPS`.
This section contains the following parameters:

View File

@ -233,7 +233,7 @@ JAVA_OPTS="-Xms{{ '{{' }} cluster.get('xms','128M') {{ '}}' }} \
-XX:+PrintSafepointStatistics \
-XX:+UseParNewGC \
-XX:+UseConcMarkSweepGC \
-XX:+CMSParallelRemarkEnabled"
-XX:+CMSParallelRemarkEnabled"
```
Salt init:

View File

@ -15,7 +15,8 @@ $ sudo service clickhouse-server restart
If you installed ClickHouse using something other than the recommended `deb` packages, use the appropriate update method.
ClickHouse does not support a distributed update. The operation should be performed consecutively on each separate server. Do not update all the servers on a cluster simultaneously, or the cluster will be unavailable for some time.
!!! note "Note"
You can update multiple servers at once as soon as there is no moment when all replicas of one shard are offline.
The upgrade of older version of ClickHouse to specific version:
@ -31,4 +32,3 @@ $ sudo service clickhouse-server restart

View File

@ -5,7 +5,7 @@ toc_title: Date
# Date {#data_type-date}
A date. Stored in two bytes as the number of days since 1970-01-01 (unsigned). Allows storing values from just after the beginning of the Unix Epoch to the upper threshold defined by a constant at the compilation stage (currently, this is until the year 2106, but the final fully-supported year is 2105).
A date. Stored in two bytes as the number of days since 1970-01-01 (unsigned). Allows storing values from just after the beginning of the Unix Epoch to the upper threshold defined by a constant at the compilation stage (currently, this is until the year 2149, but the final fully-supported year is 2148).
The date value is stored without the time zone.

View File

@ -9,7 +9,7 @@ Allows to store an instant in time, that can be expressed as a calendar date and
Tick size (precision): 10<sup>-precision</sup> seconds
Syntax:
**Syntax:**
``` sql
DateTime64(precision, [timezone])
@ -17,9 +17,11 @@ DateTime64(precision, [timezone])
Internally, stores data as a number of ticks since epoch start (1970-01-01 00:00:00 UTC) as Int64. The tick resolution is determined by the precision parameter. Additionally, the `DateTime64` type can store time zone that is the same for the entire column, that affects how the values of the `DateTime64` type values are displayed in text format and how the values specified as strings are parsed (2020-01-01 05:00:01.000). The time zone is not stored in the rows of the table (or in resultset), but is stored in the column metadata. See details in [DateTime](../../sql-reference/data-types/datetime.md).
Supported range from January 1, 1925 till December 31, 2283.
## Examples {#examples}
**1.** Creating a table with `DateTime64`-type column and inserting data into it:
1. Creating a table with `DateTime64`-type column and inserting data into it:
``` sql
CREATE TABLE dt
@ -27,15 +29,15 @@ CREATE TABLE dt
`timestamp` DateTime64(3, 'Europe/Moscow'),
`event_id` UInt8
)
ENGINE = TinyLog
ENGINE = TinyLog;
```
``` sql
INSERT INTO dt Values (1546300800000, 1), ('2019-01-01 00:00:00', 2)
INSERT INTO dt Values (1546300800000, 1), ('2019-01-01 00:00:00', 2);
```
``` sql
SELECT * FROM dt
SELECT * FROM dt;
```
``` text
@ -45,13 +47,13 @@ SELECT * FROM dt
└─────────────────────────┴──────────┘
```
- When inserting datetime as an integer, it is treated as an appropriately scaled Unix Timestamp (UTC). `1546300800000` (with precision 3) represents `'2019-01-01 00:00:00'` UTC. However, as `timestamp` column has `Europe/Moscow` (UTC+3) timezone specified, when outputting as a string the value will be shown as `'2019-01-01 03:00:00'`
- When inserting datetime as an integer, it is treated as an appropriately scaled Unix Timestamp (UTC). `1546300800000` (with precision 3) represents `'2019-01-01 00:00:00'` UTC. However, as `timestamp` column has `Europe/Moscow` (UTC+3) timezone specified, when outputting as a string the value will be shown as `'2019-01-01 03:00:00'`.
- When inserting string value as datetime, it is treated as being in column timezone. `'2019-01-01 00:00:00'` will be treated as being in `Europe/Moscow` timezone and stored as `1546290000000`.
**2.** Filtering on `DateTime64` values
2. Filtering on `DateTime64` values
``` sql
SELECT * FROM dt WHERE timestamp = toDateTime64('2019-01-01 00:00:00', 3, 'Europe/Moscow')
SELECT * FROM dt WHERE timestamp = toDateTime64('2019-01-01 00:00:00', 3, 'Europe/Moscow');
```
``` text
@ -60,12 +62,12 @@ SELECT * FROM dt WHERE timestamp = toDateTime64('2019-01-01 00:00:00', 3, 'Europ
└─────────────────────────┴──────────┘
```
Unlike `DateTime`, `DateTime64` values are not converted from `String` automatically
Unlike `DateTime`, `DateTime64` values are not converted from `String` automatically.
**3.** Getting a time zone for a `DateTime64`-type value:
3. Getting a time zone for a `DateTime64`-type value:
``` sql
SELECT toDateTime64(now(), 3, 'Europe/Moscow') AS column, toTypeName(column) AS x
SELECT toDateTime64(now(), 3, 'Europe/Moscow') AS column, toTypeName(column) AS x;
```
``` text
@ -74,13 +76,13 @@ SELECT toDateTime64(now(), 3, 'Europe/Moscow') AS column, toTypeName(column) AS
└─────────────────────────┴────────────────────────────────┘
```
**4.** Timezone conversion
4. Timezone conversion
``` sql
SELECT
toDateTime64(timestamp, 3, 'Europe/London') as lon_time,
toDateTime64(timestamp, 3, 'Europe/Moscow') as mos_time
FROM dt
FROM dt;
```
``` text
@ -90,7 +92,7 @@ FROM dt
└─────────────────────────┴─────────────────────────┘
```
## See Also {#see-also}
**See Also**
- [Type conversion functions](../../sql-reference/functions/type-conversion-functions.md)
- [Functions for working with dates and times](../../sql-reference/functions/date-time-functions.md)

View File

@ -159,14 +159,14 @@ Configuration fields:
| Tag | Description | Required |
|------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------|
| `name` | Column name. | Yes |
| `type` | ClickHouse data type.<br/>ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.<br/>[Nullable](../../../sql-reference/data-types/nullable.md) is not supported. | Yes |
| `null_value` | Default value for a non-existing element.<br/>In the example, it is an empty string. You cannot use `NULL` in this field. | Yes |
| `type` | ClickHouse data type.<br/>ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.<br/>[Nullable](../../../sql-reference/data-types/nullable.md) is currently supported for [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md) dictionaries. In [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache), [IPTrie](external-dicts-dict-layout.md#ip-trie) dictionaries `Nullable` types are not supported. | Yes |
| `null_value` | Default value for a non-existing element.<br/>In the example, it is an empty string. [NULL](../../syntax.md#null-literal) value can be used only for the `Nullable` types (see the previous line with types description). | Yes |
| `expression` | [Expression](../../../sql-reference/syntax.md#syntax-expressions) that ClickHouse executes on the value.<br/>The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.<br/><br/>Default value: no expression. | No |
| <a name="hierarchical-dict-attr"></a> `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md).<br/><br/>Default value: `false`. | No |
| `injective` | Flag that shows whether the `id -> attribute` image is [injective](https://en.wikipedia.org/wiki/Injective_function).<br/>If `true`, ClickHouse can automatically place after the `GROUP BY` clause the requests to dictionaries with injection. Usually it significantly reduces the amount of such requests.<br/><br/>Default value: `false`. | No |
| `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.<br/><br/>Default value: `false`. | No |
## See Also {#see-also}
**See Also**
- [Functions for working with external dictionaries](../../../sql-reference/functions/ext-dict-functions.md).

View File

@ -10,8 +10,6 @@ A dictionary is a mapping (`key -> attributes`) that is convenient for various t
ClickHouse supports special functions for working with dictionaries that can be used in queries. It is easier and more efficient to use dictionaries with functions than a `JOIN` with reference tables.
[NULL](../../sql-reference/syntax.md#null-literal) values cant be stored in a dictionary.
ClickHouse supports:
- [Built-in dictionaries](../../sql-reference/dictionaries/internal-dicts.md#internal_dicts) with a specific [set of functions](../../sql-reference/functions/ym-dict-functions.md).

View File

@ -79,7 +79,7 @@ The `TTL` is no longer there, so the second row is not deleted:
└───────────────────────┴─────────┴──────────────┘
```
### See Also
**See Also**
- More about the [TTL-expression](../../../sql-reference/statements/create/table.md#ttl-expression).
- Modify column [with TTL](../../../sql-reference/statements/alter/column.md#alter_modify-column).

View File

@ -169,7 +169,7 @@ SYSTEM START MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name
### STOP TTL MERGES {#query_language-stop-ttl-merges}
Provides possibility to stop background delete old data according to [TTL expression](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) for tables in the MergeTree family:
Return `Ok.` even table doesnt exists or table have not MergeTree engine. Return error when database doesnt exists:
Returns `Ok.` even if table doesnt exist or table has not MergeTree engine. Returns error when database doesnt exist:
``` sql
SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name]
@ -178,7 +178,7 @@ SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name]
### START TTL MERGES {#query_language-start-ttl-merges}
Provides possibility to start background delete old data according to [TTL expression](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) for tables in the MergeTree family:
Return `Ok.` even table doesnt exists. Return error when database doesnt exists:
Returns `Ok.` even if table doesnt exist. Returns error when database doesnt exist:
``` sql
SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name]
@ -187,7 +187,7 @@ SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name]
### STOP MOVES {#query_language-stop-moves}
Provides possibility to stop background move data according to [TTL table expression with TO VOLUME or TO DISK clause](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl) for tables in the MergeTree family:
Return `Ok.` even table doesnt exists. Return error when database doesnt exists:
Returns `Ok.` even if table doesnt exist. Returns error when database doesnt exist:
``` sql
SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
@ -196,7 +196,7 @@ SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
### START MOVES {#query_language-start-moves}
Provides possibility to start background move data according to [TTL table expression with TO VOLUME and TO DISK clause](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl) for tables in the MergeTree family:
Return `Ok.` even table doesnt exists. Return error when database doesnt exists:
Returns `Ok.` even if table doesnt exist. Returns error when database doesnt exist:
``` sql
SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
@ -209,7 +209,7 @@ ClickHouse can manage background replication related processes in [ReplicatedMer
### STOP FETCHES {#query_language-system-stop-fetches}
Provides possibility to stop background fetches for inserted parts for tables in the `ReplicatedMergeTree` family:
Always returns `Ok.` regardless of the table engine and even table or database doesnt exists.
Always returns `Ok.` regardless of the table engine and even if table or database doesnt exist.
``` sql
SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name]
@ -218,7 +218,7 @@ SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name]
### START FETCHES {#query_language-system-start-fetches}
Provides possibility to start background fetches for inserted parts for tables in the `ReplicatedMergeTree` family:
Always returns `Ok.` regardless of the table engine and even table or database doesnt exists.
Always returns `Ok.` regardless of the table engine and even if table or database doesnt exist.
``` sql
SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name]
@ -264,7 +264,7 @@ Wait until a `ReplicatedMergeTree` table will be synced with other replicas in a
SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name
```
After running this statement the `[db.]replicated_merge_tree_family_table_name` fetches commands from
After running this statement the `[db.]replicated_merge_tree_family_table_name` fetches commands from
the common replicated log into its own replication queue, and then the query waits till the replica processes all
of the fetched commands.
@ -280,4 +280,3 @@ SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name
### RESTART REPLICAS {#query_language-system-restart-replicas}
Provides possibility to reinitialize Zookeeper sessions state for all `ReplicatedMergeTree` tables, will compare current state with Zookeeper as source of true and add tasks to Zookeeper queue if needed

File diff suppressed because one or more lines are too long

View File

@ -16,4 +16,5 @@ toc_title: "Введение"
- [AMPLab Big Data Benchmark](amplab-benchmark.md)
- [Данные о такси в Нью-Йорке](nyc-taxi.md)
- [OnTime](ontime.md)
- [Вышки сотовой связи](../../getting-started/example-datasets/cell-towers.md)

View File

@ -291,7 +291,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part
## interserver_http_host {#interserver-http-host}
Имя хоста, которое могут использовать другие серверы для обращения к этому.
Имя хоста, которое могут использовать другие серверы для обращения к этому хосту.
Если не указано, то определяется аналогично команде `hostname -f`.
@ -303,11 +303,36 @@ ClickHouse проверяет условия для `min_part_size` и `min_part
<interserver_http_host>example.yandex.ru</interserver_http_host>
```
## interserver_https_port {#interserver-https-port}
Порт для обмена данными между репликами ClickHouse по протоколу `HTTPS`.
**Пример**
``` xml
<interserver_https_port>9010</interserver_https_port>
```
## interserver_https_host {#interserver-https-host}
Имя хоста, которое могут использовать другие реплики для обращения к нему по протоколу `HTTPS`.
**Пример**
``` xml
<interserver_https_host>example.yandex.ru</interserver_https_host>
```
## interserver_http_credentials {#server-settings-interserver-http-credentials}
Имя пользователя и пароль, использующиеся для аутентификации при [репликации](../../operations/server-configuration-parameters/settings.md) движками Replicated\*. Это имя пользователя и пароль используются только для взаимодействия между репликами кластера и никак не связаны с аутентификацией клиентов ClickHouse. Сервер проверяет совпадение имени и пароля для соединяющихся с ним реплик, а также использует это же имя и пароль для соединения с другими репликами. Соответственно, эти имя и пароль должны быть прописаны одинаковыми для всех реплик кластера.
По умолчанию аутентификация не используется.
!!! note "Примечание"
Эти учетные данные являются общими для обмена данными по протоколам `HTTP` и `HTTPS`.
Раздел содержит следующие параметры:
- `user` — имя пользователя.

View File

@ -3,7 +3,7 @@ toc_priority: 47
toc_title: "Обновление ClickHouse"
---
# Обновление ClickHouse {#obnovlenie-clickhouse}
# Обновление ClickHouse {#clickhouse-upgrade}
Если ClickHouse установлен с помощью deb-пакетов, выполните следующие команды на сервере:
@ -15,4 +15,19 @@ $ sudo service clickhouse-server restart
Если ClickHouse установлен не из рекомендуемых deb-пакетов, используйте соответствующий метод обновления.
ClickHouse не поддерживает распределенное обновление. Операция должна выполняться последовательно на каждом отдельном сервере. Не обновляйте все серверы в кластере одновременно, иначе кластер становится недоступным в течение некоторого времени.
!!! note "Примечание"
Вы можете обновить сразу несколько серверов, кроме случая, когда все реплики одного шарда отключены.
Обновление ClickHouse до определенной версии:
**Пример**
`xx.yy.a.b` — это номер текущей стабильной версии. Последнюю стабильную версию можно узнать [здесь](https://github.com/ClickHouse/ClickHouse/releases)
```bash
$ sudo apt-get update
$ sudo apt-get install clickhouse-server=xx.yy.a.b clickhouse-client=xx.yy.a.b clickhouse-common-static=xx.yy.a.b
$ sudo service clickhouse-server restart
```
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/update/) <!--hide-->

View File

@ -7,9 +7,9 @@ toc_title: DateTime64
Позволяет хранить момент времени, который может быть представлен как календарная дата и время, с заданной суб-секундной точностью.
Размер тика/точность: 10<sup>-precision</sup> секунд, где precision - целочисленный параметр типа.
Размер тика (точность, precision): 10<sup>-precision</sup> секунд, где precision - целочисленный параметр.
Синтаксис:
**Синтаксис:**
``` sql
DateTime64(precision, [timezone])
@ -17,9 +17,11 @@ DateTime64(precision, [timezone])
Данные хранятся в виде количества ‘тиков’, прошедших с момента начала эпохи (1970-01-01 00:00:00 UTC), в Int64. Размер тика определяется параметром precision. Дополнительно, тип `DateTime64` позволяет хранить часовой пояс, единый для всей колонки, который влияет на то, как будут отображаться значения типа `DateTime64` в текстовом виде и как будут парситься значения заданные в виде строк (2020-01-01 05:00:01.000). Часовой пояс не хранится в строках таблицы (выборки), а хранится в метаданных колонки. Подробнее см. [DateTime](datetime.md).
## Пример {#primer}
Поддерживаются значения от 1 января 1925 г. и до 31 декабря 2283 г.
**1.** Создание таблицы с столбцом типа `DateTime64` и вставка данных в неё:
## Примеры {#examples}
1. Создание таблицы со столбцом типа `DateTime64` и вставка данных в неё:
``` sql
CREATE TABLE dt
@ -27,15 +29,15 @@ CREATE TABLE dt
`timestamp` DateTime64(3, 'Europe/Moscow'),
`event_id` UInt8
)
ENGINE = TinyLog
ENGINE = TinyLog;
```
``` sql
INSERT INTO dt Values (1546300800000, 1), ('2019-01-01 00:00:00', 2)
INSERT INTO dt Values (1546300800000, 1), ('2019-01-01 00:00:00', 2);
```
``` sql
SELECT * FROM dt
SELECT * FROM dt;
```
``` text
@ -46,12 +48,12 @@ SELECT * FROM dt
```
- При вставке даты-времени как числа (аналогично Unix timestamp), время трактуется как UTC. Unix timestamp `1546300800` в часовом поясе `Europe/London (UTC+0)` представляет время `'2019-01-01 00:00:00'`. Однако, столбец `timestamp` имеет тип `DateTime('Europe/Moscow (UTC+3)')`, так что при выводе в виде строки время отобразится как `2019-01-01 03:00:00`.
- При вставке даты-времени в виде строки, время трактуется соответственно часовому поясу установленному для колонки. `'2019-01-01 00:00:00'` трактуется как время по Москве (и в базу сохраняется `'2018-12-31 21:00:00'` в виде Unix Timestamp)
- При вставке даты-времени в виде строки, время трактуется соответственно часовому поясу установленному для колонки. `'2019-01-01 00:00:00'` трактуется как время по Москве (и в базу сохраняется `'2018-12-31 21:00:00'` в виде Unix Timestamp).
**2.** Фильтрация по значениям даты-времени
2. Фильтрация по значениям даты и времени
``` sql
SELECT * FROM dt WHERE timestamp = toDateTime64('2019-01-01 00:00:00', 3, 'Europe/Moscow')
SELECT * FROM dt WHERE timestamp = toDateTime64('2019-01-01 00:00:00', 3, 'Europe/Moscow');
```
``` text
@ -60,12 +62,12 @@ SELECT * FROM dt WHERE timestamp = toDateTime64('2019-01-01 00:00:00', 3, 'Europ
└─────────────────────────┴──────────┘
```
В отличие от типа `DateTime`, `DateTime64` не конвертируется из строк автоматически
В отличие от типа `DateTime`, `DateTime64` не конвертируется из строк автоматически.
**3.** Получение часового пояса для значения типа `DateTime64`:
3. Получение часового пояса для значения типа `DateTime64`:
``` sql
SELECT toDateTime64(now(), 3, 'Europe/Moscow') AS column, toTypeName(column) AS x
SELECT toDateTime64(now(), 3, 'Europe/Moscow') AS column, toTypeName(column) AS x;
```
``` text
@ -74,13 +76,13 @@ SELECT toDateTime64(now(), 3, 'Europe/Moscow') AS column, toTypeName(column) AS
└─────────────────────────┴────────────────────────────────┘
```
**4.** Конвертация часовых поясов
4. Конвертация часовых поясов
``` sql
SELECT
toDateTime64(timestamp, 3, 'Europe/London') as lon_time,
toDateTime64(timestamp, 3, 'Europe/Moscow') as mos_time
FROM dt
FROM dt;
```
``` text
@ -90,7 +92,7 @@ FROM dt
└─────────────────────────┴─────────────────────────┘
```
## See Also {#see-also}
**See Also**
- [Функции преобразования типов](../../sql-reference/functions/type-conversion-functions.md)
- [Функции для работы с датой и временем](../../sql-reference/functions/date-time-functions.md)

View File

@ -3,7 +3,7 @@ toc_priority: 44
toc_title: "Ключ и поля словаря"
---
# Ключ и поля словаря {#kliuch-i-polia-slovaria}
# Ключ и поля словаря {#dictionary-key-and-fields}
Секция `<structure>` описывает ключ словаря и поля, доступные для запросов.
@ -88,7 +88,7 @@ PRIMARY KEY Id
- `PRIMARY KEY` имя столбца с ключами.
### Составной ключ {#sostavnoi-kliuch}
### Составной ключ {#composite-key}
Ключом может быть кортеж (`tuple`) из полей произвольных типов. В этом случае [layout](external-dicts-dict-layout.md) должен быть `complex_key_hashed` или `complex_key_cache`.
@ -159,13 +159,12 @@ CREATE DICTIONARY somename (
| Тег | Описание | Обязательный |
|------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------|
| `name` | Имя столбца. | Да |
| `type` | Тип данных ClickHouse.<br/>ClickHouse пытается привести значение из словаря к заданному типу данных. Например, в случае MySQL, в таблице-источнике поле может быть `TEXT`, `VARCHAR`, `BLOB`, но загружено может быть как `String`. [Nullable](../../../sql-reference/data-types/nullable.md) не поддерживается. | Да |
| `null_value` | Значение по умолчанию для несуществующего элемента.<br/>В примере это пустая строка. Нельзя указать значение `NULL`. | Да |
| `type` | Тип данных ClickHouse.<br/>ClickHouse пытается привести значение из словаря к заданному типу данных. Например, в случае MySQL, в таблице-источнике поле может быть `TEXT`, `VARCHAR`, `BLOB`, но загружено может быть как `String`. <br/>[Nullable](../../../sql-reference/data-types/nullable.md) в настоящее время поддерживается для словарей [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md). Для словарей [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache) и [IPTrie](external-dicts-dict-layout.md#ip-trie) `Nullable`-типы не поддерживаются. | Да |
| `null_value` | Значение по умолчанию для несуществующего элемента.<br/>В примере это пустая строка. Значение [NULL](../../syntax.md#null-literal) можно указывать только для типов `Nullable` (см. предыдущую строку с описанием типов). | Да |
| `expression` | [Выражение](../../syntax.md#syntax-expressions), которое ClickHouse выполняет со значением.<br/>Выражением может быть имя столбца в удаленной SQL базе. Таким образом, вы можете использовать его для создания псевдонима удаленного столбца.<br/><br/>Значение по умолчанию: нет выражения. | Нет |
| <a name="hierarchical-dict-attr"></a> `hierarchical` | Если `true`, то атрибут содержит ключ предка для текущего элемента. Смотрите [Иерархические словари](external-dicts-dict-hierarchical.md).<br/><br/>Default value: `false`. | No |
| <a name="hierarchical-dict-attr"></a> `hierarchical` | Если `true`, то атрибут содержит ключ предка для текущего элемента. Смотрите [Иерархические словари](external-dicts-dict-hierarchical.md).<br/><br/>Значение по умолчанию: `false`. | Нет |
| `is_object_id` | Признак того, что запрос выполняется к документу MongoDB по `ObjectID`.<br/><br/>Значение по умолчанию: `false`. | Нет |
## Смотрите также {#smotrite-takzhe}
**Смотрите также**
- [Функции для работы с внешними словарями](../../../sql-reference/functions/ext-dict-functions.md).

View File

@ -10,8 +10,6 @@ toc_title: "Введение"
ClickHouse поддерживает специальные функции для работы со словарями, которые можно использовать в запросах. Проще и эффективнее использовать словари с помощью функций, чем `JOIN` с таблицами-справочниками.
В словаре нельзя хранить значения [NULL](../../sql-reference/syntax.md#null-literal).
ClickHouse поддерживает:
- [Встроенные словари](internal-dicts.md#internal_dicts) со специфическим [набором функций](../../sql-reference/dictionaries/external-dictionaries/index.md).

View File

@ -82,4 +82,4 @@ SELECT * FROM table_with_ttl;
### Смотрите также
- Подробнее о [свойстве TTL](../../../engines/table-engines/mergetree-family/mergetree.md#mergetree-column-ttl).
- Изменить столбец [с TTL](../../../sql-reference/statements/alter/column.md#alter_modify-column).

View File

@ -21,7 +21,7 @@
#include <unordered_set>
#include <algorithm>
#include <optional>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
#include <boost/program_options.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <Poco/String.h>
@ -1610,7 +1610,7 @@ private:
{
/// Temporarily apply query settings to context.
std::optional<Settings> old_settings;
SCOPE_EXIT({ if (old_settings) context.setSettings(*old_settings); });
SCOPE_EXIT_SAFE({ if (old_settings) context.setSettings(*old_settings); });
auto apply_query_settings = [&](const IAST & settings_ast)
{
if (!old_settings)

View File

@ -37,34 +37,33 @@ namespace ErrorCodes
Field QueryFuzzer::getRandomField(int type)
{
static constexpr Int64 bad_int64_values[]
= {-2, -1, 0, 1, 2, 3, 7, 10, 100, 255, 256, 257, 1023, 1024,
1025, 65535, 65536, 65537, 1024 * 1024 - 1, 1024 * 1024,
1024 * 1024 + 1, INT_MIN - 1ll, INT_MIN, INT_MIN + 1,
INT_MAX - 1, INT_MAX, INT_MAX + 1ll, INT64_MIN, INT64_MIN + 1,
INT64_MAX - 1, INT64_MAX};
switch (type)
{
case 0:
{
static constexpr Int64 values[]
= {-2, -1, 0, 1, 2, 3, 7, 10, 100, 255, 256, 257, 1023, 1024,
1025, 65535, 65536, 65537, 1024 * 1024 - 1, 1024 * 1024,
1024 * 1024 + 1, INT64_MIN, INT64_MAX};
return values[fuzz_rand() % (sizeof(values) / sizeof(*values))];
return bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values)
/ sizeof(*bad_int64_values))];
}
case 1:
{
static constexpr float values[]
= {NAN, INFINITY, -INFINITY, 0., 0.0001, 0.5, 0.9999,
1., 1.0001, 2., 10.0001, 100.0001, 1000.0001};
return values[fuzz_rand() % (sizeof(values) / sizeof(*values))];
= {NAN, INFINITY, -INFINITY, 0., -0., 0.0001, 0.5, 0.9999,
1., 1.0001, 2., 10.0001, 100.0001, 1000.0001, 1e10, 1e20,
FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % (sizeof(values) / sizeof(*values))];
}
case 2:
{
static constexpr Int64 values[]
= {-2, -1, 0, 1, 2, 3, 7, 10, 100, 255, 256, 257, 1023, 1024,
1025, 65535, 65536, 65537, 1024 * 1024 - 1, 1024 * 1024,
1024 * 1024 + 1, INT64_MIN, INT64_MAX};
static constexpr UInt64 scales[] = {0, 1, 2, 10};
return DecimalField<Decimal64>(
values[fuzz_rand() % (sizeof(values) / sizeof(*values))],
scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))]
);
bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values)
/ sizeof(*bad_int64_values))],
scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))]);
}
default:
assert(false);

View File

@ -3,6 +3,7 @@
#include <Common/TerminalSize.h>
#include <IO/ConnectionTimeoutsContext.h>
#include <Formats/registerFormats.h>
#include <ext/scope_guard_safe.h>
#include <unistd.h>
@ -112,7 +113,7 @@ void ClusterCopierApp::mainImpl()
SharedContextHolder shared_context = Context::createShared();
auto context = std::make_unique<Context>(Context::createGlobal(shared_context.get()));
context->makeGlobalContext();
SCOPE_EXIT(context->shutdown());
SCOPE_EXIT_SAFE(context->shutdown());
context->setConfig(loaded_config.configuration);
context->setApplicationType(Context::ApplicationType::LOCAL);

View File

@ -47,6 +47,7 @@
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Interpreters/InterserverCredentials.h>
#include <Interpreters/ExpressionJIT.h>
#include <Access/AccessControlManager.h>
#include <Storages/StorageReplicatedMergeTree.h>
@ -688,16 +689,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
}
if (config().has("interserver_http_credentials"))
{
String user = config().getString("interserver_http_credentials.user", "");
String password = config().getString("interserver_http_credentials.password", "");
if (user.empty())
throw Exception("Configuration parameter interserver_http_credentials user can't be empty", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
global_context->setInterserverCredentials(user, password);
}
LOG_DEBUG(log, "Initiailizing interserver credentials.");
global_context->updateInterserverCredentials(config());
if (config().has("macros"))
global_context->setMacros(std::make_unique<Macros>(config(), "macros", log));
@ -777,6 +770,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
global_context->updateStorageConfiguration(*config);
global_context->updateInterserverCredentials(*config);
},
/* already_loaded = */ false); /// Reload it right now (initial loading)

View File

@ -174,6 +174,8 @@ public:
return "mannWhitneyUTest";
}
bool allocatesMemoryInArena() const override { return true; }
DataTypePtr getReturnType() const override
{
DataTypes types
@ -208,7 +210,7 @@ public:
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & a = this->data(place);
auto & b = this->data(rhs);
const auto & b = this->data(rhs);
a.merge(b, arena);
}

View File

@ -58,6 +58,8 @@ public:
return "rankCorr";
}
bool allocatesMemoryInArena() const override { return true; }
DataTypePtr getReturnType() const override
{
return std::make_shared<DataTypeNumber<Float64>>();

View File

@ -128,7 +128,7 @@ private:
template <size_t> friend class AlignedArenaAllocator;
public:
Arena(size_t initial_size_ = 4096, size_t growth_factor_ = 2, size_t linear_growth_threshold_ = 128 * 1024 * 1024)
explicit Arena(size_t initial_size_ = 4096, size_t growth_factor_ = 2, size_t linear_growth_threshold_ = 128 * 1024 * 1024)
: growth_factor(growth_factor_), linear_growth_threshold(linear_growth_threshold_),
head(new MemoryChunk(initial_size_, nullptr)), size_in_bytes(head->size()),
page_size(static_cast<size_t>(::getPageSize()))
@ -160,7 +160,7 @@ public:
void * head_pos = head->pos;
size_t space = head->end - head->pos;
auto res = static_cast<char *>(std::align(alignment, size, head_pos, space));
auto * res = static_cast<char *>(std::align(alignment, size, head_pos, space));
if (res)
{
head->pos = static_cast<char *>(head_pos);

View File

@ -24,17 +24,26 @@ struct PairNoInit
PairNoInit() {}
template <typename First_>
PairNoInit(First_ && first_, NoInitTag) : first(std::forward<First_>(first_))
template <typename FirstValue>
PairNoInit(FirstValue && first_, NoInitTag)
: first(std::forward<FirstValue>(first_))
{
}
template <typename First_, typename Second_>
PairNoInit(First_ && first_, Second_ && second_) : first(std::forward<First_>(first_)), second(std::forward<Second_>(second_))
template <typename FirstValue, typename SecondValue>
PairNoInit(FirstValue && first_, SecondValue && second_)
: first(std::forward<FirstValue>(first_))
, second(std::forward<SecondValue>(second_))
{
}
};
template <typename First, typename Second>
PairNoInit<std::decay_t<First>, std::decay_t<Second>> makePairNoInit(First && first, Second && second)
{
return PairNoInit<std::decay_t<First>, std::decay_t<Second>>(std::forward<First>(first), std::forward<Second>(second));
}
template <typename Key, typename TMapped, typename Hash, typename TState = HashTableNoState>
struct HashMapCell

View File

@ -146,6 +146,8 @@
M(StorageBufferPassedTimeMaxThreshold, "") \
M(StorageBufferPassedRowsMaxThreshold, "") \
M(StorageBufferPassedBytesMaxThreshold, "") \
M(StorageBufferLayerLockReadersWaitMilliseconds, "Time for waiting for Buffer layer during reading") \
M(StorageBufferLayerLockWritersWaitMilliseconds, "Time for waiting free Buffer layer to write to (can be used to tune Buffer layers)") \
\
M(DictCacheKeysRequested, "") \
M(DictCacheKeysRequestedMiss, "") \

View File

@ -243,6 +243,7 @@ Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings
auto callback = [&](const Coordination::ListResponse & response)
{
SCOPE_EXIT(event.set());
code = response.error;
if (code == Coordination::Error::ZOK)
{
@ -250,7 +251,6 @@ Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings
if (stat)
*stat = response.stat;
}
event.set();
};
impl->list(path, callback, watch_callback);
@ -303,10 +303,10 @@ Coordination::Error ZooKeeper::createImpl(const std::string & path, const std::s
auto callback = [&](const Coordination::CreateResponse & response)
{
SCOPE_EXIT(event.set());
code = response.error;
if (code == Coordination::Error::ZOK)
path_created = response.path_created;
event.set();
};
impl->create(path, data, mode & 1, mode & 2, {}, callback); /// TODO better mode
@ -371,9 +371,9 @@ Coordination::Error ZooKeeper::removeImpl(const std::string & path, int32_t vers
auto callback = [&](const Coordination::RemoveResponse & response)
{
SCOPE_EXIT(event.set());
if (response.error != Coordination::Error::ZOK)
code = response.error;
event.set();
};
impl->remove(path, version, callback);
@ -404,10 +404,10 @@ Coordination::Error ZooKeeper::existsImpl(const std::string & path, Coordination
auto callback = [&](const Coordination::ExistsResponse & response)
{
SCOPE_EXIT(event.set());
code = response.error;
if (code == Coordination::Error::ZOK && stat)
*stat = response.stat;
event.set();
};
impl->exists(path, callback, watch_callback);
@ -436,6 +436,7 @@ Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & r
auto callback = [&](const Coordination::GetResponse & response)
{
SCOPE_EXIT(event.set());
code = response.error;
if (code == Coordination::Error::ZOK)
{
@ -443,7 +444,6 @@ Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & r
if (stat)
*stat = response.stat;
}
event.set();
};
impl->get(path, callback, watch_callback);
@ -508,10 +508,10 @@ Coordination::Error ZooKeeper::setImpl(const std::string & path, const std::stri
auto callback = [&](const Coordination::SetResponse & response)
{
SCOPE_EXIT(event.set());
code = response.error;
if (code == Coordination::Error::ZOK && stat)
*stat = response.stat;
event.set();
};
impl->set(path, data, version, callback);
@ -558,9 +558,9 @@ Coordination::Error ZooKeeper::multiImpl(const Coordination::Requests & requests
auto callback = [&](const Coordination::MultiResponse & response)
{
SCOPE_EXIT(event.set());
code = response.error;
responses = response.responses;
event.set();
};
impl->multi(requests, callback);

View File

@ -796,8 +796,17 @@ void ZooKeeper::receiveEvent()
/// In case we cannot read the response, we should indicate it as the error of that type
/// when the user cannot assume whether the request was processed or not.
response->error = Error::ZCONNECTIONLOSS;
if (request_info.callback)
request_info.callback(*response);
try
{
if (request_info.callback)
request_info.callback(*response);
}
catch (...)
{
/// Throw initial exception, not exception from callback.
tryLogCurrentException(__PRETTY_FUNCTION__);
}
throw;
}

View File

@ -44,8 +44,8 @@ ChangelogFileDescription getChangelogFileDescription(const std::string & path_st
ChangelogFileDescription result;
result.prefix = filename_parts[0];
result.from_log_index = parse<size_t>(filename_parts[1]);
result.to_log_index = parse<size_t>(filename_parts[2]);
result.from_log_index = parse<uint64_t>(filename_parts[1]);
result.to_log_index = parse<uint64_t>(filename_parts[2]);
result.path = path_str;
return result;
}
@ -73,7 +73,7 @@ Checksum computeRecordChecksum(const ChangelogRecord & record)
class ChangelogWriter
{
public:
ChangelogWriter(const std::string & filepath_, WriteMode mode, size_t start_index_)
ChangelogWriter(const std::string & filepath_, WriteMode mode, uint64_t start_index_)
: filepath(filepath_)
, plain_buf(filepath, DBMS_DEFAULT_BUFFER_SIZE, mode == WriteMode::Rewrite ? -1 : (O_APPEND | O_CREAT | O_WRONLY))
, start_index(start_index_)
@ -115,22 +115,22 @@ public:
plain_buf.sync();
}
size_t getEntriesWritten() const
uint64_t getEntriesWritten() const
{
return entries_written;
}
void setEntriesWritten(size_t entries_written_)
void setEntriesWritten(uint64_t entries_written_)
{
entries_written = entries_written_;
}
size_t getStartIndex() const
uint64_t getStartIndex() const
{
return start_index;
}
void setStartIndex(size_t start_index_)
void setStartIndex(uint64_t start_index_)
{
start_index = start_index_;
}
@ -138,14 +138,14 @@ public:
private:
std::string filepath;
WriteBufferFromFile plain_buf;
size_t entries_written = 0;
size_t start_index;
uint64_t entries_written = 0;
uint64_t start_index;
};
struct ChangelogReadResult
{
size_t entries_read;
size_t first_read_index;
uint64_t entries_read;
uint64_t first_read_index;
off_t last_position;
bool error;
};
@ -158,9 +158,9 @@ public:
, read_buf(filepath)
{}
ChangelogReadResult readChangelog(IndexToLogEntry & logs, size_t start_log_index, IndexToOffset & index_to_offset, Poco::Logger * log)
ChangelogReadResult readChangelog(IndexToLogEntry & logs, uint64_t start_log_index, IndexToOffset & index_to_offset, Poco::Logger * log)
{
size_t previous_index = 0;
uint64_t previous_index = 0;
ChangelogReadResult result{};
try
{
@ -247,7 +247,7 @@ private:
ReadBufferFromFile read_buf;
};
Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval_, Poco::Logger * log_)
Changelog::Changelog(const std::string & changelogs_dir_, uint64_t rotate_interval_, Poco::Logger * log_)
: changelogs_dir(changelogs_dir_)
, rotate_interval(rotate_interval_)
, log(log_)
@ -263,15 +263,15 @@ Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval
}
}
void Changelog::readChangelogAndInitWriter(size_t last_commited_log_index, size_t logs_to_keep)
void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep)
{
size_t total_read = 0;
size_t entries_in_last = 0;
size_t incomplete_log_index = 0;
uint64_t total_read = 0;
uint64_t entries_in_last = 0;
uint64_t incomplete_log_index = 0;
ChangelogReadResult result{};
size_t first_read_index = 0;
uint64_t first_read_index = 0;
size_t start_to_read_from = last_commited_log_index;
uint64_t start_to_read_from = last_commited_log_index;
if (start_to_read_from > logs_to_keep)
start_to_read_from -= logs_to_keep;
else
@ -355,12 +355,8 @@ void Changelog::readChangelogAndInitWriter(size_t last_commited_log_index, size_
rotate(start_index + total_read);
}
void Changelog::rotate(size_t new_start_log_index)
void Changelog::rotate(uint64_t new_start_log_index)
{
//// doesn't exist on init
if (current_writer)
current_writer->flush();
ChangelogFileDescription new_description;
new_description.prefix = DEFAULT_PREFIX;
new_description.from_log_index = new_start_log_index;
@ -373,7 +369,7 @@ void Changelog::rotate(size_t new_start_log_index)
current_writer = std::make_unique<ChangelogWriter>(new_description.path, WriteMode::Rewrite, new_start_log_index);
}
ChangelogRecord Changelog::buildRecord(size_t index, const LogEntryPtr & log_entry)
ChangelogRecord Changelog::buildRecord(uint64_t index, const LogEntryPtr & log_entry)
{
ChangelogRecord record;
record.header.version = ChangelogVersion::V0;
@ -391,7 +387,7 @@ ChangelogRecord Changelog::buildRecord(size_t index, const LogEntryPtr & log_ent
return record;
}
void Changelog::appendEntry(size_t index, const LogEntryPtr & log_entry, bool force_sync)
void Changelog::appendEntry(uint64_t index, const LogEntryPtr & log_entry, bool force_sync)
{
if (!current_writer)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before appending records");
@ -409,7 +405,7 @@ void Changelog::appendEntry(size_t index, const LogEntryPtr & log_entry, bool fo
logs[index] = makeClone(log_entry);
}
void Changelog::writeAt(size_t index, const LogEntryPtr & log_entry, bool force_sync)
void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry, bool force_sync)
{
if (index_to_start_pos.count(index) == 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot write at index {} because changelog doesn't contain it", index);
@ -443,7 +439,7 @@ void Changelog::writeAt(size_t index, const LogEntryPtr & log_entry, bool force_
}
/// Remove redundant logs from memory
for (size_t i = index; ; ++i)
for (uint64_t i = index; ; ++i)
{
auto log_itr = logs.find(i);
if (log_itr == logs.end())
@ -458,7 +454,7 @@ void Changelog::writeAt(size_t index, const LogEntryPtr & log_entry, bool force_
appendEntry(index, log_entry, force_sync);
}
void Changelog::compact(size_t up_to_log_index)
void Changelog::compact(uint64_t up_to_log_index)
{
for (auto itr = existing_changelogs.begin(); itr != existing_changelogs.end();)
{
@ -480,9 +476,9 @@ void Changelog::compact(size_t up_to_log_index)
LogEntryPtr Changelog::getLastEntry() const
{
static LogEntryPtr fake_entry = nuraft::cs_new<nuraft::log_entry>(0, nuraft::buffer::alloc(sizeof(size_t)));
static LogEntryPtr fake_entry = nuraft::cs_new<nuraft::log_entry>(0, nuraft::buffer::alloc(sizeof(uint64_t)));
size_t next_index = getNextEntryIndex() - 1;
uint64_t next_index = getNextEntryIndex() - 1;
auto entry = logs.find(next_index);
if (entry == logs.end())
return fake_entry;
@ -490,13 +486,13 @@ LogEntryPtr Changelog::getLastEntry() const
return entry->second;
}
LogEntriesPtr Changelog::getLogEntriesBetween(size_t start, size_t end)
LogEntriesPtr Changelog::getLogEntriesBetween(uint64_t start, uint64_t end)
{
LogEntriesPtr ret = nuraft::cs_new<std::vector<nuraft::ptr<nuraft::log_entry>>>();
ret->resize(end - start);
size_t result_pos = 0;
for (size_t i = start; i < end; ++i)
uint64_t result_pos = 0;
for (uint64_t i = start; i < end; ++i)
{
(*ret)[result_pos] = entryAt(i);
result_pos++;
@ -504,7 +500,7 @@ LogEntriesPtr Changelog::getLogEntriesBetween(size_t start, size_t end)
return ret;
}
LogEntryPtr Changelog::entryAt(size_t index)
LogEntryPtr Changelog::entryAt(uint64_t index)
{
nuraft::ptr<nuraft::log_entry> src = nullptr;
auto entry = logs.find(index);
@ -515,12 +511,12 @@ LogEntryPtr Changelog::entryAt(size_t index)
return src;
}
nuraft::ptr<nuraft::buffer> Changelog::serializeEntriesToBuffer(size_t index, int32_t count)
nuraft::ptr<nuraft::buffer> Changelog::serializeEntriesToBuffer(uint64_t index, int32_t count)
{
std::vector<nuraft::ptr<nuraft::buffer>> returned_logs;
size_t size_total = 0;
for (size_t i = index; i < index + count; ++i)
uint64_t size_total = 0;
for (uint64_t i = index; i < index + count; ++i)
{
auto entry = logs.find(i);
if (entry == logs.end())
@ -544,14 +540,14 @@ nuraft::ptr<nuraft::buffer> Changelog::serializeEntriesToBuffer(size_t index, in
return buf_out;
}
void Changelog::applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer, bool force_sync)
void Changelog::applyEntriesFromBuffer(uint64_t index, nuraft::buffer & buffer, bool force_sync)
{
buffer.pos(0);
int num_logs = buffer.get_int();
for (int i = 0; i < num_logs; ++i)
{
size_t cur_index = index + i;
uint64_t cur_index = index + i;
int buf_size = buffer.get_int();
nuraft::ptr<nuraft::buffer> buf_local = nuraft::buffer::alloc(buf_size);

View File

@ -17,8 +17,8 @@ using LogEntries = std::vector<LogEntryPtr>;
using LogEntriesPtr = nuraft::ptr<LogEntries>;
using BufferPtr = nuraft::ptr<nuraft::buffer>;
using IndexToOffset = std::unordered_map<size_t, off_t>;
using IndexToLogEntry = std::unordered_map<size_t, LogEntryPtr>;
using IndexToOffset = std::unordered_map<uint64_t, off_t>;
using IndexToLogEntry = std::unordered_map<uint64_t, LogEntryPtr>;
enum class ChangelogVersion : uint8_t
{
@ -30,10 +30,10 @@ static constexpr auto CURRENT_CHANGELOG_VERSION = ChangelogVersion::V0;
struct ChangelogRecordHeader
{
ChangelogVersion version = CURRENT_CHANGELOG_VERSION;
size_t index; /// entry log number
size_t term;
uint64_t index; /// entry log number
uint64_t term;
nuraft::log_val_type value_type;
size_t blob_size;
uint64_t blob_size;
};
/// Changelog record on disk
@ -48,8 +48,8 @@ struct ChangelogRecord
struct ChangelogFileDescription
{
std::string prefix;
size_t from_log_index;
size_t to_log_index;
uint64_t from_log_index;
uint64_t to_log_index;
std::string path;
};
@ -63,27 +63,27 @@ class Changelog
{
public:
Changelog(const std::string & changelogs_dir_, size_t rotate_interval_, Poco::Logger * log_);
Changelog(const std::string & changelogs_dir_, uint64_t rotate_interval_, Poco::Logger * log_);
/// Read changelog from files on changelogs_dir_ skipping all entries before from_log_index
/// Truncate broken entries, remove files after broken entries.
void readChangelogAndInitWriter(size_t last_commited_log_index, size_t logs_to_keep);
void readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep);
/// Add entry to log with index. Call fsync if force_sync true.
void appendEntry(size_t index, const LogEntryPtr & log_entry, bool force_sync);
void appendEntry(uint64_t index, const LogEntryPtr & log_entry, bool force_sync);
/// Write entry at index and truncate all subsequent entries.
void writeAt(size_t index, const LogEntryPtr & log_entry, bool force_sync);
void writeAt(uint64_t index, const LogEntryPtr & log_entry, bool force_sync);
/// Remove log files with to_log_index <= up_to_log_index.
void compact(size_t up_to_log_index);
void compact(uint64_t up_to_log_index);
size_t getNextEntryIndex() const
uint64_t getNextEntryIndex() const
{
return start_index + logs.size();
}
size_t getStartIndex() const
uint64_t getStartIndex() const
{
return start_index;
}
@ -92,21 +92,21 @@ public:
LogEntryPtr getLastEntry() const;
/// Return log entries between [start, end)
LogEntriesPtr getLogEntriesBetween(size_t start_index, size_t end_index);
LogEntriesPtr getLogEntriesBetween(uint64_t start_index, uint64_t end_index);
/// Return entry at position index
LogEntryPtr entryAt(size_t index);
LogEntryPtr entryAt(uint64_t index);
/// Serialize entries from index into buffer
BufferPtr serializeEntriesToBuffer(size_t index, int32_t count);
BufferPtr serializeEntriesToBuffer(uint64_t index, int32_t count);
/// Apply entries from buffer overriding existing entries
void applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer, bool force_sync);
void applyEntriesFromBuffer(uint64_t index, nuraft::buffer & buffer, bool force_sync);
/// Fsync log to disk
void flush();
size_t size() const
uint64_t size() const
{
return logs.size();
}
@ -116,21 +116,21 @@ public:
private:
/// Pack log_entry into changelog record
static ChangelogRecord buildRecord(size_t index, const LogEntryPtr & log_entry);
static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry);
/// Starts new file [new_start_log_index, new_start_log_index + rotate_interval]
void rotate(size_t new_start_log_index);
void rotate(uint64_t new_start_log_index);
private:
const std::string changelogs_dir;
const size_t rotate_interval;
const uint64_t rotate_interval;
Poco::Logger * log;
std::map<size_t, ChangelogFileDescription> existing_changelogs;
std::map<uint64_t, ChangelogFileDescription> existing_changelogs;
std::unique_ptr<ChangelogWriter> current_writer;
IndexToOffset index_to_start_pos;
IndexToLogEntry logs;
size_t start_index = 0;
uint64_t start_index = 0;
};
}

View File

@ -16,16 +16,16 @@ ptr<log_entry> makeClone(const ptr<log_entry> & entry)
InMemoryLogStore::InMemoryLogStore()
: start_idx(1)
{
nuraft::ptr<nuraft::buffer> buf = nuraft::buffer::alloc(sizeof(size_t));
nuraft::ptr<nuraft::buffer> buf = nuraft::buffer::alloc(sizeof(uint64_t));
logs[0] = nuraft::cs_new<nuraft::log_entry>(0, buf);
}
size_t InMemoryLogStore::start_index() const
uint64_t InMemoryLogStore::start_index() const
{
return start_idx;
}
size_t InMemoryLogStore::next_slot() const
uint64_t InMemoryLogStore::next_slot() const
{
std::lock_guard<std::mutex> l(logs_lock);
// Exclude the dummy entry.
@ -34,7 +34,7 @@ size_t InMemoryLogStore::next_slot() const
nuraft::ptr<nuraft::log_entry> InMemoryLogStore::last_entry() const
{
size_t next_idx = next_slot();
uint64_t next_idx = next_slot();
std::lock_guard<std::mutex> lock(logs_lock);
auto entry = logs.find(next_idx - 1);
if (entry == logs.end())
@ -43,17 +43,17 @@ nuraft::ptr<nuraft::log_entry> InMemoryLogStore::last_entry() const
return makeClone(entry->second);
}
size_t InMemoryLogStore::append(nuraft::ptr<nuraft::log_entry> & entry)
uint64_t InMemoryLogStore::append(nuraft::ptr<nuraft::log_entry> & entry)
{
ptr<log_entry> clone = makeClone(entry);
std::lock_guard<std::mutex> l(logs_lock);
size_t idx = start_idx + logs.size() - 1;
uint64_t idx = start_idx + logs.size() - 1;
logs[idx] = clone;
return idx;
}
void InMemoryLogStore::write_at(size_t index, nuraft::ptr<nuraft::log_entry> & entry)
void InMemoryLogStore::write_at(uint64_t index, nuraft::ptr<nuraft::log_entry> & entry)
{
nuraft::ptr<log_entry> clone = makeClone(entry);
@ -65,14 +65,14 @@ void InMemoryLogStore::write_at(size_t index, nuraft::ptr<nuraft::log_entry> & e
logs[index] = clone;
}
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> InMemoryLogStore::log_entries(size_t start, size_t end)
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> InMemoryLogStore::log_entries(uint64_t start, uint64_t end)
{
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> ret =
nuraft::cs_new<std::vector<nuraft::ptr<nuraft::log_entry>>>();
ret->resize(end - start);
size_t cc = 0;
for (size_t i = start; i < end; ++i)
uint64_t cc = 0;
for (uint64_t i = start; i < end; ++i)
{
nuraft::ptr<nuraft::log_entry> src = nullptr;
{
@ -90,7 +90,7 @@ nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> InMemoryLogStore::log_e
return ret;
}
nuraft::ptr<nuraft::log_entry> InMemoryLogStore::entry_at(size_t index)
nuraft::ptr<nuraft::log_entry> InMemoryLogStore::entry_at(uint64_t index)
{
nuraft::ptr<nuraft::log_entry> src = nullptr;
{
@ -103,9 +103,9 @@ nuraft::ptr<nuraft::log_entry> InMemoryLogStore::entry_at(size_t index)
return makeClone(src);
}
size_t InMemoryLogStore::term_at(size_t index)
uint64_t InMemoryLogStore::term_at(uint64_t index)
{
size_t term = 0;
uint64_t term = 0;
{
std::lock_guard<std::mutex> l(logs_lock);
auto entry = logs.find(index);
@ -116,12 +116,12 @@ size_t InMemoryLogStore::term_at(size_t index)
return term;
}
nuraft::ptr<nuraft::buffer> InMemoryLogStore::pack(size_t index, Int32 cnt)
nuraft::ptr<nuraft::buffer> InMemoryLogStore::pack(uint64_t index, Int32 cnt)
{
std::vector<nuraft::ptr<nuraft::buffer>> returned_logs;
size_t size_total = 0;
for (size_t ii = index; ii < index + cnt; ++ii)
uint64_t uint64_total = 0;
for (uint64_t ii = index; ii < index + cnt; ++ii)
{
ptr<log_entry> le = nullptr;
{
@ -130,11 +130,11 @@ nuraft::ptr<nuraft::buffer> InMemoryLogStore::pack(size_t index, Int32 cnt)
}
assert(le.get());
nuraft::ptr<nuraft::buffer> buf = le->serialize();
size_total += buf->size();
uint64_total += buf->size();
returned_logs.push_back(buf);
}
nuraft::ptr<buffer> buf_out = nuraft::buffer::alloc(sizeof(int32) + cnt * sizeof(int32) + size_total);
nuraft::ptr<buffer> buf_out = nuraft::buffer::alloc(sizeof(int32) + cnt * sizeof(int32) + uint64_total);
buf_out->pos(0);
buf_out->put(static_cast<Int32>(cnt));
@ -147,14 +147,14 @@ nuraft::ptr<nuraft::buffer> InMemoryLogStore::pack(size_t index, Int32 cnt)
return buf_out;
}
void InMemoryLogStore::apply_pack(size_t index, nuraft::buffer & pack)
void InMemoryLogStore::apply_pack(uint64_t index, nuraft::buffer & pack)
{
pack.pos(0);
Int32 num_logs = pack.get_int();
for (Int32 i = 0; i < num_logs; ++i)
{
size_t cur_idx = index + i;
uint64_t cur_idx = index + i;
Int32 buf_size = pack.get_int();
nuraft::ptr<nuraft::buffer> buf_local = nuraft::buffer::alloc(buf_size);
@ -177,10 +177,10 @@ void InMemoryLogStore::apply_pack(size_t index, nuraft::buffer & pack)
}
}
bool InMemoryLogStore::compact(size_t last_log_index)
bool InMemoryLogStore::compact(uint64_t last_log_index)
{
std::lock_guard<std::mutex> l(logs_lock);
for (size_t ii = start_idx; ii <= last_log_index; ++ii)
for (uint64_t ii = start_idx; ii <= last_log_index; ++ii)
{
auto entry = logs.find(ii);
if (entry != logs.end())

View File

@ -14,34 +14,34 @@ class InMemoryLogStore : public nuraft::log_store
public:
InMemoryLogStore();
size_t start_index() const override;
uint64_t start_index() const override;
size_t next_slot() const override;
uint64_t next_slot() const override;
nuraft::ptr<nuraft::log_entry> last_entry() const override;
size_t append(nuraft::ptr<nuraft::log_entry> & entry) override;
uint64_t append(nuraft::ptr<nuraft::log_entry> & entry) override;
void write_at(size_t index, nuraft::ptr<nuraft::log_entry> & entry) override;
void write_at(uint64_t index, nuraft::ptr<nuraft::log_entry> & entry) override;
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> log_entries(size_t start, size_t end) override;
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> log_entries(uint64_t start, uint64_t end) override;
nuraft::ptr<nuraft::log_entry> entry_at(size_t index) override;
nuraft::ptr<nuraft::log_entry> entry_at(uint64_t index) override;
size_t term_at(size_t index) override;
uint64_t term_at(uint64_t index) override;
nuraft::ptr<nuraft::buffer> pack(size_t index, Int32 cnt) override;
nuraft::ptr<nuraft::buffer> pack(uint64_t index, Int32 cnt) override;
void apply_pack(size_t index, nuraft::buffer & pack) override;
void apply_pack(uint64_t index, nuraft::buffer & pack) override;
bool compact(size_t last_log_index) override;
bool compact(uint64_t last_log_index) override;
bool flush() override { return true; }
private:
std::map<size_t, nuraft::ptr<nuraft::log_entry>> logs;
std::map<uint64_t, nuraft::ptr<nuraft::log_entry>> logs;
mutable std::mutex logs_lock;
std::atomic<size_t> start_idx;
std::atomic<uint64_t> start_idx;
};
}

View File

@ -3,26 +3,26 @@
namespace DB
{
KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_, bool force_sync_)
KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_)
: log(&Poco::Logger::get("KeeperLogStore"))
, changelog(changelogs_path, rotate_interval_, log)
, force_sync(force_sync_)
{
}
size_t KeeperLogStore::start_index() const
uint64_t KeeperLogStore::start_index() const
{
std::lock_guard lock(changelog_lock);
return changelog.getStartIndex();
}
void KeeperLogStore::init(size_t last_commited_log_index, size_t logs_to_keep)
void KeeperLogStore::init(uint64_t last_commited_log_index, uint64_t logs_to_keep)
{
std::lock_guard lock(changelog_lock);
changelog.readChangelogAndInitWriter(last_commited_log_index, logs_to_keep);
}
size_t KeeperLogStore::next_slot() const
uint64_t KeeperLogStore::next_slot() const
{
std::lock_guard lock(changelog_lock);
return changelog.getNextEntryIndex();
@ -34,34 +34,34 @@ nuraft::ptr<nuraft::log_entry> KeeperLogStore::last_entry() const
return changelog.getLastEntry();
}
size_t KeeperLogStore::append(nuraft::ptr<nuraft::log_entry> & entry)
uint64_t KeeperLogStore::append(nuraft::ptr<nuraft::log_entry> & entry)
{
std::lock_guard lock(changelog_lock);
size_t idx = changelog.getNextEntryIndex();
uint64_t idx = changelog.getNextEntryIndex();
changelog.appendEntry(idx, entry, force_sync);
return idx;
}
void KeeperLogStore::write_at(size_t index, nuraft::ptr<nuraft::log_entry> & entry)
void KeeperLogStore::write_at(uint64_t index, nuraft::ptr<nuraft::log_entry> & entry)
{
std::lock_guard lock(changelog_lock);
changelog.writeAt(index, entry, force_sync);
}
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> KeeperLogStore::log_entries(size_t start, size_t end)
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> KeeperLogStore::log_entries(uint64_t start, uint64_t end)
{
std::lock_guard lock(changelog_lock);
return changelog.getLogEntriesBetween(start, end);
}
nuraft::ptr<nuraft::log_entry> KeeperLogStore::entry_at(size_t index)
nuraft::ptr<nuraft::log_entry> KeeperLogStore::entry_at(uint64_t index)
{
std::lock_guard lock(changelog_lock);
return changelog.entryAt(index);
}
size_t KeeperLogStore::term_at(size_t index)
uint64_t KeeperLogStore::term_at(uint64_t index)
{
std::lock_guard lock(changelog_lock);
auto entry = changelog.entryAt(index);
@ -70,13 +70,13 @@ size_t KeeperLogStore::term_at(size_t index)
return 0;
}
nuraft::ptr<nuraft::buffer> KeeperLogStore::pack(size_t index, int32_t cnt)
nuraft::ptr<nuraft::buffer> KeeperLogStore::pack(uint64_t index, int32_t cnt)
{
std::lock_guard lock(changelog_lock);
return changelog.serializeEntriesToBuffer(index, cnt);
}
bool KeeperLogStore::compact(size_t last_log_index)
bool KeeperLogStore::compact(uint64_t last_log_index)
{
std::lock_guard lock(changelog_lock);
changelog.compact(last_log_index);
@ -90,13 +90,13 @@ bool KeeperLogStore::flush()
return true;
}
void KeeperLogStore::apply_pack(size_t index, nuraft::buffer & pack)
void KeeperLogStore::apply_pack(uint64_t index, nuraft::buffer & pack)
{
std::lock_guard lock(changelog_lock);
changelog.applyEntriesFromBuffer(index, pack, force_sync);
}
size_t KeeperLogStore::size() const
uint64_t KeeperLogStore::size() const
{
std::lock_guard lock(changelog_lock);
return changelog.size();

View File

@ -12,35 +12,35 @@ namespace DB
class KeeperLogStore : public nuraft::log_store
{
public:
KeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_, bool force_sync_);
KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_);
void init(size_t last_commited_log_index, size_t logs_to_keep);
void init(uint64_t last_commited_log_index, uint64_t logs_to_keep);
size_t start_index() const override;
uint64_t start_index() const override;
size_t next_slot() const override;
uint64_t next_slot() const override;
nuraft::ptr<nuraft::log_entry> last_entry() const override;
size_t append(nuraft::ptr<nuraft::log_entry> & entry) override;
uint64_t append(nuraft::ptr<nuraft::log_entry> & entry) override;
void write_at(size_t index, nuraft::ptr<nuraft::log_entry> & entry) override;
void write_at(uint64_t index, nuraft::ptr<nuraft::log_entry> & entry) override;
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> log_entries(size_t start, size_t end) override;
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> log_entries(uint64_t start, uint64_t end) override;
nuraft::ptr<nuraft::log_entry> entry_at(size_t index) override;
nuraft::ptr<nuraft::log_entry> entry_at(uint64_t index) override;
size_t term_at(size_t index) override;
uint64_t term_at(uint64_t index) override;
nuraft::ptr<nuraft::buffer> pack(size_t index, int32_t cnt) override;
nuraft::ptr<nuraft::buffer> pack(uint64_t index, int32_t cnt) override;
void apply_pack(size_t index, nuraft::buffer & pack) override;
void apply_pack(uint64_t index, nuraft::buffer & pack) override;
bool compact(size_t last_log_index) override;
bool compact(uint64_t last_log_index) override;
bool flush() override;
size_t size() const;
uint64_t size() const;
private:
mutable std::mutex changelog_lock;

View File

@ -32,14 +32,14 @@ KeeperServer::KeeperServer(
coordination_settings))
, state_manager(nuraft::cs_new<KeeperStateManager>(server_id, "keeper_server", config, coordination_settings))
, responses_queue(responses_queue_)
, log(&Poco::Logger::get("KeeperServer"))
{
if (coordination_settings->quorum_reads)
LOG_WARNING(&Poco::Logger::get("KeeperServer"), "Quorum reads enabled, Keeper will work slower.");
LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower.");
}
void KeeperServer::startup()
{
state_machine->init();
state_manager->loadLogStore(state_machine->last_commit_index() + 1, coordination_settings->reserved_log_items);
@ -72,28 +72,90 @@ void KeeperServer::startup()
params.return_method_ = nuraft::raft_params::blocking;
nuraft::asio_service::options asio_opts{};
nuraft::raft_server::init_options init_options;
init_options.skip_initial_election_timeout_ = state_manager->shouldStartAsFollower();
init_options.raft_callback_ = [this] (nuraft::cb_func::Type type, nuraft::cb_func::Param * param)
{
return callbackFunc(type, param);
};
raft_instance = launcher.init(
state_machine, state_manager, nuraft::cs_new<LoggerWrapper>("RaftInstance", coordination_settings->raft_logs_level), state_manager->getPort(),
asio_opts, params, init_options);
launchRaftServer(params, asio_opts);
if (!raft_instance)
throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance");
}
void KeeperServer::launchRaftServer(
const nuraft::raft_params & params,
const nuraft::asio_service::options & asio_opts)
{
nuraft::raft_server::init_options init_options;
init_options.skip_initial_election_timeout_ = state_manager->shouldStartAsFollower();
init_options.start_server_in_constructor_ = false;
init_options.raft_callback_ = [this] (nuraft::cb_func::Type type, nuraft::cb_func::Param * param)
{
return callbackFunc(type, param);
};
nuraft::ptr<nuraft::logger> logger = nuraft::cs_new<LoggerWrapper>("RaftInstance", coordination_settings->raft_logs_level);
asio_service = nuraft::cs_new<nuraft::asio_service>(asio_opts, logger);
asio_listener = asio_service->create_rpc_listener(state_manager->getPort(), logger);
if (!asio_listener)
return;
nuraft::ptr<nuraft::delayed_task_scheduler> scheduler = asio_service;
nuraft::ptr<nuraft::rpc_client_factory> rpc_cli_factory = asio_service;
nuraft::ptr<nuraft::state_mgr> casted_state_manager = state_manager;
nuraft::ptr<nuraft::state_machine> casted_state_machine = state_machine;
/// raft_server creates unique_ptr from it
nuraft::context * ctx = new nuraft::context(
casted_state_manager, casted_state_machine,
asio_listener, logger, rpc_cli_factory, scheduler, params);
raft_instance = nuraft::cs_new<nuraft::raft_server>(ctx, init_options);
raft_instance->start_server(init_options.skip_initial_election_timeout_);
asio_listener->listen(raft_instance);
}
void KeeperServer::shutdownRaftServer()
{
size_t timeout = coordination_settings->shutdown_timeout.totalSeconds();
if (!raft_instance)
{
LOG_INFO(log, "RAFT doesn't start, shutdown not required");
return;
}
raft_instance->shutdown();
raft_instance.reset();
if (asio_listener)
{
asio_listener->stop();
asio_listener->shutdown();
}
if (asio_service)
{
asio_service->stop();
size_t count = 0;
while (asio_service->get_active_workers() != 0 && count < timeout * 100)
{
std::this_thread::sleep_for(std::chrono::milliseconds(10));
count++;
}
}
if (asio_service->get_active_workers() != 0)
LOG_WARNING(log, "Failed to shutdown RAFT server in {} seconds", timeout);
}
void KeeperServer::shutdown()
{
state_machine->shutdownStorage();
state_manager->flushLogStore();
auto timeout = coordination_settings->shutdown_timeout.totalSeconds();
if (!launcher.shutdown(timeout))
LOG_WARNING(&Poco::Logger::get("KeeperServer"), "Failed to shutdown RAFT server in {} seconds", timeout);
shutdownRaftServer();
}
namespace
@ -190,17 +252,17 @@ bool KeeperServer::isLeaderAlive() const
return raft_instance->is_leader_alive();
}
nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * /* param */)
nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * param)
{
if (initialized_flag)
return nuraft::cb_func::ReturnCode::Ok;
size_t last_commited = state_machine->last_commit_index();
size_t next_index = state_manager->getLogStore()->next_slot();
bool commited_store = false;
if (next_index < last_commited || next_index - last_commited <= 1)
commited_store = true;
if (initialized_flag)
return nuraft::cb_func::ReturnCode::Ok;
auto set_initialized = [this] ()
{
std::unique_lock lock(initialized_mutex);
@ -220,7 +282,7 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
case nuraft::cb_func::BecomeFollower:
case nuraft::cb_func::GotAppendEntryReqFromLeader:
{
if (isLeaderAlive())
if (param->leaderId != -1)
{
auto leader_index = raft_instance->get_leader_committed_log_idx();
auto our_index = raft_instance->get_committed_log_idx();
@ -240,7 +302,7 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
}
case nuraft::cb_func::InitialBatchCommited:
{
if (isLeader()) /// We have committed our log store and we are leader, ready to serve requests.
if (param->myId == param->leaderId) /// We have committed our log store and we are leader, ready to serve requests.
set_initialized();
initial_batch_committed = true;
return nuraft::cb_func::ReturnCode::Ok;

View File

@ -7,6 +7,7 @@
#include <Coordination/KeeperStorage.h>
#include <Coordination/CoordinationSettings.h>
#include <unordered_map>
#include <common/logger_useful.h>
namespace DB
{
@ -22,9 +23,9 @@ private:
nuraft::ptr<KeeperStateManager> state_manager;
nuraft::raft_launcher launcher;
nuraft::ptr<nuraft::raft_server> raft_instance;
nuraft::ptr<nuraft::asio_service> asio_service;
nuraft::ptr<nuraft::rpc_listener> asio_listener;
std::mutex append_entries_mutex;
@ -36,8 +37,19 @@ private:
std::atomic<bool> initial_batch_committed = false;
std::atomic<size_t> active_session_id_requests = 0;
Poco::Logger * log;
nuraft::cb_func::ReturnCode callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * param);
/// Almost copy-paste from nuraft::launcher, but with separated server init and start
/// Allows to avoid race conditions.
void launchRaftServer(
const nuraft::raft_params & params,
const nuraft::asio_service::options & asio_opts);
void shutdownRaftServer();
public:
KeeperServer(
int server_id_,

View File

@ -23,16 +23,16 @@ namespace ErrorCodes
namespace
{
size_t getSnapshotPathUpToLogIdx(const String & snapshot_path)
uint64_t getSnapshotPathUpToLogIdx(const String & snapshot_path)
{
std::filesystem::path path(snapshot_path);
std::string filename = path.stem();
Strings name_parts;
splitInto<'_'>(name_parts, filename);
return parse<size_t>(name_parts[1]);
return parse<uint64_t>(name_parts[1]);
}
std::string getSnapshotFileName(size_t up_to_log_idx)
std::string getSnapshotFileName(uint64_t up_to_log_idx)
{
return std::string{"snapshot_"} + std::to_string(up_to_log_idx) + ".bin";
}
@ -214,7 +214,7 @@ SnapshotMetadataPtr KeeperStorageSnapshot::deserialize(KeeperStorage & storage,
return result;
}
KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, size_t up_to_log_idx_)
KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_)
: storage(storage_)
, snapshot_meta(std::make_shared<SnapshotMetadata>(up_to_log_idx_, 0, std::make_shared<nuraft::cluster_config>()))
, session_id(storage->session_id_counter)
@ -266,7 +266,7 @@ KeeperSnapshotManager::KeeperSnapshotManager(const std::string & snapshots_path_
}
std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, size_t up_to_log_idx)
std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx)
{
ReadBufferFromNuraftBuffer reader(buffer);
@ -307,7 +307,7 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeLatestSnapshotBuff
return nullptr;
}
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(size_t up_to_log_idx) const
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const
{
const std::string & snapshot_path = existing_snapshots.at(up_to_log_idx);
WriteBufferFromNuraftBuffer writer;
@ -352,7 +352,7 @@ void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded()
removeSnapshot(existing_snapshots.begin()->first);
}
void KeeperSnapshotManager::removeSnapshot(size_t log_idx)
void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx)
{
auto itr = existing_snapshots.find(log_idx);
if (itr == existing_snapshots.end())

View File

@ -18,7 +18,7 @@ enum SnapshotVersion : uint8_t
struct KeeperStorageSnapshot
{
public:
KeeperStorageSnapshot(KeeperStorage * storage_, size_t up_to_log_idx_);
KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_);
KeeperStorageSnapshot(KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_);
~KeeperStorageSnapshot();
@ -51,14 +51,14 @@ public:
SnapshotMetaAndStorage restoreFromLatestSnapshot();
static nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot);
std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, size_t up_to_log_idx);
std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx);
SnapshotMetaAndStorage deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const;
nuraft::ptr<nuraft::buffer> deserializeSnapshotBufferFromDisk(size_t up_to_log_idx) const;
nuraft::ptr<nuraft::buffer> deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const;
nuraft::ptr<nuraft::buffer> deserializeLatestSnapshotBufferFromDisk();
void removeSnapshot(size_t log_idx);
void removeSnapshot(uint64_t log_idx);
size_t totalSnapshots() const
{
@ -76,7 +76,7 @@ private:
void removeOutdatedSnapshotsIfNeeded();
const std::string snapshots_path;
const size_t snapshots_to_keep;
std::map<size_t, std::string> existing_snapshots;
std::map<uint64_t, std::string> existing_snapshots;
size_t storage_tick_time;
};

View File

@ -54,7 +54,7 @@ void KeeperStateMachine::init()
bool has_snapshots = snapshot_manager.totalSnapshots() != 0;
while (snapshot_manager.totalSnapshots() != 0)
{
size_t latest_log_index = snapshot_manager.getLatestSnapshotIndex();
uint64_t latest_log_index = snapshot_manager.getLatestSnapshotIndex();
LOG_DEBUG(log, "Trying to load state machine from snapshot up to log index {}", latest_log_index);
try
@ -88,7 +88,7 @@ void KeeperStateMachine::init()
storage = std::make_unique<KeeperStorage>(coordination_settings->dead_session_check_period_ms.totalMilliseconds());
}
nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const size_t log_idx, nuraft::buffer & data)
nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data)
{
if (data.size() == sizeof(int64_t))
{
@ -205,7 +205,7 @@ void KeeperStateMachine::create_snapshot(
void KeeperStateMachine::save_logical_snp_obj(
nuraft::snapshot & s,
size_t & obj_id,
uint64_t & obj_id,
nuraft::buffer & data,
bool /*is_first_obj*/,
bool /*is_last_obj*/)
@ -246,7 +246,7 @@ void KeeperStateMachine::save_logical_snp_obj(
int KeeperStateMachine::read_logical_snp_obj(
nuraft::snapshot & s,
void* & /*user_snp_ctx*/,
ulong obj_id,
uint64_t obj_id,
nuraft::ptr<nuraft::buffer> & data_out,
bool & is_last_obj)
{

View File

@ -20,13 +20,13 @@ public:
void init();
nuraft::ptr<nuraft::buffer> pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; }
nuraft::ptr<nuraft::buffer> pre_commit(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; }
nuraft::ptr<nuraft::buffer> commit(const size_t log_idx, nuraft::buffer & data) override;
nuraft::ptr<nuraft::buffer> commit(const uint64_t log_idx, nuraft::buffer & data) override;
void rollback(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override {}
void rollback(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override {}
size_t last_commit_index() override { return last_committed_idx; }
uint64_t last_commit_index() override { return last_committed_idx; }
bool apply_snapshot(nuraft::snapshot & s) override;
@ -38,7 +38,7 @@ public:
void save_logical_snp_obj(
nuraft::snapshot & s,
size_t & obj_id,
uint64_t & obj_id,
nuraft::buffer & data,
bool is_first_obj,
bool is_last_obj) override;
@ -46,7 +46,7 @@ public:
int read_logical_snp_obj(
nuraft::snapshot & s,
void* & user_snp_ctx,
ulong obj_id,
uint64_t obj_id,
nuraft::ptr<nuraft::buffer> & data_out,
bool & is_last_obj) override;
@ -82,7 +82,7 @@ private:
std::mutex storage_lock;
/// Last committed Raft log number.
std::atomic<size_t> last_committed_idx;
std::atomic<uint64_t> last_committed_idx;
Poco::Logger * log;
};

View File

@ -64,7 +64,7 @@ KeeperStateManager::KeeperStateManager(
throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without <start_as_follower>)");
}
void KeeperStateManager::loadLogStore(size_t last_commited_index, size_t logs_to_keep)
void KeeperStateManager::loadLogStore(uint64_t last_commited_index, uint64_t logs_to_keep)
{
log_store->init(last_commited_index, logs_to_keep);
}

View File

@ -25,7 +25,7 @@ public:
int port,
const std::string & logs_path);
void loadLogStore(size_t last_commited_index, size_t logs_to_keep);
void loadLogStore(uint64_t last_commited_index, uint64_t logs_to_keep);
void flushLogStore();
@ -54,12 +54,12 @@ public:
nuraft::ptr<KeeperLogStore> getLogStore() const { return log_store; }
size_t getTotalServers() const { return total_servers; }
uint64_t getTotalServers() const { return total_servers; }
private:
int my_server_id;
int my_port;
size_t total_servers{0};
uint64_t total_servers{0};
std::unordered_set<int> start_as_follower_servers;
nuraft::ptr<KeeperLogStore> log_store;
nuraft::ptr<nuraft::srv_config> my_server_config;

View File

@ -21,7 +21,7 @@ SummingStateMachine::SummingStateMachine()
{
}
nuraft::ptr<nuraft::buffer> SummingStateMachine::commit(const size_t log_idx, nuraft::buffer & data)
nuraft::ptr<nuraft::buffer> SummingStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data)
{
int64_t value_to_add = deserializeValue(data);
@ -84,7 +84,7 @@ void SummingStateMachine::createSnapshotInternal(nuraft::snapshot & s)
void SummingStateMachine::save_logical_snp_obj(
nuraft::snapshot & s,
size_t & obj_id,
uint64_t & obj_id,
nuraft::buffer & data,
bool /*is_first_obj*/,
bool /*is_last_obj*/)
@ -112,7 +112,7 @@ void SummingStateMachine::save_logical_snp_obj(
int SummingStateMachine::read_logical_snp_obj(
nuraft::snapshot & s,
void* & /*user_snp_ctx*/,
size_t obj_id,
uint64_t obj_id,
nuraft::ptr<nuraft::buffer> & data_out,
bool & is_last_obj)
{
@ -142,7 +142,7 @@ int SummingStateMachine::read_logical_snp_obj(
else
{
// Object ID > 0: second object, put actual value.
data_out = nuraft::buffer::alloc(sizeof(size_t));
data_out = nuraft::buffer::alloc(sizeof(uint64_t));
nuraft::buffer_serializer bs(data_out);
bs.put_u64(ctx->value);
is_last_obj = true;

View File

@ -15,13 +15,13 @@ class SummingStateMachine : public nuraft::state_machine
public:
SummingStateMachine();
nuraft::ptr<nuraft::buffer> pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; }
nuraft::ptr<nuraft::buffer> pre_commit(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; }
nuraft::ptr<nuraft::buffer> commit(const size_t log_idx, nuraft::buffer & data) override;
nuraft::ptr<nuraft::buffer> commit(const uint64_t log_idx, nuraft::buffer & data) override;
void rollback(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override {}
void rollback(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override {}
size_t last_commit_index() override { return last_committed_idx; }
uint64_t last_commit_index() override { return last_committed_idx; }
bool apply_snapshot(nuraft::snapshot & s) override;
@ -33,7 +33,7 @@ public:
void save_logical_snp_obj(
nuraft::snapshot & s,
size_t & obj_id,
uint64_t & obj_id,
nuraft::buffer & data,
bool is_first_obj,
bool is_last_obj) override;
@ -41,7 +41,7 @@ public:
int read_logical_snp_obj(
nuraft::snapshot & s,
void* & user_snp_ctx,
size_t obj_id,
uint64_t obj_id,
nuraft::ptr<nuraft::buffer> & data_out,
bool & is_last_obj) override;

View File

@ -1085,7 +1085,7 @@ nuraft::ptr<nuraft::log_entry> getLogEntryFromZKRequest(size_t term, int64_t ses
return nuraft::cs_new<nuraft::log_entry>(term, buffer);
}
void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, size_t total_logs)
void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint64_t total_logs)
{
using namespace Coordination;
using namespace DB;

View File

@ -455,7 +455,11 @@ class IColumn;
M(UnionMode, union_default_mode, UnionMode::Unspecified, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) \
M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \
M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \
\
M(Bool, query_plan_enable_optimizations, true, "Apply optimizations to query plan", 0) \
M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \
M(Bool, query_plan_filter_push_down, true, "Allow to push down filter by predicate query plan step", 0) \
\
M(Bool, database_replicated_ddl_output, true, "Obsolete setting, does nothing. Will be removed after 2021-09-08", 0) \
// End of COMMON_SETTINGS

View File

@ -10,7 +10,7 @@
#include <Storages/IStorage.h>
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
#include <iomanip>
#include <Poco/File.h>
@ -61,7 +61,7 @@ void DatabaseLazy::createTable(
const StoragePtr & table,
const ASTPtr & query)
{
SCOPE_EXIT({ clearExpiredTables(); });
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
if (!endsWith(table->getName(), "Log"))
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
DatabaseOnDisk::createTable(context, table_name, table, query);
@ -78,7 +78,7 @@ void DatabaseLazy::dropTable(
const String & table_name,
bool no_delay)
{
SCOPE_EXIT({ clearExpiredTables(); });
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
DatabaseOnDisk::dropTable(context, table_name, no_delay);
}
@ -90,7 +90,7 @@ void DatabaseLazy::renameTable(
bool exchange,
bool dictionary)
{
SCOPE_EXIT({ clearExpiredTables(); });
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary);
}
@ -115,14 +115,14 @@ void DatabaseLazy::alterTable(
bool DatabaseLazy::isTableExist(const String & table_name) const
{
SCOPE_EXIT({ clearExpiredTables(); });
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
std::lock_guard lock(mutex);
return tables_cache.find(table_name) != tables_cache.end();
}
StoragePtr DatabaseLazy::tryGetTable(const String & table_name) const
{
SCOPE_EXIT({ clearExpiredTables(); });
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
{
std::lock_guard lock(mutex);
auto it = tables_cache.find(table_name);
@ -224,7 +224,7 @@ DatabaseLazy::~DatabaseLazy()
StoragePtr DatabaseLazy::loadTable(const String & table_name) const
{
SCOPE_EXIT({ clearExpiredTables(); });
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
LOG_DEBUG(log, "Load table {} to cache.", backQuote(table_name));

View File

@ -133,7 +133,7 @@ ColumnPtr CacheDictionary<dictionary_key_type>::getColumn(
template <DictionaryKeyType dictionary_key_type>
Columns CacheDictionary<dictionary_key_type>::getColumns(
const Strings & attribute_names,
const DataTypes &,
const DataTypes & result_types,
const Columns & key_columns,
const DataTypes & key_types,
const Columns & default_values_columns) const
@ -159,7 +159,7 @@ Columns CacheDictionary<dictionary_key_type>::getColumns(
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns, arena_holder.getComplexKeyArena());
auto keys = extractor.extractAllKeys();
DictionaryStorageFetchRequest request(dict_struct, attribute_names, default_values_columns);
DictionaryStorageFetchRequest request(dict_struct, attribute_names, result_types, default_values_columns);
FetchResult result_of_fetch_from_storage;
@ -277,7 +277,7 @@ ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::hasKeys(const Columns & k
const auto keys = extractor.extractAllKeys();
/// We make empty request just to fetch if keys exists
DictionaryStorageFetchRequest request(dict_struct, {}, {});
DictionaryStorageFetchRequest request(dict_struct, {}, {}, {});
FetchResult result_of_fetch_from_storage;

View File

@ -1,11 +1,14 @@
#pragma once
#include <Common/Arena.h>
#include <Common/HashTable/HashMap.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Core/Block.h>
#include <Dictionaries/IDictionary.h>
#include <Dictionaries/DictionaryStructure.h>
@ -60,7 +63,11 @@ private:
class DictionaryStorageFetchRequest
{
public:
DictionaryStorageFetchRequest(const DictionaryStructure & structure, const Strings & attributes_names_to_fetch, Columns attributes_default_values_columns)
DictionaryStorageFetchRequest(
const DictionaryStructure & structure,
const Strings & attributes_names_to_fetch,
DataTypes attributes_to_fetch_result_types,
Columns attributes_default_values_columns)
: attributes_to_fetch_names_set(attributes_names_to_fetch.begin(), attributes_names_to_fetch.end())
, attributes_to_fetch_filter(structure.attributes.size(), false)
{
@ -73,7 +80,7 @@ public:
dictionary_attributes_types.reserve(attributes_size);
attributes_default_value_providers.reserve(attributes_to_fetch_names_set.size());
size_t default_values_column_index = 0;
size_t attributes_to_fetch_index = 0;
for (size_t i = 0; i < attributes_size; ++i)
{
const auto & dictionary_attribute = structure.attributes[i];
@ -84,8 +91,16 @@ public:
if (attributes_to_fetch_names_set.find(name) != attributes_to_fetch_names_set.end())
{
attributes_to_fetch_filter[i] = true;
attributes_default_value_providers.emplace_back(dictionary_attribute.null_value, attributes_default_values_columns[default_values_column_index]);
++default_values_column_index;
auto & attribute_to_fetch_result_type = attributes_to_fetch_result_types[attributes_to_fetch_index];
if (!attribute_to_fetch_result_type->equals(*type))
throw Exception(ErrorCodes::TYPE_MISMATCH,
"Attribute type does not match, expected ({}), found ({})",
attribute_to_fetch_result_type->getName(),
type->getName());
attributes_default_value_providers.emplace_back(dictionary_attribute.null_value, attributes_default_values_columns[attributes_to_fetch_index]);
++attributes_to_fetch_index;
}
else
attributes_default_value_providers.emplace_back(dictionary_attribute.null_value);
@ -416,6 +431,105 @@ private:
Arena * complex_key_arena;
};
/** Merge block with blocks from stream. If there are duplicate keys in block they are filtered out.
* In result block_to_update will be merged with blocks from stream.
* Note: readPrefix readImpl readSuffix will be called on stream object during function execution.
*/
template <DictionaryKeyType dictionary_key_type>
void mergeBlockWithStream(
size_t key_column_size [[maybe_unused]],
Block & block_to_update [[maybe_unused]],
BlockInputStreamPtr & stream [[maybe_unused]])
{
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by updatePreviousyLoadedBlockWithStream");
Columns saved_block_key_columns;
saved_block_key_columns.reserve(key_column_size);
/// Split into keys columns and attribute columns
for (size_t i = 0; i < key_column_size; ++i)
saved_block_key_columns.emplace_back(block_to_update.safeGetByPosition(i).column);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
DictionaryKeysExtractor<dictionary_key_type> saved_keys_extractor(saved_block_key_columns, arena_holder.getComplexKeyArena());
auto saved_keys_extracted_from_block = saved_keys_extractor.extractAllKeys();
IColumn::Filter filter(saved_keys_extracted_from_block.size(), true);
HashMap<KeyType, size_t> saved_key_to_index;
saved_key_to_index.reserve(saved_keys_extracted_from_block.size());
size_t indexes_to_remove_count = 0;
for (size_t i = 0; i < saved_keys_extracted_from_block.size(); ++i)
{
auto saved_key = saved_keys_extracted_from_block[i];
auto [it, was_inserted] = saved_key_to_index.insert(makePairNoInit(saved_key, i));
if (!was_inserted)
{
size_t index_to_remove = it->getMapped();
filter[index_to_remove] = false;
it->getMapped() = i;
++indexes_to_remove_count;
}
}
auto result_fetched_columns = block_to_update.cloneEmptyColumns();
stream->readPrefix();
while (Block block = stream->read())
{
Columns block_key_columns;
block_key_columns.reserve(key_column_size);
/// Split into keys columns and attribute columns
for (size_t i = 0; i < key_column_size; ++i)
block_key_columns.emplace_back(block.safeGetByPosition(i).column);
DictionaryKeysExtractor<dictionary_key_type> update_keys_extractor(block_key_columns, arena_holder.getComplexKeyArena());
PaddedPODArray<KeyType> update_keys = update_keys_extractor.extractAllKeys();
for (auto update_key : update_keys)
{
const auto * it = saved_key_to_index.find(update_key);
if (it != nullptr)
{
size_t index_to_filter = it->getMapped();
filter[index_to_filter] = false;
++indexes_to_remove_count;
}
}
size_t rows = block.rows();
for (size_t column_index = 0; column_index < block.columns(); ++column_index)
{
const auto update_column = block.safeGetByPosition(column_index).column;
MutableColumnPtr & result_fetched_column = result_fetched_columns[column_index];
result_fetched_column->insertRangeFrom(*update_column, 0, rows);
}
}
stream->readSuffix();
size_t result_fetched_rows = result_fetched_columns.front()->size();
size_t filter_hint = filter.size() - indexes_to_remove_count;
for (size_t column_index = 0; column_index < block_to_update.columns(); ++column_index)
{
auto & column = block_to_update.getByPosition(column_index).column;
column = column->filter(filter, filter_hint);
MutableColumnPtr mutable_column = column->assumeMutable();
const IColumn & fetched_column = *result_fetched_columns[column_index];
mutable_column->insertRangeFrom(fetched_column, 0, result_fetched_rows);
}
}
/**
* Returns ColumnVector data as PaddedPodArray.

View File

@ -31,12 +31,12 @@ DirectDictionary<dictionary_key_type>::DirectDictionary(
}
template <DictionaryKeyType dictionary_key_type>
ColumnPtr DirectDictionary<dictionary_key_type>::getColumn(
const std::string & attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types [[maybe_unused]],
const ColumnPtr & default_values_column) const
Columns DirectDictionary<dictionary_key_type>::getColumns(
const Strings & attribute_names,
const DataTypes & result_types,
const Columns & key_columns,
const DataTypes & key_types [[maybe_unused]],
const Columns & default_values_columns) const
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
dict_struct.validateKeyTypes(key_types);
@ -45,16 +45,14 @@ ColumnPtr DirectDictionary<dictionary_key_type>::getColumn(
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns, arena_holder.getComplexKeyArena());
const auto requested_keys = extractor.extractAllKeys();
const DictionaryAttribute & attribute = dict_struct.getAttribute(attribute_name, result_type);
DefaultValueProvider default_value_provider(attribute.null_value, default_values_column);
DictionaryStorageFetchRequest request(dict_struct, attribute_names, result_types, default_values_columns);
HashMap<KeyType, size_t> key_to_fetched_index;
key_to_fetched_index.reserve(requested_keys.size());
auto fetched_from_storage = attribute.type->createColumn();
auto fetched_columns_from_storage = request.makeAttributesResultColumns();
size_t fetched_key_index = 0;
size_t requested_attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second;
Columns block_key_columns;
size_t dictionary_keys_size = dict_struct.getKeysNames().size();
@ -73,8 +71,14 @@ ColumnPtr DirectDictionary<dictionary_key_type>::getColumn(
DictionaryKeysExtractor<dictionary_key_type> block_keys_extractor(block_key_columns, arena_holder.getComplexKeyArena());
auto block_keys = block_keys_extractor.extractAllKeys();
const auto & block_column = block.safeGetByPosition(dictionary_keys_size + requested_attribute_index).column;
fetched_from_storage->insertRangeFrom(*block_column, 0, block_keys.size());
for (size_t attribute_index = 0; attribute_index < request.attributesSize(); ++attribute_index)
{
if (!request.shouldFillResultColumnWithIndex(attribute_index))
continue;
const auto & block_column = block.safeGetByPosition(dictionary_keys_size + attribute_index).column;
fetched_columns_from_storage[attribute_index]->insertRangeFrom(*block_column, 0, block_keys.size());
}
for (size_t block_key_index = 0; block_key_index < block_keys.size(); ++block_key_index)
{
@ -92,29 +96,54 @@ ColumnPtr DirectDictionary<dictionary_key_type>::getColumn(
size_t requested_keys_size = requested_keys.size();
auto result = fetched_from_storage->cloneEmpty();
result->reserve(requested_keys_size);
auto result_columns = request.makeAttributesResultColumns();
for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index)
for (size_t attribute_index = 0; attribute_index < result_columns.size(); ++attribute_index)
{
const auto requested_key = requested_keys[requested_key_index];
const auto * it = key_to_fetched_index.find(requested_key);
if (!request.shouldFillResultColumnWithIndex(attribute_index))
continue;
if (it)
fetched_from_storage->get(it->getMapped(), value_to_insert);
else
value_to_insert = default_value_provider.getDefaultValue(requested_key_index);
auto & result_column = result_columns[attribute_index];
result->insert(value_to_insert);
const auto & fetched_column_from_storage = fetched_columns_from_storage[attribute_index];
const auto & default_value_provider = request.defaultValueProviderAtIndex(attribute_index);
result_column->reserve(requested_keys_size);
for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index)
{
const auto requested_key = requested_keys[requested_key_index];
const auto * it = key_to_fetched_index.find(requested_key);
if (it)
fetched_column_from_storage->get(it->getMapped(), value_to_insert);
else
value_to_insert = default_value_provider.getDefaultValue(requested_key_index);
result_column->insert(value_to_insert);
}
}
query_count.fetch_add(requested_keys_size, std::memory_order_relaxed);
return result;
return request.filterRequestedColumns(result_columns);
}
template <DictionaryKeyType dictionary_key_type>
ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::hasKeys(const Columns & key_columns, const DataTypes & key_types [[maybe_unused]]) const
ColumnPtr DirectDictionary<dictionary_key_type>::getColumn(
const std::string & attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr & default_values_column) const
{
return getColumns({ attribute_name }, { result_type }, key_columns, key_types, { default_values_column }).front();
}
template <DictionaryKeyType dictionary_key_type>
ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::hasKeys(
const Columns & key_columns,
const DataTypes & key_types [[maybe_unused]]) const
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
dict_struct.validateKeyTypes(key_types);

View File

@ -66,6 +66,13 @@ public:
DictionaryKeyType getKeyType() const override { return dictionary_key_type; }
Columns getColumns(
const Strings & attribute_names,
const DataTypes & result_types,
const Columns & key_columns,
const DataTypes & key_types,
const Columns & default_values_columns) const override;
ColumnPtr getColumn(
const std::string& attribute_name,
const DataTypePtr & result_type,

View File

@ -1,8 +1,11 @@
#include "FileDictionarySource.h"
#include <Poco/File.h>
#include <filesystem>
#include <DataStreams/OwningBlockInputStream.h>
#include <IO/ReadBufferFromFile.h>
#include <Interpreters/Context.h>
#include <Poco/File.h>
#include <Common/StringUtils/StringUtils.h>
#include <common/logger_useful.h>
#include "DictionarySourceFactory.h"
@ -10,7 +13,6 @@
#include "registerDictionaries.h"
#include "DictionarySourceHelpers.h"
namespace DB
{
static const UInt64 max_block_size = 8192;
@ -32,9 +34,19 @@ FileDictionarySource::FileDictionarySource(
{
if (check_config)
{
const String user_files_path = context.getUserFilesPath();
if (!startsWith(filepath, user_files_path))
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", filepath, user_files_path);
auto source_file_path = std::filesystem::path(filepath);
auto source_file_absolute_path = std::filesystem::canonical(source_file_path);
String user_files_path_string_value = context.getUserFilesPath();
auto user_files_path = std::filesystem::path(user_files_path_string_value);
auto user_files_absolute_path = std::filesystem::canonical(user_files_path);
auto [_, user_files_absolute_path_mismatch_it] = std::mismatch(source_file_absolute_path.begin(), source_file_absolute_path.end(), user_files_absolute_path.begin(), user_files_absolute_path.end());
bool user_files_absolute_path_include_source_file_absolute_path = user_files_absolute_path_mismatch_it == user_files_absolute_path.end();
if (!user_files_absolute_path_include_source_file_absolute_path)
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", filepath, user_files_path_string_value);
}
}

View File

@ -2,6 +2,7 @@
#include <Core/Defines.h>
#include <Common/HashTable/HashMap.h>
#include <Common/HashTable/HashSet.h>
#include <DataTypes/DataTypesDecimal.h>
#include <IO/WriteHelpers.h>
@ -23,23 +24,20 @@ namespace ErrorCodes
extern const int UNSUPPORTED_METHOD;
}
static const auto initial_array_size = 1024;
static const auto max_array_size = 500000;
FlatDictionary::FlatDictionary(
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
bool require_nonempty_,
BlockPtr saved_block_)
Configuration configuration_,
BlockPtr previously_loaded_block_)
: IDictionary(dict_id_)
, dict_struct(dict_struct_)
, source_ptr{std::move(source_ptr_)}
, dict_lifetime(dict_lifetime_)
, require_nonempty(require_nonempty_)
, loaded_ids(initial_array_size, false)
, saved_block{std::move(saved_block_)}
, configuration(configuration_)
, loaded_keys(configuration.initial_array_size, false)
, previously_loaded_block(std::move(previously_loaded_block_))
{
createAttributes();
loadData();
@ -126,20 +124,19 @@ ColumnPtr FlatDictionary::getColumn(
ColumnUInt8::Ptr FlatDictionary::hasKeys(const Columns & key_columns, const DataTypes &) const
{
PaddedPODArray<UInt64> backup_storage;
const auto& ids = getColumnVectorData(this, key_columns.front(), backup_storage);
const auto & keys = getColumnVectorData(this, key_columns.front(), backup_storage);
size_t keys_size = keys.size();
auto result = ColumnUInt8::create(ext::size(ids));
auto& out = result->getData();
auto result = ColumnUInt8::create(keys_size);
auto & out = result->getData();
const auto ids_count = ext::size(ids);
for (const auto i : ext::range(0, ids_count))
for (size_t key_index = 0; key_index < keys_size; ++key_index)
{
const auto id = ids[i];
out[i] = id < loaded_ids.size() && loaded_ids[id];
const auto key = keys[key_index];
out[key_index] = key < loaded_keys.size() && loaded_keys[key];
}
query_count.fetch_add(ids_count, std::memory_order_relaxed);
query_count.fetch_add(keys_size, std::memory_order_relaxed);
return result;
}
@ -153,22 +150,14 @@ ColumnPtr FlatDictionary::getHierarchy(ColumnPtr key_column, const DataTypePtr &
const auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
const UInt64 null_value = std::get<UInt64>(hierarchical_attribute.null_values);
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.arrays);
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.container);
auto is_key_valid_func = [&, this](auto & key)
{
return key < loaded_ids.size() && loaded_ids[key];
};
auto is_key_valid_func = [&, this](auto & key) { return key < loaded_keys.size() && loaded_keys[key]; };
auto get_parent_key_func = [&, this](auto & hierarchy_key)
{
std::optional<UInt64> result;
if (hierarchy_key >= loaded_ids.size() || !loaded_ids[hierarchy_key])
return result;
result = parent_keys[hierarchy_key];
bool is_key_valid = hierarchy_key < loaded_keys.size() && loaded_keys[hierarchy_key];
std::optional<UInt64> result = is_key_valid ? std::make_optional(parent_keys[hierarchy_key]) : std::nullopt;
return result;
};
@ -194,22 +183,14 @@ ColumnUInt8::Ptr FlatDictionary::isInHierarchy(
const auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
const UInt64 null_value = std::get<UInt64>(hierarchical_attribute.null_values);
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.arrays);
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.container);
auto is_key_valid_func = [&, this](auto & key)
{
return key < loaded_ids.size() && loaded_ids[key];
};
auto is_key_valid_func = [&, this](auto & key) { return key < loaded_keys.size() && loaded_keys[key]; };
auto get_parent_key_func = [&, this](auto & hierarchy_key)
{
std::optional<UInt64> result;
if (hierarchy_key >= loaded_ids.size() || !loaded_ids[hierarchy_key])
return result;
result = parent_keys[hierarchy_key];
bool is_key_valid = hierarchy_key < loaded_keys.size() && loaded_keys[hierarchy_key];
std::optional<UInt64> result = is_key_valid ? std::make_optional(parent_keys[hierarchy_key]) : std::nullopt;
return result;
};
@ -230,7 +211,7 @@ ColumnPtr FlatDictionary::getDescendants(
size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index;
const auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.arrays);
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.container);
HashMap<UInt64, PaddedPODArray<UInt64>> parent_to_child;
@ -238,7 +219,7 @@ ColumnPtr FlatDictionary::getDescendants(
{
auto parent_key = parent_keys[i];
if (loaded_ids[i])
if (loaded_keys[i])
parent_to_child[parent_key].emplace_back(static_cast<UInt64>(i));
}
@ -260,22 +241,39 @@ void FlatDictionary::createAttributes()
void FlatDictionary::blockToAttributes(const Block & block)
{
const IColumn & id_column = *block.safeGetByPosition(0).column;
element_count += id_column.size();
const auto keys_column = block.safeGetByPosition(0).column;
for (const size_t attribute_idx : ext::range(0, attributes.size()))
DictionaryKeysArenaHolder<DictionaryKeyType::simple> arena_holder;
DictionaryKeysExtractor<DictionaryKeyType::simple> keys_extractor({ keys_column }, arena_holder.getComplexKeyArena());
auto keys = keys_extractor.extractAllKeys();
HashSet<UInt64> already_processed_keys;
size_t key_offset = 1;
for (size_t attribute_index = 0; attribute_index < attributes.size(); ++attribute_index)
{
const IColumn & attribute_column = *block.safeGetByPosition(attribute_idx + 1).column;
Attribute & attribute = attributes[attribute_idx];
const IColumn & attribute_column = *block.safeGetByPosition(attribute_index + key_offset).column;
Attribute & attribute = attributes[attribute_index];
for (const auto row_idx : ext::range(0, id_column.size()))
setAttributeValue(attribute, id_column[row_idx].get<UInt64>(), attribute_column[row_idx]);
for (size_t i = 0; i < keys.size(); ++i)
{
auto key = keys[i];
if (already_processed_keys.find(key) != nullptr)
continue;
already_processed_keys.insert(key);
setAttributeValue(attribute, key, attribute_column[i]);
++element_count;
}
already_processed_keys.clear();
}
}
void FlatDictionary::updateData()
{
if (!saved_block || saved_block->rows() == 0)
if (!previously_loaded_block || previously_loaded_block->rows() == 0)
{
auto stream = source_ptr->loadUpdatedAll();
stream->readPrefix();
@ -283,12 +281,13 @@ void FlatDictionary::updateData()
while (const auto block = stream->read())
{
/// We are using this to keep saved data if input stream consists of multiple blocks
if (!saved_block)
saved_block = std::make_shared<DB::Block>(block.cloneEmpty());
for (const auto attribute_idx : ext::range(0, attributes.size() + 1))
if (!previously_loaded_block)
previously_loaded_block = std::make_shared<DB::Block>(block.cloneEmpty());
for (size_t column_index = 0; column_index < block.columns(); ++column_index)
{
const IColumn & update_column = *block.getByPosition(attribute_idx).column.get();
MutableColumnPtr saved_column = saved_block->getByPosition(attribute_idx).column->assumeMutable();
const IColumn & update_column = *block.getByPosition(column_index).column.get();
MutableColumnPtr saved_column = previously_loaded_block->getByPosition(column_index).column->assumeMutable();
saved_column->insertRangeFrom(update_column, 0, update_column.size());
}
}
@ -297,51 +296,14 @@ void FlatDictionary::updateData()
else
{
auto stream = source_ptr->loadUpdatedAll();
stream->readPrefix();
while (Block block = stream->read())
{
const auto & saved_id_column = *saved_block->safeGetByPosition(0).column;
const auto & update_id_column = *block.safeGetByPosition(0).column;
std::unordered_map<UInt64, std::vector<size_t>> update_ids;
for (size_t row = 0; row < update_id_column.size(); ++row)
{
const auto id = update_id_column.get64(row);
update_ids[id].push_back(row);
}
const size_t saved_rows = saved_id_column.size();
IColumn::Filter filter(saved_rows);
std::unordered_map<UInt64, std::vector<size_t>>::iterator it;
for (size_t row = 0; row < saved_id_column.size(); ++row)
{
auto id = saved_id_column.get64(row);
it = update_ids.find(id);
if (it != update_ids.end())
filter[row] = 0;
else
filter[row] = 1;
}
auto block_columns = block.mutateColumns();
for (const auto attribute_idx : ext::range(0, attributes.size() + 1))
{
auto & column = saved_block->safeGetByPosition(attribute_idx).column;
const auto & filtered_column = column->filter(filter, -1);
block_columns[attribute_idx]->insertRangeFrom(*filtered_column.get(), 0, filtered_column->size());
}
saved_block->setColumns(std::move(block_columns));
}
stream->readSuffix();
mergeBlockWithStream<DictionaryKeyType::simple>(
dict_struct.getKeysSize(),
*previously_loaded_block,
stream);
}
if (saved_block)
blockToAttributes(*saved_block.get());
if (previously_loaded_block)
blockToAttributes(*previously_loaded_block.get());
}
void FlatDictionary::loadData()
@ -359,28 +321,10 @@ void FlatDictionary::loadData()
else
updateData();
if (require_nonempty && 0 == element_count)
if (configuration.require_nonempty && 0 == element_count)
throw Exception{full_name + ": dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY};
}
template <typename T>
void FlatDictionary::addAttributeSize(const Attribute & attribute)
{
const auto & array_ref = std::get<ContainerType<T>>(attribute.arrays);
bytes_allocated += sizeof(PaddedPODArray<T>) + array_ref.allocated_bytes();
bucket_count = array_ref.capacity();
}
template <>
void FlatDictionary::addAttributeSize<String>(const Attribute & attribute)
{
const auto & array_ref = std::get<ContainerType<StringRef>>(attribute.arrays);
bytes_allocated += sizeof(PaddedPODArray<StringRef>) + array_ref.allocated_bytes();
bytes_allocated += sizeof(Arena) + attribute.string_arena->size();
bucket_count = array_ref.capacity();
}
void FlatDictionary::calculateBytesAllocated()
{
bytes_allocated += attributes.size() * sizeof(attributes.front());
@ -391,102 +335,108 @@ void FlatDictionary::calculateBytesAllocated()
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
using ValueType = DictionaryValueType<AttributeType>;
addAttributeSize<AttributeType>(attribute);
const auto & container = std::get<ContainerType<ValueType>>(attribute.container);
bytes_allocated += sizeof(PaddedPODArray<ValueType>) + container.allocated_bytes();
bucket_count = container.capacity();
if constexpr (std::is_same_v<ValueType, StringRef>)
bytes_allocated += sizeof(Arena) + attribute.string_arena->size();
};
callOnDictionaryAttributeType(attribute.type, type_call);
}
}
template <typename T>
void FlatDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value)
FlatDictionary::Attribute FlatDictionary::createAttribute(const DictionaryAttribute & dictionary_attribute, const Field & null_value)
{
attribute.null_values = T(null_value.get<T>());
const auto & null_value_ref = std::get<T>(attribute.null_values);
attribute.arrays.emplace<ContainerType<T>>(initial_array_size, null_value_ref);
}
auto nullable_set = dictionary_attribute.is_nullable ? std::make_optional<NullableSet>() : std::optional<NullableSet>{};
Attribute attribute{dictionary_attribute.underlying_type, std::move(nullable_set), {}, {}, {}};
template <>
void FlatDictionary::createAttributeImpl<String>(Attribute & attribute, const Field & null_value)
{
attribute.string_arena = std::make_unique<Arena>();
const String & string = null_value.get<String>();
const char * string_in_arena = attribute.string_arena->insert(string.data(), string.size());
attribute.null_values.emplace<StringRef>(string_in_arena, string.size());
attribute.arrays.emplace<ContainerType<StringRef>>(initial_array_size, StringRef(string_in_arena, string.size()));
}
FlatDictionary::Attribute FlatDictionary::createAttribute(const DictionaryAttribute& attribute, const Field & null_value)
{
auto nullable_set = attribute.is_nullable ? std::make_optional<NullableSet>() : std::optional<NullableSet>{};
Attribute attr{attribute.underlying_type, std::move(nullable_set), {}, {}, {}};
auto type_call = [&](const auto &dictionary_attribute_type)
auto type_call = [&](const auto & dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
using ValueType = DictionaryValueType<AttributeType>;
createAttributeImpl<AttributeType>(attr, null_value);
if constexpr (std::is_same_v<ValueType, StringRef>)
{
attribute.string_arena = std::make_unique<Arena>();
const String & string = null_value.get<String>();
const char * string_in_arena = attribute.string_arena->insert(string.data(), string.size());
attribute.null_values.emplace<StringRef>(string_in_arena, string.size());
}
else
attribute.null_values = ValueType(null_value.get<NearestFieldType<ValueType>>());
const auto & null_value_ref = std::get<ValueType>(attribute.null_values);
attribute.container.emplace<ContainerType<ValueType>>(configuration.initial_array_size, null_value_ref);
};
callOnDictionaryAttributeType(attribute.underlying_type, type_call);
callOnDictionaryAttributeType(dictionary_attribute.underlying_type, type_call);
return attr;
return attribute;
}
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
void FlatDictionary::getItemsImpl(
const Attribute & attribute,
const PaddedPODArray<UInt64> & ids,
const PaddedPODArray<UInt64> & keys,
ValueSetter && set_value,
DefaultValueExtractor & default_value_extractor) const
{
const auto & attr = std::get<ContainerType<AttributeType>>(attribute.arrays);
const auto rows = ext::size(ids);
const auto & container = std::get<ContainerType<AttributeType>>(attribute.container);
const auto rows = keys.size();
for (const auto row : ext::range(0, rows))
for (size_t row = 0; row < rows; ++row)
{
const auto id = ids[row];
set_value(row, id < ext::size(attr) && loaded_ids[id] ? static_cast<OutputType>(attr[id]) : default_value_extractor[row]);
const auto key = keys[row];
if (key < loaded_keys.size() && loaded_keys[key])
set_value(row, static_cast<OutputType>(container[key]));
else
set_value(row, default_value_extractor[row]);
}
query_count.fetch_add(rows, std::memory_order_relaxed);
}
template <typename T>
void FlatDictionary::resize(Attribute & attribute, const UInt64 id)
void FlatDictionary::resize(Attribute & attribute, UInt64 key)
{
if (id >= max_array_size)
throw Exception{full_name + ": identifier should be less than " + toString(max_array_size), ErrorCodes::ARGUMENT_OUT_OF_BOUND};
if (key >= configuration.max_array_size)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND,
"({}): identifier should be less than ({})",
full_name,
toString(configuration.max_array_size));
auto & array = std::get<ContainerType<T>>(attribute.arrays);
if (id >= array.size())
auto & container = std::get<ContainerType<T>>(attribute.container);
if (key >= container.size())
{
const size_t elements_count = id + 1; //id=0 -> elements_count=1
loaded_ids.resize(elements_count, false);
array.resize_fill(elements_count, std::get<T>(attribute.null_values));
const size_t elements_count = key + 1; //id=0 -> elements_count=1
loaded_keys.resize(elements_count, false);
container.resize_fill(elements_count, std::get<T>(attribute.null_values));
}
}
template <typename T>
void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const UInt64 id, const T & value)
void FlatDictionary::setAttributeValueImpl(Attribute & attribute, UInt64 key, const T & value)
{
auto & array = std::get<ContainerType<T>>(attribute.arrays);
array[id] = value;
loaded_ids[id] = true;
auto & array = std::get<ContainerType<T>>(attribute.container);
array[key] = value;
loaded_keys[key] = true;
}
template <>
void FlatDictionary::setAttributeValueImpl<String>(Attribute & attribute, const UInt64 id, const String & value)
void FlatDictionary::setAttributeValueImpl<String>(Attribute & attribute, UInt64 key, const String & value)
{
const auto * string_in_arena = attribute.string_arena->insert(value.data(), value.size());
setAttributeValueImpl(attribute, id, StringRef{string_in_arena, value.size()});
setAttributeValueImpl(attribute, key, StringRef{string_in_arena, value.size()});
}
void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 id, const Field & value)
void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 key, const Field & value)
{
auto type_call = [&](const auto &dictionary_attribute_type)
{
@ -494,44 +444,36 @@ void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 id, c
using AttributeType = typename Type::AttributeType;
using ResizeType = std::conditional_t<std::is_same_v<AttributeType, String>, StringRef, AttributeType>;
resize<ResizeType>(attribute, id);
resize<ResizeType>(attribute, key);
if (attribute.nullable_set)
{
if (value.isNull())
{
attribute.nullable_set->insert(id);
loaded_ids[id] = true;
attribute.nullable_set->insert(key);
loaded_keys[key] = true;
return;
}
else
{
attribute.nullable_set->erase(id);
}
}
setAttributeValueImpl<AttributeType>(attribute, id, value.get<AttributeType>());
setAttributeValueImpl<AttributeType>(attribute, key, value.get<AttributeType>());
};
callOnDictionaryAttributeType(attribute.type, type_call);
}
PaddedPODArray<UInt64> FlatDictionary::getIds() const
{
const auto ids_count = ext::size(loaded_ids);
PaddedPODArray<UInt64> ids;
ids.reserve(ids_count);
for (auto idx : ext::range(0, ids_count))
if (loaded_ids[idx])
ids.push_back(idx);
return ids;
}
BlockInputStreamPtr FlatDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const
{
return std::make_shared<DictionaryBlockInputStream>(shared_from_this(), max_block_size, getIds(), column_names);
const auto keys_count = loaded_keys.size();
PaddedPODArray<UInt64> keys;
keys.reserve(keys_count);
for (size_t key_index = 0; key_index < keys_count; ++key_index)
if (loaded_keys[key_index])
keys.push_back(key_index);
return std::make_shared<DictionaryBlockInputStream>(shared_from_this(), max_block_size, std::move(keys), column_names);
}
void registerDictionaryFlat(DictionaryFactory & factory)
@ -543,19 +485,32 @@ void registerDictionaryFlat(DictionaryFactory & factory)
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
if (dict_struct.key)
throw Exception{"'key' is not supported for dictionary of layout 'flat'", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for dictionary of layout 'flat'");
if (dict_struct.range_min || dict_struct.range_max)
throw Exception{full_name
+ ": elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
full_name);
static constexpr size_t default_initial_array_size = 1024;
static constexpr size_t default_max_array_size = 500000;
String dictionary_layout_prefix = config_prefix + ".layout" + ".flat";
FlatDictionary::Configuration configuration
{
.initial_array_size = config.getUInt64(dictionary_layout_prefix + ".initial_array_size", default_initial_array_size),
.max_array_size = config.getUInt64(dictionary_layout_prefix + ".max_array_size", default_max_array_size),
.require_nonempty = config.getBool(config_prefix + ".require_nonempty", false)
};
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<FlatDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
return std::make_unique<FlatDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, std::move(configuration));
};
factory.registerLayout("flat", create_layout, false);
}

View File

@ -26,13 +26,20 @@ namespace DB
class FlatDictionary final : public IDictionary
{
public:
struct Configuration
{
size_t initial_array_size;
size_t max_array_size;
bool require_nonempty;
};
FlatDictionary(
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
bool require_nonempty_,
BlockPtr saved_block_ = nullptr);
Configuration configuration_,
BlockPtr previously_loaded_block_ = nullptr);
std::string getTypeName() const override { return "Flat"; }
@ -48,7 +55,7 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<FlatDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
return std::make_shared<FlatDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, configuration, previously_loaded_block);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }
@ -133,7 +140,7 @@ private:
ContainerType<Float32>,
ContainerType<Float64>,
ContainerType<StringRef>>
arrays;
container;
std::unique_ptr<Arena> string_arena;
};
@ -143,54 +150,39 @@ private:
void updateData();
void loadData();
template <typename T>
void addAttributeSize(const Attribute & attribute);
void calculateBytesAllocated();
template <typename T>
static void createAttributeImpl(Attribute & attribute, const Field & null_value);
static Attribute createAttribute(const DictionaryAttribute& attribute, const Field & null_value);
Attribute createAttribute(const DictionaryAttribute& attribute, const Field & null_value);
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
void getItemsImpl(
const Attribute & attribute,
const PaddedPODArray<UInt64> & ids,
const PaddedPODArray<UInt64> & keys,
ValueSetter && set_value,
DefaultValueExtractor & default_value_extractor) const;
template <typename T>
void resize(Attribute & attribute, const UInt64 id);
void resize(Attribute & attribute, UInt64 key);
template <typename T>
void setAttributeValueImpl(Attribute & attribute, const UInt64 id, const T & value);
void setAttributeValueImpl(Attribute & attribute, UInt64 key, const T & value);
void setAttributeValue(Attribute & attribute, const UInt64 id, const Field & value);
const Attribute & getAttribute(const std::string & attribute_name) const;
template <typename ChildType, typename AncestorType>
void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
PaddedPODArray<UInt64> getIds() const;
void setAttributeValue(Attribute & attribute, UInt64 key, const Field & value);
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
const bool require_nonempty;
const Configuration configuration;
std::map<std::string, size_t> attribute_index_by_name;
std::vector<Attribute> attributes;
std::vector<bool> loaded_ids;
std::vector<bool> loaded_keys;
size_t bytes_allocated = 0;
size_t element_count = 0;
size_t bucket_count = 0;
mutable std::atomic<size_t> query_count{0};
/// TODO: Remove
BlockPtr saved_block;
BlockPtr previously_loaded_block;
};
}

View File

@ -1,9 +1,5 @@
#include "HashedDictionary.h"
#include <ext/size.h>
#include <absl/container/flat_hash_map.h>
#include <Core/Defines.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Columns/ColumnsNumber.h>
@ -46,13 +42,13 @@ HashedDictionary<dictionary_key_type, sparse>::HashedDictionary(
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
bool require_nonempty_,
BlockPtr saved_block_)
BlockPtr previously_loaded_block_)
: IDictionary(dict_id_)
, dict_struct(dict_struct_)
, source_ptr(std::move(source_ptr_))
, dict_lifetime(dict_lifetime_)
, require_nonempty(require_nonempty_)
, saved_block(std::move(saved_block_))
, previously_loaded_block(std::move(previously_loaded_block_))
{
createAttributes();
loadData();
@ -347,7 +343,7 @@ void HashedDictionary<dictionary_key_type, sparse>::createAttributes()
template <DictionaryKeyType dictionary_key_type, bool sparse>
void HashedDictionary<dictionary_key_type, sparse>::updateData()
{
if (!saved_block || saved_block->rows() == 0)
if (!previously_loaded_block || previously_loaded_block->rows() == 0)
{
auto stream = source_ptr->loadUpdatedAll();
stream->readPrefix();
@ -355,13 +351,13 @@ void HashedDictionary<dictionary_key_type, sparse>::updateData()
while (const auto block = stream->read())
{
/// We are using this to keep saved data if input stream consists of multiple blocks
if (!saved_block)
saved_block = std::make_shared<DB::Block>(block.cloneEmpty());
if (!previously_loaded_block)
previously_loaded_block = std::make_shared<DB::Block>(block.cloneEmpty());
for (const auto attribute_idx : ext::range(0, attributes.size() + 1))
{
const IColumn & update_column = *block.getByPosition(attribute_idx).column.get();
MutableColumnPtr saved_column = saved_block->getByPosition(attribute_idx).column->assumeMutable();
MutableColumnPtr saved_column = previously_loaded_block->getByPosition(attribute_idx).column->assumeMutable();
saved_column->insertRangeFrom(update_column, 0, update_column.size());
}
}
@ -369,70 +365,17 @@ void HashedDictionary<dictionary_key_type, sparse>::updateData()
}
else
{
size_t skip_keys_size_offset = dict_struct.getKeysSize();
Columns saved_block_key_columns;
saved_block_key_columns.reserve(skip_keys_size_offset);
/// Split into keys columns and attribute columns
for (size_t i = 0; i < skip_keys_size_offset; ++i)
saved_block_key_columns.emplace_back(saved_block->safeGetByPosition(i).column);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
DictionaryKeysExtractor<dictionary_key_type> saved_keys_extractor(saved_block_key_columns, arena_holder.getComplexKeyArena());
auto saved_keys_extracted_from_block = saved_keys_extractor.extractAllKeys();
auto stream = source_ptr->loadUpdatedAll();
stream->readPrefix();
while (Block block = stream->read())
{
/// TODO: Rewrite
Columns block_key_columns;
block_key_columns.reserve(skip_keys_size_offset);
/// Split into keys columns and attribute columns
for (size_t i = 0; i < skip_keys_size_offset; ++i)
block_key_columns.emplace_back(block.safeGetByPosition(i).column);
DictionaryKeysExtractor<dictionary_key_type> block_keys_extractor(saved_block_key_columns, arena_holder.getComplexKeyArena());
auto keys_extracted_from_block = block_keys_extractor.extractAllKeys();
absl::flat_hash_map<KeyType, std::vector<size_t>, DefaultHash<KeyType>> update_keys;
for (size_t row = 0; row < keys_extracted_from_block.size(); ++row)
{
auto key = keys_extracted_from_block[row];
update_keys[key].push_back(row);
}
IColumn::Filter filter(saved_keys_extracted_from_block.size());
for (size_t row = 0; row < saved_keys_extracted_from_block.size(); ++row)
{
auto key = saved_keys_extracted_from_block[row];
auto it = update_keys.find(key);
filter[row] = (it == update_keys.end());
}
auto block_columns = block.mutateColumns();
for (const auto attribute_idx : ext::range(0, attributes.size() + 1))
{
auto & column = saved_block->safeGetByPosition(attribute_idx).column;
const auto & filtered_column = column->filter(filter, -1);
block_columns[attribute_idx]->insertRangeFrom(*filtered_column.get(), 0, filtered_column->size());
}
saved_block->setColumns(std::move(block_columns));
}
stream->readSuffix();
mergeBlockWithStream<dictionary_key_type>(
dict_struct.getKeysSize(),
*previously_loaded_block,
stream);
}
if (saved_block)
if (previously_loaded_block)
{
resize(saved_block->rows());
blockToAttributes(*saved_block.get());
resize(previously_loaded_block->rows());
blockToAttributes(*previously_loaded_block.get());
}
}
@ -624,7 +567,11 @@ void HashedDictionary<dictionary_key_type, sparse>::calculateBytesAllocated()
if constexpr (sparse || std::is_same_v<AttributeValueType, Field>)
{
bytes_allocated += container.max_size() * (sizeof(KeyType) + sizeof(AttributeValueType));
/// bucket_count() - Returns table size, that includes empty and deleted
/// size() - Returns table size, w/o empty and deleted
/// and since this is sparsehash, empty cells should not be significant,
/// and since items cannot be removed from the dictionary, deleted is also not important.
bytes_allocated += container.size() * (sizeof(KeyType) + sizeof(AttributeValueType));
bucket_count = container.bucket_count();
}
else

View File

@ -41,7 +41,7 @@ public:
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
bool require_nonempty_,
BlockPtr saved_block_ = nullptr);
BlockPtr previously_loaded_block_ = nullptr);
std::string getTypeName() const override
{
@ -67,7 +67,7 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<HashedDictionary<dictionary_key_type, sparse>>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
return std::make_shared<HashedDictionary<dictionary_key_type, sparse>>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, previously_loaded_block);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }
@ -219,8 +219,7 @@ private:
size_t bucket_count = 0;
mutable std::atomic<size_t> query_count{0};
/// TODO: Remove
BlockPtr saved_block;
BlockPtr previously_loaded_block;
Arena complex_key_arena;
};

View File

@ -412,11 +412,26 @@ void FormatFactory::markOutputFormatSupportsParallelFormatting(const String & na
{
auto & target = dict[name].supports_parallel_formatting;
if (target)
throw Exception("FormatFactory: Output format " + name + " is already marked as supporting parallel formatting.", ErrorCodes::LOGICAL_ERROR);
throw Exception("FormatFactory: Output format " + name + " is already marked as supporting parallel formatting", ErrorCodes::LOGICAL_ERROR);
target = true;
}
void FormatFactory::markFormatAsColumnOriented(const String & name)
{
auto & target = dict[name].is_column_oriented;
if (target)
throw Exception("FormatFactory: Format " + name + " is already marked as column oriented", ErrorCodes::LOGICAL_ERROR);
target = true;
}
bool FormatFactory::checkIfFormatIsColumnOriented(const String & name)
{
const auto & target = getCreators(name);
return target.is_column_oriented;
}
FormatFactory & FormatFactory::instance()
{
static FormatFactory ret;

View File

@ -101,6 +101,7 @@ private:
OutputProcessorCreator output_processor_creator;
FileSegmentationEngine file_segmentation_engine;
bool supports_parallel_formatting{false};
bool is_column_oriented{false};
};
using FormatsDictionary = std::unordered_map<String, Creators>;
@ -155,6 +156,9 @@ public:
void registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator);
void markOutputFormatSupportsParallelFormatting(const String & name);
void markFormatAsColumnOriented(const String & name);
bool checkIfFormatIsColumnOriented(const String & name);
const FormatsDictionary & getAllFormats() const
{

View File

@ -97,15 +97,15 @@ public:
return false;
if (sample_columns.columns() < 3)
throw Exception{"Wrong arguments count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong arguments count");
const auto * dict_name_col = checkAndGetColumnConst<ColumnString>(sample_columns.getByPosition(0).column.get());
if (!dict_name_col)
throw Exception{"First argument of function dictGet... must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "First argument of function dictGet must be a constant string");
const auto * attr_name_col = checkAndGetColumnConst<ColumnString>(sample_columns.getByPosition(1).column.get());
if (!attr_name_col)
throw Exception{"Second argument of function dictGet... must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument of function dictGet... must be a constant string");
return getDictionary(dict_name_col->getValue<String>())->isInjective(attr_name_col->getValue<String>());
}
@ -154,16 +154,22 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() < 2)
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Wrong argument count for function {}",
getName());
if (!isString(arguments[0]))
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of first argument of function, expected a string",
arguments[0]->getName(),
getName());
if (!WhichDataType(arguments[1]).isUInt64() &&
!isTuple(arguments[1]))
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of second argument of function {} must be UInt64 or tuple(...)",
arguments[1]->getName(),
getName());
return std::make_shared<DataTypeUInt8>();
}
@ -193,31 +199,39 @@ public:
if (dictionary_key_type == DictionaryKeyType::range)
{
if (arguments.size() != 3)
throw Exception{"Wrong argument count for function " + getName()
+ " when dictionary has key type range", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Wrong argument count for function {} when dictionary has key type range",
getName());
range_col = arguments[2].column;
range_col_type = arguments[2].type;
if (!(range_col_type->isValueRepresentedByInteger() && range_col_type->getSizeOfValueInMemory() <= sizeof(Int64)))
throw Exception{"Illegal type " + range_col_type->getName() + " of fourth argument of function "
+ getName() + " must be convertible to Int64.",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of fourth argument of function {} must be convertible to Int64.",
range_col_type->getName(),
getName());
}
if (dictionary_key_type == DictionaryKeyType::simple)
{
if (!key_column_type.isUInt64())
throw Exception{"Second argument of function " + getName() + " must be " + dictionary->getStructure().getKeyDescription(),
ErrorCodes::TYPE_MISMATCH};
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument of function {} must be UInt64 when dictionary is simple. Actual type {}.",
getName(),
key_column_with_type.type->getName());
return dictionary->hasKeys({key_column}, {std::make_shared<DataTypeUInt64>()});
}
else if (dictionary_key_type == DictionaryKeyType::complex)
{
if (!key_column_type.isTuple())
throw Exception{"Second argument of function " + getName() + " must be " + dictionary->getStructure().getKeyDescription(),
ErrorCodes::TYPE_MISMATCH};
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument of function {} must be tuple when dictionary is complex. Actual type {}.",
getName(),
key_column_with_type.type->getName());
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column_full = key_column->convertToFullColumnIfConst();
@ -228,7 +242,16 @@ public:
return dictionary->hasKeys(key_columns, key_types);
}
else
{
if (!key_column_type.isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument of function {} must be UInt64 when dictionary is range. Actual type {}.",
getName(),
key_column_with_type.type->getName());
return dictionary->hasKeys({key_column, range_col}, {std::make_shared<DataTypeUInt64>(), range_col_type});
}
}
private:
@ -273,14 +296,18 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() < 3)
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Wrong argument count for function {}",
getName());
String dictionary_name;
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get()))
dictionary_name = name_col->getValue<String>();
else
throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName()
+ ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of first argument of function {}, expected a const string.",
arguments[0].type->getName(),
getName());
Strings attribute_names = getAttributeNamesFromColumn(arguments[1].column, arguments[1].type);
@ -311,17 +338,20 @@ public:
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get()))
dictionary_name = name_col->getValue<String>();
else
throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName()
+ ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of first argument of function {}, expected a const string.",
arguments[0].type->getName(),
getName());
Strings attribute_names = getAttributeNamesFromColumn(arguments[1].column, arguments[1].type);
auto dictionary = helper.getDictionary(dictionary_name);
if (!WhichDataType(arguments[2].type).isUInt64() && !isTuple(arguments[2].type))
throw Exception{"Illegal type " + arguments[2].type->getName() + " of third argument of function "
+ getName() + ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of third argument of function {}, must be UInt64 or tuple(...).",
arguments[2].type->getName(),
getName());
auto dictionary_key_type = dictionary->getKeyType();
@ -333,15 +363,20 @@ public:
if (dictionary_key_type == DictionaryKeyType::range)
{
if (current_arguments_index >= arguments.size())
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {} should be {}",
getName(),
arguments.size(),
arguments.size() + 1);
range_col = arguments[current_arguments_index].column;
range_col_type = arguments[current_arguments_index].type;
if (!(range_col_type->isValueRepresentedByInteger() && range_col_type->getSizeOfValueInMemory() <= sizeof(Int64)))
throw Exception{"Illegal type " + range_col_type->getName() + " of fourth argument of function "
+ getName() + " must be convertible to Int64.",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of fourth argument of function must be convertible to Int64.",
range_col_type->getName(),
getName());
++current_arguments_index;
}
@ -351,7 +386,11 @@ public:
if (dictionary_get_function_type == DictionaryGetFunctionType::getOrDefault)
{
if (current_arguments_index >= arguments.size())
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {} should be {}",
getName(),
arguments.size(),
arguments.size() + 1);
const auto & column_before_cast = arguments[current_arguments_index];
@ -361,10 +400,14 @@ public:
for (const auto & nested_type : nested_types)
if (nested_type->isNullable())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong argument for function ({}) default values column nullable is not supported", getName());
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Wrong argument for function {} default values column nullable is not supported",
getName());
}
else if (column_before_cast.type->isNullable())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong argument for function ({}) default values column nullable is not supported", getName());
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Wrong argument for function {} default values column nullable is not supported",
getName());
auto result_type_no_nullable = removeNullable(result_type);
@ -378,11 +421,12 @@ public:
if (!tuple_column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Wrong argument for function ({}) default values column must be tuple", getName());
"Wrong argument for function {} default values column must be tuple",
getName());
if (tuple_column->tupleSize() != attribute_names.size())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Wrong argument for function ({}) default values tuple column must contain same column size as requested attributes",
"Wrong argument for function {} default values tuple column must contain same column size as requested attributes",
getName());
default_cols = tuple_column->getColumnsCopy();
@ -406,7 +450,7 @@ public:
if (!WhichDataType(key_col_with_type.type).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function ({}) must be uint64 when dictionary is simple. Actual type ({}).",
"Third argument of function {} must be UInt64 when dictionary is simple. Actual type {}.",
getName(),
key_col_with_type.type->getName());
@ -436,7 +480,7 @@ public:
if (!isTuple(key_col_with_type.type))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function ({}) must be tuple when dictionary is complex. Actual type ({}).",
"Third argument of function {} must be tuple when dictionary is complex. Actual type {}.",
getName(),
key_col_with_type.type->getName());
@ -446,7 +490,7 @@ public:
if (!isTuple(key_col_with_type.type))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function ({}) must be tuple when dictionary is complex. Actual type ({}).",
"Third argument of function {} must be tuple when dictionary is complex. Actual type {}.",
getName(),
key_col_with_type.type->getName());
@ -479,7 +523,7 @@ public:
if (!WhichDataType(key_col_with_type.type).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function ({}) must be uint64 when dictionary is range. Actual type ({}).",
"Third argument of function {} must be UInt64 when dictionary is range. Actual type {}.",
getName(),
key_col_with_type.type->getName());
@ -505,7 +549,7 @@ public:
default_cols.front());
}
else
throw Exception{"Unknown dictionary identifier type", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown dictionary identifier type");
return result;
}
@ -525,7 +569,7 @@ private:
if (tuple_size < 1)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Tuple second argument of function ({}) must contain multiple constant string columns");
"Tuple second argument of function {} must contain multiple constant string columns");
for (size_t i = 0; i < tuple_col.tupleSize(); ++i)
{
@ -535,7 +579,7 @@ private:
if (!attribute_name_column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Tuple second argument of function ({}) must contain multiple constant string columns",
"Tuple second argument of function {} must contain multiple constant string columns",
getName());
attribute_names.emplace_back(attribute_name_column->getDataAt(0));
@ -543,7 +587,7 @@ private:
}
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type ({}) of second argument of function ({}), expected a const string or const tuple of const strings.",
"Illegal type {} of second argument of function {}, expected a const string or const tuple of const strings.",
type->getName(),
getName());
@ -852,6 +896,7 @@ private:
const FunctionDictGetNoType<DictionaryGetFunctionType::get> dictionary_get_func_impl;
const FunctionDictHas dictionary_has_func_impl;
};
/// Functions to work with hierarchies.
class FunctionDictGetHierarchy final : public IFunction
@ -879,13 +924,13 @@ private:
{
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of first argument of function ({}). Expected String. Actual type ({})",
"Illegal type of first argument of function {}. Expected String. Actual type {}",
getName(),
arguments[0]->getName());
if (!WhichDataType(arguments[1]).isUInt64())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of second argument of function ({}). Expected UInt64. Actual type ({})",
"Illegal type of second argument of function {}. Expected UInt64. Actual type {}",
getName(),
arguments[1]->getName());
@ -903,7 +948,7 @@ private:
if (!dictionary->hasHierarchy())
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Dictionary ({}) does not support hierarchy",
"Dictionary {} does not support hierarchy",
dictionary->getFullName());
ColumnPtr result = dictionary->getHierarchy(arguments[1].column, std::make_shared<DataTypeUInt64>());
@ -939,19 +984,19 @@ private:
{
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of first argument of function ({}). Expected String. Actual type ({})",
"Illegal type of first argument of function {}. Expected String. Actual type {}",
getName(),
arguments[0]->getName());
if (!WhichDataType(arguments[1]).isUInt64())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of second argument of function ({}). Expected UInt64. Actual type ({})",
"Illegal type of second argument of function {}. Expected UInt64. Actual type {}",
getName(),
arguments[1]->getName());
if (!WhichDataType(arguments[2]).isUInt64())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of third argument of function ({}). Expected UInt64. Actual type ({})",
"Illegal type of third argument of function {}. Expected UInt64. Actual type {}",
getName(),
arguments[2]->getName());
@ -968,7 +1013,9 @@ private:
auto dict = helper.getDictionary(arguments[0].column);
if (!dict->hasHierarchy())
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName());
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Dictionary {} does not support hierarchy",
dict->getFullName());
ColumnPtr res = dict->isInHierarchy(arguments[1].column, arguments[2].column, std::make_shared<DataTypeUInt64>());
@ -1004,13 +1051,13 @@ private:
{
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of first argument of function ({}). Expected String. Actual type ({})",
"Illegal type of first argument of function {}. Expected String. Actual type {}",
getName(),
arguments[0]->getName());
if (!WhichDataType(arguments[1]).isUInt64())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of second argument of function ({}). Expected UInt64. Actual type ({})",
"Illegal type of second argument of function {}. Expected UInt64. Actual type {}",
getName(),
arguments[1]->getName());
@ -1026,7 +1073,7 @@ private:
if (!dictionary->hasHierarchy())
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Dictionary ({}) does not support hierarchy",
"Dictionary {} does not support hierarchy",
dictionary->getFullName());
ColumnPtr result = dictionary->getDescendants(arguments[1].column, std::make_shared<DataTypeUInt64>(), 1);
@ -1067,27 +1114,27 @@ private:
if (arguments_size < 2 || arguments_size > 3)
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Illegal arguments size of function ({}). Expects 2 or 3 arguments size. Actual size ({})",
"Illegal arguments size of function {}. Expects 2 or 3 arguments size. Actual size {}",
getName(),
arguments_size);
}
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of first argument of function ({}). Expected const String. Actual type ({})",
"Illegal type of first argument of function {}. Expected const String. Actual type {}",
getName(),
arguments[0]->getName());
if (!WhichDataType(arguments[1]).isUInt64())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of second argument of function ({}). Expected UInt64. Actual type ({})",
"Illegal type of second argument of function {}. Expected UInt64. Actual type {}",
getName(),
arguments[1]->getName());
if (arguments.size() == 3 && !isUnsignedInteger(arguments[2]))
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of third argument of function ({}). Expected const unsigned integer. Actual type ({})",
"Illegal type of third argument of function {}. Expected const unsigned integer. Actual type {}",
getName(),
arguments[2]->getName());
}
@ -1108,7 +1155,7 @@ private:
{
if (!isColumnConst(*arguments[2].column))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of third argument of function ({}). Expected const unsigned integer.",
"Illegal type of third argument of function {}. Expected const unsigned integer.",
getName());
level = static_cast<size_t>(arguments[2].column->get64(0));
@ -1116,7 +1163,7 @@ private:
if (!dictionary->hasHierarchy())
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Dictionary ({}) does not support hierarchy",
"Dictionary {} does not support hierarchy",
dictionary->getFullName());
ColumnPtr res = dictionary->getDescendants(arguments[1].column, std::make_shared<DataTypeUInt64>(), level);

View File

@ -11,7 +11,7 @@
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
#include <Common/Arena.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
namespace DB
@ -172,7 +172,7 @@ ColumnPtr FunctionArrayReduce::executeImpl(const ColumnsWithTypeAndName & argume
}
}
SCOPE_EXIT({
SCOPE_EXIT_MEMORY_SAFE({
for (size_t i = 0; i < input_rows_count; ++i)
agg_func.destroy(places[i]);
});

View File

@ -13,7 +13,7 @@
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
#include <Common/Arena.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
namespace DB
@ -252,7 +252,7 @@ ColumnPtr FunctionArrayReduceInRanges::executeImpl(const ColumnsWithTypeAndName
}
}
SCOPE_EXIT({
SCOPE_EXIT_MEMORY_SAFE({
for (size_t j = 0; j < place_total; ++j)
agg_func.destroy(places[j]);
});
@ -331,7 +331,7 @@ ColumnPtr FunctionArrayReduceInRanges::executeImpl(const ColumnsWithTypeAndName
AggregateDataPtr place = arena->alignedAlloc(agg_func.sizeOfData(), agg_func.alignOfData());
agg_func.create(place);
SCOPE_EXIT({
SCOPE_EXIT_MEMORY_SAFE({
agg_func.destroy(place);
});

View File

@ -9,7 +9,7 @@
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
#include <Common/Arena.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
namespace DB
@ -132,7 +132,7 @@ ColumnPtr FunctionInitializeAggregation::executeImpl(const ColumnsWithTypeAndNam
}
}
SCOPE_EXIT({
SCOPE_EXIT_MEMORY_SAFE({
for (size_t i = 0; i < input_rows_count; ++i)
agg_func.destroy(places[i]);
});

View File

@ -5,7 +5,7 @@
#include <DataTypes/DataTypeAggregateFunction.h>
#include <Common/AlignedBuffer.h>
#include <Common/Arena.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
namespace DB
@ -104,7 +104,7 @@ public:
const auto & states = column_with_states->getData();
bool state_created = false;
SCOPE_EXIT({
SCOPE_EXIT_MEMORY_SAFE({
if (state_created)
agg_func.destroy(place.data());
});

View File

@ -175,7 +175,7 @@ bool PeekableReadBuffer::nextImpl()
if (checkpoint_at_end)
{
checkpoint.emplace(working_buffer.begin());
checkpoint.emplace(position());
peeked_size = 0;
checkpoint_in_own_memory = false;
}

View File

@ -1349,7 +1349,7 @@ ColumnsWithTypeAndName prepareFunctionArguments(const ActionsDAG::NodeRawConstPt
/// Create actions which calculate conjunction of selected nodes.
/// Assume conjunction nodes are predicates (and may be used as arguments of function AND).
///
/// Result actions add single column with conjunction result (it is always last in index).
/// Result actions add single column with conjunction result (it is always first in index).
/// No other columns are added or removed.
ActionsDAGPtr ActionsDAG::cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs)
{
@ -1414,6 +1414,20 @@ ActionsDAGPtr ActionsDAG::cloneActionsForConjunction(NodeRawConstPtrs conjunctio
}
}
const Node * result_predicate = nodes_mapping[*conjunction.begin()];
if (conjunction.size() > 1)
{
NodeRawConstPtrs args;
args.reserve(conjunction.size());
for (const auto * predicate : conjunction)
args.emplace_back(nodes_mapping[predicate]);
result_predicate = &actions->addFunction(func_builder_and, std::move(args), {});
}
actions->index.push_back(result_predicate);
for (const auto & col : all_inputs)
{
const Node * input;
@ -1430,19 +1444,6 @@ ActionsDAGPtr ActionsDAG::cloneActionsForConjunction(NodeRawConstPtrs conjunctio
actions->index.push_back(input);
}
const Node * result_predicate = nodes_mapping[*conjunction.begin()];
if (conjunction.size() > 1)
{
NodeRawConstPtrs args;
args.reserve(conjunction.size());
for (const auto * predicate : conjunction)
args.emplace_back(nodes_mapping[predicate]);
result_predicate = &actions->addFunction(func_builder_and, std::move(args), {});
}
actions->index.push_back(result_predicate);
return actions;
}
@ -1458,6 +1459,11 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown(
"Index for ActionsDAG does not contain filter column name {}. DAG:\n{}",
filter_name, dumpDAG());
/// If condition is constant let's do nothing.
/// It means there is nothing to push down or optimization was already applied.
if (predicate->type == ActionType::COLUMN)
return nullptr;
std::unordered_set<const Node *> allowed_nodes;
/// Get input nodes from available_inputs names.
@ -1507,7 +1513,19 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown(
node.result_name = std::move(predicate->result_name);
node.result_type = std::move(predicate->result_type);
node.column = node.result_type->createColumnConst(0, 1);
*predicate = std::move(node);
if (predicate->type != ActionType::INPUT)
*predicate = std::move(node);
else
{
/// Special case. We cannot replace input to constant inplace.
/// Because we cannot affect inputs list for actions.
/// So we just add a new constant and update index.
const auto * new_predicate = &addNode(node);
for (auto & index_node : index)
if (index_node == predicate)
index_node = new_predicate;
}
}
removeUnusedActions(false);

View File

@ -220,7 +220,7 @@ public:
/// Create actions which may calculate part of filter using only available_inputs.
/// If nothing may be calculated, returns nullptr.
/// Otherwise, return actions which inputs are from available_inputs.
/// Returned actions add single column which may be used for filter.
/// Returned actions add single column which may be used for filter. Added column will be the first one.
/// Also, replace some nodes of current inputs to constant 1 in case they are filtered.
///
/// @param all_inputs should contain inputs from previous step, which will be used for result actions.
@ -231,9 +231,9 @@ public:
/// Pushed condition: z > 0
/// GROUP BY step will transform columns `x, y, z` -> `sum(x), y, z`
/// If we just add filter step with actions `z -> z > 0` before GROUP BY,
/// columns will be transformed like `x, y, z` -> `z, z > 0, x, y` -(remove filter)-> `z, x, y`.
/// columns will be transformed like `x, y, z` -> `z > 0, z, x, y` -(remove filter)-> `z, x, y`.
/// To avoid it, add inputs from `all_inputs` list,
/// so actions `x, y, z -> x, y, z, z > 0` -(remove filter)-> `x, y, z` will not change columns order.
/// so actions `x, y, z -> z > 0, x, y, z` -(remove filter)-> `x, y, z` will not change columns order.
ActionsDAGPtr cloneActionsForFilterPushDown(
const std::string & filter_name,
bool can_remove_filter,

View File

@ -156,8 +156,7 @@ void executeQuery(
for (auto & plan : plans)
input_streams.emplace_back(plan->getCurrentDataStream());
auto header = input_streams.front().header;
auto union_step = std::make_unique<UnionStep>(std::move(input_streams), header);
auto union_step = std::make_unique<UnionStep>(std::move(input_streams));
query_plan.unitePlans(std::move(union_step), std::move(plans));
}

View File

@ -49,6 +49,7 @@
#include <Interpreters/ExternalModelsLoader.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/InterserverCredentials.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/InterserverIOHandler.h>
#include <Interpreters/SystemLog.h>
@ -322,9 +323,8 @@ struct ContextShared
String interserver_io_host; /// The host name by which this server is available for other servers.
UInt16 interserver_io_port = 0; /// and port.
String interserver_io_user;
String interserver_io_password;
String interserver_scheme; /// http or https
MultiVersion<InterserverCredentials> interserver_io_credentials;
String path; /// Path to the data directory, with a slash at the end.
String flags_path; /// Path to the directory with some control flags for server maintenance.
@ -1734,6 +1734,17 @@ bool Context::hasAuxiliaryZooKeeper(const String & name) const
return getConfigRef().has("auxiliary_zookeepers." + name);
}
InterserverCredentialsPtr Context::getInterserverCredentials()
{
return shared->interserver_io_credentials.get();
}
void Context::updateInterserverCredentials(const Poco::Util::AbstractConfiguration & config)
{
auto credentials = InterserverCredentials::make(config, "interserver_http_credentials");
shared->interserver_io_credentials.set(std::move(credentials));
}
void Context::setInterserverIOAddress(const String & host, UInt16 port)
{
shared->interserver_io_host = host;
@ -1749,17 +1760,6 @@ std::pair<String, UInt16> Context::getInterserverIOAddress() const
return { shared->interserver_io_host, shared->interserver_io_port };
}
void Context::setInterserverCredentials(const String & user_, const String & password)
{
shared->interserver_io_user = user_;
shared->interserver_io_password = password;
}
std::pair<String, String> Context::getInterserverCredentials() const
{
return { shared->interserver_io_user, shared->interserver_io_password };
}
void Context::setInterserverScheme(const String & scheme)
{
shared->interserver_scheme = scheme;

View File

@ -61,6 +61,8 @@ class AccessRightsElements;
class EmbeddedDictionaries;
class ExternalDictionariesLoader;
class ExternalModelsLoader;
class InterserverCredentials;
using InterserverCredentialsPtr = std::shared_ptr<const InterserverCredentials>;
class InterserverIOHandler;
class BackgroundSchedulePool;
class MergeList;
@ -522,8 +524,8 @@ public:
std::pair<String, UInt16> getInterserverIOAddress() const;
/// Credentials which server will use to communicate with others
void setInterserverCredentials(const String & user, const String & password);
std::pair<String, String> getInterserverCredentials() const;
void updateInterserverCredentials(const Poco::Util::AbstractConfiguration & config);
InterserverCredentialsPtr getInterserverCredentials();
/// Interserver requests scheme (http or https)
void setInterserverScheme(const String & scheme);
@ -788,6 +790,9 @@ private:
StoragePolicySelectorPtr getStoragePolicySelector(std::lock_guard<std::mutex> & lock) const;
DiskSelectorPtr getDiskSelector(std::lock_guard<std::mutex> & /* lock */) const;
/// If the password is not set, the password will not be checked
void setUserImpl(const String & name, const std::optional<String> & password, const Poco::Net::SocketAddress & address);
};

View File

@ -81,86 +81,6 @@ private:
ASTTableJoin * join = nullptr;
};
/// Collect all identifiers from ast
class IdentifiersCollector
{
public:
using ASTIdentPtr = const ASTIdentifier *;
using ASTIdentifiers = std::vector<ASTIdentPtr>;
struct Data
{
ASTIdentifiers idents;
};
static void visit(const ASTPtr & node, Data & data)
{
if (const auto * ident = node->as<ASTIdentifier>())
data.idents.push_back(ident);
}
static bool needChildVisit(const ASTPtr &, const ASTPtr &)
{
return true;
}
static ASTIdentifiers collect(const ASTPtr & node)
{
IdentifiersCollector::Data ident_data;
ConstInDepthNodeVisitor<IdentifiersCollector, true> ident_visitor(ident_data);
ident_visitor.visit(node);
return ident_data.idents;
}
};
/// Split expression `expr_1 AND expr_2 AND ... AND expr_n` into vector `[expr_1, expr_2, ..., expr_n]`
void collectConjunctions(const ASTPtr & node, std::vector<ASTPtr> & members)
{
if (const auto * func = node->as<ASTFunction>(); func && func->name == NameAnd::name)
{
for (const auto & child : func->arguments->children)
collectConjunctions(child, members);
return;
}
members.push_back(node);
}
std::vector<ASTPtr> collectConjunctions(const ASTPtr & node)
{
std::vector<ASTPtr> members;
collectConjunctions(node, members);
return members;
}
std::optional<size_t> getIdentMembership(const ASTIdentifier & ident, const std::vector<TableWithColumnNamesAndTypes> & tables)
{
std::optional<size_t> table_pos = IdentifierSemantic::getMembership(ident);
if (table_pos)
return table_pos;
return IdentifierSemantic::chooseTableColumnMatch(ident, tables, true);
}
std::optional<size_t> getIdentsMembership(const ASTPtr ast,
const std::vector<TableWithColumnNamesAndTypes> & tables,
const Aliases & aliases)
{
auto idents = IdentifiersCollector::collect(ast);
std::optional<size_t> result;
for (const auto * ident : idents)
{
/// Moving expressions that use column aliases is not supported.
if (ident->isShort() && aliases.count(ident->shortName()))
return {};
const auto pos = getIdentMembership(*ident, tables);
if (!pos)
return {};
if (result && *pos != *result)
return {};
result = pos;
}
return result;
}
bool isAllowedToRewriteCrossJoin(const ASTPtr & node, const Aliases & aliases)
{
if (node->as<ASTFunction>())
@ -193,8 +113,8 @@ std::map<size_t, std::vector<ASTPtr>> moveExpressionToJoinOn(
/// Check if the identifiers are from different joined tables.
/// If it's a self joint, tables should have aliases.
auto left_table_pos = getIdentsMembership(func->arguments->children[0], tables, aliases);
auto right_table_pos = getIdentsMembership(func->arguments->children[1], tables, aliases);
auto left_table_pos = IdentifierSemantic::getIdentsMembership(func->arguments->children[0], tables, aliases);
auto right_table_pos = IdentifierSemantic::getIdentsMembership(func->arguments->children[1], tables, aliases);
/// Identifiers from different table move to JOIN ON
if (left_table_pos && right_table_pos && *left_table_pos != *right_table_pos)

View File

@ -29,6 +29,7 @@
#include <common/logger_useful.h>
#include <random>
#include <pcg_random.hpp>
#include <ext/scope_guard_safe.h>
namespace fs = std::filesystem;
@ -820,7 +821,7 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica(
zookeeper->set(tries_to_execute_path, toString(counter + 1));
task.ops.push_back(create_shard_flag);
SCOPE_EXIT({ if (!executed_by_us && !task.ops.empty()) task.ops.pop_back(); });
SCOPE_EXIT_MEMORY({ if (!executed_by_us && !task.ops.empty()) task.ops.pop_back(); });
/// If the leader will unexpectedly changed this method will return false
/// and on the next iteration new leader will take lock

View File

@ -75,7 +75,7 @@ struct TableWithColumnNamesAndTypes
void addMaterializedColumns(const NamesAndTypesList & addition)
{
addAdditionalColumns(alias_columns, addition);
addAdditionalColumns(materialized_columns, addition);
}
private:

View File

@ -152,7 +152,7 @@ void DatabaseCatalog::loadDatabases()
/// Another background thread which drops temporary LiveViews.
/// We should start it after loadMarkedAsDroppedTables() to avoid race condition.
TemporaryLiveViewCleaner::instance().startupIfNecessary();
TemporaryLiveViewCleaner::instance().startup();
}
void DatabaseCatalog::shutdownImpl()

View File

@ -134,6 +134,8 @@ class HashJoin : public IJoin
public:
HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_sample_block, bool any_take_last_row_ = false);
const TableJoin & getTableJoin() const override { return *table_join; }
/** Add block of data from right hand of JOIN to the map.
* Returns false, if some limit was exceeded and you should not insert more data.
*/

View File

@ -14,11 +14,15 @@ class Block;
struct ExtraBlock;
using ExtraBlockPtr = std::shared_ptr<ExtraBlock>;
class TableJoin;
class IJoin
{
public:
virtual ~IJoin() = default;
virtual const TableJoin & getTableJoin() const = 0;
/// Add block of data from right hand of JOIN.
/// @returns false, if some limit was exceeded and you should not insert more data.
virtual bool addJoinedBlock(const Block & block, bool check_limits = true) = 0;

View File

@ -3,6 +3,8 @@
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/StorageID.h>
#include <Parsers/ASTFunction.h>
namespace DB
{
@ -249,4 +251,86 @@ void IdentifierSemantic::setColumnLongName(ASTIdentifier & identifier, const Dat
}
}
std::optional<size_t> IdentifierSemantic::getIdentMembership(const ASTIdentifier & ident, const std::vector<TableWithColumnNamesAndTypes> & tables)
{
std::optional<size_t> table_pos = IdentifierSemantic::getMembership(ident);
if (table_pos)
return table_pos;
return IdentifierSemantic::chooseTableColumnMatch(ident, tables, true);
}
std::optional<size_t>
IdentifierSemantic::getIdentsMembership(ASTPtr ast, const std::vector<TableWithColumnNamesAndTypes> & tables, const Aliases & aliases)
{
auto idents = IdentifiersCollector::collect(ast);
std::optional<size_t> result;
for (const auto * ident : idents)
{
/// short name clashes with alias, ambiguous
if (ident->isShort() && aliases.count(ident->shortName()))
return {};
const auto pos = getIdentMembership(*ident, tables);
if (!pos)
return {};
/// identifiers from different tables
if (result && *pos != *result)
return {};
result = pos;
}
return result;
}
IdentifiersCollector::ASTIdentifiers IdentifiersCollector::collect(const ASTPtr & node)
{
IdentifiersCollector::Data ident_data;
ConstInDepthNodeVisitor<IdentifiersCollector, true> ident_visitor(ident_data);
ident_visitor.visit(node);
return ident_data.idents;
}
bool IdentifiersCollector::needChildVisit(const ASTPtr &, const ASTPtr &)
{
return true;
}
void IdentifiersCollector::visit(const ASTPtr & node, IdentifiersCollector::Data & data)
{
if (const auto * ident = node->as<ASTIdentifier>())
data.idents.push_back(ident);
}
IdentifierMembershipCollector::IdentifierMembershipCollector(const ASTSelectQuery & select, const Context & context)
{
if (ASTPtr with = select.with())
QueryAliasesNoSubqueriesVisitor(aliases).visit(with);
QueryAliasesNoSubqueriesVisitor(aliases).visit(select.select());
tables = getDatabaseAndTablesWithColumns(getTableExpressions(select), context);
}
std::optional<size_t> IdentifierMembershipCollector::getIdentsMembership(ASTPtr ast) const
{
return IdentifierSemantic::getIdentsMembership(ast, tables, aliases);
}
static void collectConjunctions(const ASTPtr & node, std::vector<ASTPtr> & members)
{
if (const auto * func = node->as<ASTFunction>(); func && func->name == "and")
{
for (const auto & child : func->arguments->children)
collectConjunctions(child, members);
return;
}
members.push_back(node);
}
std::vector<ASTPtr> collectConjunctions(const ASTPtr & node)
{
std::vector<ASTPtr> members;
collectConjunctions(node, members);
return members;
}
}

View File

@ -2,8 +2,15 @@
#include <optional>
#include <Parsers/ASTIdentifier.h>
#include <Interpreters/Aliases.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/QueryAliasesVisitor.h>
#include <Interpreters/getHeaderForProcessingStage.h>
#include <Interpreters/getTableExpressions.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>
namespace DB
{
@ -59,9 +66,48 @@ struct IdentifierSemantic
static std::optional<size_t> chooseTableColumnMatch(const ASTIdentifier &, const TablesWithColumns & tables,
bool allow_ambiguous = false);
static std::optional<size_t> getIdentMembership(const ASTIdentifier & ident, const std::vector<TableWithColumnNamesAndTypes> & tables);
/// Collect common table membership for identifiers in expression
/// If membership cannot be established or there are several identifies from different tables, return empty optional
static std::optional<size_t>
getIdentsMembership(ASTPtr ast, const std::vector<TableWithColumnNamesAndTypes> & tables, const Aliases & aliases);
private:
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table);
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table);
};
/// Collect all identifies from AST recursively
class IdentifiersCollector
{
public:
using ASTIdentPtr = const ASTIdentifier *;
using ASTIdentifiers = std::vector<ASTIdentPtr>;
struct Data
{
ASTIdentifiers idents;
};
static void visit(const ASTPtr & node, Data & data);
static bool needChildVisit(const ASTPtr &, const ASTPtr &);
static ASTIdentifiers collect(const ASTPtr & node);
};
/// Collect identifier table membership considering aliases
class IdentifierMembershipCollector
{
public:
IdentifierMembershipCollector(const ASTSelectQuery & select, const Context & context);
std::optional<size_t> getIdentsMembership(ASTPtr ast) const;
private:
std::vector<TableWithColumnNamesAndTypes> tables;
Aliases aliases;
};
/// Split expression `expr_1 AND expr_2 AND ... AND expr_n` into vector `[expr_1, expr_2, ..., expr_n]`
std::vector<ASTPtr> collectConjunctions(const ASTPtr & node);
}

View File

@ -37,7 +37,6 @@
#include <Interpreters/replaceAliasColumnsInQuery.h>
#include <Processors/Pipe.h>
#include <Processors/QueryPlan/AddingDelayedSourceStep.h>
#include <Processors/QueryPlan/AggregatingStep.h>
#include <Processors/QueryPlan/ArrayJoinStep.h>
#include <Processors/QueryPlan/CreatingSetsStep.h>
@ -82,7 +81,7 @@
#include <Common/typeid_cast.h>
#include <Common/checkStackSize.h>
#include <ext/map.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
#include <memory>
@ -1081,26 +1080,14 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
if (expressions.hasJoin())
{
JoinPtr join = expressions.join;
QueryPlanStepPtr join_step = std::make_unique<JoinStep>(
query_plan.getCurrentDataStream(),
expressions.join);
expressions.join,
expressions.join_has_delayed_stream,
settings.max_block_size);
join_step->setStepDescription("JOIN");
query_plan.addStep(std::move(join_step));
if (expressions.join_has_delayed_stream)
{
const Block & join_result_sample = query_plan.getCurrentDataStream().header;
auto stream = std::make_shared<LazyNonJoinedBlockInputStream>(*join, join_result_sample, settings.max_block_size);
auto source = std::make_shared<SourceFromInputStream>(std::move(stream));
auto add_non_joined_rows_step = std::make_unique<AddingDelayedSourceStep>(
query_plan.getCurrentDataStream(), std::move(source));
add_non_joined_rows_step->setStepDescription("Add non-joined rows after JOIN");
query_plan.addStep(std::move(add_non_joined_rows_step));
}
}
if (expressions.hasWhere())
@ -1401,7 +1388,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
AggregateDataPtr place = state.data();
agg_count.create(place);
SCOPE_EXIT(agg_count.destroy(place));
SCOPE_EXIT_MEMORY_SAFE(agg_count.destroy(place));
agg_count.set(place, *num_rows);
@ -1624,6 +1611,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
&& !query.limitBy()
&& query.limitLength()
&& !query_analyzer->hasAggregation()
&& !query_analyzer->hasWindow()
&& limit_length <= std::numeric_limits<UInt64>::max() - limit_offset
&& limit_length + limit_offset < max_block_size)
{

View File

@ -6,6 +6,7 @@
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/queryToString.h>
#include <Processors/QueryPlan/DistinctStep.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/IQueryPlanStep.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/UnionStep.h>
@ -251,11 +252,23 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan)
{
plans[i] = std::make_unique<QueryPlan>();
nested_interpreters[i]->buildQueryPlan(*plans[i]);
if (!blocksHaveEqualStructure(plans[i]->getCurrentDataStream().header, result_header))
{
auto actions_dag = ActionsDAG::makeConvertingActions(
plans[i]->getCurrentDataStream().header.getColumnsWithTypeAndName(),
result_header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Position);
auto converting_step = std::make_unique<ExpressionStep>(plans[i]->getCurrentDataStream(), std::move(actions_dag));
converting_step->setStepDescription("Conversion before UNION");
plans[i]->addStep(std::move(converting_step));
}
data_streams[i] = plans[i]->getCurrentDataStream();
}
auto max_threads = context->getSettingsRef().max_threads;
auto union_step = std::make_unique<UnionStep>(std::move(data_streams), result_header, max_threads);
auto union_step = std::make_unique<UnionStep>(std::move(data_streams), max_threads);
query_plan.unitePlans(std::move(union_step), std::move(plans));

View File

@ -0,0 +1,87 @@
#include <Interpreters/InterserverCredentials.h>
#include <common/logger_useful.h>
#include <Common/StringUtils/StringUtils.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NO_ELEMENTS_IN_CONFIG;
}
std::unique_ptr<InterserverCredentials>
InterserverCredentials::make(const Poco::Util::AbstractConfiguration & config, const std::string & root_tag)
{
if (config.has("user") && !config.has("password"))
throw Exception("Configuration parameter interserver_http_credentials.password can't be empty", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
if (!config.has("user") && config.has("password"))
throw Exception("Configuration parameter interserver_http_credentials.user can't be empty if user specified", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
/// They both can be empty
auto user = config.getString(root_tag + ".user", "");
auto password = config.getString(root_tag + ".password", "");
auto store = parseCredentialsFromConfig(user, password, config, root_tag);
return std::make_unique<InterserverCredentials>(user, password, store);
}
InterserverCredentials::CurrentCredentials InterserverCredentials::parseCredentialsFromConfig(
const std::string & current_user_,
const std::string & current_password_,
const Poco::Util::AbstractConfiguration & config,
const std::string & root_tag)
{
auto * log = &Poco::Logger::get("InterserverCredentials");
CurrentCredentials store;
store.emplace_back(current_user_, current_password_);
if (config.getBool(root_tag + ".allow_empty", false))
{
LOG_DEBUG(log, "Allowing empty credentials");
/// Allow empty credential to support migrating from no auth
store.emplace_back("", "");
}
Poco::Util::AbstractConfiguration::Keys old_users;
config.keys(root_tag, old_users);
for (const auto & user_key : old_users)
{
if (startsWith(user_key, "old"))
{
std::string full_prefix = root_tag + "." + user_key;
std::string old_user_name = config.getString(full_prefix + ".user");
LOG_DEBUG(log, "Adding credentials for old user {}", old_user_name);
std::string old_user_password = config.getString(full_prefix + ".password");
store.emplace_back(old_user_name, old_user_password);
}
}
return store;
}
InterserverCredentials::CheckResult InterserverCredentials::isValidUser(const UserWithPassword & credentials) const
{
auto itr = std::find(all_users_store.begin(), all_users_store.end(), credentials);
if (itr == all_users_store.end())
{
if (credentials.first.empty())
return {"Server requires HTTP Basic authentication, but client doesn't provide it", false};
return {"Incorrect user or password in HTTP basic authentication: " + credentials.first, false};
}
return {"", true};
}
InterserverCredentials::CheckResult InterserverCredentials::isValidUser(const std::string & user, const std::string & password) const
{
return isValidUser(std::make_pair(user, password));
}
}

View File

@ -0,0 +1,70 @@
#pragma once
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/Exception.h>
#include <common/logger_useful.h>
#include <unordered_set>
namespace DB
{
/// InterserverCredentials implements authentication using a CurrentCredentials, which
/// is configured, e.g.
/// <interserver_http_credentials>
/// <user>admin</user>
/// <password>222</password>
/// <!-- To support mix of un/authenticated clients -->
/// <!-- <allow_empty>true</allow_empty> -->
/// <old>
/// <!-- Allow authentication using previous passwords during rotation -->
/// <user>admin</user>
/// <password>qqq</password>
/// </old>
/// <old>
/// <!-- Allow authentication using previous users during rotation -->
/// <user>johny</user>
/// <password>333</password>
/// </old>
/// </interserver_http_credentials>
class InterserverCredentials
{
public:
using UserWithPassword = std::pair<std::string, std::string>;
using CheckResult = std::pair<std::string, bool>;
using CurrentCredentials = std::vector<UserWithPassword>;
InterserverCredentials(const InterserverCredentials &) = delete;
static std::unique_ptr<InterserverCredentials> make(const Poco::Util::AbstractConfiguration & config, const std::string & root_tag);
InterserverCredentials(const std::string & current_user_, const std::string & current_password_, const CurrentCredentials & all_users_store_)
: current_user(current_user_)
, current_password(current_password_)
, all_users_store(all_users_store_)
{}
CheckResult isValidUser(const UserWithPassword & credentials) const;
CheckResult isValidUser(const std::string & user, const std::string & password) const;
std::string getUser() const { return current_user; }
std::string getPassword() const { return current_password; }
private:
std::string current_user;
std::string current_password;
/// In common situation this store contains one record
CurrentCredentials all_users_store;
static CurrentCredentials parseCredentialsFromConfig(
const std::string & current_user_,
const std::string & current_password_,
const Poco::Util::AbstractConfiguration & config,
const std::string & root_tag);
};
using InterserverCredentialsPtr = std::shared_ptr<const InterserverCredentials>;
}

View File

@ -19,6 +19,8 @@ class JoinSwitcher : public IJoin
public:
JoinSwitcher(std::shared_ptr<TableJoin> table_join_, const Block & right_sample_block_);
const TableJoin & getTableJoin() const override { return *table_join; }
/// Add block of data from right hand of JOIN into current join object.
/// If join-in-memory memory limit exceeded switches to join-on-disk and continue with it.
/// @returns false, if join-on-disk disk limit exceeded

View File

@ -23,6 +23,7 @@ class MergeJoin : public IJoin
public:
MergeJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_sample_block);
const TableJoin & getTableJoin() const override { return *table_join; }
bool addJoinedBlock(const Block & block, bool check_limits) override;
void joinBlock(Block &, ExtraBlockPtr & not_processed) override;
void joinTotals(Block &) const override;

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