Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into postgresql-protocol-with-row-policy

This commit is contained in:
kssenii 2021-04-08 14:46:56 +00:00
commit 80fc675258
143 changed files with 3145 additions and 1003 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

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

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

@ -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 |

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

@ -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

@ -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

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

@ -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

@ -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

@ -357,10 +357,6 @@ void Changelog::readChangelogAndInitWriter(size_t last_commited_log_index, size_
void Changelog::rotate(size_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;

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

@ -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());
}
}

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

@ -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

@ -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

@ -82,7 +82,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>
@ -1401,7 +1401,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 +1624,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

@ -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

@ -184,12 +184,13 @@ private:
// synchronous log flushing for SYSTEM FLUSH LOGS.
uint64_t queue_front_index = 0;
bool is_shutdown = false;
// A flag that says we must create the tables even if the queue is empty.
bool is_force_prepare_tables = false;
std::condition_variable flush_event;
// Requested to flush logs up to this index, exclusive
uint64_t requested_flush_before = 0;
uint64_t requested_flush_up_to = 0;
// Flushed log up to this index, exclusive
uint64_t flushed_before = 0;
uint64_t flushed_up_to = 0;
// Logged overflow message at this queue front index
uint64_t logged_queue_full_at_index = -1;
@ -267,8 +268,8 @@ void SystemLog<LogElement>::add(const LogElement & element)
// It is enough to only wake the flushing thread once, after the message
// count increases past half available size.
const uint64_t queue_end = queue_front_index + queue.size();
if (requested_flush_before < queue_end)
requested_flush_before = queue_end;
if (requested_flush_up_to < queue_end)
requested_flush_up_to = queue_end;
flush_event.notify_all();
}
@ -304,24 +305,34 @@ void SystemLog<LogElement>::add(const LogElement & element)
template <typename LogElement>
void SystemLog<LogElement>::flush(bool force)
{
std::unique_lock lock(mutex);
uint64_t this_thread_requested_offset;
if (is_shutdown)
return;
const uint64_t queue_end = queue_front_index + queue.size();
is_force_prepare_tables = force;
if (requested_flush_before < queue_end || force)
{
requested_flush_before = queue_end;
std::unique_lock lock(mutex);
if (is_shutdown)
return;
this_thread_requested_offset = queue_front_index + queue.size();
// Publish our flush request, taking care not to overwrite the requests
// made by other threads.
is_force_prepare_tables |= force;
requested_flush_up_to = std::max(requested_flush_up_to,
this_thread_requested_offset);
flush_event.notify_all();
}
LOG_DEBUG(log, "Requested flush up to offset {}",
this_thread_requested_offset);
// Use an arbitrary timeout to avoid endless waiting.
const int timeout_seconds = 60;
std::unique_lock lock(mutex);
bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds),
[&] { return flushed_before >= queue_end && !is_force_prepare_tables; });
[&] { return flushed_up_to >= this_thread_requested_offset
&& !is_force_prepare_tables; });
if (!result)
{
@ -371,6 +382,8 @@ void SystemLog<LogElement>::savingThreadFunction()
// The end index (exclusive, like std end()) of the messages we are
// going to flush.
uint64_t to_flush_end = 0;
// Should we prepare table even if there are no new messages.
bool should_prepare_tables_anyway = false;
{
std::unique_lock lock(mutex);
@ -378,7 +391,7 @@ void SystemLog<LogElement>::savingThreadFunction()
std::chrono::milliseconds(flush_interval_milliseconds),
[&] ()
{
return requested_flush_before > flushed_before || is_shutdown || is_force_prepare_tables;
return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables;
}
);
@ -389,18 +402,14 @@ void SystemLog<LogElement>::savingThreadFunction()
to_flush.resize(0);
queue.swap(to_flush);
should_prepare_tables_anyway = is_force_prepare_tables;
exit_this_thread = is_shutdown;
}
if (to_flush.empty())
{
bool force;
{
std::lock_guard lock(mutex);
force = is_force_prepare_tables;
}
if (force)
if (should_prepare_tables_anyway)
{
prepareTable();
LOG_TRACE(log, "Table created (force)");
@ -429,7 +438,8 @@ void SystemLog<LogElement>::flushImpl(const std::vector<LogElement> & to_flush,
{
try
{
LOG_TRACE(log, "Flushing system log, {} entries to flush", to_flush.size());
LOG_TRACE(log, "Flushing system log, {} entries to flush up to offset {}",
to_flush.size(), to_flush_end);
/// We check for existence of the table and create it as needed at every
/// flush. This is done to allow user to drop the table at any moment
@ -468,12 +478,12 @@ void SystemLog<LogElement>::flushImpl(const std::vector<LogElement> & to_flush,
{
std::lock_guard lock(mutex);
flushed_before = to_flush_end;
flushed_up_to = to_flush_end;
is_force_prepare_tables = false;
flush_event.notify_all();
}
LOG_TRACE(log, "Flushed system log");
LOG_TRACE(log, "Flushed system log up to offset {}", to_flush_end);
}

View File

@ -316,6 +316,8 @@ void ThreadStatus::finalizeQueryProfiler()
void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits)
{
MemoryTracker::LockExceptionInThread lock;
if (exit_if_already_detached && thread_state == ThreadState::DetachedFromQuery)
{
thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery;

View File

@ -408,7 +408,7 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTSelectQuery & sele
const TablesWithColumns & tables, const Aliases & aliases, ASTPtr & new_where_conditions)
{
const ASTTablesInSelectQueryElement * node = select_query.join();
if (!node)
if (!node || tables.size() < 2)
return;
auto & table_join = node->table_join->as<ASTTableJoin &>();

View File

@ -86,6 +86,38 @@ void WindowFrame::toString(WriteBuffer & buf) const
void WindowFrame::checkValid() const
{
// Check the validity of offsets.
if (type == WindowFrame::FrameType::Rows
|| type == WindowFrame::FrameType::Groups)
{
if (begin_type == BoundaryType::Offset
&& !((begin_offset.getType() == Field::Types::UInt64
|| begin_offset.getType() == Field::Types::Int64)
&& begin_offset.get<Int64>() >= 0
&& begin_offset.get<Int64>() < INT_MAX))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given.",
toString(type),
applyVisitor(FieldVisitorToString(), begin_offset),
Field::Types::toString(begin_offset.getType()));
}
if (end_type == BoundaryType::Offset
&& !((end_offset.getType() == Field::Types::UInt64
|| end_offset.getType() == Field::Types::Int64)
&& end_offset.get<Int64>() >= 0
&& end_offset.get<Int64>() < INT_MAX))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given.",
toString(type),
applyVisitor(FieldVisitorToString(), end_offset),
Field::Types::toString(end_offset.getType()));
}
}
// Check relative positioning of offsets.
// UNBOUNDED PRECEDING end and UNBOUNDED FOLLOWING start should have been
// forbidden at the parsing level.
assert(!(begin_type == BoundaryType::Unbounded && !begin_preceding));

View File

@ -12,21 +12,27 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
/// Rewrite original query removing joined tables from it
bool removeJoin(ASTSelectQuery & select)
bool hasJoin(const ASTSelectQuery & select)
{
const auto & tables = select.tables();
if (!tables || tables->children.size() < 2)
return false;
const auto & joined_table = tables->children[1]->as<ASTTablesInSelectQueryElement &>();
if (!joined_table.table_join)
return false;
return joined_table.table_join != nullptr;
}
/// The most simple temporary solution: leave only the first table in query.
/// TODO: we also need to remove joined columns and related functions (taking in account aliases if any).
tables->children.resize(1);
return true;
/// Rewrite original query removing joined tables from it
bool removeJoin(ASTSelectQuery & select)
{
if (hasJoin(select))
{
/// The most simple temporary solution: leave only the first table in query.
/// TODO: we also need to remove joined columns and related functions (taking in account aliases if any).
select.tables()->children.resize(1);
return true;
}
return false;
}
Block getHeaderForProcessingStage(

View File

@ -14,6 +14,7 @@ struct SelectQueryInfo;
class Context;
class ASTSelectQuery;
bool hasJoin(const ASTSelectQuery & select);
bool removeJoin(ASTSelectQuery & select);
Block getHeaderForProcessingStage(

View File

@ -103,6 +103,7 @@ SRCS(
InterpreterSystemQuery.cpp
InterpreterUseQuery.cpp
InterpreterWatchQuery.cpp
InterserverCredentials.cpp
JoinSwitcher.cpp
JoinToSubqueryTransformVisitor.cpp
JoinedTables.cpp

View File

@ -491,14 +491,14 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
if (!written && 0 == strcmp(name.c_str(), "map"))
{
settings.ostr << (settings.hilite ? hilite_operator : "") << '{' << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_operator : "") << "map(" << (settings.hilite ? hilite_none : "");
for (size_t i = 0; i < arguments->children.size(); ++i)
{
if (i != 0)
settings.ostr << ", ";
arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens);
}
settings.ostr << (settings.hilite ? hilite_operator : "") << '}' << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_operator : "") << ')' << (settings.hilite ? hilite_none : "");
written = true;
}
}

View File

@ -580,18 +580,6 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p
else if (parser_literal.parse(pos, ast_literal, expected))
{
const Field & value = ast_literal->as<ASTLiteral &>().value;
if ((node->frame.type == WindowFrame::FrameType::Rows
|| node->frame.type == WindowFrame::FrameType::Groups)
&& !(value.getType() == Field::Types::UInt64
|| (value.getType() == Field::Types::Int64
&& value.get<Int64>() >= 0)))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Frame offset for '{}' frame must be a nonnegative integer, '{}' of type '{}' given.",
WindowFrame::toString(node->frame.type),
applyVisitor(FieldVisitorToString(), value),
Field::Types::toString(value.getType()));
}
node->frame.begin_offset = value;
node->frame.begin_type = WindowFrame::BoundaryType::Offset;
}
@ -641,18 +629,6 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p
else if (parser_literal.parse(pos, ast_literal, expected))
{
const Field & value = ast_literal->as<ASTLiteral &>().value;
if ((node->frame.type == WindowFrame::FrameType::Rows
|| node->frame.type == WindowFrame::FrameType::Groups)
&& !(value.getType() == Field::Types::UInt64
|| (value.getType() == Field::Types::Int64
&& value.get<Int64>() >= 0)))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Frame offset for '{}' frame must be a nonnegative integer, '{}' of type '{}' given.",
WindowFrame::toString(node->frame.type),
applyVisitor(FieldVisitorToString(), value),
Field::Types::toString(value.getType()));
}
node->frame.end_offset = value;
node->frame.end_type = WindowFrame::BoundaryType::Offset;
}

View File

@ -1,14 +1,15 @@
#include <Processors/Executors/PipelineExecutor.h>
#include <queue>
#include <IO/WriteBufferFromString.h>
#include <Processors/printPipeline.h>
#include <Common/EventCounter.h>
#include <ext/scope_guard.h>
#include <Common/CurrentThread.h>
#include <Processors/ISource.h>
#include <Common/setThreadName.h>
#include <Common/MemoryTracker.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Processors/printPipeline.h>
#include <Processors/ISource.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/OpenTelemetrySpanLog.h>
#include <ext/scope_guard_safe.h>
#ifndef NDEBUG
#include <Common/Stopwatch.h>
@ -740,7 +741,7 @@ void PipelineExecutor::executeImpl(size_t num_threads)
bool finished_flag = false;
SCOPE_EXIT(
SCOPE_EXIT_SAFE(
if (!finished_flag)
{
finish();
@ -766,9 +767,9 @@ void PipelineExecutor::executeImpl(size_t num_threads)
if (thread_group)
CurrentThread::attachTo(thread_group);
SCOPE_EXIT(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);
try

View File

@ -5,7 +5,7 @@
#include <Processors/QueryPipeline.h>
#include <Common/setThreadName.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
namespace DB
{
@ -72,7 +72,7 @@ static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGrou
if (thread_group)
CurrentThread::attachTo(thread_group);
SCOPE_EXIT(
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);

View File

@ -101,7 +101,7 @@ void ArrowBlockInputFormat::prepareReader()
record_batch_current = 0;
}
void registerInputFormatProcessorArrow(FormatFactory &factory)
void registerInputFormatProcessorArrow(FormatFactory & factory)
{
factory.registerInputFormatProcessor(
"Arrow",
@ -112,7 +112,7 @@ void registerInputFormatProcessorArrow(FormatFactory &factory)
{
return std::make_shared<ArrowBlockInputFormat>(buf, sample, false);
});
factory.markFormatAsColumnOriented("Arrow");
factory.registerInputFormatProcessor(
"ArrowStream",
[](ReadBuffer & buf,

View File

@ -64,6 +64,7 @@ void registerInputFormatProcessorORC(FormatFactory &factory)
{
return std::make_shared<ORCBlockInputFormat>(buf, sample);
});
factory.markFormatAsColumnOriented("ORC");
}
}

View File

@ -2,14 +2,14 @@
#include <IO/ReadHelpers.h>
#include <Common/CurrentThread.h>
#include <Common/setThreadName.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
namespace DB
{
void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr thread_group)
{
SCOPE_EXIT(
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);
@ -60,7 +60,7 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr
void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupStatusPtr thread_group, size_t current_ticket_number)
{
SCOPE_EXIT(
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);

View File

@ -94,6 +94,7 @@ void registerInputFormatProcessorParquet(FormatFactory &factory)
{
return std::make_shared<ParquetBlockInputFormat>(buf, sample);
});
factory.markFormatAsColumnOriented("Parquet");
}
}

View File

@ -64,6 +64,11 @@ WindowStep::WindowStep(const DataStream & input_stream_,
void WindowStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
{
// This resize is needed for cases such as `over ()` when we don't have a
// sort node, and the input might have multiple streams. The sort node would
// have resized it.
pipeline.resize(1);
pipeline.addSimpleTransform([&](const Block & /*header*/)
{
return std::make_shared<WindowTransform>(input_header,

View File

@ -257,10 +257,9 @@ WindowTransform::WindowTransform(const Block & input_header_,
const IColumn * column = entry.column.get();
APPLY_FOR_TYPES(compareValuesWithOffset)
// Check that the offset type matches the window type.
// Convert the offsets to the ORDER BY column type. We can't just check
// that it matches, because e.g. the int literals are always (U)Int64,
// but the column might be Int8 and so on.
// that the type matches, because e.g. the int literals are always
// (U)Int64, but the column might be Int8 and so on.
if (window_description.frame.begin_type
== WindowFrame::BoundaryType::Offset)
{
@ -435,6 +434,9 @@ auto WindowTransform::moveRowNumberNoCheck(const RowNumber & _x, int offset) con
assertValid(x);
assert(offset <= 0);
// abs(offset) is less than INT_MAX, as checked in the parser, so
// this negation should always work.
assert(offset >= -INT_MAX);
if (x.row >= static_cast<uint64_t>(-offset))
{
x.row -= -offset;
@ -1500,6 +1502,12 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction
"The offset for function {} must be nonnegative, {} given",
getName(), offset);
}
if (offset > INT_MAX)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"The offset for function {} must be less than {}, {} given",
getName(), INT_MAX, offset);
}
}
const auto [target_row, offset_left] = transform->moveRowNumber(

View File

@ -51,7 +51,7 @@ void readHeaders(
if (name.size() > max_name_length)
throw Poco::Net::MessageException("Field name is too long");
if (ch != ':')
throw Poco::Net::MessageException("Field name is invalid or no colon found");
throw Poco::Net::MessageException(fmt::format("Field name is invalid or no colon found: \"{}\"", name));
}
in.ignore();

View File

@ -25,29 +25,26 @@ namespace ErrorCodes
std::pair<String, bool> InterserverIOHTTPHandler::checkAuthentication(HTTPServerRequest & request) const
{
const auto & config = server.config();
if (config.has("interserver_http_credentials.user"))
auto server_credentials = server.context().getInterserverCredentials();
if (server_credentials)
{
if (!request.hasCredentials())
return {"Server requires HTTP Basic authentication, but client doesn't provide it", false};
return server_credentials->isValidUser("", "");
String scheme, info;
request.getCredentials(scheme, info);
if (scheme != "Basic")
return {"Server requires HTTP Basic authentication but client provides another method", false};
String user = config.getString("interserver_http_credentials.user");
String password = config.getString("interserver_http_credentials.password", "");
Poco::Net::HTTPBasicCredentials credentials(info);
if (std::make_pair(user, password) != std::make_pair(credentials.getUsername(), credentials.getPassword()))
return {"Incorrect user or password in HTTP Basic authentication", false};
return server_credentials->isValidUser(credentials.getUsername(), credentials.getPassword());
}
else if (request.hasCredentials())
{
return {"Client requires HTTP Basic authentication, but server doesn't provide it", false};
}
return {"", true};
}

View File

@ -2,10 +2,12 @@
#include <Server/HTTP/HTTPRequestHandler.h>
#include <Common/CurrentMetrics.h>
#include <Interpreters/InterserverCredentials.h>
#include <Poco/Logger.h>
#include <memory>
#include <string>
namespace CurrentMetrics

View File

@ -104,12 +104,14 @@ namespace
size_t rows = 0;
size_t bytes = 0;
std::string header;
/// dumpStructure() of the header -- obsolete
std::string block_header_string;
Block block_header;
};
DistributedHeader readDistributedHeader(ReadBuffer & in, Poco::Logger * log)
DistributedHeader readDistributedHeader(ReadBufferFromFile & in, Poco::Logger * log)
{
DistributedHeader header;
DistributedHeader distributed_header;
UInt64 query_size;
readVarUInt(query_size, in);
@ -135,17 +137,25 @@ namespace
LOG_WARNING(log, "ClickHouse shard version is older than ClickHouse initiator version. It may lack support for new features.");
}
readStringBinary(header.insert_query, header_buf);
header.insert_settings.read(header_buf);
readStringBinary(distributed_header.insert_query, header_buf);
distributed_header.insert_settings.read(header_buf);
if (header_buf.hasPendingData())
header.client_info.read(header_buf, initiator_revision);
distributed_header.client_info.read(header_buf, initiator_revision);
if (header_buf.hasPendingData())
{
readVarUInt(header.rows, header_buf);
readVarUInt(header.bytes, header_buf);
readStringBinary(header.header, header_buf);
readVarUInt(distributed_header.rows, header_buf);
readVarUInt(distributed_header.bytes, header_buf);
readStringBinary(distributed_header.block_header_string, header_buf);
}
if (header_buf.hasPendingData())
{
NativeBlockInputStream header_block_in(header_buf, DBMS_TCP_PROTOCOL_VERSION);
distributed_header.block_header = header_block_in.read();
if (!distributed_header.block_header)
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read header from the {} batch", in.getFileName());
}
/// Add handling new data here, for example:
@ -155,20 +165,20 @@ namespace
///
/// And note that it is safe, because we have checksum and size for header.
return header;
return distributed_header;
}
if (query_size == DBMS_DISTRIBUTED_SIGNATURE_HEADER_OLD_FORMAT)
{
header.insert_settings.read(in, SettingsWriteFormat::BINARY);
readStringBinary(header.insert_query, in);
return header;
distributed_header.insert_settings.read(in, SettingsWriteFormat::BINARY);
readStringBinary(distributed_header.insert_query, in);
return distributed_header;
}
header.insert_query.resize(query_size);
in.readStrict(header.insert_query.data(), query_size);
distributed_header.insert_query.resize(query_size);
in.readStrict(distributed_header.insert_query.data(), query_size);
return header;
return distributed_header;
}
/// remote_error argument is used to decide whether some errors should be
@ -200,35 +210,58 @@ namespace
return nullptr;
}
void writeRemoteConvert(const DistributedHeader & header, RemoteBlockOutputStream & remote, ReadBufferFromFile & in, Poco::Logger * log)
void writeAndConvert(RemoteBlockOutputStream & remote, ReadBufferFromFile & in)
{
if (remote.getHeader() && header.header != remote.getHeader().dumpStructure())
CompressedReadBuffer decompressing_in(in);
NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION);
block_in.readPrefix();
while (Block block = block_in.read())
{
LOG_WARNING(log,
"Structure does not match (remote: {}, local: {}), implicit conversion will be done",
remote.getHeader().dumpStructure(), header.header);
CompressedReadBuffer decompressing_in(in);
/// Lack of header, requires to read blocks
NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION);
block_in.readPrefix();
while (Block block = block_in.read())
{
ConvertingBlockInputStream convert(
std::make_shared<OneBlockInputStream>(block),
remote.getHeader(),
ConvertingBlockInputStream::MatchColumnsMode::Name);
auto adopted_block = convert.read();
remote.write(adopted_block);
}
block_in.readSuffix();
ConvertingBlockInputStream convert(
std::make_shared<OneBlockInputStream>(block),
remote.getHeader(),
ConvertingBlockInputStream::MatchColumnsMode::Name);
auto adopted_block = convert.read();
remote.write(adopted_block);
}
else
block_in.readSuffix();
}
void writeRemoteConvert(const DistributedHeader & distributed_header, RemoteBlockOutputStream & remote, ReadBufferFromFile & in, Poco::Logger * log)
{
if (!remote.getHeader())
{
CheckingCompressedReadBuffer checking_in(in);
remote.writePrepared(checking_in);
return;
}
/// This is old format, that does not have header for the block in the file header,
/// applying ConvertingBlockInputStream in this case is not a big overhead.
///
/// Anyway we can get header only from the first block, which contain all rows anyway.
if (!distributed_header.block_header)
{
LOG_TRACE(log, "Processing batch {} with old format (no header)", in.getFileName());
writeAndConvert(remote, in);
return;
}
if (!blocksHaveEqualStructure(distributed_header.block_header, remote.getHeader()))
{
LOG_WARNING(log,
"Structure does not match (remote: {}, local: {}), implicit conversion will be done",
remote.getHeader().dumpStructure(), distributed_header.block_header.dumpStructure());
writeAndConvert(remote, in);
return;
}
CheckingCompressedReadBuffer checking_in(in);
remote.writePrepared(checking_in);
}
}
@ -498,13 +531,15 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa
CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend};
ReadBufferFromFile in(file_path);
const auto & header = readDistributedHeader(in, log);
const auto & distributed_header = readDistributedHeader(in, log);
auto connection = pool->get(timeouts, &header.insert_settings);
auto connection = pool->get(timeouts, &distributed_header.insert_settings);
RemoteBlockOutputStream remote{*connection, timeouts,
header.insert_query, header.insert_settings, header.client_info};
distributed_header.insert_query,
distributed_header.insert_settings,
distributed_header.client_info};
remote.writePrefix();
writeRemoteConvert(header, remote, in, log);
writeRemoteConvert(distributed_header, remote, in, log);
remote.writeSuffix();
}
catch (const Exception & e)
@ -523,20 +558,21 @@ struct StorageDistributedDirectoryMonitor::BatchHeader
Settings settings;
String query;
ClientInfo client_info;
String sample_block_structure;
Block header;
BatchHeader(Settings settings_, String query_, ClientInfo client_info_, String sample_block_structure_)
BatchHeader(Settings settings_, String query_, ClientInfo client_info_, Block header_)
: settings(std::move(settings_))
, query(std::move(query_))
, client_info(std::move(client_info_))
, sample_block_structure(std::move(sample_block_structure_))
, header(std::move(header_))
{
}
bool operator==(const BatchHeader & other) const
{
return std::tie(settings, query, client_info.query_kind, sample_block_structure) ==
std::tie(other.settings, other.query, other.client_info.query_kind, other.sample_block_structure);
return std::tie(settings, query, client_info.query_kind) ==
std::tie(other.settings, other.query, other.client_info.query_kind) &&
blocksHaveEqualStructure(header, other.header);
}
struct Hash
@ -545,7 +581,7 @@ struct StorageDistributedDirectoryMonitor::BatchHeader
{
SipHash hash_state;
hash_state.update(batch_header.query.data(), batch_header.query.size());
hash_state.update(batch_header.sample_block_structure.data(), batch_header.sample_block_structure.size());
batch_header.header.updateHash(hash_state);
return hash_state.get64();
}
};
@ -632,16 +668,17 @@ struct StorageDistributedDirectoryMonitor::Batch
}
ReadBufferFromFile in(file_path->second);
const auto & header = readDistributedHeader(in, parent.log);
const auto & distributed_header = readDistributedHeader(in, parent.log);
if (!remote)
{
remote = std::make_unique<RemoteBlockOutputStream>(*connection, timeouts,
header.insert_query, header.insert_settings, header.client_info);
distributed_header.insert_query,
distributed_header.insert_settings,
distributed_header.client_info);
remote->writePrefix();
}
writeRemoteConvert(header, *remote, in, parent.log);
writeRemoteConvert(distributed_header, *remote, in, parent.log);
}
if (remote)
@ -808,22 +845,27 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map
size_t total_rows = 0;
size_t total_bytes = 0;
std::string sample_block_structure;
DistributedHeader header;
Block header;
DistributedHeader distributed_header;
try
{
/// Determine metadata of the current file and check if it is not broken.
ReadBufferFromFile in{file_path};
header = readDistributedHeader(in, log);
distributed_header = readDistributedHeader(in, log);
if (header.rows)
if (distributed_header.rows)
{
total_rows += header.rows;
total_bytes += header.bytes;
sample_block_structure = header.header;
total_rows += distributed_header.rows;
total_bytes += distributed_header.bytes;
}
else
if (distributed_header.block_header)
header = distributed_header.block_header;
if (!total_rows || !header)
{
LOG_TRACE(log, "Processing batch {} with old format (no header/rows)", in.getFileName());
CompressedReadBuffer decompressing_in(in);
NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION);
block_in.readPrefix();
@ -833,8 +875,8 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map
total_rows += block.rows();
total_bytes += block.bytes();
if (sample_block_structure.empty())
sample_block_structure = block.cloneEmpty().dumpStructure();
if (!header)
header = block.cloneEmpty();
}
block_in.readSuffix();
}
@ -850,7 +892,12 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map
throw;
}
BatchHeader batch_header(std::move(header.insert_settings), std::move(header.insert_query), std::move(header.client_info), std::move(sample_block_structure));
BatchHeader batch_header(
std::move(distributed_header.insert_settings),
std::move(distributed_header.insert_query),
std::move(distributed_header.client_info),
std::move(header)
);
Batch & batch = header_to_batch.try_emplace(batch_header, *this, files).first->second;
batch.file_indices.push_back(file_idx);

View File

@ -679,7 +679,13 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std::
context.getClientInfo().write(header_buf, DBMS_TCP_PROTOCOL_VERSION);
writeVarUInt(block.rows(), header_buf);
writeVarUInt(block.bytes(), header_buf);
writeStringBinary(block.cloneEmpty().dumpStructure(), header_buf);
writeStringBinary(block.cloneEmpty().dumpStructure(), header_buf); /// obsolete
/// Write block header separately in the batch header.
/// It is required for checking does conversion is required or not.
{
NativeBlockOutputStream header_stream{header_buf, DBMS_TCP_PROTOCOL_VERSION, block.cloneEmpty()};
header_stream.write(block.cloneEmpty());
}
/// Add new fields here, for example:
/// writeVarUInt(my_new_data, header_buf);

View File

@ -1356,6 +1356,24 @@ String IMergeTreeDataPart::getUniqueId() const
return id;
}
String IMergeTreeDataPart::getZeroLevelPartBlockID() const
{
if (info.level != 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to get block id for non zero level part {}", name);
SipHash hash;
checksums.computeTotalChecksumDataOnly(hash);
union
{
char bytes[16];
UInt64 words[2];
} hash_value;
hash.get128(hash_value.bytes);
return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]);
}
bool isCompactPart(const MergeTreeDataPartPtr & data_part)
{
return (data_part && data_part->getType() == MergeTreeDataPartType::COMPACT);
@ -1372,4 +1390,3 @@ bool isInMemoryPart(const MergeTreeDataPartPtr & data_part)
}
}

View File

@ -164,6 +164,9 @@ public:
bool isEmpty() const { return rows_count == 0; }
/// Compute part block id for zero level part. Otherwise throws an exception.
String getZeroLevelPartBlockID() const;
const MergeTreeData & storage;
String name;

View File

@ -35,12 +35,14 @@ void MergeTreeBlockOutputStream::write(const Block & block)
if (!part)
continue;
storage.renameTempPartAndAdd(part, &storage.increment);
/// Part can be deduplicated, so increment counters and add to part log only if it's really added
if (storage.renameTempPartAndAdd(part, &storage.increment, nullptr, storage.getDeduplicationLog()))
{
PartLog::addNewPart(storage.global_context, part, watch.elapsed());
PartLog::addNewPart(storage.global_context, part, watch.elapsed());
/// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'.
storage.background_executor.triggerTask();
/// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'.
storage.background_executor.triggerTask();
}
}
}

View File

@ -71,6 +71,7 @@ namespace ProfileEvents
extern const Event RejectedInserts;
extern const Event DelayedInserts;
extern const Event DelayedInsertsMilliseconds;
extern const Event DuplicatedInsertedBlocks;
}
namespace CurrentMetrics
@ -2022,7 +2023,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getActivePartsToReplace(
}
bool MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction)
bool MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction, MergeTreeDeduplicationLog * deduplication_log)
{
if (out_transaction && &out_transaction->data != this)
throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.",
@ -2031,7 +2032,7 @@ bool MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrem
DataPartsVector covered_parts;
{
auto lock = lockParts();
if (!renameTempPartAndReplace(part, increment, out_transaction, lock, &covered_parts))
if (!renameTempPartAndReplace(part, increment, out_transaction, lock, &covered_parts, deduplication_log))
return false;
}
if (!covered_parts.empty())
@ -2044,7 +2045,7 @@ bool MergeTreeData::renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrem
bool MergeTreeData::renameTempPartAndReplace(
MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction,
std::unique_lock<std::mutex> & lock, DataPartsVector * out_covered_parts)
std::unique_lock<std::mutex> & lock, DataPartsVector * out_covered_parts, MergeTreeDeduplicationLog * deduplication_log)
{
if (out_transaction && &out_transaction->data != this)
throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.",
@ -2099,6 +2100,22 @@ bool MergeTreeData::renameTempPartAndReplace(
return false;
}
/// Deduplication log used only from non-replicated MergeTree. Replicated
/// tables have their own mechanism. We try to deduplicate at such deep
/// level, because only here we know real part name which is required for
/// deduplication.
if (deduplication_log)
{
String block_id = part->getZeroLevelPartBlockID();
auto res = deduplication_log->addPart(block_id, part_info);
if (!res.second)
{
ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks);
LOG_INFO(log, "Block with ID {} already exists as part {}; ignoring it", block_id, res.first.getPartName());
return false;
}
}
/// All checks are passed. Now we can rename the part on disk.
/// So, we maintain invariant: if a non-temporary part in filesystem then it is in data_parts
///
@ -2155,7 +2172,7 @@ bool MergeTreeData::renameTempPartAndReplace(
}
MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction)
MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction, MergeTreeDeduplicationLog * deduplication_log)
{
if (out_transaction && &out_transaction->data != this)
throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.",
@ -2164,7 +2181,7 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
DataPartsVector covered_parts;
{
auto lock = lockParts();
renameTempPartAndReplace(part, increment, out_transaction, lock, &covered_parts);
renameTempPartAndReplace(part, increment, out_transaction, lock, &covered_parts, deduplication_log);
}
return covered_parts;
}

View File

@ -54,6 +54,7 @@ struct CurrentlySubmergingEmergingTagger;
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
using ManyExpressionActions = std::vector<ExpressionActionsPtr>;
class MergeTreeDeduplicationLog;
namespace ErrorCodes
{
@ -447,18 +448,18 @@ public:
/// active set later with out_transaction->commit()).
/// Else, commits the part immediately.
/// Returns true if part was added. Returns false if part is covered by bigger part.
bool renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr);
bool renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr, MergeTreeDeduplicationLog * deduplication_log = nullptr);
/// The same as renameTempPartAndAdd but the block range of the part can contain existing parts.
/// Returns all parts covered by the added part (in ascending order).
/// If out_transaction == nullptr, marks covered parts as Outdated.
DataPartsVector renameTempPartAndReplace(
MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr);
MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr, MergeTreeDeduplicationLog * deduplication_log = nullptr);
/// Low-level version of previous one, doesn't lock mutex
bool renameTempPartAndReplace(
MutableDataPartPtr & part, SimpleIncrement * increment, Transaction * out_transaction, DataPartsLock & lock,
DataPartsVector * out_covered_parts = nullptr);
DataPartsVector * out_covered_parts = nullptr, MergeTreeDeduplicationLog * deduplication_log = nullptr);
/// Remove parts from working set immediately (without wait for background

View File

@ -1,5 +1,5 @@
#include <boost/rational.hpp> /// For calculations related to sampling coefficients.
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
#include <optional>
#include <unordered_set>
@ -704,7 +704,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
for (size_t part_index = 0; part_index < parts.size(); ++part_index)
pool.scheduleOrThrowOnError([&, part_index, thread_group = CurrentThread::getGroup()] {
SCOPE_EXIT(
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);

View File

@ -0,0 +1,311 @@
#include <Storages/MergeTree/MergeTreeDeduplicationLog.h>
#include <filesystem>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/trim.hpp>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
namespace DB
{
namespace
{
/// Deduplication operation part was dropped or added
enum class MergeTreeDeduplicationOp : uint8_t
{
ADD = 1,
DROP = 2,
};
/// Record for deduplication on disk
struct MergeTreeDeduplicationLogRecord
{
MergeTreeDeduplicationOp operation;
std::string part_name;
std::string block_id;
};
void writeRecord(const MergeTreeDeduplicationLogRecord & record, WriteBuffer & out)
{
writeIntText(static_cast<uint8_t>(record.operation), out);
writeChar('\t', out);
writeString(record.part_name, out);
writeChar('\t', out);
writeString(record.block_id, out);
writeChar('\n', out);
out.next();
}
void readRecord(MergeTreeDeduplicationLogRecord & record, ReadBuffer & in)
{
uint8_t op;
readIntText(op, in);
record.operation = static_cast<MergeTreeDeduplicationOp>(op);
assertChar('\t', in);
readString(record.part_name, in);
assertChar('\t', in);
readString(record.block_id, in);
assertChar('\n', in);
}
std::string getLogPath(const std::string & prefix, size_t number)
{
std::filesystem::path path(prefix);
path /= std::filesystem::path(std::string{"deduplication_log_"} + std::to_string(number) + ".txt");
return path;
}
size_t getLogNumber(const std::string & path_str)
{
std::filesystem::path path(path_str);
std::string filename = path.stem();
Strings filename_parts;
boost::split(filename_parts, filename, boost::is_any_of("_"));
return parse<size_t>(filename_parts[2]);
}
}
MergeTreeDeduplicationLog::MergeTreeDeduplicationLog(
const std::string & logs_dir_,
size_t deduplication_window_,
const MergeTreeDataFormatVersion & format_version_)
: logs_dir(logs_dir_)
, deduplication_window(deduplication_window_)
, rotate_interval(deduplication_window_ * 2) /// actually it doesn't matter
, format_version(format_version_)
, deduplication_map(deduplication_window)
{
namespace fs = std::filesystem;
if (deduplication_window != 0 && !fs::exists(logs_dir))
fs::create_directories(logs_dir);
}
void MergeTreeDeduplicationLog::load()
{
namespace fs = std::filesystem;
if (!fs::exists(logs_dir))
return;
for (const auto & p : fs::directory_iterator(logs_dir))
{
const auto & path = p.path();
auto log_number = getLogNumber(path);
existing_logs[log_number] = {path, 0};
}
/// We should know which logs are exist even in case
/// of deduplication_window = 0
if (!existing_logs.empty())
current_log_number = existing_logs.rbegin()->first;
if (deduplication_window != 0)
{
/// Order important, we load history from the begging to the end
for (auto & [log_number, desc] : existing_logs)
{
try
{
desc.entries_count = loadSingleLog(desc.path);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__, "Error while loading MergeTree deduplication log on path " + desc.path);
}
}
/// Start new log, drop previous
rotateAndDropIfNeeded();
/// Can happen in case we have unfinished log
if (!current_writer)
current_writer = std::make_unique<WriteBufferFromFile>(existing_logs.rbegin()->second.path, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY);
}
}
size_t MergeTreeDeduplicationLog::loadSingleLog(const std::string & path)
{
ReadBufferFromFile read_buf(path);
size_t total_entries = 0;
while (!read_buf.eof())
{
MergeTreeDeduplicationLogRecord record;
readRecord(record, read_buf);
if (record.operation == MergeTreeDeduplicationOp::DROP)
deduplication_map.erase(record.block_id);
else
deduplication_map.insert(record.block_id, MergeTreePartInfo::fromPartName(record.part_name, format_version));
total_entries++;
}
return total_entries;
}
void MergeTreeDeduplicationLog::rotate()
{
/// We don't deduplicate anything so we don't need any writers
if (deduplication_window == 0)
return;
current_log_number++;
auto new_path = getLogPath(logs_dir, current_log_number);
MergeTreeDeduplicationLogNameDescription log_description{new_path, 0};
existing_logs.emplace(current_log_number, log_description);
if (current_writer)
current_writer->sync();
current_writer = std::make_unique<WriteBufferFromFile>(log_description.path, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY);
}
void MergeTreeDeduplicationLog::dropOutdatedLogs()
{
size_t current_sum = 0;
size_t remove_from_value = 0;
/// Go from end to the beginning
for (auto itr = existing_logs.rbegin(); itr != existing_logs.rend(); ++itr)
{
if (current_sum > deduplication_window)
{
/// We have more logs than required, all older files (including current) can be dropped
remove_from_value = itr->first;
break;
}
auto & description = itr->second;
current_sum += description.entries_count;
}
/// If we found some logs to drop
if (remove_from_value != 0)
{
/// Go from the beginning to the end and drop all outdated logs
for (auto itr = existing_logs.begin(); itr != existing_logs.end();)
{
size_t number = itr->first;
std::filesystem::remove(itr->second.path);
itr = existing_logs.erase(itr);
if (remove_from_value == number)
break;
}
}
}
void MergeTreeDeduplicationLog::rotateAndDropIfNeeded()
{
/// If we don't have logs at all or already have enough records in current
if (existing_logs.empty() || existing_logs[current_log_number].entries_count >= rotate_interval)
{
rotate();
dropOutdatedLogs();
}
}
std::pair<MergeTreePartInfo, bool> MergeTreeDeduplicationLog::addPart(const std::string & block_id, const MergeTreePartInfo & part_info)
{
std::lock_guard lock(state_mutex);
/// We support zero case because user may want to disable deduplication with
/// ALTER MODIFY SETTING query. It's much more simpler to handle zero case
/// here then destroy whole object, check for null pointer from different
/// threads and so on.
if (deduplication_window == 0)
return std::make_pair(part_info, true);
/// If we already have this block let's deduplicate it
if (deduplication_map.contains(block_id))
{
auto info = deduplication_map.get(block_id);
return std::make_pair(info, false);
}
assert(current_writer != nullptr);
/// Create new record
MergeTreeDeduplicationLogRecord record;
record.operation = MergeTreeDeduplicationOp::ADD;
record.part_name = part_info.getPartName();
record.block_id = block_id;
/// Write it to disk
writeRecord(record, *current_writer);
/// We have one more record in current log
existing_logs[current_log_number].entries_count++;
/// Add to deduplication map
deduplication_map.insert(record.block_id, part_info);
/// Rotate and drop old logs if needed
rotateAndDropIfNeeded();
return std::make_pair(part_info, true);
}
void MergeTreeDeduplicationLog::dropPart(const MergeTreePartInfo & drop_part_info)
{
std::lock_guard lock(state_mutex);
/// We support zero case because user may want to disable deduplication with
/// ALTER MODIFY SETTING query. It's much more simpler to handle zero case
/// here then destroy whole object, check for null pointer from different
/// threads and so on.
if (deduplication_window == 0)
return;
assert(current_writer != nullptr);
for (auto itr = deduplication_map.begin(); itr != deduplication_map.end(); /* no increment here, we erasing from map */)
{
const auto & part_info = itr->value;
/// Part is covered by dropped part, let's remove it from
/// deduplication history
if (drop_part_info.contains(part_info))
{
/// Create drop record
MergeTreeDeduplicationLogRecord record;
record.operation = MergeTreeDeduplicationOp::DROP;
record.part_name = part_info.getPartName();
record.block_id = itr->key;
/// Write it to disk
writeRecord(record, *current_writer);
/// We have one more record on disk
existing_logs[current_log_number].entries_count++;
/// Increment itr before erase, otherwise it will invalidated
++itr;
/// Remove block_id from in-memory table
deduplication_map.erase(record.block_id);
/// Rotate and drop old logs if needed
rotateAndDropIfNeeded();
}
else
{
++itr;
}
}
}
void MergeTreeDeduplicationLog::setDeduplicationWindowSize(size_t deduplication_window_)
{
std::lock_guard lock(state_mutex);
deduplication_window = deduplication_window_;
rotate_interval = deduplication_window * 2;
/// If settings was set for the first time with ALTER MODIFY SETTING query
if (deduplication_window != 0 && !std::filesystem::exists(logs_dir))
std::filesystem::create_directories(logs_dir);
deduplication_map.setMaxSize(deduplication_window);
rotateAndDropIfNeeded();
/// Can happen in case we have unfinished log
if (!current_writer)
current_writer = std::make_unique<WriteBufferFromFile>(existing_logs.rbegin()->second.path, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY);
}
}

View File

@ -0,0 +1,192 @@
#pragma once
#include <Core/Types.h>
#include <common/StringRef.h>
#include <IO/WriteBufferFromFile.h>
#include <Storages/MergeTree/MergeTreePartInfo.h>
#include <map>
#include <list>
#include <mutex>
#include <string>
#include <unordered_map>
namespace DB
{
/// Description of dedupliction log
struct MergeTreeDeduplicationLogNameDescription
{
/// Path to log
std::string path;
/// How many entries we have in log
size_t entries_count;
};
/// Simple string-key HashTable with fixed size based on STL containers.
/// Preserves order using linked list and remove elements
/// on overflow in FIFO order.
template <typename V>
class LimitedOrderedHashMap
{
private:
struct ListNode
{
std::string key;
V value;
};
using Queue = std::list<ListNode>;
using IndexMap = std::unordered_map<StringRef, typename Queue::iterator, StringRefHash>;
Queue queue;
IndexMap map;
size_t max_size;
public:
using iterator = typename Queue::iterator;
using const_iterator = typename Queue::const_iterator;
using reverse_iterator = typename Queue::reverse_iterator;
using const_reverse_iterator = typename Queue::const_reverse_iterator;
explicit LimitedOrderedHashMap(size_t max_size_)
: max_size(max_size_)
{}
bool contains(const std::string & key) const
{
return map.find(key) != map.end();
}
V get(const std::string & key) const
{
return map.at(key)->value;
}
size_t size() const
{
return queue.size();
}
void setMaxSize(size_t max_size_)
{
max_size = max_size_;
while (size() > max_size)
{
map.erase(queue.front().key);
queue.pop_front();
}
}
bool erase(const std::string & key)
{
auto it = map.find(key);
if (it == map.end())
return false;
auto queue_itr = it->second;
map.erase(it);
queue.erase(queue_itr);
return true;
}
bool insert(const std::string & key, const V & value)
{
auto it = map.find(key);
if (it != map.end())
return false;
if (size() == max_size)
{
map.erase(queue.front().key);
queue.pop_front();
}
ListNode elem{key, value};
auto itr = queue.insert(queue.end(), elem);
map.emplace(itr->key, itr);
return true;
}
void clear()
{
map.clear();
queue.clear();
}
iterator begin() { return queue.begin(); }
const_iterator begin() const { return queue.cbegin(); }
iterator end() { return queue.end(); }
const_iterator end() const { return queue.cend(); }
reverse_iterator rbegin() { return queue.rbegin(); }
const_reverse_iterator rbegin() const { return queue.crbegin(); }
reverse_iterator rend() { return queue.rend(); }
const_reverse_iterator rend() const { return queue.crend(); }
};
/// Fixed-size log for deduplication in non-replicated MergeTree.
/// Stores records on disk for zero-level parts in human-readable format:
/// operation part_name partition_id_check_sum
/// 1 88_18_18_0 88_10619499460461868496_9553701830997749308
/// 2 77_14_14_0 77_15147918179036854170_6725063583757244937
/// 2 77_15_15_0 77_14977227047908934259_8047656067364802772
/// 1 77_20_20_0 77_15147918179036854170_6725063583757244937
/// Also stores them in memory in hash table with limited size.
class MergeTreeDeduplicationLog
{
public:
MergeTreeDeduplicationLog(
const std::string & logs_dir_,
size_t deduplication_window_,
const MergeTreeDataFormatVersion & format_version_);
/// Add part into in-memory hash table and to disk
/// Return true and part info if insertion was successful.
/// Otherwise, in case of duplicate, return false and previous part name with same hash (useful for logging)
std::pair<MergeTreePartInfo, bool> addPart(const std::string & block_id, const MergeTreePartInfo & part);
/// Remove all covered parts from in memory table and add DROP records to the disk
void dropPart(const MergeTreePartInfo & drop_part_info);
/// Load history from disk. Ignores broken logs.
void load();
void setDeduplicationWindowSize(size_t deduplication_window_);
private:
const std::string logs_dir;
/// Size of deduplication window
size_t deduplication_window;
/// How often we create new logs. Not very important,
/// default value equals deduplication_window * 2
size_t rotate_interval;
const MergeTreeDataFormatVersion format_version;
/// Current log number. Always growing number.
size_t current_log_number = 0;
/// All existing logs in order of their numbers
std::map<size_t, MergeTreeDeduplicationLogNameDescription> existing_logs;
/// In memory hash-table
LimitedOrderedHashMap<MergeTreePartInfo> deduplication_map;
/// Writer to the current log file
std::unique_ptr<WriteBufferFromFile> current_writer;
/// Overall mutex because we can have a lot of cocurrent inserts
std::mutex state_mutex;
/// Start new log
void rotate();
/// Remove all old logs with non-needed records for deduplication_window
void dropOutdatedLogs();
/// Execute both previous methods if needed
void rotateAndDropIfNeeded();
/// Load single log from disk. In case of corruption throws exceptions
size_t loadSingleLog(const std::string & path);
};
}

View File

@ -2,6 +2,7 @@
#include <Core/Defines.h>
#include <Core/BaseSettings.h>
#include <Storages/MergeTree/MergeTreeDataFormatVersion.h>
namespace Poco::Util
@ -54,6 +55,7 @@ struct Settings;
M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Amount of bytes, accumulated in WAL to do fsync.", 0) \
M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \
M(Bool, in_memory_parts_insert_sync, false, "If true insert of part with in-memory format will wait for fsync of WAL", 0) \
M(UInt64, non_replicated_deduplication_window, 0, "How many last blocks of hashes should be kept on disk (0 - disabled).", 0) \
\
/** Inserts settings. */ \
M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \

View File

@ -155,18 +155,9 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block)
if (deduplicate)
{
SipHash hash;
part->checksums.computeTotalChecksumDataOnly(hash);
union
{
char bytes[16];
UInt64 words[2];
} hash_value;
hash.get128(hash_value.bytes);
/// We add the hash from the data and partition identifier to deduplication ID.
/// That is, do not insert the same data to the same partition twice.
block_id = part->info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]);
block_id = part->getZeroLevelPartBlockID();
LOG_DEBUG(log, "Wrote block with ID '{}', {} rows", block_id, current_block.block.rows());
}

View File

@ -40,6 +40,8 @@ namespace ProfileEvents
extern const Event StorageBufferPassedTimeMaxThreshold;
extern const Event StorageBufferPassedRowsMaxThreshold;
extern const Event StorageBufferPassedBytesMaxThreshold;
extern const Event StorageBufferLayerLockReadersWaitMilliseconds;
extern const Event StorageBufferLayerLockWritersWaitMilliseconds;
}
namespace CurrentMetrics
@ -63,6 +65,36 @@ namespace ErrorCodes
}
std::unique_lock<std::mutex> StorageBuffer::Buffer::lockForReading() const
{
return lockImpl(/* read= */true);
}
std::unique_lock<std::mutex> StorageBuffer::Buffer::lockForWriting() const
{
return lockImpl(/* read= */false);
}
std::unique_lock<std::mutex> StorageBuffer::Buffer::tryLock() const
{
std::unique_lock lock(mutex, std::try_to_lock);
return lock;
}
std::unique_lock<std::mutex> StorageBuffer::Buffer::lockImpl(bool read) const
{
std::unique_lock lock(mutex, std::defer_lock);
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
lock.lock();
UInt64 elapsed = watch.elapsedMilliseconds();
if (read)
ProfileEvents::increment(ProfileEvents::StorageBufferLayerLockReadersWaitMilliseconds, elapsed);
else
ProfileEvents::increment(ProfileEvents::StorageBufferLayerLockWritersWaitMilliseconds, elapsed);
return lock;
}
StorageBuffer::StorageBuffer(
const StorageID & table_id_,
const ColumnsDescription & columns_,
@ -111,7 +143,7 @@ protected:
return res;
has_been_read = true;
std::lock_guard lock(buffer.mutex);
std::unique_lock lock(buffer.lockForReading());
if (!buffer.data.rows())
return res;
@ -528,7 +560,7 @@ public:
for (size_t try_no = 0; try_no < storage.num_shards; ++try_no)
{
std::unique_lock lock(storage.buffers[shard_num].mutex, std::try_to_lock);
std::unique_lock lock(storage.buffers[shard_num].tryLock());
if (lock.owns_lock())
{
@ -548,7 +580,7 @@ public:
if (!least_busy_buffer)
{
least_busy_buffer = &storage.buffers[start_shard_num];
least_busy_lock = std::unique_lock(least_busy_buffer->mutex);
least_busy_lock = least_busy_buffer->lockForWriting();
}
insertIntoBuffer(block, *least_busy_buffer);
least_busy_lock.unlock();
@ -740,9 +772,9 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc
size_t bytes = 0;
time_t time_passed = 0;
std::unique_lock lock(buffer.mutex, std::defer_lock);
std::optional<std::unique_lock<std::mutex>> lock;
if (!locked)
lock.lock();
lock.emplace(buffer.lockForReading());
block_to_write = buffer.data.cloneEmpty();
@ -910,7 +942,7 @@ void StorageBuffer::reschedule()
/// try_to_lock is also ok for background flush, since if there is
/// INSERT contended, then the reschedule will be done after
/// INSERT will be done.
std::unique_lock lock(buffer.mutex, std::try_to_lock);
std::unique_lock lock(buffer.tryLock());
if (lock.owns_lock())
{
min_first_write_time = buffer.first_write_time;
@ -967,7 +999,7 @@ std::optional<UInt64> StorageBuffer::totalRows(const Settings & settings) const
UInt64 rows = 0;
for (const auto & buffer : buffers)
{
std::lock_guard lock(buffer.mutex);
const auto lock(buffer.lockForReading());
rows += buffer.data.rows();
}
return rows + *underlying_rows;
@ -978,7 +1010,7 @@ std::optional<UInt64> StorageBuffer::totalBytes(const Settings & /*settings*/) c
UInt64 bytes = 0;
for (const auto & buffer : buffers)
{
std::lock_guard lock(buffer.mutex);
const auto lock(buffer.lockForReading());
bytes += buffer.data.allocatedBytes();
}
return bytes;

View File

@ -118,7 +118,15 @@ private:
{
time_t first_write_time = 0;
Block data;
std::unique_lock<std::mutex> lockForReading() const;
std::unique_lock<std::mutex> lockForWriting() const;
std::unique_lock<std::mutex> tryLock() const;
private:
mutable std::mutex mutex;
std::unique_lock<std::mutex> lockImpl(bool read) const;
};
/// There are `num_shards` of independent buffers.

View File

@ -22,6 +22,8 @@
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
#include <Common/parseGlobs.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/StorageInMemoryMetadata.h>
#include <fcntl.h>
#include <unistd.h>
@ -149,6 +151,11 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user
return paths;
}
bool StorageFile::isColumnOriented() const
{
return format_name != "Distributed" && FormatFactory::instance().checkIfFormatIsColumnOriented(format_name);
}
StorageFile::StorageFile(int table_fd_, CommonArguments args)
: StorageFile(args)
{
@ -227,6 +234,8 @@ static std::chrono::seconds getLockTimeout(const Context & context)
return std::chrono::seconds{lock_timeout};
}
using StorageFilePtr = std::shared_ptr<StorageFile>;
class StorageFileSource : public SourceWithProgress
{
@ -257,6 +266,18 @@ public:
return header;
}
static Block getBlockForSource(
const StorageFilePtr & storage,
const StorageMetadataPtr & metadata_snapshot,
const ColumnsDescription & columns_description,
const FilesInfoPtr & files_info)
{
if (storage->isColumnOriented())
return metadata_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical(), storage->getVirtuals(), storage->getStorageID());
else
return getHeader(metadata_snapshot, files_info->need_path_column, files_info->need_file_column);
}
StorageFileSource(
std::shared_ptr<StorageFile> storage_,
const StorageMetadataPtr & metadata_snapshot_,
@ -264,7 +285,7 @@ public:
UInt64 max_block_size_,
FilesInfoPtr files_info_,
ColumnsDescription columns_description_)
: SourceWithProgress(getHeader(metadata_snapshot_, files_info_->need_path_column, files_info_->need_file_column))
: SourceWithProgress(getBlockForSource(storage_, metadata_snapshot_, columns_description_, files_info_))
, storage(std::move(storage_))
, metadata_snapshot(metadata_snapshot_)
, files_info(std::move(files_info_))
@ -344,8 +365,16 @@ public:
}
read_buf = wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method);
auto get_block_for_format = [&]() -> Block
{
if (storage->isColumnOriented())
return metadata_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical());
return metadata_snapshot->getSampleBlock();
};
auto format = FormatFactory::instance().getInput(
storage->format_name, *read_buf, metadata_snapshot->getSampleBlock(), context, max_block_size, storage->format_settings);
storage->format_name, *read_buf, get_block_for_format(), context, max_block_size, storage->format_settings);
reader = std::make_shared<InputStreamFromInputFormat>(format);
@ -412,7 +441,6 @@ private:
std::unique_lock<std::shared_timed_mutex> unique_lock;
};
Pipe StorageFile::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
@ -457,9 +485,16 @@ Pipe StorageFile::read(
for (size_t i = 0; i < num_streams; ++i)
{
const auto get_columns_for_format = [&]() -> ColumnsDescription
{
if (isColumnOriented())
return ColumnsDescription{
metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()).getNamesAndTypesList()};
else
return metadata_snapshot->getColumns();
};
pipes.emplace_back(std::make_shared<StorageFileSource>(
this_ptr, metadata_snapshot, context, max_block_size, files_info,
metadata_snapshot->getColumns()));
this_ptr, metadata_snapshot, context, max_block_size, files_info, get_columns_for_format()));
}
return Pipe::unitePipes(std::move(pipes));

View File

@ -64,6 +64,12 @@ public:
static Strings getPathsList(const String & table_path, const String & user_files_path, const Context & context);
/// Check if the format is column-oriented.
/// Is is useful because column oriented formats could effectively skip unknown columns
/// So we can create a header of only required columns in read method and ask
/// format to read only them. Note: this hack cannot be done with ordinary formats like TSV.
bool isColumnOriented() const;
protected:
friend class StorageFileSource;
friend class StorageFileBlockOutputStream;

View File

@ -291,9 +291,10 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns(
{
Block res;
std::unordered_map<String, DataTypePtr> columns_map;
auto all_columns = getColumns().getAllWithSubcolumns();
std::unordered_map<String, DataTypePtr> columns_map;
columns_map.reserve(all_columns.size());
for (const auto & elem : all_columns)
columns_map.emplace(elem.name, elem.type);
@ -306,15 +307,11 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns(
{
auto it = columns_map.find(name);
if (it != columns_map.end())
{
res.insert({it->second->createColumn(), it->second, it->first});
}
else
{
throw Exception(
"Column " + backQuote(name) + " not found in table " + storage_id.getNameForLogs(),
"Column " + backQuote(name) + " not found in table " + (storage_id.empty() ? "" : storage_id.getNameForLogs()),
ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
}
}
return res;

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