diff --git a/.gitmodules b/.gitmodules index de7250166b8..f7dcf5f4ac1 100644 --- a/.gitmodules +++ b/.gitmodules @@ -133,7 +133,7 @@ url = https://github.com/unicode-org/icu.git [submodule "contrib/flatbuffers"] path = contrib/flatbuffers - url = https://github.com/google/flatbuffers.git + url = https://github.com/ClickHouse-Extras/flatbuffers.git [submodule "contrib/libc-headers"] path = contrib/libc-headers url = https://github.com/ClickHouse-Extras/libc-headers.git @@ -221,6 +221,9 @@ [submodule "contrib/NuRaft"] path = contrib/NuRaft url = https://github.com/ClickHouse-Extras/NuRaft.git +[submodule "contrib/nanodbc"] + path = contrib/nanodbc + url = https://github.com/ClickHouse-Extras/nanodbc.git [submodule "contrib/datasketches-cpp"] path = contrib/datasketches-cpp url = https://github.com/ClickHouse-Extras/datasketches-cpp.git diff --git a/CHANGELOG.md b/CHANGELOG.md index 43531b60267..0f895c7c482 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,155 @@ +## ClickHouse release 21.4 + +### ClickHouse release 21.4.1 2021-04-12 + +#### 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)). Not every time and date functions are working for extended range of dates. +* Added support of Kerberos authentication for preconfigured users and HTTP requests (GSS-SPNEGO). [#14995](https://github.com/ClickHouse/ClickHouse/pull/14995) ([Denis Glazachev](https://github.com/traceon)). +* 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 @@ -26,7 +178,7 @@ #### Experimental feature * Add experimental `Replicated` database engine. It replicates DDL queries across multiple hosts. [#16193](https://github.com/ClickHouse/ClickHouse/pull/16193) ([tavplubix](https://github.com/tavplubix)). -* Introduce experimental support for window functions, enabled with `allow_experimental_functions = 1`. This is a preliminary, alpha-quality implementation that is not suitable for production use and will change in backward-incompatible ways in future releases. Please see [the documentation](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/sql-reference/window-functions/index.md#experimental-window-functions) for the list of supported features. [#20337](https://github.com/ClickHouse/ClickHouse/pull/20337) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Introduce experimental support for window functions, enabled with `allow_experimental_window_functions = 1`. This is a preliminary, alpha-quality implementation that is not suitable for production use and will change in backward-incompatible ways in future releases. Please see [the documentation](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/sql-reference/window-functions/index.md#experimental-window-functions) for the list of supported features. [#20337](https://github.com/ClickHouse/ClickHouse/pull/20337) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Add the ability to backup/restore metadata files for DiskS3. [#18377](https://github.com/ClickHouse/ClickHouse/pull/18377) ([Pavel Kovalenko](https://github.com/Jokser)). #### Performance Improvement diff --git a/CMakeLists.txt b/CMakeLists.txt index c4d429c565f..736a6577660 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -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") @@ -499,6 +512,7 @@ include (cmake/find/fastops.cmake) include (cmake/find/odbc.cmake) include (cmake/find/rocksdb.cmake) include (cmake/find/libpqxx.cmake) +include (cmake/find/nanodbc.cmake) include (cmake/find/nuraft.cmake) diff --git a/base/CMakeLists.txt b/base/CMakeLists.txt index 46bd57eda12..023dcaaccae 100644 --- a/base/CMakeLists.txt +++ b/base/CMakeLists.txt @@ -8,6 +8,7 @@ add_subdirectory (loggers) add_subdirectory (pcg-random) add_subdirectory (widechar_width) add_subdirectory (readpassphrase) +add_subdirectory (bridge) if (USE_MYSQL) add_subdirectory (mysqlxx) diff --git a/base/bridge/CMakeLists.txt b/base/bridge/CMakeLists.txt new file mode 100644 index 00000000000..20b0b651677 --- /dev/null +++ b/base/bridge/CMakeLists.txt @@ -0,0 +1,7 @@ +add_library (bridge + IBridge.cpp +) + +target_include_directories (daemon PUBLIC ..) +target_link_libraries (bridge PRIVATE daemon dbms Poco::Data Poco::Data::ODBC) + diff --git a/base/bridge/IBridge.cpp b/base/bridge/IBridge.cpp new file mode 100644 index 00000000000..b1f71315fef --- /dev/null +++ b/base/bridge/IBridge.cpp @@ -0,0 +1,238 @@ +#include "IBridge.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if USE_ODBC +# include +#endif + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ARGUMENT_OUT_OF_BOUND; +} + +namespace +{ + Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, Poco::Logger * log) + { + Poco::Net::SocketAddress socket_address; + try + { + socket_address = Poco::Net::SocketAddress(host, port); + } + catch (const Poco::Net::DNSException & e) + { + const auto code = e.code(); + if (code == EAI_FAMILY +#if defined(EAI_ADDRFAMILY) + || code == EAI_ADDRFAMILY +#endif + ) + { + LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. If it is an IPv6 address and your host has disabled IPv6, then consider to specify IPv4 address to listen in element of configuration file. Example: 0.0.0.0", host, e.code(), e.message()); + } + + throw; + } + return socket_address; + } + + Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, Poco::Logger * log) + { + auto address = makeSocketAddress(host, port, log); +#if POCO_VERSION < 0x01080000 + socket.bind(address, /* reuseAddress = */ true); +#else + socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ false); +#endif + + socket.listen(/* backlog = */ 64); + + return address; + } +} + + +void IBridge::handleHelp(const std::string &, const std::string &) +{ + Poco::Util::HelpFormatter help_formatter(options()); + help_formatter.setCommand(commandName()); + help_formatter.setHeader("HTTP-proxy for odbc requests"); + help_formatter.setUsage("--http-port "); + help_formatter.format(std::cerr); + + stopOptionsProcessing(); +} + + +void IBridge::defineOptions(Poco::Util::OptionSet & options) +{ + options.addOption( + Poco::Util::Option("http-port", "", "port to listen").argument("http-port", true) .binding("http-port")); + + options.addOption( + Poco::Util::Option("listen-host", "", "hostname or address to listen, default 127.0.0.1").argument("listen-host").binding("listen-host")); + + options.addOption( + Poco::Util::Option("http-timeout", "", "http timeout for socket, default 1800").argument("http-timeout").binding("http-timeout")); + + options.addOption( + Poco::Util::Option("max-server-connections", "", "max connections to server, default 1024").argument("max-server-connections").binding("max-server-connections")); + + options.addOption( + Poco::Util::Option("keep-alive-timeout", "", "keepalive timeout, default 10").argument("keep-alive-timeout").binding("keep-alive-timeout")); + + options.addOption( + Poco::Util::Option("log-level", "", "sets log level, default info") .argument("log-level").binding("logger.level")); + + options.addOption( + Poco::Util::Option("log-path", "", "log path for all logs, default console").argument("log-path").binding("logger.log")); + + options.addOption( + Poco::Util::Option("err-log-path", "", "err log path for all logs, default no").argument("err-log-path").binding("logger.errorlog")); + + options.addOption( + Poco::Util::Option("stdout-path", "", "stdout log path, default console").argument("stdout-path").binding("logger.stdout")); + + options.addOption( + Poco::Util::Option("stderr-path", "", "stderr log path, default console").argument("stderr-path").binding("logger.stderr")); + + using Me = std::decay_t; + + options.addOption( + Poco::Util::Option("help", "", "produce this help message").binding("help").callback(Poco::Util::OptionCallback(this, &Me::handleHelp))); + + ServerApplication::defineOptions(options); // NOLINT Don't need complex BaseDaemon's .xml config +} + + +void IBridge::initialize(Application & self) +{ + BaseDaemon::closeFDs(); + is_help = config().has("help"); + + if (is_help) + return; + + config().setString("logger", bridgeName()); + + /// Redirect stdout, stderr to specified files. + /// Some libraries and sanitizers write to stderr in case of errors. + const auto stdout_path = config().getString("logger.stdout", ""); + if (!stdout_path.empty()) + { + if (!freopen(stdout_path.c_str(), "a+", stdout)) + throw Poco::OpenFileException("Cannot attach stdout to " + stdout_path); + + /// Disable buffering for stdout. + setbuf(stdout, nullptr); + } + const auto stderr_path = config().getString("logger.stderr", ""); + if (!stderr_path.empty()) + { + if (!freopen(stderr_path.c_str(), "a+", stderr)) + throw Poco::OpenFileException("Cannot attach stderr to " + stderr_path); + + /// Disable buffering for stderr. + setbuf(stderr, nullptr); + } + + buildLoggers(config(), logger(), self.commandName()); + + BaseDaemon::logRevision(); + + log = &logger(); + hostname = config().getString("listen-host", "127.0.0.1"); + port = config().getUInt("http-port"); + if (port > 0xFFFF) + throw Exception("Out of range 'http-port': " + std::to_string(port), ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + http_timeout = config().getUInt("http-timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT); + max_server_connections = config().getUInt("max-server-connections", 1024); + keep_alive_timeout = config().getUInt("keep-alive-timeout", 10); + + initializeTerminationAndSignalProcessing(); + +#if USE_ODBC + if (bridgeName() == "ODBCBridge") + Poco::Data::ODBC::Connector::registerConnector(); +#endif + + ServerApplication::initialize(self); // NOLINT +} + + +void IBridge::uninitialize() +{ + BaseDaemon::uninitialize(); +} + + +int IBridge::main(const std::vector & /*args*/) +{ + if (is_help) + return Application::EXIT_OK; + + registerFormats(); + LOG_INFO(log, "Starting up {} on host: {}, port: {}", bridgeName(), hostname, port); + + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, hostname, port, log); + socket.setReceiveTimeout(http_timeout); + socket.setSendTimeout(http_timeout); + + Poco::ThreadPool server_pool(3, max_server_connections); + + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(http_timeout); + http_params->setKeepAliveTimeout(keep_alive_timeout); + + auto shared_context = Context::createShared(); + auto context = Context::createGlobal(shared_context.get()); + context->makeGlobalContext(); + + if (config().has("query_masking_rules")) + SensitiveDataMasker::setInstance(std::make_unique(config(), "query_masking_rules")); + + auto server = HTTPServer( + context, + getHandlerFactoryPtr(context), + server_pool, + socket, + http_params); + + SCOPE_EXIT({ + LOG_DEBUG(log, "Received termination signal."); + LOG_DEBUG(log, "Waiting for current connections to close."); + + server.stop(); + + for (size_t count : ext::range(1, 6)) + { + if (server.currentConnections() == 0) + break; + LOG_DEBUG(log, "Waiting for {} connections, try {}", server.currentConnections(), count); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + } + }); + + server.start(); + LOG_INFO(log, "Listening http://{}", address.toString()); + + waitForTerminationRequest(); + return Application::EXIT_OK; +} + +} diff --git a/base/bridge/IBridge.h b/base/bridge/IBridge.h new file mode 100644 index 00000000000..c64003d9959 --- /dev/null +++ b/base/bridge/IBridge.h @@ -0,0 +1,51 @@ +#pragma once + +#include +#include +#include + +#include +#include + + +namespace DB +{ + +/// Class represents base for clickhouse-odbc-bridge and clickhouse-library-bridge servers. +/// Listens to incoming HTTP POST and GET requests on specified port and host. +/// Has two handlers '/' for all incoming POST requests and /ping for GET request about service status. +class IBridge : public BaseDaemon +{ + +public: + /// Define command line arguments + void defineOptions(Poco::Util::OptionSet & options) override; + +protected: + using HandlerFactoryPtr = std::shared_ptr; + + void initialize(Application & self) override; + + void uninitialize() override; + + int main(const std::vector & args) override; + + virtual std::string bridgeName() const = 0; + + virtual HandlerFactoryPtr getHandlerFactoryPtr(ContextPtr context) const = 0; + + size_t keep_alive_timeout; + +private: + void handleHelp(const std::string &, const std::string &); + + bool is_help; + std::string hostname; + size_t port; + std::string log_level; + size_t max_server_connections; + size_t http_timeout; + + Poco::Logger * log; +}; +} diff --git a/base/daemon/SentryWriter.cpp b/base/daemon/SentryWriter.cpp index 29430b65983..1b7d0064b99 100644 --- a/base/daemon/SentryWriter.cpp +++ b/base/daemon/SentryWriter.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include #include diff --git a/base/ext/scope_guard_safe.h b/base/ext/scope_guard_safe.h new file mode 100644 index 00000000000..7cfb3959a81 --- /dev/null +++ b/base/ext/scope_guard_safe.h @@ -0,0 +1,66 @@ +#pragma once + +#include +#include +#include + +/// 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__); \ + } \ +) diff --git a/base/mysqlxx/PoolWithFailover.cpp b/base/mysqlxx/PoolWithFailover.cpp index 5e9f70f4ac1..ea2d060e596 100644 --- a/base/mysqlxx/PoolWithFailover.cpp +++ b/base/mysqlxx/PoolWithFailover.cpp @@ -2,7 +2,6 @@ #include #include #include - #include @@ -15,9 +14,12 @@ static bool startsWith(const std::string & s, const char * prefix) using namespace mysqlxx; -PoolWithFailover::PoolWithFailover(const Poco::Util::AbstractConfiguration & config_, - const std::string & config_name_, const unsigned default_connections_, - const unsigned max_connections_, const size_t max_tries_) +PoolWithFailover::PoolWithFailover( + const Poco::Util::AbstractConfiguration & config_, + const std::string & config_name_, + const unsigned default_connections_, + const unsigned max_connections_, + const size_t max_tries_) : max_tries(max_tries_) { shareable = config_.getBool(config_name_ + ".share_connection", false); @@ -59,16 +61,38 @@ PoolWithFailover::PoolWithFailover(const Poco::Util::AbstractConfiguration & con } } -PoolWithFailover::PoolWithFailover(const std::string & config_name_, const unsigned default_connections_, - const unsigned max_connections_, const size_t max_tries_) - : PoolWithFailover{ - Poco::Util::Application::instance().config(), config_name_, - default_connections_, max_connections_, max_tries_} + +PoolWithFailover::PoolWithFailover( + const std::string & config_name_, + const unsigned default_connections_, + const unsigned max_connections_, + const size_t max_tries_) + : PoolWithFailover{Poco::Util::Application::instance().config(), + config_name_, default_connections_, max_connections_, max_tries_} { } + +PoolWithFailover::PoolWithFailover( + const std::string & database, + const RemoteDescription & addresses, + const std::string & user, + const std::string & password, + size_t max_tries_) + : max_tries(max_tries_) + , shareable(false) +{ + /// Replicas have the same priority, but traversed replicas are moved to the end of the queue. + for (const auto & [host, port] : addresses) + { + replicas_by_priority[0].emplace_back(std::make_shared(database, host, user, password, port)); + } +} + + PoolWithFailover::PoolWithFailover(const PoolWithFailover & other) - : max_tries{other.max_tries}, shareable{other.shareable} + : max_tries{other.max_tries} + , shareable{other.shareable} { if (shareable) { diff --git a/base/mysqlxx/PoolWithFailover.h b/base/mysqlxx/PoolWithFailover.h index 029fc3ebad3..5154fc3e253 100644 --- a/base/mysqlxx/PoolWithFailover.h +++ b/base/mysqlxx/PoolWithFailover.h @@ -11,6 +11,8 @@ namespace mysqlxx { /** MySQL connection pool with support of failover. + * + * For dictionary source: * Have information about replicas and their priorities. * Tries to connect to replica in an order of priority. When equal priority, choose replica with maximum time without connections. * @@ -68,42 +70,58 @@ namespace mysqlxx using PoolPtr = std::shared_ptr; using Replicas = std::vector; - /// [priority][index] -> replica. + /// [priority][index] -> replica. Highest priority is 0. using ReplicasByPriority = std::map; - ReplicasByPriority replicas_by_priority; /// Number of connection tries. size_t max_tries; /// Mutex for set of replicas. std::mutex mutex; - /// Can the Pool be shared bool shareable; public: using Entry = Pool::Entry; + using RemoteDescription = std::vector>; /** - * config_name Name of parameter in configuration file. + * * Mysql dictionary sourse related params: + * config_name Name of parameter in configuration file for dictionary source. + * + * * Mysql storage related parameters: + * replicas_description + * + * * Mutual parameters: * default_connections Number of connection in pool to each replica at start. * max_connections Maximum number of connections in pool to each replica. * max_tries_ Max number of connection tries. */ - PoolWithFailover(const std::string & config_name_, + PoolWithFailover( + const std::string & config_name_, unsigned default_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, unsigned max_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS, size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES); - PoolWithFailover(const Poco::Util::AbstractConfiguration & config_, + PoolWithFailover( + const Poco::Util::AbstractConfiguration & config_, const std::string & config_name_, unsigned default_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, unsigned max_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS, size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES); + PoolWithFailover( + const std::string & database, + const RemoteDescription & addresses, + const std::string & user, + const std::string & password, + size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES); + PoolWithFailover(const PoolWithFailover & other); /** Allocates a connection to use. */ Entry get(); }; + + using PoolWithFailoverPtr = std::shared_ptr; } diff --git a/cmake/darwin/default_libs.cmake b/cmake/darwin/default_libs.cmake index 79ac675f234..6c298106c6b 100644 --- a/cmake/darwin/default_libs.cmake +++ b/cmake/darwin/default_libs.cmake @@ -1,11 +1,11 @@ set (DEFAULT_LIBS "-nodefaultlibs") -if (NOT COMPILER_CLANG) - message (FATAL_ERROR "Darwin build is supported only for Clang") -endif () - set (DEFAULT_LIBS "${DEFAULT_LIBS} ${COVERAGE_OPTION} -lc -lm -lpthread -ldl") +if (COMPILER_GCC) + set (DEFAULT_LIBS "${DEFAULT_LIBS} -lgcc_eh") +endif () + message(STATUS "Default libraries: ${DEFAULT_LIBS}") set(CMAKE_CXX_STANDARD_LIBRARIES ${DEFAULT_LIBS}) diff --git a/cmake/find/amqpcpp.cmake b/cmake/find/amqpcpp.cmake index 4191dce26bb..e3eaaf33ddb 100644 --- a/cmake/find/amqpcpp.cmake +++ b/cmake/find/amqpcpp.cmake @@ -1,3 +1,8 @@ +if (OS_DARWIN AND COMPILER_GCC) + # AMQP-CPP requires libuv which cannot be built with GCC in macOS due to a bug: https://gcc.gnu.org/bugzilla/show_bug.cgi?id=93082 + set (ENABLE_AMQPCPP OFF CACHE INTERNAL "") +endif() + option(ENABLE_AMQPCPP "Enalbe AMQP-CPP" ${ENABLE_LIBRARIES}) if (NOT ENABLE_AMQPCPP) diff --git a/cmake/find/cassandra.cmake b/cmake/find/cassandra.cmake index 037d6c3f131..ded25a5bf41 100644 --- a/cmake/find/cassandra.cmake +++ b/cmake/find/cassandra.cmake @@ -1,3 +1,8 @@ +if (OS_DARWIN AND COMPILER_GCC) + # Cassandra requires libuv which cannot be built with GCC in macOS due to a bug: https://gcc.gnu.org/bugzilla/show_bug.cgi?id=93082 + set (ENABLE_CASSANDRA OFF CACHE INTERNAL "") +endif() + option(ENABLE_CASSANDRA "Enable Cassandra" ${ENABLE_LIBRARIES}) if (NOT ENABLE_CASSANDRA) diff --git a/cmake/find/ccache.cmake b/cmake/find/ccache.cmake index fea1f8b4c97..986c9cb5fe2 100644 --- a/cmake/find/ccache.cmake +++ b/cmake/find/ccache.cmake @@ -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 diff --git a/cmake/find/nanodbc.cmake b/cmake/find/nanodbc.cmake new file mode 100644 index 00000000000..2c913abb13e --- /dev/null +++ b/cmake/find/nanodbc.cmake @@ -0,0 +1,35 @@ +option(ENABLE_NANODBC "Enalbe nanodbc" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_NANODBC) + set (USE_ODBC 0) + return() +endif() + +if (NOT ENABLE_ODBC) + set (USE_NANODBC 0) + message (STATUS "Using nanodbc=${USE_NANODBC}") + return() +endif() + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/nanodbc/CMakeLists.txt") + message (WARNING "submodule contrib/nanodbc is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal nanodbc library") + set (USE_NANODBC 0) + return() +endif() + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/unixodbc/include") + message (ERROR "submodule contrib/unixodbc is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal unixodbc needed for nanodbc") + set (USE_NANODBC 0) + return() +endif() + +set (USE_NANODBC 1) + +set (NANODBC_LIBRARY nanodbc) + +set (NANODBC_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/nanodbc/nanodbce") + +message (STATUS "Using nanodbc=${USE_NANODBC}: ${NANODBC_INCLUDE_DIR} : ${NANODBC_LIBRARY}") +message (STATUS "Using unixodbc") diff --git a/cmake/find/nuraft.cmake b/cmake/find/nuraft.cmake index 7fa5251946e..4e5258e132f 100644 --- a/cmake/find/nuraft.cmake +++ b/cmake/find/nuraft.cmake @@ -11,7 +11,7 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/NuRaft/CMakeLists.txt") return() endif () -if (NOT OS_FREEBSD AND NOT OS_DARWIN) +if (NOT OS_FREEBSD) set (USE_NURAFT 1) set (NURAFT_LIBRARY nuraft) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 4aeb67a5085..d05177739fe 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -47,7 +47,10 @@ add_subdirectory (lz4-cmake) add_subdirectory (murmurhash) add_subdirectory (replxx-cmake) add_subdirectory (unixodbc-cmake) -add_subdirectory (xz) + +if (USE_INTERNAL_XZ_LIBRARY) + add_subdirectory (xz) +endif() add_subdirectory (poco-cmake) add_subdirectory (croaring-cmake) @@ -323,6 +326,10 @@ if (USE_LIBPQXX) add_subdirectory (libpqxx-cmake) endif() +if (USE_NANODBC) + add_subdirectory (nanodbc-cmake) +endif() + if (USE_NURAFT) add_subdirectory(nuraft-cmake) endif() diff --git a/contrib/NuRaft b/contrib/NuRaft index 241fd3754a1..d2feb5978b9 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 241fd3754a1eb4d82ab68a9a875dc99391ec9f02 +Subproject commit d2feb5978b979729a07c3ca76eaa4ab94cef4ceb diff --git a/contrib/antlr4-runtime b/contrib/antlr4-runtime index a2fa7b76e2e..672643e9a42 160000 --- a/contrib/antlr4-runtime +++ b/contrib/antlr4-runtime @@ -1 +1 @@ -Subproject commit a2fa7b76e2ee16d2ad955e9214a90bbf79da66fc +Subproject commit 672643e9a427ef803abf13bc8cb4989606553d64 diff --git a/contrib/boringssl b/contrib/boringssl index fd9ce1a0406..83c1cda8a02 160000 --- a/contrib/boringssl +++ b/contrib/boringssl @@ -1 +1 @@ -Subproject commit fd9ce1a0406f571507068b9555d0b545b8a18332 +Subproject commit 83c1cda8a0224dc817cbad2966c7ed4acc35f02a diff --git a/contrib/boringssl-cmake/CMakeLists.txt b/contrib/boringssl-cmake/CMakeLists.txt index 017a8a64c0e..adfee82dda4 100644 --- a/contrib/boringssl-cmake/CMakeLists.txt +++ b/contrib/boringssl-cmake/CMakeLists.txt @@ -16,7 +16,7 @@ endif() if(CMAKE_COMPILER_IS_GNUCXX OR CLANG) set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11 -fvisibility=hidden -fno-common -fno-exceptions -fno-rtti") - if(APPLE) + if(APPLE AND CLANG) set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") endif() diff --git a/contrib/flatbuffers b/contrib/flatbuffers index 6df40a24717..22e3ffc66d2 160000 --- a/contrib/flatbuffers +++ b/contrib/flatbuffers @@ -1 +1 @@ -Subproject commit 6df40a2471737b27271bdd9b900ab5f3aec746c7 +Subproject commit 22e3ffc66d2d7d72d1414390aa0f04ffd114a5a1 diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 73afa99f1d8..b174d4d361e 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -121,12 +121,14 @@ target_include_directories(jemalloc SYSTEM PRIVATE target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE) if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") - target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_DEBUG=1 -DJEMALLOC_PROF=1) + target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_DEBUG=1) +endif () - if (USE_UNWIND) - target_compile_definitions (jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1) - target_link_libraries (jemalloc PRIVATE unwind) - endif () +target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_PROF=1) + +if (USE_UNWIND) + target_compile_definitions (jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1) + target_link_libraries (jemalloc PRIVATE unwind) endif () target_compile_options(jemalloc PRIVATE -Wno-redundant-decls) diff --git a/contrib/libcxx b/contrib/libcxx index 8b80a151d12..2fa892f69ac 160000 --- a/contrib/libcxx +++ b/contrib/libcxx @@ -1 +1 @@ -Subproject commit 8b80a151d12b98ffe2d0c22f7cec12c3b9ff88d7 +Subproject commit 2fa892f69acbaa40f8a18c6484854a6183a34482 diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index 3b5d53cd1c0..59d23b2cd9e 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -56,6 +56,11 @@ if (USE_UNWIND) target_compile_definitions(cxx PUBLIC -DSTD_EXCEPTION_HAS_STACK_TRACE=1) endif () +# Override the deduced attribute support that causes error. +if (OS_DARWIN AND COMPILER_GCC) + add_compile_definitions(_LIBCPP_INIT_PRIORITY_MAX) +endif () + target_compile_options(cxx PUBLIC $<$:-nostdinc++>) # Third party library may have substandard code. diff --git a/contrib/librdkafka-cmake/config.h.in b/contrib/librdkafka-cmake/config.h.in index 80b6ea61b6e..b3450c40a60 100644 --- a/contrib/librdkafka-cmake/config.h.in +++ b/contrib/librdkafka-cmake/config.h.in @@ -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 diff --git a/contrib/mariadb-connector-c b/contrib/mariadb-connector-c index f4476ee7311..5f4034a3a63 160000 --- a/contrib/mariadb-connector-c +++ b/contrib/mariadb-connector-c @@ -1 +1 @@ -Subproject commit f4476ee7311b35b593750f6ae2cbdb62a4006374 +Subproject commit 5f4034a3a6376416504f17186c55fe401c6d8e5e diff --git a/contrib/nanodbc b/contrib/nanodbc new file mode 160000 index 00000000000..9fc45967551 --- /dev/null +++ b/contrib/nanodbc @@ -0,0 +1 @@ +Subproject commit 9fc459675515d491401727ec67fca38db721f28c diff --git a/contrib/nanodbc-cmake/CMakeLists.txt b/contrib/nanodbc-cmake/CMakeLists.txt new file mode 100644 index 00000000000..5de46d52a61 --- /dev/null +++ b/contrib/nanodbc-cmake/CMakeLists.txt @@ -0,0 +1,14 @@ +set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/nanodbc) + +if (NOT TARGET unixodbc) + message(FATAL_ERROR "Configuration error: unixodbc is not a target") +endif() + +set (SRCS + ${LIBRARY_DIR}/nanodbc/nanodbc.cpp +) + +add_library(nanodbc ${SRCS}) + +target_link_libraries (nanodbc PUBLIC unixodbc) +target_include_directories (nanodbc SYSTEM PUBLIC ${LIBRARY_DIR}/) diff --git a/debian/clickhouse-common-static.install b/debian/clickhouse-common-static.install index 17c955a12a9..bd65f17ad42 100644 --- a/debian/clickhouse-common-static.install +++ b/debian/clickhouse-common-static.install @@ -1,5 +1,6 @@ usr/bin/clickhouse usr/bin/clickhouse-odbc-bridge +usr/bin/clickhouse-library-bridge usr/bin/clickhouse-extract-from-config usr/share/bash-completion/completions etc/security/limits.d/clickhouse.conf diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 94c7f934f6e..6948aeb3b18 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -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"] diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index a42789c6186..cf74105fbbb 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -11,17 +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 ||: -ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: 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 \; @@ -65,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 + diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 8fd89d60f85..902929a2644 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -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. diff --git a/docker/packager/deb/build.sh b/docker/packager/deb/build.sh index 6450e21d289..c1a0b27db5d 100755 --- a/docker/packager/deb/build.sh +++ b/docker/packager/deb/build.sh @@ -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,5 +28,19 @@ then mv /build/obj-*/src/unit_tests_dbms /output/binary fi fi + +ccache --show-config ||: ccache --show-stats ||: -ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: + +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 diff --git a/docker/packager/unbundled/build.sh b/docker/packager/unbundled/build.sh index 54575ab977c..99fc34fd9f3 100755 --- a/docker/packager/unbundled/build.sh +++ b/docker/packager/unbundled/build.sh @@ -13,4 +13,3 @@ mv /*.rpm /output ||: # if exists mv /*.tgz /output ||: # if exists ccache --show-stats ||: -ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index c53bf65aa68..033c03e6281 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -300,6 +300,7 @@ function run_tests 01663_aes_msan # Depends on OpenSSL 01667_aes_args_check # Depends on OpenSSL 01776_decrypt_aead_size_check # Depends on OpenSSL + 01811_filter_by_null # Depends on OpenSSL 01281_unsucceeded_insert_select_queries_counter 01292_create_user 01294_lazy_database_concurrent diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 4bd3fa717a2..626bedb453c 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -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 diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 938d8d45ffd..1c962f1bf8f 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -19,7 +19,8 @@ RUN apt-get update \ tar \ krb5-user \ iproute2 \ - lsof + lsof \ + g++ RUN rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index e0e5e36a3d6..783e689ed01 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -31,6 +31,7 @@ RUN apt-get update \ software-properties-common \ libkrb5-dev \ krb5-user \ + g++ \ && rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml b/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml new file mode 100644 index 00000000000..d0674362709 --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml @@ -0,0 +1,23 @@ +version: '2.3' +services: + mysql2: + image: mysql:5.7 + restart: always + environment: + MYSQL_ROOT_PASSWORD: clickhouse + ports: + - 3348:3306 + mysql3: + image: mysql:5.7 + restart: always + environment: + MYSQL_ROOT_PASSWORD: clickhouse + ports: + - 3388:3306 + mysql4: + image: mysql:5.7 + restart: always + environment: + MYSQL_ROOT_PASSWORD: clickhouse + ports: + - 3368:3306 diff --git a/docker/test/integration/runner/compose/docker_compose_postgres.yml b/docker/test/integration/runner/compose/docker_compose_postgres.yml index 58ed97251fb..5657352e1b3 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres.yml @@ -11,10 +11,3 @@ services: default: aliases: - postgre-sql.local - postgres2: - image: postgres - restart: always - environment: - POSTGRES_PASSWORD: mysecretpassword - ports: - - 5441:5432 diff --git a/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml b/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml new file mode 100644 index 00000000000..d04c8a2f3a6 --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml @@ -0,0 +1,23 @@ +version: '2.3' +services: + postgres2: + image: postgres + restart: always + environment: + POSTGRES_PASSWORD: mysecretpassword + ports: + - 5421:5432 + postgres3: + image: postgres + restart: always + environment: + POSTGRES_PASSWORD: mysecretpassword + ports: + - 5441:5432 + postgres4: + image: postgres + restart: always + environment: + POSTGRES_PASSWORD: mysecretpassword + ports: + - 5461:5432 diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index c0255d3d706..bda6f5a719d 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -21,6 +21,7 @@ export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge +export CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH=/clickhouse-library-bridge export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 3594eead992..ad8ec731f23 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -108,6 +108,11 @@ zgrep -Fav "ASan doesn't fully support makecontext/swapcontext functions" > /dev || echo -e 'No sanitizer asserts\tOK' >> /test_output/test_results.tsv rm -f /test_output/tmp +# OOM +zgrep -Fa " Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \ + && echo -e 'OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \ + || echo -e 'No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv + # Logical errors zgrep -Fa "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \ && echo -e 'Logical error thrown (see clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \ @@ -118,7 +123,7 @@ zgrep -Fa "########################################" /var/log/clickhouse-server/ && echo -e 'Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Not crashed\tOK' >> /test_output/test_results.tsv -# It also checks for OOM or crash without stacktrace (printed by watchdog) +# It also checks for crash without stacktrace (printed by watchdog) zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.log > /dev/null \ && echo -e 'Fatal message in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index 76a2f647231..64ca2387029 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -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 | diff --git a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md index 1a997b6b237..818830646cb 100644 --- a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -3,7 +3,7 @@ toc_priority: 35 toc_title: AggregatingMergeTree --- -# Aggregatingmergetree {#aggregatingmergetree} +# AggregatingMergeTree {#aggregatingmergetree} The engine inherits from [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md#table_engines-mergetree), altering the logic for data parts merging. ClickHouse replaces all rows with the same primary key (or more accurately, with the same [sorting key](../../../engines/table-engines/mergetree-family/mergetree.md)) with a single row (within a one data part) that stores a combination of states of aggregate functions. diff --git a/docs/en/getting-started/example-datasets/cell-towers.md b/docs/en/getting-started/example-datasets/cell-towers.md index 76effdd4c62..7028b650ad1 100644 --- a/docs/en/getting-started/example-datasets/cell-towers.md +++ b/docs/en/getting-started/example-datasets/cell-towers.md @@ -3,31 +3,31 @@ toc_priority: 21 toc_title: Cell Towers --- -# Cell Towers +# Cell Towers {#cell-towers} This dataset is from [OpenCellid](https://www.opencellid.org/) - The world's largest Open Database of Cell Towers. -As of 2021 it contains more than 40 million records about cell towers (GSM, LTE, UMTS, etc.) around the world with their geographical coordinates and metadata (country code, network, etc). +As of 2021, it contains more than 40 million records about cell towers (GSM, LTE, UMTS, etc.) around the world with their geographical coordinates and metadata (country code, network, etc). -OpenCelliD Project is licensed under a Creative Commons Attribution-ShareAlike 4.0 International License, and we redistribute a snapshot of this dataset under the terms of the same license. The up to date version of the dataset is available to download after sign in. +OpenCelliD Project is licensed under a Creative Commons Attribution-ShareAlike 4.0 International License, and we redistribute a snapshot of this dataset under the terms of the same license. The up-to-date version of the dataset is available to download after sign in. -## Get the Dataset +## Get the Dataset {#get-the-dataset} -Download the snapshot of the dataset from Feb 2021: [https://datasets.clickhouse.tech/cell_towers.csv.xz] (729 MB). +1. Download the snapshot of the dataset from February 2021: [https://datasets.clickhouse.tech/cell_towers.csv.xz] (729 MB). -Optionally validate the integrity: +2. Validate the integrity (optional step): ``` md5sum cell_towers.csv.xz 8cf986f4a0d9f12c6f384a0e9192c908 cell_towers.csv.xz ``` -Decompress it with the following command: +3. Decompress it with the following command: ``` xz -d cell_towers.csv.xz ``` -Create a table: +4. Create a table: ``` CREATE TABLE cell_towers @@ -50,15 +50,15 @@ CREATE TABLE cell_towers ENGINE = MergeTree ORDER BY (radio, mcc, net, created); ``` -Insert the dataset: +5. Insert the dataset: ``` clickhouse-client --query "INSERT INTO cell_towers FORMAT CSVWithNames" < cell_towers.csv ``` +## Examples {#examples} -## Run some queries +1. A number of cell towers by type: -Number of cell towers by type: ``` SELECT radio, count() AS c FROM cell_towers GROUP BY radio ORDER BY c DESC @@ -73,7 +73,8 @@ SELECT radio, count() AS c FROM cell_towers GROUP BY radio ORDER BY c DESC 5 rows in set. Elapsed: 0.011 sec. Processed 43.28 million rows, 43.28 MB (3.83 billion rows/s., 3.83 GB/s.) ``` -Cell towers by mobile country code (MCC): +2. Cell towers by [mobile country code (MCC)](https://en.wikipedia.org/wiki/Mobile_country_code): + ``` SELECT mcc, count() FROM cell_towers GROUP BY mcc ORDER BY count() DESC LIMIT 10 @@ -93,28 +94,28 @@ SELECT mcc, count() FROM cell_towers GROUP BY mcc ORDER BY count() DESC LIMIT 10 10 rows in set. Elapsed: 0.019 sec. Processed 43.28 million rows, 86.55 MB (2.33 billion rows/s., 4.65 GB/s.) ``` -See the dictionary here: [https://en.wikipedia.org/wiki/Mobile_country_code](https://en.wikipedia.org/wiki/Mobile_country_code). +So, the top countries are: the USA, Germany, and Russia. -So, the top countries are USA, Germany and Russia. - -You may want to create an [External Dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts/) in ClickHouse to decode these values. +You may want to create an [External Dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) in ClickHouse to decode these values. -### Example of using `pointInPolygon` function +## Use case {#use-case} -Create a table where we will store polygons: +Using `pointInPolygon` function. + +1. Create a table where we will store polygons: ``` CREATE TEMPORARY TABLE moscow (polygon Array(Tuple(Float64, Float64))); ``` -This is a rough shape of Moscow (without "new Moscow"): +2. This is a rough shape of Moscow (without "new Moscow"): ``` INSERT INTO moscow VALUES ([(37.84172564285271, 55.78000432402266), (37.8381207618713, 55.775874525970494), (37.83979446823122, 55.775626746008065), (37.84243326983639, 55.77446586811748), (37.84262672750849, 55.771974101091104), (37.84153238623039, 55.77114545193181), (37.841124690460184, 55.76722010265554), (37.84239076983644, 55.76654891107098), (37.842283558197025, 55.76258709833121), (37.8421759312134, 55.758073999993734), (37.84198330422974, 55.75381499999371), (37.8416827275085, 55.749277102484484), (37.84157576190186, 55.74794544108413), (37.83897929098507, 55.74525257875241), (37.83739676451868, 55.74404373042019), (37.838732481460525, 55.74298009816793), (37.841183997352545, 55.743060321833575), (37.84097476190185, 55.73938799999373), (37.84048155819702, 55.73570799999372), (37.840095812164286, 55.73228210777237), (37.83983814285274, 55.73080491981639), (37.83846476321406, 55.729799917464675), (37.83835745269769, 55.72919751082619), (37.838636380279524, 55.72859509486539), (37.8395161005249, 55.727705075632784), (37.83897964285276, 55.722727886185154), (37.83862557539366, 55.72034817326636), (37.83559735744853, 55.71944437307499), (37.835370708803126, 55.71831419154461), (37.83738169402022, 55.71765218986692), (37.83823396494291, 55.71691750159089), (37.838056931213345, 55.71547311301385), (37.836812846557606, 55.71221445615604), (37.83522525396725, 55.709331054395555), (37.83269301586908, 55.70953687463627), (37.829667367706236, 55.70903403789297), (37.83311126588435, 55.70552351822608), (37.83058993121339, 55.70041317726053), (37.82983872750851, 55.69883771404813), (37.82934501586913, 55.69718947487017), (37.828926414016685, 55.69504441658371), (37.82876530422971, 55.69287499999378), (37.82894754100031, 55.690759754047335), (37.827697554878185, 55.68951421135665), (37.82447346292115, 55.68965045405069), (37.83136543914793, 55.68322046195302), (37.833554015869154, 55.67814012759211), (37.83544184655761, 55.67295011628339), (37.837480388885474, 55.6672498719639), (37.838960677246064, 55.66316274139358), (37.83926093121332, 55.66046999999383), (37.839025050262435, 55.65869897264431), (37.83670784390257, 55.65794084879904), (37.835656529083245, 55.65694309303843), (37.83704060449217, 55.65689306460552), (37.83696819873806, 55.65550363526252), (37.83760389616388, 55.65487847246661), (37.83687972750851, 55.65356745541324), (37.83515216004943, 55.65155951234079), (37.83312418518067, 55.64979413590619), (37.82801726983639, 55.64640836412121), (37.820614174591, 55.64164525405531), (37.818908190475426, 55.6421883258084), (37.81717543386075, 55.64112490388471), (37.81690987037274, 55.63916106913107), (37.815099354492155, 55.637925371757085), (37.808769150787356, 55.633798276884455), (37.80100123544311, 55.62873670012244), (37.79598013491824, 55.62554336109055), (37.78634567724606, 55.62033499605651), (37.78334147619623, 55.618768681480326), (37.77746201055901, 55.619855533402706), (37.77527329626457, 55.61909966711279), (37.77801986242668, 55.618770300976294), (37.778212973541216, 55.617257701952106), (37.77784818518065, 55.61574504433011), (37.77016867724609, 55.61148576294007), (37.760191219573976, 55.60599579539028), (37.75338926983641, 55.60227892751446), (37.746329965606634, 55.59920577639331), (37.73939925396728, 55.59631430313617), (37.73273665739439, 55.5935318803559), (37.7299954450912, 55.59350760316188), (37.7268679946899, 55.59469840523759), (37.72626726983634, 55.59229549697373), (37.7262673598022, 55.59081598950582), (37.71897193121335, 55.5877595845419), (37.70871550793456, 55.58393177431724), (37.700497489410374, 55.580917323756644), (37.69204305026244, 55.57778089778455), (37.68544477378839, 55.57815154690915), (37.68391050793454, 55.57472945079756), (37.678803592590306, 55.57328235936491), (37.6743402539673, 55.57255251445782), (37.66813862698363, 55.57216388774464), (37.617927457672096, 55.57505691895805), (37.60443099999999, 55.5757737568051), (37.599683515869145, 55.57749105910326), (37.59754177842709, 55.57796291823627), (37.59625834786988, 55.57906686095235), (37.59501783265684, 55.57746616444403), (37.593090671936025, 55.57671634534502), (37.587018007904, 55.577944600233785), (37.578692203704804, 55.57982895000019), (37.57327546607398, 55.58116294118248), (37.57385012109279, 55.581550362779), (37.57399562266922, 55.5820107079112), (37.5735356072979, 55.58226289171689), (37.57290393054962, 55.582393529795155), (37.57037722355653, 55.581919415056234), (37.5592298306885, 55.584471614867844), (37.54189249206543, 55.58867650795186), (37.5297256269836, 55.59158133551745), (37.517837865081766, 55.59443656218868), (37.51200186508174, 55.59635625174229), (37.506808949737554, 55.59907823904434), (37.49820432275389, 55.6062944994944), (37.494406071441674, 55.60967103463367), (37.494760001358024, 55.61066689753365), (37.49397137107085, 55.61220931698269), (37.49016528606031, 55.613417718449064), (37.48773249206542, 55.61530616333343), (37.47921386508177, 55.622640129112334), (37.470652153442394, 55.62993723476164), (37.46273446298218, 55.6368075123157), (37.46350692265317, 55.64068225239439), (37.46050283203121, 55.640794546982576), (37.457627470916734, 55.64118904154646), (37.450718034393326, 55.64690488145138), (37.44239252645875, 55.65397824729769), (37.434587576721185, 55.66053543155961), (37.43582144975277, 55.661693766520735), (37.43576786245721, 55.662755031737014), (37.430982915344174, 55.664610641628116), (37.428547447097685, 55.66778515273695), (37.42945134592044, 55.668633314343566), (37.42859571562949, 55.66948145750025), (37.4262836402282, 55.670813882451405), (37.418709037048295, 55.6811141674414), (37.41922139651101, 55.68235377885389), (37.419218771842885, 55.68359335082235), (37.417196501327446, 55.684375235224735), (37.41607020370478, 55.68540557585352), (37.415640857147146, 55.68686637150793), (37.414632153442334, 55.68903015131686), (37.413344899475064, 55.690896881757396), (37.41171432275391, 55.69264232162232), (37.40948282275393, 55.69455101638112), (37.40703674603271, 55.69638690385348), (37.39607169577025, 55.70451821283731), (37.38952706878662, 55.70942491932811), (37.387778313491815, 55.71149057784176), (37.39049275399779, 55.71419814298992), (37.385557272491454, 55.7155489617061), (37.38388335714726, 55.71849856042102), (37.378368238098155, 55.7292763261685), (37.37763597123337, 55.730845879211614), (37.37890062088197, 55.73167906388319), (37.37750451918789, 55.734703664681774), (37.375610832015965, 55.734851959522246), (37.3723813571472, 55.74105626086403), (37.37014935714723, 55.746115620904355), (37.36944173016362, 55.750883999993725), (37.36975304365541, 55.76335905525834), (37.37244070571134, 55.76432079697595), (37.3724259757175, 55.76636979670426), (37.369922155757884, 55.76735417953104), (37.369892695770275, 55.76823419316575), (37.370214730163575, 55.782312184391266), (37.370493611114505, 55.78436801120489), (37.37120164550783, 55.78596427165359), (37.37284851456452, 55.7874378183096), (37.37608325135799, 55.7886695054807), (37.3764587460632, 55.78947647305964), (37.37530000265506, 55.79146512926804), (37.38235915344241, 55.79899647809345), (37.384344043655396, 55.80113596939471), (37.38594269577028, 55.80322699999366), (37.38711208598329, 55.804919036911976), (37.3880239841309, 55.806610999993666), (37.38928977249147, 55.81001864976979), (37.39038389947512, 55.81348641242801), (37.39235781481933, 55.81983538336746), (37.393709457672124, 55.82417822811877), (37.394685720901464, 55.82792275755836), (37.39557615344238, 55.830447148154136), (37.39844478226658, 55.83167107969975), (37.40019761214057, 55.83151823557964), (37.400398790382326, 55.83264967594742), (37.39659544313046, 55.83322180909622), (37.39667059524539, 55.83402792148566), (37.39682089947515, 55.83638877400216), (37.39643489154053, 55.83861656112751), (37.3955338994751, 55.84072348043264), (37.392680272491454, 55.84502158126453), (37.39241188227847, 55.84659117913199), (37.392529730163616, 55.84816071336481), (37.39486835714723, 55.85288092980303), (37.39873052645878, 55.859893456073635), (37.40272161111449, 55.86441833633205), (37.40697072750854, 55.867579567544375), (37.410007082016016, 55.868369880337), (37.4120992989502, 55.86920843741314), (37.412668021163924, 55.87055369615854), (37.41482461111453, 55.87170587948249), (37.41862266137694, 55.873183961039565), (37.42413732540892, 55.874879126654704), (37.4312182698669, 55.875614937236705), (37.43111093783558, 55.8762723478417), (37.43332105622856, 55.87706546369396), (37.43385747619623, 55.87790681284802), (37.441303050262405, 55.88027084462084), (37.44747234260555, 55.87942070143253), (37.44716141796871, 55.88072960917233), (37.44769797085568, 55.88121221323979), (37.45204320500181, 55.882080694420715), (37.45673176190186, 55.882346110794586), (37.463383999999984, 55.88252729504517), (37.46682797486874, 55.88294937719063), (37.470014457672086, 55.88361266759345), (37.47751410450743, 55.88546991372396), (37.47860317658232, 55.88534929207307), (37.48165826025772, 55.882563306475106), (37.48316434442331, 55.8815803226785), (37.483831555817645, 55.882427612793315), (37.483182967125686, 55.88372791409729), (37.483092277908824, 55.88495581062434), (37.4855716508179, 55.8875561994203), (37.486440636245746, 55.887827444039566), (37.49014203439328, 55.88897899871799), (37.493210285705544, 55.890208937135604), (37.497512451065035, 55.891342397444696), (37.49780744510645, 55.89174030252967), (37.49940333499519, 55.89239745507079), (37.50018383334346, 55.89339220941865), (37.52421672750851, 55.903869074155224), (37.52977457672118, 55.90564076517974), (37.53503220370484, 55.90661661218259), (37.54042858064267, 55.90714113744566), (37.54320461007303, 55.905645048442985), (37.545686966066306, 55.906608607018505), (37.54743976120755, 55.90788552162358), (37.55796999999999, 55.90901557907218), (37.572711542327866, 55.91059395704873), (37.57942799999998, 55.91073854155573), (37.58502865872187, 55.91009969268444), (37.58739968913264, 55.90794809960554), (37.59131567193598, 55.908713267595054), (37.612687423278814, 55.902866854295375), (37.62348079629517, 55.90041967242986), (37.635797880950896, 55.898141151686396), (37.649487626983664, 55.89639275532968), (37.65619302513125, 55.89572360207488), (37.66294133862307, 55.895295577183965), (37.66874564418033, 55.89505457604897), (37.67375601586915, 55.89254677027454), (37.67744661901856, 55.8947775867987), (37.688347, 55.89450045676125), (37.69480554232789, 55.89422926332761), (37.70107096560668, 55.89322256101114), (37.705962965606716, 55.891763491662616), (37.711885134918205, 55.889110234998974), (37.71682005026245, 55.886577568759876), (37.7199315476074, 55.88458159806678), (37.72234560316464, 55.882281005794134), (37.72364385977171, 55.8809452036196), (37.725371142837474, 55.8809722706006), (37.727870902099546, 55.88037213862385), (37.73394330422971, 55.877941504088696), (37.745339592590376, 55.87208120378722), (37.75525267724611, 55.86703807949492), (37.76919976190188, 55.859821640197474), (37.827835219574, 55.82962968399116), (37.83341438888553, 55.82575289922351), (37.83652584655761, 55.82188784027888), (37.83809213491821, 55.81612575504693), (37.83605359521481, 55.81460347077685), (37.83632178569025, 55.81276696067908), (37.838623105812026, 55.811486181656385), (37.83912198147584, 55.807329380532785), (37.839079078033414, 55.80510270463816), (37.83965844708251, 55.79940712529036), (37.840581150787344, 55.79131399999368), (37.84172564285271, 55.78000432402266)]); ``` -Check how many cell towers are in Moscow: +3. Check how many cell towers are in Moscow: ``` SELECT count() FROM cell_towers WHERE pointInPolygon((lon, lat), (SELECT * FROM moscow)) @@ -128,6 +129,4 @@ SELECT count() FROM cell_towers WHERE pointInPolygon((lon, lat), (SELECT * FROM The data is also available for interactive queries in the [Playground](https://gh-api.clickhouse.tech/play?user=play), [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIG1jYywgY291bnQoKSBGUk9NIGNlbGxfdG93ZXJzIEdST1VQIEJZIG1jYyBPUkRFUiBCWSBjb3VudCgpIERFU0M=). -Although you cannot create temporary tables there. - -[Original article](https://clickhouse.tech/docs/en/getting_started/example_datasets/cell-towers/) +Although you cannot create temporary tables there. \ No newline at end of file diff --git a/docs/en/interfaces/third-party/gui.md b/docs/en/interfaces/third-party/gui.md index 5d14b3aa3cc..d2fd24ab9a4 100644 --- a/docs/en/interfaces/third-party/gui.md +++ b/docs/en/interfaces/third-party/gui.md @@ -184,4 +184,9 @@ SeekTable is [free](https://www.seektable.com/help/cloud-pricing) for personal/i [How to configure ClickHouse connection in SeekTable.](https://www.seektable.com/help/clickhouse-pivot-table) + +### Chadmin {#chadmin} + +[Chadmin](https://github.com/bun4uk/chadmin) is a simple UI where you can visualize your currently running queries on your ClickHouse cluster and info about them and kill them if you want. + [Original article](https://clickhouse.tech/docs/en/interfaces/third-party/gui/) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 012d86b1ef7..3b3c1203b50 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -13,6 +13,7 @@ toc_title: Adopters | 2gis | Maps | Monitoring | — | — | [Talk in Russian, July 2019](https://youtu.be/58sPkXfq6nw) | | Admiral | Martech | Engagement Management | — | — | [Webinar Slides, June 2020](https://altinity.com/presentations/2020/06/16/big-data-in-real-time-how-clickhouse-powers-admirals-visitor-relationships-for-publishers) | | AdScribe | Ads | TV Analytics | — | — | [A quote from CTO](https://altinity.com/24x7-support/) | +| Ahrefs | SEO | Analytics | — | — | [Job listing](https://ahrefs.com/jobs/data-scientist-search) | | Alibaba Cloud | Cloud | Managed Service | — | — | [Official Website](https://help.aliyun.com/product/144466.html) | | Aloha Browser | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.tech/meetup22/aloha.pdf) | | Altinity | Cloud, SaaS | Main product | — | — | [Official Website](https://altinity.com/) | @@ -47,7 +48,8 @@ toc_title: Adopters | Diva-e | 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) | | Ecwid | E-commerce SaaS | Metrics, Logging | — | — | [Slides in Russian, April 2019](https://nastachku.ru/var/files/1/presentation/backend/2_Backend_6.pdf) | | eBay | E-commerce | Logs, Metrics and Events | — | — | [Official website, Sep 2020](https://tech.ebayinc.com/engineering/ou-online-analytical-processing/) | -| Exness | Trading | Metrics, Logging | — | — | [Talk in Russian, May 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) | +| Exness | Trading | Metrics, Logging | — | — | [Talk in Russian, May 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) | +| EventBunker.io | Serverless Data Processing | — | — | — | [Tweet, April 2021](https://twitter.com/Halil_D_/status/1379839133472985091) | | FastNetMon | DDoS Protection | Main Product | | — | [Official website](https://fastnetmon.com/docs-fnm-advanced/fastnetmon-advanced-traffic-persistency/) | | Flipkart | e-Commerce | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=239) | | FunCorp | Games | | — | 14 bn records/day as of Jan 2021 | [Article](https://www.altinity.com/blog/migrating-from-redshift-to-clickhouse) | diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 2b3bdbd51ef..f86e9668f00 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -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: diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 77b68715ba9..b2470207dcc 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -56,6 +56,26 @@ Default value: 150. ClickHouse artificially executes `INSERT` longer (adds ‘sleep’) so that the background merge process can merge parts faster than they are added. +## inactive_parts_to_throw_insert {#inactive-parts-to-throw-insert} + +If the number of inactive parts in a single partition more than the `inactive_parts_to_throw_insert` value, `INSERT` is interrupted with the "Too many inactive parts (N). Parts cleaning are processing significantly slower than inserts" exception. + +Possible values: + +- Any positive integer. + +Default value: 0 (unlimited). + +## inactive_parts_to_delay_insert {#inactive-parts-to-delay-insert} + +If the number of inactive parts in a single partition in the table at least that many the `inactive_parts_to_delay_insert` value, an `INSERT` artificially slows down. It is useful when a server fails to clean up parts quickly enough. + +Possible values: + +- Any positive integer. + +Default value: 0 (unlimited). + ## max_delay_to_insert {#max-delay-to-insert} The value in seconds, which is used to calculate the `INSERT` delay, if the number of active parts in a single partition exceeds the [parts_to_delay_insert](#parts-to-delay-insert) value. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 27bb6fee0e2..6a1b25982a0 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1565,6 +1565,17 @@ Possible values: Default value: 0 +## optimize_skip_unused_shards_rewrite_in {#optimize-skip-unused-shardslrewrite-in} + +Rewrite IN in query for remote shards to exclude values that does not belong to the shard (requires optimize_skip_unused_shards). + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: 1 (since it requires `optimize_skip_unused_shards` anyway, which `0` by default) + ## allow_nondeterministic_optimize_skip_unused_shards {#allow-nondeterministic-optimize-skip-unused-shards} Allow nondeterministic (like `rand` or `dictGet`, since later has some caveats with updates) functions in sharding key. @@ -2844,4 +2855,15 @@ Sets the interval in seconds after which periodically refreshed [live view](../. Default value: `60`. +## check_query_single_value_result {#check_query_single_value_result} + +Defines the level of detail for the [CHECK TABLE](../../sql-reference/statements/check-table.md#checking-mergetree-tables) query result for `MergeTree` family engines . + +Possible values: + +- 0 — the query shows a check status for every individual data part of a table. +- 1 — the query shows the general table check status. + +Default value: `0`. + [Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index e62dea0b04e..865fe58d7cd 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -191,8 +191,9 @@ dynamicConfigFile=/etc/zookeeper-{{ '{{' }} cluster['name'] {{ '}}' }}/conf/zoo. Java version: ``` text -Java(TM) SE Runtime Environment (build 1.8.0_25-b17) -Java HotSpot(TM) 64-Bit Server VM (build 25.25-b02, mixed mode) +openjdk 11.0.5-shenandoah 2019-10-15 +OpenJDK Runtime Environment (build 11.0.5-shenandoah+10-adhoc.heretic.src) +OpenJDK 64-Bit Server VM (build 11.0.5-shenandoah+10-adhoc.heretic.src, mixed mode) ``` JVM parameters: @@ -204,7 +205,7 @@ ZOOCFGDIR=/etc/$NAME/conf # TODO this is really ugly # How to find out, which jars are needed? # seems, that log4j requires the log4j.properties file to be in the classpath -CLASSPATH="$ZOOCFGDIR:/usr/build/classes:/usr/build/lib/*.jar:/usr/share/zookeeper/zookeeper-3.5.1-metrika.jar:/usr/share/zookeeper/slf4j-log4j12-1.7.5.jar:/usr/share/zookeeper/slf4j-api-1.7.5.jar:/usr/share/zookeeper/servlet-api-2.5-20081211.jar:/usr/share/zookeeper/netty-3.7.0.Final.jar:/usr/share/zookeeper/log4j-1.2.16.jar:/usr/share/zookeeper/jline-2.11.jar:/usr/share/zookeeper/jetty-util-6.1.26.jar:/usr/share/zookeeper/jetty-6.1.26.jar:/usr/share/zookeeper/javacc.jar:/usr/share/zookeeper/jackson-mapper-asl-1.9.11.jar:/usr/share/zookeeper/jackson-core-asl-1.9.11.jar:/usr/share/zookeeper/commons-cli-1.2.jar:/usr/src/java/lib/*.jar:/usr/etc/zookeeper" +CLASSPATH="$ZOOCFGDIR:/usr/build/classes:/usr/build/lib/*.jar:/usr/share/zookeeper-3.6.2/lib/audience-annotations-0.5.0.jar:/usr/share/zookeeper-3.6.2/lib/commons-cli-1.2.jar:/usr/share/zookeeper-3.6.2/lib/commons-lang-2.6.jar:/usr/share/zookeeper-3.6.2/lib/jackson-annotations-2.10.3.jar:/usr/share/zookeeper-3.6.2/lib/jackson-core-2.10.3.jar:/usr/share/zookeeper-3.6.2/lib/jackson-databind-2.10.3.jar:/usr/share/zookeeper-3.6.2/lib/javax.servlet-api-3.1.0.jar:/usr/share/zookeeper-3.6.2/lib/jetty-http-9.4.24.v20191120.jar:/usr/share/zookeeper-3.6.2/lib/jetty-io-9.4.24.v20191120.jar:/usr/share/zookeeper-3.6.2/lib/jetty-security-9.4.24.v20191120.jar:/usr/share/zookeeper-3.6.2/lib/jetty-server-9.4.24.v20191120.jar:/usr/share/zookeeper-3.6.2/lib/jetty-servlet-9.4.24.v20191120.jar:/usr/share/zookeeper-3.6.2/lib/jetty-util-9.4.24.v20191120.jar:/usr/share/zookeeper-3.6.2/lib/jline-2.14.6.jar:/usr/share/zookeeper-3.6.2/lib/json-simple-1.1.1.jar:/usr/share/zookeeper-3.6.2/lib/log4j-1.2.17.jar:/usr/share/zookeeper-3.6.2/lib/metrics-core-3.2.5.jar:/usr/share/zookeeper-3.6.2/lib/netty-buffer-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-codec-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-common-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-handler-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-resolver-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-transport-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-transport-native-epoll-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-transport-native-unix-common-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/simpleclient-0.6.0.jar:/usr/share/zookeeper-3.6.2/lib/simpleclient_common-0.6.0.jar:/usr/share/zookeeper-3.6.2/lib/simpleclient_hotspot-0.6.0.jar:/usr/share/zookeeper-3.6.2/lib/simpleclient_servlet-0.6.0.jar:/usr/share/zookeeper-3.6.2/lib/slf4j-api-1.7.25.jar:/usr/share/zookeeper-3.6.2/lib/slf4j-log4j12-1.7.25.jar:/usr/share/zookeeper-3.6.2/lib/snappy-java-1.1.7.jar:/usr/share/zookeeper-3.6.2/lib/zookeeper-3.6.2.jar:/usr/share/zookeeper-3.6.2/lib/zookeeper-jute-3.6.2.jar:/usr/share/zookeeper-3.6.2/lib/zookeeper-prometheus-metrics-3.6.2.jar:/usr/share/zookeeper-3.6.2/etc" ZOOCFG="$ZOOCFGDIR/zoo.cfg" ZOO_LOG_DIR=/var/log/$NAME @@ -213,27 +214,17 @@ GROUP=zookeeper PIDDIR=/var/run/$NAME PIDFILE=$PIDDIR/$NAME.pid SCRIPTNAME=/etc/init.d/$NAME -JAVA=/usr/bin/java +JAVA=/usr/local/jdk-11/bin/java ZOOMAIN="org.apache.zookeeper.server.quorum.QuorumPeerMain" ZOO_LOG4J_PROP="INFO,ROLLINGFILE" JMXLOCALONLY=false JAVA_OPTS="-Xms{{ '{{' }} cluster.get('xms','128M') {{ '}}' }} \ -Xmx{{ '{{' }} cluster.get('xmx','1G') {{ '}}' }} \ - -Xloggc:/var/log/$NAME/zookeeper-gc.log \ - -XX:+UseGCLogFileRotation \ - -XX:NumberOfGCLogFiles=16 \ - -XX:GCLogFileSize=16M \ + -Xlog:safepoint,gc*=info,age*=debug:file=/var/log/$NAME/zookeeper-gc.log:time,level,tags:filecount=16,filesize=16M -verbose:gc \ - -XX:+PrintGCTimeStamps \ - -XX:+PrintGCDateStamps \ - -XX:+PrintGCDetails - -XX:+PrintTenuringDistribution \ - -XX:+PrintGCApplicationStoppedTime \ - -XX:+PrintGCApplicationConcurrentTime \ - -XX:+PrintSafepointStatistics \ - -XX:+UseParNewGC \ - -XX:+UseConcMarkSweepGC \ --XX:+CMSParallelRemarkEnabled" + -XX:+UseG1GC \ + -Djute.maxbuffer=8388608 \ + -XX:MaxGCPauseMillis=50" ``` Salt init: diff --git a/docs/en/operations/update.md b/docs/en/operations/update.md index 9fa9c44e130..22995705af9 100644 --- a/docs/en/operations/update.md +++ b/docs/en/operations/update.md @@ -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 - diff --git a/docs/en/sql-reference/data-types/date.md b/docs/en/sql-reference/data-types/date.md index 886e93f433c..0cfac4d59fe 100644 --- a/docs/en/sql-reference/data-types/date.md +++ b/docs/en/sql-reference/data-types/date.md @@ -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. diff --git a/docs/en/sql-reference/data-types/datetime64.md b/docs/en/sql-reference/data-types/datetime64.md index 5cba8315090..1d3725b9fb3 100644 --- a/docs/en/sql-reference/data-types/datetime64.md +++ b/docs/en/sql-reference/data-types/datetime64.md @@ -9,7 +9,7 @@ Allows to store an instant in time, that can be expressed as a calendar date and Tick size (precision): 10-precision 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) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md index dbf2fa67ac5..f22d2a0b59e 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md @@ -159,14 +159,14 @@ Configuration fields: | Tag | Description | Required | |------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------| | `name` | Column name. | Yes | -| `type` | ClickHouse data type.
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.
[Nullable](../../../sql-reference/data-types/nullable.md) is not supported. | Yes | -| `null_value` | Default value for a non-existing element.
In the example, it is an empty string. You cannot use `NULL` in this field. | Yes | +| `type` | ClickHouse data type.
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.
[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.
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.
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.

Default value: no expression. | No | | `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).

Default value: `false`. | No | | `injective` | Flag that shows whether the `id -> attribute` image is [injective](https://en.wikipedia.org/wiki/Injective_function).
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.

Default value: `false`. | No | | `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.

Default value: `false`. | No | -## See Also {#see-also} +**See Also** - [Functions for working with external dictionaries](../../../sql-reference/functions/ext-dict-functions.md). diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index fa127dab103..22f4182a1c0 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -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 can’t 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). diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 6b26dae4546..b0636b0305e 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -147,6 +147,9 @@ Result: └────────────────┘ ``` +!!! attention "Attention" + The return type `toStartOf*` functions described below is `Date` or `DateTime`. Though these functions can take `DateTime64` as an argument, passing them a `DateTime64` that is out of normal range (years 1970 - 2105) will give incorrect result. + ## toStartOfYear {#tostartofyear} Rounds down a date or date with time to the first day of the year. @@ -388,13 +391,13 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d Truncates date and time data to the specified part of date. -**Syntax** +**Syntax** ``` sql date_trunc(unit, value[, timezone]) ``` -Alias: `dateTrunc`. +Alias: `dateTrunc`. **Arguments** @@ -457,13 +460,13 @@ Result: Adds the time interval or date interval to the provided date or date with time. -**Syntax** +**Syntax** ``` sql date_add(unit, value, date) ``` -Aliases: `dateAdd`, `DATE_ADD`. +Aliases: `dateAdd`, `DATE_ADD`. **Arguments** @@ -478,7 +481,7 @@ Aliases: `dateAdd`, `DATE_ADD`. - `month` - `quarter` - `year` - + - `value` — Value of interval to add. [Int](../../sql-reference/data-types/int-uint.md). - `date` — The date or date with time to which `value` is added. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). @@ -583,7 +586,7 @@ Aliases: `dateSub`, `DATE_SUB`. - `month` - `quarter` - `year` - + - `value` — Value of interval to subtract. [Int](../../sql-reference/data-types/int-uint.md). - `date` — The date or date with time from which `value` is subtracted. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). @@ -613,16 +616,16 @@ Result: Adds the specified time value with the provided date or date time value. -**Syntax** +**Syntax** ``` sql timestamp_add(date, INTERVAL value unit) ``` -Aliases: `timeStampAdd`, `TIMESTAMP_ADD`. +Aliases: `timeStampAdd`, `TIMESTAMP_ADD`. **Arguments** - + - `date` — Date or date with time. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). - `value` — Value of interval to add. [Int](../../sql-reference/data-types/int-uint.md). - `unit` — The type of interval to add. [String](../../sql-reference/data-types/string.md). @@ -642,7 +645,7 @@ Aliases: `timeStampAdd`, `TIMESTAMP_ADD`. Date or date with time with the specified `value` expressed in `unit` added to `date`. Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). - + **Example** Query: @@ -663,13 +666,13 @@ Result: Subtracts the time interval from the provided date or date with time. -**Syntax** +**Syntax** ``` sql timestamp_sub(unit, value, date) ``` -Aliases: `timeStampSub`, `TIMESTAMP_SUB`. +Aliases: `timeStampSub`, `TIMESTAMP_SUB`. **Arguments** @@ -684,7 +687,7 @@ Aliases: `timeStampSub`, `TIMESTAMP_SUB`. - `month` - `quarter` - `year` - + - `value` — Value of interval to subtract. [Int](../../sql-reference/data-types/int-uint.md). - `date` — Date or date with time. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). @@ -709,12 +712,12 @@ Result: │ 2018-07-18 01:02:03 │ └──────────────────────────────────────────────────────────────┘ ``` - + ## now {#now} -Returns the current date and time. +Returns the current date and time. -**Syntax** +**Syntax** ``` sql now([timezone]) @@ -1069,4 +1072,3 @@ Result: │ 2020-01-01 │ └────────────────────────────────────┘ ``` - diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 3d3caaf6e23..85570cb408d 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -649,3 +649,65 @@ Result: - [List of XML and HTML character entity references](https://en.wikipedia.org/wiki/List_of_XML_and_HTML_character_entity_references) + +## extractTextFromHTML {#extracttextfromhtml} + +A function to extract text from HTML or XHTML. +It does not necessarily 100% conform to any of the HTML, XML or XHTML standards, but the implementation is reasonably accurate and it is fast. The rules are the following: + +1. Comments are skipped. Example: ``. Comment must end with `-->`. Nested comments are not possible. +Note: constructions like `` and `` are not valid comments in HTML but they are skipped by other rules. +2. CDATA is pasted verbatim. Note: CDATA is XML/XHTML specific. But it is processed for "best-effort" approach. +3. `script` and `style` elements are removed with all their content. Note: it is assumed that closing tag cannot appear inside content. For example, in JS string literal has to be escaped like `"<\/script>"`. +Note: comments and CDATA are possible inside `script` or `style` - then closing tags are not searched inside CDATA. Example: `]]>`. But they are still searched inside comments. Sometimes it becomes complicated: ` var y = "-->"; alert(x + y);` +Note: `script` and `style` can be the names of XML namespaces - then they are not treated like usual `script` or `style` elements. Example: `Hello`. +Note: whitespaces are possible after closing tag name: `` but not before: `< / script>`. +4. Other tags or tag-like elements are skipped without inner content. Example: `.` +Note: it is expected that this HTML is illegal: `` +Note: it also skips something like tags: `<>`, ``, etc. +Note: tag without end is skipped to the end of input: `world`, `Helloworld` - there is no whitespace in HTML, but the function inserts it. Also consider: `Hello

world

`, `Hello
world`. This behavior is reasonable for data analysis, e.g. to convert HTML to a bag of words. +7. Also note that correct handling of whitespaces requires the support of `
` and CSS `display` and `white-space` properties.
+
+**Syntax**
+
+``` sql
+extractTextFromHTML(x)
+```
+
+**Arguments**
+
+-   `x` — input text. [String](../../sql-reference/data-types/string.md). 
+
+**Returned value**
+
+-   Extracted text.
+
+Type: [String](../../sql-reference/data-types/string.md).
+
+**Example**
+
+The first example contains several tags and a comment and also shows whitespace processing.
+The second example shows `CDATA` and `script` tag processing.
+In the third example text is extracted from the full HTML response received by the [url](../../sql-reference/table-functions/url.md) function.
+
+Query:
+
+``` sql
+SELECT extractTextFromHTML(' 

A text withtags.

'); +SELECT extractTextFromHTML('CDATA]]> '); +SELECT extractTextFromHTML(html) FROM url('http://www.donothingfor2minutes.com/', RawBLOB, 'html String'); +``` + +Result: + +``` text +A text with tags . +The content within CDATA +Do Nothing for 2 Minutes 2:00   +``` diff --git a/docs/en/sql-reference/statements/alter/ttl.md b/docs/en/sql-reference/statements/alter/ttl.md index aa7ee838e10..9cd63d3b8fe 100644 --- a/docs/en/sql-reference/statements/alter/ttl.md +++ b/docs/en/sql-reference/statements/alter/ttl.md @@ -79,7 +79,7 @@ The `TTL` is no longer there, so the second row is not deleted: └───────────────────────┴─────────┴──────────────┘ ``` -### See Also +**See Also** - More about the [TTL-expression](../../../sql-reference/statements/create/table.md#ttl-expression). - Modify column [with TTL](../../../sql-reference/statements/alter/column.md#alter_modify-column). diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index 450447acaf8..65e6238ebbc 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -30,9 +30,36 @@ Performed over the tables with another table engines causes an exception. Engines from the `*Log` family don’t provide automatic data recovery on failure. Use the `CHECK TABLE` query to track data loss in a timely manner. -For `MergeTree` family engines, the `CHECK TABLE` query shows a check status for every individual data part of a table on the local server. +## Checking the MergeTree Family Tables {#checking-mergetree-tables} -**If the data is corrupted** +For `MergeTree` family engines, if [check_query_single_value_result](../../operations/settings/settings.md#check_query_single_value_result) = 0, the `CHECK TABLE` query shows a check status for every individual data part of a table on the local server. + +```sql +SET check_query_single_value_result = 0; +CHECK TABLE test_table; +``` + +```text +┌─part_path─┬─is_passed─┬─message─┐ +│ all_1_4_1 │ 1 │ │ +│ all_1_4_2 │ 1 │ │ +└───────────┴───────────┴─────────┘ +``` + +If `check_query_single_value_result` = 0, the `CHECK TABLE` query shows the general table check status. + +```sql +SET check_query_single_value_result = 1; +CHECK TABLE test_table; +``` + +```text +┌─result─┐ +│ 1 │ +└────────┘ +``` + +## If the Data Is Corrupted {#if-data-is-corrupted} If the table is corrupted, you can copy the non-corrupted data to another table. To do this: diff --git a/docs/en/sql-reference/statements/rename.md b/docs/en/sql-reference/statements/rename.md index 4f14ad016a3..a9dda6ed3b2 100644 --- a/docs/en/sql-reference/statements/rename.md +++ b/docs/en/sql-reference/statements/rename.md @@ -5,6 +5,14 @@ toc_title: RENAME # RENAME Statement {#misc_operations-rename} +## RENAME DATABASE {#misc_operations-rename_database} +Renames database, support only for Atomic database engine + +``` +RENAME DATABASE atomic_database1 TO atomic_database2 [ON CLUSTER cluster] +``` + +## RENAME TABLE {#misc_operations-rename_table} Renames one or more tables. ``` sql diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 2348a2a2668..919bd65d56b 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -169,7 +169,7 @@ SYSTEM START MERGES [ON VOLUME | [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 doesn’t exists or table have not MergeTree engine. Return error when database doesn’t exists: +Returns `Ok.` even if table doesn’t exist or table has not MergeTree engine. Returns error when database doesn’t 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 doesn’t exists. Return error when database doesn’t exists: +Returns `Ok.` even if table doesn’t exist. Returns error when database doesn’t 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 doesn’t exists. Return error when database doesn’t exists: +Returns `Ok.` even if table doesn’t exist. Returns error when database doesn’t 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 doesn’t exists. Return error when database doesn’t exists: +Returns `Ok.` even if table doesn’t exist. Returns error when database doesn’t 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 doesn’t exists. +Always returns `Ok.` regardless of the table engine and even if table or database doesn’t 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 doesn’t exists. +Always returns `Ok.` regardless of the table engine and even if table or database doesn’t 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 - diff --git a/docs/ru/engines/table-engines/integrations/postgresql.md b/docs/ru/engines/table-engines/integrations/postgresql.md index 8964b1dbf02..064665d49c1 100644 --- a/docs/ru/engines/table-engines/integrations/postgresql.md +++ b/docs/ru/engines/table-engines/integrations/postgresql.md @@ -22,7 +22,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Структура таблицы может отличаться от исходной структуры таблицы PostgreSQL: -- Имена столбцов должны быть такими же, как в исходной таблице MySQL, но вы можете использовать только некоторые из этих столбцов и в любом порядке. +- Имена столбцов должны быть такими же, как в исходной таблице PostgreSQL, но вы можете использовать только некоторые из этих столбцов и в любом порядке. - Типы столбцов могут отличаться от типов в исходной таблице PostgreSQL. ClickHouse пытается [приводить](../../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) values to the ClickHouse data types. - Настройка `external_table_functions_use_nulls` определяет как обрабатывать Nullable столбцы. По умолчанию 1, если 0 - табличная функция не будет делать nullable столбцы и будет вместо null выставлять значения по умолчанию для скалярного типа. Это также применимо для null значений внутри массивов. diff --git a/docs/ru/getting-started/example-datasets/cell-towers.md b/docs/ru/getting-started/example-datasets/cell-towers.md new file mode 100644 index 00000000000..a5524248019 --- /dev/null +++ b/docs/ru/getting-started/example-datasets/cell-towers.md @@ -0,0 +1,128 @@ +--- +toc_priority: 21 +toc_title: Вышки сотовой связи +--- + +# Вышки сотовой связи {#cell-towers} + +Источник этого набора данных (dataset) - самая большая в мире открытая база данных о сотовых вышках - [OpenCellid](https://www.opencellid.org/). К 2021-му году здесь накопилось более, чем 40 миллионов записей о сотовых вышках (GSM, LTE, UMTS, и т.д.) по всему миру с их географическими координатами и метаданными (код страны, сети, и т.д.). + +OpenCelliD Project имеет лицензию Creative Commons Attribution-ShareAlike 4.0 International License, и мы распространяем снэпшот набора данных по условиям этой же лицензии. После авторизации можно загрузить последнюю версию набора данных. + +## Как получить набор данных {#get-the-dataset} + +1. Загрузите снэпшот набора данных за февраль 2021 [отсюда](https://datasets.clickhouse.tech/cell_towers.csv.xz) (729 MB). + +2. Если нужно, проверьте полноту и целостность при помощи команды: + +``` +md5sum cell_towers.csv.xz +8cf986f4a0d9f12c6f384a0e9192c908 cell_towers.csv.xz +``` + +3. Распакуйте набор данных при помощи команды: + +``` +xz -d cell_towers.csv.xz +``` + +4. Создайте таблицу: + +``` +CREATE TABLE cell_towers +( + radio Enum8('' = 0, 'CDMA' = 1, 'GSM' = 2, 'LTE' = 3, 'NR' = 4, 'UMTS' = 5), + mcc UInt16, + net UInt16, + area UInt16, + cell UInt64, + unit Int16, + lon Float64, + lat Float64, + range UInt32, + samples UInt32, + changeable UInt8, + created DateTime, + updated DateTime, + averageSignal UInt8 +) +ENGINE = MergeTree ORDER BY (radio, mcc, net, created); +``` + +5. Вставьте данные: +``` +clickhouse-client --query "INSERT INTO cell_towers FORMAT CSVWithNames" < cell_towers.csv +``` + +## Примеры {#examples} + +1. Количество вышек по типам: + +``` +SELECT radio, count() AS c FROM cell_towers GROUP BY radio ORDER BY c DESC + +┌─radio─┬────────c─┐ +│ UMTS │ 20686487 │ +│ LTE │ 12101148 │ +│ GSM │ 9931312 │ +│ CDMA │ 556344 │ +│ NR │ 867 │ +└───────┴──────────┘ + +5 rows in set. Elapsed: 0.011 sec. Processed 43.28 million rows, 43.28 MB (3.83 billion rows/s., 3.83 GB/s.) +``` + +2. Количество вышек по [мобильному коду страны (MCC)](https://ru.wikipedia.org/wiki/Mobile_Country_Code): + +``` +SELECT mcc, count() FROM cell_towers GROUP BY mcc ORDER BY count() DESC LIMIT 10 + +┌─mcc─┬─count()─┐ +│ 310 │ 5024650 │ +│ 262 │ 2622423 │ +│ 250 │ 1953176 │ +│ 208 │ 1891187 │ +│ 724 │ 1836150 │ +│ 404 │ 1729151 │ +│ 234 │ 1618924 │ +│ 510 │ 1353998 │ +│ 440 │ 1343355 │ +│ 311 │ 1332798 │ +└─────┴─────────┘ + +10 rows in set. Elapsed: 0.019 sec. Processed 43.28 million rows, 86.55 MB (2.33 billion rows/s., 4.65 GB/s.) +``` + +Можно увидеть, что по количеству вышек лидируют следующие страны: США, Германия, Россия. + +Вы также можете создать [внешний словарь](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) в ClickHouse для того, чтобы расшифровать эти значения. + +## Пример использования {#use-case} + +Рассмотрим применение функции `pointInPolygon`. + +1. Создаем таблицу, в которой будем хранить многоугольники: + +``` +CREATE TEMPORARY TABLE moscow (polygon Array(Tuple(Float64, Float64))); +``` + +2. Очертания Москвы выглядят приблизительно так ("Новая Москва" в них не включена): + +``` +INSERT INTO moscow VALUES ([(37.84172564285271, 55.78000432402266), (37.8381207618713, 55.775874525970494), (37.83979446823122, 55.775626746008065), (37.84243326983639, 55.77446586811748), (37.84262672750849, 55.771974101091104), (37.84153238623039, 55.77114545193181), (37.841124690460184, 55.76722010265554), (37.84239076983644, 55.76654891107098), (37.842283558197025, 55.76258709833121), (37.8421759312134, 55.758073999993734), (37.84198330422974, 55.75381499999371), (37.8416827275085, 55.749277102484484), (37.84157576190186, 55.74794544108413), (37.83897929098507, 55.74525257875241), (37.83739676451868, 55.74404373042019), (37.838732481460525, 55.74298009816793), (37.841183997352545, 55.743060321833575), (37.84097476190185, 55.73938799999373), (37.84048155819702, 55.73570799999372), (37.840095812164286, 55.73228210777237), (37.83983814285274, 55.73080491981639), (37.83846476321406, 55.729799917464675), (37.83835745269769, 55.72919751082619), (37.838636380279524, 55.72859509486539), (37.8395161005249, 55.727705075632784), (37.83897964285276, 55.722727886185154), (37.83862557539366, 55.72034817326636), (37.83559735744853, 55.71944437307499), (37.835370708803126, 55.71831419154461), (37.83738169402022, 55.71765218986692), (37.83823396494291, 55.71691750159089), (37.838056931213345, 55.71547311301385), (37.836812846557606, 55.71221445615604), (37.83522525396725, 55.709331054395555), (37.83269301586908, 55.70953687463627), (37.829667367706236, 55.70903403789297), (37.83311126588435, 55.70552351822608), (37.83058993121339, 55.70041317726053), (37.82983872750851, 55.69883771404813), (37.82934501586913, 55.69718947487017), (37.828926414016685, 55.69504441658371), (37.82876530422971, 55.69287499999378), (37.82894754100031, 55.690759754047335), (37.827697554878185, 55.68951421135665), (37.82447346292115, 55.68965045405069), (37.83136543914793, 55.68322046195302), (37.833554015869154, 55.67814012759211), (37.83544184655761, 55.67295011628339), (37.837480388885474, 55.6672498719639), (37.838960677246064, 55.66316274139358), (37.83926093121332, 55.66046999999383), (37.839025050262435, 55.65869897264431), (37.83670784390257, 55.65794084879904), (37.835656529083245, 55.65694309303843), (37.83704060449217, 55.65689306460552), (37.83696819873806, 55.65550363526252), (37.83760389616388, 55.65487847246661), (37.83687972750851, 55.65356745541324), (37.83515216004943, 55.65155951234079), (37.83312418518067, 55.64979413590619), (37.82801726983639, 55.64640836412121), (37.820614174591, 55.64164525405531), (37.818908190475426, 55.6421883258084), (37.81717543386075, 55.64112490388471), (37.81690987037274, 55.63916106913107), (37.815099354492155, 55.637925371757085), (37.808769150787356, 55.633798276884455), (37.80100123544311, 55.62873670012244), (37.79598013491824, 55.62554336109055), (37.78634567724606, 55.62033499605651), (37.78334147619623, 55.618768681480326), (37.77746201055901, 55.619855533402706), (37.77527329626457, 55.61909966711279), (37.77801986242668, 55.618770300976294), (37.778212973541216, 55.617257701952106), (37.77784818518065, 55.61574504433011), (37.77016867724609, 55.61148576294007), (37.760191219573976, 55.60599579539028), (37.75338926983641, 55.60227892751446), (37.746329965606634, 55.59920577639331), (37.73939925396728, 55.59631430313617), (37.73273665739439, 55.5935318803559), (37.7299954450912, 55.59350760316188), (37.7268679946899, 55.59469840523759), (37.72626726983634, 55.59229549697373), (37.7262673598022, 55.59081598950582), (37.71897193121335, 55.5877595845419), (37.70871550793456, 55.58393177431724), (37.700497489410374, 55.580917323756644), (37.69204305026244, 55.57778089778455), (37.68544477378839, 55.57815154690915), (37.68391050793454, 55.57472945079756), (37.678803592590306, 55.57328235936491), (37.6743402539673, 55.57255251445782), (37.66813862698363, 55.57216388774464), (37.617927457672096, 55.57505691895805), (37.60443099999999, 55.5757737568051), (37.599683515869145, 55.57749105910326), (37.59754177842709, 55.57796291823627), (37.59625834786988, 55.57906686095235), (37.59501783265684, 55.57746616444403), (37.593090671936025, 55.57671634534502), (37.587018007904, 55.577944600233785), (37.578692203704804, 55.57982895000019), (37.57327546607398, 55.58116294118248), (37.57385012109279, 55.581550362779), (37.57399562266922, 55.5820107079112), (37.5735356072979, 55.58226289171689), (37.57290393054962, 55.582393529795155), (37.57037722355653, 55.581919415056234), (37.5592298306885, 55.584471614867844), (37.54189249206543, 55.58867650795186), (37.5297256269836, 55.59158133551745), (37.517837865081766, 55.59443656218868), (37.51200186508174, 55.59635625174229), (37.506808949737554, 55.59907823904434), (37.49820432275389, 55.6062944994944), (37.494406071441674, 55.60967103463367), (37.494760001358024, 55.61066689753365), (37.49397137107085, 55.61220931698269), (37.49016528606031, 55.613417718449064), (37.48773249206542, 55.61530616333343), (37.47921386508177, 55.622640129112334), (37.470652153442394, 55.62993723476164), (37.46273446298218, 55.6368075123157), (37.46350692265317, 55.64068225239439), (37.46050283203121, 55.640794546982576), (37.457627470916734, 55.64118904154646), (37.450718034393326, 55.64690488145138), (37.44239252645875, 55.65397824729769), (37.434587576721185, 55.66053543155961), (37.43582144975277, 55.661693766520735), (37.43576786245721, 55.662755031737014), (37.430982915344174, 55.664610641628116), (37.428547447097685, 55.66778515273695), (37.42945134592044, 55.668633314343566), (37.42859571562949, 55.66948145750025), (37.4262836402282, 55.670813882451405), (37.418709037048295, 55.6811141674414), (37.41922139651101, 55.68235377885389), (37.419218771842885, 55.68359335082235), (37.417196501327446, 55.684375235224735), (37.41607020370478, 55.68540557585352), (37.415640857147146, 55.68686637150793), (37.414632153442334, 55.68903015131686), (37.413344899475064, 55.690896881757396), (37.41171432275391, 55.69264232162232), (37.40948282275393, 55.69455101638112), (37.40703674603271, 55.69638690385348), (37.39607169577025, 55.70451821283731), (37.38952706878662, 55.70942491932811), (37.387778313491815, 55.71149057784176), (37.39049275399779, 55.71419814298992), (37.385557272491454, 55.7155489617061), (37.38388335714726, 55.71849856042102), (37.378368238098155, 55.7292763261685), (37.37763597123337, 55.730845879211614), (37.37890062088197, 55.73167906388319), (37.37750451918789, 55.734703664681774), (37.375610832015965, 55.734851959522246), (37.3723813571472, 55.74105626086403), (37.37014935714723, 55.746115620904355), (37.36944173016362, 55.750883999993725), (37.36975304365541, 55.76335905525834), (37.37244070571134, 55.76432079697595), (37.3724259757175, 55.76636979670426), (37.369922155757884, 55.76735417953104), (37.369892695770275, 55.76823419316575), (37.370214730163575, 55.782312184391266), (37.370493611114505, 55.78436801120489), (37.37120164550783, 55.78596427165359), (37.37284851456452, 55.7874378183096), (37.37608325135799, 55.7886695054807), (37.3764587460632, 55.78947647305964), (37.37530000265506, 55.79146512926804), (37.38235915344241, 55.79899647809345), (37.384344043655396, 55.80113596939471), (37.38594269577028, 55.80322699999366), (37.38711208598329, 55.804919036911976), (37.3880239841309, 55.806610999993666), (37.38928977249147, 55.81001864976979), (37.39038389947512, 55.81348641242801), (37.39235781481933, 55.81983538336746), (37.393709457672124, 55.82417822811877), (37.394685720901464, 55.82792275755836), (37.39557615344238, 55.830447148154136), (37.39844478226658, 55.83167107969975), (37.40019761214057, 55.83151823557964), (37.400398790382326, 55.83264967594742), (37.39659544313046, 55.83322180909622), (37.39667059524539, 55.83402792148566), (37.39682089947515, 55.83638877400216), (37.39643489154053, 55.83861656112751), (37.3955338994751, 55.84072348043264), (37.392680272491454, 55.84502158126453), (37.39241188227847, 55.84659117913199), (37.392529730163616, 55.84816071336481), (37.39486835714723, 55.85288092980303), (37.39873052645878, 55.859893456073635), (37.40272161111449, 55.86441833633205), (37.40697072750854, 55.867579567544375), (37.410007082016016, 55.868369880337), (37.4120992989502, 55.86920843741314), (37.412668021163924, 55.87055369615854), (37.41482461111453, 55.87170587948249), (37.41862266137694, 55.873183961039565), (37.42413732540892, 55.874879126654704), (37.4312182698669, 55.875614937236705), (37.43111093783558, 55.8762723478417), (37.43332105622856, 55.87706546369396), (37.43385747619623, 55.87790681284802), (37.441303050262405, 55.88027084462084), (37.44747234260555, 55.87942070143253), (37.44716141796871, 55.88072960917233), (37.44769797085568, 55.88121221323979), (37.45204320500181, 55.882080694420715), (37.45673176190186, 55.882346110794586), (37.463383999999984, 55.88252729504517), (37.46682797486874, 55.88294937719063), (37.470014457672086, 55.88361266759345), (37.47751410450743, 55.88546991372396), (37.47860317658232, 55.88534929207307), (37.48165826025772, 55.882563306475106), (37.48316434442331, 55.8815803226785), (37.483831555817645, 55.882427612793315), (37.483182967125686, 55.88372791409729), (37.483092277908824, 55.88495581062434), (37.4855716508179, 55.8875561994203), (37.486440636245746, 55.887827444039566), (37.49014203439328, 55.88897899871799), (37.493210285705544, 55.890208937135604), (37.497512451065035, 55.891342397444696), (37.49780744510645, 55.89174030252967), (37.49940333499519, 55.89239745507079), (37.50018383334346, 55.89339220941865), (37.52421672750851, 55.903869074155224), (37.52977457672118, 55.90564076517974), (37.53503220370484, 55.90661661218259), (37.54042858064267, 55.90714113744566), (37.54320461007303, 55.905645048442985), (37.545686966066306, 55.906608607018505), (37.54743976120755, 55.90788552162358), (37.55796999999999, 55.90901557907218), (37.572711542327866, 55.91059395704873), (37.57942799999998, 55.91073854155573), (37.58502865872187, 55.91009969268444), (37.58739968913264, 55.90794809960554), (37.59131567193598, 55.908713267595054), (37.612687423278814, 55.902866854295375), (37.62348079629517, 55.90041967242986), (37.635797880950896, 55.898141151686396), (37.649487626983664, 55.89639275532968), (37.65619302513125, 55.89572360207488), (37.66294133862307, 55.895295577183965), (37.66874564418033, 55.89505457604897), (37.67375601586915, 55.89254677027454), (37.67744661901856, 55.8947775867987), (37.688347, 55.89450045676125), (37.69480554232789, 55.89422926332761), (37.70107096560668, 55.89322256101114), (37.705962965606716, 55.891763491662616), (37.711885134918205, 55.889110234998974), (37.71682005026245, 55.886577568759876), (37.7199315476074, 55.88458159806678), (37.72234560316464, 55.882281005794134), (37.72364385977171, 55.8809452036196), (37.725371142837474, 55.8809722706006), (37.727870902099546, 55.88037213862385), (37.73394330422971, 55.877941504088696), (37.745339592590376, 55.87208120378722), (37.75525267724611, 55.86703807949492), (37.76919976190188, 55.859821640197474), (37.827835219574, 55.82962968399116), (37.83341438888553, 55.82575289922351), (37.83652584655761, 55.82188784027888), (37.83809213491821, 55.81612575504693), (37.83605359521481, 55.81460347077685), (37.83632178569025, 55.81276696067908), (37.838623105812026, 55.811486181656385), (37.83912198147584, 55.807329380532785), (37.839079078033414, 55.80510270463816), (37.83965844708251, 55.79940712529036), (37.840581150787344, 55.79131399999368), (37.84172564285271, 55.78000432402266)]); +``` + +3. Проверяем, сколько сотовых вышек находится в Москве: + +``` +SELECT count() FROM cell_towers WHERE pointInPolygon((lon, lat), (SELECT * FROM moscow)) + +┌─count()─┐ +│ 310463 │ +└─────────┘ + +1 rows in set. Elapsed: 0.067 sec. Processed 43.28 million rows, 692.42 MB (645.83 million rows/s., 10.33 GB/s.) +``` + +Вы можете протестировать другие запросы с помощью интерактивного ресурса [Playground](https://gh-api.clickhouse.tech/play?user=play). Например, [вот так](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIG1jYywgY291bnQoKSBGUk9NIGNlbGxfdG93ZXJzIEdST1VQIEJZIG1jYyBPUkRFUiBCWSBjb3VudCgpIERFU0M=). Однако, обратите внимание, что здесь нельзя создавать временные таблицы. diff --git a/docs/ru/getting-started/example-datasets/index.md b/docs/ru/getting-started/example-datasets/index.md index f590300adda..756b3a75dee 100644 --- a/docs/ru/getting-started/example-datasets/index.md +++ b/docs/ru/getting-started/example-datasets/index.md @@ -16,4 +16,5 @@ toc_title: "Введение" - [AMPLab Big Data Benchmark](amplab-benchmark.md) - [Данные о такси в Нью-Йорке](nyc-taxi.md) - [OnTime](ontime.md) +- [Вышки сотовой связи](../../getting-started/example-datasets/cell-towers.md) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 84ef62650e1..be9e2deab74 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -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 example.yandex.ru ``` +## interserver_https_port {#interserver-https-port} + +Порт для обмена данными между репликами ClickHouse по протоколу `HTTPS`. + +**Пример** + +``` xml +9010 +``` + +## interserver_https_host {#interserver-https-host} + +Имя хоста, которое могут использовать другие реплики для обращения к нему по протоколу `HTTPS`. + +**Пример** + +``` xml +example.yandex.ru +``` + + + ## interserver_http_credentials {#server-settings-interserver-http-credentials} Имя пользователя и пароль, использующиеся для аутентификации при [репликации](../../operations/server-configuration-parameters/settings.md) движками Replicated\*. Это имя пользователя и пароль используются только для взаимодействия между репликами кластера и никак не связаны с аутентификацией клиентов ClickHouse. Сервер проверяет совпадение имени и пароля для соединяющихся с ним реплик, а также использует это же имя и пароль для соединения с другими репликами. Соответственно, эти имя и пароль должны быть прописаны одинаковыми для всех реплик кластера. По умолчанию аутентификация не используется. +!!! note "Примечание" + Эти учетные данные являются общими для обмена данными по протоколам `HTTP` и `HTTPS`. + Раздел содержит следующие параметры: - `user` — имя пользователя. diff --git a/docs/ru/operations/settings/merge-tree-settings.md b/docs/ru/operations/settings/merge-tree-settings.md index bfc0b0a2644..f9093d379e3 100644 --- a/docs/ru/operations/settings/merge-tree-settings.md +++ b/docs/ru/operations/settings/merge-tree-settings.md @@ -55,6 +55,26 @@ Eсли число кусков в партиции превышает знач ClickHouse искусственно выполняет `INSERT` дольше (добавляет ‘sleep’), чтобы фоновый механизм слияния успевал слиять куски быстрее, чем они добавляются. +## inactive_parts_to_throw_insert {#inactive-parts-to-throw-insert} + +Если число неактивных кусков в партиции превышает значение `inactive_parts_to_throw_insert`, `INSERT` прерывается с исключением «Too many inactive parts (N). Parts cleaning are processing significantly slower than inserts». + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: 0 (не ограничено). + +## inactive_parts_to_delay_insert {#inactive-parts-to-delay-insert} + +Если число неактивных кусков в партиции больше или равно значению `inactive_parts_to_delay_insert`, `INSERT` искусственно замедляется. Это полезно, когда сервер не может быстро очистить неактивные куски. + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: 0 (не ограничено). + ## max_delay_to_insert {#max-delay-to-insert} Величина в секундах, которая используется для расчета задержки `INSERT`, если число кусков в партиции превышает значение [parts_to_delay_insert](#parts-to-delay-insert). diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 263b6904841..7acdd65051b 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -2746,4 +2746,15 @@ SELECT * FROM test2; Значение по умолчанию: `60`. +## check_query_single_value_result {#check_query_single_value_result} + +Определяет уровень детализации результата для запросов [CHECK TABLE](../../sql-reference/statements/check-table.md#checking-mergetree-tables) для таблиц семейства `MergeTree`. + +Возможные значения: + +- 0 — запрос возвращает статус каждого куска данных таблицы. +- 1 — запрос возвращает статус таблицы в целом. + +Значение по умолчанию: `0`. + [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) diff --git a/docs/ru/operations/update.md b/docs/ru/operations/update.md index 5c187ed1604..34f86656b61 100644 --- a/docs/ru/operations/update.md +++ b/docs/ru/operations/update.md @@ -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/) diff --git a/docs/ru/sql-reference/data-types/datetime64.md b/docs/ru/sql-reference/data-types/datetime64.md index 6576bf9dc0d..3a08da75bb7 100644 --- a/docs/ru/sql-reference/data-types/datetime64.md +++ b/docs/ru/sql-reference/data-types/datetime64.md @@ -7,9 +7,9 @@ toc_title: DateTime64 Позволяет хранить момент времени, который может быть представлен как календарная дата и время, с заданной суб-секундной точностью. -Размер тика/точность: 10-precision секунд, где precision - целочисленный параметр типа. +Размер тика (точность, precision): 10-precision секунд, где 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) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index 0948153362b..33a64f6bf26 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -15,10 +15,12 @@ - [`groupBitOr`](../../sql-reference/aggregate-functions/reference/groupbitor.md#groupbitor) - [`groupBitXor`](../../sql-reference/aggregate-functions/reference/groupbitxor.md#groupbitxor) - [`groupArrayArray`](../../sql-reference/aggregate-functions/reference/grouparray.md#agg_function-grouparray) -- [`groupUniqArrayArray`](../../sql-reference/aggregate-functions/reference/groupuniqarray.md#groupuniqarray) +- [`groupUniqArrayArray`](../../sql-reference/aggregate-functions/reference/groupuniqarray.md) - [`sumMap`](../../sql-reference/aggregate-functions/reference/summap.md#agg_functions-summap) - [`minMap`](../../sql-reference/aggregate-functions/reference/minmap.md#agg_functions-minmap) - [`maxMap`](../../sql-reference/aggregate-functions/reference/maxmap.md#agg_functions-maxmap) +- [`argMin`](../../sql-reference/aggregate-functions/reference/argmin.md) +- [`argMax`](../../sql-reference/aggregate-functions/reference/argmax.md) !!! note "Примечание" Значения `SimpleAggregateFunction(func, Type)` отображаются и хранятся так же, как и `Type`, поэтому комбинаторы [-Merge](../../sql-reference/aggregate-functions/combinators.md#aggregate_functions_combinators-merge) и [-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state) не требуются. diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md index 57f53390d1c..609ee225ce2 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md @@ -3,7 +3,7 @@ toc_priority: 44 toc_title: "Ключ и поля словаря" --- -# Ключ и поля словаря {#kliuch-i-polia-slovaria} +# Ключ и поля словаря {#dictionary-key-and-fields} Секция `` описывает ключ словаря и поля, доступные для запросов. @@ -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.
ClickHouse пытается привести значение из словаря к заданному типу данных. Например, в случае MySQL, в таблице-источнике поле может быть `TEXT`, `VARCHAR`, `BLOB`, но загружено может быть как `String`. [Nullable](../../../sql-reference/data-types/nullable.md) не поддерживается. | Да | -| `null_value` | Значение по умолчанию для несуществующего элемента.
В примере это пустая строка. Нельзя указать значение `NULL`. | Да | +| `type` | Тип данных ClickHouse.
ClickHouse пытается привести значение из словаря к заданному типу данных. Например, в случае MySQL, в таблице-источнике поле может быть `TEXT`, `VARCHAR`, `BLOB`, но загружено может быть как `String`.
[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` | Значение по умолчанию для несуществующего элемента.
В примере это пустая строка. Значение [NULL](../../syntax.md#null-literal) можно указывать только для типов `Nullable` (см. предыдущую строку с описанием типов). | Да | | `expression` | [Выражение](../../syntax.md#syntax-expressions), которое ClickHouse выполняет со значением.
Выражением может быть имя столбца в удаленной SQL базе. Таким образом, вы можете использовать его для создания псевдонима удаленного столбца.

Значение по умолчанию: нет выражения. | Нет | -| `hierarchical` | Если `true`, то атрибут содержит ключ предка для текущего элемента. Смотрите [Иерархические словари](external-dicts-dict-hierarchical.md).

Default value: `false`. | No | +| `hierarchical` | Если `true`, то атрибут содержит ключ предка для текущего элемента. Смотрите [Иерархические словари](external-dicts-dict-hierarchical.md).

Значение по умолчанию: `false`. | Нет | | `is_object_id` | Признак того, что запрос выполняется к документу MongoDB по `ObjectID`.

Значение по умолчанию: `false`. | Нет | -## Смотрите также {#smotrite-takzhe} +**Смотрите также** - [Функции для работы с внешними словарями](../../../sql-reference/functions/ext-dict-functions.md). - diff --git a/docs/ru/sql-reference/dictionaries/index.md b/docs/ru/sql-reference/dictionaries/index.md index bd432497be8..59c7518d0c5 100644 --- a/docs/ru/sql-reference/dictionaries/index.md +++ b/docs/ru/sql-reference/dictionaries/index.md @@ -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). diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 6ef7dc01b6a..04af599c09a 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -645,3 +645,66 @@ SELECT decodeXMLComponent('< Σ >'); - [Мнемоники в HTML](https://ru.wikipedia.org/wiki/%D0%9C%D0%BD%D0%B5%D0%BC%D0%BE%D0%BD%D0%B8%D0%BA%D0%B8_%D0%B2_HTML) + + +## extractTextFromHTML {#extracttextfromhtml} + +Функция для извлечения текста из HTML или XHTML. +Она не соответствует всем HTML, XML или XHTML стандартам на 100%, но ее реализация достаточно точная и быстрая. Правила обработки следующие: + +1. Комментарии удаляются. Пример: ``. Комментарий должен оканчиваться символами `-->`. Вложенные комментарии недопустимы. +Примечание: конструкции наподобие `` и `` не являются допустимыми комментариями в HTML, но они будут удалены согласно другим правилам. +2. Содержимое CDATA вставляется дословно. Примечание: формат CDATA специфичен для XML/XHTML. Но он обрабатывается всегда по принципу "наилучшего возможного результата". +3. Элементы `script` и `style` удаляются вместе со всем содержимым. Примечание: предполагается, что закрывающий тег не может появиться внутри содержимого. Например, в JS строковый литерал должен быть экранирован как `"<\/script>"`. +Примечание: комментарии и CDATA возможны внутри `script` или `style` - тогда закрывающие теги не ищутся внутри CDATA. Пример: `]]>`. Но они ищутся внутри комментариев. Иногда возникают сложные случаи: ` var y = "-->"; alert(x + y);` +Примечание: `script` и `style` могут быть названиями пространств имен XML - тогда они не обрабатываются как обычные элементы `script` или `style`. Пример: `Hello`. +Примечание: пробелы возможны после имени закрывающего тега: ``, но не перед ним: `< / script>`. +4. Другие теги или элементы, подобные тегам, удаляются, а их внутреннее содержимое остается. Пример: `.` +Примечание: ожидается, что такой HTML является недопустимым: `` +Примечание: функция также удаляет подобные тегам элементы: `<>`, ``, и т. д. +Примечание: если встречается тег без завершающего символа `>`, то удаляется этот тег и весь следующий за ним текст: `world`, `Helloworld` — в HTML нет пробелов, но функция вставляет их. Также следует учитывать такие варианты написания: `Hello

world

`, `Hello
world`. Подобные результаты выполнения функции могут использоваться для анализа данных, например, для преобразования HTML-текста в набор используемых слов. +7. Также обратите внимание, что правильная обработка пробелов требует поддержки `
` и свойств CSS `display` и `white-space`.
+
+**Синтаксис**
+
+``` sql
+extractTextFromHTML(x)
+```
+
+**Аргументы**
+
+-   `x` — текст для обработки. [String](../../sql-reference/data-types/string.md). 
+
+**Возвращаемое значение**
+
+-   Извлеченный текст.
+
+Тип: [String](../../sql-reference/data-types/string.md).
+
+**Пример**
+
+Первый пример содержит несколько тегов и комментарий. На этом примере также видно, как обрабатываются пробелы.
+Второй пример показывает обработку `CDATA` и тега `script`.
+В третьем примере текст выделяется из полного HTML ответа, полученного с помощью функции [url](../../sql-reference/table-functions/url.md).
+
+Запрос:
+
+``` sql
+SELECT extractTextFromHTML(' 

A text withtags.

'); +SELECT extractTextFromHTML('CDATA]]> '); +SELECT extractTextFromHTML(html) FROM url('http://www.donothingfor2minutes.com/', RawBLOB, 'html String'); +``` + +Результат: + +``` text +A text with tags . +The content within CDATA +Do Nothing for 2 Minutes 2:00   +``` diff --git a/docs/ru/sql-reference/statements/alter/ttl.md b/docs/ru/sql-reference/statements/alter/ttl.md index e949c992bbe..2a2d10b69de 100644 --- a/docs/ru/sql-reference/statements/alter/ttl.md +++ b/docs/ru/sql-reference/statements/alter/ttl.md @@ -82,4 +82,4 @@ SELECT * FROM table_with_ttl; ### Смотрите также - Подробнее о [свойстве TTL](../../../engines/table-engines/mergetree-family/mergetree.md#mergetree-column-ttl). - +- Изменить столбец [с TTL](../../../sql-reference/statements/alter/column.md#alter_modify-column). \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/check-table.md b/docs/ru/sql-reference/statements/check-table.md index 10336f821d0..9592c1a5bc2 100644 --- a/docs/ru/sql-reference/statements/check-table.md +++ b/docs/ru/sql-reference/statements/check-table.md @@ -29,9 +29,36 @@ CHECK TABLE [db.]name В движках `*Log` не предусмотрено автоматическое восстановление данных после сбоя. Используйте запрос `CHECK TABLE`, чтобы своевременно выявлять повреждение данных. -Для движков из семейства `MergeTree` запрос `CHECK TABLE` показывает статус проверки для каждого отдельного куска данных таблицы на локальном сервере. +## Проверка таблиц семейства MergeTree {#checking-mergetree-tables} -**Что делать, если данные повреждены** +Для таблиц семейства `MergeTree` если [check_query_single_value_result](../../operations/settings/settings.md#check_query_single_value_result) = 0, запрос `CHECK TABLE` возвращает статус каждого куска данных таблицы на локальном сервере. + +```sql +SET check_query_single_value_result = 0; +CHECK TABLE test_table; +``` + +```text +┌─part_path─┬─is_passed─┬─message─┐ +│ all_1_4_1 │ 1 │ │ +│ all_1_4_2 │ 1 │ │ +└───────────┴───────────┴─────────┘ +``` + +Если `check_query_single_value_result` = 0, запрос `CHECK TABLE` возвращает статус таблицы в целом. + +```sql +SET check_query_single_value_result = 1; +CHECK TABLE test_table; +``` + +```text +┌─result─┐ +│ 1 │ +└────────┘ +``` + +## Что делать, если данные повреждены {#if-data-is-corrupted} В этом случае можно скопировать оставшиеся неповрежденные данные в другую таблицу. Для этого: diff --git a/docs/ru/sql-reference/statements/rename.md b/docs/ru/sql-reference/statements/rename.md index 104918c1a73..192426dbafa 100644 --- a/docs/ru/sql-reference/statements/rename.md +++ b/docs/ru/sql-reference/statements/rename.md @@ -3,8 +3,16 @@ toc_priority: 48 toc_title: RENAME --- -# RENAME {#misc_operations-rename} +# RENAME Statement {#misc_operations-rename} +## RENAME DATABASE {#misc_operations-rename_database} +Переименование базы данных + +``` +RENAME DATABASE atomic_database1 TO atomic_database2 [ON CLUSTER cluster] +``` + +## RENAME TABLE {#misc_operations-rename_table} Переименовывает одну или несколько таблиц. ``` sql @@ -12,5 +20,3 @@ RENAME TABLE [db11.]name11 TO [db12.]name12, [db21.]name21 TO [db22.]name22, ... ``` Переименовывание таблицы является лёгкой операцией. Если вы указали после `TO` другую базу данных, то таблица будет перенесена в эту базу данных. При этом, директории с базами данных должны быть расположены в одной файловой системе (иначе возвращается ошибка). В случае переименования нескольких таблиц в одном запросе — это неатомарная операция, может выполнится частично, запросы в других сессиях могут получить ошибку `Table ... doesn't exist...`. - - diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index c3600e5812a..c917dbe30a3 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -36,6 +36,9 @@ option (ENABLE_CLICKHOUSE_OBFUSCATOR "Table data obfuscator (convert real data t option (ENABLE_CLICKHOUSE_ODBC_BRIDGE "HTTP-server working like a proxy to ODBC driver" ${ENABLE_CLICKHOUSE_ALL}) +option (ENABLE_CLICKHOUSE_LIBRARY_BRIDGE "HTTP-server working like a proxy to Library dictionary source" + ${ENABLE_CLICKHOUSE_ALL}) + # https://presentations.clickhouse.tech/matemarketing_2020/ option (ENABLE_CLICKHOUSE_GIT_IMPORT "A tool to analyze Git repositories" ${ENABLE_CLICKHOUSE_ALL}) @@ -109,6 +112,12 @@ else() message(STATUS "ODBC bridge mode: OFF") endif() +if (ENABLE_CLICKHOUSE_LIBRARY_BRIDGE) + message(STATUS "Library bridge mode: ON") +else() + message(STATUS "Library bridge mode: OFF") +endif() + if (ENABLE_CLICKHOUSE_INSTALL) message(STATUS "ClickHouse install: ON") else() @@ -194,6 +203,10 @@ if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) add_subdirectory (odbc-bridge) endif () +if (ENABLE_CLICKHOUSE_LIBRARY_BRIDGE) + add_subdirectory (library-bridge) +endif () + if (CLICKHOUSE_ONE_SHARED) add_library(clickhouse-lib SHARED ${CLICKHOUSE_SERVER_SOURCES} ${CLICKHOUSE_CLIENT_SOURCES} ${CLICKHOUSE_LOCAL_SOURCES} ${CLICKHOUSE_BENCHMARK_SOURCES} ${CLICKHOUSE_COPIER_SOURCES} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_SOURCES} ${CLICKHOUSE_COMPRESSOR_SOURCES} ${CLICKHOUSE_FORMAT_SOURCES} ${CLICKHOUSE_OBFUSCATOR_SOURCES} ${CLICKHOUSE_GIT_IMPORT_SOURCES} ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) target_link_libraries(clickhouse-lib ${CLICKHOUSE_SERVER_LINK} ${CLICKHOUSE_CLIENT_LINK} ${CLICKHOUSE_LOCAL_LINK} ${CLICKHOUSE_BENCHMARK_LINK} ${CLICKHOUSE_COPIER_LINK} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_LINK} ${CLICKHOUSE_COMPRESSOR_LINK} ${CLICKHOUSE_FORMAT_LINK} ${CLICKHOUSE_OBFUSCATOR_LINK} ${CLICKHOUSE_GIT_IMPORT_LINK} ${CLICKHOUSE_ODBC_BRIDGE_LINK}) @@ -209,6 +222,10 @@ if (CLICKHOUSE_SPLIT_BINARY) list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-odbc-bridge) endif () + if (ENABLE_CLICKHOUSE_LIBRARY_BRIDGE) + list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-library-bridge) + endif () + set_target_properties(${CLICKHOUSE_ALL_TARGETS} PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..) add_custom_target (clickhouse-bundle ALL DEPENDS ${CLICKHOUSE_ALL_TARGETS}) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index a0e2ea155ba..1d2b579db3a 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -95,8 +95,8 @@ public: comparison_info_total.emplace_back(std::make_shared()); } - global_context.makeGlobalContext(); - global_context.setSettings(settings); + global_context->makeGlobalContext(); + global_context->setSettings(settings); std::cerr << std::fixed << std::setprecision(3); @@ -159,7 +159,7 @@ private: bool print_stacktrace; const Settings & settings; SharedContextHolder shared_context; - Context global_context; + ContextPtr global_context; QueryProcessingStage::Enum query_processing_stage; /// Don't execute new queries after timelimit or SIGINT or exception diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 0c5bbaf3edd..1aec3677b41 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -21,7 +21,7 @@ #include #include #include -#include +#include #include #include #include @@ -191,7 +191,7 @@ private: bool has_vertical_output_suffix = false; /// Is \G present at the end of the query string? SharedContextHolder shared_context = Context::createShared(); - Context context = Context::createGlobal(shared_context.get()); + ContextPtr context = Context::createGlobal(shared_context.get()); /// Buffer that reads from stdin in batch mode. ReadBufferFromFileDescriptor std_in {STDIN_FILENO}; @@ -274,20 +274,20 @@ private: configReadClient(config(), home_path); - context.setApplicationType(Context::ApplicationType::CLIENT); - context.setQueryParameters(query_parameters); + context->setApplicationType(Context::ApplicationType::CLIENT); + context->setQueryParameters(query_parameters); /// settings and limits could be specified in config file, but passed settings has higher priority - for (const auto & setting : context.getSettingsRef().allUnchanged()) + for (const auto & setting : context->getSettingsRef().allUnchanged()) { const auto & name = setting.getName(); if (config().has(name)) - context.setSetting(name, config().getString(name)); + context->setSetting(name, config().getString(name)); } /// Set path for format schema files if (config().has("format_schema_path")) - context.setFormatSchemaPath(Poco::Path(config().getString("format_schema_path")).toString()); + context->setFormatSchemaPath(Poco::Path(config().getString("format_schema_path")).toString()); /// Initialize query_id_formats if any if (config().has("query_id_formats")) @@ -538,15 +538,15 @@ private: else format = config().getString("format", is_interactive ? "PrettyCompact" : "TabSeparated"); - format_max_block_size = config().getInt("format_max_block_size", context.getSettingsRef().max_block_size); + format_max_block_size = config().getInt("format_max_block_size", context->getSettingsRef().max_block_size); insert_format = "Values"; /// Setting value from cmd arg overrides one from config - if (context.getSettingsRef().max_insert_block_size.changed) - insert_format_max_block_size = context.getSettingsRef().max_insert_block_size; + if (context->getSettingsRef().max_insert_block_size.changed) + insert_format_max_block_size = context->getSettingsRef().max_insert_block_size; else - insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size); + insert_format_max_block_size = config().getInt("insert_format_max_block_size", context->getSettingsRef().max_insert_block_size); if (!is_interactive) { @@ -555,7 +555,7 @@ private: ignore_error = config().getBool("ignore-error", false); } - ClientInfo & client_info = context.getClientInfo(); + ClientInfo & client_info = context->getClientInfo(); client_info.setInitialQuery(); client_info.quota_key = config().getString("quota_key", ""); @@ -563,7 +563,7 @@ private: /// Initialize DateLUT here to avoid counting time spent here as query execution time. const auto local_tz = DateLUT::instance().getTimeZone(); - if (!context.getSettingsRef().use_client_time_zone) + if (!context->getSettingsRef().use_client_time_zone) { const auto & time_zone = connection->getServerTimezone(connection_parameters.timeouts); if (!time_zone.empty()) @@ -738,7 +738,7 @@ private: { auto query_id = config().getString("query_id", ""); if (!query_id.empty()) - context.setCurrentQueryId(query_id); + context->setCurrentQueryId(query_id); nonInteractive(); @@ -1038,7 +1038,7 @@ private: { Tokens tokens(this_query_begin, all_queries_end); IParser::Pos token_iterator(tokens, - context.getSettingsRef().max_parser_depth); + context->getSettingsRef().max_parser_depth); if (!token_iterator.isValid()) { break; @@ -1087,7 +1087,7 @@ private: if (ignore_error) { Tokens tokens(this_query_begin, all_queries_end); - IParser::Pos token_iterator(tokens, context.getSettingsRef().max_parser_depth); + IParser::Pos token_iterator(tokens, context->getSettingsRef().max_parser_depth); while (token_iterator->type != TokenType::Semicolon && token_iterator.isValid()) ++token_iterator; this_query_begin = token_iterator->end; @@ -1133,7 +1133,7 @@ private: // beneficial so that we see proper trailing comments in "echo" and // server log. adjustQueryEnd(this_query_end, all_queries_end, - context.getSettingsRef().max_parser_depth); + context->getSettingsRef().max_parser_depth); // full_query is the query + inline INSERT data + trailing comments // (the latter is our best guess for now). @@ -1173,7 +1173,7 @@ private: { this_query_end = insert_ast->end; adjustQueryEnd(this_query_end, all_queries_end, - context.getSettingsRef().max_parser_depth); + context->getSettingsRef().max_parser_depth); } // Now we know for sure where the query ends. @@ -1290,7 +1290,7 @@ private: // Prints changed settings to stderr. Useful for debugging fuzzing failures. void printChangedSettings() const { - const auto & changes = context.getSettingsRef().changes(); + const auto & changes = context->getSettingsRef().changes(); if (!changes.empty()) { fmt::print(stderr, "Changed settings: "); @@ -1590,11 +1590,11 @@ private: if (is_interactive) { // Generate a new query_id - context.setCurrentQueryId(""); + context->setCurrentQueryId(""); for (const auto & query_id_format : query_id_formats) { writeString(query_id_format.first, std_out); - writeString(fmt::format(query_id_format.second, fmt::arg("query_id", context.getCurrentQueryId())), std_out); + writeString(fmt::format(query_id_format.second, fmt::arg("query_id", context->getCurrentQueryId())), std_out); writeChar('\n', std_out); std_out.next(); } @@ -1610,12 +1610,12 @@ private: { /// Temporarily apply query settings to context. std::optional 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) - old_settings.emplace(context.getSettingsRef()); - context.applySettingsChanges(settings_ast.as()->changes); + old_settings.emplace(context->getSettingsRef()); + context->applySettingsChanges(settings_ast.as()->changes); }; const auto * insert = parsed_query->as(); if (insert && insert->settings_ast) @@ -1653,7 +1653,7 @@ private: if (change.name == "profile") current_profile = change.value.safeGet(); else - context.applySettingChange(change); + context->applySettingChange(change); } } @@ -1725,10 +1725,10 @@ private: connection->sendQuery( connection_parameters.timeouts, query_to_send, - context.getCurrentQueryId(), + context->getCurrentQueryId(), query_processing_stage, - &context.getSettingsRef(), - &context.getClientInfo(), + &context->getSettingsRef(), + &context->getClientInfo(), true); sendExternalTables(); @@ -1766,10 +1766,10 @@ private: connection->sendQuery( connection_parameters.timeouts, query_to_send, - context.getCurrentQueryId(), + context->getCurrentQueryId(), query_processing_stage, - &context.getSettingsRef(), - &context.getClientInfo(), + &context->getSettingsRef(), + &context->getClientInfo(), true); sendExternalTables(); @@ -1792,7 +1792,7 @@ private: ParserQuery parser(end); ASTPtr res; - const auto & settings = context.getSettingsRef(); + const auto & settings = context->getSettingsRef(); size_t max_length = 0; if (!allow_multi_statements) max_length = settings.max_query_size; @@ -1880,7 +1880,7 @@ private: current_format = insert->format; } - BlockInputStreamPtr block_input = context.getInputFormat( + BlockInputStreamPtr block_input = context->getInputFormat( current_format, buf, sample, insert_format_max_block_size); if (columns_description.hasDefaults()) @@ -2204,9 +2204,9 @@ private: /// It is not clear how to write progress with parallel formatting. It may increase code complexity significantly. if (!need_render_progress) - block_out_stream = context.getOutputStreamParallelIfPossible(current_format, *out_buf, block); + block_out_stream = context->getOutputStreamParallelIfPossible(current_format, *out_buf, block); else - block_out_stream = context.getOutputStream(current_format, *out_buf, block); + block_out_stream = context->getOutputStream(current_format, *out_buf, block); block_out_stream->writePrefix(); } @@ -2710,12 +2710,12 @@ public: } } - context.makeGlobalContext(); - context.setSettings(cmd_settings); + context->makeGlobalContext(); + context->setSettings(cmd_settings); /// Copy settings-related program options to config. /// TODO: Is this code necessary? - for (const auto & setting : context.getSettingsRef().all()) + for (const auto & setting : context->getSettingsRef().all()) { const auto & name = setting.getName(); if (options.count(name)) @@ -2807,7 +2807,7 @@ public: { std::string traceparent = options["opentelemetry-traceparent"].as(); std::string error; - if (!context.getClientInfo().client_trace_context.parseTraceparentHeader( + if (!context->getClientInfo().client_trace_context.parseTraceparentHeader( traceparent, error)) { throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -2818,7 +2818,7 @@ public: if (options.count("opentelemetry-tracestate")) { - context.getClientInfo().client_trace_context.tracestate = + context->getClientInfo().client_trace_context.tracestate = options["opentelemetry-tracestate"].as(); } diff --git a/programs/client/QueryFuzzer.cpp b/programs/client/QueryFuzzer.cpp index 0c8dc0731f9..6243e2c82ec 100644 --- a/programs/client/QueryFuzzer.cpp +++ b/programs/client/QueryFuzzer.cpp @@ -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( - 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); diff --git a/programs/config_tools.h.in b/programs/config_tools.h.in index 7cb5a6d883a..abe9ef8c562 100644 --- a/programs/config_tools.h.in +++ b/programs/config_tools.h.in @@ -15,3 +15,4 @@ #cmakedefine01 ENABLE_CLICKHOUSE_GIT_IMPORT #cmakedefine01 ENABLE_CLICKHOUSE_INSTALL #cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE +#cmakedefine01 ENABLE_CLICKHOUSE_LIBRARY_BRIDGE diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index bede40d65f5..aa9b359993e 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes void ClusterCopier::init() { - auto zookeeper = context.getZooKeeper(); + auto zookeeper = getContext()->getZooKeeper(); task_description_watch_callback = [this] (const Coordination::WatchResponse & response) { @@ -39,14 +39,14 @@ void ClusterCopier::init() task_cluster_initial_config = task_cluster_current_config; task_cluster->loadTasks(*task_cluster_initial_config); - context.setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix); + getContext()->setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix); /// Set up shards and their priority task_cluster->random_engine.seed(task_cluster->random_device()); for (auto & task_table : task_cluster->table_tasks) { - task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); - task_table.cluster_push = context.getCluster(task_table.cluster_push_name); + task_table.cluster_pull = getContext()->getCluster(task_table.cluster_pull_name); + task_table.cluster_push = getContext()->getCluster(task_table.cluster_push_name); task_table.initShards(task_cluster->random_engine); } @@ -206,7 +206,7 @@ void ClusterCopier::uploadTaskDescription(const std::string & task_path, const s if (task_config_str.empty()) return; - auto zookeeper = context.getZooKeeper(); + auto zookeeper = getContext()->getZooKeeper(); zookeeper->createAncestors(local_task_description_path); auto code = zookeeper->tryCreate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); @@ -219,7 +219,7 @@ void ClusterCopier::uploadTaskDescription(const std::string & task_path, const s void ClusterCopier::reloadTaskDescription() { - auto zookeeper = context.getZooKeeper(); + auto zookeeper = getContext()->getZooKeeper(); task_description_watch_zookeeper = zookeeper; String task_config_str; @@ -235,7 +235,7 @@ void ClusterCopier::reloadTaskDescription() /// Setup settings task_cluster->reloadSettings(*config); - context.setSettings(task_cluster->settings_common); + getContext()->setSettings(task_cluster->settings_common); task_cluster_current_config = config; task_description_current_stat = stat; @@ -440,7 +440,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons { LOG_DEBUG(log, "Check that all shards processed partition {} piece {} successfully", partition_name, piece_number); - auto zookeeper = context.getZooKeeper(); + auto zookeeper = getContext()->getZooKeeper(); /// Collect all shards that contain partition piece number piece_number. Strings piece_status_paths; @@ -532,7 +532,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t LOG_DEBUG(log, "Try to move {} to destination table", partition_name); - auto zookeeper = context.getZooKeeper(); + auto zookeeper = getContext()->getZooKeeper(); const auto current_partition_attach_is_active = task_table.getPartitionAttachIsActivePath(partition_name); const auto current_partition_attach_is_done = task_table.getPartitionAttachIsDonePath(partition_name); @@ -1095,7 +1095,7 @@ TaskStatus ClusterCopier::tryCreateDestinationTable(const ConnectionTimeouts & t = rewriteCreateQueryStorage(task_shard->current_pull_table_create_query, task_table.table_push, task_table.engine_push_ast); auto & create = create_query_push_ast->as(); create.if_not_exists = true; - InterpreterCreateQuery::prepareOnClusterQuery(create, context, task_table.cluster_push_name); + InterpreterCreateQuery::prepareOnClusterQuery(create, getContext(), task_table.cluster_push_name); String query = queryToString(create_query_push_ast); LOG_DEBUG(log, "Create destination tables. Query: {}", query); @@ -1211,7 +1211,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto split_table_for_current_piece = task_shard.list_of_split_tables_on_shard[current_piece_number]; - auto zookeeper = context.getZooKeeper(); + auto zookeeper = getContext()->getZooKeeper(); const String piece_is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); const String piece_is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); @@ -1262,7 +1262,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( ParserQuery p_query(query.data() + query.size()); - const auto & settings = context.getSettingsRef(); + const auto & settings = getContext()->getSettingsRef(); return parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth); }; @@ -1366,10 +1366,10 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( ASTPtr query_select_ast = get_select_query(split_table_for_current_piece, "count()", /*enable_splitting*/ true); UInt64 count; { - Context local_context = context; + auto local_context = Context::createCopy(context); // Use pull (i.e. readonly) settings, but fetch data from destination servers - local_context.setSettings(task_cluster->settings_pull); - local_context.setSetting("skip_unavailable_shards", true); + local_context->setSettings(task_cluster->settings_pull); + local_context->setSetting("skip_unavailable_shards", true); Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().getInputStream()); count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; @@ -1468,7 +1468,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( query += "INSERT INTO " + getQuotedTable(split_table_for_current_piece) + " VALUES "; ParserQuery p_query(query.data() + query.size()); - const auto & settings = context.getSettingsRef(); + const auto & settings = getContext()->getSettingsRef(); query_insert_ast = parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth); LOG_DEBUG(log, "Executing INSERT query: {}", query); @@ -1476,18 +1476,18 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( try { - std::unique_ptr context_select = std::make_unique(context); + auto context_select = Context::createCopy(context); context_select->setSettings(task_cluster->settings_pull); - std::unique_ptr context_insert = std::make_unique(context); + auto context_insert = Context::createCopy(context); context_insert->setSettings(task_cluster->settings_push); /// Custom INSERT SELECT implementation BlockInputStreamPtr input; BlockOutputStreamPtr output; { - BlockIO io_select = InterpreterFactory::get(query_select_ast, *context_select)->execute(); - BlockIO io_insert = InterpreterFactory::get(query_insert_ast, *context_insert)->execute(); + BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); + BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); input = io_select.getInputStream(); output = io_insert.out; @@ -1581,7 +1581,7 @@ void ClusterCopier::dropAndCreateLocalTable(const ASTPtr & create_ast) const auto & create = create_ast->as(); dropLocalTableIfExists({create.database, create.table}); - InterpreterCreateQuery interpreter(create_ast, context); + InterpreterCreateQuery interpreter(create_ast, getContext()); interpreter.execute(); } @@ -1592,7 +1592,7 @@ void ClusterCopier::dropLocalTableIfExists(const DatabaseAndTableName & table_na drop_ast->database = table_name.first; drop_ast->table = table_name.second; - InterpreterDropQuery interpreter(drop_ast, context); + InterpreterDropQuery interpreter(drop_ast, getContext()); interpreter.execute(); } @@ -1654,8 +1654,8 @@ void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskT String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings & settings) { - Context remote_context(context); - remote_context.setSettings(settings); + auto remote_context = Context::createCopy(context); + remote_context->setSettings(settings); String query = "SHOW CREATE TABLE " + getQuotedTable(table); Block block = getBlockWithAllStreamData(std::make_shared( @@ -1674,7 +1674,7 @@ ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & time task_cluster->settings_pull); ParserCreateQuery parser_create_query; - const auto & settings = context.getSettingsRef(); + const auto & settings = getContext()->getSettingsRef(); return parseQuery(parser_create_query, create_query_pull_str, settings.max_query_size, settings.max_parser_depth); } @@ -1703,7 +1703,7 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout /// Create special cluster with single shard String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; ClusterPtr cluster_pull_current_shard = task_table.cluster_pull->getClusterWithSingleShard(task_shard.indexInCluster()); - context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); + getContext()->setCluster(shard_read_cluster_name, cluster_pull_current_shard); auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); @@ -1763,13 +1763,13 @@ std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti } ParserQuery parser_query(query.data() + query.size()); - const auto & settings = context.getSettingsRef(); + const auto & settings = getContext()->getSettingsRef(); ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth); LOG_DEBUG(log, "Computing destination partition set, executing query: {}", query); - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); + auto local_context = Context::createCopy(context); + local_context->setSettings(task_cluster->settings_pull); Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()); if (block) @@ -1809,11 +1809,11 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, LOG_DEBUG(log, "Checking shard {} for partition {} existence, executing query: {}", task_shard.getDescription(), partition_quoted_name, query); ParserQuery parser_query(query.data() + query.size()); -const auto & settings = context.getSettingsRef(); + const auto & settings = getContext()->getSettingsRef(); ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth); - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); + auto local_context = Context::createCopy(context); + local_context->setSettings(task_cluster->settings_pull); return InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows() != 0; } @@ -1848,11 +1848,11 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi LOG_DEBUG(log, "Checking shard {} for partition {} piece {} existence, executing query: {}", task_shard.getDescription(), partition_quoted_name, std::to_string(current_piece_number), query); ParserQuery parser_query(query.data() + query.size()); - const auto & settings = context.getSettingsRef(); + const auto & settings = getContext()->getSettingsRef(); ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth); - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); + auto local_context = Context::createCopy(context); + local_context->setSettings(task_cluster->settings_pull); auto result = InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows(); if (result != 0) LOG_DEBUG(log, "Partition {} piece number {} is PRESENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription()); @@ -1908,7 +1908,7 @@ UInt64 ClusterCopier::executeQueryOnCluster( /// In that case we don't have local replicas, but do it just in case for (UInt64 i = 0; i < num_local_replicas; ++i) { - auto interpreter = InterpreterFactory::get(query_ast, context); + auto interpreter = InterpreterFactory::get(query_ast, getContext()); interpreter->execute(); if (increment_and_check_exit()) @@ -1923,8 +1923,8 @@ UInt64 ClusterCopier::executeQueryOnCluster( auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(shard_settings).getSaturated(shard_settings.max_execution_time); auto connections = shard.pool->getMany(timeouts, &shard_settings, pool_mode); - Context shard_context(context); - shard_context.setSettings(shard_settings); + auto shard_context = Context::createCopy(context); + shard_context->setSettings(shard_settings); for (auto & connection : connections) { diff --git a/programs/copier/ClusterCopier.h b/programs/copier/ClusterCopier.h index 95bb54cf4e1..e875ca7df2e 100644 --- a/programs/copier/ClusterCopier.h +++ b/programs/copier/ClusterCopier.h @@ -12,18 +12,17 @@ namespace DB { -class ClusterCopier +class ClusterCopier : WithContext { public: ClusterCopier(const String & task_path_, const String & host_id_, const String & proxy_database_name_, - Context & context_) - : + ContextPtr context_) + : WithContext(context_), task_zookeeper_path(task_path_), host_id(host_id_), working_database_name(proxy_database_name_), - context(context_), log(&Poco::Logger::get("ClusterCopier")) {} void init(); @@ -36,7 +35,7 @@ public: /// Compute set of partitions, assume set of partitions aren't changed during the processing void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0); - void uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force); + void uploadTaskDescription(const std::string & task_path, const std::string & task_file, bool force); void reloadTaskDescription(); @@ -120,7 +119,7 @@ protected: /// Removes MATERIALIZED and ALIAS columns from create table query static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); - bool tryDropPartitionPiece(ShardPartition & task_partition, const size_t current_piece_number, + bool tryDropPartitionPiece(ShardPartition & task_partition, size_t current_piece_number, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); static constexpr UInt64 max_table_tries = 3; @@ -141,7 +140,7 @@ protected: TaskStatus processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, - const size_t current_piece_number, + size_t current_piece_number, bool is_unprioritized_task); void dropAndCreateLocalTable(const ASTPtr & create_ast); @@ -219,7 +218,6 @@ private: bool experimental_use_sample_offset{false}; - Context & context; Poco::Logger * log; std::chrono::milliseconds default_sleep_time{1000}; diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index e3169a49ecf..d3fff616b65 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -110,9 +111,9 @@ void ClusterCopierApp::mainImpl() LOG_INFO(log, "Starting clickhouse-copier (id {}, host_id {}, path {}, revision {})", process_id, host_id, process_path, ClickHouseRevision::getVersionRevision()); SharedContextHolder shared_context = Context::createShared(); - auto context = std::make_unique(Context::createGlobal(shared_context.get())); + auto 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); @@ -127,13 +128,13 @@ void ClusterCopierApp::mainImpl() registerFormats(); static const std::string default_database = "_local"; - DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, *context)); + DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, context)); context->setCurrentDatabase(default_database); /// Initialize query scope just in case. - CurrentThread::QueryScope query_scope(*context); + CurrentThread::QueryScope query_scope(context); - auto copier = std::make_unique(task_path, host_id, default_database, *context); + auto copier = std::make_unique(task_path, host_id, default_database, context); copier->setSafeMode(is_safe_mode); copier->setCopyFaultProbability(copy_fault_probability); copier->setMoveFaultProbability(move_fault_probability); diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index ea2be469945..bec612a8226 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -222,8 +222,8 @@ Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) { String pk_column = primary_key_expr_list->children[i]->getColumnName(); if (pk_column != sorting_key_column) - throw Exception("Primary key must be a prefix of the sorting key, but in position " - + toString(i) + " its column is " + pk_column + ", not " + sorting_key_column, + throw Exception("Primary key must be a prefix of the sorting key, but the column in the position " + + toString(i) + " is " + sorting_key_column +", not " + pk_column, ErrorCodes::BAD_ARGUMENTS); if (!primary_key_columns_set.emplace(pk_column).second) diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index ba3d6e8557b..5bf19191353 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -102,8 +102,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv) } SharedContextHolder shared_context = Context::createShared(); - Context context = Context::createGlobal(shared_context.get()); - context.makeGlobalContext(); + auto context = Context::createGlobal(shared_context.get()); + context->makeGlobalContext(); registerFunctions(); registerAggregateFunctions(); diff --git a/programs/library-bridge/CMakeLists.txt b/programs/library-bridge/CMakeLists.txt new file mode 100644 index 00000000000..5ceff47ee0c --- /dev/null +++ b/programs/library-bridge/CMakeLists.txt @@ -0,0 +1,25 @@ +set (CLICKHOUSE_LIBRARY_BRIDGE_SOURCES + library-bridge.cpp + library-log.cpp + LibraryBridge.cpp + Handlers.cpp + HandlerFactory.cpp + SharedLibraryHandler.cpp + SharedLibraryHandlerFactory.cpp +) + +if (OS_LINUX) + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") +endif () + +add_executable(clickhouse-library-bridge ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES}) + +target_link_libraries(clickhouse-library-bridge PRIVATE + daemon + dbms + bridge +) + +set_target_properties(clickhouse-library-bridge PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..) + +install(TARGETS clickhouse-library-bridge RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) diff --git a/programs/library-bridge/HandlerFactory.cpp b/programs/library-bridge/HandlerFactory.cpp new file mode 100644 index 00000000000..9f53a24156f --- /dev/null +++ b/programs/library-bridge/HandlerFactory.cpp @@ -0,0 +1,23 @@ +#include "HandlerFactory.h" + +#include +#include +#include "Handlers.h" + + +namespace DB +{ + std::unique_ptr LibraryBridgeHandlerFactory::createRequestHandler(const HTTPServerRequest & request) + { + Poco::URI uri{request.getURI()}; + LOG_DEBUG(log, "Request URI: {}", uri.toString()); + + if (uri == "/ping" && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) + return std::make_unique(keep_alive_timeout); + + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) + return std::make_unique(keep_alive_timeout, getContext()); + + return nullptr; + } +} diff --git a/programs/library-bridge/HandlerFactory.h b/programs/library-bridge/HandlerFactory.h new file mode 100644 index 00000000000..93f0721bf01 --- /dev/null +++ b/programs/library-bridge/HandlerFactory.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class SharedLibraryHandler; +using SharedLibraryHandlerPtr = std::shared_ptr; + +/// Factory for '/ping', '/' handlers. +class LibraryBridgeHandlerFactory : public HTTPRequestHandlerFactory, WithContext +{ +public: + LibraryBridgeHandlerFactory( + const std::string & name_, + size_t keep_alive_timeout_, + ContextPtr context_) + : WithContext(context_) + , log(&Poco::Logger::get(name_)) + , name(name_) + , keep_alive_timeout(keep_alive_timeout_) + { + } + + std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; + +private: + Poco::Logger * log; + std::string name; + size_t keep_alive_timeout; +}; + +} diff --git a/programs/library-bridge/Handlers.cpp b/programs/library-bridge/Handlers.cpp new file mode 100644 index 00000000000..6a1bfbbccb7 --- /dev/null +++ b/programs/library-bridge/Handlers.cpp @@ -0,0 +1,288 @@ +#include "Handlers.h" +#include "SharedLibraryHandlerFactory.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + std::shared_ptr parseColumns(std::string && column_string) + { + auto sample_block = std::make_shared(); + auto names_and_types = NamesAndTypesList::parse(column_string); + + for (const NameAndTypePair & column_data : names_and_types) + sample_block->insert({column_data.type, column_data.name}); + + return sample_block; + } + + std::vector parseIdsFromBinary(const std::string & ids_string) + { + ReadBufferFromString buf(ids_string); + std::vector ids; + readVectorBinary(ids, buf); + return ids; + } + + std::vector parseNamesFromBinary(const std::string & names_string) + { + ReadBufferFromString buf(names_string); + std::vector names; + readVectorBinary(names, buf); + return names; + } +} + + +void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) +{ + LOG_TRACE(log, "Request URI: {}", request.getURI()); + HTMLForm params(request); + + if (!params.has("method")) + { + processError(response, "No 'method' in request URL"); + return; + } + + if (!params.has("dictionary_id")) + { + processError(response, "No 'dictionary_id in request URL"); + return; + } + + std::string method = params.get("method"); + std::string dictionary_id = params.get("dictionary_id"); + LOG_TRACE(log, "Library method: '{}', dictionary id: {}", method, dictionary_id); + + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); + + try + { + if (method == "libNew") + { + auto & read_buf = request.getStream(); + params.read(read_buf); + + if (!params.has("library_path")) + { + processError(response, "No 'library_path' in request URL"); + return; + } + + if (!params.has("library_settings")) + { + processError(response, "No 'library_settings' in request URL"); + return; + } + + std::string library_path = params.get("library_path"); + const auto & settings_string = params.get("library_settings"); + std::vector library_settings = parseNamesFromBinary(settings_string); + + /// Needed for library dictionary + if (!params.has("attributes_names")) + { + processError(response, "No 'attributes_names' in request URL"); + return; + } + + const auto & attributes_string = params.get("attributes_names"); + std::vector attributes_names = parseNamesFromBinary(attributes_string); + + /// Needed to parse block from binary string format + if (!params.has("sample_block")) + { + processError(response, "No 'sample_block' in request URL"); + return; + } + std::string sample_block_string = params.get("sample_block"); + + std::shared_ptr sample_block; + try + { + sample_block = parseColumns(std::move(sample_block_string)); + } + catch (const Exception & ex) + { + processError(response, "Invalid 'sample_block' parameter in request body '" + ex.message() + "'"); + LOG_WARNING(log, ex.getStackTraceString()); + return; + } + + if (!params.has("null_values")) + { + processError(response, "No 'null_values' in request URL"); + return; + } + + ReadBufferFromString read_block_buf(params.get("null_values")); + auto format = FormatFactory::instance().getInput(FORMAT, read_block_buf, *sample_block, getContext(), DEFAULT_BLOCK_SIZE); + auto reader = std::make_shared(format); + auto sample_block_with_nulls = reader->read(); + + LOG_DEBUG(log, "Dictionary sample block with null values: {}", sample_block_with_nulls.dumpStructure()); + + SharedLibraryHandlerFactory::instance().create(dictionary_id, library_path, library_settings, sample_block_with_nulls, attributes_names); + writeStringBinary("1", out); + } + else if (method == "libClone") + { + if (!params.has("from_dictionary_id")) + { + processError(response, "No 'from_dictionary_id' in request URL"); + return; + } + + std::string from_dictionary_id = params.get("from_dictionary_id"); + LOG_TRACE(log, "Calling libClone from {} to {}", from_dictionary_id, dictionary_id); + SharedLibraryHandlerFactory::instance().clone(from_dictionary_id, dictionary_id); + writeStringBinary("1", out); + } + else if (method == "libDelete") + { + SharedLibraryHandlerFactory::instance().remove(dictionary_id); + writeStringBinary("1", out); + } + else if (method == "isModified") + { + auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id); + bool res = library_handler->isModified(); + writeStringBinary(std::to_string(res), out); + } + else if (method == "supportsSelectiveLoad") + { + auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id); + bool res = library_handler->supportsSelectiveLoad(); + writeStringBinary(std::to_string(res), out); + } + else if (method == "loadAll") + { + auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id); + const auto & sample_block = library_handler->getSampleBlock(); + auto input = library_handler->loadAll(); + + BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, getContext()); + copyData(*input, *output); + } + else if (method == "loadIds") + { + params.read(request.getStream()); + + if (!params.has("ids")) + { + processError(response, "No 'ids' in request URL"); + return; + } + + std::vector ids = parseIdsFromBinary(params.get("ids")); + auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id); + const auto & sample_block = library_handler->getSampleBlock(); + auto input = library_handler->loadIds(ids); + BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, getContext()); + copyData(*input, *output); + } + else if (method == "loadKeys") + { + if (!params.has("requested_block_sample")) + { + processError(response, "No 'requested_block_sample' in request URL"); + return; + } + + std::string requested_block_string = params.get("requested_block_sample"); + + std::shared_ptr requested_sample_block; + try + { + requested_sample_block = parseColumns(std::move(requested_block_string)); + } + catch (const Exception & ex) + { + processError(response, "Invalid 'requested_block' parameter in request body '" + ex.message() + "'"); + LOG_WARNING(log, ex.getStackTraceString()); + return; + } + + auto & read_buf = request.getStream(); + auto format = FormatFactory::instance().getInput(FORMAT, read_buf, *requested_sample_block, getContext(), DEFAULT_BLOCK_SIZE); + auto reader = std::make_shared(format); + auto block = reader->read(); + + auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id); + const auto & sample_block = library_handler->getSampleBlock(); + auto input = library_handler->loadKeys(block.getColumns()); + BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, getContext()); + copyData(*input, *output); + } + } + catch (...) + { + auto message = getCurrentExceptionMessage(true); + response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR, message); // can't call process_error, because of too soon response sending + + try + { + writeStringBinary(message, out); + out.finalize(); + } + catch (...) + { + tryLogCurrentException(log); + } + + tryLogCurrentException(log); + } + + try + { + out.finalize(); + } + catch (...) + { + tryLogCurrentException(log); + } +} + + +void LibraryRequestHandler::processError(HTTPServerResponse & response, const std::string & message) +{ + response.setStatusAndReason(HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); + + if (!response.sent()) + *response.send() << message << std::endl; + + LOG_WARNING(log, message); +} + + +void PingHandler::handleRequest(HTTPServerRequest & /* request */, HTTPServerResponse & response) +{ + try + { + setResponseDefaultHeaders(response, keep_alive_timeout); + const char * data = "Ok.\n"; + response.sendBuffer(data, strlen(data)); + } + catch (...) + { + tryLogCurrentException("PingHandler"); + } +} + + +} diff --git a/programs/library-bridge/Handlers.h b/programs/library-bridge/Handlers.h new file mode 100644 index 00000000000..dac61d3a735 --- /dev/null +++ b/programs/library-bridge/Handlers.h @@ -0,0 +1,59 @@ +#pragma once + +#include +#include +#include +#include "SharedLibraryHandler.h" + + +namespace DB +{ + + +/// Handler for requests to Library Dictionary Source, returns response in RowBinary format. +/// When a library dictionary source is created, it sends libNew request to library bridge (which is started on first +/// request to it, if it was not yet started). On this request a new sharedLibrayHandler is added to a +/// sharedLibraryHandlerFactory by a dictionary uuid. With libNew request come: library_path, library_settings, +/// names of dictionary attributes, sample block to parse block of null values, block of null values. Everything is +/// passed in binary format and is urlencoded. When dictionary is cloned, a new handler is created. +/// Each handler is unique to dictionary. +class LibraryRequestHandler : public HTTPRequestHandler, WithContext +{ +public: + + LibraryRequestHandler( + size_t keep_alive_timeout_, + ContextPtr context_) + : WithContext(context_) + , log(&Poco::Logger::get("LibraryRequestHandler")) + , keep_alive_timeout(keep_alive_timeout_) + { + } + + void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override; + +private: + static constexpr inline auto FORMAT = "RowBinary"; + + void processError(HTTPServerResponse & response, const std::string & message); + + Poco::Logger * log; + size_t keep_alive_timeout; +}; + + +class PingHandler : public HTTPRequestHandler +{ +public: + explicit PingHandler(size_t keep_alive_timeout_) + : keep_alive_timeout(keep_alive_timeout_) + { + } + + void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override; + +private: + const size_t keep_alive_timeout; +}; + +} diff --git a/programs/library-bridge/LibraryBridge.cpp b/programs/library-bridge/LibraryBridge.cpp new file mode 100644 index 00000000000..2e5d6041151 --- /dev/null +++ b/programs/library-bridge/LibraryBridge.cpp @@ -0,0 +1,17 @@ +#include "LibraryBridge.h" + +#pragma GCC diagnostic ignored "-Wmissing-declarations" +int mainEntryClickHouseLibraryBridge(int argc, char ** argv) +{ + DB::LibraryBridge app; + try + { + return app.run(argc, argv); + } + catch (...) + { + std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; + auto code = DB::getCurrentExceptionCode(); + return code ? code : 1; + } +} diff --git a/programs/library-bridge/LibraryBridge.h b/programs/library-bridge/LibraryBridge.h new file mode 100644 index 00000000000..9f2dafb89ab --- /dev/null +++ b/programs/library-bridge/LibraryBridge.h @@ -0,0 +1,26 @@ +#pragma once + +#include +#include +#include "HandlerFactory.h" + + +namespace DB +{ + +class LibraryBridge : public IBridge +{ + +protected: + std::string bridgeName() const override + { + return "LibraryBridge"; + } + + HandlerFactoryPtr getHandlerFactoryPtr(ContextPtr context) const override + { + return std::make_shared("LibraryRequestHandlerFactory-factory", keep_alive_timeout, context); + } +}; + +} diff --git a/programs/library-bridge/LibraryUtils.h b/programs/library-bridge/LibraryUtils.h new file mode 100644 index 00000000000..359d1de93e3 --- /dev/null +++ b/programs/library-bridge/LibraryUtils.h @@ -0,0 +1,43 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ + +class CStringsHolder +{ + +public: + using Container = std::vector; + + explicit CStringsHolder(const Container & strings_pass) + { + strings_holder = strings_pass; + strings.size = strings_holder.size(); + + ptr_holder = std::make_unique(strings.size); + strings.data = ptr_holder.get(); + + size_t i = 0; + for (auto & str : strings_holder) + { + strings.data[i] = str.c_str(); + ++i; + } + } + + ClickHouseLibrary::CStrings strings; // will pass pointer to lib + +private: + std::unique_ptr ptr_holder = nullptr; + Container strings_holder; +}; + + +} diff --git a/programs/library-bridge/SharedLibraryHandler.cpp b/programs/library-bridge/SharedLibraryHandler.cpp new file mode 100644 index 00000000000..ab8cf2417c2 --- /dev/null +++ b/programs/library-bridge/SharedLibraryHandler.cpp @@ -0,0 +1,219 @@ +#include "SharedLibraryHandler.h" + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int EXTERNAL_LIBRARY_ERROR; + extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; +} + + +SharedLibraryHandler::SharedLibraryHandler( + const std::string & library_path_, + const std::vector & library_settings, + const Block & sample_block_, + const std::vector & attributes_names_) + : library_path(library_path_) + , sample_block(sample_block_) + , attributes_names(attributes_names_) +{ + library = std::make_shared(library_path, RTLD_LAZY); + settings_holder = std::make_shared(CStringsHolder(library_settings)); + + auto lib_new = library->tryGet(ClickHouseLibrary::LIBRARY_CREATE_NEW_FUNC_NAME); + + if (lib_new) + lib_data = lib_new(&settings_holder->strings, ClickHouseLibrary::log); + else + throw Exception("Method libNew failed", ErrorCodes::EXTERNAL_LIBRARY_ERROR); +} + + +SharedLibraryHandler::SharedLibraryHandler(const SharedLibraryHandler & other) + : library_path{other.library_path} + , sample_block{other.sample_block} + , attributes_names{other.attributes_names} + , library{other.library} + , settings_holder{other.settings_holder} +{ + + auto lib_clone = library->tryGet(ClickHouseLibrary::LIBRARY_CLONE_FUNC_NAME); + + if (lib_clone) + { + lib_data = lib_clone(other.lib_data); + } + else + { + auto lib_new = library->tryGet(ClickHouseLibrary::LIBRARY_CREATE_NEW_FUNC_NAME); + + if (lib_new) + lib_data = lib_new(&settings_holder->strings, ClickHouseLibrary::log); + } +} + + +SharedLibraryHandler::~SharedLibraryHandler() +{ + auto lib_delete = library->tryGet(ClickHouseLibrary::LIBRARY_DELETE_FUNC_NAME); + + if (lib_delete) + lib_delete(lib_data); +} + + +bool SharedLibraryHandler::isModified() +{ + auto func_is_modified = library->tryGet(ClickHouseLibrary::LIBRARY_IS_MODIFIED_FUNC_NAME); + + if (func_is_modified) + return func_is_modified(lib_data, &settings_holder->strings); + + return true; +} + + +bool SharedLibraryHandler::supportsSelectiveLoad() +{ + auto func_supports_selective_load = library->tryGet(ClickHouseLibrary::LIBRARY_SUPPORTS_SELECTIVE_LOAD_FUNC_NAME); + + if (func_supports_selective_load) + return func_supports_selective_load(lib_data, &settings_holder->strings); + + return true; +} + + +BlockInputStreamPtr SharedLibraryHandler::loadAll() +{ + auto columns_holder = std::make_unique(attributes_names.size()); + ClickHouseLibrary::CStrings columns{static_cast(columns_holder.get()), attributes_names.size()}; + for (size_t i = 0; i < attributes_names.size(); ++i) + columns.data[i] = attributes_names[i].c_str(); + + auto load_all_func = library->get(ClickHouseLibrary::LIBRARY_LOAD_ALL_FUNC_NAME); + auto data_new_func = library->get(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME); + auto data_delete_func = library->get(ClickHouseLibrary::LIBRARY_DATA_DELETE_FUNC_NAME); + + ClickHouseLibrary::LibraryData data_ptr = data_new_func(lib_data); + SCOPE_EXIT(data_delete_func(lib_data, data_ptr)); + + ClickHouseLibrary::RawClickHouseLibraryTable data = load_all_func(data_ptr, &settings_holder->strings, &columns); + auto block = dataToBlock(data); + + return std::make_shared(block); +} + + +BlockInputStreamPtr SharedLibraryHandler::loadIds(const std::vector & ids) +{ + const ClickHouseLibrary::VectorUInt64 ids_data{ext::bit_cast(ids.data()), ids.size()}; + + auto columns_holder = std::make_unique(attributes_names.size()); + ClickHouseLibrary::CStrings columns_pass{static_cast(columns_holder.get()), attributes_names.size()}; + + auto load_ids_func = library->get(ClickHouseLibrary::LIBRARY_LOAD_IDS_FUNC_NAME); + auto data_new_func = library->get(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME); + auto data_delete_func = library->get(ClickHouseLibrary::LIBRARY_DATA_DELETE_FUNC_NAME); + + ClickHouseLibrary::LibraryData data_ptr = data_new_func(lib_data); + SCOPE_EXIT(data_delete_func(lib_data, data_ptr)); + + ClickHouseLibrary::RawClickHouseLibraryTable data = load_ids_func(data_ptr, &settings_holder->strings, &columns_pass, &ids_data); + auto block = dataToBlock(data); + + return std::make_shared(block); +} + + +BlockInputStreamPtr SharedLibraryHandler::loadKeys(const Columns & key_columns) +{ + auto holder = std::make_unique(key_columns.size()); + std::vector> column_data_holders; + + for (size_t i = 0; i < key_columns.size(); ++i) + { + auto cell_holder = std::make_unique(key_columns[i]->size()); + + for (size_t j = 0; j < key_columns[i]->size(); ++j) + { + auto data_ref = key_columns[i]->getDataAt(j); + + cell_holder[j] = ClickHouseLibrary::Field{ + .data = static_cast(data_ref.data), + .size = data_ref.size}; + } + + holder[i] = ClickHouseLibrary::Row{ + .data = static_cast(cell_holder.get()), + .size = key_columns[i]->size()}; + + column_data_holders.push_back(std::move(cell_holder)); + } + + ClickHouseLibrary::Table request_cols{ + .data = static_cast(holder.get()), + .size = key_columns.size()}; + + auto load_keys_func = library->get(ClickHouseLibrary::LIBRARY_LOAD_KEYS_FUNC_NAME); + auto data_new_func = library->get(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME); + auto data_delete_func = library->get(ClickHouseLibrary::LIBRARY_DATA_DELETE_FUNC_NAME); + + ClickHouseLibrary::LibraryData data_ptr = data_new_func(lib_data); + SCOPE_EXIT(data_delete_func(lib_data, data_ptr)); + + ClickHouseLibrary::RawClickHouseLibraryTable data = load_keys_func(data_ptr, &settings_holder->strings, &request_cols); + auto block = dataToBlock(data); + + return std::make_shared(block); +} + + +Block SharedLibraryHandler::dataToBlock(const ClickHouseLibrary::RawClickHouseLibraryTable data) +{ + if (!data) + throw Exception("LibraryDictionarySource: No data returned", ErrorCodes::EXTERNAL_LIBRARY_ERROR); + + const auto * columns_received = static_cast(data); + if (columns_received->error_code) + throw Exception( + "LibraryDictionarySource: Returned error: " + std::to_string(columns_received->error_code) + " " + (columns_received->error_string ? columns_received->error_string : ""), + ErrorCodes::EXTERNAL_LIBRARY_ERROR); + + MutableColumns columns = sample_block.cloneEmptyColumns(); + + for (size_t col_n = 0; col_n < columns_received->size; ++col_n) + { + if (columns.size() != columns_received->data[col_n].size) + throw Exception( + "LibraryDictionarySource: Returned unexpected number of columns: " + std::to_string(columns_received->data[col_n].size) + ", must be " + std::to_string(columns.size()), + ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + for (size_t row_n = 0; row_n < columns_received->data[col_n].size; ++row_n) + { + const auto & field = columns_received->data[col_n].data[row_n]; + if (!field.data) + { + /// sample_block contains null_value (from config) inside corresponding column + const auto & col = sample_block.getByPosition(row_n); + columns[row_n]->insertFrom(*(col.column), 0); + } + else + { + const auto & size = field.size; + columns[row_n]->insertData(static_cast(field.data), size); + } + } + } + + return sample_block.cloneWithColumns(std::move(columns)); +} + +} diff --git a/programs/library-bridge/SharedLibraryHandler.h b/programs/library-bridge/SharedLibraryHandler.h new file mode 100644 index 00000000000..5c0334ac89f --- /dev/null +++ b/programs/library-bridge/SharedLibraryHandler.h @@ -0,0 +1,54 @@ +#pragma once + +#include +#include +#include +#include "LibraryUtils.h" + + +namespace DB +{ + +/// A class that manages all operations with library dictionary. +/// Every library dictionary source has its own object of this class, accessed by UUID. +class SharedLibraryHandler +{ + +public: + SharedLibraryHandler( + const std::string & library_path_, + const std::vector & library_settings, + const Block & sample_block_, + const std::vector & attributes_names_); + + SharedLibraryHandler(const SharedLibraryHandler & other); + + ~SharedLibraryHandler(); + + BlockInputStreamPtr loadAll(); + + BlockInputStreamPtr loadIds(const std::vector & ids); + + BlockInputStreamPtr loadKeys(const Columns & key_columns); + + bool isModified(); + + bool supportsSelectiveLoad(); + + const Block & getSampleBlock() { return sample_block; } + +private: + Block dataToBlock(const ClickHouseLibrary::RawClickHouseLibraryTable data); + + std::string library_path; + const Block sample_block; + std::vector attributes_names; + + SharedLibraryPtr library; + std::shared_ptr settings_holder; + void * lib_data; +}; + +using SharedLibraryHandlerPtr = std::shared_ptr; + +} diff --git a/programs/library-bridge/SharedLibraryHandlerFactory.cpp b/programs/library-bridge/SharedLibraryHandlerFactory.cpp new file mode 100644 index 00000000000..05494c313c4 --- /dev/null +++ b/programs/library-bridge/SharedLibraryHandlerFactory.cpp @@ -0,0 +1,67 @@ +#include "SharedLibraryHandlerFactory.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +SharedLibraryHandlerPtr SharedLibraryHandlerFactory::get(const std::string & dictionary_id) +{ + std::lock_guard lock(mutex); + auto library_handler = library_handlers.find(dictionary_id); + + if (library_handler != library_handlers.end()) + return library_handler->second; + + return nullptr; +} + + +void SharedLibraryHandlerFactory::create( + const std::string & dictionary_id, + const std::string & library_path, + const std::vector & library_settings, + const Block & sample_block, + const std::vector & attributes_names) +{ + std::lock_guard lock(mutex); + library_handlers[dictionary_id] = std::make_shared(library_path, library_settings, sample_block, attributes_names); +} + + +void SharedLibraryHandlerFactory::clone(const std::string & from_dictionary_id, const std::string & to_dictionary_id) +{ + std::lock_guard lock(mutex); + auto from_library_handler = library_handlers.find(from_dictionary_id); + + /// This is not supposed to happen as libClone is called from copy constructor of LibraryDictionarySource + /// object, and shared library handler of from_dictionary is removed only in its destructor. + /// And if for from_dictionary there was no shared library handler, it would have received and exception in + /// its constructor, so no libClone would be made from it. + if (from_library_handler == library_handlers.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No shared library handler found"); + + /// libClone method will be called in copy constructor + library_handlers[to_dictionary_id] = std::make_shared(*from_library_handler->second); +} + + +void SharedLibraryHandlerFactory::remove(const std::string & dictionary_id) +{ + std::lock_guard lock(mutex); + /// libDelete is called in destructor. + library_handlers.erase(dictionary_id); +} + + +SharedLibraryHandlerFactory & SharedLibraryHandlerFactory::instance() +{ + static SharedLibraryHandlerFactory ret; + return ret; +} + +} diff --git a/programs/library-bridge/SharedLibraryHandlerFactory.h b/programs/library-bridge/SharedLibraryHandlerFactory.h new file mode 100644 index 00000000000..473d90618a2 --- /dev/null +++ b/programs/library-bridge/SharedLibraryHandlerFactory.h @@ -0,0 +1,37 @@ +#pragma once + +#include "SharedLibraryHandler.h" +#include +#include + + +namespace DB +{ + +/// Each library dictionary source has unique UUID. When clone() method is called, a new UUID is generated. +/// There is a unique mapping from diciotnary UUID to sharedLibraryHandler. +class SharedLibraryHandlerFactory final : private boost::noncopyable +{ +public: + static SharedLibraryHandlerFactory & instance(); + + SharedLibraryHandlerPtr get(const std::string & dictionary_id); + + void create( + const std::string & dictionary_id, + const std::string & library_path, + const std::vector & library_settings, + const Block & sample_block, + const std::vector & attributes_names); + + void clone(const std::string & from_dictionary_id, const std::string & to_dictionary_id); + + void remove(const std::string & dictionary_id); + +private: + /// map: dict_id -> sharedLibraryHandler + std::unordered_map library_handlers; + std::mutex mutex; +}; + +} diff --git a/programs/library-bridge/library-bridge.cpp b/programs/library-bridge/library-bridge.cpp new file mode 100644 index 00000000000..5fff2ffe525 --- /dev/null +++ b/programs/library-bridge/library-bridge.cpp @@ -0,0 +1,3 @@ +int mainEntryClickHouseLibraryBridge(int argc, char ** argv); +int main(int argc_, char ** argv_) { return mainEntryClickHouseLibraryBridge(argc_, argv_); } + diff --git a/programs/library-bridge/library-log.cpp b/programs/library-bridge/library-log.cpp new file mode 100644 index 00000000000..89fb31623b3 --- /dev/null +++ b/programs/library-bridge/library-log.cpp @@ -0,0 +1,66 @@ +#include +#include + +namespace +{ +const char DICT_LOGGER_NAME[] = "LibraryDictionarySourceExternal"; +} + +namespace ClickHouseLibrary +{ + +std::string_view LIBRARY_CREATE_NEW_FUNC_NAME = "ClickHouseDictionary_v3_libNew"; +std::string_view LIBRARY_CLONE_FUNC_NAME = "ClickHouseDictionary_v3_libClone"; +std::string_view LIBRARY_DELETE_FUNC_NAME = "ClickHouseDictionary_v3_libDelete"; + +std::string_view LIBRARY_DATA_NEW_FUNC_NAME = "ClickHouseDictionary_v3_dataNew"; +std::string_view LIBRARY_DATA_DELETE_FUNC_NAME = "ClickHouseDictionary_v3_dataDelete"; + +std::string_view LIBRARY_LOAD_ALL_FUNC_NAME = "ClickHouseDictionary_v3_loadAll"; +std::string_view LIBRARY_LOAD_IDS_FUNC_NAME = "ClickHouseDictionary_v3_loadIds"; +std::string_view LIBRARY_LOAD_KEYS_FUNC_NAME = "ClickHouseDictionary_v3_loadKeys"; + +std::string_view LIBRARY_IS_MODIFIED_FUNC_NAME = "ClickHouseDictionary_v3_isModified"; +std::string_view LIBRARY_SUPPORTS_SELECTIVE_LOAD_FUNC_NAME = "ClickHouseDictionary_v3_supportsSelectiveLoad"; + +void log(LogLevel level, CString msg) +{ + auto & logger = Poco::Logger::get(DICT_LOGGER_NAME); + switch (level) + { + case LogLevel::TRACE: + if (logger.trace()) + logger.trace(msg); + break; + case LogLevel::DEBUG: + if (logger.debug()) + logger.debug(msg); + break; + case LogLevel::INFORMATION: + if (logger.information()) + logger.information(msg); + break; + case LogLevel::NOTICE: + if (logger.notice()) + logger.notice(msg); + break; + case LogLevel::WARNING: + if (logger.warning()) + logger.warning(msg); + break; + case LogLevel::ERROR: + if (logger.error()) + logger.error(msg); + break; + case LogLevel::CRITICAL: + if (logger.critical()) + logger.critical(msg); + break; + case LogLevel::FATAL: + if (logger.fatal()) + logger.fatal(msg); + break; + } +} + +} diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 2909b838c84..f680c2c2da6 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -99,9 +99,9 @@ void LocalServer::initialize(Poco::Util::Application & self) } } -void LocalServer::applyCmdSettings(Context & context) +void LocalServer::applyCmdSettings(ContextPtr context) { - context.applySettingsChanges(cmd_settings.changes()); + context->applySettingsChanges(cmd_settings.changes()); } /// If path is specified and not empty, will try to setup server environment and load existing metadata @@ -176,7 +176,7 @@ void LocalServer::tryInitPath() } -static void attachSystemTables(const Context & context) +static void attachSystemTables(ContextPtr context) { DatabasePtr system_database = DatabaseCatalog::instance().tryGetDatabase(DatabaseCatalog::SYSTEM_DATABASE); if (!system_database) @@ -211,7 +211,7 @@ try } shared_context = Context::createShared(); - global_context = std::make_unique(Context::createGlobal(shared_context.get())); + global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); global_context->setApplicationType(Context::ApplicationType::LOCAL); tryInitPath(); @@ -274,9 +274,9 @@ try * if such tables will not be dropped, clickhouse-server will not be able to load them due to security reasons. */ std::string default_database = config().getString("default_database", "_local"); - DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, *global_context)); + DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, global_context)); global_context->setCurrentDatabase(default_database); - applyCmdOptions(*global_context); + applyCmdOptions(global_context); if (config().has("path")) { @@ -288,15 +288,15 @@ try LOG_DEBUG(log, "Loading metadata from {}", path); Poco::File(path + "data/").createDirectories(); Poco::File(path + "metadata/").createDirectories(); - loadMetadataSystem(*global_context); - attachSystemTables(*global_context); - loadMetadata(*global_context); + loadMetadataSystem(global_context); + attachSystemTables(global_context); + loadMetadata(global_context); DatabaseCatalog::instance().loadDatabases(); LOG_DEBUG(log, "Loaded metadata."); } else if (!config().has("no-system-tables")) { - attachSystemTables(*global_context); + attachSystemTables(global_context); } processQueries(); @@ -375,13 +375,13 @@ void LocalServer::processQueries() /// we can't mutate global global_context (can lead to races, as it was already passed to some background threads) /// so we can't reuse it safely as a query context and need a copy here - auto context = Context(*global_context); + auto context = Context::createCopy(global_context); - context.makeSessionContext(); - context.makeQueryContext(); + context->makeSessionContext(); + context->makeQueryContext(); - context.setUser("default", "", Poco::Net::SocketAddress{}); - context.setCurrentQueryId(""); + context->setUser("default", "", Poco::Net::SocketAddress{}); + context->setCurrentQueryId(""); applyCmdSettings(context); /// Use the same query_id (and thread group) for all queries @@ -618,9 +618,9 @@ void LocalServer::init(int argc, char ** argv) argsToConfig(arguments, config(), 100); } -void LocalServer::applyCmdOptions(Context & context) +void LocalServer::applyCmdOptions(ContextPtr context) { - context.setDefaultFormat(config().getString("output-format", config().getString("format", "TSV"))); + context->setDefaultFormat(config().getString("output-format", config().getString("format", "TSV"))); applyCmdSettings(context); } diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index 02778bd86cb..3555e8a38ad 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -36,15 +36,15 @@ private: std::string getInitialCreateTableQuery(); void tryInitPath(); - void applyCmdOptions(Context & context); - void applyCmdSettings(Context & context); + void applyCmdOptions(ContextPtr context); + void applyCmdSettings(ContextPtr context); void processQueries(); void setupUsers(); void cleanup(); protected: SharedContextHolder shared_context; - std::unique_ptr global_context; + ContextPtr global_context; /// Settings specified via command line args Settings cmd_settings; diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index aea70ba0986..c92eb5c6647 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1129,8 +1129,8 @@ try } SharedContextHolder shared_context = Context::createShared(); - Context context = Context::createGlobal(shared_context.get()); - context.makeGlobalContext(); + ContextPtr context = Context::createGlobal(shared_context.get()); + context->makeGlobalContext(); ReadBufferFromFileDescriptor file_in(STDIN_FILENO); WriteBufferFromFileDescriptor file_out(STDOUT_FILENO); @@ -1152,7 +1152,7 @@ try if (!silent) std::cerr << "Training models\n"; - BlockInputStreamPtr input = context.getInputFormat(input_format, file_in, header, max_block_size); + BlockInputStreamPtr input = context->getInputFormat(input_format, file_in, header, max_block_size); input->readPrefix(); while (Block block = input->read()) @@ -1179,8 +1179,8 @@ try file_in.seek(0, SEEK_SET); - BlockInputStreamPtr input = context.getInputFormat(input_format, file_in, header, max_block_size); - BlockOutputStreamPtr output = context.getOutputStreamParallelIfPossible(output_format, file_out, header); + BlockInputStreamPtr input = context->getInputFormat(input_format, file_in, header, max_block_size); + BlockOutputStreamPtr output = context->getOutputStreamParallelIfPossible(output_format, file_out, header); if (processed_rows + source_rows > limit) input = std::make_shared(input, limit - processed_rows, 0); diff --git a/programs/odbc-bridge/CMakeLists.txt b/programs/odbc-bridge/CMakeLists.txt index 11864354619..7b232f2b5dc 100644 --- a/programs/odbc-bridge/CMakeLists.txt +++ b/programs/odbc-bridge/CMakeLists.txt @@ -24,12 +24,14 @@ add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) target_link_libraries(clickhouse-odbc-bridge PRIVATE daemon dbms + bridge clickhouse_parsers - Poco::Data - Poco::Data::ODBC + nanodbc + unixodbc ) set_target_properties(clickhouse-odbc-bridge PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..) +target_compile_options (clickhouse-odbc-bridge PRIVATE -Wno-reserved-id-macro -Wno-keyword-macro) if (USE_GDB_ADD_INDEX) add_custom_command(TARGET clickhouse-odbc-bridge POST_BUILD COMMAND ${GDB_ADD_INDEX_EXE} ../clickhouse-odbc-bridge COMMENT "Adding .gdb-index to clickhouse-odbc-bridge" VERBATIM) diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index 14fa734f246..e33858583c2 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -2,29 +2,36 @@ #if USE_ODBC -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include "getIdentifierQuote.h" -# include "validateODBCConnectionString.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "getIdentifierQuote.h" +#include "validateODBCConnectionString.h" +#include "ODBCConnectionFactory.h" + +#include +#include -# define POCO_SQL_ODBC_CLASS Poco::Data::ODBC namespace DB { + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; +} + namespace { DataTypePtr getDataType(SQLSMALLINT type) @@ -59,6 +66,7 @@ namespace } } + void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) { HTMLForm params(request, request.getStream()); @@ -77,88 +85,79 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ process_error("No 'table' param in request URL"); return; } + if (!params.has("connection_string")) { process_error("No 'connection_string' in request URL"); return; } + std::string schema_name; std::string table_name = params.get("table"); std::string connection_string = params.get("connection_string"); if (params.has("schema")) - { schema_name = params.get("schema"); - LOG_TRACE(log, "Will fetch info for table '{}'", schema_name + "." + table_name); - } - else - LOG_TRACE(log, "Will fetch info for table '{}'", table_name); + LOG_TRACE(log, "Got connection str '{}'", connection_string); try { const bool external_table_functions_use_nulls = Poco::NumberParser::parseBool(params.get("external_table_functions_use_nulls", "false")); - POCO_SQL_ODBC_CLASS::SessionImpl session(validateODBCConnectionString(connection_string), DBMS_DEFAULT_CONNECT_TIMEOUT_SEC); - SQLHDBC hdbc = session.dbc().handle(); + auto connection = ODBCConnectionFactory::instance().get( + validateODBCConnectionString(connection_string), + getContext()->getSettingsRef().odbc_bridge_connection_pool_size); - SQLHSTMT hstmt = nullptr; + nanodbc::catalog catalog(*connection); + std::string catalog_name; - if (POCO_SQL_ODBC_CLASS::Utility::isError(SQLAllocStmt(hdbc, &hstmt))) - throw POCO_SQL_ODBC_CLASS::ODBCException("Could not allocate connection handle."); - - SCOPE_EXIT(SQLFreeStmt(hstmt, SQL_DROP)); - - const auto & context_settings = context.getSettingsRef(); - - /// TODO Why not do SQLColumns instead? - std::string name = schema_name.empty() ? backQuoteIfNeed(table_name) : backQuoteIfNeed(schema_name) + "." + backQuoteIfNeed(table_name); - WriteBufferFromOwnString buf; - std::string input = "SELECT * FROM " + name + " WHERE 1 = 0"; - ParserQueryWithOutput parser(input.data() + input.size()); - ASTPtr select = parseQuery(parser, input.data(), input.data() + input.size(), "", context_settings.max_query_size, context_settings.max_parser_depth); - - IAST::FormatSettings settings(buf, true); - settings.always_quote_identifiers = true; - settings.identifier_quoting_style = getQuotingStyle(hdbc); - select->format(settings); - std::string query = buf.str(); - - LOG_TRACE(log, "Inferring structure with query '{}'", query); - - if (POCO_SQL_ODBC_CLASS::Utility::isError(POCO_SQL_ODBC_CLASS::SQLPrepare(hstmt, reinterpret_cast(query.data()), query.size()))) - throw POCO_SQL_ODBC_CLASS::DescriptorException(session.dbc()); - - if (POCO_SQL_ODBC_CLASS::Utility::isError(SQLExecute(hstmt))) - throw POCO_SQL_ODBC_CLASS::StatementException(hstmt); - - SQLSMALLINT cols = 0; - if (POCO_SQL_ODBC_CLASS::Utility::isError(SQLNumResultCols(hstmt, &cols))) - throw POCO_SQL_ODBC_CLASS::StatementException(hstmt); - - /// TODO cols not checked - - NamesAndTypesList columns; - for (SQLSMALLINT ncol = 1; ncol <= cols; ++ncol) + /// In XDBC tables it is allowed to pass either database_name or schema_name in table definion, but not both of them. + /// They both are passed as 'schema' parameter in request URL, so it is not clear whether it is database_name or schema_name passed. + /// If it is schema_name then we know that database is added in odbc.ini. But if we have database_name as 'schema', + /// it is not guaranteed. For nanodbc database_name must be either in odbc.ini or passed as catalog_name. + auto get_columns = [&]() { - SQLSMALLINT type = 0; - /// TODO Why 301? - SQLCHAR column_name[301]; - - SQLSMALLINT is_nullable; - const auto result = POCO_SQL_ODBC_CLASS::SQLDescribeCol(hstmt, ncol, column_name, sizeof(column_name), nullptr, &type, nullptr, nullptr, &is_nullable); - if (POCO_SQL_ODBC_CLASS::Utility::isError(result)) - throw POCO_SQL_ODBC_CLASS::StatementException(hstmt); - - auto column_type = getDataType(type); - if (external_table_functions_use_nulls && is_nullable == SQL_NULLABLE) + nanodbc::catalog::tables tables = catalog.find_tables(table_name, /* type = */ "", /* schema = */ "", /* catalog = */ schema_name); + if (tables.next()) { - column_type = std::make_shared(column_type); + catalog_name = tables.table_catalog(); + LOG_TRACE(log, "Will fetch info for table '{}.{}'", catalog_name, table_name); + return catalog.find_columns(/* column = */ "", table_name, /* schema = */ "", catalog_name); } - columns.emplace_back(reinterpret_cast(column_name), std::move(column_type)); + tables = catalog.find_tables(table_name, /* type = */ "", /* schema = */ schema_name); + if (tables.next()) + { + catalog_name = tables.table_catalog(); + LOG_TRACE(log, "Will fetch info for table '{}.{}.{}'", catalog_name, schema_name, table_name); + return catalog.find_columns(/* column = */ "", table_name, schema_name, catalog_name); + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table {} not found", schema_name.empty() ? table_name : schema_name + '.' + table_name); + }; + + nanodbc::catalog::columns columns_definition = get_columns(); + + NamesAndTypesList columns; + while (columns_definition.next()) + { + SQLSMALLINT type = columns_definition.sql_data_type(); + std::string column_name = columns_definition.column_name(); + + bool is_nullable = columns_definition.nullable() == SQL_NULLABLE; + + auto column_type = getDataType(type); + + if (external_table_functions_use_nulls && is_nullable == SQL_NULLABLE) + column_type = std::make_shared(column_type); + + columns.emplace_back(column_name, std::move(column_type)); } + if (columns.empty()) + throw Exception("Columns definition was not returned", ErrorCodes::LOGICAL_ERROR); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); try { diff --git a/programs/odbc-bridge/ColumnInfoHandler.h b/programs/odbc-bridge/ColumnInfoHandler.h index 9b5b470b31d..bc976f54aee 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.h +++ b/programs/odbc-bridge/ColumnInfoHandler.h @@ -2,24 +2,23 @@ #if USE_ODBC -# include -# include -# include +#include +#include +#include +#include +#include -# include -/** The structure of the table is taken from the query "SELECT * FROM table WHERE 1=0". - * TODO: It would be much better to utilize ODBC methods dedicated for columns description. - * If there is no such table, an exception is thrown. - */ namespace DB { -class ODBCColumnsInfoHandler : public HTTPRequestHandler +class ODBCColumnsInfoHandler : public HTTPRequestHandler, WithContext { public: - ODBCColumnsInfoHandler(size_t keep_alive_timeout_, Context & context_) - : log(&Poco::Logger::get("ODBCColumnsInfoHandler")), keep_alive_timeout(keep_alive_timeout_), context(context_) + ODBCColumnsInfoHandler(size_t keep_alive_timeout_, ContextPtr context_) + : WithContext(context_) + , log(&Poco::Logger::get("ODBCColumnsInfoHandler")) + , keep_alive_timeout(keep_alive_timeout_) { } @@ -28,7 +27,6 @@ public: private: Poco::Logger * log; size_t keep_alive_timeout; - Context & context; }; } diff --git a/programs/odbc-bridge/HandlerFactory.cpp b/programs/odbc-bridge/HandlerFactory.cpp index 9ac48af4ace..49984453d33 100644 --- a/programs/odbc-bridge/HandlerFactory.cpp +++ b/programs/odbc-bridge/HandlerFactory.cpp @@ -8,7 +8,7 @@ namespace DB { -std::unique_ptr HandlerFactory::createRequestHandler(const HTTPServerRequest & request) +std::unique_ptr ODBCBridgeHandlerFactory::createRequestHandler(const HTTPServerRequest & request) { Poco::URI uri{request.getURI()}; LOG_TRACE(log, "Request URI: {}", uri.toString()); @@ -21,26 +21,26 @@ std::unique_ptr HandlerFactory::createRequestHandler(const H if (uri.getPath() == "/columns_info") #if USE_ODBC - return std::make_unique(keep_alive_timeout, context); + return std::make_unique(keep_alive_timeout, getContext()); #else return nullptr; #endif else if (uri.getPath() == "/identifier_quote") #if USE_ODBC - return std::make_unique(keep_alive_timeout, context); + return std::make_unique(keep_alive_timeout, getContext()); #else return nullptr; #endif else if (uri.getPath() == "/schema_allowed") #if USE_ODBC - return std::make_unique(keep_alive_timeout, context); + return std::make_unique(keep_alive_timeout, getContext()); #else return nullptr; #endif else if (uri.getPath() == "/write") - return std::make_unique(pool_map, keep_alive_timeout, context, "write"); + return std::make_unique(keep_alive_timeout, getContext(), "write"); else - return std::make_unique(pool_map, keep_alive_timeout, context, "read"); + return std::make_unique(keep_alive_timeout, getContext(), "read"); } return nullptr; } diff --git a/programs/odbc-bridge/HandlerFactory.h b/programs/odbc-bridge/HandlerFactory.h index 5dce6f02ecd..ffbbe3670af 100644 --- a/programs/odbc-bridge/HandlerFactory.h +++ b/programs/odbc-bridge/HandlerFactory.h @@ -1,32 +1,28 @@ #pragma once -#include +#include #include #include "ColumnInfoHandler.h" #include "IdentifierQuoteHandler.h" #include "MainHandler.h" #include "SchemaAllowedHandler.h" - #include -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" -#include -#pragma GCC diagnostic pop - namespace DB { /** Factory for '/ping', '/', '/columns_info', '/identifier_quote', '/schema_allowed' handlers. * Also stores Session pools for ODBC connections */ -class HandlerFactory : public HTTPRequestHandlerFactory +class ODBCBridgeHandlerFactory : public HTTPRequestHandlerFactory, WithContext { public: - HandlerFactory(const std::string & name_, size_t keep_alive_timeout_, Context & context_) - : log(&Poco::Logger::get(name_)), name(name_), keep_alive_timeout(keep_alive_timeout_), context(context_) + ODBCBridgeHandlerFactory(const std::string & name_, size_t keep_alive_timeout_, ContextPtr context_) + : WithContext(context_) + , log(&Poco::Logger::get(name_)) + , name(name_) + , keep_alive_timeout(keep_alive_timeout_) { - pool_map = std::make_shared(); } std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; @@ -35,7 +31,6 @@ private: Poco::Logger * log; std::string name; size_t keep_alive_timeout; - Context & context; - std::shared_ptr pool_map; }; + } diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.cpp b/programs/odbc-bridge/IdentifierQuoteHandler.cpp index 5060d37c479..a5a97cb8086 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.cpp +++ b/programs/odbc-bridge/IdentifierQuoteHandler.cpp @@ -2,23 +2,20 @@ #if USE_ODBC -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include "getIdentifierQuote.h" -# include "validateODBCConnectionString.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "getIdentifierQuote.h" +#include "validateODBCConnectionString.h" +#include "ODBCConnectionFactory.h" -# define POCO_SQL_ODBC_CLASS Poco::Data::ODBC namespace DB { @@ -44,10 +41,12 @@ void IdentifierQuoteHandler::handleRequest(HTTPServerRequest & request, HTTPServ try { std::string connection_string = params.get("connection_string"); - POCO_SQL_ODBC_CLASS::SessionImpl session(validateODBCConnectionString(connection_string), DBMS_DEFAULT_CONNECT_TIMEOUT_SEC); - SQLHDBC hdbc = session.dbc().handle(); - auto identifier = getIdentifierQuote(hdbc); + auto connection = ODBCConnectionFactory::instance().get( + validateODBCConnectionString(connection_string), + getContext()->getSettingsRef().odbc_bridge_connection_pool_size); + + auto identifier = getIdentifierQuote(*connection); WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); try diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.h b/programs/odbc-bridge/IdentifierQuoteHandler.h index dad88c72ad8..ef3806fd802 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.h +++ b/programs/odbc-bridge/IdentifierQuoteHandler.h @@ -11,11 +11,13 @@ namespace DB { -class IdentifierQuoteHandler : public HTTPRequestHandler +class IdentifierQuoteHandler : public HTTPRequestHandler, WithContext { public: - IdentifierQuoteHandler(size_t keep_alive_timeout_, Context &) - : log(&Poco::Logger::get("IdentifierQuoteHandler")), keep_alive_timeout(keep_alive_timeout_) + IdentifierQuoteHandler(size_t keep_alive_timeout_, ContextPtr context_) + : WithContext(context_) + , log(&Poco::Logger::get("IdentifierQuoteHandler")) + , keep_alive_timeout(keep_alive_timeout_) { } diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index 079fc371ab4..24bcaf63c69 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -18,18 +18,17 @@ #include #include #include +#include "ODBCConnectionFactory.h" #include #include +#include -#if USE_ODBC -#include -#define POCO_SQL_ODBC_CLASS Poco::Data::ODBC -#endif namespace DB { + namespace { std::unique_ptr parseColumns(std::string && column_string) @@ -42,37 +41,6 @@ namespace } } -using PocoSessionPoolConstructor = std::function()>; -/** Is used to adjust max size of default Poco thread pool. See issue #750 - * Acquire the lock, resize pool and construct new Session. - */ -static std::shared_ptr createAndCheckResizePocoSessionPool(PocoSessionPoolConstructor pool_constr) -{ - static std::mutex mutex; - - Poco::ThreadPool & pool = Poco::ThreadPool::defaultPool(); - - /// NOTE: The lock don't guarantee that external users of the pool don't change its capacity - std::unique_lock lock(mutex); - - if (pool.available() == 0) - pool.addCapacity(2 * std::max(pool.capacity(), 1)); - - return pool_constr(); -} - -ODBCHandler::PoolPtr ODBCHandler::getPool(const std::string & connection_str) -{ - std::lock_guard lock(mutex); - if (!pool_map->count(connection_str)) - { - pool_map->emplace(connection_str, createAndCheckResizePocoSessionPool([connection_str] - { - return std::make_shared("ODBC", validateODBCConnectionString(connection_str)); - })); - } - return pool_map->at(connection_str); -} void ODBCHandler::processError(HTTPServerResponse & response, const std::string & message) { @@ -82,6 +50,7 @@ void ODBCHandler::processError(HTTPServerResponse & response, const std::string LOG_WARNING(log, message); } + void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) { HTMLForm params(request); @@ -141,6 +110,10 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse try { + auto connection = ODBCConnectionFactory::instance().get( + validateODBCConnectionString(connection_string), + getContext()->getSettingsRef().odbc_bridge_connection_pool_size); + if (mode == "write") { if (!params.has("db_name")) @@ -159,15 +132,12 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse auto quoting_style = IdentifierQuotingStyle::None; #if USE_ODBC - POCO_SQL_ODBC_CLASS::SessionImpl session(validateODBCConnectionString(connection_string), DBMS_DEFAULT_CONNECT_TIMEOUT_SEC); - quoting_style = getQuotingStyle(session.dbc().handle()); + quoting_style = getQuotingStyle(*connection); #endif - - auto pool = getPool(connection_string); auto & read_buf = request.getStream(); - auto input_format = FormatFactory::instance().getInput(format, read_buf, *sample_block, context, max_block_size); + auto input_format = FormatFactory::instance().getInput(format, read_buf, *sample_block, getContext(), max_block_size); auto input_stream = std::make_shared(input_format); - ODBCBlockOutputStream output_stream(pool->get(), db_name, table_name, *sample_block, quoting_style); + ODBCBlockOutputStream output_stream(*connection, db_name, table_name, *sample_block, getContext(), quoting_style); copyData(*input_stream, output_stream); writeStringBinary("Ok.", out); } @@ -176,9 +146,8 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse std::string query = params.get("query"); LOG_TRACE(log, "Query: {}", query); - BlockOutputStreamPtr writer = FormatFactory::instance().getOutputStreamParallelIfPossible(format, out, *sample_block, context); - auto pool = getPool(connection_string); - ODBCBlockInputStream inp(pool->get(), query, *sample_block, max_block_size); + BlockOutputStreamPtr writer = FormatFactory::instance().getOutputStreamParallelIfPossible(format, out, *sample_block, getContext()); + ODBCBlockInputStream inp(*connection, query, *sample_block, max_block_size); copyData(inp, *writer); } } diff --git a/programs/odbc-bridge/MainHandler.h b/programs/odbc-bridge/MainHandler.h index e237ede5814..bc0fca8b9a5 100644 --- a/programs/odbc-bridge/MainHandler.h +++ b/programs/odbc-bridge/MainHandler.h @@ -1,14 +1,13 @@ #pragma once -#include +#include #include - #include -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" -#include -#pragma GCC diagnostic pop + +#include +#include + namespace DB { @@ -17,20 +16,16 @@ namespace DB * and also query in request body * response in RowBinary format */ -class ODBCHandler : public HTTPRequestHandler +class ODBCHandler : public HTTPRequestHandler, WithContext { public: - using PoolPtr = std::shared_ptr; - using PoolMap = std::unordered_map; - - ODBCHandler(std::shared_ptr pool_map_, + ODBCHandler( size_t keep_alive_timeout_, - Context & context_, + ContextPtr context_, const String & mode_) - : log(&Poco::Logger::get("ODBCHandler")) - , pool_map(pool_map_) + : WithContext(context_) + , log(&Poco::Logger::get("ODBCHandler")) , keep_alive_timeout(keep_alive_timeout_) - , context(context_) , mode(mode_) { } @@ -40,14 +35,11 @@ public: private: Poco::Logger * log; - std::shared_ptr pool_map; size_t keep_alive_timeout; - Context & context; String mode; static inline std::mutex mutex; - PoolPtr getPool(const std::string & connection_str); void processError(HTTPServerResponse & response, const std::string & message); }; diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index b8a4209ac94..3a73cb9f601 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -1,5 +1,7 @@ #include "ODBCBlockInputStream.h" #include +#include +#include #include #include #include @@ -14,137 +16,143 @@ namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; extern const int UNKNOWN_TYPE; } ODBCBlockInputStream::ODBCBlockInputStream( - Poco::Data::Session && session_, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_) - : session{session_} - , statement{(this->session << query_str, Poco::Data::Keywords::now)} - , result{statement} - , iterator{result.begin()} + nanodbc::connection & connection_, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_) + : log(&Poco::Logger::get("ODBCBlockInputStream")) , max_block_size{max_block_size_} - , log(&Poco::Logger::get("ODBCBlockInputStream")) + , connection(connection_) + , query(query_str) { - if (sample_block.columns() != result.columnCount()) - throw Exception{"RecordSet contains " + toString(result.columnCount()) + " columns while " + toString(sample_block.columns()) - + " expected", - ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH}; - description.init(sample_block); -} - - -namespace -{ - using ValueType = ExternalResultDescription::ValueType; - - void insertValue(IColumn & column, const ValueType type, const Poco::Dynamic::Var & value) - { - switch (type) - { - case ValueType::vtUInt8: - assert_cast(column).insertValue(value.convert()); - break; - case ValueType::vtUInt16: - assert_cast(column).insertValue(value.convert()); - break; - case ValueType::vtUInt32: - assert_cast(column).insertValue(value.convert()); - break; - case ValueType::vtUInt64: - assert_cast(column).insertValue(value.convert()); - break; - case ValueType::vtInt8: - assert_cast(column).insertValue(value.convert()); - break; - case ValueType::vtInt16: - assert_cast(column).insertValue(value.convert()); - break; - case ValueType::vtInt32: - assert_cast(column).insertValue(value.convert()); - break; - case ValueType::vtInt64: - assert_cast(column).insertValue(value.convert()); - break; - case ValueType::vtFloat32: - assert_cast(column).insertValue(value.convert()); - break; - case ValueType::vtFloat64: - assert_cast(column).insertValue(value.convert()); - break; - case ValueType::vtString: - assert_cast(column).insert(value.convert()); - break; - case ValueType::vtDate: - { - Poco::DateTime date = value.convert(); - assert_cast(column).insertValue(UInt16{LocalDate(date.year(), date.month(), date.day()).getDayNum()}); - break; - } - case ValueType::vtDateTime: - { - Poco::DateTime datetime = value.convert(); - assert_cast(column).insertValue(DateLUT::instance().makeDateTime( - datetime.year(), datetime.month(), datetime.day(), datetime.hour(), datetime.minute(), datetime.second())); - break; - } - case ValueType::vtUUID: - assert_cast(column).insert(parse(value.convert())); - break; - default: - throw Exception("Unsupported value type", ErrorCodes::UNKNOWN_TYPE); - } - } - - void insertDefaultValue(IColumn & column, const IColumn & sample_column) { column.insertFrom(sample_column, 0); } + result = execute(connection, NANODBC_TEXT(query)); } Block ODBCBlockInputStream::readImpl() { - if (iterator == result.end()) - return {}; - - MutableColumns columns(description.sample_block.columns()); - for (const auto i : ext::range(0, columns.size())) - columns[i] = description.sample_block.getByPosition(i).column->cloneEmpty(); + if (finished) + return Block(); + MutableColumns columns(description.sample_block.cloneEmptyColumns()); size_t num_rows = 0; - while (iterator != result.end()) + + while (true) { - Poco::Data::Row & row = *iterator; - - for (const auto idx : ext::range(0, row.fieldCount())) + if (!result.next()) { - /// TODO This is extremely slow. - const Poco::Dynamic::Var & value = row[idx]; + finished = true; + break; + } - if (!value.isEmpty()) + for (int idx = 0; idx < result.columns(); ++idx) + { + const auto & sample = description.sample_block.getByPosition(idx); + + if (!result.is_null(idx)) { - if (description.types[idx].second) + bool is_nullable = description.types[idx].second; + + if (is_nullable) { ColumnNullable & column_nullable = assert_cast(*columns[idx]); - insertValue(column_nullable.getNestedColumn(), description.types[idx].first, value); + const auto & data_type = assert_cast(*sample.type); + insertValue(column_nullable.getNestedColumn(), data_type.getNestedType(), description.types[idx].first, result, idx); column_nullable.getNullMapData().emplace_back(0); } else - insertValue(*columns[idx], description.types[idx].first, value); + { + insertValue(*columns[idx], sample.type, description.types[idx].first, result, idx); + } } else - insertDefaultValue(*columns[idx], *description.sample_block.getByPosition(idx).column); + insertDefaultValue(*columns[idx], *sample.column); } - ++iterator; - - ++num_rows; - if (num_rows == max_block_size) + if (++num_rows == max_block_size) break; } return description.sample_block.cloneWithColumns(std::move(columns)); } + +void ODBCBlockInputStream::insertValue( + IColumn & column, const DataTypePtr data_type, const ValueType type, nanodbc::result & row, size_t idx) +{ + switch (type) + { + case ValueType::vtUInt8: + assert_cast(column).insertValue(row.get(idx)); + break; + case ValueType::vtUInt16: + assert_cast(column).insertValue(row.get(idx)); + break; + case ValueType::vtUInt32: + assert_cast(column).insertValue(row.get(idx)); + break; + case ValueType::vtUInt64: + assert_cast(column).insertValue(row.get(idx)); + break; + case ValueType::vtInt8: + assert_cast(column).insertValue(row.get(idx)); + break; + case ValueType::vtInt16: + assert_cast(column).insertValue(row.get(idx)); + break; + case ValueType::vtInt32: + assert_cast(column).insertValue(row.get(idx)); + break; + case ValueType::vtInt64: + assert_cast(column).insertValue(row.get(idx)); + break; + case ValueType::vtFloat32: + assert_cast(column).insertValue(row.get(idx)); + break; + case ValueType::vtFloat64: + assert_cast(column).insertValue(row.get(idx)); + break; + case ValueType::vtFixedString:[[fallthrough]]; + case ValueType::vtString: + assert_cast(column).insert(row.get(idx)); + break; + case ValueType::vtUUID: + { + auto value = row.get(idx); + assert_cast(column).insert(parse(value.data(), value.size())); + break; + } + case ValueType::vtDate: + assert_cast(column).insertValue(UInt16{LocalDate{row.get(idx)}.getDayNum()}); + break; + case ValueType::vtDateTime: + { + auto value = row.get(idx); + ReadBufferFromString in(value); + time_t time = 0; + readDateTimeText(time, in); + if (time < 0) + time = 0; + assert_cast(column).insertValue(time); + break; + } + case ValueType::vtDateTime64:[[fallthrough]]; + case ValueType::vtDecimal32: [[fallthrough]]; + case ValueType::vtDecimal64: [[fallthrough]]; + case ValueType::vtDecimal128: [[fallthrough]]; + case ValueType::vtDecimal256: + { + auto value = row.get(idx); + ReadBufferFromString istr(value); + data_type->getDefaultSerialization()->deserializeWholeText(column, istr, FormatSettings{}); + break; + } + default: + throw Exception("Unsupported value type", ErrorCodes::UNKNOWN_TYPE); + } +} + } diff --git a/programs/odbc-bridge/ODBCBlockInputStream.h b/programs/odbc-bridge/ODBCBlockInputStream.h index 13491e05822..bbd90ce4d6c 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.h +++ b/programs/odbc-bridge/ODBCBlockInputStream.h @@ -3,10 +3,8 @@ #include #include #include -#include -#include -#include #include +#include namespace DB @@ -15,25 +13,33 @@ namespace DB class ODBCBlockInputStream final : public IBlockInputStream { public: - ODBCBlockInputStream( - Poco::Data::Session && session_, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_); + ODBCBlockInputStream(nanodbc::connection & connection_, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_); String getName() const override { return "ODBC"; } Block getHeader() const override { return description.sample_block.cloneEmpty(); } private: + using QueryResult = std::shared_ptr; + using ValueType = ExternalResultDescription::ValueType; + Block readImpl() override; - Poco::Data::Session session; - Poco::Data::Statement statement; - Poco::Data::RecordSet result; - Poco::Data::RecordSet::Iterator iterator; + static void insertValue(IColumn & column, const DataTypePtr data_type, const ValueType type, nanodbc::result & row, size_t idx); + static void insertDefaultValue(IColumn & column, const IColumn & sample_column) + { + column.insertFrom(sample_column, 0); + } + + Poco::Logger * log; const UInt64 max_block_size; ExternalResultDescription description; - Poco::Logger * log; + nanodbc::connection & connection; + nanodbc::result result; + String query; + bool finished = false; }; } diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.cpp b/programs/odbc-bridge/ODBCBlockOutputStream.cpp index db3c9441419..e4614204178 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockOutputStream.cpp @@ -8,16 +8,14 @@ #include #include #include "getIdentifierQuote.h" +#include +#include +#include namespace DB { -namespace ErrorCodes -{ - extern const int UNKNOWN_TYPE; -} - namespace { using ValueType = ExternalResultDescription::ValueType; @@ -40,69 +38,21 @@ namespace return buf.str(); } - std::string getQuestionMarks(size_t n) - { - std::string result = "("; - for (size_t i = 0; i < n; ++i) - { - if (i > 0) - result += ","; - result += "?"; - } - return result + ")"; - } - - Poco::Dynamic::Var getVarFromField(const Field & field, const ValueType type) - { - switch (type) - { - case ValueType::vtUInt8: - return Poco::Dynamic::Var(static_cast(field.get())).convert(); - case ValueType::vtUInt16: - return Poco::Dynamic::Var(static_cast(field.get())).convert(); - case ValueType::vtUInt32: - return Poco::Dynamic::Var(static_cast(field.get())).convert(); - case ValueType::vtUInt64: - return Poco::Dynamic::Var(field.get()).convert(); - case ValueType::vtInt8: - return Poco::Dynamic::Var(static_cast(field.get())).convert(); - case ValueType::vtInt16: - return Poco::Dynamic::Var(static_cast(field.get())).convert(); - case ValueType::vtInt32: - return Poco::Dynamic::Var(static_cast(field.get())).convert(); - case ValueType::vtInt64: - return Poco::Dynamic::Var(field.get()).convert(); - case ValueType::vtFloat32: - return Poco::Dynamic::Var(field.get()).convert(); - case ValueType::vtFloat64: - return Poco::Dynamic::Var(field.get()).convert(); - case ValueType::vtString: - return Poco::Dynamic::Var(field.get()).convert(); - case ValueType::vtDate: - return Poco::Dynamic::Var(LocalDate(DayNum(field.get())).toString()).convert(); - case ValueType::vtDateTime: - return Poco::Dynamic::Var(DateLUT::instance().timeToString(time_t(field.get()))).convert(); - case ValueType::vtUUID: - return Poco::Dynamic::Var(UUID(field.get()).toUnderType().toHexString()).convert(); - default: - throw Exception("Unsupported value type", ErrorCodes::UNKNOWN_TYPE); - - } - __builtin_unreachable(); - } } -ODBCBlockOutputStream::ODBCBlockOutputStream(Poco::Data::Session && session_, +ODBCBlockOutputStream::ODBCBlockOutputStream(nanodbc::connection & connection_, const std::string & remote_database_name_, const std::string & remote_table_name_, const Block & sample_block_, + ContextPtr local_context_, IdentifierQuotingStyle quoting_) - : session(session_) + : log(&Poco::Logger::get("ODBCBlockOutputStream")) + , connection(connection_) , db_name(remote_database_name_) , table_name(remote_table_name_) , sample_block(sample_block_) + , local_context(local_context_) , quoting(quoting_) - , log(&Poco::Logger::get("ODBCBlockOutputStream")) { description.init(sample_block); } @@ -114,28 +64,12 @@ Block ODBCBlockOutputStream::getHeader() const void ODBCBlockOutputStream::write(const Block & block) { - ColumnsWithTypeAndName columns; - for (size_t i = 0; i < block.columns(); ++i) - columns.push_back({block.getColumns()[i], sample_block.getDataTypes()[i], sample_block.getNames()[i]}); + WriteBufferFromOwnString values_buf; + auto writer = FormatFactory::instance().getOutputStream("Values", values_buf, sample_block, local_context); + writer->write(block); - std::vector row_to_insert(block.columns()); - Poco::Data::Statement statement(session << getInsertQuery(db_name, table_name, columns, quoting) + getQuestionMarks(block.columns())); - for (size_t i = 0; i < block.columns(); ++i) - statement.addBind(Poco::Data::Keywords::use(row_to_insert[i])); - - for (size_t i = 0; i < block.rows(); ++i) - { - for (size_t col_idx = 0; col_idx < block.columns(); ++col_idx) - { - Field val; - columns[col_idx].column->get(i, val); - if (val.isNull()) - row_to_insert[col_idx] = Poco::Dynamic::Var(); - else - row_to_insert[col_idx] = getVarFromField(val, description.types[col_idx].first); - } - statement.execute(); - } + std::string query = getInsertQuery(db_name, table_name, block.getColumnsWithTypeAndName(), quoting) + values_buf.str(); + execute(connection, query); } } diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.h b/programs/odbc-bridge/ODBCBlockOutputStream.h index 39e1d6f77ac..0b13f7039b5 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.h +++ b/programs/odbc-bridge/ODBCBlockOutputStream.h @@ -2,30 +2,41 @@ #include #include -#include #include #include +#include +#include + namespace DB { + class ODBCBlockOutputStream : public IBlockOutputStream { + public: - ODBCBlockOutputStream(Poco::Data::Session && session_, const std::string & remote_database_name_, - const std::string & remote_table_name_, const Block & sample_block_, IdentifierQuotingStyle quoting); + ODBCBlockOutputStream( + nanodbc::connection & connection_, + const std::string & remote_database_name_, + const std::string & remote_table_name_, + const Block & sample_block_, + ContextPtr local_context_, + IdentifierQuotingStyle quoting); Block getHeader() const override; void write(const Block & block) override; private: - Poco::Data::Session session; + Poco::Logger * log; + + nanodbc::connection & connection; std::string db_name; std::string table_name; Block sample_block; + ContextPtr local_context; IdentifierQuotingStyle quoting; ExternalResultDescription description; - Poco::Logger * log; }; } diff --git a/programs/odbc-bridge/ODBCBridge.cpp b/programs/odbc-bridge/ODBCBridge.cpp index 8869a2639c1..0deefe46014 100644 --- a/programs/odbc-bridge/ODBCBridge.cpp +++ b/programs/odbc-bridge/ODBCBridge.cpp @@ -1,244 +1,4 @@ #include "ODBCBridge.h" -#include "HandlerFactory.h" - -#include -#include -#include -#include - -#if USE_ODBC -// It doesn't make much sense to build this bridge without ODBC, but we still do this. -# include -#endif - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int ARGUMENT_OUT_OF_BOUND; -} - -namespace -{ - Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, Poco::Logger * log) - { - Poco::Net::SocketAddress socket_address; - try - { - socket_address = Poco::Net::SocketAddress(host, port); - } - catch (const Poco::Net::DNSException & e) - { - const auto code = e.code(); - if (code == EAI_FAMILY -#if defined(EAI_ADDRFAMILY) - || code == EAI_ADDRFAMILY -#endif - ) - { - LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. If it is an IPv6 address and your host has disabled IPv6, then consider to specify IPv4 address to listen in element of configuration file. Example: 0.0.0.0", host, e.code(), e.message()); - } - - throw; - } - return socket_address; - } - - Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, Poco::Logger * log) - { - auto address = makeSocketAddress(host, port, log); -#if POCO_VERSION < 0x01080000 - socket.bind(address, /* reuseAddress = */ true); -#else - socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ false); -#endif - - socket.listen(/* backlog = */ 64); - - return address; - } -} - -void ODBCBridge::handleHelp(const std::string &, const std::string &) -{ - Poco::Util::HelpFormatter help_formatter(options()); - help_formatter.setCommand(commandName()); - help_formatter.setHeader("HTTP-proxy for odbc requests"); - help_formatter.setUsage("--http-port "); - help_formatter.format(std::cerr); - - stopOptionsProcessing(); -} - - -void ODBCBridge::defineOptions(Poco::Util::OptionSet & options) -{ - options.addOption(Poco::Util::Option("http-port", "", "port to listen").argument("http-port", true).binding("http-port")); - options.addOption( - Poco::Util::Option("listen-host", "", "hostname or address to listen, default 127.0.0.1").argument("listen-host").binding("listen-host")); - options.addOption( - Poco::Util::Option("http-timeout", "", "http timeout for socket, default 1800").argument("http-timeout").binding("http-timeout")); - - options.addOption(Poco::Util::Option("max-server-connections", "", "max connections to server, default 1024") - .argument("max-server-connections") - .binding("max-server-connections")); - options.addOption(Poco::Util::Option("keep-alive-timeout", "", "keepalive timeout, default 10") - .argument("keep-alive-timeout") - .binding("keep-alive-timeout")); - - options.addOption(Poco::Util::Option("log-level", "", "sets log level, default info").argument("log-level").binding("logger.level")); - - options.addOption( - Poco::Util::Option("log-path", "", "log path for all logs, default console").argument("log-path").binding("logger.log")); - - options.addOption(Poco::Util::Option("err-log-path", "", "err log path for all logs, default no") - .argument("err-log-path") - .binding("logger.errorlog")); - - options.addOption(Poco::Util::Option("stdout-path", "", "stdout log path, default console") - .argument("stdout-path") - .binding("logger.stdout")); - - options.addOption(Poco::Util::Option("stderr-path", "", "stderr log path, default console") - .argument("stderr-path") - .binding("logger.stderr")); - - using Me = std::decay_t; - options.addOption(Poco::Util::Option("help", "", "produce this help message") - .binding("help") - .callback(Poco::Util::OptionCallback(this, &Me::handleHelp))); - - ServerApplication::defineOptions(options); // NOLINT Don't need complex BaseDaemon's .xml config -} - -void ODBCBridge::initialize(Application & self) -{ - BaseDaemon::closeFDs(); - is_help = config().has("help"); - - if (is_help) - return; - - config().setString("logger", "ODBCBridge"); - - /// Redirect stdout, stderr to specified files. - /// Some libraries and sanitizers write to stderr in case of errors. - const auto stdout_path = config().getString("logger.stdout", ""); - if (!stdout_path.empty()) - { - if (!freopen(stdout_path.c_str(), "a+", stdout)) - throw Poco::OpenFileException("Cannot attach stdout to " + stdout_path); - - /// Disable buffering for stdout. - setbuf(stdout, nullptr); - } - const auto stderr_path = config().getString("logger.stderr", ""); - if (!stderr_path.empty()) - { - if (!freopen(stderr_path.c_str(), "a+", stderr)) - throw Poco::OpenFileException("Cannot attach stderr to " + stderr_path); - - /// Disable buffering for stderr. - setbuf(stderr, nullptr); - } - - buildLoggers(config(), logger(), self.commandName()); - - BaseDaemon::logRevision(); - - log = &logger(); - hostname = config().getString("listen-host", "127.0.0.1"); - port = config().getUInt("http-port"); - if (port > 0xFFFF) - throw Exception("Out of range 'http-port': " + std::to_string(port), ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - http_timeout = config().getUInt("http-timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT); - max_server_connections = config().getUInt("max-server-connections", 1024); - keep_alive_timeout = config().getUInt("keep-alive-timeout", 10); - - initializeTerminationAndSignalProcessing(); - -#if USE_ODBC - // It doesn't make much sense to build this bridge without ODBC, but we - // still do this. - Poco::Data::ODBC::Connector::registerConnector(); -#endif - - ServerApplication::initialize(self); // NOLINT -} - -void ODBCBridge::uninitialize() -{ - BaseDaemon::uninitialize(); -} - -int ODBCBridge::main(const std::vector & /*args*/) -{ - if (is_help) - return Application::EXIT_OK; - - registerFormats(); - - LOG_INFO(log, "Starting up"); - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, hostname, port, log); - socket.setReceiveTimeout(http_timeout); - socket.setSendTimeout(http_timeout); - Poco::ThreadPool server_pool(3, max_server_connections); - Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; - http_params->setTimeout(http_timeout); - http_params->setKeepAliveTimeout(keep_alive_timeout); - - auto shared_context = Context::createShared(); - Context context(Context::createGlobal(shared_context.get())); - context.makeGlobalContext(); - - if (config().has("query_masking_rules")) - { - SensitiveDataMasker::setInstance(std::make_unique(config(), "query_masking_rules")); - } - - auto server = HTTPServer( - context, - std::make_shared("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context), - server_pool, - socket, - http_params); - server.start(); - - LOG_INFO(log, "Listening http://{}", address.toString()); - - SCOPE_EXIT({ - LOG_DEBUG(log, "Received termination signal."); - LOG_DEBUG(log, "Waiting for current connections to close."); - server.stop(); - for (size_t count : ext::range(1, 6)) - { - if (server.currentConnections() == 0) - break; - LOG_DEBUG(log, "Waiting for {} connections, try {}", server.currentConnections(), count); - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); - } - }); - - waitForTerminationRequest(); - return Application::EXIT_OK; -} -} #pragma GCC diagnostic ignored "-Wmissing-declarations" int mainEntryClickHouseODBCBridge(int argc, char ** argv) diff --git a/programs/odbc-bridge/ODBCBridge.h b/programs/odbc-bridge/ODBCBridge.h index 9a0d37fa0f9..b17051dce91 100644 --- a/programs/odbc-bridge/ODBCBridge.h +++ b/programs/odbc-bridge/ODBCBridge.h @@ -2,38 +2,25 @@ #include #include -#include +#include +#include "HandlerFactory.h" + namespace DB { -/** Class represents clickhouse-odbc-bridge server, which listen - * incoming HTTP POST and GET requests on specified port and host. - * Has two handlers '/' for all incoming POST requests to ODBC driver - * and /ping for GET request about service status - */ -class ODBCBridge : public BaseDaemon + +class ODBCBridge : public IBridge { -public: - void defineOptions(Poco::Util::OptionSet & options) override; protected: - void initialize(Application & self) override; + std::string bridgeName() const override + { + return "ODBCBridge"; + } - void uninitialize() override; - - int main(const std::vector & args) override; - -private: - void handleHelp(const std::string &, const std::string &); - - bool is_help; - std::string hostname; - size_t port; - size_t http_timeout; - std::string log_level; - size_t max_server_connections; - size_t keep_alive_timeout; - - Poco::Logger * log; + HandlerFactoryPtr getHandlerFactoryPtr(ContextPtr context) const override + { + return std::make_shared("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context); + } }; } diff --git a/programs/odbc-bridge/ODBCConnectionFactory.h b/programs/odbc-bridge/ODBCConnectionFactory.h new file mode 100644 index 00000000000..958cf03cfce --- /dev/null +++ b/programs/odbc-bridge/ODBCConnectionFactory.h @@ -0,0 +1,82 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace nanodbc +{ + +static constexpr inline auto ODBC_CONNECT_TIMEOUT = 100; + +using ConnectionPtr = std::shared_ptr; +using Pool = BorrowedObjectPool; +using PoolPtr = std::shared_ptr; + +class ConnectionHolder +{ + +public: + ConnectionHolder(const std::string & connection_string_, PoolPtr pool_) : connection_string(connection_string_), pool(pool_) {} + + ~ConnectionHolder() + { + if (connection) + pool->returnObject(std::move(connection)); + } + + nanodbc::connection & operator*() + { + if (!connection) + { + pool->borrowObject(connection, [&]() + { + return std::make_shared(connection_string, ODBC_CONNECT_TIMEOUT); + }); + } + + return *connection; + } + +private: + std::string connection_string; + PoolPtr pool; + ConnectionPtr connection; +}; + +} + + +namespace DB +{ + +class ODBCConnectionFactory final : private boost::noncopyable +{ +public: + static ODBCConnectionFactory & instance() + { + static ODBCConnectionFactory ret; + return ret; + } + + nanodbc::ConnectionHolder get(const std::string & connection_string, size_t pool_size) + { + std::lock_guard lock(mutex); + + if (!factory.count(connection_string)) + factory.emplace(std::make_pair(connection_string, std::make_shared(pool_size))); + + return nanodbc::ConnectionHolder(connection_string, factory[connection_string]); + } + +private: + /// [connection_settings_string] -> [connection_pool] + using PoolFactory = std::unordered_map; + PoolFactory factory; + std::mutex mutex; +}; + +} diff --git a/programs/odbc-bridge/SchemaAllowedHandler.cpp b/programs/odbc-bridge/SchemaAllowedHandler.cpp index d4a70db61f4..4cceaee962c 100644 --- a/programs/odbc-bridge/SchemaAllowedHandler.cpp +++ b/programs/odbc-bridge/SchemaAllowedHandler.cpp @@ -2,33 +2,26 @@ #if USE_ODBC -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include "validateODBCConnectionString.h" +#include +#include +#include +#include +#include +#include +#include "validateODBCConnectionString.h" +#include "ODBCConnectionFactory.h" +#include +#include -# define POCO_SQL_ODBC_CLASS Poco::Data::ODBC namespace DB { namespace { - bool isSchemaAllowed(SQLHDBC hdbc) + bool isSchemaAllowed(nanodbc::connection & connection) { - SQLUINTEGER value; - SQLSMALLINT value_length = sizeof(value); - SQLRETURN r = POCO_SQL_ODBC_CLASS::SQLGetInfo(hdbc, SQL_SCHEMA_USAGE, &value, sizeof(value), &value_length); - - if (POCO_SQL_ODBC_CLASS::Utility::isError(r)) - throw POCO_SQL_ODBC_CLASS::ConnectionException(hdbc); - - return value != 0; + uint32_t result = connection.get_info(SQL_SCHEMA_USAGE); + return result != 0; } } @@ -55,10 +48,12 @@ void SchemaAllowedHandler::handleRequest(HTTPServerRequest & request, HTTPServer try { std::string connection_string = params.get("connection_string"); - POCO_SQL_ODBC_CLASS::SessionImpl session(validateODBCConnectionString(connection_string), DBMS_DEFAULT_CONNECT_TIMEOUT_SEC); - SQLHDBC hdbc = session.dbc().handle(); - bool result = isSchemaAllowed(hdbc); + auto connection = ODBCConnectionFactory::instance().get( + validateODBCConnectionString(connection_string), + getContext()->getSettingsRef().odbc_bridge_connection_pool_size); + + bool result = isSchemaAllowed(*connection); WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); try diff --git a/programs/odbc-bridge/SchemaAllowedHandler.h b/programs/odbc-bridge/SchemaAllowedHandler.h index 91eddf67803..d7b922ed05b 100644 --- a/programs/odbc-bridge/SchemaAllowedHandler.h +++ b/programs/odbc-bridge/SchemaAllowedHandler.h @@ -1,22 +1,25 @@ #pragma once +#include #include - #include #if USE_ODBC + namespace DB { class Context; /// This handler establishes connection to database, and retrieves whether schema is allowed. -class SchemaAllowedHandler : public HTTPRequestHandler +class SchemaAllowedHandler : public HTTPRequestHandler, WithContext { public: - SchemaAllowedHandler(size_t keep_alive_timeout_, Context &) - : log(&Poco::Logger::get("SchemaAllowedHandler")), keep_alive_timeout(keep_alive_timeout_) + SchemaAllowedHandler(size_t keep_alive_timeout_, ContextPtr context_) + : WithContext(context_) + , log(&Poco::Logger::get("SchemaAllowedHandler")) + , keep_alive_timeout(keep_alive_timeout_) { } diff --git a/programs/odbc-bridge/getIdentifierQuote.cpp b/programs/odbc-bridge/getIdentifierQuote.cpp index 15b3749d37d..d16d2a9eea0 100644 --- a/programs/odbc-bridge/getIdentifierQuote.cpp +++ b/programs/odbc-bridge/getIdentifierQuote.cpp @@ -2,11 +2,10 @@ #if USE_ODBC -# include -# include -# include - -# define POCO_SQL_ODBC_CLASS Poco::Data::ODBC +#include +#include +#include +#include namespace DB @@ -17,33 +16,16 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } -std::string getIdentifierQuote(SQLHDBC hdbc) + +std::string getIdentifierQuote(nanodbc::connection & connection) { - std::string identifier; - - SQLSMALLINT t; - SQLRETURN r = POCO_SQL_ODBC_CLASS::SQLGetInfo(hdbc, SQL_IDENTIFIER_QUOTE_CHAR, nullptr, 0, &t); - - if (POCO_SQL_ODBC_CLASS::Utility::isError(r)) - throw POCO_SQL_ODBC_CLASS::ConnectionException(hdbc); - - if (t > 0) - { - // I have no idea, why to add '2' here, got from: contrib/poco/Data/ODBC/src/ODBCStatementImpl.cpp:60 (SQL_DRIVER_NAME) - identifier.resize(static_cast(t) + 2); - - if (POCO_SQL_ODBC_CLASS::Utility::isError(POCO_SQL_ODBC_CLASS::SQLGetInfo( - hdbc, SQL_IDENTIFIER_QUOTE_CHAR, &identifier[0], SQLSMALLINT((identifier.length() - 1) * sizeof(identifier[0])), &t))) - throw POCO_SQL_ODBC_CLASS::ConnectionException(hdbc); - - identifier.resize(static_cast(t)); - } - return identifier; + return connection.get_info(SQL_IDENTIFIER_QUOTE_CHAR); } -IdentifierQuotingStyle getQuotingStyle(SQLHDBC hdbc) + +IdentifierQuotingStyle getQuotingStyle(nanodbc::connection & connection) { - auto identifier_quote = getIdentifierQuote(hdbc); + auto identifier_quote = getIdentifierQuote(connection); if (identifier_quote.length() == 0) return IdentifierQuotingStyle::None; else if (identifier_quote[0] == '`') diff --git a/programs/odbc-bridge/getIdentifierQuote.h b/programs/odbc-bridge/getIdentifierQuote.h index 0fb4c3bddb1..7f7156eff82 100644 --- a/programs/odbc-bridge/getIdentifierQuote.h +++ b/programs/odbc-bridge/getIdentifierQuote.h @@ -2,20 +2,19 @@ #if USE_ODBC -# include -# include -# include - -# include - +#include +#include +#include #include +#include + namespace DB { -std::string getIdentifierQuote(SQLHDBC hdbc); +std::string getIdentifierQuote(nanodbc::connection & connection); -IdentifierQuotingStyle getQuotingStyle(SQLHDBC hdbc); +IdentifierQuotingStyle getQuotingStyle(nanodbc::connection & connection); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f2f43aabc7d..8a96612721d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -47,6 +47,7 @@ #include #include #include +#include #include #include #include @@ -425,8 +426,7 @@ int Server::main(const std::vector & /*args*/) * settings, available functions, data types, aggregate functions, databases, ... */ auto shared_context = Context::createShared(); - auto global_context = std::make_unique(Context::createGlobal(shared_context.get())); - global_context_ptr = global_context.get(); + global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); global_context->setApplicationType(Context::ApplicationType::SERVER); @@ -688,16 +688,8 @@ int Server::main(const std::vector & /*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(config(), "macros", log)); @@ -758,6 +750,7 @@ int Server::main(const std::vector & /*args*/) global_context->setClustersConfig(config); global_context->setMacros(std::make_unique(*config, "macros", log)); global_context->setExternalAuthenticatorsConfig(*config); + global_context->setExternalModelsConfig(config); /// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default) if (config->has("max_table_size_to_drop")) @@ -777,6 +770,7 @@ int Server::main(const std::vector & /*args*/) } global_context->updateStorageConfiguration(*config); + global_context->updateInterserverCredentials(*config); }, /* already_loaded = */ false); /// Reload it right now (initial loading) @@ -885,10 +879,30 @@ int Server::main(const std::vector & /*args*/) servers_to_start_before_tables->emplace_back( port_name, std::make_unique( - new KeeperTCPHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); + new KeeperTCPHandlerFactory(*this, false), server_pool, socket, new Poco::Net::TCPServerParams)); LOG_INFO(log, "Listening for connections to Keeper (tcp): {}", address.toString()); }); + + const char * secure_port_name = "keeper_server.tcp_port_secure"; + createServer(listen_host, secure_port_name, listen_try, [&](UInt16 port) + { +#if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + servers_to_start_before_tables->emplace_back( + secure_port_name, + std::make_unique( + new KeeperTCPHandlerFactory(*this, true), server_pool, socket, new Poco::Net::TCPServerParams)); + LOG_INFO(log, "Listening for connections to Keeper with secure protocol (tcp_secure): {}", address.toString()); +#else + UNUSED(port); + throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", + ErrorCodes::SUPPORT_IS_DISABLED}; +#endif + }); } #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination."); @@ -940,7 +954,6 @@ int Server::main(const std::vector & /*args*/) /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. * At this moment, no one could own shared part of Context. */ - global_context_ptr = nullptr; global_context.reset(); shared_context.reset(); LOG_DEBUG(log, "Destroyed global context."); @@ -954,14 +967,14 @@ int Server::main(const std::vector & /*args*/) try { - loadMetadataSystem(*global_context); + loadMetadataSystem(global_context); /// After attaching system databases we can initialize system log. global_context->initializeSystemLogs(); auto & database_catalog = DatabaseCatalog::instance(); /// After the system database is created, attach virtual system tables (in addition to query_log and part_log) attachSystemTablesServer(*database_catalog.getSystemDatabase(), has_zookeeper); /// Then, load remaining databases - loadMetadata(*global_context, default_database); + loadMetadata(global_context, default_database); database_catalog.loadDatabases(); /// After loading validate that default database exists database_catalog.assertDatabaseExists(default_database); @@ -1041,7 +1054,7 @@ int Server::main(const std::vector & /*args*/) else { /// Initialize a watcher periodically updating DNS cache - dns_cache_updater = std::make_unique(*global_context, config().getInt("dns_cache_update_period", 15)); + dns_cache_updater = std::make_unique(global_context, config().getInt("dns_cache_update_period", 15)); } #if defined(OS_LINUX) @@ -1073,7 +1086,7 @@ int Server::main(const std::vector & /*args*/) { /// This object will periodically calculate some metrics. AsynchronousMetrics async_metrics( - *global_context, config().getUInt("asynchronous_metrics_update_period_s", 60), servers_to_start_before_tables, servers); + global_context, config().getUInt("asynchronous_metrics_update_period_s", 60), servers_to_start_before_tables, servers); attachSystemTablesAsync(*DatabaseCatalog::instance().getSystemDatabase(), async_metrics); for (const auto & listen_host : listen_hosts) @@ -1310,7 +1323,7 @@ int Server::main(const std::vector & /*args*/) } /// try to load dictionaries immediately, throw on error and die - ext::scope_guard dictionaries_xmls, models_xmls; + ext::scope_guard dictionaries_xmls; try { if (!config().getBool("dictionaries_lazy_load", true)) @@ -1320,8 +1333,6 @@ int Server::main(const std::vector & /*args*/) } dictionaries_xmls = global_context->getExternalDictionariesLoader().addConfigRepository( std::make_unique(config(), "dictionaries_config")); - models_xmls = global_context->getExternalModelsLoader().addConfigRepository( - std::make_unique(config(), "models_config")); } catch (...) { @@ -1336,7 +1347,7 @@ int Server::main(const std::vector & /*args*/) int pool_size = config().getInt("distributed_ddl.pool_size", 1); if (pool_size < 1) throw Exception("distributed_ddl.pool_size should be greater then 0", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - global_context->setDDLWorker(std::make_unique(pool_size, ddl_zookeeper_path, *global_context, &config(), + global_context->setDDLWorker(std::make_unique(pool_size, ddl_zookeeper_path, global_context, &config(), "distributed_ddl", "DDLWorker", &CurrentMetrics::MaxDDLEntryID)); } diff --git a/programs/server/Server.h b/programs/server/Server.h index fbfc26f6ee5..c698108767c 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -40,9 +40,9 @@ public: return BaseDaemon::logger(); } - Context & context() const override + ContextPtr context() const override { - return *global_context_ptr; + return global_context; } bool isCancelled() const override @@ -64,8 +64,7 @@ protected: std::string getDefaultCorePath() const override; private: - Context * global_context_ptr = nullptr; - + ContextPtr global_context; Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const; using CreateServerFunc = std::function; diff --git a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp index c699dd4f217..09e343b2dc5 100644 --- a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp +++ b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp @@ -60,6 +60,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void create(AggregateDataPtr __restrict place) const override { if (std::uniform_real_distribution<>(0.0, 1.0)(thread_local_rng) <= throw_probability) diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 7bf742294b4..96d48941f8b 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -98,6 +98,8 @@ public: DataTypePtr getReturnType() const final { return std::make_shared>(); } + bool allocatesMemoryInArena() const override { return false; } + void NO_SANITIZE_UNDEFINED merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { this->data(place).numerator += this->data(rhs).numerator; diff --git a/src/AggregateFunctions/AggregateFunctionBitwise.h b/src/AggregateFunctions/AggregateFunctionBitwise.h index 3ba8e045069..b48b1960329 100644 --- a/src/AggregateFunctions/AggregateFunctionBitwise.h +++ b/src/AggregateFunctions/AggregateFunctionBitwise.h @@ -54,6 +54,8 @@ public: return std::make_shared>(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { this->data(place).update(assert_cast &>(*columns[0]).getData()[row_num]); diff --git a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index 32ae22fd573..0cfb33efc10 100644 --- a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -127,6 +127,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override { /// NOTE Slightly inefficient. diff --git a/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.h b/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.h index ba8acb208ea..fb3e35fbcf1 100644 --- a/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.h +++ b/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.h @@ -33,6 +33,8 @@ public: return "categoricalInformationValue"; } + bool allocatesMemoryInArena() const override { return false; } + void create(AggregateDataPtr __restrict place) const override { memset(place, 0, sizeOfData()); diff --git a/src/AggregateFunctions/AggregateFunctionCombinatorFactory.cpp b/src/AggregateFunctions/AggregateFunctionCombinatorFactory.cpp index a20d355bb2f..e4ff8c134c5 100644 --- a/src/AggregateFunctions/AggregateFunctionCombinatorFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionCombinatorFactory.cpp @@ -13,17 +13,25 @@ namespace ErrorCodes void AggregateFunctionCombinatorFactory::registerCombinator(const AggregateFunctionCombinatorPtr & value) { - if (!dict.emplace(value->getName(), value).second) - throw Exception("AggregateFunctionCombinatorFactory: the name '" + value->getName() + "' is not unique", - ErrorCodes::LOGICAL_ERROR); + CombinatorPair pair{ + .name = value->getName(), + .combinator_ptr = value, + }; + + /// lower_bound() cannot be used since sort order of the dict is by length of the combinator + /// but there are just a few combiners, so not a problem. + if (std::find(dict.begin(), dict.end(), pair) != dict.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionCombinatorFactory: the name '{}' is not unique", + value->getName()); + dict.emplace(std::lower_bound(dict.begin(), dict.end(), pair), pair); } AggregateFunctionCombinatorPtr AggregateFunctionCombinatorFactory::tryFindSuffix(const std::string & name) const { /// O(N) is ok for just a few combinators. for (const auto & suffix_value : dict) - if (endsWith(name, suffix_value.first)) - return suffix_value.second; + if (endsWith(name, suffix_value.name)) + return suffix_value.combinator_ptr; return {}; } diff --git a/src/AggregateFunctions/AggregateFunctionCombinatorFactory.h b/src/AggregateFunctions/AggregateFunctionCombinatorFactory.h index b535475d111..5f7658c16af 100644 --- a/src/AggregateFunctions/AggregateFunctionCombinatorFactory.h +++ b/src/AggregateFunctions/AggregateFunctionCombinatorFactory.h @@ -15,7 +15,17 @@ namespace DB class AggregateFunctionCombinatorFactory final: private boost::noncopyable { private: - using Dict = std::unordered_map; + struct CombinatorPair + { + std::string name; + AggregateFunctionCombinatorPtr combinator_ptr; + + bool operator==(const CombinatorPair & rhs) const { return name == rhs.name; } + /// Sort by the length of the combinator name for proper tryFindSuffix() + /// for combiners with common prefix (i.e. "State" and "SimpleState"). + bool operator<(const CombinatorPair & rhs) const { return name.length() > rhs.name.length(); } + }; + using Dict = std::vector; Dict dict; public: diff --git a/src/AggregateFunctions/AggregateFunctionCount.h b/src/AggregateFunctions/AggregateFunctionCount.h index 1b3a0acb528..71db28390af 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.h +++ b/src/AggregateFunctions/AggregateFunctionCount.h @@ -38,6 +38,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn **, size_t, Arena *) const override { ++data(place).count; @@ -126,6 +128,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { data(place).count += !assert_cast(*columns[0]).isNullAt(row_num); diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSum.h b/src/AggregateFunctions/AggregateFunctionDeltaSum.h index d5760de84ae..99406618da5 100644 --- a/src/AggregateFunctions/AggregateFunctionDeltaSum.h +++ b/src/AggregateFunctions/AggregateFunctionDeltaSum.h @@ -43,6 +43,8 @@ public: DataTypePtr getReturnType() const override { return std::make_shared>(); } + bool allocatesMemoryInArena() const override { return false; } + void NO_SANITIZE_UNDEFINED ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { auto value = assert_cast &>(*columns[0]).getData()[row_num]; diff --git a/src/AggregateFunctions/AggregateFunctionEntropy.h b/src/AggregateFunctions/AggregateFunctionEntropy.h index 9bb1bc039c5..587f5aa3fc7 100644 --- a/src/AggregateFunctions/AggregateFunctionEntropy.h +++ b/src/AggregateFunctions/AggregateFunctionEntropy.h @@ -103,6 +103,8 @@ public: return std::make_shared>(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { if constexpr (!std::is_same_v) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index c0011b6ebec..eec8b374424 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -121,7 +121,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( is_case_insensitive = true; } - const Context * query_context = nullptr; + ContextPtr query_context; if (CurrentThread::isInitialized()) query_context = CurrentThread::get().getQueryContext(); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h index 35913f133b1..723ee7140bc 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h @@ -104,6 +104,8 @@ public: return std::make_shared(type); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { /// TODO Do positions need to be 1-based for this function? diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h index 4628410286d..83db274752b 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h @@ -22,6 +22,8 @@ public: DataTypePtr getReturnType() const override { return std::make_shared>(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { this->data(place).rbs.add(assert_cast &>(*columns[0]).getData()[row_num]); @@ -56,6 +58,8 @@ public: DataTypePtr getReturnType() const override { return std::make_shared>(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { Data & data_lhs = this->data(place); diff --git a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index 435efdd2373..16911184f31 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -59,6 +59,8 @@ public: return std::make_shared(this->argument_types[0]); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { if (limit_num_elems && this->data(place).value.size() >= max_elems) diff --git a/src/AggregateFunctions/AggregateFunctionHistogram.h b/src/AggregateFunctions/AggregateFunctionHistogram.h index 76aa96ba663..c44cb61b275 100644 --- a/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -332,6 +332,8 @@ public: return std::make_shared(tuple); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { auto val = assert_cast &>(*columns[0]).getData()[row_num]; diff --git a/src/AggregateFunctions/AggregateFunctionMLMethod.cpp b/src/AggregateFunctions/AggregateFunctionMLMethod.cpp index 6c5c5af2f1d..145660e881b 100644 --- a/src/AggregateFunctions/AggregateFunctionMLMethod.cpp +++ b/src/AggregateFunctions/AggregateFunctionMLMethod.cpp @@ -146,7 +146,7 @@ void LinearModelData::predict( const ColumnsWithTypeAndName & arguments, size_t offset, size_t limit, - const Context & context) const + ContextPtr context) const { gradient_computer->predict(container, arguments, offset, limit, weights, bias, context); } @@ -453,7 +453,7 @@ void LogisticRegression::predict( size_t limit, const std::vector & weights, Float64 bias, - const Context & /*context*/) const + ContextPtr /*context*/) const { size_t rows_num = arguments.front().column->size(); @@ -521,7 +521,7 @@ void LinearRegression::predict( size_t limit, const std::vector & weights, Float64 bias, - const Context & /*context*/) const + ContextPtr /*context*/) const { if (weights.size() + 1 != arguments.size()) { diff --git a/src/AggregateFunctions/AggregateFunctionMLMethod.h b/src/AggregateFunctions/AggregateFunctionMLMethod.h index 0c88f9d877d..6d97feb3193 100644 --- a/src/AggregateFunctions/AggregateFunctionMLMethod.h +++ b/src/AggregateFunctions/AggregateFunctionMLMethod.h @@ -3,10 +3,10 @@ #include #include #include -#include -#include -#include #include +#include +#include +#include #include "IAggregateFunction.h" namespace DB @@ -44,7 +44,7 @@ public: size_t limit, const std::vector & weights, Float64 bias, - const Context & context) const = 0; + ContextPtr context) const = 0; }; @@ -69,7 +69,7 @@ public: size_t limit, const std::vector & weights, Float64 bias, - const Context & context) const override; + ContextPtr context) const override; }; @@ -94,7 +94,7 @@ public: size_t limit, const std::vector & weights, Float64 bias, - const Context & context) const override; + ContextPtr context) const override; }; @@ -264,7 +264,7 @@ public: const ColumnsWithTypeAndName & arguments, size_t offset, size_t limit, - const Context & context) const; + ContextPtr context) const; void returnWeights(IColumn & to) const; private: @@ -323,6 +323,8 @@ public: return std::make_shared(std::make_shared()); } + bool allocatesMemoryInArena() const override { return false; } + /// This function is called from evalMLMethod function for correct predictValues call DataTypePtr getReturnTypeToPredict() const override { @@ -363,7 +365,7 @@ public: const ColumnsWithTypeAndName & arguments, size_t offset, size_t limit, - const Context & context) const override + ContextPtr context) const override { if (arguments.size() != param_num + 1) throw Exception( diff --git a/src/AggregateFunctions/AggregateFunctionMannWhitney.h b/src/AggregateFunctions/AggregateFunctionMannWhitney.h index 1451536d519..7efdebf1639 100644 --- a/src/AggregateFunctions/AggregateFunctionMannWhitney.h +++ b/src/AggregateFunctions/AggregateFunctionMannWhitney.h @@ -174,6 +174,8 @@ public: return "mannWhitneyUTest"; } + bool allocatesMemoryInArena() const override { return true; } + DataTypePtr getReturnType() const override { DataTypes types @@ -208,7 +210,7 @@ public: void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override { auto & a = this->data(place); - auto & b = this->data(rhs); + const auto & b = this->data(rhs); a.merge(b, arena); } diff --git a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index d4946ad2c9d..94509a40ada 100644 --- a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -87,6 +87,8 @@ public: return std::make_shared>(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override { PointType left = assert_cast &>(*columns[0]).getData()[row_num]; diff --git a/src/AggregateFunctions/AggregateFunctionNothing.h b/src/AggregateFunctions/AggregateFunctionNothing.h index f373b3b55b0..a094c1e0fac 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.h +++ b/src/AggregateFunctions/AggregateFunctionNothing.h @@ -28,6 +28,8 @@ public: return argument_types.front(); } + bool allocatesMemoryInArena() const override { return false; } + void create(AggregateDataPtr) const override { } diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.h b/src/AggregateFunctions/AggregateFunctionQuantile.h index edd24add736..209784361dd 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -103,6 +103,8 @@ public: return res; } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { auto value = static_cast(*columns[0]).getData()[row_num]; diff --git a/src/AggregateFunctions/AggregateFunctionRankCorrelation.h b/src/AggregateFunctions/AggregateFunctionRankCorrelation.h index a7e0852378c..fbba1b99ab9 100644 --- a/src/AggregateFunctions/AggregateFunctionRankCorrelation.h +++ b/src/AggregateFunctions/AggregateFunctionRankCorrelation.h @@ -58,6 +58,8 @@ public: return "rankCorr"; } + bool allocatesMemoryInArena() const override { return true; } + DataTypePtr getReturnType() const override { return std::make_shared>(); diff --git a/src/AggregateFunctions/AggregateFunctionRetention.h b/src/AggregateFunctions/AggregateFunctionRetention.h index 5f0d9907280..aee74cb7324 100644 --- a/src/AggregateFunctions/AggregateFunctionRetention.h +++ b/src/AggregateFunctions/AggregateFunctionRetention.h @@ -94,6 +94,8 @@ public: return std::make_shared(std::make_shared()); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override { for (const auto i : ext::range(0, events_size)) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceMatch.h b/src/AggregateFunctions/AggregateFunctionSequenceMatch.h index 48015a6d282..2c2a4d4c75a 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -560,6 +560,8 @@ public: DataTypePtr getReturnType() const override { return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { this->data(place).sort(); @@ -588,6 +590,8 @@ public: DataTypePtr getReturnType() const override { return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { this->data(place).sort(); diff --git a/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h b/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h index 8c029855a26..61f10895de6 100644 --- a/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h +++ b/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h @@ -168,6 +168,8 @@ public: ); } + bool allocatesMemoryInArena() const override { return false; } + void insertResultInto( AggregateDataPtr place, IColumn & to, diff --git a/src/AggregateFunctions/AggregateFunctionStatistics.h b/src/AggregateFunctions/AggregateFunctionStatistics.h index 76b6e843c15..473bce1d89a 100644 --- a/src/AggregateFunctions/AggregateFunctionStatistics.h +++ b/src/AggregateFunctions/AggregateFunctionStatistics.h @@ -123,6 +123,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { this->data(place).update(*columns[0], row_num); @@ -375,6 +377,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { this->data(place).update(*columns[0], *columns[1], row_num); diff --git a/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h b/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h index 9903e2f6eaa..a5d6bbb7dc8 100644 --- a/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h +++ b/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h @@ -121,6 +121,8 @@ public: return std::make_shared>(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { if constexpr (StatFunc::num_args == 2) diff --git a/src/AggregateFunctions/AggregateFunctionSum.h b/src/AggregateFunctions/AggregateFunctionSum.h index bd1f9fc302e..1748458f6d8 100644 --- a/src/AggregateFunctions/AggregateFunctionSum.h +++ b/src/AggregateFunctions/AggregateFunctionSum.h @@ -314,6 +314,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { const auto & column = assert_cast(*columns[0]); diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index d8b19bad62f..7819bb8752c 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -140,6 +140,8 @@ public: return std::make_shared(types); } + bool allocatesMemoryInArena() const override { return false; } + static const auto & getArgumentColumns(const IColumn**& columns) { if constexpr (tuple_argument) diff --git a/src/AggregateFunctions/AggregateFunctionTTest.h b/src/AggregateFunctions/AggregateFunctionTTest.h index 3c9873ebd1e..5617adf38dd 100644 --- a/src/AggregateFunctions/AggregateFunctionTTest.h +++ b/src/AggregateFunctions/AggregateFunctionTTest.h @@ -109,6 +109,8 @@ public: ); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { Float64 value = columns[0]->getFloat64(row_num); diff --git a/src/AggregateFunctions/AggregateFunctionTopK.h b/src/AggregateFunctions/AggregateFunctionTopK.h index 43320a96b99..dbc471687ea 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.h +++ b/src/AggregateFunctions/AggregateFunctionTopK.h @@ -50,6 +50,8 @@ public: return std::make_shared(this->argument_types[0]); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { auto & set = this->data(place).value; diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index bddd28e9718..191fc873bb6 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -230,6 +230,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + /// ALWAYS_INLINE is required to have better code layout for uniqHLL12 function void ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { @@ -285,6 +287,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { this->data(place).set.insert(typename Data::Set::value_type( diff --git a/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/src/AggregateFunctions/AggregateFunctionUniqCombined.h index c9327594670..31bd8021dcf 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -141,6 +141,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { if constexpr (!std::is_same_v) @@ -211,6 +213,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { this->data(place).set.insert(typename AggregateFunctionUniqCombinedData::Set::value_type( diff --git a/src/AggregateFunctions/AggregateFunctionUniqUpTo.h b/src/AggregateFunctions/AggregateFunctionUniqUpTo.h index d2c765137bc..88f26ab312e 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqUpTo.h +++ b/src/AggregateFunctions/AggregateFunctionUniqUpTo.h @@ -184,6 +184,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + /// ALWAYS_INLINE is required to have better code layout for uniqUpTo function void ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { @@ -247,6 +249,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { this->data(place).insert(UInt64(UniqVariadicHash::apply(num_args, columns, row_num)), threshold); diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index 9644b8d9ea4..9f399a9f25b 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -247,6 +247,8 @@ public: return std::make_shared(); } + bool allocatesMemoryInArena() const override { return false; } + AggregateFunctionPtr getOwnNullAdapter( const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, const AggregateFunctionProperties & /*properties*/) const override diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 168d12fb2b9..dbb0a9f4349 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -1,21 +1,23 @@ #pragma once +#include +#include +#include +#include +#include +#include +#include +#include + #include #include #include #include -#include -#include -#include -#include -#include -#include -#include - namespace DB { + namespace ErrorCodes { extern const int NOT_IMPLEMENTED; @@ -104,7 +106,7 @@ public: virtual void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const = 0; /// Returns true if a function requires Arena to handle own states (see add(), merge(), deserialize()). - virtual bool allocatesMemoryInArena() const { return false; } + virtual bool allocatesMemoryInArena() const = 0; /// Inserts results into a column. This method might modify the state (e.g. /// sort an array), so must be called once, from single thread. The state @@ -122,7 +124,7 @@ public: const ColumnsWithTypeAndName & /*arguments*/, size_t /*offset*/, size_t /*limit*/, - const Context & /*context*/) const + ContextPtr /*context*/) const { throw Exception("Method predictValues is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Bridge/IBridgeHelper.cpp b/src/Bridge/IBridgeHelper.cpp new file mode 100644 index 00000000000..16886bae4fe --- /dev/null +++ b/src/Bridge/IBridgeHelper.cpp @@ -0,0 +1,132 @@ +#include "IBridgeHelper.h" + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int EXTERNAL_SERVER_IS_NOT_RESPONDING; +} + + +Poco::URI IBridgeHelper::getMainURI() const +{ + auto uri = createBaseURI(); + uri.setPath(MAIN_HANDLER); + return uri; +} + + +Poco::URI IBridgeHelper::getPingURI() const +{ + auto uri = createBaseURI(); + uri.setPath(PING_HANDLER); + return uri; +} + + +bool IBridgeHelper::checkBridgeIsRunning() const +{ + try + { + ReadWriteBufferFromHTTP buf( + getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, ConnectionTimeouts::getHTTPTimeouts(getContext())); + return checkString(PING_OK_ANSWER, buf); + } + catch (...) + { + return false; + } +} + + +void IBridgeHelper::startBridgeSync() const +{ + if (!checkBridgeIsRunning()) + { + LOG_TRACE(getLog(), "{} is not running, will try to start it", serviceAlias()); + startBridge(startBridgeCommand()); + bool started = false; + + uint64_t milliseconds_to_wait = 10; /// Exponential backoff + uint64_t counter = 0; + + while (milliseconds_to_wait < 10000) + { + ++counter; + LOG_TRACE(getLog(), "Checking {} is running, try {}", serviceAlias(), counter); + + if (checkBridgeIsRunning()) + { + started = true; + break; + } + + std::this_thread::sleep_for(std::chrono::milliseconds(milliseconds_to_wait)); + milliseconds_to_wait *= 2; + } + + if (!started) + throw Exception("BridgeHelper: " + serviceAlias() + " is not responding", + ErrorCodes::EXTERNAL_SERVER_IS_NOT_RESPONDING); + } +} + + +std::unique_ptr IBridgeHelper::startBridgeCommand() const +{ + if (startBridgeManually()) + throw Exception(serviceAlias() + " is not running. Please, start it manually", ErrorCodes::EXTERNAL_SERVER_IS_NOT_RESPONDING); + + const auto & config = getConfig(); + /// Path to executable folder + Poco::Path path{config.getString("application.dir", "/usr/bin")}; + + std::vector cmd_args; + path.setFileName(serviceFileName()); + + cmd_args.push_back("--http-port"); + cmd_args.push_back(std::to_string(config.getUInt(configPrefix() + ".port", getDefaultPort()))); + cmd_args.push_back("--listen-host"); + cmd_args.push_back(config.getString(configPrefix() + ".listen_host", DEFAULT_HOST)); + cmd_args.push_back("--http-timeout"); + cmd_args.push_back(std::to_string(getHTTPTimeout().totalMicroseconds())); + if (config.has("logger." + configPrefix() + "_log")) + { + cmd_args.push_back("--log-path"); + cmd_args.push_back(config.getString("logger." + configPrefix() + "_log")); + } + if (config.has("logger." + configPrefix() + "_errlog")) + { + cmd_args.push_back("--err-log-path"); + cmd_args.push_back(config.getString("logger." + configPrefix() + "_errlog")); + } + if (config.has("logger." + configPrefix() + "_stdout")) + { + cmd_args.push_back("--stdout-path"); + cmd_args.push_back(config.getString("logger." + configPrefix() + "_stdout")); + } + if (config.has("logger." + configPrefix() + "_stderr")) + { + cmd_args.push_back("--stderr-path"); + cmd_args.push_back(config.getString("logger." + configPrefix() + "_stderr")); + } + if (config.has("logger." + configPrefix() + "_level")) + { + cmd_args.push_back("--log-level"); + cmd_args.push_back(config.getString("logger." + configPrefix() + "_level")); + } + + LOG_TRACE(getLog(), "Starting {}", serviceAlias()); + + return ShellCommand::executeDirect(path.toString(), cmd_args, ShellCommandDestructorStrategy(true)); +} + +} diff --git a/src/Bridge/IBridgeHelper.h b/src/Bridge/IBridgeHelper.h new file mode 100644 index 00000000000..3473e24047e --- /dev/null +++ b/src/Bridge/IBridgeHelper.h @@ -0,0 +1,69 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Common base class for XDBC and Library bridge helpers. +/// Contains helper methods to check/start bridge sync. +class IBridgeHelper: protected WithContext +{ + +public: + static constexpr inline auto DEFAULT_HOST = "127.0.0.1"; + static constexpr inline auto PING_HANDLER = "/ping"; + static constexpr inline auto MAIN_HANDLER = "/"; + static constexpr inline auto DEFAULT_FORMAT = "RowBinary"; + static constexpr inline auto PING_OK_ANSWER = "Ok."; + + static const inline std::string PING_METHOD = Poco::Net::HTTPRequest::HTTP_GET; + static const inline std::string MAIN_METHOD = Poco::Net::HTTPRequest::HTTP_POST; + + explicit IBridgeHelper(ContextPtr context_) : WithContext(context_) {} + virtual ~IBridgeHelper() = default; + + void startBridgeSync() const; + + Poco::URI getMainURI() const; + + Poco::URI getPingURI() const; + + +protected: + /// clickhouse-odbc-bridge, clickhouse-library-bridge + virtual String serviceAlias() const = 0; + + virtual String serviceFileName() const = 0; + + virtual size_t getDefaultPort() const = 0; + + virtual bool startBridgeManually() const = 0; + + virtual void startBridge(std::unique_ptr cmd) const = 0; + + virtual String configPrefix() const = 0; + + virtual const Poco::Util::AbstractConfiguration & getConfig() const = 0; + + virtual Poco::Logger * getLog() const = 0; + + virtual const Poco::Timespan & getHTTPTimeout() const = 0; + + virtual Poco::URI createBaseURI() const = 0; + + +private: + bool checkBridgeIsRunning() const; + + std::unique_ptr startBridgeCommand() const; +}; + +} diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp new file mode 100644 index 00000000000..3ab01e18b11 --- /dev/null +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -0,0 +1,182 @@ +#include "LibraryBridgeHelper.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +LibraryBridgeHelper::LibraryBridgeHelper( + ContextPtr context_, + const Block & sample_block_, + const Field & dictionary_id_) + : IBridgeHelper(context_) + , log(&Poco::Logger::get("LibraryBridgeHelper")) + , sample_block(sample_block_) + , config(context_->getConfigRef()) + , http_timeout(context_->getSettingsRef().http_receive_timeout.value.totalSeconds()) + , dictionary_id(dictionary_id_) +{ + bridge_port = config.getUInt("library_bridge.port", DEFAULT_PORT); + bridge_host = config.getString("library_bridge.host", DEFAULT_HOST); +} + + +Poco::URI LibraryBridgeHelper::createRequestURI(const String & method) const +{ + auto uri = getMainURI(); + uri.addQueryParameter("dictionary_id", toString(dictionary_id)); + uri.addQueryParameter("method", method); + return uri; +} + + +Poco::URI LibraryBridgeHelper::createBaseURI() const +{ + Poco::URI uri; + uri.setHost(bridge_host); + uri.setPort(bridge_port); + uri.setScheme("http"); + return uri; +} + + +void LibraryBridgeHelper::startBridge(std::unique_ptr cmd) const +{ + getContext()->addBridgeCommand(std::move(cmd)); +} + + +bool LibraryBridgeHelper::initLibrary(const std::string & library_path, const std::string library_settings, const std::string attributes_names) +{ + startBridgeSync(); + auto uri = createRequestURI(LIB_NEW_METHOD); + + /// Sample block must contain null values + WriteBufferFromOwnString out; + auto output_stream = getContext()->getOutputStream(LibraryBridgeHelper::DEFAULT_FORMAT, out, sample_block); + formatBlock(output_stream, sample_block); + auto block_string = out.str(); + + auto out_stream_callback = [library_path, library_settings, attributes_names, block_string, this](std::ostream & os) + { + os << "library_path=" << escapeForFileName(library_path) << "&"; + os << "library_settings=" << escapeForFileName(library_settings) << "&"; + os << "attributes_names=" << escapeForFileName(attributes_names) << "&"; + os << "sample_block=" << escapeForFileName(sample_block.getNamesAndTypesList().toString()) << "&"; + os << "null_values=" << escapeForFileName(block_string); + }; + return executeRequest(uri, out_stream_callback); +} + + +bool LibraryBridgeHelper::cloneLibrary(const Field & other_dictionary_id) +{ + startBridgeSync(); + auto uri = createRequestURI(LIB_CLONE_METHOD); + uri.addQueryParameter("from_dictionary_id", toString(other_dictionary_id)); + return executeRequest(uri); +} + + +bool LibraryBridgeHelper::removeLibrary() +{ + startBridgeSync(); + auto uri = createRequestURI(LIB_DELETE_METHOD); + return executeRequest(uri); +} + + +bool LibraryBridgeHelper::isModified() +{ + startBridgeSync(); + auto uri = createRequestURI(IS_MODIFIED_METHOD); + return executeRequest(uri); +} + + +bool LibraryBridgeHelper::supportsSelectiveLoad() +{ + startBridgeSync(); + auto uri = createRequestURI(SUPPORTS_SELECTIVE_LOAD_METHOD); + return executeRequest(uri); +} + + +BlockInputStreamPtr LibraryBridgeHelper::loadAll() +{ + startBridgeSync(); + auto uri = createRequestURI(LOAD_ALL_METHOD); + return loadBase(uri); +} + + +BlockInputStreamPtr LibraryBridgeHelper::loadIds(const std::string ids_string) +{ + startBridgeSync(); + auto uri = createRequestURI(LOAD_IDS_METHOD); + return loadBase(uri, [ids_string](std::ostream & os) { os << "ids=" << ids_string; }); +} + + +BlockInputStreamPtr LibraryBridgeHelper::loadKeys(const Block & requested_block) +{ + startBridgeSync(); + auto uri = createRequestURI(LOAD_KEYS_METHOD); + /// Sample block to parse block from callback + uri.addQueryParameter("requested_block_sample", requested_block.getNamesAndTypesList().toString()); + ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [requested_block, this](std::ostream & os) + { + WriteBufferFromOStream out_buffer(os); + auto output_stream = getContext()->getOutputStream(LibraryBridgeHelper::DEFAULT_FORMAT, out_buffer, sample_block); + formatBlock(output_stream, requested_block); + }; + return loadBase(uri, out_stream_callback); +} + + +bool LibraryBridgeHelper::executeRequest(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback) +{ + ReadWriteBufferFromHTTP buf( + uri, + Poco::Net::HTTPRequest::HTTP_POST, + std::move(out_stream_callback), + ConnectionTimeouts::getHTTPTimeouts(getContext())); + + bool res; + readBoolText(res, buf); + return res; +} + + +BlockInputStreamPtr LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback) +{ + auto read_buf_ptr = std::make_unique( + uri, + Poco::Net::HTTPRequest::HTTP_POST, + std::move(out_stream_callback), + ConnectionTimeouts::getHTTPTimeouts(getContext()), + 0, + Poco::Net::HTTPBasicCredentials{}, + DBMS_DEFAULT_BUFFER_SIZE, + ReadWriteBufferFromHTTP::HTTPHeaderEntries{}); + + auto input_stream = getContext()->getInputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, DEFAULT_BLOCK_SIZE); + return std::make_shared>(input_stream, std::move(read_buf_ptr)); +} + +} diff --git a/src/Bridge/LibraryBridgeHelper.h b/src/Bridge/LibraryBridgeHelper.h new file mode 100644 index 00000000000..dc3f9020e71 --- /dev/null +++ b/src/Bridge/LibraryBridgeHelper.h @@ -0,0 +1,86 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class LibraryBridgeHelper : public IBridgeHelper +{ + +public: + static constexpr inline size_t DEFAULT_PORT = 9012; + + LibraryBridgeHelper(ContextPtr context_, const Block & sample_block, const Field & dictionary_id_); + + bool initLibrary(const std::string & library_path, std::string library_settings, std::string attributes_names); + + bool cloneLibrary(const Field & other_dictionary_id); + + bool removeLibrary(); + + bool isModified(); + + bool supportsSelectiveLoad(); + + BlockInputStreamPtr loadAll(); + + BlockInputStreamPtr loadIds(std::string ids_string); + + BlockInputStreamPtr loadKeys(const Block & requested_block); + + BlockInputStreamPtr loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = {}); + + bool executeRequest(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = {}); + + +protected: + void startBridge(std::unique_ptr cmd) const override; + + String serviceAlias() const override { return "clickhouse-library-bridge"; } + + String serviceFileName() const override { return serviceAlias(); } + + size_t getDefaultPort() const override { return DEFAULT_PORT; } + + bool startBridgeManually() const override { return false; } + + String configPrefix() const override { return "library_bridge"; } + + const Poco::Util::AbstractConfiguration & getConfig() const override { return config; } + + Poco::Logger * getLog() const override { return log; } + + const Poco::Timespan & getHTTPTimeout() const override { return http_timeout; } + + Poco::URI createBaseURI() const override; + +private: + static constexpr inline auto LIB_NEW_METHOD = "libNew"; + static constexpr inline auto LIB_CLONE_METHOD = "libClone"; + static constexpr inline auto LIB_DELETE_METHOD = "libDelete"; + static constexpr inline auto LOAD_ALL_METHOD = "loadAll"; + static constexpr inline auto LOAD_IDS_METHOD = "loadIds"; + static constexpr inline auto LOAD_KEYS_METHOD = "loadKeys"; + static constexpr inline auto IS_MODIFIED_METHOD = "isModified"; + static constexpr inline auto SUPPORTS_SELECTIVE_LOAD_METHOD = "supportsSelectiveLoad"; + + Poco::URI createRequestURI(const String & method) const; + + Poco::Logger * log; + const Block sample_block; + const Poco::Util::AbstractConfiguration & config; + const Poco::Timespan http_timeout; + + Field dictionary_id; + std::string bridge_host; + size_t bridge_port; +}; + +} diff --git a/src/Bridge/XDBCBridgeHelper.h b/src/Bridge/XDBCBridgeHelper.h new file mode 100644 index 00000000000..8a31171111d --- /dev/null +++ b/src/Bridge/XDBCBridgeHelper.h @@ -0,0 +1,266 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if !defined(ARCADIA_BUILD) +# include +#endif + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +/// Class for Helpers for XDBC-bridges, provide utility methods, not main request. +class IXDBCBridgeHelper : public IBridgeHelper +{ + +public: + explicit IXDBCBridgeHelper(ContextPtr context_) : IBridgeHelper(context_) {} + + virtual std::vector> getURLParams(const std::string & cols, UInt64 max_block_size) const = 0; + + virtual Poco::URI getColumnsInfoURI() const = 0; + + virtual IdentifierQuotingStyle getIdentifierQuotingStyle() = 0; + + virtual bool isSchemaAllowed() = 0; + + virtual String getName() const = 0; +}; + +using BridgeHelperPtr = std::shared_ptr; + + +template +class XDBCBridgeHelper : public IXDBCBridgeHelper +{ + +public: + static constexpr inline auto DEFAULT_PORT = BridgeHelperMixin::DEFAULT_PORT; + static constexpr inline auto COL_INFO_HANDLER = "/columns_info"; + static constexpr inline auto IDENTIFIER_QUOTE_HANDLER = "/identifier_quote"; + static constexpr inline auto SCHEMA_ALLOWED_HANDLER = "/schema_allowed"; + + XDBCBridgeHelper( + ContextPtr global_context_, + const Poco::Timespan & http_timeout_, + const std::string & connection_string_) + : IXDBCBridgeHelper(global_context_) + , log(&Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper")) + , connection_string(connection_string_) + , http_timeout(http_timeout_) + , config(global_context_->getConfigRef()) +{ + bridge_host = config.getString(BridgeHelperMixin::configPrefix() + ".host", DEFAULT_HOST); + bridge_port = config.getUInt(BridgeHelperMixin::configPrefix() + ".port", DEFAULT_PORT); +} + + +protected: + auto getConnectionString() const { return connection_string; } + + String getName() const override { return BridgeHelperMixin::getName(); } + + size_t getDefaultPort() const override { return DEFAULT_PORT; } + + String serviceAlias() const override { return BridgeHelperMixin::serviceAlias(); } + + /// Same for odbc and jdbc + String serviceFileName() const override { return "clickhouse-odbc-bridge"; } + + String configPrefix() const override { return BridgeHelperMixin::configPrefix(); } + + const Poco::Timespan & getHTTPTimeout() const override { return http_timeout; } + + const Poco::Util::AbstractConfiguration & getConfig() const override { return config; } + + Poco::Logger * getLog() const override { return log; } + + bool startBridgeManually() const override { return BridgeHelperMixin::startBridgeManually(); } + + Poco::URI createBaseURI() const override + { + Poco::URI uri; + uri.setHost(bridge_host); + uri.setPort(bridge_port); + uri.setScheme("http"); + return uri; + } + + void startBridge(std::unique_ptr cmd) const override + { + getContext()->addBridgeCommand(std::move(cmd)); + } + + +private: + using Configuration = Poco::Util::AbstractConfiguration; + + Poco::Logger * log; + std::string connection_string; + const Poco::Timespan & http_timeout; + std::string bridge_host; + size_t bridge_port; + + const Configuration & config; + + std::optional quote_style; + std::optional is_schema_allowed; + + +protected: + using URLParams = std::vector>; + + Poco::URI getColumnsInfoURI() const override + { + auto uri = createBaseURI(); + uri.setPath(COL_INFO_HANDLER); + return uri; + } + + URLParams getURLParams(const std::string & cols, UInt64 max_block_size) const override + { + std::vector> result; + + result.emplace_back("connection_string", connection_string); /// already validated + result.emplace_back("columns", cols); + result.emplace_back("max_block_size", std::to_string(max_block_size)); + + return result; + } + + bool isSchemaAllowed() override + { + if (!is_schema_allowed.has_value()) + { + startBridgeSync(); + + auto uri = createBaseURI(); + uri.setPath(SCHEMA_ALLOWED_HANDLER); + uri.addQueryParameter("connection_string", getConnectionString()); + + ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(getContext())); + + bool res; + readBoolText(res, buf); + is_schema_allowed = res; + } + + return *is_schema_allowed; + } + + IdentifierQuotingStyle getIdentifierQuotingStyle() override + { + if (!quote_style.has_value()) + { + startBridgeSync(); + + auto uri = createBaseURI(); + uri.setPath(IDENTIFIER_QUOTE_HANDLER); + uri.addQueryParameter("connection_string", getConnectionString()); + + ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(getContext())); + + std::string character; + readStringBinary(character, buf); + if (character.length() > 1) + throw Exception( + "Failed to parse quoting style from '" + character + "' for service " + BridgeHelperMixin::serviceAlias(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + else if (character.length() == 0) + quote_style = IdentifierQuotingStyle::None; + else if (character[0] == '`') + quote_style = IdentifierQuotingStyle::Backticks; + else if (character[0] == '"') + quote_style = IdentifierQuotingStyle::DoubleQuotes; + else + throw Exception("Can not map quote identifier '" + character + "' to enum value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + return *quote_style; + } +}; + + +struct JDBCBridgeMixin +{ + static constexpr inline auto DEFAULT_PORT = 9019; + + static String configPrefix() + { + return "jdbc_bridge"; + } + + static String serviceAlias() + { + return "clickhouse-jdbc-bridge"; + } + + static String getName() + { + return "JDBC"; + } + + static AccessType getSourceAccessType() + { + return AccessType::JDBC; + } + + static bool startBridgeManually() + { + return true; + } +}; + + +struct ODBCBridgeMixin +{ + static constexpr inline auto DEFAULT_PORT = 9018; + + static String configPrefix() + { + return "odbc_bridge"; + } + + static String serviceAlias() + { + return "clickhouse-odbc-bridge"; + } + + static String getName() + { + return "ODBC"; + } + + static AccessType getSourceAccessType() + { + return AccessType::ODBC; + } + + static bool startBridgeManually() + { + return false; + } +}; + +} diff --git a/src/Bridge/ya.make b/src/Bridge/ya.make new file mode 100644 index 00000000000..e900bab036e --- /dev/null +++ b/src/Bridge/ya.make @@ -0,0 +1,17 @@ +# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. +OWNER(g:clickhouse) + +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + + +SRCS( + IBridgeHelper.cpp + LibraryBridgeHelper.cpp + +) + +END() diff --git a/src/Bridge/ya.make.in b/src/Bridge/ya.make.in new file mode 100644 index 00000000000..01edf8dca82 --- /dev/null +++ b/src/Bridge/ya.make.in @@ -0,0 +1,14 @@ +OWNER(g:clickhouse) + +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + + +SRCS( + +) + +END() diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index a41ddada5a9..43f6ae8fea5 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -190,6 +190,7 @@ add_object_library(clickhouse_storages_distributed Storages/Distributed) add_object_library(clickhouse_storages_mergetree Storages/MergeTree) add_object_library(clickhouse_storages_liveview Storages/LiveView) add_object_library(clickhouse_client Client) +add_object_library(clickhouse_bridge Bridge) add_object_library(clickhouse_server Server) add_object_library(clickhouse_server_http Server/HTTP) add_object_library(clickhouse_formats Formats) @@ -507,5 +508,10 @@ if (ENABLE_TESTS AND USE_GTEST) clickhouse_common_zookeeper string_utils) + # For __udivmodti4 referenced in Core/tests/gtest_DecimalFunctions.cpp + if (OS_DARWIN AND COMPILER_GCC) + target_link_libraries(unit_tests_dbms PRIVATE gcc) + endif () + add_check(unit_tests_dbms) endif () diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 7e3b9b7ab61..65ed956a60b 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -21,6 +21,7 @@ #include #include +#include #include @@ -52,8 +53,6 @@ class Connection; using ConnectionPtr = std::shared_ptr; using Connections = std::vector; -using Scalars = std::map; - /// Packet that could be received from server. struct Packet @@ -111,7 +110,7 @@ public: setDescription(); } - virtual ~Connection() {} + virtual ~Connection() = default; /// Set throttler of network traffic. One throttler could be used for multiple connections to limit total traffic. void setThrottler(const ThrottlerPtr & throttler_) diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 695b2e34fd6..b0c5a3d22f5 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -55,7 +55,7 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode { size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1; - size_t max_entries; + size_t max_entries = 1; switch (pool_mode) { case PoolMode::GET_ALL: diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index df7b3cce729..8178802f3bd 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -162,7 +162,7 @@ MutableColumnPtr ColumnAggregateFunction::convertToValues(MutableColumnPtr colum return res; } -MutableColumnPtr ColumnAggregateFunction::predictValues(const ColumnsWithTypeAndName & arguments, const Context & context) const +MutableColumnPtr ColumnAggregateFunction::predictValues(const ColumnsWithTypeAndName & arguments, ContextPtr context) const { MutableColumnPtr res = func->getReturnTypeToPredict()->createColumn(); res->reserve(data.size()); diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index 5cb9aaa4ad5..8eb1a04b174 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -82,7 +82,7 @@ private: /// Name of the type to distinguish different aggregation states. String type_string; - ColumnAggregateFunction() {} + ColumnAggregateFunction() = default; /// Create a new column that has another column as a source. MutablePtr createView() const; @@ -119,7 +119,7 @@ public: const char * getFamilyName() const override { return "AggregateFunction"; } TypeIndex getDataType() const override { return TypeIndex::AggregateFunction; } - MutableColumnPtr predictValues(const ColumnsWithTypeAndName & arguments, const Context & context) const; + MutableColumnPtr predictValues(const ColumnsWithTypeAndName & arguments, ContextPtr context) const; size_t size() const override { diff --git a/src/Common/Arena.h b/src/Common/Arena.h index 0f4f0420c38..4d14c15197d 100644 --- a/src/Common/Arena.h +++ b/src/Common/Arena.h @@ -128,7 +128,7 @@ private: template friend class AlignedArenaAllocator; public: - Arena(size_t initial_size_ = 4096, size_t growth_factor_ = 2, size_t linear_growth_threshold_ = 128 * 1024 * 1024) + explicit Arena(size_t initial_size_ = 4096, size_t growth_factor_ = 2, size_t linear_growth_threshold_ = 128 * 1024 * 1024) : growth_factor(growth_factor_), linear_growth_threshold(linear_growth_threshold_), head(new MemoryChunk(initial_size_, nullptr)), size_in_bytes(head->size()), page_size(static_cast(::getPageSize())) @@ -160,7 +160,7 @@ public: void * head_pos = head->pos; size_t space = head->end - head->pos; - auto res = static_cast(std::align(alignment, size, head_pos, space)); + auto * res = static_cast(std::align(alignment, size, head_pos, space)); if (res) { head->pos = static_cast(head_pos); diff --git a/src/Common/Config/AbstractConfigurationComparison.cpp b/src/Common/Config/AbstractConfigurationComparison.cpp index 59c0c895a89..eb677debb02 100644 --- a/src/Common/Config/AbstractConfigurationComparison.cpp +++ b/src/Common/Config/AbstractConfigurationComparison.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -31,6 +32,23 @@ bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const P return isSameConfiguration(left, key, right, key); } +bool isSameConfigurationWithMultipleKeys(const Poco::Util::AbstractConfiguration & left, const Poco::Util::AbstractConfiguration & right, const String & root, const String & name) +{ + if (&left == &right) + return true; + + auto left_multiple_keys = getMultipleKeysFromConfig(left, root, name); + auto right_multiple_keys = getMultipleKeysFromConfig(right, root, name); + if (left_multiple_keys.size() != right_multiple_keys.size()) + return false; + + for (auto & key : left_multiple_keys) + if (!isSameConfiguration(left, right, concatKeyAndSubKey(root, key))) + return false; + + return true; +} + bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const String & left_key, const Poco::Util::AbstractConfiguration & right, const String & right_key) { diff --git a/src/Common/Config/AbstractConfigurationComparison.h b/src/Common/Config/AbstractConfigurationComparison.h index 795fca2af8e..6e1d8a890bb 100644 --- a/src/Common/Config/AbstractConfigurationComparison.h +++ b/src/Common/Config/AbstractConfigurationComparison.h @@ -13,6 +13,17 @@ namespace DB bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const Poco::Util::AbstractConfiguration & right); + /// Config may have multiple keys with one name. For example: + /// + /// ... + /// ... + /// + /// Returns true if the specified subview of the two configurations contains + /// the same keys and values for each key with the given name. + bool isSameConfigurationWithMultipleKeys(const Poco::Util::AbstractConfiguration & left, + const Poco::Util::AbstractConfiguration & right, + const String & root, const String & name); + /// Returns true if the specified subview of the two configurations contains the same keys and values. bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const Poco::Util::AbstractConfiguration & right, diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index 7ab57ea7fab..069f9cf2af7 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -1,11 +1,12 @@ #pragma once +#include +#include +#include + #include #include -#include -#include - namespace ProfileEvents { @@ -18,7 +19,6 @@ class MemoryTracker; namespace DB { -class Context; class QueryStatus; struct Progress; class InternalTextLogsQueue; @@ -87,7 +87,7 @@ public: /// Initializes query with current thread as master thread in constructor, and detaches it in destructor struct QueryScope { - explicit QueryScope(Context & query_context); + explicit QueryScope(ContextPtr query_context); ~QueryScope(); void logPeakMemoryUsage(); @@ -99,7 +99,7 @@ private: /// Sets query_context for current thread group /// Can by used only through QueryScope - static void attachQueryContext(Context & query_context); + static void attachQueryContext(ContextPtr query_context); }; } diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 0f85ad5c792..ad0463db889 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -548,6 +548,7 @@ M(578, INVALID_FORMAT_INSERT_QUERY_WITH_DATA) \ M(579, INCORRECT_PART_TYPE) \ M(580, CANNOT_SET_ROUNDING_MODE) \ + M(581, TOO_LARGE_DISTRIBUTED_DEPTH) \ \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ @@ -590,7 +591,7 @@ namespace ErrorCodes void increment(ErrorCode error_code, bool remote, const std::string & message, const FramePointers & trace) { - if (error_code >= end()) + if (error_code < 0 || error_code >= end()) { /// For everything outside the range, use END. /// (end() is the pointer pass the end, while END is the last value that has an element in values array). diff --git a/src/Common/FieldVisitors.h b/src/Common/FieldVisitors.h index 954cd004e84..5adbf2b01ac 100644 --- a/src/Common/FieldVisitors.h +++ b/src/Common/FieldVisitors.h @@ -185,12 +185,20 @@ public: /// Conversion of infinite values to integer is undefined. throw Exception("Cannot convert infinite value to integer type", ErrorCodes::CANNOT_CONVERT_TYPE); } + else if (x > std::numeric_limits::max() || x < std::numeric_limits::lowest()) + { + throw Exception("Cannot convert out of range floating point value to integer type", ErrorCodes::CANNOT_CONVERT_TYPE); + } } if constexpr (std::is_same_v) + { return Int256(x); + } else + { return T(x); + } } T operator() (const UInt128 &) const diff --git a/src/Common/HashTable/HashMap.h b/src/Common/HashTable/HashMap.h index c3cd09eccb2..50324fe64c7 100644 --- a/src/Common/HashTable/HashMap.h +++ b/src/Common/HashTable/HashMap.h @@ -24,17 +24,26 @@ struct PairNoInit PairNoInit() {} - template - PairNoInit(First_ && first_, NoInitTag) : first(std::forward(first_)) + template + PairNoInit(FirstValue && first_, NoInitTag) + : first(std::forward(first_)) { } - template - PairNoInit(First_ && first_, Second_ && second_) : first(std::forward(first_)), second(std::forward(second_)) + template + PairNoInit(FirstValue && first_, SecondValue && second_) + : first(std::forward(first_)) + , second(std::forward(second_)) { } }; +template +PairNoInit, std::decay_t> makePairNoInit(First && first, Second && second) +{ + return PairNoInit, std::decay_t>(std::forward(first), std::forward(second)); +} + template struct HashMapCell diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 35703790d82..d0876c5e69c 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -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, "") \ diff --git a/src/Common/StackTrace.h b/src/Common/StackTrace.h index 35df77b6b8a..ef90a0d587d 100644 --- a/src/Common/StackTrace.h +++ b/src/Common/StackTrace.h @@ -11,7 +11,9 @@ #ifdef __APPLE__ // ucontext is not available without _XOPEN_SOURCE -# pragma clang diagnostic ignored "-Wreserved-id-macro" +# ifdef __clang__ +# pragma clang diagnostic ignored "-Wreserved-id-macro" +# endif # define _XOPEN_SOURCE 700 #endif #include diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 8c01ed2d46f..1493d30ea01 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -101,7 +101,8 @@ ThreadStatus::~ThreadStatus() #if !defined(ARCADIA_BUILD) /// It may cause segfault if query_context was destroyed, but was not detached - assert((!query_context && query_id.empty()) || (query_context && query_id == query_context->getCurrentQueryId())); + auto query_context_ptr = query_context.lock(); + assert((!query_context_ptr && query_id.empty()) || (query_context_ptr && query_id == query_context_ptr->getCurrentQueryId())); #endif if (deleter) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index dc5f09c5f3d..3b39e462fa6 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -1,20 +1,20 @@ #pragma once -#include -#include +#include +#include +#include #include #include +#include +#include -#include +#include -#include - -#include +#include #include +#include #include #include -#include -#include namespace Poco @@ -26,7 +26,6 @@ namespace Poco namespace DB { -class Context; class QueryStatus; class ThreadStatus; class QueryProfilerReal; @@ -58,8 +57,8 @@ public: ProfileEvents::Counters performance_counters{VariableContext::Process}; MemoryTracker memory_tracker{VariableContext::Process}; - Context * query_context = nullptr; - Context * global_context = nullptr; + ContextWeakPtr query_context; + ContextWeakPtr global_context; InternalTextLogsQueueWeakPtr logs_queue_ptr; std::function fatal_error_callback; @@ -122,9 +121,9 @@ protected: std::atomic thread_state{ThreadState::DetachedFromQuery}; /// Is set once - Context * global_context = nullptr; + ContextWeakPtr global_context; /// Use it only from current thread - Context * query_context = nullptr; + ContextWeakPtr query_context; String query_id; @@ -178,9 +177,9 @@ public: return query_id; } - const Context * getQueryContext() const + auto getQueryContext() const { - return query_context; + return query_context.lock(); } /// Starts new query and create new thread group for it, current thread becomes master thread of the query @@ -203,7 +202,7 @@ public: /// Sets query context for current master thread and its thread group /// NOTE: query_context have to be alive until detachQuery() is called - void attachQueryContext(Context & query_context); + void attachQueryContext(ContextPtr query_context); /// Update several ProfileEvents counters void updatePerformanceCounters(); diff --git a/src/Common/XDBCBridgeHelper.h b/src/Common/XDBCBridgeHelper.h deleted file mode 100644 index 1e552157a56..00000000000 --- a/src/Common/XDBCBridgeHelper.h +++ /dev/null @@ -1,351 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(ARCADIA_BUILD) -# include -#endif - -namespace DB -{ -namespace ErrorCodes -{ - extern const int EXTERNAL_SERVER_IS_NOT_RESPONDING; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; -} - -/** - * Class for Helpers for Xdbc-bridges, provide utility methods, not main request - */ -class IXDBCBridgeHelper -{ -public: - static constexpr inline auto DEFAULT_FORMAT = "RowBinary"; - - virtual std::vector> getURLParams(const std::string & cols, UInt64 max_block_size) const = 0; - virtual void startBridgeSync() const = 0; - virtual Poco::URI getMainURI() const = 0; - virtual Poco::URI getColumnsInfoURI() const = 0; - virtual IdentifierQuotingStyle getIdentifierQuotingStyle() = 0; - virtual bool isSchemaAllowed() = 0; - virtual String getName() const = 0; - - virtual ~IXDBCBridgeHelper() = default; -}; - -using BridgeHelperPtr = std::shared_ptr; - -template -class XDBCBridgeHelper : public IXDBCBridgeHelper -{ -private: - Poco::Timespan http_timeout; - - std::string connection_string; - - Poco::URI ping_url; - - Poco::Logger * log = &Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper"); - - std::optional quote_style; - std::optional is_schema_allowed; - -protected: - auto getConnectionString() const - { - return connection_string; - } - -public: - using Configuration = Poco::Util::AbstractConfiguration; - - const Context & context; - const Configuration & config; - - static constexpr inline auto DEFAULT_HOST = "127.0.0.1"; - static constexpr inline auto DEFAULT_PORT = BridgeHelperMixin::DEFAULT_PORT; - static constexpr inline auto PING_HANDLER = "/ping"; - static constexpr inline auto MAIN_HANDLER = "/"; - static constexpr inline auto COL_INFO_HANDLER = "/columns_info"; - static constexpr inline auto IDENTIFIER_QUOTE_HANDLER = "/identifier_quote"; - static constexpr inline auto SCHEMA_ALLOWED_HANDLER = "/schema_allowed"; - static constexpr inline auto PING_OK_ANSWER = "Ok."; - - XDBCBridgeHelper(const Context & global_context_, const Poco::Timespan & http_timeout_, const std::string & connection_string_) - : http_timeout(http_timeout_), connection_string(connection_string_), context(global_context_), config(context.getConfigRef()) - { - size_t bridge_port = config.getUInt(BridgeHelperMixin::configPrefix() + ".port", DEFAULT_PORT); - std::string bridge_host = config.getString(BridgeHelperMixin::configPrefix() + ".host", DEFAULT_HOST); - - ping_url.setHost(bridge_host); - ping_url.setPort(bridge_port); - ping_url.setScheme("http"); - ping_url.setPath(PING_HANDLER); - } - - String getName() const override - { - return BridgeHelperMixin::getName(); - } - - IdentifierQuotingStyle getIdentifierQuotingStyle() override - { - if (!quote_style.has_value()) - { - startBridgeSync(); - - auto uri = createBaseURI(); - uri.setPath(IDENTIFIER_QUOTE_HANDLER); - uri.addQueryParameter("connection_string", getConnectionString()); - - ReadWriteBufferFromHTTP buf( - uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context)); - std::string character; - readStringBinary(character, buf); - if (character.length() > 1) - throw Exception("Failed to parse quoting style from '" + character + "' for service " + BridgeHelperMixin::serviceAlias(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - else if (character.length() == 0) - quote_style = IdentifierQuotingStyle::None; - else if (character[0] == '`') - quote_style = IdentifierQuotingStyle::Backticks; - else if (character[0] == '"') - quote_style = IdentifierQuotingStyle::DoubleQuotes; - else - throw Exception("Can not map quote identifier '" + character + "' to enum value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - return *quote_style; - } - - bool isSchemaAllowed() override - { - if (!is_schema_allowed.has_value()) - { - startBridgeSync(); - - auto uri = createBaseURI(); - uri.setPath(SCHEMA_ALLOWED_HANDLER); - uri.addQueryParameter("connection_string", getConnectionString()); - - ReadWriteBufferFromHTTP buf( - uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context)); - - bool res; - readBoolText(res, buf); - is_schema_allowed = res; - } - - return *is_schema_allowed; - } - - /** - * @todo leaky abstraction - used by external API's - */ - std::vector> getURLParams(const std::string & cols, UInt64 max_block_size) const override - { - std::vector> result; - - result.emplace_back("connection_string", connection_string); /// already validated - result.emplace_back("columns", cols); - result.emplace_back("max_block_size", std::to_string(max_block_size)); - - return result; - } - - /** - * Performs spawn of external daemon - */ - void startBridgeSync() const override - { - if (!checkBridgeIsRunning()) - { - LOG_TRACE(log, "{} is not running, will try to start it", BridgeHelperMixin::serviceAlias()); - startBridge(); - bool started = false; - - uint64_t milliseconds_to_wait = 10; /// Exponential backoff - uint64_t counter = 0; - while (milliseconds_to_wait < 10000) - { - ++counter; - LOG_TRACE(log, "Checking {} is running, try {}", BridgeHelperMixin::serviceAlias(), counter); - if (checkBridgeIsRunning()) - { - started = true; - break; - } - std::this_thread::sleep_for(std::chrono::milliseconds(milliseconds_to_wait)); - milliseconds_to_wait *= 2; - } - - if (!started) - throw Exception(BridgeHelperMixin::getName() + "BridgeHelper: " + BridgeHelperMixin::serviceAlias() + " is not responding", - ErrorCodes::EXTERNAL_SERVER_IS_NOT_RESPONDING); - } - } - - /** - * URI to fetch the data from external service - */ - Poco::URI getMainURI() const override - { - auto uri = createBaseURI(); - uri.setPath(MAIN_HANDLER); - return uri; - } - - /** - * URI to retrieve column description from external service - */ - Poco::URI getColumnsInfoURI() const override - { - auto uri = createBaseURI(); - uri.setPath(COL_INFO_HANDLER); - return uri; - } - -protected: - Poco::URI createBaseURI() const - { - Poco::URI uri; - uri.setHost(ping_url.getHost()); - uri.setPort(ping_url.getPort()); - uri.setScheme("http"); - return uri; - } - -private: - bool checkBridgeIsRunning() const - { - try - { - ReadWriteBufferFromHTTP buf( - ping_url, Poco::Net::HTTPRequest::HTTP_GET, {}, ConnectionTimeouts::getHTTPTimeouts(context)); - return checkString(XDBCBridgeHelper::PING_OK_ANSWER, buf); - } - catch (...) - { - return false; - } - } - - /* Contains logic for instantiation of the bridge instance */ - void startBridge() const - { - auto cmd = BridgeHelperMixin::startBridge(config, log, http_timeout); - context.addXDBCBridgeCommand(std::move(cmd)); - } -}; - -struct JDBCBridgeMixin -{ - static constexpr inline auto DEFAULT_PORT = 9019; - static const String configPrefix() - { - return "jdbc_bridge"; - } - static const String serviceAlias() - { - return "clickhouse-jdbc-bridge"; - } - static const String getName() - { - return "JDBC"; - } - static AccessType getSourceAccessType() - { - return AccessType::JDBC; - } - - static std::unique_ptr startBridge(const Poco::Util::AbstractConfiguration &, const Poco::Logger *, const Poco::Timespan &) - { - throw Exception("jdbc-bridge is not running. Please, start it manually", ErrorCodes::EXTERNAL_SERVER_IS_NOT_RESPONDING); - } -}; - -struct ODBCBridgeMixin -{ - static constexpr inline auto DEFAULT_PORT = 9018; - - static const String configPrefix() - { - return "odbc_bridge"; - } - static const String serviceAlias() - { - return "clickhouse-odbc-bridge"; - } - static const String getName() - { - return "ODBC"; - } - static AccessType getSourceAccessType() - { - return AccessType::ODBC; - } - - static std::unique_ptr startBridge( - const Poco::Util::AbstractConfiguration & config, Poco::Logger * log, const Poco::Timespan & http_timeout) - { - /// Path to executable folder - Poco::Path path{config.getString("application.dir", "/usr/bin")}; - - std::vector cmd_args; - path.setFileName("clickhouse-odbc-bridge"); - -#if !CLICKHOUSE_SPLIT_BINARY - cmd_args.push_back("odbc-bridge"); -#endif - - cmd_args.push_back("--http-port"); - cmd_args.push_back(std::to_string(config.getUInt(configPrefix() + ".port", DEFAULT_PORT))); - cmd_args.push_back("--listen-host"); - cmd_args.push_back(config.getString(configPrefix() + ".listen_host", XDBCBridgeHelper::DEFAULT_HOST)); - cmd_args.push_back("--http-timeout"); - cmd_args.push_back(std::to_string(http_timeout.totalMicroseconds())); - if (config.has("logger." + configPrefix() + "_log")) - { - cmd_args.push_back("--log-path"); - cmd_args.push_back(config.getString("logger." + configPrefix() + "_log")); - } - if (config.has("logger." + configPrefix() + "_errlog")) - { - cmd_args.push_back("--err-log-path"); - cmd_args.push_back(config.getString("logger." + configPrefix() + "_errlog")); - } - if (config.has("logger." + configPrefix() + "_stdout")) - { - cmd_args.push_back("--stdout-path"); - cmd_args.push_back(config.getString("logger." + configPrefix() + "_stdout")); - } - if (config.has("logger." + configPrefix() + "_stderr")) - { - cmd_args.push_back("--stderr-path"); - cmd_args.push_back(config.getString("logger." + configPrefix() + "_stderr")); - } - if (config.has("logger." + configPrefix() + "_level")) - { - cmd_args.push_back("--log-level"); - cmd_args.push_back(config.getString("logger." + configPrefix() + "_level")); - } - - LOG_TRACE(log, "Starting {}", serviceAlias()); - - return ShellCommand::executeDirect(path.toString(), cmd_args, ShellCommandDestructorStrategy(true)); - } -}; -} diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 330985e1599..9f59da233fc 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -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); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 2314139af69..93ecaef8365 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -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; } diff --git a/src/Common/formatReadable.cpp b/src/Common/formatReadable.cpp index fc5c6c19b50..fd8ff9abd86 100644 --- a/src/Common/formatReadable.cpp +++ b/src/Common/formatReadable.cpp @@ -13,7 +13,10 @@ namespace DB } } -static void formatReadable(double size, DB::WriteBuffer & out, int precision, const char ** units, size_t units_size, double delimiter) +// I wanted to make this ALWAYS_INLINE to prevent flappy performance tests, +// but GCC complains it may not be inlined. +static void formatReadable(double size, DB::WriteBuffer & out, + int precision, const char ** units, size_t units_size, double delimiter) { size_t i = 0; for (; i + 1 < units_size && fabs(size) >= delimiter; ++i) diff --git a/src/Common/parseRemoteDescription.cpp b/src/Common/parseRemoteDescription.cpp index cc89af26d99..6bb1186bfea 100644 --- a/src/Common/parseRemoteDescription.cpp +++ b/src/Common/parseRemoteDescription.cpp @@ -1,6 +1,9 @@ #include "parseRemoteDescription.h" #include #include +#include +#include + namespace DB { @@ -167,4 +170,27 @@ std::vector parseRemoteDescription(const String & description, size_t l, return res; } + +std::vector> parseRemoteDescriptionForExternalDatabase(const String & description, size_t max_addresses, UInt16 default_port) +{ + auto addresses = parseRemoteDescription(description, 0, description.size(), '|', max_addresses); + std::vector> result; + + for (const auto & address : addresses) + { + size_t colon = address.find(':'); + if (colon == String::npos) + { + LOG_WARNING(&Poco::Logger::get("ParseRemoteDescription"), "Port is not found for host: {}. Using default port {}", default_port); + result.emplace_back(std::make_pair(address, default_port)); + } + else + { + result.emplace_back(std::make_pair(address.substr(0, colon), DB::parseFromString(address.substr(colon + 1)))); + } + } + + return result; +} + } diff --git a/src/Common/parseRemoteDescription.h b/src/Common/parseRemoteDescription.h index 6ba0bb4737f..f691a57dc73 100644 --- a/src/Common/parseRemoteDescription.h +++ b/src/Common/parseRemoteDescription.h @@ -17,4 +17,7 @@ namespace DB */ std::vector parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses); +/// Parse remote description for external database (MySQL or PostgreSQL). +std::vector> parseRemoteDescriptionForExternalDatabase(const String & description, size_t max_addresses, UInt16 default_port); + } diff --git a/src/Common/tests/gtest_global_context.h b/src/Common/tests/gtest_global_context.h index b6529f09b46..30ebf1dbca9 100644 --- a/src/Common/tests/gtest_global_context.h +++ b/src/Common/tests/gtest_global_context.h @@ -5,14 +5,14 @@ struct ContextHolder { DB::SharedContextHolder shared_context; - DB::Context context; + DB::ContextPtr context; ContextHolder() : shared_context(DB::Context::createShared()) , context(DB::Context::createGlobal(shared_context.get())) { - context.makeGlobalContext(); - context.setPath("./"); + context->makeGlobalContext(); + context->setPath("./"); } ContextHolder(ContextHolder &&) = default; diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 227192fdf8a..7fc24de1cb9 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -44,8 +44,8 @@ ChangelogFileDescription getChangelogFileDescription(const std::string & path_st ChangelogFileDescription result; result.prefix = filename_parts[0]; - result.from_log_index = parse(filename_parts[1]); - result.to_log_index = parse(filename_parts[2]); + result.from_log_index = parse(filename_parts[1]); + result.to_log_index = parse(filename_parts[2]); result.path = path_str; return result; } @@ -73,7 +73,7 @@ Checksum computeRecordChecksum(const ChangelogRecord & record) class ChangelogWriter { public: - ChangelogWriter(const std::string & filepath_, WriteMode mode, size_t start_index_) + ChangelogWriter(const std::string & filepath_, WriteMode mode, uint64_t start_index_) : filepath(filepath_) , plain_buf(filepath, DBMS_DEFAULT_BUFFER_SIZE, mode == WriteMode::Rewrite ? -1 : (O_APPEND | O_CREAT | O_WRONLY)) , start_index(start_index_) @@ -115,22 +115,22 @@ public: plain_buf.sync(); } - size_t getEntriesWritten() const + uint64_t getEntriesWritten() const { return entries_written; } - void setEntriesWritten(size_t entries_written_) + void setEntriesWritten(uint64_t entries_written_) { entries_written = entries_written_; } - size_t getStartIndex() const + uint64_t getStartIndex() const { return start_index; } - void setStartIndex(size_t start_index_) + void setStartIndex(uint64_t start_index_) { start_index = start_index_; } @@ -138,14 +138,14 @@ public: private: std::string filepath; WriteBufferFromFile plain_buf; - size_t entries_written = 0; - size_t start_index; + uint64_t entries_written = 0; + uint64_t start_index; }; struct ChangelogReadResult { - size_t entries_read; - size_t first_read_index; + uint64_t entries_read; + uint64_t first_read_index; off_t last_position; bool error; }; @@ -158,9 +158,9 @@ public: , read_buf(filepath) {} - ChangelogReadResult readChangelog(IndexToLogEntry & logs, size_t start_log_index, IndexToOffset & index_to_offset, Poco::Logger * log) + ChangelogReadResult readChangelog(IndexToLogEntry & logs, uint64_t start_log_index, IndexToOffset & index_to_offset, Poco::Logger * log) { - size_t previous_index = 0; + uint64_t previous_index = 0; ChangelogReadResult result{}; try { @@ -247,7 +247,7 @@ private: ReadBufferFromFile read_buf; }; -Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval_, Poco::Logger * log_) +Changelog::Changelog(const std::string & changelogs_dir_, uint64_t rotate_interval_, Poco::Logger * log_) : changelogs_dir(changelogs_dir_) , rotate_interval(rotate_interval_) , log(log_) @@ -263,15 +263,15 @@ Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval } } -void Changelog::readChangelogAndInitWriter(size_t last_commited_log_index, size_t logs_to_keep) +void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep) { - size_t total_read = 0; - size_t entries_in_last = 0; - size_t incomplete_log_index = 0; + uint64_t total_read = 0; + uint64_t entries_in_last = 0; + uint64_t incomplete_log_index = 0; ChangelogReadResult result{}; - size_t first_read_index = 0; + uint64_t first_read_index = 0; - size_t start_to_read_from = last_commited_log_index; + uint64_t start_to_read_from = last_commited_log_index; if (start_to_read_from > logs_to_keep) start_to_read_from -= logs_to_keep; else @@ -355,12 +355,8 @@ void Changelog::readChangelogAndInitWriter(size_t last_commited_log_index, size_ rotate(start_index + total_read); } -void Changelog::rotate(size_t new_start_log_index) +void Changelog::rotate(uint64_t new_start_log_index) { - //// doesn't exist on init - if (current_writer) - current_writer->flush(); - ChangelogFileDescription new_description; new_description.prefix = DEFAULT_PREFIX; new_description.from_log_index = new_start_log_index; @@ -373,7 +369,7 @@ void Changelog::rotate(size_t new_start_log_index) current_writer = std::make_unique(new_description.path, WriteMode::Rewrite, new_start_log_index); } -ChangelogRecord Changelog::buildRecord(size_t index, const LogEntryPtr & log_entry) +ChangelogRecord Changelog::buildRecord(uint64_t index, const LogEntryPtr & log_entry) { ChangelogRecord record; record.header.version = ChangelogVersion::V0; @@ -391,7 +387,7 @@ ChangelogRecord Changelog::buildRecord(size_t index, const LogEntryPtr & log_ent return record; } -void Changelog::appendEntry(size_t index, const LogEntryPtr & log_entry, bool force_sync) +void Changelog::appendEntry(uint64_t index, const LogEntryPtr & log_entry, bool force_sync) { if (!current_writer) throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before appending records"); @@ -409,7 +405,7 @@ void Changelog::appendEntry(size_t index, const LogEntryPtr & log_entry, bool fo logs[index] = makeClone(log_entry); } -void Changelog::writeAt(size_t index, const LogEntryPtr & log_entry, bool force_sync) +void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry, bool force_sync) { if (index_to_start_pos.count(index) == 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot write at index {} because changelog doesn't contain it", index); @@ -443,7 +439,7 @@ void Changelog::writeAt(size_t index, const LogEntryPtr & log_entry, bool force_ } /// Remove redundant logs from memory - for (size_t i = index; ; ++i) + for (uint64_t i = index; ; ++i) { auto log_itr = logs.find(i); if (log_itr == logs.end()) @@ -458,7 +454,7 @@ void Changelog::writeAt(size_t index, const LogEntryPtr & log_entry, bool force_ appendEntry(index, log_entry, force_sync); } -void Changelog::compact(size_t up_to_log_index) +void Changelog::compact(uint64_t up_to_log_index) { for (auto itr = existing_changelogs.begin(); itr != existing_changelogs.end();) { @@ -480,9 +476,9 @@ void Changelog::compact(size_t up_to_log_index) LogEntryPtr Changelog::getLastEntry() const { - static LogEntryPtr fake_entry = nuraft::cs_new(0, nuraft::buffer::alloc(sizeof(size_t))); + static LogEntryPtr fake_entry = nuraft::cs_new(0, nuraft::buffer::alloc(sizeof(uint64_t))); - size_t next_index = getNextEntryIndex() - 1; + uint64_t next_index = getNextEntryIndex() - 1; auto entry = logs.find(next_index); if (entry == logs.end()) return fake_entry; @@ -490,13 +486,13 @@ LogEntryPtr Changelog::getLastEntry() const return entry->second; } -LogEntriesPtr Changelog::getLogEntriesBetween(size_t start, size_t end) +LogEntriesPtr Changelog::getLogEntriesBetween(uint64_t start, uint64_t end) { LogEntriesPtr ret = nuraft::cs_new>>(); ret->resize(end - start); - size_t result_pos = 0; - for (size_t i = start; i < end; ++i) + uint64_t result_pos = 0; + for (uint64_t i = start; i < end; ++i) { (*ret)[result_pos] = entryAt(i); result_pos++; @@ -504,7 +500,7 @@ LogEntriesPtr Changelog::getLogEntriesBetween(size_t start, size_t end) return ret; } -LogEntryPtr Changelog::entryAt(size_t index) +LogEntryPtr Changelog::entryAt(uint64_t index) { nuraft::ptr src = nullptr; auto entry = logs.find(index); @@ -515,12 +511,12 @@ LogEntryPtr Changelog::entryAt(size_t index) return src; } -nuraft::ptr Changelog::serializeEntriesToBuffer(size_t index, int32_t count) +nuraft::ptr Changelog::serializeEntriesToBuffer(uint64_t index, int32_t count) { std::vector> returned_logs; - size_t size_total = 0; - for (size_t i = index; i < index + count; ++i) + uint64_t size_total = 0; + for (uint64_t i = index; i < index + count; ++i) { auto entry = logs.find(i); if (entry == logs.end()) @@ -544,14 +540,14 @@ nuraft::ptr Changelog::serializeEntriesToBuffer(size_t index, in return buf_out; } -void Changelog::applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer, bool force_sync) +void Changelog::applyEntriesFromBuffer(uint64_t index, nuraft::buffer & buffer, bool force_sync) { buffer.pos(0); int num_logs = buffer.get_int(); for (int i = 0; i < num_logs; ++i) { - size_t cur_index = index + i; + uint64_t cur_index = index + i; int buf_size = buffer.get_int(); nuraft::ptr buf_local = nuraft::buffer::alloc(buf_size); diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 24b98d6b892..21d96cd1438 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -17,8 +17,8 @@ using LogEntries = std::vector; using LogEntriesPtr = nuraft::ptr; using BufferPtr = nuraft::ptr; -using IndexToOffset = std::unordered_map; -using IndexToLogEntry = std::unordered_map; +using IndexToOffset = std::unordered_map; +using IndexToLogEntry = std::unordered_map; enum class ChangelogVersion : uint8_t { @@ -30,10 +30,10 @@ static constexpr auto CURRENT_CHANGELOG_VERSION = ChangelogVersion::V0; struct ChangelogRecordHeader { ChangelogVersion version = CURRENT_CHANGELOG_VERSION; - size_t index; /// entry log number - size_t term; + uint64_t index; /// entry log number + uint64_t term; nuraft::log_val_type value_type; - size_t blob_size; + uint64_t blob_size; }; /// Changelog record on disk @@ -48,8 +48,8 @@ struct ChangelogRecord struct ChangelogFileDescription { std::string prefix; - size_t from_log_index; - size_t to_log_index; + uint64_t from_log_index; + uint64_t to_log_index; std::string path; }; @@ -63,27 +63,27 @@ class Changelog { public: - Changelog(const std::string & changelogs_dir_, size_t rotate_interval_, Poco::Logger * log_); + Changelog(const std::string & changelogs_dir_, uint64_t rotate_interval_, Poco::Logger * log_); /// Read changelog from files on changelogs_dir_ skipping all entries before from_log_index /// Truncate broken entries, remove files after broken entries. - void readChangelogAndInitWriter(size_t last_commited_log_index, size_t logs_to_keep); + void readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep); /// Add entry to log with index. Call fsync if force_sync true. - void appendEntry(size_t index, const LogEntryPtr & log_entry, bool force_sync); + void appendEntry(uint64_t index, const LogEntryPtr & log_entry, bool force_sync); /// Write entry at index and truncate all subsequent entries. - void writeAt(size_t index, const LogEntryPtr & log_entry, bool force_sync); + void writeAt(uint64_t index, const LogEntryPtr & log_entry, bool force_sync); /// Remove log files with to_log_index <= up_to_log_index. - void compact(size_t up_to_log_index); + void compact(uint64_t up_to_log_index); - size_t getNextEntryIndex() const + uint64_t getNextEntryIndex() const { return start_index + logs.size(); } - size_t getStartIndex() const + uint64_t getStartIndex() const { return start_index; } @@ -92,21 +92,21 @@ public: LogEntryPtr getLastEntry() const; /// Return log entries between [start, end) - LogEntriesPtr getLogEntriesBetween(size_t start_index, size_t end_index); + LogEntriesPtr getLogEntriesBetween(uint64_t start_index, uint64_t end_index); /// Return entry at position index - LogEntryPtr entryAt(size_t index); + LogEntryPtr entryAt(uint64_t index); /// Serialize entries from index into buffer - BufferPtr serializeEntriesToBuffer(size_t index, int32_t count); + BufferPtr serializeEntriesToBuffer(uint64_t index, int32_t count); /// Apply entries from buffer overriding existing entries - void applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer, bool force_sync); + void applyEntriesFromBuffer(uint64_t index, nuraft::buffer & buffer, bool force_sync); /// Fsync log to disk void flush(); - size_t size() const + uint64_t size() const { return logs.size(); } @@ -116,21 +116,21 @@ public: private: /// Pack log_entry into changelog record - static ChangelogRecord buildRecord(size_t index, const LogEntryPtr & log_entry); + static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry); /// Starts new file [new_start_log_index, new_start_log_index + rotate_interval] - void rotate(size_t new_start_log_index); + void rotate(uint64_t new_start_log_index); private: const std::string changelogs_dir; - const size_t rotate_interval; + const uint64_t rotate_interval; Poco::Logger * log; - std::map existing_changelogs; + std::map existing_changelogs; std::unique_ptr current_writer; IndexToOffset index_to_start_pos; IndexToLogEntry logs; - size_t start_index = 0; + uint64_t start_index = 0; }; } diff --git a/src/Coordination/InMemoryLogStore.cpp b/src/Coordination/InMemoryLogStore.cpp index 877c8a60a2a..b7bccdf588e 100644 --- a/src/Coordination/InMemoryLogStore.cpp +++ b/src/Coordination/InMemoryLogStore.cpp @@ -16,16 +16,16 @@ ptr makeClone(const ptr & entry) InMemoryLogStore::InMemoryLogStore() : start_idx(1) { - nuraft::ptr buf = nuraft::buffer::alloc(sizeof(size_t)); + nuraft::ptr buf = nuraft::buffer::alloc(sizeof(uint64_t)); logs[0] = nuraft::cs_new(0, buf); } -size_t InMemoryLogStore::start_index() const +uint64_t InMemoryLogStore::start_index() const { return start_idx; } -size_t InMemoryLogStore::next_slot() const +uint64_t InMemoryLogStore::next_slot() const { std::lock_guard l(logs_lock); // Exclude the dummy entry. @@ -34,7 +34,7 @@ size_t InMemoryLogStore::next_slot() const nuraft::ptr InMemoryLogStore::last_entry() const { - size_t next_idx = next_slot(); + uint64_t next_idx = next_slot(); std::lock_guard lock(logs_lock); auto entry = logs.find(next_idx - 1); if (entry == logs.end()) @@ -43,17 +43,17 @@ nuraft::ptr InMemoryLogStore::last_entry() const return makeClone(entry->second); } -size_t InMemoryLogStore::append(nuraft::ptr & entry) +uint64_t InMemoryLogStore::append(nuraft::ptr & entry) { ptr clone = makeClone(entry); std::lock_guard l(logs_lock); - size_t idx = start_idx + logs.size() - 1; + uint64_t idx = start_idx + logs.size() - 1; logs[idx] = clone; return idx; } -void InMemoryLogStore::write_at(size_t index, nuraft::ptr & entry) +void InMemoryLogStore::write_at(uint64_t index, nuraft::ptr & entry) { nuraft::ptr clone = makeClone(entry); @@ -65,14 +65,14 @@ void InMemoryLogStore::write_at(size_t index, nuraft::ptr & e logs[index] = clone; } -nuraft::ptr>> InMemoryLogStore::log_entries(size_t start, size_t end) +nuraft::ptr>> InMemoryLogStore::log_entries(uint64_t start, uint64_t end) { nuraft::ptr>> ret = nuraft::cs_new>>(); ret->resize(end - start); - size_t cc = 0; - for (size_t i = start; i < end; ++i) + uint64_t cc = 0; + for (uint64_t i = start; i < end; ++i) { nuraft::ptr src = nullptr; { @@ -90,7 +90,7 @@ nuraft::ptr>> InMemoryLogStore::log_e return ret; } -nuraft::ptr InMemoryLogStore::entry_at(size_t index) +nuraft::ptr InMemoryLogStore::entry_at(uint64_t index) { nuraft::ptr src = nullptr; { @@ -103,9 +103,9 @@ nuraft::ptr InMemoryLogStore::entry_at(size_t index) return makeClone(src); } -size_t InMemoryLogStore::term_at(size_t index) +uint64_t InMemoryLogStore::term_at(uint64_t index) { - size_t term = 0; + uint64_t term = 0; { std::lock_guard l(logs_lock); auto entry = logs.find(index); @@ -116,12 +116,12 @@ size_t InMemoryLogStore::term_at(size_t index) return term; } -nuraft::ptr InMemoryLogStore::pack(size_t index, Int32 cnt) +nuraft::ptr InMemoryLogStore::pack(uint64_t index, Int32 cnt) { std::vector> returned_logs; - size_t size_total = 0; - for (size_t ii = index; ii < index + cnt; ++ii) + uint64_t uint64_total = 0; + for (uint64_t ii = index; ii < index + cnt; ++ii) { ptr le = nullptr; { @@ -130,11 +130,11 @@ nuraft::ptr InMemoryLogStore::pack(size_t index, Int32 cnt) } assert(le.get()); nuraft::ptr buf = le->serialize(); - size_total += buf->size(); + uint64_total += buf->size(); returned_logs.push_back(buf); } - nuraft::ptr buf_out = nuraft::buffer::alloc(sizeof(int32) + cnt * sizeof(int32) + size_total); + nuraft::ptr buf_out = nuraft::buffer::alloc(sizeof(int32) + cnt * sizeof(int32) + uint64_total); buf_out->pos(0); buf_out->put(static_cast(cnt)); @@ -147,14 +147,14 @@ nuraft::ptr InMemoryLogStore::pack(size_t index, Int32 cnt) return buf_out; } -void InMemoryLogStore::apply_pack(size_t index, nuraft::buffer & pack) +void InMemoryLogStore::apply_pack(uint64_t index, nuraft::buffer & pack) { pack.pos(0); Int32 num_logs = pack.get_int(); for (Int32 i = 0; i < num_logs; ++i) { - size_t cur_idx = index + i; + uint64_t cur_idx = index + i; Int32 buf_size = pack.get_int(); nuraft::ptr buf_local = nuraft::buffer::alloc(buf_size); @@ -177,10 +177,10 @@ void InMemoryLogStore::apply_pack(size_t index, nuraft::buffer & pack) } } -bool InMemoryLogStore::compact(size_t last_log_index) +bool InMemoryLogStore::compact(uint64_t last_log_index) { std::lock_guard l(logs_lock); - for (size_t ii = start_idx; ii <= last_log_index; ++ii) + for (uint64_t ii = start_idx; ii <= last_log_index; ++ii) { auto entry = logs.find(ii); if (entry != logs.end()) diff --git a/src/Coordination/InMemoryLogStore.h b/src/Coordination/InMemoryLogStore.h index 425b056a81d..caa5e31698d 100644 --- a/src/Coordination/InMemoryLogStore.h +++ b/src/Coordination/InMemoryLogStore.h @@ -14,34 +14,34 @@ class InMemoryLogStore : public nuraft::log_store public: InMemoryLogStore(); - size_t start_index() const override; + uint64_t start_index() const override; - size_t next_slot() const override; + uint64_t next_slot() const override; nuraft::ptr last_entry() const override; - size_t append(nuraft::ptr & entry) override; + uint64_t append(nuraft::ptr & entry) override; - void write_at(size_t index, nuraft::ptr & entry) override; + void write_at(uint64_t index, nuraft::ptr & entry) override; - nuraft::ptr>> log_entries(size_t start, size_t end) override; + nuraft::ptr>> log_entries(uint64_t start, uint64_t end) override; - nuraft::ptr entry_at(size_t index) override; + nuraft::ptr entry_at(uint64_t index) override; - size_t term_at(size_t index) override; + uint64_t term_at(uint64_t index) override; - nuraft::ptr pack(size_t index, Int32 cnt) override; + nuraft::ptr pack(uint64_t index, Int32 cnt) override; - void apply_pack(size_t index, nuraft::buffer & pack) override; + void apply_pack(uint64_t index, nuraft::buffer & pack) override; - bool compact(size_t last_log_index) override; + bool compact(uint64_t last_log_index) override; bool flush() override { return true; } private: - std::map> logs; + std::map> logs; mutable std::mutex logs_lock; - std::atomic start_idx; + std::atomic start_idx; }; } diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index 2b1306bffb9..f78d57ee1ce 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -3,26 +3,26 @@ namespace DB { -KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_, bool force_sync_) +KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_) : log(&Poco::Logger::get("KeeperLogStore")) , changelog(changelogs_path, rotate_interval_, log) , force_sync(force_sync_) { } -size_t KeeperLogStore::start_index() const +uint64_t KeeperLogStore::start_index() const { std::lock_guard lock(changelog_lock); return changelog.getStartIndex(); } -void KeeperLogStore::init(size_t last_commited_log_index, size_t logs_to_keep) +void KeeperLogStore::init(uint64_t last_commited_log_index, uint64_t logs_to_keep) { std::lock_guard lock(changelog_lock); changelog.readChangelogAndInitWriter(last_commited_log_index, logs_to_keep); } -size_t KeeperLogStore::next_slot() const +uint64_t KeeperLogStore::next_slot() const { std::lock_guard lock(changelog_lock); return changelog.getNextEntryIndex(); @@ -34,34 +34,34 @@ nuraft::ptr KeeperLogStore::last_entry() const return changelog.getLastEntry(); } -size_t KeeperLogStore::append(nuraft::ptr & entry) +uint64_t KeeperLogStore::append(nuraft::ptr & entry) { std::lock_guard lock(changelog_lock); - size_t idx = changelog.getNextEntryIndex(); + uint64_t idx = changelog.getNextEntryIndex(); changelog.appendEntry(idx, entry, force_sync); return idx; } -void KeeperLogStore::write_at(size_t index, nuraft::ptr & entry) +void KeeperLogStore::write_at(uint64_t index, nuraft::ptr & entry) { std::lock_guard lock(changelog_lock); changelog.writeAt(index, entry, force_sync); } -nuraft::ptr>> KeeperLogStore::log_entries(size_t start, size_t end) +nuraft::ptr>> KeeperLogStore::log_entries(uint64_t start, uint64_t end) { std::lock_guard lock(changelog_lock); return changelog.getLogEntriesBetween(start, end); } -nuraft::ptr KeeperLogStore::entry_at(size_t index) +nuraft::ptr KeeperLogStore::entry_at(uint64_t index) { std::lock_guard lock(changelog_lock); return changelog.entryAt(index); } -size_t KeeperLogStore::term_at(size_t index) +uint64_t KeeperLogStore::term_at(uint64_t index) { std::lock_guard lock(changelog_lock); auto entry = changelog.entryAt(index); @@ -70,13 +70,13 @@ size_t KeeperLogStore::term_at(size_t index) return 0; } -nuraft::ptr KeeperLogStore::pack(size_t index, int32_t cnt) +nuraft::ptr KeeperLogStore::pack(uint64_t index, int32_t cnt) { std::lock_guard lock(changelog_lock); return changelog.serializeEntriesToBuffer(index, cnt); } -bool KeeperLogStore::compact(size_t last_log_index) +bool KeeperLogStore::compact(uint64_t last_log_index) { std::lock_guard lock(changelog_lock); changelog.compact(last_log_index); @@ -90,13 +90,13 @@ bool KeeperLogStore::flush() return true; } -void KeeperLogStore::apply_pack(size_t index, nuraft::buffer & pack) +void KeeperLogStore::apply_pack(uint64_t index, nuraft::buffer & pack) { std::lock_guard lock(changelog_lock); changelog.applyEntriesFromBuffer(index, pack, force_sync); } -size_t KeeperLogStore::size() const +uint64_t KeeperLogStore::size() const { std::lock_guard lock(changelog_lock); return changelog.size(); diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index c9c20e144ba..bc2ae719f0e 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -12,35 +12,35 @@ namespace DB class KeeperLogStore : public nuraft::log_store { public: - KeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_, bool force_sync_); + KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_); - void init(size_t last_commited_log_index, size_t logs_to_keep); + void init(uint64_t last_commited_log_index, uint64_t logs_to_keep); - size_t start_index() const override; + uint64_t start_index() const override; - size_t next_slot() const override; + uint64_t next_slot() const override; nuraft::ptr last_entry() const override; - size_t append(nuraft::ptr & entry) override; + uint64_t append(nuraft::ptr & entry) override; - void write_at(size_t index, nuraft::ptr & entry) override; + void write_at(uint64_t index, nuraft::ptr & entry) override; - nuraft::ptr>> log_entries(size_t start, size_t end) override; + nuraft::ptr>> log_entries(uint64_t start, uint64_t end) override; - nuraft::ptr entry_at(size_t index) override; + nuraft::ptr entry_at(uint64_t index) override; - size_t term_at(size_t index) override; + uint64_t term_at(uint64_t index) override; - nuraft::ptr pack(size_t index, int32_t cnt) override; + nuraft::ptr pack(uint64_t index, int32_t cnt) override; - void apply_pack(size_t index, nuraft::buffer & pack) override; + void apply_pack(uint64_t index, nuraft::buffer & pack) override; - bool compact(size_t last_log_index) override; + bool compact(uint64_t last_log_index) override; bool flush() override; - size_t size() const; + uint64_t size() const; private: mutable std::mutex changelog_lock; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index c7446c296f0..7827a25afdd 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -1,4 +1,9 @@ #include + +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + #include #include #include @@ -9,6 +14,7 @@ #include #include #include +#include namespace DB { @@ -16,6 +22,42 @@ namespace DB namespace ErrorCodes { extern const int RAFT_ERROR; + extern const int NO_ELEMENTS_IN_CONFIG; + extern const int SUPPORT_IS_DISABLED; +} + +namespace +{ + +#if USE_SSL +void setSSLParams(nuraft::asio_service::options & asio_opts) +{ + const Poco::Util::LayeredConfiguration & config = Poco::Util::Application::instance().config(); + String certificate_file_property = "openSSL.server.certificateFile"; + String private_key_file_property = "openSSL.server.privateKeyFile"; + String root_ca_file_property = "openSSL.server.caConfig"; + + if (!config.has(certificate_file_property)) + throw Exception("Server certificate file is not set.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); + + if (!config.has(private_key_file_property)) + throw Exception("Server private key file is not set.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); + + asio_opts.enable_ssl_ = true; + asio_opts.server_cert_file_ = config.getString(certificate_file_property); + asio_opts.server_key_file_ = config.getString(private_key_file_property); + + if (config.has(root_ca_file_property)) + asio_opts.root_cert_file_ = config.getString(root_ca_file_property); + + if (config.getBool("openSSL.server.loadDefaultCAFile", false)) + asio_opts.load_default_ca_file_ = true; + + if (config.getString("openSSL.server.verificationMode", "none") == "none") + asio_opts.skip_verification_ = true; +} +#endif + } KeeperServer::KeeperServer( @@ -32,14 +74,14 @@ KeeperServer::KeeperServer( coordination_settings)) , state_manager(nuraft::cs_new(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 +114,99 @@ 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) + if (state_manager->isSecure()) { - return callbackFunc(type, param); - }; +#if USE_SSL + setSSLParams(asio_opts); +#else + throw Exception{"SSL support for NuRaft is disabled because ClickHouse was built without SSL support.", + ErrorCodes::SUPPORT_IS_DISABLED}; +#endif + } - raft_instance = launcher.init( - state_machine, state_manager, nuraft::cs_new("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 logger = nuraft::cs_new("RaftInstance", coordination_settings->raft_logs_level); + asio_service = nuraft::cs_new(asio_opts, logger); + asio_listener = asio_service->create_rpc_listener(state_manager->getPort(), logger); + + if (!asio_listener) + return; + + nuraft::ptr scheduler = asio_service; + nuraft::ptr rpc_cli_factory = asio_service; + + nuraft::ptr casted_state_manager = state_manager; + nuraft::ptr 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(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 +303,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 +333,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 +353,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; diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 9a0260cda94..5af948305ef 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -22,9 +23,9 @@ private: nuraft::ptr state_manager; - nuraft::raft_launcher launcher; - nuraft::ptr raft_instance; + nuraft::ptr asio_service; + nuraft::ptr asio_listener; std::mutex append_entries_mutex; @@ -36,8 +37,19 @@ private: std::atomic initial_batch_committed = false; std::atomic 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_, diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 70d540ca06b..3cfa07fb542 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -23,16 +23,16 @@ namespace ErrorCodes namespace { - size_t getSnapshotPathUpToLogIdx(const String & snapshot_path) + uint64_t getSnapshotPathUpToLogIdx(const String & snapshot_path) { std::filesystem::path path(snapshot_path); std::string filename = path.stem(); Strings name_parts; splitInto<'_'>(name_parts, filename); - return parse(name_parts[1]); + return parse(name_parts[1]); } - std::string getSnapshotFileName(size_t up_to_log_idx) + std::string getSnapshotFileName(uint64_t up_to_log_idx) { return std::string{"snapshot_"} + std::to_string(up_to_log_idx) + ".bin"; } @@ -214,7 +214,7 @@ SnapshotMetadataPtr KeeperStorageSnapshot::deserialize(KeeperStorage & storage, return result; } -KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, size_t up_to_log_idx_) +KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_) : storage(storage_) , snapshot_meta(std::make_shared(up_to_log_idx_, 0, std::make_shared())) , session_id(storage->session_id_counter) @@ -266,7 +266,7 @@ KeeperSnapshotManager::KeeperSnapshotManager(const std::string & snapshots_path_ } -std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, size_t up_to_log_idx) +std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) { ReadBufferFromNuraftBuffer reader(buffer); @@ -307,7 +307,7 @@ nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBuff return nullptr; } -nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(size_t up_to_log_idx) const +nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const { const std::string & snapshot_path = existing_snapshots.at(up_to_log_idx); WriteBufferFromNuraftBuffer writer; @@ -352,7 +352,7 @@ void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded() removeSnapshot(existing_snapshots.begin()->first); } -void KeeperSnapshotManager::removeSnapshot(size_t log_idx) +void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) { auto itr = existing_snapshots.find(log_idx); if (itr == existing_snapshots.end()) diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index e53473d71c7..95d1ce831d4 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -18,7 +18,7 @@ enum SnapshotVersion : uint8_t struct KeeperStorageSnapshot { public: - KeeperStorageSnapshot(KeeperStorage * storage_, size_t up_to_log_idx_); + KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_); KeeperStorageSnapshot(KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_); ~KeeperStorageSnapshot(); @@ -51,14 +51,14 @@ public: SnapshotMetaAndStorage restoreFromLatestSnapshot(); static nuraft::ptr serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot); - std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, size_t up_to_log_idx); + std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx); SnapshotMetaAndStorage deserializeSnapshotFromBuffer(nuraft::ptr buffer) const; - nuraft::ptr deserializeSnapshotBufferFromDisk(size_t up_to_log_idx) const; + nuraft::ptr deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const; nuraft::ptr deserializeLatestSnapshotBufferFromDisk(); - void removeSnapshot(size_t log_idx); + void removeSnapshot(uint64_t log_idx); size_t totalSnapshots() const { @@ -76,7 +76,7 @@ private: void removeOutdatedSnapshotsIfNeeded(); const std::string snapshots_path; const size_t snapshots_to_keep; - std::map existing_snapshots; + std::map existing_snapshots; size_t storage_tick_time; }; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 227bb1aaa4e..c909cfc68e2 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -54,7 +54,7 @@ void KeeperStateMachine::init() bool has_snapshots = snapshot_manager.totalSnapshots() != 0; while (snapshot_manager.totalSnapshots() != 0) { - size_t latest_log_index = snapshot_manager.getLatestSnapshotIndex(); + uint64_t latest_log_index = snapshot_manager.getLatestSnapshotIndex(); LOG_DEBUG(log, "Trying to load state machine from snapshot up to log index {}", latest_log_index); try @@ -88,7 +88,7 @@ void KeeperStateMachine::init() storage = std::make_unique(coordination_settings->dead_session_check_period_ms.totalMilliseconds()); } -nuraft::ptr KeeperStateMachine::commit(const size_t log_idx, nuraft::buffer & data) +nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) { if (data.size() == sizeof(int64_t)) { @@ -205,7 +205,7 @@ void KeeperStateMachine::create_snapshot( void KeeperStateMachine::save_logical_snp_obj( nuraft::snapshot & s, - size_t & obj_id, + uint64_t & obj_id, nuraft::buffer & data, bool /*is_first_obj*/, bool /*is_last_obj*/) @@ -246,7 +246,7 @@ void KeeperStateMachine::save_logical_snp_obj( int KeeperStateMachine::read_logical_snp_obj( nuraft::snapshot & s, void* & /*user_snp_ctx*/, - ulong obj_id, + uint64_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) { diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index aefdbe3f9f3..8712adaf4b1 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -20,13 +20,13 @@ public: void init(); - nuraft::ptr pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; } + nuraft::ptr pre_commit(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; } - nuraft::ptr commit(const size_t log_idx, nuraft::buffer & data) override; + nuraft::ptr commit(const uint64_t log_idx, nuraft::buffer & data) override; - void rollback(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override {} + void rollback(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override {} - size_t last_commit_index() override { return last_committed_idx; } + uint64_t last_commit_index() override { return last_committed_idx; } bool apply_snapshot(nuraft::snapshot & s) override; @@ -38,7 +38,7 @@ public: void save_logical_snp_obj( nuraft::snapshot & s, - size_t & obj_id, + uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override; @@ -46,7 +46,7 @@ public: int read_logical_snp_obj( nuraft::snapshot & s, void* & user_snp_ctx, - ulong obj_id, + uint64_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) override; @@ -82,7 +82,7 @@ private: std::mutex storage_lock; /// Last committed Raft log number. - std::atomic last_committed_idx; + std::atomic last_committed_idx; Poco::Logger * log; }; diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index ad656cd1f95..e57ae7e7c19 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -12,6 +12,7 @@ namespace ErrorCodes KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path) : my_server_id(server_id_) , my_port(port) + , secure(false) , log_store(nuraft::cs_new(logs_path, 5000, false)) , cluster_config(nuraft::cs_new()) { @@ -25,6 +26,7 @@ KeeperStateManager::KeeperStateManager( const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings) : my_server_id(my_server_id_) + , secure(config.getBool(config_prefix + ".raft_configuration.secure", false)) , log_store(nuraft::cs_new( config.getString(config_prefix + ".log_storage_path", config.getString("path", DBMS_DEFAULT_PATH) + "coordination/logs"), coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync)) @@ -37,6 +39,9 @@ KeeperStateManager::KeeperStateManager( for (const auto & server_key : keys) { + if (!startsWith(server_key, "server")) + continue; + std::string full_prefix = config_prefix + ".raft_configuration." + server_key; int server_id = config.getInt(full_prefix + ".id"); std::string hostname = config.getString(full_prefix + ".hostname"); @@ -44,6 +49,7 @@ KeeperStateManager::KeeperStateManager( bool can_become_leader = config.getBool(full_prefix + ".can_become_leader", true); int32_t priority = config.getInt(full_prefix + ".priority", 1); bool start_as_follower = config.getBool(full_prefix + ".start_as_follower", false); + if (start_as_follower) start_as_follower_servers.insert(server_id); @@ -57,6 +63,7 @@ KeeperStateManager::KeeperStateManager( cluster_config->get_servers().push_back(peer_config); } + if (!my_server_config) throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section", my_server_id); @@ -64,7 +71,7 @@ KeeperStateManager::KeeperStateManager( throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without )"); } -void KeeperStateManager::loadLogStore(size_t last_commited_index, size_t logs_to_keep) +void KeeperStateManager::loadLogStore(uint64_t last_commited_index, uint64_t logs_to_keep) { log_store->init(last_commited_index, logs_to_keep); } diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index f0575082b02..cb5181760cb 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -25,7 +25,7 @@ public: int port, const std::string & logs_path); - void loadLogStore(size_t last_commited_index, size_t logs_to_keep); + void loadLogStore(uint64_t last_commited_index, uint64_t logs_to_keep); void flushLogStore(); @@ -52,14 +52,20 @@ public: return start_as_follower_servers.count(my_server_id); } + bool isSecure() const + { + return secure; + } + nuraft::ptr getLogStore() const { return log_store; } - size_t getTotalServers() const { return total_servers; } + uint64_t getTotalServers() const { return total_servers; } private: int my_server_id; int my_port; - size_t total_servers{0}; + bool secure; + uint64_t total_servers{0}; std::unordered_set start_as_follower_servers; nuraft::ptr log_store; nuraft::ptr my_server_config; diff --git a/src/Coordination/SummingStateMachine.cpp b/src/Coordination/SummingStateMachine.cpp index 0cb7a7da6c3..ae3d2b06d75 100644 --- a/src/Coordination/SummingStateMachine.cpp +++ b/src/Coordination/SummingStateMachine.cpp @@ -21,7 +21,7 @@ SummingStateMachine::SummingStateMachine() { } -nuraft::ptr SummingStateMachine::commit(const size_t log_idx, nuraft::buffer & data) +nuraft::ptr SummingStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) { int64_t value_to_add = deserializeValue(data); @@ -84,7 +84,7 @@ void SummingStateMachine::createSnapshotInternal(nuraft::snapshot & s) void SummingStateMachine::save_logical_snp_obj( nuraft::snapshot & s, - size_t & obj_id, + uint64_t & obj_id, nuraft::buffer & data, bool /*is_first_obj*/, bool /*is_last_obj*/) @@ -112,7 +112,7 @@ void SummingStateMachine::save_logical_snp_obj( int SummingStateMachine::read_logical_snp_obj( nuraft::snapshot & s, void* & /*user_snp_ctx*/, - size_t obj_id, + uint64_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) { @@ -142,7 +142,7 @@ int SummingStateMachine::read_logical_snp_obj( else { // Object ID > 0: second object, put actual value. - data_out = nuraft::buffer::alloc(sizeof(size_t)); + data_out = nuraft::buffer::alloc(sizeof(uint64_t)); nuraft::buffer_serializer bs(data_out); bs.put_u64(ctx->value); is_last_obj = true; diff --git a/src/Coordination/SummingStateMachine.h b/src/Coordination/SummingStateMachine.h index c8594ba7e8d..03572840739 100644 --- a/src/Coordination/SummingStateMachine.h +++ b/src/Coordination/SummingStateMachine.h @@ -15,13 +15,13 @@ class SummingStateMachine : public nuraft::state_machine public: SummingStateMachine(); - nuraft::ptr pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; } + nuraft::ptr pre_commit(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; } - nuraft::ptr commit(const size_t log_idx, nuraft::buffer & data) override; + nuraft::ptr commit(const uint64_t log_idx, nuraft::buffer & data) override; - void rollback(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override {} + void rollback(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override {} - size_t last_commit_index() override { return last_committed_idx; } + uint64_t last_commit_index() override { return last_committed_idx; } bool apply_snapshot(nuraft::snapshot & s) override; @@ -33,7 +33,7 @@ public: void save_logical_snp_obj( nuraft::snapshot & s, - size_t & obj_id, + uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override; @@ -41,7 +41,7 @@ public: int read_logical_snp_obj( nuraft::snapshot & s, void* & user_snp_ctx, - size_t obj_id, + uint64_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) override; diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 06e60e5592f..0e06af1a7f3 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -1085,7 +1085,7 @@ nuraft::ptr getLogEntryFromZKRequest(size_t term, int64_t ses return nuraft::cs_new(term, buffer); } -void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, size_t total_logs) +void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint64_t total_logs) { using namespace Coordination; using namespace DB; diff --git a/src/Core/Block.h b/src/Core/Block.h index 914d6b361b9..0cfd17b27dc 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -1,15 +1,15 @@ #pragma once -#include -#include -#include -#include -#include - #include -#include #include #include +#include + +#include +#include +#include +#include +#include namespace DB @@ -22,8 +22,6 @@ namespace DB * Allows to insert, remove columns in arbitrary position, to change order of columns. */ -class Context; - class Block { private: diff --git a/src/Core/BlockInfo.h b/src/Core/BlockInfo.h index c8dd1576b22..125c90b1b23 100644 --- a/src/Core/BlockInfo.h +++ b/src/Core/BlockInfo.h @@ -1,9 +1,10 @@ #pragma once -#include - #include +#include +#include + namespace DB { diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 468c1187e91..e7c1c86a23e 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -81,8 +81,9 @@ #define DBMS_MIN_REVISION_WITH_REFERER_IN_CLIENT_INFO 54447 /// Version of ClickHouse TCP protocol. Increment it manually when you change the protocol. -#define DBMS_TCP_PROTOCOL_VERSION 54447 +#define DBMS_TCP_PROTOCOL_VERSION 54448 +#define DBMS_MIN_PROTOCOL_VERSION_WITH_DISTRIBUTED_DEPTH 54448 /// The boundary on which the blocks for asynchronous file operations should be aligned. #define DEFAULT_AIO_FILE_BLOCK_SIZE 4096 diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index afc9fe00ef5..87945dd1ce6 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -31,11 +31,11 @@ namespace ErrorCodes } -ExternalTableDataPtr BaseExternalTable::getData(const Context & context) +ExternalTableDataPtr BaseExternalTable::getData(ContextPtr context) { initReadBuffer(); initSampleBlock(); - auto input = context.getInputFormat(format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE); + auto input = context->getInputFormat(format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE); auto stream = std::make_shared(input); auto data = std::make_unique(); @@ -127,7 +127,7 @@ ExternalTable::ExternalTable(const boost::program_options::variables_map & exter void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, ReadBuffer & stream) { - const Settings & settings = context.getSettingsRef(); + const Settings & settings = getContext()->getSettingsRef(); if (settings.http_max_multipart_form_data_size) read_buffer = std::make_unique( @@ -152,14 +152,14 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, else throw Exception("Neither structure nor types have not been provided for external table " + name + ". Use fields " + name + "_structure or " + name + "_types to do so.", ErrorCodes::BAD_ARGUMENTS); - ExternalTableDataPtr data = getData(context); + ExternalTableDataPtr data = getData(getContext()); /// Create table NamesAndTypesList columns = sample_block.getNamesAndTypesList(); - auto temporary_table = TemporaryTableHolder(context, ColumnsDescription{columns}, {}); + auto temporary_table = TemporaryTableHolder(getContext(), ColumnsDescription{columns}, {}); auto storage = temporary_table.getTable(); - context.addExternalTable(data->table_name, std::move(temporary_table)); - BlockOutputStreamPtr output = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), context); + getContext()->addExternalTable(data->table_name, std::move(temporary_table)); + BlockOutputStreamPtr output = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), getContext()); /// Write data data->pipe->resize(1); diff --git a/src/Core/ExternalTable.h b/src/Core/ExternalTable.h index aa15846d48a..fcefa3d7fe3 100644 --- a/src/Core/ExternalTable.h +++ b/src/Core/ExternalTable.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -11,30 +12,21 @@ #include -namespace Poco +namespace Poco::Net { - namespace Net - { - class NameValueCollection; - class MessageHeader; - } +class NameValueCollection; +class MessageHeader; } -namespace boost +namespace boost::program_options { - namespace program_options - { - class variables_map; - } +class variables_map; } namespace DB { -class Context; - - /// The base class containing the basic information about external table and /// basic functions for extracting this information from text fields. class BaseExternalTable @@ -56,7 +48,7 @@ public: virtual void initReadBuffer() {} /// Get the table data - a pair (a stream with the contents of the table, the name of the table) - ExternalTableDataPtr getData(const Context & context); + ExternalTableDataPtr getData(ContextPtr context); protected: /// Clear all accumulated information @@ -88,15 +80,14 @@ public: /// Parsing of external table used when sending tables via http /// The `handlePart` function will be called for each table passed, /// so it's also necessary to call `clean` at the end of the `handlePart`. -class ExternalTablesHandler : public HTMLForm::PartHandler, BaseExternalTable +class ExternalTablesHandler : public HTMLForm::PartHandler, BaseExternalTable, WithContext { public: - ExternalTablesHandler(Context & context_, const Poco::Net::NameValueCollection & params_) : context(context_), params(params_) {} + ExternalTablesHandler(ContextPtr context_, const Poco::Net::NameValueCollection & params_) : WithContext(context_), params(params_) {} void handlePart(const Poco::Net::MessageHeader & header, ReadBuffer & stream) override; private: - Context & context; const Poco::Net::NameValueCollection & params; }; diff --git a/src/Core/MySQL/Authentication.cpp b/src/Core/MySQL/Authentication.cpp index e685ad0394d..b8dff9972c7 100644 --- a/src/Core/MySQL/Authentication.cpp +++ b/src/Core/MySQL/Authentication.cpp @@ -73,7 +73,7 @@ Native41::Native41(const String & password, const String & auth_plugin_data) } void Native41::authenticate( - const String & user_name, std::optional auth_response, Context & context, + const String & user_name, std::optional auth_response, ContextPtr context, std::shared_ptr packet_endpoint, bool, const Poco::Net::SocketAddress & address) { if (!auth_response) @@ -86,15 +86,17 @@ void Native41::authenticate( if (auth_response->empty()) { - context.setUser(user_name, "", address); + context->setUser(user_name, "", address); return; } if (auth_response->size() != Poco::SHA1Engine::DIGEST_SIZE) - throw Exception("Wrong size of auth response. Expected: " + std::to_string(Poco::SHA1Engine::DIGEST_SIZE) + " bytes, received: " + std::to_string(auth_response->size()) + " bytes.", - ErrorCodes::UNKNOWN_EXCEPTION); + throw Exception( + "Wrong size of auth response. Expected: " + std::to_string(Poco::SHA1Engine::DIGEST_SIZE) + + " bytes, received: " + std::to_string(auth_response->size()) + " bytes.", + ErrorCodes::UNKNOWN_EXCEPTION); - auto user = context.getAccessControlManager().read(user_name); + auto user = context->getAccessControlManager().read(user_name); Poco::SHA1Engine::Digest double_sha1_value = user->authentication.getPasswordDoubleSHA1(); assert(double_sha1_value.size() == Poco::SHA1Engine::DIGEST_SIZE); @@ -109,7 +111,7 @@ void Native41::authenticate( { password_sha1[i] = digest[i] ^ static_cast((*auth_response)[i]); } - context.setUser(user_name, password_sha1, address); + context->setUser(user_name, password_sha1, address); } #if USE_SSL @@ -134,7 +136,7 @@ Sha256Password::Sha256Password(RSA & public_key_, RSA & private_key_, Poco::Logg } void Sha256Password::authenticate( - const String & user_name, std::optional auth_response, Context & context, + const String & user_name, std::optional auth_response, ContextPtr context, std::shared_ptr packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) { if (!auth_response) @@ -229,7 +231,7 @@ void Sha256Password::authenticate( password.pop_back(); } - context.setUser(user_name, password, address); + context->setUser(user_name, password, address); } #endif diff --git a/src/Core/MySQL/Authentication.h b/src/Core/MySQL/Authentication.h index e1b7c174139..5358e2da737 100644 --- a/src/Core/MySQL/Authentication.h +++ b/src/Core/MySQL/Authentication.h @@ -32,7 +32,7 @@ public: virtual String getAuthPluginData() = 0; virtual void authenticate( - const String & user_name, std::optional auth_response, Context & context, + const String & user_name, std::optional auth_response, ContextPtr context, std::shared_ptr packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) = 0; }; @@ -49,7 +49,7 @@ public: String getAuthPluginData() override { return scramble; } void authenticate( - const String & user_name, std::optional auth_response, Context & context, + const String & user_name, std::optional auth_response, ContextPtr context, std::shared_ptr packet_endpoint, bool /* is_secure_connection */, const Poco::Net::SocketAddress & address) override; private: @@ -69,7 +69,7 @@ public: String getAuthPluginData() override { return scramble; } void authenticate( - const String & user_name, std::optional auth_response, Context & context, + const String & user_name, std::optional auth_response, ContextPtr context, std::shared_ptr packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) override; private: diff --git a/src/Core/PostgreSQLProtocol.h b/src/Core/PostgreSQLProtocol.h index 471811e969b..62b87e78642 100644 --- a/src/Core/PostgreSQLProtocol.h +++ b/src/Core/PostgreSQLProtocol.h @@ -797,15 +797,15 @@ namespace PGAuthentication class AuthenticationMethod { protected: - void setPassword( + static void setPassword( const String & user_name, const String & password, - Context & context, + ContextPtr context, Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) { try { - context.setUser(user_name, password, address); + context->setUser(user_name, password, address); } catch (const Exception &) { @@ -819,7 +819,7 @@ protected: public: virtual void authenticate( const String & user_name, - Context & context, + ContextPtr context, Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) = 0; @@ -833,7 +833,7 @@ class NoPasswordAuth : public AuthenticationMethod public: void authenticate( const String & /* user_name */, - Context & /* context */, + ContextPtr /* context */, Messaging::MessageTransport & /* mt */, const Poco::Net::SocketAddress & /* address */) override {} @@ -848,7 +848,7 @@ class CleartextPasswordAuth : public AuthenticationMethod public: void authenticate( const String & user_name, - Context & context, + ContextPtr context, Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) override { @@ -891,11 +891,11 @@ public: void authenticate( const String & user_name, - Context & context, + ContextPtr context, Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) { - auto user = context.getAccessControlManager().read(user_name); + auto user = context->getAccessControlManager().read(user_name); Authentication::Type user_auth_type = user->authentication.getType(); if (type_to_method.find(user_auth_type) != type_to_method.end()) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 045433dc895..d31073ae932 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -78,7 +78,7 @@ class IColumn; M(UInt64, background_buffer_flush_schedule_pool_size, 16, "Number of threads performing background flush for tables with Buffer engine. Only has meaning at server startup.", 0) \ M(UInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \ M(UInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \ - M(UInt64, background_fetches_pool_size, 3, "Number of threads performing background fetches for replicated tables. Only has meaning at server startup.", 0) \ + M(UInt64, background_fetches_pool_size, 8, "Number of threads performing background fetches for replicated tables. Only has meaning at server startup.", 0) \ M(UInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables, dns cache updates. Only has meaning at server startup.", 0) \ M(UInt64, background_message_broker_schedule_pool_size, 16, "Number of threads performing background tasks for message streaming. Only has meaning at server startup.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "Number of threads performing background tasks for distributed sends. Only has meaning at server startup.", 0) \ @@ -118,6 +118,7 @@ class IColumn; M(Bool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avoiding costly aggregation on the initiator server).", 0) \ M(UInt64, optimize_skip_unused_shards_limit, 1000, "Limit for number of sharding key values, turns off optimize_skip_unused_shards if the limit is reached", 0) \ M(Bool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ + M(Bool, optimize_skip_unused_shards_rewrite_in, true, "Rewrite IN in query for remote shards to exclude values that does not belong to the shard (requires optimize_skip_unused_shards)", 0) \ M(Bool, allow_nondeterministic_optimize_skip_unused_shards, false, "Allow non-deterministic functions (includes dictGet) in sharding_key for optimize_skip_unused_shards", 0) \ M(UInt64, force_optimize_skip_unused_shards, 0, "Throw an exception if unused shards cannot be skipped (1 - throw only if the table has the sharding key, 2 - always throw.", 0) \ M(UInt64, optimize_skip_unused_shards_nesting, 0, "Same as optimize_skip_unused_shards, but accept nesting level until which it will work.", 0) \ @@ -228,7 +229,7 @@ class IColumn; M(Seconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \ M(Seconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \ M(Seconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout", 0) \ - M(UInt64, http_max_uri_size, 16384, "Maximum URI length of HTTP request", 0) \ + M(UInt64, http_max_uri_size, 1048576, "Maximum URI length of HTTP request", 0) \ M(Bool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \ M(Bool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \ M(Bool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \ @@ -371,6 +372,8 @@ class IColumn; \ M(UInt64, postgresql_connection_pool_size, 16, "Connection pool size for PostgreSQL table engine and database engine.", 0) \ M(Int64, postgresql_connection_pool_wait_timeout, -1, "Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.", 0) \ + M(UInt64, glob_expansion_max_elements, 1000, "Maximum number of allowed addresses (For external storages, table functions, etc).", 0) \ + M(UInt64, odbc_bridge_connection_pool_size, 16, "Connection pool size for each connection settings string in ODBC bridge.", 0) \ \ M(Seconds, distributed_replica_error_half_life, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD, "Time period reduces replica error counter by 2 times.", 0) \ M(UInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up an incredible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.", 0) \ @@ -440,6 +443,7 @@ class IColumn; M(Bool, engine_file_truncate_on_insert, false, "Enables or disables truncate before insert in file engine tables", 0) \ M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ + M(UInt64, max_distributed_depth, 5, "Maximum distributed query depth", 0) \ M(Bool, database_replicated_always_detach_permanently, false, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result", 0) \ M(UInt64, distributed_ddl_entry_format_version, 1, "Version of DDL entry to write into ZooKeeper", 0) \ @@ -453,8 +457,13 @@ class IColumn; M(UnionMode, union_default_mode, UnionMode::Unspecified, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) \ M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \ M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \ + \ + M(Bool, query_plan_enable_optimizations, true, "Apply optimizations to query plan", 0) \ M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \ + M(Bool, query_plan_filter_push_down, true, "Allow to push down filter by predicate query plan step", 0) \ + \ M(Bool, database_replicated_ddl_output, true, "Obsolete setting, does nothing. Will be removed after 2021-09-08", 0) \ + M(HandleKafkaErrorMode, handle_kafka_error_mode, HandleKafkaErrorMode::DEFAULT, "How to handle errors for Kafka engine. Passible values: default, stream.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS below. diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 64ba51d1c68..26c2bd9b6af 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -108,4 +108,7 @@ IMPLEMENT_SETTING_ENUM(DistributedDDLOutputMode, ErrorCodes::BAD_ARGUMENTS, {"null_status_on_timeout", DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT}, {"never_throw", DistributedDDLOutputMode::NEVER_THROW}}) +IMPLEMENT_SETTING_ENUM(HandleKafkaErrorMode, ErrorCodes::BAD_ARGUMENTS, + {{"default", HandleKafkaErrorMode::DEFAULT}, + {"stream", HandleKafkaErrorMode::STREAM}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 7615b185a61..f0dd10aacfb 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -138,7 +138,6 @@ enum class UnionMode DECLARE_SETTING_ENUM(UnionMode) - enum class DistributedDDLOutputMode { NONE, @@ -149,4 +148,13 @@ enum class DistributedDDLOutputMode DECLARE_SETTING_ENUM(DistributedDDLOutputMode) +enum class HandleKafkaErrorMode +{ + DEFAULT = 0, // Ignore errors whit threshold. + STREAM, // Put errors to stream in the virtual column named ``_error. + /*FIXED_SYSTEM_TABLE, Put errors to in a fixed system table likey system.kafka_errors. This is not implemented now. */ + /*CUSTOM_SYSTEM_TABLE, Put errors to in a custom system table. This is not implemented now. */ +}; + +DECLARE_SETTING_ENUM(HandleKafkaErrorMode) } diff --git a/src/DataStreams/AddingDefaultBlockOutputStream.cpp b/src/DataStreams/AddingDefaultBlockOutputStream.cpp index f241d417205..f4d8f6954c1 100644 --- a/src/DataStreams/AddingDefaultBlockOutputStream.cpp +++ b/src/DataStreams/AddingDefaultBlockOutputStream.cpp @@ -10,7 +10,7 @@ AddingDefaultBlockOutputStream::AddingDefaultBlockOutputStream( const BlockOutputStreamPtr & output_, const Block & header_, const ColumnsDescription & columns_, - const Context & context_) + ContextPtr context_) : output(output_), header(header_) { auto dag = addMissingDefaults(header_, output->getHeader().getNamesAndTypesList(), columns_, context_); diff --git a/src/DataStreams/AddingDefaultBlockOutputStream.h b/src/DataStreams/AddingDefaultBlockOutputStream.h index 5fbbe2aed60..a8235d6623a 100644 --- a/src/DataStreams/AddingDefaultBlockOutputStream.h +++ b/src/DataStreams/AddingDefaultBlockOutputStream.h @@ -26,7 +26,7 @@ public: const BlockOutputStreamPtr & output_, const Block & header_, const ColumnsDescription & columns_, - const Context & context_); + ContextPtr context_); Block getHeader() const override { return header; } void write(const Block & block) override; diff --git a/src/DataStreams/AddingDefaultsBlockInputStream.cpp b/src/DataStreams/AddingDefaultsBlockInputStream.cpp index 81408b0aaa6..e3f0906cb03 100644 --- a/src/DataStreams/AddingDefaultsBlockInputStream.cpp +++ b/src/DataStreams/AddingDefaultsBlockInputStream.cpp @@ -130,7 +130,7 @@ static MutableColumnPtr mixColumns(const ColumnWithTypeAndName & col_read, AddingDefaultsBlockInputStream::AddingDefaultsBlockInputStream( const BlockInputStreamPtr & input, const ColumnsDescription & columns_, - const Context & context_) + ContextPtr context_) : columns(columns_) , column_defaults(columns.getDefaults()) , context(context_) diff --git a/src/DataStreams/AddingDefaultsBlockInputStream.h b/src/DataStreams/AddingDefaultsBlockInputStream.h index b2efa5863ef..957f14caff3 100644 --- a/src/DataStreams/AddingDefaultsBlockInputStream.h +++ b/src/DataStreams/AddingDefaultsBlockInputStream.h @@ -7,8 +7,6 @@ namespace DB { -class Context; - /// Adds defaults to columns using BlockDelayedDefaults bitmask attached to Block by child InputStream. class AddingDefaultsBlockInputStream : public IBlockInputStream { @@ -16,7 +14,7 @@ public: AddingDefaultsBlockInputStream( const BlockInputStreamPtr & input, const ColumnsDescription & columns_, - const Context & context_); + ContextPtr context_); String getName() const override { return "AddingDefaults"; } Block getHeader() const override { return header; } @@ -28,7 +26,7 @@ private: Block header; const ColumnsDescription columns; const ColumnDefaults column_defaults; - const Context & context; + ContextPtr context; }; } diff --git a/src/DataStreams/BlockIO.h b/src/DataStreams/BlockIO.h index 91d7efac8d1..31a0e1020d2 100644 --- a/src/DataStreams/BlockIO.h +++ b/src/DataStreams/BlockIO.h @@ -50,7 +50,7 @@ struct BlockIO } } - void onException() + void onException() const { if (exception_callback) exception_callback(); diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 3b0b4db72f9..c4556162323 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -27,7 +27,7 @@ CheckConstraintsBlockOutputStream::CheckConstraintsBlockOutputStream( const BlockOutputStreamPtr & output_, const Block & header_, const ConstraintsDescription & constraints_, - const Context & context_) + ContextPtr context_) : table_id(table_id_), output(output_), header(header_), diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.h b/src/DataStreams/CheckConstraintsBlockOutputStream.h index a1be720f21e..0f115550eb8 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.h +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.h @@ -20,7 +20,7 @@ public: const BlockOutputStreamPtr & output_, const Block & header_, const ConstraintsDescription & constraints_, - const Context & context_); + ContextPtr context_); Block getHeader() const override { return header; } void write(const Block & block) override; diff --git a/src/DataStreams/IBlockInputStream.h b/src/DataStreams/IBlockInputStream.h index 4e314ef2980..b077f87e8a8 100644 --- a/src/DataStreams/IBlockInputStream.h +++ b/src/DataStreams/IBlockInputStream.h @@ -2,9 +2,8 @@ #include #include -#include -#include #include +#include #include #include #include diff --git a/src/DataStreams/InputStreamFromASTInsertQuery.cpp b/src/DataStreams/InputStreamFromASTInsertQuery.cpp index 70d69227ac0..0848d838276 100644 --- a/src/DataStreams/InputStreamFromASTInsertQuery.cpp +++ b/src/DataStreams/InputStreamFromASTInsertQuery.cpp @@ -24,7 +24,7 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery( const ASTPtr & ast, ReadBuffer * input_buffer_tail_part, const Block & header, - const Context & context, + ContextPtr context, const ASTPtr & input_function) { const auto * ast_insert_query = ast->as(); @@ -58,9 +58,9 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery( input_buffer_contacenated = std::make_unique(buffers); - res_stream = context.getInputFormat(format, *input_buffer_contacenated, header, context.getSettings().max_insert_block_size); + res_stream = context->getInputFormat(format, *input_buffer_contacenated, header, context->getSettings().max_insert_block_size); - if (context.getSettingsRef().input_format_defaults_for_omitted_fields && ast_insert_query->table_id && !input_function) + if (context->getSettingsRef().input_format_defaults_for_omitted_fields && ast_insert_query->table_id && !input_function) { StoragePtr storage = DatabaseCatalog::instance().getTable(ast_insert_query->table_id, context); auto metadata_snapshot = storage->getInMemoryMetadataPtr(); diff --git a/src/DataStreams/InputStreamFromASTInsertQuery.h b/src/DataStreams/InputStreamFromASTInsertQuery.h index d4c6443c77d..15b698a2d68 100644 --- a/src/DataStreams/InputStreamFromASTInsertQuery.h +++ b/src/DataStreams/InputStreamFromASTInsertQuery.h @@ -25,7 +25,7 @@ public: const ASTPtr & ast, ReadBuffer * input_buffer_tail_part, const Block & header, - const Context & context, + ContextPtr context, const ASTPtr & input_function); Block readImpl() override { return res_stream->read(); } diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 478df9e2259..f4af15df011 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -28,7 +28,7 @@ namespace ErrorCodes } PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( - PostgreSQLConnectionHolderPtr connection_, + postgres::ConnectionHolderPtr connection_, const std::string & query_str_, const Block & sample_block, const UInt64 max_block_size_) @@ -120,8 +120,15 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view switch (type) { case ValueType::vtUInt8: - assert_cast(column).insertValue(pqxx::from_string(value)); + { + if (value == "t") + assert_cast(column).insertValue(1); + else if (value == "f") + assert_cast(column).insertValue(0); + else + assert_cast(column).insertValue(pqxx::from_string(value)); break; + } case ValueType::vtUInt16: assert_cast(column).insertValue(pqxx::from_string(value)); break; diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 7795cc2d36a..b172cae5b62 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -19,7 +19,7 @@ class PostgreSQLBlockInputStream : public IBlockInputStream { public: PostgreSQLBlockInputStream( - PostgreSQLConnectionHolderPtr connection_, + postgres::ConnectionHolderPtr connection_, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_); @@ -46,7 +46,7 @@ private: const UInt64 max_block_size; ExternalResultDescription description; - PostgreSQLConnectionHolderPtr connection; + postgres::ConnectionHolderPtr connection; std::unique_ptr tx; std::unique_ptr stream; diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 4d1990ffe18..56100205b0d 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -26,13 +26,13 @@ namespace DB PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( const StoragePtr & storage_, const StorageMetadataPtr & metadata_snapshot_, - const Context & context_, + ContextPtr context_, const ASTPtr & query_ptr_, bool no_destination) - : storage(storage_) + : WithContext(context_) + , storage(storage_) , metadata_snapshot(metadata_snapshot_) , log(&Poco::Logger::get("PushingToViewsBlockOutputStream")) - , context(context_) , query_ptr(query_ptr_) { checkStackSize(); @@ -42,12 +42,12 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( * but it's clear that here is not the best place for this functionality. */ addTableLock( - storage->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout)); + storage->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout)); /// If the "root" table deduplicates blocks, there are no need to make deduplication for children /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks bool disable_deduplication_for_children = false; - if (!context.getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) + if (!getContext()->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) disable_deduplication_for_children = !no_destination && storage->supportsDeduplication(); auto table_id = storage->getStorageID(); @@ -56,8 +56,8 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( /// We need special context for materialized views insertions if (!dependencies.empty()) { - select_context = std::make_unique(context); - insert_context = std::make_unique(context); + select_context = Context::createCopy(context); + insert_context = Context::createCopy(context); const auto & insert_settings = insert_context->getSettingsRef(); @@ -74,7 +74,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( for (const auto & database_table : dependencies) { - auto dependent_table = DatabaseCatalog::instance().getTable(database_table, context); + auto dependent_table = DatabaseCatalog::instance().getTable(database_table, getContext()); auto dependent_metadata_snapshot = dependent_table->getInMemoryMetadataPtr(); ASTPtr query; @@ -83,7 +83,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( if (auto * materialized_view = dynamic_cast(dependent_table.get())) { addTableLock( - materialized_view->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout)); + materialized_view->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout)); StoragePtr inner_table = materialized_view->getTargetTable(); auto inner_table_id = inner_table->getStorageID(); @@ -94,7 +94,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( insert->table_id = inner_table_id; /// Get list of columns we get from select query. - auto header = InterpreterSelectQuery(query, *select_context, SelectQueryOptions().analyze()) + auto header = InterpreterSelectQuery(query, select_context, SelectQueryOptions().analyze()) .getSampleBlock(); /// Insert only columns returned by select. @@ -110,16 +110,16 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( insert->columns = std::move(list); ASTPtr insert_query_ptr(insert.release()); - InterpreterInsertQuery interpreter(insert_query_ptr, *insert_context); + InterpreterInsertQuery interpreter(insert_query_ptr, insert_context); BlockIO io = interpreter.execute(); out = io.out; } else if (dynamic_cast(dependent_table.get())) out = std::make_shared( - dependent_table, dependent_metadata_snapshot, *insert_context, ASTPtr(), true); + dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true); else out = std::make_shared( - dependent_table, dependent_metadata_snapshot, *insert_context, ASTPtr()); + dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr()); views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr, 0 /* elapsed_ms */}); } @@ -127,7 +127,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( /// Do not push to destination table if the flag is set if (!no_destination) { - output = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), context); + output = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), getContext()); replicated_output = dynamic_cast(output.get()); } } @@ -155,7 +155,7 @@ void PushingToViewsBlockOutputStream::write(const Block & block) if (auto * live_view = dynamic_cast(storage.get())) { - StorageLiveView::writeIntoLiveView(*live_view, block, context); + StorageLiveView::writeIntoLiveView(*live_view, block, getContext()); } else { @@ -166,11 +166,11 @@ void PushingToViewsBlockOutputStream::write(const Block & block) } /// Don't process materialized views if this block is duplicate - if (!context.getSettingsRef().deduplicate_blocks_in_dependent_materialized_views && replicated_output && replicated_output->lastBlockIsDuplicate()) + if (!getContext()->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views && replicated_output && replicated_output->lastBlockIsDuplicate()) return; // Insert data into materialized views only after successful insert into main table - const Settings & settings = context.getSettingsRef(); + const Settings & settings = getContext()->getSettingsRef(); if (settings.parallel_view_processing && views.size() > 1) { // Push to views concurrently if enabled and more than one view is attached @@ -228,7 +228,7 @@ void PushingToViewsBlockOutputStream::writeSuffix() std::exception_ptr first_exception; - const Settings & settings = context.getSettingsRef(); + const Settings & settings = getContext()->getSettingsRef(); bool parallel_processing = false; /// Run writeSuffix() for views in separate thread pool. @@ -353,10 +353,9 @@ void PushingToViewsBlockOutputStream::process(const Block & block, ViewInfo & vi /// but it will contain single block (that is INSERT-ed into main table). /// InterpreterSelectQuery will do processing of alias columns. - Context local_context = *select_context; - local_context.addViewSource( - StorageValues::create( - storage->getStorageID(), metadata_snapshot->getColumns(), block, storage->getVirtuals())); + auto local_context = Context::createCopy(select_context); + local_context->addViewSource( + StorageValues::create(storage->getStorageID(), metadata_snapshot->getColumns(), block, storage->getVirtuals())); select.emplace(view.query, local_context, SelectQueryOptions()); in = std::make_shared(select->execute().getInputStream()); @@ -364,7 +363,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, ViewInfo & vi /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY /// and two-level aggregation is triggered). in = std::make_shared( - in, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); + in, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes); in = std::make_shared(in, view.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Name); } else diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index 6b32607b294..2ae941efc2e 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -1,9 +1,9 @@ #pragma once #include -#include #include #include +#include namespace Poco { @@ -14,18 +14,16 @@ namespace DB { class ReplicatedMergeTreeBlockOutputStream; -class Context; - /** Writes data to the specified table and to all dependent materialized views. */ -class PushingToViewsBlockOutputStream : public IBlockOutputStream +class PushingToViewsBlockOutputStream : public IBlockOutputStream, WithContext { public: PushingToViewsBlockOutputStream( const StoragePtr & storage_, const StorageMetadataPtr & metadata_snapshot_, - const Context & context_, + ContextPtr context_, const ASTPtr & query_ptr_, bool no_destination = false); @@ -43,7 +41,6 @@ private: ReplicatedMergeTreeBlockOutputStream * replicated_output = nullptr; Poco::Logger * log; - const Context & context; ASTPtr query_ptr; Stopwatch main_watch; @@ -57,8 +54,8 @@ private: }; std::vector views; - std::unique_ptr select_context; - std::unique_ptr insert_context; + ContextPtr select_context; + ContextPtr insert_context; void process(const Block & block, ViewInfo & view); }; diff --git a/src/DataStreams/RemoteBlockInputStream.cpp b/src/DataStreams/RemoteBlockInputStream.cpp index a62f7fca0b7..c633600d37f 100644 --- a/src/DataStreams/RemoteBlockInputStream.cpp +++ b/src/DataStreams/RemoteBlockInputStream.cpp @@ -6,7 +6,7 @@ namespace DB RemoteBlockInputStream::RemoteBlockInputStream( Connection & connection, - const String & query_, const Block & header_, const Context & context_, + const String & query_, const Block & header_, ContextPtr context_, const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_) : query_executor(connection, query_, header_, context_, throttler, scalars_, external_tables_, stage_) { @@ -15,7 +15,7 @@ RemoteBlockInputStream::RemoteBlockInputStream( RemoteBlockInputStream::RemoteBlockInputStream( std::vector && connections, - const String & query_, const Block & header_, const Context & context_, + const String & query_, const Block & header_, ContextPtr context_, const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_) : query_executor(std::move(connections), query_, header_, context_, throttler, scalars_, external_tables_, stage_) { @@ -24,7 +24,7 @@ RemoteBlockInputStream::RemoteBlockInputStream( RemoteBlockInputStream::RemoteBlockInputStream( const ConnectionPoolWithFailoverPtr & pool, - const String & query_, const Block & header_, const Context & context_, + const String & query_, const Block & header_, ContextPtr context_, const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_) : query_executor(pool, query_, header_, context_, throttler, scalars_, external_tables_, stage_) { diff --git a/src/DataStreams/RemoteBlockInputStream.h b/src/DataStreams/RemoteBlockInputStream.h index 5ef05ee99eb..b0029da91bb 100644 --- a/src/DataStreams/RemoteBlockInputStream.h +++ b/src/DataStreams/RemoteBlockInputStream.h @@ -25,21 +25,21 @@ public: /// Takes already set connection. RemoteBlockInputStream( Connection & connection, - const String & query_, const Block & header_, const Context & context_, + const String & query_, const Block & header_, ContextPtr context_, const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); /// Accepts several connections already taken from pool. RemoteBlockInputStream( std::vector && connections, - const String & query_, const Block & header_, const Context & context_, + const String & query_, const Block & header_, ContextPtr context_, const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); /// Takes a pool and gets one or several connections from it. RemoteBlockInputStream( const ConnectionPoolWithFailoverPtr & pool, - const String & query_, const Block & header_, const Context & context_, + const String & query_, const Block & header_, ContextPtr context_, const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 9cac638ceb5..4aa659854b9 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -28,41 +28,51 @@ namespace ErrorCodes RemoteQueryExecutor::RemoteQueryExecutor( Connection & connection, - const String & query_, const Block & header_, const Context & context_, - ThrottlerPtr throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_) - : header(header_), query(query_), context(context_) - , scalars(scalars_), external_tables(external_tables_), stage(stage_) + const String & query_, + const Block & header_, + ContextPtr context_, + ThrottlerPtr throttler, + const Scalars & scalars_, + const Tables & external_tables_, + QueryProcessingStage::Enum stage_) + : header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_) { create_connections = [this, &connection, throttler]() { - return std::make_unique(connection, context.getSettingsRef(), throttler); + return std::make_unique(connection, context->getSettingsRef(), throttler); }; } RemoteQueryExecutor::RemoteQueryExecutor( std::vector && connections_, - const String & query_, const Block & header_, const Context & context_, - const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_) - : header(header_), query(query_), context(context_) - , scalars(scalars_), external_tables(external_tables_), stage(stage_) + const String & query_, + const Block & header_, + ContextPtr context_, + const ThrottlerPtr & throttler, + const Scalars & scalars_, + const Tables & external_tables_, + QueryProcessingStage::Enum stage_) + : header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_) { - create_connections = [this, connections_, throttler]() mutable - { - return std::make_unique( - std::move(connections_), context.getSettingsRef(), throttler); + create_connections = [this, connections_, throttler]() mutable { + return std::make_unique(std::move(connections_), context->getSettingsRef(), throttler); }; } RemoteQueryExecutor::RemoteQueryExecutor( const ConnectionPoolWithFailoverPtr & pool, - const String & query_, const Block & header_, const Context & context_, - const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_) - : header(header_), query(query_), context(context_) - , scalars(scalars_), external_tables(external_tables_), stage(stage_) + const String & query_, + const Block & header_, + ContextPtr context_, + const ThrottlerPtr & throttler, + const Scalars & scalars_, + const Tables & external_tables_, + QueryProcessingStage::Enum stage_) + : header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_) { create_connections = [this, pool, throttler]()->std::unique_ptr { - const Settings & current_settings = context.getSettingsRef(); + const Settings & current_settings = context->getSettingsRef(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); #if defined(OS_LINUX) @@ -157,7 +167,7 @@ void RemoteQueryExecutor::sendQuery() connections = create_connections(); - const auto & settings = context.getSettingsRef(); + const auto & settings = context->getSettingsRef(); if (settings.skip_unavailable_shards && 0 == connections->size()) return; @@ -176,7 +186,7 @@ void RemoteQueryExecutor::sendQuery() was_cancelled = false; auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings); - ClientInfo modified_client_info = context.getClientInfo(); + ClientInfo modified_client_info = context->getClientInfo(); modified_client_info.query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; if (CurrentThread::isInitialized()) { @@ -205,7 +215,7 @@ Block RemoteQueryExecutor::read() { sendQuery(); - if (context.getSettingsRef().skip_unavailable_shards && (0 == connections->size())) + if (context->getSettingsRef().skip_unavailable_shards && (0 == connections->size())) return {}; } @@ -231,7 +241,7 @@ std::variant RemoteQueryExecutor::read(std::unique_ptr { sendQuery(); - if (context.getSettingsRef().skip_unavailable_shards && (0 == connections->size())) + if (context->getSettingsRef().skip_unavailable_shards && (0 == connections->size())) return Block(); } @@ -363,8 +373,8 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) bool RemoteQueryExecutor::setPartUUIDs(const std::vector & uuids) { - Context & query_context = const_cast(context).getQueryContext(); - auto duplicates = query_context.getPartUUIDs()->add(uuids); + auto query_context = context->getQueryContext(); + auto duplicates = query_context->getPartUUIDs()->add(uuids); if (!duplicates.empty()) { diff --git a/src/DataStreams/RemoteQueryExecutor.h b/src/DataStreams/RemoteQueryExecutor.h index f55a023231b..45a633230b7 100644 --- a/src/DataStreams/RemoteQueryExecutor.h +++ b/src/DataStreams/RemoteQueryExecutor.h @@ -35,21 +35,21 @@ public: /// Takes already set connection. RemoteQueryExecutor( Connection & connection, - const String & query_, const Block & header_, const Context & context_, + const String & query_, const Block & header_, ContextPtr context_, ThrottlerPtr throttler_ = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); /// Accepts several connections already taken from pool. RemoteQueryExecutor( std::vector && connections_, - const String & query_, const Block & header_, const Context & context_, + const String & query_, const Block & header_, ContextPtr context_, const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); /// Takes a pool and gets one or several connections from it. RemoteQueryExecutor( const ConnectionPoolWithFailoverPtr & pool, - const String & query_, const Block & header_, const Context & context_, + const String & query_, const Block & header_, ContextPtr context_, const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); @@ -108,8 +108,8 @@ private: std::unique_ptr connections; const String query; - String query_id = ""; - Context context; + String query_id; + ContextPtr context; ProgressCallback progress_callback; ProfileInfoCallback profile_info_callback; diff --git a/src/DataStreams/TTLAggregationAlgorithm.cpp b/src/DataStreams/TTLAggregationAlgorithm.cpp index ebe08159c55..9a1cf45772f 100644 --- a/src/DataStreams/TTLAggregationAlgorithm.cpp +++ b/src/DataStreams/TTLAggregationAlgorithm.cpp @@ -28,12 +28,12 @@ TTLAggregationAlgorithm::TTLAggregationAlgorithm( descr.arguments.push_back(header.getPositionByName(name)); columns_for_aggregator.resize(description.aggregate_descriptions.size()); - const Settings & settings = storage_.global_context.getSettingsRef(); + const Settings & settings = storage_.getContext()->getSettingsRef(); Aggregator::Params params(header, keys, aggregates, false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0, settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - storage_.global_context.getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); + storage_.getContext()->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); aggregator = std::make_unique(params); } diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 4f141a03475..8b31da6d2f1 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -68,8 +68,9 @@ TTLBlockInputStream::TTLBlockInputStream( auto default_ast = it->second.expression->clone(); default_ast = addTypeConversionToAST(std::move(default_ast), column.type->getName()); - auto syntax_result = TreeRewriter(storage_.global_context).analyze(default_ast, metadata_snapshot_->getColumns().getAllPhysical()); - default_expression = ExpressionAnalyzer{default_ast, syntax_result, storage_.global_context}.getActions(true); + auto syntax_result + = TreeRewriter(storage_.getContext()).analyze(default_ast, metadata_snapshot_->getColumns().getAllPhysical()); + default_expression = ExpressionAnalyzer{default_ast, syntax_result, storage_.getContext()}.getActions(true); default_column_name = default_ast->getColumnName(); } diff --git a/src/DataTypes/DataTypeDecimalBase.cpp b/src/DataTypes/DataTypeDecimalBase.cpp index 5f64fca6704..683710b8880 100644 --- a/src/DataTypes/DataTypeDecimalBase.cpp +++ b/src/DataTypes/DataTypeDecimalBase.cpp @@ -19,9 +19,14 @@ namespace ErrorCodes { } - -bool decimalCheckComparisonOverflow(const Context & context) { return context.getSettingsRef().decimal_check_overflow; } -bool decimalCheckArithmeticOverflow(const Context & context) { return context.getSettingsRef().decimal_check_overflow; } +bool decimalCheckComparisonOverflow(ContextPtr context) +{ + return context->getSettingsRef().decimal_check_overflow; +} +bool decimalCheckArithmeticOverflow(ContextPtr context) +{ + return context->getSettingsRef().decimal_check_overflow; +} template Field DataTypeDecimalBase::getDefault() const diff --git a/src/DataTypes/DataTypeDecimalBase.h b/src/DataTypes/DataTypeDecimalBase.h index 85cb9fa8363..9e44310a0ff 100644 --- a/src/DataTypes/DataTypeDecimalBase.h +++ b/src/DataTypes/DataTypeDecimalBase.h @@ -1,11 +1,12 @@ #pragma once -#include #include #include #include #include +#include +#include #include @@ -17,9 +18,8 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; } -class Context; -bool decimalCheckComparisonOverflow(const Context & context); -bool decimalCheckArithmeticOverflow(const Context & context); +bool decimalCheckComparisonOverflow(ContextPtr context); +bool decimalCheckArithmeticOverflow(ContextPtr context); inline UInt32 leastDecimalPrecisionFor(TypeIndex int_type) { diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index eee3f22f9f1..c28de15565c 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -156,7 +156,7 @@ void DataTypeFactory::registerSimpleDataTypeCustom(const String &name, SimpleCre const DataTypeFactory::Value & DataTypeFactory::findCreatorByName(const String & family_name) const { - const Context * query_context = nullptr; + ContextPtr query_context; if (CurrentThread::isInitialized()) query_context = CurrentThread::get().getQueryContext(); diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index b4222a7e349..c9e533dfafb 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -35,10 +35,10 @@ public: UUID uuid() const override { return table()->getStorageID().uuid; } }; -DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, const Context & context_) +DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, ContextPtr context_) : DatabaseOrdinary(name_, std::move(metadata_path_), "store/", logger_name, context_) - , path_to_table_symlinks(global_context.getPath() + "data/" + escapeForFileName(name_) + "/") - , path_to_metadata_symlink(global_context.getPath() + "metadata/" + escapeForFileName(name_)) + , path_to_table_symlinks(getContext()->getPath() + "data/" + escapeForFileName(name_) + "/") + , path_to_metadata_symlink(getContext()->getPath() + "metadata/" + escapeForFileName(name_)) , db_uuid(uuid) { assert(db_uuid != UUIDHelpers::Nil); @@ -46,7 +46,7 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, c tryCreateMetadataSymlink(); } -DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const Context & context_) +DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, ContextPtr context_) : DatabaseAtomic(name_, std::move(metadata_path_), uuid, "DatabaseAtomic (" + name_ + ")", context_) { } @@ -68,7 +68,7 @@ String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const return tmp; } -void DatabaseAtomic::drop(const Context &) +void DatabaseAtomic::drop(ContextPtr) { assert(tables.empty()); try @@ -106,13 +106,13 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) return table; } -void DatabaseAtomic::dropTable(const Context & context, const String & table_name, bool no_delay) +void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool no_delay) { - if (auto * mv = dynamic_cast(tryGetTable(table_name, context).get())) + if (auto * mv = dynamic_cast(tryGetTable(table_name, local_context).get())) { /// Remove the inner table (if any) to avoid deadlock /// (due to attempt to execute DROP from the worker thread) - mv->dropInnerTable(no_delay, context); + mv->dropInnerTable(no_delay, local_context); } String table_metadata_path = getObjectMetadataPath(table_name); @@ -122,8 +122,8 @@ void DatabaseAtomic::dropTable(const Context & context, const String & table_nam std::unique_lock lock(mutex); table = getTableUnlocked(table_name, lock); table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID()); - auto txn = context.getZooKeeperMetadataTransaction(); - if (txn && !context.isInternalSubquery()) + auto txn = local_context->getZooKeeperMetadataTransaction(); + if (txn && !local_context->isInternalSubquery()) txn->commit(); /// Commit point (a sort of) for Replicated database /// NOTE: replica will be lost if server crashes before the following rename @@ -144,7 +144,7 @@ void DatabaseAtomic::dropTable(const Context & context, const String & table_nam DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay); } -void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database, +void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_name, IDatabase & to_database, const String & to_table_name, bool exchange, bool dictionary) { if (typeid(*this) != typeid(to_database)) @@ -152,7 +152,7 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n if (!typeid_cast(&to_database)) throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); /// Allow moving tables between Atomic and Ordinary (with table lock) - DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary); + DatabaseOnDisk::renameTable(local_context, table_name, to_database, to_table_name, exchange, dictionary); return; } @@ -248,8 +248,8 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n } /// Table renaming actually begins here - auto txn = context.getZooKeeperMetadataTransaction(); - if (txn && !context.isInternalSubquery()) + auto txn = local_context->getZooKeeperMetadataTransaction(); + if (txn && !local_context->isInternalSubquery()) txn->commit(); /// Commit point (a sort of) for Replicated database /// NOTE: replica will be lost if server crashes before the following rename @@ -291,7 +291,7 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, const String & table_metadata_tmp_path, const String & table_metadata_path, - const Context & query_context) + ContextPtr query_context) { DetachedTables not_in_use; auto table_data_path = getTableDataPath(query); @@ -309,8 +309,8 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora DatabaseCatalog::instance().addUUIDMapping(query.uuid); locked_uuid = true; - auto txn = query_context.getZooKeeperMetadataTransaction(); - if (txn && !query_context.isInternalSubquery()) + auto txn = query_context->getZooKeeperMetadataTransaction(); + if (txn && !query_context->isInternalSubquery()) txn->commit(); /// Commit point (a sort of) for Replicated database /// NOTE: replica will be lost if server crashes before the following renameNoReplace(...) @@ -333,7 +333,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora } void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, - const String & /*statement*/, const Context & query_context) + const String & /*statement*/, ContextPtr query_context) { bool check_file_exists = true; SCOPE_EXIT({ std::error_code code; if (check_file_exists) std::filesystem::remove(table_metadata_tmp_path, code); }); @@ -344,8 +344,8 @@ void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & if (table_id.uuid != actual_table_id.uuid) throw Exception("Cannot alter table because it was renamed", ErrorCodes::CANNOT_ASSIGN_ALTER); - auto txn = query_context.getZooKeeperMetadataTransaction(); - if (txn && !query_context.isInternalSubquery()) + auto txn = query_context->getZooKeeperMetadataTransaction(); + if (txn && !query_context->isInternalSubquery()) txn->commit(); /// Commit point (a sort of) for Replicated database /// NOTE: replica will be lost if server crashes before the following rename @@ -409,26 +409,27 @@ void DatabaseAtomic::assertCanBeDetached(bool cleanup) "because some tables are still in use. Retry later.", ErrorCodes::DATABASE_NOT_EMPTY); } -DatabaseTablesIteratorPtr DatabaseAtomic::getTablesIterator(const Context & context, const IDatabase::FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr +DatabaseAtomic::getTablesIterator(ContextPtr local_context, const IDatabase::FilterByNameFunction & filter_by_table_name) { - auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(context, filter_by_table_name); + auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(local_context, filter_by_table_name); return std::make_unique(std::move(typeid_cast(*base_iter))); } UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const { - if (auto table = tryGetTable(table_name, global_context)) + if (auto table = tryGetTable(table_name, getContext())) return table->getStorageID().uuid; return UUIDHelpers::Nil; } -void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) +void DatabaseAtomic::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach) { /// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken if (has_force_restore_data_flag) Poco::File(path_to_table_symlinks).remove(true); - DatabaseOrdinary::loadStoredObjects(context, has_force_restore_data_flag, force_attach); + DatabaseOrdinary::loadStoredObjects(local_context, has_force_restore_data_flag, force_attach); if (has_force_restore_data_flag) { @@ -449,7 +450,7 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & try { String link = path_to_table_symlinks + escapeForFileName(table_name); - Poco::File data = Poco::Path(global_context.getPath()).makeAbsolute().toString() + actual_data_path; + Poco::File data = Poco::Path(getContext()->getPath()).makeAbsolute().toString() + actual_data_path; if (!if_data_path_exist || data.exists()) data.linkTo(link, Poco::File::LINK_SYMBOLIC); } @@ -509,8 +510,8 @@ void DatabaseAtomic::renameDatabase(const String & new_name) } auto new_name_escaped = escapeForFileName(new_name); - auto old_database_metadata_path = global_context.getPath() + "metadata/" + escapeForFileName(getDatabaseName()) + ".sql"; - auto new_database_metadata_path = global_context.getPath() + "metadata/" + new_name_escaped + ".sql"; + auto old_database_metadata_path = getContext()->getPath() + "metadata/" + escapeForFileName(getDatabaseName()) + ".sql"; + auto new_database_metadata_path = getContext()->getPath() + "metadata/" + new_name_escaped + ".sql"; renameNoReplace(old_database_metadata_path, new_database_metadata_path); String old_path_to_table_symlinks; @@ -535,9 +536,9 @@ void DatabaseAtomic::renameDatabase(const String & new_name) renameDictionaryInMemoryUnlocked(old_name, name); } - path_to_metadata_symlink = global_context.getPath() + "metadata/" + new_name_escaped; + path_to_metadata_symlink = getContext()->getPath() + "metadata/" + new_name_escaped; old_path_to_table_symlinks = path_to_table_symlinks; - path_to_table_symlinks = global_context.getPath() + "data/" + new_name_escaped + "/"; + path_to_table_symlinks = getContext()->getPath() + "data/" + new_name_escaped + "/"; } Poco::File(old_path_to_table_symlinks).renameTo(path_to_table_symlinks); diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 09cdf269b35..d35495fc962 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -19,8 +19,8 @@ namespace DB class DatabaseAtomic : public DatabaseOrdinary { public: - DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, const Context & context_); - DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const Context & context_); + DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, ContextPtr context_); + DatabaseAtomic(String name_, String metadata_path_, UUID uuid, ContextPtr context_); String getEngineName() const override { return "Atomic"; } UUID getUUID() const override { return db_uuid; } @@ -28,14 +28,14 @@ public: void renameDatabase(const String & new_name) override; void renameTable( - const Context & context, + ContextPtr context, const String & table_name, IDatabase & to_database, const String & to_table_name, bool exchange, bool dictionary) override; - void dropTable(const Context & context, const String & table_name, bool no_delay) override; + void dropTable(ContextPtr context, const String & table_name, bool no_delay) override; void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override; StoragePtr detachTable(const String & name) override; @@ -43,11 +43,11 @@ public: String getTableDataPath(const String & table_name) const override; String getTableDataPath(const ASTCreateQuery & query) const override; - void drop(const Context & /*context*/) override; + void drop(ContextPtr /*context*/) override; - DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override; - void loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) override; + void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override; /// Atomic database cannot be detached if there is detached table which still in use void assertCanBeDetached(bool cleanup) override; @@ -61,12 +61,12 @@ public: void setDetachedTableNotInUseForce(const UUID & uuid); protected: - void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, const Context & query_context) override; + void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, ContextPtr query_context) override; void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, - const String & table_metadata_tmp_path, const String & table_metadata_path, const Context & query_context) override; + const String & table_metadata_tmp_path, const String & table_metadata_path, ContextPtr query_context) override; void assertDetachedTableNotInUse(const UUID & uuid); - typedef std::unordered_map DetachedTables; + using DetachedTables = std::unordered_map; [[nodiscard]] DetachedTables cleanupDetachedTables(); void tryCreateMetadataSymlink(); @@ -74,7 +74,7 @@ protected: void renameDictionaryInMemoryUnlocked(const StorageID & old_name, const StorageID & new_name); //TODO store path in DatabaseWithOwnTables::tables - typedef std::unordered_map NameToPathMap; + using NameToPathMap = std::unordered_map; NameToPathMap table_name_to_path; DetachedTables detached_tables; diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index ff5510f0bf9..c00201145eb 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -44,17 +44,16 @@ namespace } } -DatabaseDictionary::DatabaseDictionary(const String & name_, const Context & context_) - : IDatabase(name_) +DatabaseDictionary::DatabaseDictionary(const String & name_, ContextPtr context_) + : IDatabase(name_), WithContext(context_->getGlobalContext()) , log(&Poco::Logger::get("DatabaseDictionary(" + database_name + ")")) - , global_context(context_.getGlobalContext()) { } Tables DatabaseDictionary::listTables(const FilterByNameFunction & filter_by_name) { Tables tables; - auto load_results = global_context.getExternalDictionariesLoader().getLoadResults(filter_by_name); + auto load_results = getContext()->getExternalDictionariesLoader().getLoadResults(filter_by_name); String db_name = getDatabaseName(); for (auto & load_result : load_results) { @@ -65,34 +64,34 @@ Tables DatabaseDictionary::listTables(const FilterByNameFunction & filter_by_nam return tables; } -bool DatabaseDictionary::isTableExist(const String & table_name, const Context &) const +bool DatabaseDictionary::isTableExist(const String & table_name, ContextPtr) const { - return global_context.getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST; + return getContext()->getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST; } -StoragePtr DatabaseDictionary::tryGetTable(const String & table_name, const Context &) const +StoragePtr DatabaseDictionary::tryGetTable(const String & table_name, ContextPtr) const { - auto load_result = global_context.getExternalDictionariesLoader().getLoadResult(table_name); + auto load_result = getContext()->getExternalDictionariesLoader().getLoadResult(table_name); return createStorageDictionary(getDatabaseName(), load_result); } -DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) { return std::make_unique(listTables(filter_by_table_name), getDatabaseName()); } bool DatabaseDictionary::empty() const { - return !global_context.getExternalDictionariesLoader().hasObjects(); + return !getContext()->getExternalDictionariesLoader().hasObjects(); } -ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const +ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, ContextPtr, bool throw_on_error) const { String query; { WriteBufferFromString buffer(query); - auto load_result = global_context.getExternalDictionariesLoader().getLoadResult(table_name); + auto load_result = getContext()->getExternalDictionariesLoader().getLoadResult(table_name); if (!load_result.config) { if (throw_on_error) @@ -106,7 +105,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, co buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")"; } - auto settings = global_context.getSettingsRef(); + auto settings = getContext()->getSettingsRef(); ParserCreateQuery parser; const char * pos = query.data(); std::string error_message; @@ -126,7 +125,7 @@ ASTPtr DatabaseDictionary::getCreateDatabaseQuery() const WriteBufferFromString buffer(query); buffer << "CREATE DATABASE " << backQuoteIfNeed(getDatabaseName()) << " ENGINE = Dictionary"; } - auto settings = global_context.getSettingsRef(); + auto settings = getContext()->getSettingsRef(); ParserCreateQuery parser; return parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth); } diff --git a/src/Databases/DatabaseDictionary.h b/src/Databases/DatabaseDictionary.h index 2cfc6ef3285..06402a96381 100644 --- a/src/Databases/DatabaseDictionary.h +++ b/src/Databases/DatabaseDictionary.h @@ -1,11 +1,12 @@ #pragma once -#include -#include #include #include #include +#include +#include + namespace Poco { @@ -19,21 +20,21 @@ namespace DB /* Database to store StorageDictionary tables * automatically creates tables for all dictionaries */ -class DatabaseDictionary final : public IDatabase +class DatabaseDictionary final : public IDatabase, WithContext { public: - DatabaseDictionary(const String & name_, const Context & context_); + DatabaseDictionary(const String & name_, ContextPtr context_); String getEngineName() const override { return "Dictionary"; } - bool isTableExist(const String & table_name, const Context & context) const override; + bool isTableExist(const String & table_name, ContextPtr context) const override; - StoragePtr tryGetTable(const String & table_name, const Context & context) const override; + StoragePtr tryGetTable(const String & table_name, ContextPtr context) const override; - DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override; bool empty() const override; @@ -44,11 +45,10 @@ public: void shutdown() override; protected: - ASTPtr getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const override; + ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr context, bool throw_on_error) const override; private: Poco::Logger * log; - const Context & global_context; Tables listTables(const FilterByNameFunction & filter_by_name); }; diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 523ce7d925e..03c2849ac94 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -30,6 +30,7 @@ #endif #if USE_MYSQL || USE_LIBPQXX +#include #include #include #endif @@ -50,7 +51,7 @@ namespace ErrorCodes extern const int CANNOT_CREATE_DATABASE; } -DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & metadata_path, Context & context) +DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context) { bool created = false; @@ -65,8 +66,8 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m DatabasePtr impl = getImpl(create, metadata_path, context); - if (impl && context.hasQueryContext() && context.getSettingsRef().log_queries) - context.getQueryContext().addQueryFactoriesInfo(Context::QueryLogFactories::Database, impl->getEngineName()); + if (impl && context->hasQueryContext() && context->getSettingsRef().log_queries) + context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Database, impl->getEngineName()); return impl; @@ -91,7 +92,7 @@ static inline ValueType safeGetLiteralValue(const ASTPtr &ast, const String &eng return ast->as()->value.safeGet(); } -DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String & metadata_path, Context & context) +DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context) { auto * engine_define = create.storage; const String & database_name = create.database; @@ -133,19 +134,20 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String ASTs & arguments = engine->arguments->children; arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context); - const auto & host_name_and_port = safeGetLiteralValue(arguments[0], engine_name); + const auto & host_port = safeGetLiteralValue(arguments[0], engine_name); const auto & mysql_database_name = safeGetLiteralValue(arguments[1], engine_name); const auto & mysql_user_name = safeGetLiteralValue(arguments[2], engine_name); const auto & mysql_user_password = safeGetLiteralValue(arguments[3], engine_name); try { - const auto & [remote_host_name, remote_port] = parseAddress(host_name_and_port, 3306); - auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password, remote_port); - if (engine_name == "MySQL") { auto mysql_database_settings = std::make_unique(); + /// Split into replicas if needed. + size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; + auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306); + auto mysql_pool = mysqlxx::PoolWithFailover(mysql_database_name, addresses, mysql_user_name, mysql_user_password); mysql_database_settings->loadFromQueryContext(context); mysql_database_settings->loadFromQuery(*engine_define); /// higher priority @@ -154,7 +156,10 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_database_settings), std::move(mysql_pool)); } + const auto & [remote_host_name, remote_port] = parseAddress(host_port, 3306); MySQLClient client(remote_host_name, remote_port, mysql_user_name, mysql_user_password); + auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password); + auto materialize_mode_settings = std::make_unique(); @@ -204,9 +209,9 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String String shard_name = safeGetLiteralValue(arguments[1], "Replicated"); String replica_name = safeGetLiteralValue(arguments[2], "Replicated"); - zookeeper_path = context.getMacros()->expand(zookeeper_path); - shard_name = context.getMacros()->expand(shard_name); - replica_name = context.getMacros()->expand(replica_name); + zookeeper_path = context->getMacros()->expand(zookeeper_path); + shard_name = context->getMacros()->expand(shard_name); + replica_name = context->getMacros()->expand(replica_name); DatabaseReplicatedSettings database_replicated_settings{}; if (engine_define->settings) @@ -243,15 +248,17 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (engine->arguments->children.size() == 5) use_table_cache = safeGetLiteralValue(engine_args[4], engine_name); - auto parsed_host_port = parseAddress(host_port, 5432); + /// Split into replicas if needed. + size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; + auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432); /// no connection is made here - auto connection_pool = std::make_shared( + auto connection_pool = std::make_shared( postgres_database_name, - parsed_host_port.first, parsed_host_port.second, + addresses, username, password, - context.getSettingsRef().postgresql_connection_pool_size, - context.getSettingsRef().postgresql_connection_pool_wait_timeout); + context->getSettingsRef().postgresql_connection_pool_size, + context->getSettingsRef().postgresql_connection_pool_wait_timeout); return std::make_shared( context, metadata_path, engine_define, database_name, postgres_database_name, connection_pool, use_table_cache); diff --git a/src/Databases/DatabaseFactory.h b/src/Databases/DatabaseFactory.h index 88d33dc1cd5..8992ea27093 100644 --- a/src/Databases/DatabaseFactory.h +++ b/src/Databases/DatabaseFactory.h @@ -11,9 +11,9 @@ class ASTCreateQuery; class DatabaseFactory { public: - static DatabasePtr get(const ASTCreateQuery & create, const String & metadata_path, Context & context); + static DatabasePtr get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context); - static DatabasePtr getImpl(const ASTCreateQuery & create, const String & metadata_path, Context & context); + static DatabasePtr getImpl(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context); }; } diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index f297bf2c82f..d498cb96062 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include @@ -27,7 +27,7 @@ namespace ErrorCodes } -DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_) +DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, ContextPtr context_) : DatabaseOnDisk(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseLazy (" + name_ + ")", context_) , expiration_time(expiration_time_) { @@ -35,11 +35,11 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, void DatabaseLazy::loadStoredObjects( - Context & context, + ContextPtr local_context, bool /* has_force_restore_data_flag */, bool /*force_attach*/) { - iterateMetadataFiles(context, [this](const String & file_name) + iterateMetadataFiles(local_context, [this](const String & file_name) { const std::string table_name = file_name.substr(0, file_name.size() - 4); @@ -56,15 +56,15 @@ void DatabaseLazy::loadStoredObjects( void DatabaseLazy::createTable( - const Context & context, + ContextPtr local_context, const String & table_name, 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); + DatabaseOnDisk::createTable(local_context, table_name, table, query); /// DatabaseOnDisk::createTable renames file, so we need to get new metadata_modification_time. std::lock_guard lock(mutex); @@ -74,24 +74,24 @@ void DatabaseLazy::createTable( } void DatabaseLazy::dropTable( - const Context & context, + ContextPtr local_context, const String & table_name, bool no_delay) { - SCOPE_EXIT({ clearExpiredTables(); }); - DatabaseOnDisk::dropTable(context, table_name, no_delay); + SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); }); + DatabaseOnDisk::dropTable(local_context, table_name, no_delay); } void DatabaseLazy::renameTable( - const Context & context, + ContextPtr local_context, const String & table_name, IDatabase & to_database, const String & to_table_name, bool exchange, bool dictionary) { - SCOPE_EXIT({ clearExpiredTables(); }); - DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary); + SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); }); + DatabaseOnDisk::renameTable(local_context, table_name, to_database, to_table_name, exchange, dictionary); } @@ -105,7 +105,7 @@ time_t DatabaseLazy::getObjectMetadataModificationTime(const String & table_name } void DatabaseLazy::alterTable( - const Context & /* context */, + ContextPtr /* context */, const StorageID & /*table_id*/, const StorageInMemoryMetadata & /* metadata */) { @@ -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); @@ -142,7 +142,7 @@ StoragePtr DatabaseLazy::tryGetTable(const String & table_name) const return loadTable(table_name); } -DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) { std::lock_guard lock(mutex); Strings filtered_tables; @@ -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)); @@ -233,9 +233,9 @@ StoragePtr DatabaseLazy::loadTable(const String & table_name) const try { StoragePtr table; - Context context_copy(global_context); /// some tables can change context, but not LogTables + auto context_copy = Context::createCopy(context); /// some tables can change context, but not LogTables - auto ast = parseQueryFromMetadata(log, global_context, table_metadata_path, /*throw_on_error*/ true, /*remove_empty*/false); + auto ast = parseQueryFromMetadata(log, getContext(), table_metadata_path, /*throw_on_error*/ true, /*remove_empty*/false); if (ast) { const auto & ast_create = ast->as(); diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 2d091297c91..99a71b342fa 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -18,7 +18,7 @@ class Context; class DatabaseLazy final : public DatabaseOnDisk { public: - DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_); + DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, ContextPtr context_); String getEngineName() const override { return "Lazy"; } @@ -27,22 +27,22 @@ public: bool canContainDistributedTables() const override { return false; } void loadStoredObjects( - Context & context, + ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override; void createTable( - const Context & context, + ContextPtr context, const String & table_name, const StoragePtr & table, const ASTPtr & query) override; void dropTable( - const Context & context, + ContextPtr context, const String & table_name, bool no_delay) override; void renameTable( - const Context & context, + ContextPtr context, const String & table_name, IDatabase & to_database, const String & to_table_name, @@ -50,21 +50,21 @@ public: bool dictionary) override; void alterTable( - const Context & context, + ContextPtr context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; time_t getObjectMetadataModificationTime(const String & table_name) const override; - bool isTableExist(const String & table_name, const Context &) const override { return isTableExist(table_name); } + bool isTableExist(const String & table_name, ContextPtr) const override { return isTableExist(table_name); } bool isTableExist(const String & table_name) const; - StoragePtr tryGetTable(const String & table_name, const Context &) const override { return tryGetTable(table_name); } + StoragePtr tryGetTable(const String & table_name, ContextPtr) const override { return tryGetTable(table_name); } StoragePtr tryGetTable(const String & table_name) const; bool empty() const override; - DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override; void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) override; diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 357acb32371..f21a145df55 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -16,13 +16,13 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } -DatabaseMemory::DatabaseMemory(const String & name_, const Context & context) - : DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")", context) +DatabaseMemory::DatabaseMemory(const String & name_, ContextPtr context_) + : DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")", context_) , data_path("data/" + escapeForFileName(database_name) + "/") {} void DatabaseMemory::createTable( - const Context & /*context*/, + ContextPtr /*context*/, const String & table_name, const StoragePtr & table, const ASTPtr & query) @@ -33,7 +33,7 @@ void DatabaseMemory::createTable( } void DatabaseMemory::dropTable( - const Context & /*context*/, + ContextPtr /*context*/, const String & table_name, bool /*no_delay*/) { @@ -67,7 +67,7 @@ ASTPtr DatabaseMemory::getCreateDatabaseQuery() const return create_query; } -ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const +ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, ContextPtr, bool throw_on_error) const { std::lock_guard lock{mutex}; auto it = create_queries.find(table_name); @@ -83,15 +83,15 @@ ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, const UUID DatabaseMemory::tryGetTableUUID(const String & table_name) const { - if (auto table = tryGetTable(table_name, global_context)) + if (auto table = tryGetTable(table_name, getContext())) return table->getStorageID().uuid; return UUIDHelpers::Nil; } -void DatabaseMemory::drop(const Context & context) +void DatabaseMemory::drop(ContextPtr local_context) { /// Remove data on explicit DROP DATABASE - std::filesystem::remove_all(context.getPath() + data_path); + std::filesystem::remove_all(local_context->getPath() + data_path); } } diff --git a/src/Databases/DatabaseMemory.h b/src/Databases/DatabaseMemory.h index 40cc808e42b..8c9cf86ec19 100644 --- a/src/Databases/DatabaseMemory.h +++ b/src/Databases/DatabaseMemory.h @@ -19,22 +19,22 @@ namespace DB class DatabaseMemory final : public DatabaseWithOwnTablesBase { public: - DatabaseMemory(const String & name_, const Context & context); + DatabaseMemory(const String & name_, ContextPtr context); String getEngineName() const override { return "Memory"; } void createTable( - const Context & context, + ContextPtr context, const String & table_name, const StoragePtr & table, const ASTPtr & query) override; void dropTable( - const Context & context, + ContextPtr context, const String & table_name, bool no_delay) override; - ASTPtr getCreateTableQueryImpl(const String & name, const Context & context, bool throw_on_error) const override; + ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override; ASTPtr getCreateDatabaseQuery() const override; /// DatabaseMemory allows to create tables, which store data on disk. @@ -46,7 +46,7 @@ public: UUID tryGetTableUUID(const String & table_name) const override; - void drop(const Context & context) override; + void drop(ContextPtr context) override; private: String data_path; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 79373128713..358f9030db5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -46,7 +46,7 @@ std::pair createTableFromAST( ASTCreateQuery ast_create_query, const String & database_name, const String & table_data_path_relative, - Context & context, + ContextPtr context, bool has_force_restore_data_flag) { ast_create_query.attach = true; @@ -79,7 +79,7 @@ std::pair createTableFromAST( ast_create_query, table_data_path_relative, context, - context.getGlobalContext(), + context->getGlobalContext(), columns, constraints, has_force_restore_data_flag) @@ -188,23 +188,23 @@ DatabaseOnDisk::DatabaseOnDisk( const String & metadata_path_, const String & data_path_, const String & logger, - const Context & context) - : DatabaseWithOwnTablesBase(name, logger, context) + ContextPtr local_context) + : DatabaseWithOwnTablesBase(name, logger, local_context) , metadata_path(metadata_path_) , data_path(data_path_) { - Poco::File(context.getPath() + data_path).createDirectories(); + Poco::File(local_context->getPath() + data_path).createDirectories(); Poco::File(metadata_path).createDirectories(); } void DatabaseOnDisk::createTable( - const Context & context, + ContextPtr local_context, const String & table_name, const StoragePtr & table, const ASTPtr & query) { - const auto & settings = context.getSettingsRef(); + const auto & settings = local_context->getSettingsRef(); const auto & create = query->as(); assert(table_name == create.table); @@ -221,17 +221,19 @@ void DatabaseOnDisk::createTable( /// But there is protection from it - see using DDLGuard in InterpreterCreateQuery. if (isDictionaryExist(table_name)) - throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS, "Dictionary {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name)); + throw Exception( + ErrorCodes::DICTIONARY_ALREADY_EXISTS, "Dictionary {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name)); - if (isTableExist(table_name, global_context)) - throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name)); + if (isTableExist(table_name, getContext())) + throw Exception( + ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name)); String table_metadata_path = getObjectMetadataPath(table_name); if (create.attach_short_syntax) { /// Metadata already exists, table was detached - removeDetachedPermanentlyFlag(context, table_name, table_metadata_path, true); + removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, true); attachTable(table_name, table, getTableDataPath(create)); return; } @@ -241,7 +243,7 @@ void DatabaseOnDisk::createTable( if (create.attach && Poco::File(table_metadata_path).exists()) { - ASTPtr ast_detached = parseQueryFromMetadata(log, context, table_metadata_path); + ASTPtr ast_detached = parseQueryFromMetadata(log, local_context, table_metadata_path); auto & create_detached = ast_detached->as(); // either both should be Nil, either values should be equal @@ -268,14 +270,14 @@ void DatabaseOnDisk::createTable( out.close(); } - commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, context); + commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, local_context); - removeDetachedPermanentlyFlag(context, table_name, table_metadata_path, false); + removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, false); } /// If the table was detached permanently we will have a flag file with /// .sql.detached extension, is not needed anymore since we attached the table back -void DatabaseOnDisk::removeDetachedPermanentlyFlag(const Context &, const String & table_name, const String & table_metadata_path, bool) const +void DatabaseOnDisk::removeDetachedPermanentlyFlag(ContextPtr, const String & table_name, const String & table_metadata_path, bool) const { try { @@ -293,7 +295,7 @@ void DatabaseOnDisk::removeDetachedPermanentlyFlag(const Context &, const String void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, const String & table_metadata_tmp_path, const String & table_metadata_path, - const Context & /*query_context*/) + ContextPtr /*query_context*/) { try { @@ -311,7 +313,7 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora } } -void DatabaseOnDisk::detachTablePermanently(const Context &, const String & table_name) +void DatabaseOnDisk::detachTablePermanently(ContextPtr, const String & table_name) { auto table = detachTable(table_name); @@ -327,7 +329,7 @@ void DatabaseOnDisk::detachTablePermanently(const Context &, const String & tabl } } -void DatabaseOnDisk::dropTable(const Context & context, const String & table_name, bool /*no_delay*/) +void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_name, bool /*no_delay*/) { String table_metadata_path = getObjectMetadataPath(table_name); String table_metadata_path_drop = table_metadata_path + drop_suffix; @@ -349,7 +351,7 @@ void DatabaseOnDisk::dropTable(const Context & context, const String & table_nam table->drop(); table->is_dropped = true; - Poco::File table_data_dir{context.getPath() + table_data_path_relative}; + Poco::File table_data_dir{local_context->getPath() + table_data_path_relative}; if (table_data_dir.exists()) table_data_dir.remove(true); } @@ -387,7 +389,7 @@ void DatabaseOnDisk::checkMetadataFilenameAvailabilityUnlocked(const String & to } void DatabaseOnDisk::renameTable( - const Context & context, + ContextPtr local_context, const String & table_name, IDatabase & to_database, const String & to_table_name, @@ -418,15 +420,16 @@ void DatabaseOnDisk::renameTable( String table_metadata_path; ASTPtr attach_query; /// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case. - StoragePtr table = tryGetTable(table_name, global_context); + StoragePtr table = tryGetTable(table_name, getContext()); detachTable(table_name); UUID prev_uuid = UUIDHelpers::Nil; try { - table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); + table_lock = table->lockExclusively( + local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); table_metadata_path = getObjectMetadataPath(table_name); - attach_query = parseQueryFromMetadata(log, context, table_metadata_path); + attach_query = parseQueryFromMetadata(log, local_context, table_metadata_path); auto & create = attach_query->as(); create.database = to_database.getDatabaseName(); create.table = to_table_name; @@ -454,7 +457,7 @@ void DatabaseOnDisk::renameTable( } /// Now table data are moved to new database, so we must add metadata and attach table to new database - to_database.createTable(context, to_table_name, table, attach_query); + to_database.createTable(local_context, to_table_name, table, attach_query); Poco::File(table_metadata_path).remove(); @@ -473,10 +476,10 @@ void DatabaseOnDisk::renameTable( /// It returns create table statement (even if table is detached) -ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const +ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, ContextPtr, bool throw_on_error) const { ASTPtr ast; - bool has_table = tryGetTable(table_name, global_context) != nullptr; + bool has_table = tryGetTable(table_name, getContext()) != nullptr; auto table_metadata_path = getObjectMetadataPath(table_name); try { @@ -497,11 +500,11 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const { ASTPtr ast; - auto settings = global_context.getSettingsRef(); + auto settings = getContext()->getSettingsRef(); { std::lock_guard lock(mutex); - auto database_metadata_path = global_context.getPath() + "metadata/" + escapeForFileName(database_name) + ".sql"; - ast = parseQueryFromMetadata(log, global_context, database_metadata_path, true); + auto database_metadata_path = getContext()->getPath() + "metadata/" + escapeForFileName(database_name) + ".sql"; + ast = parseQueryFromMetadata(log, getContext(), database_metadata_path, true); auto & ast_create_query = ast->as(); ast_create_query.attach = false; ast_create_query.database = database_name; @@ -518,10 +521,10 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const return ast; } -void DatabaseOnDisk::drop(const Context & context) +void DatabaseOnDisk::drop(ContextPtr local_context) { assert(tables.empty()); - Poco::File(context.getPath() + getDataPath()).remove(false); + Poco::File(local_context->getPath() + getDataPath()).remove(false); Poco::File(getMetadataPath()).remove(false); } @@ -541,14 +544,14 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n return static_cast(0); } -void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file) const +void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const IteratingFunction & process_metadata_file) const { auto process_tmp_drop_metadata_file = [&](const String & file_name) { assert(getUUID() == UUIDHelpers::Nil); static const char * tmp_drop_ext = ".sql.tmp_drop"; const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext)); - if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists()) + if (Poco::File(local_context->getPath() + getDataPath() + '/' + object_name).exists()) { Poco::File(getMetadataPath() + file_name).renameTo(getMetadataPath() + object_name + ".sql"); LOG_WARNING(log, "Object {} was not dropped previously and will be restored", backQuote(object_name)); @@ -615,7 +618,12 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const Iterati pool.wait(); } -ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Context & context, const String & metadata_file_path, bool throw_on_error /*= true*/, bool remove_empty /*= false*/) +ASTPtr DatabaseOnDisk::parseQueryFromMetadata( + Poco::Logger * logger, + ContextPtr local_context, + const String & metadata_file_path, + bool throw_on_error /*= true*/, + bool remove_empty /*= false*/) { String query; @@ -643,7 +651,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte return nullptr; } - auto settings = context.getSettingsRef(); + auto settings = local_context->getSettingsRef(); ParserCreateQuery parser; const char * pos = query.data(); std::string error_message; @@ -662,7 +670,12 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte table_name = unescapeForFileName(table_name); if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER && logger) - LOG_WARNING(logger, "File {} contains both UUID and table name. Will use name `{}` instead of `{}`", metadata_file_path, table_name, create.table); + LOG_WARNING( + logger, + "File {} contains both UUID and table name. Will use name `{}` instead of `{}`", + metadata_file_path, + table_name, + create.table); create.table = table_name; } @@ -671,7 +684,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const String & database_metadata_path, bool throw_on_error) const { - ASTPtr ast = parseQueryFromMetadata(log, global_context, database_metadata_path, throw_on_error); + ASTPtr ast = parseQueryFromMetadata(log, getContext(), database_metadata_path, throw_on_error); if (ast) { diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index b96a24f3345..677465e306e 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -16,7 +16,7 @@ std::pair createTableFromAST( ASTCreateQuery ast_create_query, const String & database_name, const String & table_data_path_relative, - Context & context, + ContextPtr context, bool has_force_restore_data_flag); /** Get the string with the table definition based on the CREATE query. @@ -33,23 +33,23 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo class DatabaseOnDisk : public DatabaseWithOwnTablesBase { public: - DatabaseOnDisk(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context); + DatabaseOnDisk(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, ContextPtr context); void createTable( - const Context & context, + ContextPtr context, const String & table_name, const StoragePtr & table, const ASTPtr & query) override; - void detachTablePermanently(const Context & context, const String & table_name) override; + void detachTablePermanently(ContextPtr context, const String & table_name) override; void dropTable( - const Context & context, + ContextPtr context, const String & table_name, bool no_delay) override; void renameTable( - const Context & context, + ContextPtr context, const String & table_name, IDatabase & to_database, const String & to_table_name, @@ -58,7 +58,7 @@ public: ASTPtr getCreateDatabaseQuery() const override; - void drop(const Context & context) override; + void drop(ContextPtr context) override; String getObjectMetadataPath(const String & object_name) const override; @@ -69,7 +69,7 @@ public: String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.table); } String getMetadataPath() const override { return metadata_path; } - static ASTPtr parseQueryFromMetadata(Poco::Logger * log, const Context & context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false); + static ASTPtr parseQueryFromMetadata(Poco::Logger * log, ContextPtr context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false); /// will throw when the table we want to attach already exists (in active / detached / detached permanently form) void checkMetadataFilenameAvailability(const String & to_table_name) const; @@ -82,19 +82,19 @@ protected: using IteratingFunction = std::function; - void iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file) const; + void iterateMetadataFiles(ContextPtr context, const IteratingFunction & process_metadata_file) const; ASTPtr getCreateTableQueryImpl( const String & table_name, - const Context & context, + ContextPtr context, bool throw_on_error) const override; ASTPtr getCreateQueryFromMetadata(const String & metadata_path, bool throw_on_error) const; virtual void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, - const String & table_metadata_tmp_path, const String & table_metadata_path, const Context & query_context); + const String & table_metadata_tmp_path, const String & table_metadata_path, ContextPtr query_context); - virtual void removeDetachedPermanentlyFlag(const Context & context, const String & table_name, const String & table_metadata_path, bool attach) const; + virtual void removeDetachedPermanentlyFlag(ContextPtr context, const String & table_name, const String & table_metadata_path, bool attach) const; const String metadata_path; const String data_path; diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index a94668dacf7..840be0e006a 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -36,7 +36,7 @@ static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; namespace { void tryAttachTable( - Context & context, + ContextPtr context, const ASTCreateQuery & query, DatabaseOrdinary & database, const String & database_name, @@ -62,7 +62,7 @@ namespace } - void tryAttachDictionary(const ASTPtr & query, DatabaseOrdinary & database, const String & metadata_path, const Context & context) + void tryAttachDictionary(const ASTPtr & query, DatabaseOrdinary & database, const String & metadata_path, ContextPtr context) { auto & create_query = query->as(); assert(create_query.is_dictionary); @@ -94,18 +94,18 @@ namespace } -DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context_) +DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, ContextPtr context_) : DatabaseOrdinary(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseOrdinary (" + name_ + ")", context_) { } DatabaseOrdinary::DatabaseOrdinary( - const String & name_, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context_) + const String & name_, const String & metadata_path_, const String & data_path_, const String & logger, ContextPtr context_) : DatabaseWithDictionaries(name_, metadata_path_, data_path_, logger, context_) { } -void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool /*force_attach*/) +void DatabaseOrdinary::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool /*force_attach*/) { /** Tables load faster if they are loaded in sorted (by name) order. * Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order, @@ -117,7 +117,8 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto size_t total_dictionaries = 0; - auto process_metadata = [&context, &file_names, &total_dictionaries, &file_names_mutex, this](const String & file_name) + auto process_metadata = [context_weak = ContextWeakPtr(local_context), &file_names, &total_dictionaries, &file_names_mutex, this]( + const String & file_name) { fs::path path(getMetadataPath()); fs::path file_path(file_name); @@ -125,7 +126,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto try { - auto ast = parseQueryFromMetadata(log, context, full_path.string(), /*throw_on_error*/ true, /*remove_empty*/ false); + auto ast = parseQueryFromMetadata(log, getContext(), full_path.string(), /*throw_on_error*/ true, /*remove_empty*/ false); if (ast) { auto * create_query = ast->as(); @@ -155,7 +156,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto } }; - iterateMetadataFiles(context, process_metadata); + iterateMetadataFiles(local_context, process_metadata); size_t total_tables = file_names.size() - total_dictionaries; @@ -180,7 +181,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto auto create_query = query->as(); if (create_query.is_dictionary) { - tryAttachDictionary(query, *this, getMetadataPath() + name, context); + tryAttachDictionary(query, *this, getMetadataPath() + name, local_context); /// Messages, so that it's not boring to wait for the server to load for a long time. logAboutProgress(log, ++dictionaries_processed, total_dictionaries, watch); @@ -195,7 +196,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto pool.scheduleOrThrowOnError([&]() { tryAttachTable( - context, + local_context, create_query, *this, database_name, @@ -245,7 +246,7 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool) thread_pool.wait(); } -void DatabaseOrdinary::alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) +void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) { String table_name = table_id.table_name; /// Read the definition of the table and replace the necessary parts with new ones. @@ -265,7 +266,7 @@ void DatabaseOrdinary::alterTable(const Context & context, const StorageID & tab statement.data() + statement.size(), "in file " + table_metadata_path, 0, - context.getSettingsRef().max_parser_depth); + local_context->getSettingsRef().max_parser_depth); applyMetadataChangesToCreateQuery(ast, metadata); @@ -274,15 +275,15 @@ void DatabaseOrdinary::alterTable(const Context & context, const StorageID & tab WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL); writeString(statement, out); out.next(); - if (context.getSettingsRef().fsync_metadata) + if (local_context->getSettingsRef().fsync_metadata) out.sync(); out.close(); } - commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, context); + commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, local_context); } -void DatabaseOrdinary::commitAlterTable(const StorageID &, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & /*statement*/, const Context & /*query_context*/) +void DatabaseOrdinary::commitAlterTable(const StorageID &, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & /*statement*/, ContextPtr /*query_context*/) { try { diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index c1ad32345f6..4cf58cef9f0 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -14,23 +14,26 @@ namespace DB class DatabaseOrdinary : public DatabaseWithDictionaries { public: - DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context); - DatabaseOrdinary(const String & name_, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context_); + DatabaseOrdinary(const String & name_, const String & metadata_path_, ContextPtr context); + DatabaseOrdinary( + const String & name_, const String & metadata_path_, const String & data_path_, const String & logger, ContextPtr context_); String getEngineName() const override { return "Ordinary"; } - void loadStoredObjects( - Context & context, - bool has_force_restore_data_flag, - bool force_attach) override; + void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override; void alterTable( - const Context & context, + ContextPtr context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; protected: - virtual void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, const Context & query_context); + virtual void commitAlterTable( + const StorageID & table_id, + const String & table_metadata_tmp_path, + const String & table_metadata_path, + const String & statement, + ContextPtr query_context); void startupTables(ThreadPool & thread_pool); }; diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 664452abe6f..46d67e275ba 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -47,12 +47,12 @@ static constexpr const char * BROKEN_TABLES_SUFFIX = "_broken_tables"; zkutil::ZooKeeperPtr DatabaseReplicated::getZooKeeper() const { - return global_context.getZooKeeper(); + return getContext()->getZooKeeper(); } -static inline String getHostID(const Context & global_context, const UUID & db_uuid) +static inline String getHostID(ContextPtr global_context, const UUID & db_uuid) { - return Cluster::Address::toString(getFQDNOrHostName(), global_context.getTCPPort()) + ':' + toString(db_uuid); + return Cluster::Address::toString(getFQDNOrHostName(), global_context->getTCPPort()) + ':' + toString(db_uuid); } @@ -66,7 +66,7 @@ DatabaseReplicated::DatabaseReplicated( const String & shard_name_, const String & replica_name_, DatabaseReplicatedSettings db_settings_, - const Context & context_) + ContextPtr context_) : DatabaseAtomic(name_, metadata_path_, uuid, "DatabaseReplicated (" + name_ + ")", context_) , zookeeper_path(zookeeper_path_) , shard_name(shard_name_) @@ -126,7 +126,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const Strings hosts; Strings host_ids; - auto zookeeper = global_context.getZooKeeper(); + auto zookeeper = getContext()->getZooKeeper(); constexpr int max_retries = 10; int iteration = 0; bool success = false; @@ -189,22 +189,22 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const String username = db_settings.cluster_username; String password = db_settings.cluster_password; - UInt16 default_port = global_context.getTCPPort(); + UInt16 default_port = getContext()->getTCPPort(); bool secure = db_settings.cluster_secure_connection; - return std::make_shared(global_context.getSettingsRef(), shards, username, password, default_port, false, secure); + return std::make_shared(getContext()->getSettingsRef(), shards, username, password, default_port, false, secure); } void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(bool force_attach) { try { - if (!global_context.hasZooKeeper()) + if (!getContext()->hasZooKeeper()) { throw Exception("Can't create replicated database without ZooKeeper", ErrorCodes::NO_ZOOKEEPER); } - auto current_zookeeper = global_context.getZooKeeper(); + auto current_zookeeper = getContext()->getZooKeeper(); if (!current_zookeeper->exists(zookeeper_path)) { @@ -217,7 +217,7 @@ void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(bool force_attach) String replica_host_id; if (current_zookeeper->tryGet(replica_path, replica_host_id)) { - String host_id = getHostID(global_context, db_uuid); + String host_id = getHostID(getContext(), db_uuid); if (replica_host_id != host_id) throw Exception(ErrorCodes::REPLICA_IS_ALREADY_EXIST, "Replica {} of shard {} of replicated database at {} already exists. Replica host ID: '{}', current host ID: '{}'", @@ -291,7 +291,7 @@ void DatabaseReplicated::createEmptyLogEntry(Coordination::Requests & ops, const void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPtr & current_zookeeper) { /// Write host name to replica_path, it will protect from multiple replicas with the same name - auto host_id = getHostID(global_context, db_uuid); + auto host_id = getHostID(getContext(), db_uuid); Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent)); @@ -300,17 +300,17 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt current_zookeeper->multi(ops); } -void DatabaseReplicated::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) +void DatabaseReplicated::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach) { tryConnectToZooKeeperAndInitDatabase(force_attach); - DatabaseAtomic::loadStoredObjects(context, has_force_restore_data_flag, force_attach); + DatabaseAtomic::loadStoredObjects(local_context, has_force_restore_data_flag, force_attach); - ddl_worker = std::make_unique(this, global_context); + ddl_worker = std::make_unique(this, getContext()); ddl_worker->startup(); } -void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & query_context) const +void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_context) const { /// Replicas will set correct name of current database in query context (database name can be different on replicas) if (auto * ddl_query = dynamic_cast(query.get())) @@ -343,12 +343,12 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & q Macros::MacroExpansionInfo info; info.table_id = {getDatabaseName(), create->table, create->uuid}; - query_context.getMacros()->expand(maybe_path, info); + query_context->getMacros()->expand(maybe_path, info); bool maybe_shard_macros = info.expanded_other; info.expanded_other = false; - query_context.getMacros()->expand(maybe_replica, info); + query_context->getMacros()->expand(maybe_replica, info); bool maybe_replica_macros = info.expanded_other; - bool enable_functional_tests_helper = global_context.getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros"); + bool enable_functional_tests_helper = getContext()->getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros"); if (!enable_functional_tests_helper) LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments"); @@ -383,7 +383,7 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & q if (auto * query_drop = query->as()) { - if (query_drop->kind == ASTDropQuery::Kind::Detach && query_context.getSettingsRef().database_replicated_always_detach_permanently) + if (query_drop->kind == ASTDropQuery::Kind::Detach && query_context->getSettingsRef().database_replicated_always_detach_permanently) query_drop->permanently = true; if (query_drop->kind == ASTDropQuery::Kind::Detach && !query_drop->permanently) throw Exception(ErrorCodes::INCORRECT_QUERY, "DETACH TABLE is not allowed for Replicated databases. " @@ -392,12 +392,12 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & q } } -BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const Context & query_context) +BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context) { if (is_readonly) throw Exception(ErrorCodes::NO_ZOOKEEPER, "Database is in readonly mode, because it cannot connect to ZooKeeper"); - if (query_context.getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY) + if (query_context->getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY) throw Exception(ErrorCodes::INCORRECT_QUERY, "It's not initial query. ON CLUSTER is not allowed for Replicated database."); checkQueryValid(query, query_context); @@ -413,15 +413,15 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const return getDistributedDDLStatus(node_path, entry, query_context, hosts_to_wait); } -static UUID getTableUUIDIfReplicated(const String & metadata, const Context & context) +static UUID getTableUUIDIfReplicated(const String & metadata, ContextPtr context) { bool looks_like_replicated = metadata.find("ReplicatedMergeTree") != std::string::npos; if (!looks_like_replicated) return UUIDHelpers::Nil; ParserCreateQuery parser; - auto size = context.getSettingsRef().max_query_size; - auto depth = context.getSettingsRef().max_parser_depth; + auto size = context->getSettingsRef().max_query_size; + auto depth = context->getSettingsRef().max_parser_depth; ASTPtr query = parseQuery(parser, metadata, size, depth); const ASTCreateQuery & create = query->as(); if (!create.storage || !create.storage->engine) @@ -457,7 +457,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep std::unordered_map zk_replicated_id_to_name; for (const auto & zk_table : table_name_to_metadata) { - UUID zk_replicated_id = getTableUUIDIfReplicated(zk_table.second, global_context); + UUID zk_replicated_id = getTableUUIDIfReplicated(zk_table.second, getContext()); if (zk_replicated_id != UUIDHelpers::Nil) zk_replicated_id_to_name.emplace(zk_replicated_id, zk_table.first); } @@ -467,7 +467,8 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep std::vector> replicated_tables_to_rename; size_t total_tables = 0; std::vector replicated_ids; - for (auto existing_tables_it = getTablesIterator(global_context, {}); existing_tables_it->isValid(); existing_tables_it->next(), ++total_tables) + for (auto existing_tables_it = getTablesIterator(getContext(), {}); existing_tables_it->isValid(); + existing_tables_it->next(), ++total_tables) { String name = existing_tables_it->name(); UUID local_replicated_id = UUIDHelpers::Nil; @@ -506,7 +507,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep /// We use Ordinary engine for destination database, because it's the only way to discard table UUID /// and make possible creation of new table with the same UUID. String query = fmt::format("CREATE DATABASE IF NOT EXISTS {} ENGINE=Ordinary", backQuoteIfNeed(to_db_name)); - Context query_context = global_context; + auto query_context = Context::createCopy(getContext()); executeQuery(query, query_context, true); } @@ -519,12 +520,12 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep if (getDatabaseName() != db_name) throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed, will retry"); - auto table = tryGetTable(table_name, global_context); + auto table = tryGetTable(table_name, getContext()); if (isDictionaryExist(table_name)) { /// We can safely drop any dictionaries because they do not store data LOG_DEBUG(log, "Will DROP DICTIONARY {}", backQuoteIfNeed(table_name)); - DatabaseAtomic::removeDictionary(global_context, table_name); + DatabaseAtomic::removeDictionary(getContext(), table_name); ++dropped_dicts; } else if (!table->storesDataOnDisk()) @@ -532,7 +533,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep LOG_DEBUG(log, "Will DROP TABLE {}, because it does not store data on disk and can be safely dropped", backQuoteIfNeed(table_name)); dropped_tables.push_back(tryGetTableUUID(table_name)); table->shutdown(); - DatabaseAtomic::dropTable(global_context, table_name, true); + DatabaseAtomic::dropTable(getContext(), table_name, true); } else { @@ -542,7 +543,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep assert(db_name < to_db_name); DDLGuardPtr to_table_guard = DatabaseCatalog::instance().getDDLGuard(to_db_name, to_name); auto to_db_ptr = DatabaseCatalog::instance().getDatabase(to_db_name); - DatabaseAtomic::renameTable(global_context, table_name, *to_db_ptr, to_name, false, false); + DatabaseAtomic::renameTable(getContext(), table_name, *to_db_ptr, to_name, false, false); ++moved_tables; } } @@ -561,7 +562,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep /// TODO Maybe we should do it in two steps: rename all tables to temporary names and then rename them to actual names? DDLGuardPtr table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::min(from, to)); DDLGuardPtr to_table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::max(from, to)); - DatabaseAtomic::renameTable(global_context, from, *this, to, false, false); + DatabaseAtomic::renameTable(getContext(), from, *this, to, false, false); } for (const auto & id : dropped_tables) @@ -569,7 +570,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep for (const auto & name_and_meta : table_name_to_metadata) { - if (isTableExist(name_and_meta.first, global_context)) + if (isTableExist(name_and_meta.first, getContext())) { assert(name_and_meta.second == readMetadataFile(name_and_meta.first)); continue; @@ -577,11 +578,11 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep auto query_ast = parseQueryFromMetadataInZooKeeper(name_and_meta.first, name_and_meta.second); - Context query_context = global_context; - query_context.makeQueryContext(); - query_context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - query_context.setCurrentDatabase(database_name); - query_context.setCurrentQueryId(""); // generate random query_id + auto query_context = Context::createCopy(getContext()); + query_context->makeQueryContext(); + query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + query_context->setCurrentDatabase(database_name); + query_context->setCurrentQueryId(""); // generate random query_id LOG_INFO(log, "Executing {}", serializeAST(*query_ast)); InterpreterCreateQuery(query_ast, query_context).execute(); @@ -643,7 +644,7 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node { ParserCreateQuery parser; String description = "in ZooKeeper " + zookeeper_path + "/metadata/" + node_name; - auto ast = parseQuery(parser, query, description, 0, global_context.getSettingsRef().max_parser_depth); + auto ast = parseQuery(parser, query, description, 0, getContext()->getSettingsRef().max_parser_depth); auto & create = ast->as(); if (create.uuid == UUIDHelpers::Nil || create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER || !create.database.empty()) @@ -658,7 +659,7 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node return ast; } -void DatabaseReplicated::drop(const Context & context_) +void DatabaseReplicated::drop(ContextPtr context_) { auto current_zookeeper = getZooKeeper(); Coordination::Requests ops; @@ -691,22 +692,22 @@ void DatabaseReplicated::shutdown() } -void DatabaseReplicated::dropTable(const Context & context, const String & table_name, bool no_delay) +void DatabaseReplicated::dropTable(ContextPtr local_context, const String & table_name, bool no_delay) { - auto txn = context.getZooKeeperMetadataTransaction(); + auto txn = local_context->getZooKeeperMetadataTransaction(); assert(!ddl_worker->isCurrentlyActive() || txn || startsWith(table_name, ".inner_id.")); if (txn && txn->isInitialQuery()) { String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name); txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1)); } - DatabaseAtomic::dropTable(context, table_name, no_delay); + DatabaseAtomic::dropTable(local_context, table_name, no_delay); } -void DatabaseReplicated::renameTable(const Context & context, const String & table_name, IDatabase & to_database, +void DatabaseReplicated::renameTable(ContextPtr local_context, const String & table_name, IDatabase & to_database, const String & to_table_name, bool exchange, bool dictionary) { - auto txn = context.getZooKeeperMetadataTransaction(); + auto txn = local_context->getZooKeeperMetadataTransaction(); assert(txn); if (txn->isInitialQuery()) @@ -715,9 +716,9 @@ void DatabaseReplicated::renameTable(const Context & context, const String & tab throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Moving tables between databases is not supported for Replicated engine"); if (table_name == to_table_name) throw Exception(ErrorCodes::INCORRECT_QUERY, "Cannot rename table to itself"); - if (!isTableExist(table_name, context)) + if (!isTableExist(table_name, local_context)) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist", table_name); - if (exchange && !to_database.isTableExist(to_table_name, context)) + if (exchange && !to_database.isTableExist(to_table_name, local_context)) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist", to_table_name); String statement = readMetadataFile(table_name); @@ -733,14 +734,14 @@ void DatabaseReplicated::renameTable(const Context & context, const String & tab txn->addOp(zkutil::makeCreateRequest(metadata_zk_path_to, statement, zkutil::CreateMode::Persistent)); } - DatabaseAtomic::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary); + DatabaseAtomic::renameTable(local_context, table_name, to_database, to_table_name, exchange, dictionary); } void DatabaseReplicated::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, const String & table_metadata_tmp_path, const String & table_metadata_path, - const Context & query_context) + ContextPtr query_context) { - auto txn = query_context.getZooKeeperMetadataTransaction(); + auto txn = query_context->getZooKeeperMetadataTransaction(); assert(!ddl_worker->isCurrentlyActive() || txn); if (txn && txn->isInitialQuery()) { @@ -754,9 +755,9 @@ void DatabaseReplicated::commitCreateTable(const ASTCreateQuery & query, const S void DatabaseReplicated::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, - const String & statement, const Context & query_context) + const String & statement, ContextPtr query_context) { - auto txn = query_context.getZooKeeperMetadataTransaction(); + auto txn = query_context->getZooKeeperMetadataTransaction(); if (txn && txn->isInitialQuery()) { String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_id.table_name); @@ -765,11 +766,11 @@ void DatabaseReplicated::commitAlterTable(const StorageID & table_id, DatabaseAtomic::commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, query_context); } -void DatabaseReplicated::createDictionary(const Context & context, +void DatabaseReplicated::createDictionary(ContextPtr local_context, const String & dictionary_name, const ASTPtr & query) { - auto txn = context.getZooKeeperMetadataTransaction(); + auto txn = local_context->getZooKeeperMetadataTransaction(); assert(!ddl_worker->isCurrentlyActive() || txn); if (txn && txn->isInitialQuery()) { @@ -777,24 +778,24 @@ void DatabaseReplicated::createDictionary(const Context & context, String statement = getObjectDefinitionFromCreateQuery(query->clone()); txn->addOp(zkutil::makeCreateRequest(metadata_zk_path, statement, zkutil::CreateMode::Persistent)); } - DatabaseAtomic::createDictionary(context, dictionary_name, query); + DatabaseAtomic::createDictionary(local_context, dictionary_name, query); } -void DatabaseReplicated::removeDictionary(const Context & context, const String & dictionary_name) +void DatabaseReplicated::removeDictionary(ContextPtr local_context, const String & dictionary_name) { - auto txn = context.getZooKeeperMetadataTransaction(); + auto txn = local_context->getZooKeeperMetadataTransaction(); assert(!ddl_worker->isCurrentlyActive() || txn); if (txn && txn->isInitialQuery()) { String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(dictionary_name); txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1)); } - DatabaseAtomic::removeDictionary(context, dictionary_name); + DatabaseAtomic::removeDictionary(local_context, dictionary_name); } -void DatabaseReplicated::detachTablePermanently(const Context & context, const String & table_name) +void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const String & table_name) { - auto txn = context.getZooKeeperMetadataTransaction(); + auto txn = local_context->getZooKeeperMetadataTransaction(); assert(!ddl_worker->isCurrentlyActive() || txn); if (txn && txn->isInitialQuery()) { @@ -803,12 +804,12 @@ void DatabaseReplicated::detachTablePermanently(const Context & context, const S String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name); txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1)); } - DatabaseAtomic::detachTablePermanently(context, table_name); + DatabaseAtomic::detachTablePermanently(local_context, table_name); } -void DatabaseReplicated::removeDetachedPermanentlyFlag(const Context & context, const String & table_name, const String & table_metadata_path, bool attach) const +void DatabaseReplicated::removeDetachedPermanentlyFlag(ContextPtr local_context, const String & table_name, const String & table_metadata_path, bool attach) const { - auto txn = context.getZooKeeperMetadataTransaction(); + auto txn = local_context->getZooKeeperMetadataTransaction(); assert(!ddl_worker->isCurrentlyActive() || txn); if (txn && txn->isInitialQuery() && attach) { @@ -816,7 +817,7 @@ void DatabaseReplicated::removeDetachedPermanentlyFlag(const Context & context, String statement = readMetadataFile(table_name); txn->addOp(zkutil::makeCreateRequest(metadata_zk_path, statement, zkutil::CreateMode::Persistent)); } - DatabaseAtomic::removeDetachedPermanentlyFlag(context, table_name, table_metadata_path, attach); + DatabaseAtomic::removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, attach); } diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 46bcdc71ef1..5220535f095 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -24,32 +24,32 @@ public: DatabaseReplicated(const String & name_, const String & metadata_path_, UUID uuid, const String & zookeeper_path_, const String & shard_name_, const String & replica_name_, DatabaseReplicatedSettings db_settings_, - const Context & context); + ContextPtr context); ~DatabaseReplicated() override; String getEngineName() const override { return "Replicated"; } /// If current query is initial, then the following methods add metadata updating ZooKeeper operations to current ZooKeeperMetadataTransaction. - void dropTable(const Context &, const String & table_name, bool no_delay) override; - void renameTable(const Context & context, const String & table_name, IDatabase & to_database, + void dropTable(ContextPtr, const String & table_name, bool no_delay) override; + void renameTable(ContextPtr context, const String & table_name, IDatabase & to_database, const String & to_table_name, bool exchange, bool dictionary) override; void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, const String & table_metadata_tmp_path, const String & table_metadata_path, - const Context & query_context) override; + ContextPtr query_context) override; void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, - const String & statement, const Context & query_context) override; - void createDictionary(const Context & context, + const String & statement, ContextPtr query_context) override; + void createDictionary(ContextPtr context, const String & dictionary_name, const ASTPtr & query) override; - void removeDictionary(const Context & context, const String & dictionary_name) override; - void detachTablePermanently(const Context & context, const String & table_name) override; - void removeDetachedPermanentlyFlag(const Context & context, const String & table_name, const String & table_metadata_path, bool attach) const override; + void removeDictionary(ContextPtr context, const String & dictionary_name) override; + void detachTablePermanently(ContextPtr context, const String & table_name) override; + void removeDetachedPermanentlyFlag(ContextPtr context, const String & table_name, const String & table_metadata_path, bool attach) const override; /// Try to execute DLL query on current host as initial query. If query is succeed, /// then it will be executed on all replicas. - BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, const Context & query_context); + BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context); void stopReplication(); @@ -59,9 +59,9 @@ public: /// Returns cluster consisting of database replicas ClusterPtr getCluster() const; - void drop(const Context & /*context*/) override; + void drop(ContextPtr /*context*/) override; - void loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) override; + void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override; void shutdown() override; friend struct DatabaseReplicatedTask; @@ -71,7 +71,7 @@ private: bool createDatabaseNodesInZooKeeper(const ZooKeeperPtr & current_zookeeper); void createReplicaNodesInZooKeeper(const ZooKeeperPtr & current_zookeeper); - void checkQueryValid(const ASTPtr & query, const Context & query_context) const; + void checkQueryValid(const ASTPtr & query, ContextPtr query_context) const; void recoverLostReplica(const ZooKeeperPtr & current_zookeeper, UInt32 our_log_ptr, UInt32 max_log_ptr); std::map tryGetConsistentMetadataSnapshot(const ZooKeeperPtr & zookeeper, UInt32 & max_log_ptr); diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 63f4d89b216..9ae4d026bf0 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -13,7 +13,7 @@ namespace ErrorCodes extern const int UNFINISHED; } -DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db, const Context & context_) +DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db, ContextPtr context_) : DDLWorker(/* pool_size */ 1, db->zookeeper_path + "/log", context_, nullptr, {}, fmt::format("DDLWorker({})", db->getDatabaseName())) , database(db) { @@ -91,7 +91,7 @@ String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry) return node_path; } -String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entry, const Context & query_context) +String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entry, ContextPtr query_context) { /// NOTE Possibly it would be better to execute initial query on the most up-to-date node, /// but it requires more complex logic around /try node. @@ -115,7 +115,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr task->is_initial_query = true; LOG_DEBUG(log, "Waiting for worker thread to process all entries before {}", entry_name); - UInt64 timeout = query_context.getSettingsRef().database_replicated_initial_query_timeout_sec; + UInt64 timeout = query_context->getSettingsRef().database_replicated_initial_query_timeout_sec; { std::unique_lock lock{mutex}; bool processed = wait_current_task_change.wait_for(lock, std::chrono::seconds(timeout), [&]() diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 3a45817c755..16ad100b81a 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -21,11 +21,11 @@ class DatabaseReplicated; class DatabaseReplicatedDDLWorker : public DDLWorker { public: - DatabaseReplicatedDDLWorker(DatabaseReplicated * db, const Context & context_); + DatabaseReplicatedDDLWorker(DatabaseReplicated * db, ContextPtr context_); String enqueueQuery(DDLLogEntry & entry) override; - String tryEnqueueAndExecuteEntry(DDLLogEntry & entry, const Context & query_context); + String tryEnqueueAndExecuteEntry(DDLLogEntry & entry, ContextPtr query_context); void shutdown() override; diff --git a/src/Databases/DatabaseWithDictionaries.cpp b/src/Databases/DatabaseWithDictionaries.cpp index 55b04f27c58..5180b251a5f 100644 --- a/src/Databases/DatabaseWithDictionaries.cpp +++ b/src/Databases/DatabaseWithDictionaries.cpp @@ -125,9 +125,9 @@ void DatabaseWithDictionaries::detachDictionaryImpl(const String & dictionary_na detachTable(dictionary_name); } -void DatabaseWithDictionaries::createDictionary(const Context & context, const String & dictionary_name, const ASTPtr & query) +void DatabaseWithDictionaries::createDictionary(ContextPtr local_context, const String & dictionary_name, const ASTPtr & query) { - const auto & settings = context.getSettingsRef(); + const auto & settings = local_context->getSettingsRef(); /** The code is based on the assumption that all threads share the same order of operations: * - create the .sql.tmp file; @@ -151,7 +151,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS, "Dictionary {} already exists.", dict_id.getFullNameNotQuoted()); - if (isTableExist(dictionary_name, global_context)) + if (isTableExist(dictionary_name, getContext())) throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", dict_id.getFullTableName()); String dictionary_metadata_path = getObjectMetadataPath(dictionary_name); @@ -177,9 +177,9 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S /// Add a temporary repository containing the dictionary. /// We need this temp repository to try loading the dictionary before actually attaching it to the database. auto temp_repository = external_loader.addConfigRepository(std::make_unique( - getDatabaseName(), dictionary_metadata_tmp_path, getDictionaryConfigurationFromAST(query->as(), context))); + getDatabaseName(), dictionary_metadata_tmp_path, getDictionaryConfigurationFromAST(query->as(), local_context))); - bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true); + bool lazy_load = local_context->getConfigRef().getBool("dictionaries_lazy_load", true); if (!lazy_load) { /// load() is called here to force loading the dictionary, wait until the loading is finished, @@ -187,15 +187,15 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S external_loader.load(dict_id.getInternalDictionaryName()); } - auto config = getDictionaryConfigurationFromAST(query->as(), context); + auto config = getDictionaryConfigurationFromAST(query->as(), local_context); attachDictionary(dictionary_name, DictionaryAttachInfo{query, config, time(nullptr)}); SCOPE_EXIT({ if (!succeeded) detachDictionary(dictionary_name); }); - auto txn = context.getZooKeeperMetadataTransaction(); - if (txn && !context.isInternalSubquery()) + auto txn = local_context->getZooKeeperMetadataTransaction(); + if (txn && !local_context->isInternalSubquery()) txn->commit(); /// Commit point (a sort of) for Replicated database /// If it was ATTACH query and file with dictionary metadata already exist @@ -210,7 +210,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S succeeded = true; } -void DatabaseWithDictionaries::removeDictionary(const Context & context, const String & dictionary_name) +void DatabaseWithDictionaries::removeDictionary(ContextPtr local_context, const String & dictionary_name) { DictionaryAttachInfo attach_info; detachDictionaryImpl(dictionary_name, attach_info); @@ -219,8 +219,8 @@ void DatabaseWithDictionaries::removeDictionary(const Context & context, const S { String dictionary_metadata_path = getObjectMetadataPath(dictionary_name); - auto txn = context.getZooKeeperMetadataTransaction(); - if (txn && !context.isInternalSubquery()) + auto txn = local_context->getZooKeeperMetadataTransaction(); + if (txn && !local_context->isInternalSubquery()) txn->commit(); /// Commit point (a sort of) for Replicated database Poco::File(dictionary_metadata_path).remove(); @@ -335,7 +335,7 @@ void DatabaseWithDictionaries::reloadDictionaryConfig(const String & full_name) /// Ensure that this database is attached to ExternalLoader as a config repository. if (!database_as_config_repo_for_external_loader.load()) { - auto repository = std::make_unique(*this, global_context); + auto repository = std::make_unique(*this, getContext()); auto remove_repository_callback = external_loader.addConfigRepository(std::move(repository)); database_as_config_repo_for_external_loader = boost::make_shared(std::move(remove_repository_callback)); } @@ -359,9 +359,9 @@ void DatabaseWithDictionaries::shutdown() DatabaseWithDictionaries::DatabaseWithDictionaries( - const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context) - : DatabaseOnDisk(name, metadata_path_, data_path_, logger, context) - , external_loader(context.getExternalDictionariesLoader()) + const String & name, const String & metadata_path_, const String & data_path_, const String & logger, ContextPtr context_) + : DatabaseOnDisk(name, metadata_path_, data_path_, logger, context_) + , external_loader(context_->getExternalDictionariesLoader()) { } diff --git a/src/Databases/DatabaseWithDictionaries.h b/src/Databases/DatabaseWithDictionaries.h index d69289d7456..d10908c7c06 100644 --- a/src/Databases/DatabaseWithDictionaries.h +++ b/src/Databases/DatabaseWithDictionaries.h @@ -17,11 +17,11 @@ public: void detachDictionary(const String & dictionary_name) override; - void createDictionary(const Context & context, + void createDictionary(ContextPtr context, const String & dictionary_name, const ASTPtr & query) override; - void removeDictionary(const Context & context, const String & dictionary_name) override; + void removeDictionary(ContextPtr context, const String & dictionary_name) override; bool isDictionaryExist(const String & dictionary_name) const override; @@ -38,7 +38,7 @@ public: ~DatabaseWithDictionaries() override; protected: - DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context); + DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, ContextPtr context); ASTPtr getCreateDictionaryQueryImpl(const String & dictionary_name, bool throw_on_error) const override; diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 2d3d206162b..9329b0a4210 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -20,18 +20,18 @@ namespace ErrorCodes extern const int UNKNOWN_DATABASE; } -DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, const Context & context) - : IDatabase(name_), log(&Poco::Logger::get(logger)), global_context(context.getGlobalContext()) +DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context_) + : IDatabase(name_), WithContext(context_->getGlobalContext()), log(&Poco::Logger::get(logger)) { } -bool DatabaseWithOwnTablesBase::isTableExist(const String & table_name, const Context &) const +bool DatabaseWithOwnTablesBase::isTableExist(const String & table_name, ContextPtr) const { std::lock_guard lock(mutex); return tables.find(table_name) != tables.end(); } -StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, const Context &) const +StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, ContextPtr) const { std::lock_guard lock(mutex); auto it = tables.find(table_name); @@ -40,7 +40,7 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, con return {}; } -DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) { std::lock_guard lock(mutex); if (!filter_by_table_name) diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index 5e1e555a524..da1bd6c1852 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -16,12 +16,12 @@ namespace DB class Context; /// A base class for databases that manage their own list of tables. -class DatabaseWithOwnTablesBase : public IDatabase +class DatabaseWithOwnTablesBase : public IDatabase, protected WithContext { public: - bool isTableExist(const String & table_name, const Context & context) const override; + bool isTableExist(const String & table_name, ContextPtr context) const override; - StoragePtr tryGetTable(const String & table_name, const Context & context) const override; + StoragePtr tryGetTable(const String & table_name, ContextPtr context) const override; bool empty() const override; @@ -29,7 +29,7 @@ public: StoragePtr detachTable(const String & table_name) override; - DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override; void shutdown() override; @@ -38,9 +38,8 @@ public: protected: Tables tables; Poco::Logger * log; - const Context & global_context; - DatabaseWithOwnTablesBase(const String & name_, const String & logger, const Context & context); + DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context); void attachTableUnlocked(const String & table_name, const StoragePtr & table, std::unique_lock & lock); StoragePtr detachTableUnlocked(const String & table_name, std::unique_lock & lock); diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 3a196f827b7..6cd0e468709 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -10,6 +10,7 @@ #include #include + #include #include #include @@ -18,7 +19,6 @@ namespace DB { -class Context; struct Settings; struct ConstraintsDescription; struct IndicesDescription; @@ -153,10 +153,10 @@ public: /// Load a set of existing tables. /// You can call only once, right after the object is created. - virtual void loadStoredObjects(Context & /*context*/, bool /*has_force_restore_data_flag*/, bool /*force_attach*/ = false) {} + virtual void loadStoredObjects(ContextPtr /*context*/, bool /*has_force_restore_data_flag*/, bool /*force_attach*/ = false) {} /// Check the existence of the table. - virtual bool isTableExist(const String & name, const Context & context) const = 0; + virtual bool isTableExist(const String & name, ContextPtr context) const = 0; /// Check the existence of the dictionary virtual bool isDictionaryExist(const String & /*name*/) const @@ -165,7 +165,7 @@ public: } /// Get the table for work. Return nullptr if there is no table. - virtual StoragePtr tryGetTable(const String & name, const Context & context) const = 0; + virtual StoragePtr tryGetTable(const String & name, ContextPtr context) const = 0; virtual UUID tryGetTableUUID(const String & /*table_name*/) const { return UUIDHelpers::Nil; } @@ -173,7 +173,7 @@ public: /// Get an iterator that allows you to pass through all the tables. /// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above. - virtual DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0; + virtual DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name = {}) = 0; /// Get an iterator to pass through all the dictionaries. virtual DatabaseDictionariesIteratorPtr getDictionariesIterator([[maybe_unused]] const FilterByNameFunction & filter_by_dictionary_name = {}) @@ -186,7 +186,7 @@ public: /// Add the table to the database. Record its presence in the metadata. virtual void createTable( - const Context & /*context*/, + ContextPtr /*context*/, const String & /*name*/, const StoragePtr & /*table*/, const ASTPtr & /*query*/) @@ -196,7 +196,7 @@ public: /// Add the dictionary to the database. Record its presence in the metadata. virtual void createDictionary( - const Context & /*context*/, + ContextPtr /*context*/, const String & /*dictionary_name*/, const ASTPtr & /*query*/) { @@ -205,7 +205,7 @@ public: /// Delete the table from the database, drop table and delete the metadata. virtual void dropTable( - const Context & /*context*/, + ContextPtr /*context*/, const String & /*name*/, [[maybe_unused]] bool no_delay = false) { @@ -214,7 +214,7 @@ public: /// Delete the dictionary from the database. Delete the metadata. virtual void removeDictionary( - const Context & /*context*/, + ContextPtr /*context*/, const String & /*dictionary_name*/) { throw Exception("There is no DROP DICTIONARY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); @@ -249,14 +249,14 @@ public: /// Forget about the table without deleting it's data, but rename metadata file to prevent reloading it /// with next restart. The database may not support this method. - virtual void detachTablePermanently(const Context & /*context*/, const String & /*name*/) + virtual void detachTablePermanently(ContextPtr /*context*/, const String & /*name*/) { throw Exception("There is no DETACH TABLE PERMANENTLY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); } /// Rename the table and possibly move the table to another database. virtual void renameTable( - const Context & /*context*/, + ContextPtr /*context*/, const String & /*name*/, IDatabase & /*to_database*/, const String & /*to_name*/, @@ -271,7 +271,7 @@ public: /// Change the table structure in metadata. /// You must call under the alter_lock of the corresponding table . If engine_modifier is empty, then engine does not change. virtual void alterTable( - const Context & /*context*/, + ContextPtr /*context*/, const StorageID & /*table_id*/, const StorageInMemoryMetadata & /*metadata*/) { @@ -285,12 +285,12 @@ public: } /// Get the CREATE TABLE query for the table. It can also provide information for detached tables for which there is metadata. - ASTPtr tryGetCreateTableQuery(const String & name, const Context & context) const noexcept + ASTPtr tryGetCreateTableQuery(const String & name, ContextPtr context) const noexcept { return getCreateTableQueryImpl(name, context, false); } - ASTPtr getCreateTableQuery(const String & name, const Context & context) const + ASTPtr getCreateTableQuery(const String & name, ContextPtr context) const { return getCreateTableQueryImpl(name, context, true); } @@ -351,12 +351,12 @@ public: virtual void shutdown() = 0; /// Delete data and metadata stored inside the database, if exists. - virtual void drop(const Context & /*context*/) {} + virtual void drop(ContextPtr /*context*/) {} - virtual ~IDatabase() {} + virtual ~IDatabase() = default; protected: - virtual ASTPtr getCreateTableQueryImpl(const String & /*name*/, const Context & /*context*/, bool throw_on_error) const + virtual ASTPtr getCreateTableQueryImpl(const String & /*name*/, ContextPtr /*context*/, bool throw_on_error) const { if (throw_on_error) throw Exception("There is no SHOW CREATE TABLE query for Database" + getEngineName(), ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY); diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.cpp b/src/Databases/MySQL/ConnectionMySQLSettings.cpp index fa92e793225..1026d14018b 100644 --- a/src/Databases/MySQL/ConnectionMySQLSettings.cpp +++ b/src/Databases/MySQL/ConnectionMySQLSettings.cpp @@ -50,12 +50,12 @@ void ConnectionMySQLSettings::loadFromQuery(ASTStorage & storage_def) #undef ADD_IF_ABSENT } -void ConnectionMySQLSettings::loadFromQueryContext(const Context & context) +void ConnectionMySQLSettings::loadFromQueryContext(ContextPtr context) { - if (!context.hasQueryContext()) + if (!context->hasQueryContext()) return; - const Settings & settings = context.getQueryContext().getSettingsRef(); + const Settings & settings = context->getQueryContext()->getSettingsRef(); if (settings.mysql_datatypes_support_level.value != mysql_datatypes_support_level.value) set("mysql_datatypes_support_level", settings.mysql_datatypes_support_level.toString()); diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.h b/src/Databases/MySQL/ConnectionMySQLSettings.h index ce2773307c5..f05985a0cda 100644 --- a/src/Databases/MySQL/ConnectionMySQLSettings.h +++ b/src/Databases/MySQL/ConnectionMySQLSettings.h @@ -1,13 +1,13 @@ #pragma once -#include #include +#include #include +#include namespace DB { -class Context; class ASTStorage; #define LIST_OF_CONNECTION_MYSQL_SETTINGS(M) \ @@ -27,7 +27,7 @@ struct ConnectionMySQLSettings : public BaseSettings settings_, mysqlxx::Pool && pool) +DatabaseConnectionMySQL::DatabaseConnectionMySQL( + ContextPtr context_, + const String & database_name_, + const String & metadata_path_, + const ASTStorage * database_engine_define_, + const String & database_name_in_mysql_, + std::unique_ptr settings_, + mysqlxx::PoolWithFailover && pool) : IDatabase(database_name_) - , global_context(context.getGlobalContext()) + , WithContext(context_->getGlobalContext()) , metadata_path(metadata_path_) , database_engine_define(database_engine_define_->clone()) , database_name_in_mysql(database_name_in_mysql_) @@ -63,7 +69,7 @@ bool DatabaseConnectionMySQL::empty() const { std::lock_guard lock(mutex); - fetchTablesIntoLocalCache(global_context); + fetchTablesIntoLocalCache(getContext()); if (local_tables_cache.empty()) return true; @@ -75,12 +81,12 @@ bool DatabaseConnectionMySQL::empty() const return true; } -DatabaseTablesIteratorPtr DatabaseConnectionMySQL::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseConnectionMySQL::getTablesIterator(ContextPtr local_context, const FilterByNameFunction & filter_by_table_name) { Tables tables; std::lock_guard lock(mutex); - fetchTablesIntoLocalCache(context); + fetchTablesIntoLocalCache(local_context); for (const auto & [table_name, modify_time_and_storage] : local_tables_cache) if (!remove_or_detach_tables.count(table_name) && (!filter_by_table_name || filter_by_table_name(table_name))) @@ -89,16 +95,16 @@ DatabaseTablesIteratorPtr DatabaseConnectionMySQL::getTablesIterator(const Conte return std::make_unique(tables, database_name); } -bool DatabaseConnectionMySQL::isTableExist(const String & name, const Context & context) const +bool DatabaseConnectionMySQL::isTableExist(const String & name, ContextPtr local_context) const { - return bool(tryGetTable(name, context)); + return bool(tryGetTable(name, local_context)); } -StoragePtr DatabaseConnectionMySQL::tryGetTable(const String & mysql_table_name, const Context & context) const +StoragePtr DatabaseConnectionMySQL::tryGetTable(const String & mysql_table_name, ContextPtr local_context) const { std::lock_guard lock(mutex); - fetchTablesIntoLocalCache(context); + fetchTablesIntoLocalCache(local_context); if (!remove_or_detach_tables.count(mysql_table_name) && local_tables_cache.find(mysql_table_name) != local_tables_cache.end()) return local_tables_cache[mysql_table_name].second; @@ -153,11 +159,11 @@ static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr return create_table_query; } -ASTPtr DatabaseConnectionMySQL::getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const +ASTPtr DatabaseConnectionMySQL::getCreateTableQueryImpl(const String & table_name, ContextPtr local_context, bool throw_on_error) const { std::lock_guard lock(mutex); - fetchTablesIntoLocalCache(context); + fetchTablesIntoLocalCache(local_context); if (local_tables_cache.find(table_name) == local_tables_cache.end()) { @@ -174,7 +180,7 @@ time_t DatabaseConnectionMySQL::getObjectMetadataModificationTime(const String & { std::lock_guard lock(mutex); - fetchTablesIntoLocalCache(global_context); + fetchTablesIntoLocalCache(getContext()); if (local_tables_cache.find(table_name) == local_tables_cache.end()) throw Exception("MySQL table " + database_name_in_mysql + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); @@ -190,12 +196,12 @@ ASTPtr DatabaseConnectionMySQL::getCreateDatabaseQuery() const return create_query; } -void DatabaseConnectionMySQL::fetchTablesIntoLocalCache(const Context & context) const +void DatabaseConnectionMySQL::fetchTablesIntoLocalCache(ContextPtr local_context) const { const auto & tables_with_modification_time = fetchTablesWithModificationTime(); destroyLocalCacheExtraTables(tables_with_modification_time); - fetchLatestTablesStructureIntoCache(tables_with_modification_time, context); + fetchLatestTablesStructureIntoCache(tables_with_modification_time, local_context); } void DatabaseConnectionMySQL::destroyLocalCacheExtraTables(const std::map & tables_with_modification_time) const @@ -212,7 +218,8 @@ void DatabaseConnectionMySQL::destroyLocalCacheExtraTables(const std::map &tables_modification_time, const Context & context) const +void DatabaseConnectionMySQL::fetchLatestTablesStructureIntoCache( + const std::map & tables_modification_time, ContextPtr local_context) const { std::vector wait_update_tables_name; for (const auto & table_modification_time : tables_modification_time) @@ -224,7 +231,7 @@ void DatabaseConnectionMySQL::fetchLatestTablesStructureIntoCache(const std::map wait_update_tables_name.emplace_back(table_modification_time.first); } - std::map tables_and_columns = fetchTablesColumnsList(wait_update_tables_name, context); + std::map tables_and_columns = fetchTablesColumnsList(wait_update_tables_name, local_context); for (const auto & table_and_columns : tables_and_columns) { @@ -241,7 +248,7 @@ void DatabaseConnectionMySQL::fetchLatestTablesStructureIntoCache(const std::map local_tables_cache[table_name] = std::make_pair(table_modification_time, StorageMySQL::create( StorageID(database_name, table_name), std::move(mysql_pool), database_name_in_mysql, table_name, - false, "", ColumnsDescription{columns_name_and_type}, ConstraintsDescription{}, global_context)); + false, "", ColumnsDescription{columns_name_and_type}, ConstraintsDescription{}, getContext())); } } @@ -276,9 +283,10 @@ std::map DatabaseConnectionMySQL::fetchTablesWithModificationTim return tables_with_modification_time; } -std::map DatabaseConnectionMySQL::fetchTablesColumnsList(const std::vector & tables_name, const Context & context) const +std::map +DatabaseConnectionMySQL::fetchTablesColumnsList(const std::vector & tables_name, ContextPtr local_context) const { - const auto & settings = context.getSettingsRef(); + const auto & settings = local_context->getSettingsRef(); return DB::fetchTablesColumnsList( mysql_pool, @@ -303,7 +311,7 @@ void DatabaseConnectionMySQL::shutdown() local_tables_cache.clear(); } -void DatabaseConnectionMySQL::drop(const Context & /*context*/) +void DatabaseConnectionMySQL::drop(ContextPtr /*context*/) { Poco::File(getMetadataPath()).remove(true); } @@ -378,7 +386,7 @@ String DatabaseConnectionMySQL::getMetadataPath() const return metadata_path; } -void DatabaseConnectionMySQL::loadStoredObjects(Context &, bool, bool /*force_attach*/) +void DatabaseConnectionMySQL::loadStoredObjects(ContextPtr, bool, bool /*force_attach*/) { std::lock_guard lock{mutex}; @@ -395,7 +403,7 @@ void DatabaseConnectionMySQL::loadStoredObjects(Context &, bool, bool /*force_at } } -void DatabaseConnectionMySQL::detachTablePermanently(const Context &, const String & table_name) +void DatabaseConnectionMySQL::detachTablePermanently(ContextPtr, const String & table_name) { std::lock_guard lock{mutex}; @@ -429,9 +437,9 @@ void DatabaseConnectionMySQL::detachTablePermanently(const Context &, const Stri table_iter->second.second->is_dropped = true; } -void DatabaseConnectionMySQL::dropTable(const Context & context, const String & table_name, bool /*no_delay*/) +void DatabaseConnectionMySQL::dropTable(ContextPtr local_context, const String & table_name, bool /*no_delay*/) { - detachTablePermanently(context, table_name); + detachTablePermanently(local_context, table_name); } DatabaseConnectionMySQL::~DatabaseConnectionMySQL() @@ -456,7 +464,7 @@ DatabaseConnectionMySQL::~DatabaseConnectionMySQL() } } -void DatabaseConnectionMySQL::createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) +void DatabaseConnectionMySQL::createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) { const auto & create = create_query->as(); @@ -467,7 +475,7 @@ void DatabaseConnectionMySQL::createTable(const Context &, const String & table_ /// XXX: hack /// In order to prevent users from broken the table structure by executing attach table database_name.table_name (...) /// we should compare the old and new create_query to make them completely consistent - const auto & origin_create_query = getCreateTableQuery(table_name, global_context); + const auto & origin_create_query = getCreateTableQuery(table_name, getContext()); origin_create_query->as()->attach = true; if (queryToString(origin_create_query) != queryToString(create_query)) diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.h b/src/Databases/MySQL/DatabaseConnectionMySQL.h index d0a5c041d7b..a626924d6dd 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.h +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.h @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -30,15 +31,19 @@ enum class MySQLDataTypesSupport; * It doesn't make any manipulations with filesystem. * All tables are created by calling code after real-time pull-out structure from remote MySQL */ -class DatabaseConnectionMySQL final : public IDatabase +class DatabaseConnectionMySQL final : public IDatabase, WithContext { public: ~DatabaseConnectionMySQL() override; DatabaseConnectionMySQL( - const Context & context, const String & database_name, const String & metadata_path, - const ASTStorage * database_engine_define, const String & database_name_in_mysql, std::unique_ptr settings_, - mysqlxx::Pool && pool); + ContextPtr context, + const String & database_name, + const String & metadata_path, + const ASTStorage * database_engine_define, + const String & database_name_in_mysql, + std::unique_ptr settings_, + mysqlxx::PoolWithFailover && pool); String getEngineName() const override { return "MySQL"; } @@ -50,39 +55,38 @@ public: bool empty() const override; - DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override; ASTPtr getCreateDatabaseQuery() const override; - bool isTableExist(const String & name, const Context & context) const override; + bool isTableExist(const String & name, ContextPtr context) const override; - StoragePtr tryGetTable(const String & name, const Context & context) const override; + StoragePtr tryGetTable(const String & name, ContextPtr context) const override; time_t getObjectMetadataModificationTime(const String & name) const override; void shutdown() override; - void drop(const Context & /*context*/) override; + void drop(ContextPtr /*context*/) override; String getMetadataPath() const override; - void createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override; + void createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override; - void loadStoredObjects(Context &, bool, bool force_attach) override; + void loadStoredObjects(ContextPtr, bool, bool force_attach) override; StoragePtr detachTable(const String & table_name) override; - void detachTablePermanently(const Context & context, const String & table_name) override; + void detachTablePermanently(ContextPtr context, const String & table_name) override; - void dropTable(const Context & context, const String & table_name, bool no_delay) override; + void dropTable(ContextPtr context, const String & table_name, bool no_delay) override; void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; protected: - ASTPtr getCreateTableQueryImpl(const String & name, const Context & context, bool throw_on_error) const override; + ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override; private: - const Context & global_context; String metadata_path; ASTPtr database_engine_define; String database_name_in_mysql; @@ -91,7 +95,7 @@ private: std::atomic quit{false}; std::condition_variable cond; - using MySQLPool = mysqlxx::Pool; + using MySQLPool = mysqlxx::PoolWithFailover; using ModifyTimeAndStorage = std::pair; mutable MySQLPool mysql_pool; @@ -102,15 +106,15 @@ private: void cleanOutdatedTables(); - void fetchTablesIntoLocalCache(const Context & context) const; + void fetchTablesIntoLocalCache(ContextPtr context) const; std::map fetchTablesWithModificationTime() const; - std::map fetchTablesColumnsList(const std::vector & tables_name, const Context & context) const; + std::map fetchTablesColumnsList(const std::vector & tables_name, ContextPtr context) const; void destroyLocalCacheExtraTables(const std::map & tables_with_modification_time) const; - void fetchLatestTablesStructureIntoCache(const std::map & tables_modification_time, const Context & context) const; + void fetchLatestTablesStructureIntoCache(const std::map & tables_modification_time, ContextPtr context) const; ThreadFromGlobalPool thread; }; diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 8edeabee004..62a66b22c93 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -26,27 +26,40 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -template<> +template <> DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( - const Context & context, const String & database_name_, const String & metadata_path_, UUID /*uuid*/, - const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, std::unique_ptr settings_) - : DatabaseOrdinary(database_name_ - , metadata_path_ - , "data/" + escapeForFileName(database_name_) + "/" - , "DatabaseMaterializeMySQL (" + database_name_ + ")", context - ) + ContextPtr context_, + const String & database_name_, + const String & metadata_path_, + UUID /*uuid*/, + const String & mysql_database_name_, + mysqlxx::Pool && pool_, + MySQLClient && client_, + std::unique_ptr settings_) + : DatabaseOrdinary( + database_name_, + metadata_path_, + "data/" + escapeForFileName(database_name_) + "/", + "DatabaseMaterializeMySQL (" + database_name_ + ")", + context_) , settings(std::move(settings_)) - , materialize_thread(context, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get()) + , materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get()) { } -template<> +template <> DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( - const Context & context, const String & database_name_, const String & metadata_path_, UUID uuid, - const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, std::unique_ptr settings_) - : DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabaseMaterializeMySQL (" + database_name_ + ")", context) + ContextPtr context_, + const String & database_name_, + const String & metadata_path_, + UUID uuid, + const String & mysql_database_name_, + mysqlxx::Pool && pool_, + MySQLClient && client_, + std::unique_ptr settings_) + : DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabaseMaterializeMySQL (" + database_name_ + ")", context_) , settings(std::move(settings_)) - , materialize_thread(context, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get()) + , materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get()) { } @@ -79,9 +92,9 @@ void DatabaseMaterializeMySQL::setException(const std::exception_ptr & exc } template -void DatabaseMaterializeMySQL::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) +void DatabaseMaterializeMySQL::loadStoredObjects(ContextPtr context_, bool has_force_restore_data_flag, bool force_attach) { - Base::loadStoredObjects(context, has_force_restore_data_flag, force_attach); + Base::loadStoredObjects(context_, has_force_restore_data_flag, force_attach); if (!force_attach) materialize_thread.assertMySQLAvailable(); @@ -90,17 +103,17 @@ void DatabaseMaterializeMySQL::loadStoredObjects(Context & context, bool h } template -void DatabaseMaterializeMySQL::createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) +void DatabaseMaterializeMySQL::createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) { assertCalledFromSyncThreadOrDrop("create table"); - Base::createTable(context, name, table, query); + Base::createTable(context_, name, table, query); } template -void DatabaseMaterializeMySQL::dropTable(const Context & context, const String & name, bool no_delay) +void DatabaseMaterializeMySQL::dropTable(ContextPtr context_, const String & name, bool no_delay) { assertCalledFromSyncThreadOrDrop("drop table"); - Base::dropTable(context, name, no_delay); + Base::dropTable(context_, name, no_delay); } template @@ -118,7 +131,7 @@ StoragePtr DatabaseMaterializeMySQL::detachTable(const String & name) } template -void DatabaseMaterializeMySQL::renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) +void DatabaseMaterializeMySQL::renameTable(ContextPtr context_, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) { assertCalledFromSyncThreadOrDrop("rename table"); @@ -131,18 +144,18 @@ void DatabaseMaterializeMySQL::renameTable(const Context & context, const if (to_database.getDatabaseName() != Base::getDatabaseName()) throw Exception("Cannot rename with other database for MaterializeMySQL database.", ErrorCodes::NOT_IMPLEMENTED); - Base::renameTable(context, name, *this, to_name, exchange, dictionary); + Base::renameTable(context_, name, *this, to_name, exchange, dictionary); } template -void DatabaseMaterializeMySQL::alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) +void DatabaseMaterializeMySQL::alterTable(ContextPtr context_, const StorageID & table_id, const StorageInMemoryMetadata & metadata) { assertCalledFromSyncThreadOrDrop("alter table"); - Base::alterTable(context, table_id, metadata); + Base::alterTable(context_, table_id, metadata); } template -void DatabaseMaterializeMySQL::drop(const Context & context) +void DatabaseMaterializeMySQL::drop(ContextPtr context_) { /// Remove metadata info Poco::File metadata(Base::getMetadataPath() + "/.metadata"); @@ -150,15 +163,15 @@ void DatabaseMaterializeMySQL::drop(const Context & context) if (metadata.exists()) metadata.remove(false); - Base::drop(context); + Base::drop(context_); } template -StoragePtr DatabaseMaterializeMySQL::tryGetTable(const String & name, const Context & context) const +StoragePtr DatabaseMaterializeMySQL::tryGetTable(const String & name, ContextPtr context_) const { if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) { - StoragePtr nested_storage = Base::tryGetTable(name, context); + StoragePtr nested_storage = Base::tryGetTable(name, context_); if (!nested_storage) return {}; @@ -166,19 +179,20 @@ StoragePtr DatabaseMaterializeMySQL::tryGetTable(const String & name, cons return std::make_shared(std::move(nested_storage), this); } - return Base::tryGetTable(name, context); + return Base::tryGetTable(name, context_); } -template -DatabaseTablesIteratorPtr DatabaseMaterializeMySQL::getTablesIterator(const Context & context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) +template +DatabaseTablesIteratorPtr +DatabaseMaterializeMySQL::getTablesIterator(ContextPtr context_, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) { if (!MaterializeMySQLSyncThread::isMySQLSyncThread()) { - DatabaseTablesIteratorPtr iterator = Base::getTablesIterator(context, filter_by_table_name); + DatabaseTablesIteratorPtr iterator = Base::getTablesIterator(context_, filter_by_table_name); return std::make_unique(std::move(iterator), this); } - return Base::getTablesIterator(context, filter_by_table_name); + return Base::getTablesIterator(context_, filter_by_table_name); } template diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h index e1229269a33..cd9fe640239 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -23,7 +23,7 @@ class DatabaseMaterializeMySQL : public Base public: DatabaseMaterializeMySQL( - const Context & context, const String & database_name_, const String & metadata_path_, UUID uuid, + ContextPtr context, const String & database_name_, const String & metadata_path_, UUID uuid, const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, std::unique_ptr settings_); @@ -43,25 +43,25 @@ protected: public: String getEngineName() const override { return "MaterializeMySQL"; } - void loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) override; + void loadStoredObjects(ContextPtr context_, bool has_force_restore_data_flag, bool force_attach) override; - void createTable(const Context & context, const String & name, const StoragePtr & table, const ASTPtr & query) override; + void createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) override; - void dropTable(const Context & context, const String & name, bool no_delay) override; + void dropTable(ContextPtr context_, const String & name, bool no_delay) override; void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override; StoragePtr detachTable(const String & name) override; - void renameTable(const Context & context, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) override; + void renameTable(ContextPtr context_, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) override; - void alterTable(const Context & context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; + void alterTable(ContextPtr context_, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; - void drop(const Context & context) override; + void drop(ContextPtr context_) override; - StoragePtr tryGetTable(const String & name, const Context & context) const override; + StoragePtr tryGetTable(const String & name, ContextPtr context_) const override; - DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context_, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) override; void assertCalledFromSyncThreadOrDrop(const char * method) const; diff --git a/src/Databases/MySQL/FetchTablesColumnsList.cpp b/src/Databases/MySQL/FetchTablesColumnsList.cpp index 3e25c703a1d..6344553ba69 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.cpp +++ b/src/Databases/MySQL/FetchTablesColumnsList.cpp @@ -41,7 +41,7 @@ namespace DB { std::map fetchTablesColumnsList( - mysqlxx::Pool & pool, + mysqlxx::PoolWithFailover & pool, const String & database_name, const std::vector & tables_name, bool external_table_functions_use_nulls, diff --git a/src/Databases/MySQL/FetchTablesColumnsList.h b/src/Databases/MySQL/FetchTablesColumnsList.h index 52191c2ecb8..d609f4896e7 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.h +++ b/src/Databases/MySQL/FetchTablesColumnsList.h @@ -3,7 +3,7 @@ #include "config_core.h" #if USE_MYSQL -#include +#include #include #include @@ -17,7 +17,7 @@ namespace DB { std::map fetchTablesColumnsList( - mysqlxx::Pool & pool, + mysqlxx::PoolWithFailover & pool, const String & database_name, const std::vector & tables_name, bool external_table_functions_use_nulls, diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 8d1c1e109ac..b8e135eef32 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -41,30 +41,30 @@ namespace ErrorCodes static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; -static Context createQueryContext(const Context & global_context) +static ContextPtr createQueryContext(ContextPtr context) { - Settings new_query_settings = global_context.getSettings(); + Settings new_query_settings = context->getSettings(); new_query_settings.insert_allow_materialized_columns = true; /// To avoid call AST::format /// TODO: We need to implement the format function for MySQLAST new_query_settings.enable_global_with_statement = false; - Context query_context(global_context); - query_context.setSettings(new_query_settings); + auto query_context = Context::createCopy(context); + query_context->setSettings(new_query_settings); CurrentThread::QueryScope query_scope(query_context); - query_context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - query_context.setCurrentQueryId(""); // generate random query_id + query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + query_context->setCurrentQueryId(""); // generate random query_id return query_context; } -static BlockIO tryToExecuteQuery(const String & query_to_execute, Context & query_context, const String & database, const String & comment) +static BlockIO tryToExecuteQuery(const String & query_to_execute, ContextPtr query_context, const String & database, const String & comment) { try { if (!database.empty()) - query_context.setCurrentDatabase(database); + query_context->setCurrentDatabase(database); return executeQuery("/*" + comment + "*/ " + query_to_execute, query_context, true); } @@ -143,10 +143,19 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection) } MaterializeMySQLSyncThread::MaterializeMySQLSyncThread( - const Context & context, const String & database_name_, const String & mysql_database_name_, - mysqlxx::Pool && pool_, MySQLClient && client_, MaterializeMySQLSettings * settings_) - : log(&Poco::Logger::get("MaterializeMySQLSyncThread")), global_context(context.getGlobalContext()), database_name(database_name_) - , mysql_database_name(mysql_database_name_), pool(std::move(pool_)), client(std::move(client_)), settings(settings_) + ContextPtr context_, + const String & database_name_, + const String & mysql_database_name_, + mysqlxx::Pool && pool_, + MySQLClient && client_, + MaterializeMySQLSettings * settings_) + : WithContext(context_->getGlobalContext()) + , log(&Poco::Logger::get("MaterializeMySQLSyncThread")) + , database_name(database_name_) + , mysql_database_name(mysql_database_name_) + , pool(std::move(pool_)) + , client(std::move(client_)) + , settings(settings_) { query_prefix = "EXTERNAL DDL FROM MySQL(" + backQuoteIfNeed(database_name) + ", " + backQuoteIfNeed(mysql_database_name) + ") "; } @@ -248,7 +257,7 @@ void MaterializeMySQLSyncThread::assertMySQLAvailable() } } -static inline void cleanOutdatedTables(const String & database_name, const Context & context) +static inline void cleanOutdatedTables(const String & database_name, ContextPtr context) { String cleaning_table_name; try @@ -258,7 +267,7 @@ static inline void cleanOutdatedTables(const String & database_name, const Conte for (auto iterator = clean_database->getTablesIterator(context); iterator->isValid(); iterator->next()) { - Context query_context = createQueryContext(context); + auto query_context = createQueryContext(context); String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data"; cleaning_table_name = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(iterator->name()); tryToExecuteQuery(" DROP TABLE " + cleaning_table_name, query_context, database_name, comment); @@ -271,7 +280,8 @@ static inline void cleanOutdatedTables(const String & database_name, const Conte } } -static inline BlockOutputStreamPtr getTableOutput(const String & database_name, const String & table_name, Context & query_context, bool insert_materialized = false) +static inline BlockOutputStreamPtr +getTableOutput(const String & database_name, const String & table_name, ContextPtr query_context, bool insert_materialized = false) { const StoragePtr & storage = DatabaseCatalog::instance().getTable(StorageID(database_name, table_name), query_context); @@ -303,7 +313,7 @@ static inline BlockOutputStreamPtr getTableOutput(const String & database_name, static inline void dumpDataForTables( mysqlxx::Pool::Entry & connection, const std::unordered_map & need_dumping_tables, const String & query_prefix, const String & database_name, const String & mysql_database_name, - const Context & context, const std::function & is_cancelled) + ContextPtr context, const std::function & is_cancelled) { auto iterator = need_dumping_tables.begin(); for (; iterator != need_dumping_tables.end() && !is_cancelled(); ++iterator) @@ -311,7 +321,7 @@ static inline void dumpDataForTables( try { const auto & table_name = iterator->first; - Context query_context = createQueryContext(context); + auto query_context = createQueryContext(context); String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data"; tryToExecuteQuery(query_prefix + " " + iterator->second, query_context, database_name, comment); /// create table. @@ -376,8 +386,12 @@ bool MaterializeMySQLSyncThread::prepareSynchronized(MaterializeMetadata & metad metadata.transaction(position, [&]() { - cleanOutdatedTables(database_name, global_context); - dumpDataForTables(connection, need_dumping_tables, query_prefix, database_name, mysql_database_name, global_context, [this] { return isCancelled(); }); + cleanOutdatedTables(database_name, getContext()); + dumpDataForTables( + connection, need_dumping_tables, query_prefix, database_name, mysql_database_name, getContext(), [this] + { + return isCancelled(); + }); }); const auto & position_message = [&]() @@ -431,7 +445,7 @@ void MaterializeMySQLSyncThread::flushBuffersData(Buffers & buffers, Materialize if (buffers.data.empty()) return; - metadata.transaction(client.getPosition(), [&]() { buffers.commit(global_context); }); + metadata.transaction(client.getPosition(), [&]() { buffers.commit(getContext()); }); const auto & position_message = [&]() { @@ -664,21 +678,21 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr if (receive_event->type() == MYSQL_WRITE_ROWS_EVENT) { WriteRowsEvent & write_rows_event = static_cast(*receive_event); - Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(write_rows_event.table, global_context); + Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(write_rows_event.table, getContext()); size_t bytes = onWriteOrDeleteData<1>(write_rows_event.rows, buffer->first, ++metadata.data_version); buffers.add(buffer->first.rows(), buffer->first.bytes(), write_rows_event.rows.size(), bytes); } else if (receive_event->type() == MYSQL_UPDATE_ROWS_EVENT) { UpdateRowsEvent & update_rows_event = static_cast(*receive_event); - Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(update_rows_event.table, global_context); + Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(update_rows_event.table, getContext()); size_t bytes = onUpdateData(update_rows_event.rows, buffer->first, ++metadata.data_version, buffer->second); buffers.add(buffer->first.rows(), buffer->first.bytes(), update_rows_event.rows.size(), bytes); } else if (receive_event->type() == MYSQL_DELETE_ROWS_EVENT) { DeleteRowsEvent & delete_rows_event = static_cast(*receive_event); - Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(delete_rows_event.table, global_context); + Buffers::BufferAndSortingColumnsPtr buffer = buffers.getTableDataBuffer(delete_rows_event.table, getContext()); size_t bytes = onWriteOrDeleteData<-1>(delete_rows_event.rows, buffer->first, ++metadata.data_version); buffers.add(buffer->first.rows(), buffer->first.bytes(), delete_rows_event.rows.size(), bytes); } @@ -687,7 +701,7 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr QueryEvent & query_event = static_cast(*receive_event); Position position_before_ddl; position_before_ddl.update(metadata.binlog_position, metadata.binlog_file, metadata.executed_gtid_set); - metadata.transaction(position_before_ddl, [&]() { buffers.commit(global_context); }); + metadata.transaction(position_before_ddl, [&]() { buffers.commit(getContext()); }); metadata.transaction(client.getPosition(),[&](){ executeDDLAtomic(query_event); }); } else @@ -718,7 +732,7 @@ void MaterializeMySQLSyncThread::executeDDLAtomic(const QueryEvent & query_event { try { - Context query_context = createQueryContext(global_context); + auto query_context = createQueryContext(getContext()); String comment = "Materialize MySQL step 2: execute MySQL DDL for sync data"; String event_database = query_event.schema == mysql_database_name ? database_name : ""; tryToExecuteQuery(query_prefix + query_event.query, query_context, event_database, comment); @@ -761,13 +775,13 @@ bool MaterializeMySQLSyncThread::Buffers::checkThresholds(size_t check_block_row || total_blocks_bytes >= check_total_bytes; } -void MaterializeMySQLSyncThread::Buffers::commit(const Context & context) +void MaterializeMySQLSyncThread::Buffers::commit(ContextPtr context) { try { for (auto & table_name_and_buffer : data) { - Context query_context = createQueryContext(context); + auto query_context = createQueryContext(context); OneBlockInputStream input(table_name_and_buffer.second->first); BlockOutputStreamPtr out = getTableOutput(database, table_name_and_buffer.first, query_context, true); copyData(input, *out); @@ -787,7 +801,7 @@ void MaterializeMySQLSyncThread::Buffers::commit(const Context & context) } MaterializeMySQLSyncThread::Buffers::BufferAndSortingColumnsPtr MaterializeMySQLSyncThread::Buffers::getTableDataBuffer( - const String & table_name, const Context & context) + const String & table_name, ContextPtr context) { const auto & iterator = data.find(table_name); if (iterator == data.end()) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 54a6cbbdda2..03958fe10cc 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -36,14 +36,18 @@ namespace DB * real-time pull incremental data: * We will pull the binlog event of MySQL to parse and execute when the full data synchronization is completed. */ -class MaterializeMySQLSyncThread +class MaterializeMySQLSyncThread : WithContext { public: ~MaterializeMySQLSyncThread(); MaterializeMySQLSyncThread( - const Context & context, const String & database_name_, const String & mysql_database_name_ - , mysqlxx::Pool && pool_, MySQLClient && client_, MaterializeMySQLSettings * settings_); + ContextPtr context, + const String & database_name_, + const String & mysql_database_name_, + mysqlxx::Pool && pool_, + MySQLClient && client_, + MaterializeMySQLSettings * settings_); void stopSynchronization(); @@ -55,7 +59,6 @@ public: private: Poco::Logger * log; - const Context & global_context; String database_name; String mysql_database_name; @@ -90,13 +93,13 @@ private: Buffers(const String & database_) : database(database_) {} - void commit(const Context & context); + void commit(ContextPtr context); void add(size_t block_rows, size_t block_bytes, size_t written_rows, size_t written_bytes); bool checkThresholds(size_t check_block_rows, size_t check_block_bytes, size_t check_total_rows, size_t check_total_bytes) const; - BufferAndSortingColumnsPtr getTableDataBuffer(const String & table, const Context & context); + BufferAndSortingColumnsPtr getTableDataBuffer(const String & table, ContextPtr context); }; void synchronization(); diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 229c338d765..e12d7f975ce 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -35,22 +35,22 @@ static const auto suffix = ".removed"; static const auto cleaner_reschedule_ms = 60000; DatabasePostgreSQL::DatabasePostgreSQL( - const Context & context, + ContextPtr context_, const String & metadata_path_, const ASTStorage * database_engine_define_, const String & dbname_, const String & postgres_dbname, - PostgreSQLConnectionPoolPtr connection_pool_, + postgres::PoolWithFailoverPtr connection_pool_, const bool cache_tables_) : IDatabase(dbname_) - , global_context(context.getGlobalContext()) + , WithContext(context_->getGlobalContext()) , metadata_path(metadata_path_) , database_engine_define(database_engine_define_->clone()) , dbname(postgres_dbname) , connection_pool(std::move(connection_pool_)) , cache_tables(cache_tables_) { - cleaner_task = context.getSchedulePool().createTask("PostgreSQLCleanerTask", [this]{ removeOutdatedTables(); }); + cleaner_task = getContext()->getSchedulePool().createTask("PostgreSQLCleanerTask", [this]{ removeOutdatedTables(); }); cleaner_task->deactivate(); } @@ -69,8 +69,7 @@ bool DatabasePostgreSQL::empty() const } -DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator( - const Context & context, const FilterByNameFunction & /* filter_by_table_name */) +DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator(ContextPtr local_context, const FilterByNameFunction & /* filter_by_table_name */) { std::lock_guard lock(mutex); @@ -79,7 +78,7 @@ DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator( for (const auto & table_name : table_names) if (!detached_or_dropped.count(table_name)) - tables[table_name] = fetchTable(table_name, context, true); + tables[table_name] = fetchTable(table_name, local_context, true); return std::make_unique(tables, database_name); } @@ -135,7 +134,7 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const } -bool DatabasePostgreSQL::isTableExist(const String & table_name, const Context & /* context */) const +bool DatabasePostgreSQL::isTableExist(const String & table_name, ContextPtr /* context */) const { std::lock_guard lock(mutex); @@ -146,33 +145,33 @@ bool DatabasePostgreSQL::isTableExist(const String & table_name, const Context & } -StoragePtr DatabasePostgreSQL::tryGetTable(const String & table_name, const Context & context) const +StoragePtr DatabasePostgreSQL::tryGetTable(const String & table_name, ContextPtr local_context) const { std::lock_guard lock(mutex); if (!detached_or_dropped.count(table_name)) - return fetchTable(table_name, context, false); + return fetchTable(table_name, local_context, false); return StoragePtr{}; } -StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, const Context & context, const bool table_checked) const +StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr local_context, const bool table_checked) const { if (!cache_tables || !cached_tables.count(table_name)) { if (!table_checked && !checkPostgresTable(table_name)) return StoragePtr{}; - auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; + auto use_nulls = local_context->getSettingsRef().external_table_functions_use_nulls; auto columns = fetchPostgreSQLTableStructure(connection_pool->get(), doubleQuoteString(table_name), use_nulls); if (!columns) return StoragePtr{}; auto storage = StoragePostgreSQL::create( - StorageID(database_name, table_name), table_name, std::make_shared(*connection_pool), - ColumnsDescription{*columns}, ConstraintsDescription{}, context); + StorageID(database_name, table_name), *connection_pool, table_name, + ColumnsDescription{*columns}, ConstraintsDescription{}, local_context); if (cache_tables) cached_tables[table_name] = storage; @@ -232,7 +231,7 @@ StoragePtr DatabasePostgreSQL::detachTable(const String & table_name) } -void DatabasePostgreSQL::createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) +void DatabasePostgreSQL::createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) { const auto & create = create_query->as(); @@ -243,7 +242,7 @@ void DatabasePostgreSQL::createTable(const Context &, const String & table_name, } -void DatabasePostgreSQL::dropTable(const Context &, const String & table_name, bool /* no_delay */) +void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool /* no_delay */) { std::lock_guard lock{mutex}; @@ -271,13 +270,13 @@ void DatabasePostgreSQL::dropTable(const Context &, const String & table_name, b } -void DatabasePostgreSQL::drop(const Context & /*context*/) +void DatabasePostgreSQL::drop(ContextPtr /*context*/) { Poco::File(getMetadataPath()).remove(true); } -void DatabasePostgreSQL::loadStoredObjects(Context & /* context */, bool, bool /*force_attach*/) +void DatabasePostgreSQL::loadStoredObjects(ContextPtr /* context */, bool, bool /*force_attach*/) { { std::lock_guard lock{mutex}; @@ -349,9 +348,9 @@ ASTPtr DatabasePostgreSQL::getCreateDatabaseQuery() const } -ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const +ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, ContextPtr local_context, bool throw_on_error) const { - auto storage = fetchTable(table_name, context, false); + auto storage = fetchTable(table_name, local_context, false); if (!storage) { if (throw_on_error) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 7be34df2123..3505c38e499 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -9,14 +9,13 @@ #include #include #include +#include namespace DB { class Context; -class PostgreSQLConnectionPool; -using PostgreSQLConnectionPoolPtr = std::shared_ptr; /** Real-time access to table list and table structure from remote PostgreSQL. @@ -24,18 +23,18 @@ using PostgreSQLConnectionPoolPtr = std::shared_ptr; * If `cache_tables` == 1 (default: 0) table structure is cached and not checked for being modififed, * but it will be updated during detach->attach. */ -class DatabasePostgreSQL final : public IDatabase +class DatabasePostgreSQL final : public IDatabase, WithContext { public: DatabasePostgreSQL( - const Context & context, + ContextPtr context, const String & metadata_path_, const ASTStorage * database_engine_define, const String & dbname_, const String & postgres_dbname, - PostgreSQLConnectionPoolPtr connection_pool_, - const bool cache_tables_); + postgres::PoolWithFailoverPtr connection_pool_, + bool cache_tables_); String getEngineName() const override { return "PostgreSQL"; } String getMetadataPath() const override { return metadata_path; } @@ -48,31 +47,30 @@ public: bool empty() const override; - void loadStoredObjects(Context &, bool, bool force_attach) override; + void loadStoredObjects(ContextPtr, bool, bool force_attach) override; - DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override; - bool isTableExist(const String & name, const Context & context) const override; - StoragePtr tryGetTable(const String & name, const Context & context) const override; + bool isTableExist(const String & name, ContextPtr context) const override; + StoragePtr tryGetTable(const String & name, ContextPtr context) const override; - void createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override; - void dropTable(const Context &, const String & table_name, bool no_delay) override; + void createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override; + void dropTable(ContextPtr, const String & table_name, bool no_delay) override; void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; StoragePtr detachTable(const String & table_name) override; - void drop(const Context & /*context*/) override; + void drop(ContextPtr /*context*/) override; void shutdown() override; protected: - ASTPtr getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const override; + ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr context, bool throw_on_error) const override; private: - const Context & global_context; String metadata_path; ASTPtr database_engine_define; String dbname; - PostgreSQLConnectionPoolPtr connection_pool; + postgres::PoolWithFailoverPtr connection_pool; const bool cache_tables; mutable Tables cached_tables; @@ -81,7 +79,7 @@ private: bool checkPostgresTable(const String & table_name) const; std::unordered_set fetchTablesList() const; - StoragePtr fetchTable(const String & table_name, const Context & context, const bool table_checked) const; + StoragePtr fetchTable(const String & table_name, ContextPtr context, bool table_checked) const; void removeOutdatedTables(); ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const; }; diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 4887b47d98a..066090d02d6 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -40,6 +40,8 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl res = std::make_shared(); else if (type == "bigint") res = std::make_shared(); + else if (type == "boolean") + res = std::make_shared(); else if (type == "real") res = std::make_shared(); else if (type == "double precision") @@ -94,7 +96,7 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl std::shared_ptr fetchPostgreSQLTableStructure( - PostgreSQLConnectionHolderPtr connection, const String & postgres_table_name, bool use_nulls) + postgres::ConnectionHolderPtr connection, const String & postgres_table_name, bool use_nulls) { auto columns = NamesAndTypesList(); diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index 96305349062..f40929aa91d 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -12,7 +12,7 @@ namespace DB { std::shared_ptr fetchPostgreSQLTableStructure( - PostgreSQLConnectionHolderPtr connection, const String & postgres_table_name, bool use_nulls); + postgres::ConnectionHolderPtr connection, const String & postgres_table_name, bool use_nulls); } diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 535e862af40..b1b8ebed5bd 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -70,7 +70,7 @@ CacheDictionary::CacheDictionary( , rnd_engine(randomSeed()) { if (!source_ptr->supportsSelectiveLoad()) - throw Exception{full_name + ": source cannot be used with CacheDictionary", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: source cannot be used with CacheDictionary", full_name); } template @@ -133,7 +133,7 @@ ColumnPtr CacheDictionary::getColumn( template Columns CacheDictionary::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::getColumns( DictionaryKeysExtractor 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::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; diff --git a/src/Dictionaries/CacheDictionaryStorage.h b/src/Dictionaries/CacheDictionaryStorage.h index 98031801b11..720800e6357 100644 --- a/src/Dictionaries/CacheDictionaryStorage.h +++ b/src/Dictionaries/CacheDictionaryStorage.h @@ -77,7 +77,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::simple) return fetchColumnsForKeysImpl(keys, fetch_request); else - throw Exception("Method fetchColumnsForKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for complex key storage"); } void insertColumnsForKeys(const PaddedPODArray & keys, Columns columns) override @@ -85,7 +85,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::simple) insertColumnsForKeysImpl(keys, columns); else - throw Exception("Method insertColumnsForKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage"); } void insertDefaultKeys(const PaddedPODArray & keys) override @@ -93,7 +93,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::simple) insertDefaultKeysImpl(keys); else - throw Exception("Method insertDefaultKeysImpl is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage"); } PaddedPODArray getCachedSimpleKeys() const override @@ -101,7 +101,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::simple) return getCachedKeysImpl(); else - throw Exception("Method getCachedSimpleKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage"); } bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; } @@ -113,7 +113,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::complex) return fetchColumnsForKeysImpl(keys, column_fetch_requests); else - throw Exception("Method fetchColumnsForKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage"); } void insertColumnsForKeys(const PaddedPODArray & keys, Columns columns) override @@ -121,7 +121,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::complex) insertColumnsForKeysImpl(keys, columns); else - throw Exception("Method insertColumnsForKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage"); } void insertDefaultKeys(const PaddedPODArray & keys) override @@ -129,7 +129,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::complex) insertDefaultKeysImpl(keys); else - throw Exception("Method insertDefaultKeysImpl is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage"); } PaddedPODArray getCachedComplexKeys() const override @@ -137,7 +137,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::complex) return getCachedKeysImpl(); else - throw Exception("Method getCachedComplexKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedComplexKeys is not supported for simple key storage"); } size_t getSize() const override { return size; } @@ -601,6 +601,7 @@ private: PaddedPODArray, PaddedPODArray, PaddedPODArray, + PaddedPODArray, PaddedPODArray, PaddedPODArray, PaddedPODArray, diff --git a/src/Dictionaries/CacheDictionaryUpdateQueue.cpp b/src/Dictionaries/CacheDictionaryUpdateQueue.cpp index eca833f62da..310abed822f 100644 --- a/src/Dictionaries/CacheDictionaryUpdateQueue.cpp +++ b/src/Dictionaries/CacheDictionaryUpdateQueue.cpp @@ -49,11 +49,10 @@ template void CacheDictionaryUpdateQueue::tryPushToUpdateQueueOrThrow(CacheDictionaryUpdateUnitPtr & update_unit_ptr) { if (finished) - throw Exception{"CacheDictionaryUpdateQueue finished", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "CacheDictionaryUpdateQueue finished"); if (!update_queue.tryPush(update_unit_ptr, configuration.update_queue_push_timeout_milliseconds)) - throw DB::Exception( - ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL, + throw DB::Exception(ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL, "Cannot push to internal update queue in dictionary {}. " "Timelimit of {} ms. exceeded. Current queue size is {}", dictionary_name_for_logs, @@ -65,7 +64,7 @@ template void CacheDictionaryUpdateQueue::waitForCurrentUpdateFinish(CacheDictionaryUpdateUnitPtr & update_unit_ptr) const { if (finished) - throw Exception{"CacheDictionaryUpdateQueue finished", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "CacheDictionaryUpdateQueue finished"); std::unique_lock update_lock(update_mutex); diff --git a/src/Dictionaries/CassandraBlockInputStream.cpp b/src/Dictionaries/CassandraBlockInputStream.cpp index 721cb44a82e..4e71c212451 100644 --- a/src/Dictionaries/CassandraBlockInputStream.cpp +++ b/src/Dictionaries/CassandraBlockInputStream.cpp @@ -142,7 +142,7 @@ void CassandraBlockInputStream::insertValue(IColumn & column, ValueType type, co break; } default: - throw Exception("Unknown type : " + std::to_string(static_cast(type)), ErrorCodes::UNKNOWN_TYPE); + throw Exception(ErrorCodes::UNKNOWN_TYPE, "Unknown type : {}", std::to_string(static_cast(type))); } } @@ -256,7 +256,7 @@ void CassandraBlockInputStream::assertTypes(const CassResultPtr & result) expected_text = "uuid"; break; default: - throw Exception("Unknown type : " + std::to_string(static_cast(description.types[i].first)), ErrorCodes::UNKNOWN_TYPE); + throw Exception(ErrorCodes::UNKNOWN_TYPE, "Unknown type : {}", std::to_string(static_cast(description.types[i].first))); } CassValueType got = cass_result_column_type(result, i); @@ -267,8 +267,10 @@ void CassandraBlockInputStream::assertTypes(const CassResultPtr & result) continue; const auto & column_name = description.sample_block.getColumnsWithTypeAndName()[i].name; - throw Exception("Type mismatch for column " + column_name + ": expected Cassandra type " + expected_text, - ErrorCodes::TYPE_MISMATCH); + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Type mismatch for column {} : expected Cassandra type {}", + column_name, + expected_text); } } diff --git a/src/Dictionaries/CassandraDictionarySource.cpp b/src/Dictionaries/CassandraDictionarySource.cpp index 643e9af54b8..b4f7f3221bc 100644 --- a/src/Dictionaries/CassandraDictionarySource.cpp +++ b/src/Dictionaries/CassandraDictionarySource.cpp @@ -17,7 +17,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory) [[maybe_unused]] const Poco::Util::AbstractConfiguration & config, [[maybe_unused]] const std::string & config_prefix, [[maybe_unused]] Block & sample_block, - const Context & /* context */, + ContextPtr /* context */, const std::string & /* default_database */, bool /*check_config*/) -> DictionarySourcePtr { @@ -25,8 +25,8 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory) setupCassandraDriverLibraryLogging(CASS_LOG_INFO); return std::make_unique(dict_struct, config, config_prefix + ".cassandra", sample_block); #else - throw Exception{"Dictionary source of type `cassandra` is disabled because ClickHouse was built without cassandra support.", - ErrorCodes::SUPPORT_IS_DISABLED}; + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Dictionary source of type `cassandra` is disabled because ClickHouse was built without cassandra support."); #endif }; factory.registerSource("cassandra", create_table_source); @@ -90,7 +90,7 @@ void CassandraSettings::setConsistency(const String & config_str) else if (config_str == "LocalSerial") consistency = CASS_CONSISTENCY_LOCAL_SERIAL; else /// CASS_CONSISTENCY_ANY is only valid for writes - throw Exception("Unsupported consistency level: " + config_str, ErrorCodes::INVALID_CONFIG_PARAMETER); + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Unsupported consistency level: {}", config_str); } static const size_t max_block_size = 8192; @@ -156,7 +156,7 @@ BlockInputStreamPtr CassandraDictionarySource::loadIds(const std::vector BlockInputStreamPtr CassandraDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { if (requested_rows.empty()) - throw Exception("No rows requested", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No rows requested"); /// TODO is there a better way to load data by complex keys? std::unordered_map> partitions; @@ -185,7 +185,7 @@ BlockInputStreamPtr CassandraDictionarySource::loadKeys(const Columns & key_colu BlockInputStreamPtr CassandraDictionarySource::loadUpdatedAll() { - throw Exception("Method loadUpdatedAll is unsupported for CassandraDictionarySource", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for CassandraDictionarySource"); } CassSessionShared CassandraDictionarySource::getSession() diff --git a/src/Dictionaries/CassandraHelpers.cpp b/src/Dictionaries/CassandraHelpers.cpp index 6de80a455c7..81f7d6d9a63 100644 --- a/src/Dictionaries/CassandraHelpers.cpp +++ b/src/Dictionaries/CassandraHelpers.cpp @@ -16,8 +16,10 @@ extern const int CASSANDRA_INTERNAL_ERROR; void cassandraCheck(CassError code) { if (code != CASS_OK) - throw Exception("Cassandra driver error " + std::to_string(code) + ": " + cass_error_desc(code), - ErrorCodes::CASSANDRA_INTERNAL_ERROR); + throw Exception(ErrorCodes::CASSANDRA_INTERNAL_ERROR, + "Cassandra driver error {}: {}", + std::to_string(code), + cass_error_desc(code)); } @@ -31,8 +33,12 @@ void cassandraWaitAndCheck(CassFuturePtr & future) const char * message; size_t message_len; cass_future_error_message(future, &message, & message_len); - std::string full_message = "Cassandra driver error " + std::to_string(code) + ": " + cass_error_desc(code) + ": " + message; - throw Exception(full_message, ErrorCodes::CASSANDRA_INTERNAL_ERROR); + + throw Exception(ErrorCodes::CASSANDRA_INTERNAL_ERROR, + "Cassandra driver error {}: {}: {}", + std::to_string(code), + cass_error_desc(code), + message); } static std::once_flag setup_logging_flag; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index d89271932eb..fdb0d76a8d7 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -27,9 +27,9 @@ namespace { constexpr size_t MAX_CONNECTIONS = 16; - inline UInt16 getPortFromContext(const Context & context, bool secure) + inline UInt16 getPortFromContext(ContextPtr context, bool secure) { - return secure ? context.getTCPPortSecure().value_or(0) : context.getTCPPort(); + return secure ? context->getTCPPortSecure().value_or(0) : context->getTCPPort(); } ConnectionPoolWithFailoverPtr createPool(const ClickHouseDictionarySource::Configuration & configuration) @@ -60,19 +60,19 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, const Block & sample_block_, - const Context & context_) + ContextPtr context_) : update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , configuration{configuration_} , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.where, IdentifierQuotingStyle::Backticks} , sample_block{sample_block_} - , context{context_} + , context(Context::createCopy(context_)) , pool{createPool(configuration)} , load_all_query{query_builder.composeLoadAllQuery()} { /// Query context is needed because some code in executeQuery function may assume it exists. /// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock. - context.makeQueryContext(); + context->makeQueryContext(); } ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionarySource & other) @@ -82,11 +82,11 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionar , invalidate_query_response{other.invalidate_query_response} , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.where, IdentifierQuotingStyle::Backticks} , sample_block{other.sample_block} - , context{other.context} + , context(Context::createCopy(other.context)) , pool{createPool(configuration)} , load_all_query{other.load_all_query} { - context.makeQueryContext(); + context->makeQueryContext(); } std::string ClickHouseDictionarySource::getUpdateFieldAndDate() @@ -172,7 +172,7 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re LOG_TRACE(log, "Performing invalidate query"); if (configuration.is_local) { - Context query_context = context; + auto query_context = Context::createCopy(context); auto input_block = executeQuery(request, query_context, true).getInputStream(); return readInvalidateQuery(*input_block); } @@ -191,12 +191,12 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - const Context & context, + ContextPtr context, const std::string & default_database [[maybe_unused]], bool /* check_config */) -> DictionarySourcePtr { bool secure = config.getBool(config_prefix + ".secure", false); - Context context_copy = context; + auto context_copy = Context::createCopy(context); UInt16 default_port = getPortFromContext(context_copy, secure); @@ -221,7 +221,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). if (configuration.is_local) { - context_copy.setUser(configuration.user, configuration.password, Poco::Net::SocketAddress("127.0.0.1", 0)); + context_copy->setUser(configuration.user, configuration.password, Poco::Net::SocketAddress("127.0.0.1", 0)); context_copy = copyContextAndApplySettings(config_prefix, context_copy, config); } diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index ac615848b49..21c290ab23b 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -37,7 +37,7 @@ public: const DictionaryStructure & dict_struct_, const Configuration & configuration_, const Block & sample_block_, - const Context & context); + ContextPtr context); /// copy-constructor is provided in order to support cloneability ClickHouseDictionarySource(const ClickHouseDictionarySource & other); @@ -77,7 +77,7 @@ private: mutable std::string invalidate_query_response; ExternalQueryBuilder query_builder; Block sample_block; - Context context; + ContextPtr context; ConnectionPoolWithFailoverPtr pool; const std::string load_all_query; Poco::Logger * log = &Poco::Logger::get("ClickHouseDictionarySource"); diff --git a/src/Dictionaries/DictionaryFactory.cpp b/src/Dictionaries/DictionaryFactory.cpp index 0ab7d199186..413bbd4f967 100644 --- a/src/Dictionaries/DictionaryFactory.cpp +++ b/src/Dictionaries/DictionaryFactory.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex) { if (!registered_layouts.emplace(layout_type, std::move(create_layout)).second) - throw Exception("DictionaryFactory: the layout name '" + layout_type + "' is not unique", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "DictionaryFactory: the layout name '{}' is not unique", layout_type); layout_complexity[layout_type] = is_complex; @@ -31,15 +31,16 @@ DictionaryPtr DictionaryFactory::create( const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - const Context & context, + ContextPtr context, bool check_source_config) const { Poco::Util::AbstractConfiguration::Keys keys; const auto & layout_prefix = config_prefix + ".layout"; config.keys(layout_prefix, keys); if (keys.size() != 1) - throw Exception{name + ": element dictionary.layout should have exactly one child element", - ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG}; + throw Exception(ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, + "{}: element dictionary.layout should have exactly one child element", + name); const DictionaryStructure dict_struct{config, config_prefix}; @@ -47,8 +48,8 @@ DictionaryPtr DictionaryFactory::create( name, config, config_prefix + ".source", dict_struct, context, config.getString(config_prefix + ".database", ""), check_source_config); LOG_TRACE(&Poco::Logger::get("DictionaryFactory"), "Created dictionary source '{}' for dictionary '{}'", source_ptr->toString(), name); - if (context.hasQueryContext() && context.getSettingsRef().log_queries) - context.getQueryContext().addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, name); + if (context->hasQueryContext() && context->getSettingsRef().log_queries) + context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, name); const auto & layout_type = keys.front(); @@ -61,10 +62,13 @@ DictionaryPtr DictionaryFactory::create( } } - throw Exception{name + ": unknown dictionary layout type: " + layout_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG}; + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, + "{}: unknown dictionary layout type: {}", + name, + layout_type); } -DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, const Context & context) const +DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, ContextPtr context) const { auto configuration = getDictionaryConfigurationFromAST(ast, context); return DictionaryFactory::create(name, *configuration, "dictionary", context, true); @@ -77,7 +81,9 @@ bool DictionaryFactory::isComplex(const std::string & layout_type) const if (found != layout_complexity.end()) return found->second; - throw Exception{"Unknown dictionary layout type: " + layout_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG}; + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, + "Unknown dictionary layout type: {}", + layout_type); } diff --git a/src/Dictionaries/DictionaryFactory.h b/src/Dictionaries/DictionaryFactory.h index 3d3f793e2b2..b869550a9af 100644 --- a/src/Dictionaries/DictionaryFactory.h +++ b/src/Dictionaries/DictionaryFactory.h @@ -1,5 +1,6 @@ #pragma once +#include #include "IDictionary.h" #include "registerDictionaries.h" #include @@ -21,8 +22,6 @@ class Logger; namespace DB { -class Context; - /** Create dictionary according to its layout. */ class DictionaryFactory : private boost::noncopyable @@ -37,13 +36,13 @@ public: const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - const Context & context, + ContextPtr context, bool check_source_config = false) const; /// Create dictionary from DDL-query DictionaryPtr create(const std::string & name, const ASTCreateQuery & ast, - const Context & context) const; + ContextPtr context) const; using Creator = std::function +#include #include #include #include #include +#include #include +#include #include #include @@ -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); @@ -236,7 +251,7 @@ public: else if constexpr (IsNumber) return ColumnType::create(size); else - throw Exception{"Unsupported attribute type.", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Unsupported attribute type."); } }; @@ -273,7 +288,7 @@ public: use_default_value_from_column = false; } else - throw Exception{"Type of default column is not the same as dictionary attribute type.", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Type of default column is not the same as dictionary attribute type."); } } @@ -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 +void mergeBlockWithStream( + size_t key_column_size [[maybe_unused]], + Block & block_to_update [[maybe_unused]], + BlockInputStreamPtr & stream [[maybe_unused]]) +{ + using KeyType = std::conditional_t; + 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 arena_holder; + DictionaryKeysExtractor 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 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 update_keys_extractor(block_key_columns, arena_holder.getComplexKeyArena()); + PaddedPODArray 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. @@ -434,10 +548,10 @@ static const PaddedPODArray & getColumnVectorData( if (!vector_col) { - throw Exception{ErrorCodes::TYPE_MISMATCH, + throw Exception(ErrorCodes::TYPE_MISMATCH, "{}: type mismatch: column has wrong type expected {}", dictionary->getDictionaryID().getNameForLogs(), - TypeName::get()}; + TypeName::get()); } if (is_const_column) diff --git a/src/Dictionaries/DictionarySourceFactory.cpp b/src/Dictionaries/DictionarySourceFactory.cpp index af3552364ba..50ba6405074 100644 --- a/src/Dictionaries/DictionarySourceFactory.cpp +++ b/src/Dictionaries/DictionarySourceFactory.cpp @@ -71,7 +71,7 @@ DictionarySourceFactory::DictionarySourceFactory() : log(&Poco::Logger::get("Dic void DictionarySourceFactory::registerSource(const std::string & source_type, Creator create_source) { if (!registered_sources.emplace(source_type, std::move(create_source)).second) - throw Exception("DictionarySourceFactory: the source name '" + source_type + "' is not unique", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "DictionarySourceFactory: the source name '{}' is not unique", source_type); } DictionarySourcePtr DictionarySourceFactory::create( @@ -79,7 +79,7 @@ DictionarySourcePtr DictionarySourceFactory::create( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const DictionaryStructure & dict_struct, - const Context & context, + ContextPtr context, const std::string & default_database, bool check_config) const { @@ -87,8 +87,9 @@ DictionarySourcePtr DictionarySourceFactory::create( config.keys(config_prefix, keys); if (keys.empty() || keys.size() > 2) - throw Exception{name + ": element dictionary.source should have one or two child elements", - ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG}; + throw Exception(ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, + "{}: element dictionary.source should have one or two child elements", + name); const std::string & source_type = keys.front() == "settings" ? keys.back() : keys.front(); @@ -100,7 +101,10 @@ DictionarySourcePtr DictionarySourceFactory::create( return create_source(dict_struct, config, config_prefix, sample_block, context, default_database, check_config); } - throw Exception{name + ": unknown dictionary source type: " + source_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG}; + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, + "{}: unknown dictionary source type: {}", + name, + source_type); } DictionarySourceFactory & DictionarySourceFactory::instance() diff --git a/src/Dictionaries/DictionarySourceFactory.h b/src/Dictionaries/DictionarySourceFactory.h index 1406660dfb4..bb583927ac4 100644 --- a/src/Dictionaries/DictionarySourceFactory.h +++ b/src/Dictionaries/DictionarySourceFactory.h @@ -2,6 +2,7 @@ #include "IDictionarySource.h" #include +#include #include @@ -17,7 +18,7 @@ class Logger; namespace DB { -class Context; + struct DictionaryStructure; /// creates IDictionarySource instance from config and DictionaryStructure @@ -34,7 +35,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - const Context & context, + ContextPtr context, const std::string & default_database, bool check_config)>; @@ -47,7 +48,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const DictionaryStructure & dict_struct, - const Context & context, + ContextPtr context, const std::string & default_database, bool check_config) const; diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 64527a78e8e..e175c6c3eae 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -67,12 +67,12 @@ Block blockForKeys( return block; } -Context copyContextAndApplySettings( +ContextPtr copyContextAndApplySettings( const std::string & config_prefix, - const Context & context, + ContextPtr context, const Poco::Util::AbstractConfiguration & config) { - Context local_context(context); + auto local_context = Context::createCopy(context); if (config.has(config_prefix + ".settings")) { const auto prefix = config_prefix + ".settings"; @@ -88,7 +88,7 @@ Context copyContextAndApplySettings( changes.emplace_back(key, value); } - local_context.applySettingsChanges(changes); + local_context->applySettingsChanges(changes); } return local_context; } @@ -125,9 +125,8 @@ Block BlockInputStreamWithAdditionalColumns::readImpl() auto cut_block = block_to_add.cloneWithCutColumns(current_range_index, block_rows); if (cut_block.rows() != block_rows) - throw Exception( - "Number of rows in block to add after cut must equal to number of rows in block from inner stream", - ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, + "Number of rows in block to add after cut must equal to number of rows in block from inner stream"); for (Int64 i = static_cast(cut_block.columns() - 1); i >= 0; --i) block.insert(0, cut_block.getByPosition(i)); diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index b429136a981..1febf921e07 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB { @@ -17,7 +18,6 @@ class IBlockOutputStream; using BlockOutputStreamPtr = std::shared_ptr; struct DictionaryStructure; -class Context; /// Write keys to block output stream. @@ -37,14 +37,14 @@ Block blockForKeys( const std::vector & requested_rows); /// Used for applying settings to copied context in some register[...]Source functions -Context copyContextAndApplySettings( +ContextPtr copyContextAndApplySettings( const std::string & config_prefix, - const Context & context, + ContextPtr context, const Poco::Util::AbstractConfiguration & config); void applySettingsToContext( const std::string & config_prefix, - Context & context, + ContextPtr context, const Poco::Util::AbstractConfiguration & config); /** A stream, adds additional columns to each block that it will read from inner stream. diff --git a/src/Dictionaries/DictionaryStructure.cpp b/src/Dictionaries/DictionaryStructure.cpp index 806ee0b80e0..dd53e31041b 100644 --- a/src/Dictionaries/DictionaryStructure.cpp +++ b/src/Dictionaries/DictionaryStructure.cpp @@ -33,7 +33,7 @@ namespace const auto expression = config.getString(config_prefix + ".expression", ""); if (name.empty() && !expression.empty()) - throw Exception{"Element " + config_prefix + ".name is empty", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Element {}.name is empty"); const auto type_name = config.getString(config_prefix + ".type", default_type); return DictionaryTypedSpecialAttribute{std::move(name), std::move(expression), DataTypeFactory::instance().get(type_name)}; @@ -65,6 +65,7 @@ AttributeUnderlyingType getAttributeUnderlyingType(const DataTypePtr & type) case TypeIndex::Decimal32: return AttributeUnderlyingType::utDecimal32; case TypeIndex::Decimal64: return AttributeUnderlyingType::utDecimal64; case TypeIndex::Decimal128: return AttributeUnderlyingType::utDecimal128; + case TypeIndex::Decimal256: return AttributeUnderlyingType::utDecimal256; case TypeIndex::Date: return AttributeUnderlyingType::utUInt16; case TypeIndex::DateTime: return AttributeUnderlyingType::utUInt32; @@ -81,11 +82,11 @@ AttributeUnderlyingType getAttributeUnderlyingType(const DataTypePtr & type) default: break; } - throw Exception{"Unknown type for dictionary" + type->getName(), ErrorCodes::UNKNOWN_TYPE}; + throw Exception(ErrorCodes::UNKNOWN_TYPE, "Unknown type {} for dictionary attribute", type->getName()); } -std::string toString(const AttributeUnderlyingType type) +std::string toString(AttributeUnderlyingType type) { switch (type) { @@ -117,11 +118,13 @@ std::string toString(const AttributeUnderlyingType type) return "Decimal64"; case AttributeUnderlyingType::utDecimal128: return "Decimal128"; + case AttributeUnderlyingType::utDecimal256: + return "Decimal256"; case AttributeUnderlyingType::utString: return "String"; } - throw Exception{"Unknown attribute_type " + toString(static_cast(type)), ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Unknown dictionary attribute type {}", toString(static_cast(type))); } @@ -129,7 +132,7 @@ DictionarySpecialAttribute::DictionarySpecialAttribute(const Poco::Util::Abstrac : name{config.getString(config_prefix + ".name", "")}, expression{config.getString(config_prefix + ".expression", "")} { if (name.empty() && !expression.empty()) - throw Exception{"Element " + config_prefix + ".name is empty", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Element {}.name is empty", config_prefix); } @@ -141,7 +144,7 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration const auto has_key = config.has(structure_prefix + ".key"); if (has_key && has_id) - throw Exception{"Only one of 'id' and 'key' should be specified", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only one of 'id' and 'key' should be specified"); if (has_id) id.emplace(config, structure_prefix + ".id"); @@ -149,15 +152,15 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration { key.emplace(getAttributes(config, structure_prefix + ".key", true)); if (key->empty()) - throw Exception{"Empty 'key' supplied", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty 'key' supplied"); } else - throw Exception{"Dictionary structure should specify either 'id' or 'key'", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary structure should specify either 'id' or 'key'"); if (id) { if (id->name.empty()) - throw Exception{"'id' cannot be empty", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'id' cannot be empty"); const char * range_default_type = "Date"; if (config.has(structure_prefix + ".range_min")) @@ -168,28 +171,27 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration if (range_min.has_value() != range_max.has_value()) { - throw Exception{"Dictionary structure should have both 'range_min' and 'range_max' either specified or not.", - ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Dictionary structure should have both 'range_min' and 'range_max' either specified or not."); } if (range_min && range_max && !range_min->type->equals(*range_max->type)) { - throw Exception{"Dictionary structure 'range_min' and 'range_max' should have same type, " - "'range_min' type: " - + range_min->type->getName() - + ", " - "'range_max' type: " - + range_max->type->getName(), - ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Dictionary structure 'range_min' and 'range_max' should have same type, " + "'range_min' type: {}," + "'range_max' type: {}", + range_min->type->getName(), + range_max->type->getName()); } if (range_min) { if (!range_min->type->isValueRepresentedByInteger()) - throw Exception{"Dictionary structure type of 'range_min' and 'range_max' should be an integer, Date, DateTime, or Enum." - " Actual 'range_min' and 'range_max' type is " - + range_min->type->getName(), - ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Dictionary structure type of 'range_min' and 'range_max' should be an integer, Date, DateTime, or Enum." + " Actual 'range_min' and 'range_max' type is {}", + range_min->type->getName()); } if (!id->expression.empty() || (range_min && !range_min->expression.empty()) || (range_max && !range_max->expression.empty())) @@ -208,8 +210,9 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration { if (id && attribute.underlying_type != AttributeUnderlyingType::utUInt64) throw Exception(ErrorCodes::TYPE_MISMATCH, - "Hierarchical attribute type for dictionary with simple key must be UInt64. Actual ({})", + "Hierarchical attribute type for dictionary with simple key must be UInt64. Actual {}", toString(attribute.underlying_type)); + else if (key) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary with complex key does not support hierarchy"); @@ -218,7 +221,7 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration } if (attributes.empty()) - throw Exception{"Dictionary has no attributes defined", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary has no attributes defined"); if (config.getBool(config_prefix + ".layout.ip_trie.access_to_key_from_attributes", false)) access_to_key_from_attributes = true; @@ -228,7 +231,7 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration void DictionaryStructure::validateKeyTypes(const DataTypes & key_types) const { if (key_types.size() != key->size()) - throw Exception{"Key structure does not match, expected " + getKeyDescription(), ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Key structure does not match, expected {}", getKeyDescription()); for (const auto i : ext::range(0, key_types.size())) { @@ -236,9 +239,11 @@ void DictionaryStructure::validateKeyTypes(const DataTypes & key_types) const const auto & actual_type = key_types[i]; if (!areTypesEqual(expected_type, actual_type)) - throw Exception{"Key type at position " + std::to_string(i) + " does not match, expected " + expected_type->getName() + ", found " - + actual_type->getName(), - ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Key type at position {} does not match, expected {}, found {}", + std::to_string(i), + expected_type->getName(), + actual_type->getName()); } } @@ -249,13 +254,13 @@ const DictionaryAttribute & DictionaryStructure::getAttribute(const std::string if (it == attribute_name_to_index.end()) { if (!access_to_key_from_attributes) - throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such attribute '{}'", attribute_name); for (const auto & key_attribute : *key) if (key_attribute.name == attribute_name) return key_attribute; - throw Exception{"No such attribute '" + attribute_name + "' in keys", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such attribute '{}' in keys", attribute_name); } size_t attribute_index = it->second; @@ -267,8 +272,10 @@ const DictionaryAttribute & DictionaryStructure::getAttribute(const std::string const auto & attribute = getAttribute(attribute_name); if (!areTypesEqual(attribute.type, type)) - throw Exception{"Attribute type does not match, expected " + attribute.type->getName() + ", found " + type->getName(), - ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Attribute type does not match, expected {}, found {}", + attribute.type->getName(), + type->getName()); return attribute; } @@ -343,11 +350,10 @@ static void checkAttributeKeys(const Poco::Util::AbstractConfiguration::Keys & k for (const auto & key : keys) { if (valid_keys.find(key) == valid_keys.end()) - throw Exception{"Unknown key '" + key + "' inside attribute section", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown key '{}' inside attribute section", key); } } - std::vector DictionaryStructure::getAttributes( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, @@ -388,8 +394,7 @@ std::vector DictionaryStructure::getAttributes( bool inserted = insert_result.second; if (!inserted) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary attributes names must be unique. Attribute name ({}) is not unique", name); @@ -442,13 +447,13 @@ std::vector DictionaryStructure::getAttributes( const auto injective = config.getBool(prefix + "injective", false); const auto is_object_id = config.getBool(prefix + "is_object_id", false); if (name.empty()) - throw Exception{"Properties 'name' and 'type' of an attribute cannot be empty", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Properties 'name' and 'type' of an attribute cannot be empty"); if (has_hierarchy && !hierarchy_allowed) - throw Exception{"Hierarchy not allowed in '" + prefix, ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Hierarchy not allowed in '{}'", prefix); if (has_hierarchy && hierarchical) - throw Exception{"Only one hierarchical attribute supported", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only one hierarchical attribute supported"); has_hierarchy = has_hierarchy || hierarchical; diff --git a/src/Dictionaries/DictionaryStructure.h b/src/Dictionaries/DictionaryStructure.h index 4f03b4ff09e..ce5dd3dd422 100644 --- a/src/Dictionaries/DictionaryStructure.h +++ b/src/Dictionaries/DictionaryStructure.h @@ -32,13 +32,14 @@ enum class AttributeUnderlyingType utDecimal32, utDecimal64, utDecimal128, + utDecimal256, utString }; AttributeUnderlyingType getAttributeUnderlyingType(const std::string & type); -std::string toString(const AttributeUnderlyingType type); +std::string toString(AttributeUnderlyingType type); /// Min and max lifetimes for a dictionary or it's entry using DictionaryLifetime = ExternalLoadableLifetime; @@ -125,6 +126,9 @@ void callOnDictionaryAttributeType(AttributeUnderlyingType type, F&& func) case AttributeUnderlyingType::utDecimal128: func(DictionaryAttributeType()); break; + case AttributeUnderlyingType::utDecimal256: + func(DictionaryAttributeType()); + break; } }; diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 96ef259106a..0a3e685df1a 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -27,16 +27,16 @@ DirectDictionary::DirectDictionary( , source_ptr{std::move(source_ptr_)} { if (!source_ptr->supportsSelectiveLoad()) - throw Exception{full_name + ": source cannot be used with DirectDictionary", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: source cannot be used with DirectDictionary", full_name); } template -ColumnPtr DirectDictionary::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::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::getColumn( DictionaryKeysExtractor 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 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::getColumn( DictionaryKeysExtractor 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::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 -ColumnUInt8::Ptr DirectDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types [[maybe_unused]]) const +ColumnPtr DirectDictionary::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 +ColumnUInt8::Ptr DirectDictionary::hasKeys( + const Columns & key_columns, + const DataTypes & key_types [[maybe_unused]]) const { if constexpr (dictionary_key_type == DictionaryKeyType::complex) dict_struct.validateKeyTypes(key_types); @@ -261,20 +290,20 @@ namespace { if (dict_struct.key) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "'key' is not supported for dictionary of layout '({})'", + "'key' is not supported for dictionary of layout '{}'", layout_name); } else { if (dict_struct.id) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "'id' is not supported for dictionary of layout '({})'", + "'id' is not supported for dictionary of layout '{}'", layout_name); } if (dict_struct.range_min || dict_struct.range_max) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): elements .structure.range_min and .structure.range_max should be defined only " \ + "{}: elements .structure.range_min and .structure.range_max should be defined only " "for a dictionary of layout 'range_hashed'", full_name); @@ -282,7 +311,7 @@ namespace if (config.has(config_prefix + ".lifetime.min") || config.has(config_prefix + ".lifetime.max")) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "'lifetime' parameter is redundant for the dictionary' of layout '({})'", + "'lifetime' parameter is redundant for the dictionary' of layout '{}'", layout_name); return std::make_unique>(dict_id, dict_struct, std::move(source_ptr)); diff --git a/src/Dictionaries/DirectDictionary.h b/src/Dictionaries/DirectDictionary.h index 6bca6ac6a18..e547e10433c 100644 --- a/src/Dictionaries/DirectDictionary.h +++ b/src/Dictionaries/DirectDictionary.h @@ -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, diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 8066de41b20..13feab2071a 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -62,7 +62,7 @@ ExecutableDictionarySource::ExecutableDictionarySource( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block_, - const Context & context_) + ContextPtr context_) : log(&Poco::Logger::get("ExecutableDictionarySource")) , dict_struct{dict_struct_} , implicit_key{config.getBool(config_prefix + ".implicit_key", false)} @@ -97,25 +97,25 @@ ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionar , update_field{other.update_field} , format{other.format} , sample_block{other.sample_block} - , context(other.context) + , context(Context::createCopy(other.context)) { } BlockInputStreamPtr ExecutableDictionarySource::loadAll() { if (implicit_key) - throw Exception("ExecutableDictionarySource with implicit_key does not support loadAll method", ErrorCodes::UNSUPPORTED_METHOD); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutableDictionarySource with implicit_key does not support loadAll method"); LOG_TRACE(log, "loadAll {}", toString()); auto process = ShellCommand::execute(command); - auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size); + auto input_stream = context->getInputFormat(format, process->out, sample_block, max_block_size); return std::make_shared(log, input_stream, std::move(process)); } BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll() { if (implicit_key) - throw Exception("ExecutableDictionarySource with implicit_key does not support loadUpdatedAll method", ErrorCodes::UNSUPPORTED_METHOD); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutableDictionarySource with implicit_key does not support loadUpdatedAll method"); time_t new_update_time = time(nullptr); SCOPE_EXIT(update_time = new_update_time); @@ -126,7 +126,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll() LOG_TRACE(log, "loadUpdatedAll {}", command_with_update_field); auto process = ShellCommand::execute(command_with_update_field); - auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size); + auto input_stream = context->getInputFormat(format, process->out, sample_block, max_block_size); return std::make_shared(log, input_stream, std::move(process)); } @@ -139,7 +139,7 @@ namespace { public: BlockInputStreamWithBackgroundThread( - const Context & context, + ContextPtr context, const std::string & format, const Block & sample_block, const std::string & command_str, @@ -150,7 +150,7 @@ namespace send_data(std::move(send_data_)), thread([this] { send_data(command->in); }) { - stream = context.getInputFormat(format, command->out, sample_block, max_block_size); + stream = context->getInputFormat(format, command->out, sample_block, max_block_size); } ~BlockInputStreamWithBackgroundThread() override @@ -222,7 +222,7 @@ BlockInputStreamPtr ExecutableDictionarySource::getStreamForBlock(const Block & context, format, sample_block, command, log, [block, this](WriteBufferFromFile & out) mutable { - auto output_stream = context.getOutputStream(format, out, block.cloneEmpty()); + auto output_stream = context->getOutputStream(format, out, block.cloneEmpty()); formatBlock(output_stream, block); out.close(); }); @@ -264,7 +264,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - const Context & context, + ContextPtr context, const std::string & /* default_database */, bool check_config) -> DictionarySourcePtr { @@ -277,7 +277,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) if (check_config) throw Exception(ErrorCodes::DICTIONARY_ACCESS_DENIED, "Dictionaries with executable dictionary source are not allowed to be created from DDL query"); - Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config); return std::make_unique( dict_struct, config, config_prefix + ".executable", diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index 9dc152bf580..878cb086873 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -20,7 +20,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block_, - const Context & context_); + ContextPtr context_); ExecutableDictionarySource(const ExecutableDictionarySource & other); ExecutableDictionarySource & operator=(const ExecutableDictionarySource &) = delete; @@ -58,7 +58,7 @@ private: const std::string update_field; const std::string format; Block sample_block; - Context context; + ContextPtr context; }; } diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 0c9ca4ce714..e920b8392d6 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -32,7 +32,7 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, Block & sample_block_, - const Context & context_) + ContextPtr context_) : log(&Poco::Logger::get("ExecutablePoolDictionarySource")) , dict_struct{dict_struct_} , configuration{configuration_} @@ -63,19 +63,19 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource(const ExecutableP , dict_struct{other.dict_struct} , configuration{other.configuration} , sample_block{other.sample_block} - , context{other.context} + , context{Context::createCopy(other.context)} , process_pool{std::make_shared(configuration.pool_size)} { } BlockInputStreamPtr ExecutablePoolDictionarySource::loadAll() { - throw Exception("ExecutablePoolDictionarySource with implicit_key does not support loadAll method", ErrorCodes::UNSUPPORTED_METHOD); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutablePoolDictionarySource with implicit_key does not support loadAll method"); } BlockInputStreamPtr ExecutablePoolDictionarySource::loadUpdatedAll() { - throw Exception("ExecutablePoolDictionarySource with implicit_key does not support loadAll method", ErrorCodes::UNSUPPORTED_METHOD); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutablePoolDictionarySource with implicit_key does not support loadAll method"); } namespace @@ -226,13 +226,13 @@ BlockInputStreamPtr ExecutablePoolDictionarySource::getStreamForBlock(const Bloc configuration.max_command_execution_time); size_t rows_to_read = block.rows(); - auto read_stream = context.getInputFormat(configuration.format, process->out, sample_block, rows_to_read); + auto read_stream = context->getInputFormat(configuration.format, process->out, sample_block, rows_to_read); auto stream = std::make_unique( process_pool, std::move(process), std::move(read_stream), rows_to_read, log, [block, this](WriteBufferFromFile & out) mutable { - auto output_stream = context.getOutputStream(configuration.format, out, block.cloneEmpty()); + auto output_stream = context->getOutputStream(configuration.format, out, block.cloneEmpty()); formatBlock(output_stream, block); }); @@ -273,7 +273,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - const Context & context, + ContextPtr context, const std::string & /* default_database */, bool check_config) -> DictionarySourcePtr { @@ -286,20 +286,20 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) if (check_config) throw Exception(ErrorCodes::DICTIONARY_ACCESS_DENIED, "Dictionaries with executable pool dictionary source are not allowed to be created from DDL query"); - Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config); /** Currently parallel parsing input format cannot read exactly max_block_size rows from input, * so it will be blocked on ReadBufferFromFileDescriptor because this file descriptor represent pipe that does not have eof. */ - auto settings_no_parallel_parsing = context_local_copy.getSettings(); + auto settings_no_parallel_parsing = context_local_copy->getSettings(); settings_no_parallel_parsing.input_format_parallel_parsing = false; - context_local_copy.setSettings(settings_no_parallel_parsing); + context_local_copy->setSettings(settings_no_parallel_parsing); String configuration_config_prefix = config_prefix + ".executable_pool"; size_t max_command_execution_time = config.getUInt64(configuration_config_prefix + ".max_command_execution_time", 10); - size_t max_execution_time_seconds = static_cast(context.getSettings().max_execution_time.totalSeconds()); + size_t max_execution_time_seconds = static_cast(context->getSettings().max_execution_time.totalSeconds()); if (max_execution_time_seconds != 0 && max_command_execution_time > max_execution_time_seconds) max_command_execution_time = max_execution_time_seconds; diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index da75881b799..7f24e56257a 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -41,7 +41,7 @@ public: const DictionaryStructure & dict_struct_, const Configuration & configuration_, Block & sample_block_, - const Context & context_); + ContextPtr context_); ExecutablePoolDictionarySource(const ExecutablePoolDictionarySource & other); ExecutablePoolDictionarySource & operator=(const ExecutablePoolDictionarySource &) = delete; @@ -77,7 +77,7 @@ private: const Configuration configuration; Block sample_block; - Context context; + ContextPtr context; std::shared_ptr process_pool; }; diff --git a/src/Dictionaries/ExternalQueryBuilder.cpp b/src/Dictionaries/ExternalQueryBuilder.cpp index 454c918fd1f..8c6e9f60afb 100644 --- a/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/src/Dictionaries/ExternalQueryBuilder.cpp @@ -173,7 +173,7 @@ std::string ExternalQueryBuilder::composeUpdateQuery(const std::string & update_ std::string ExternalQueryBuilder::composeLoadIdsQuery(const std::vector & ids) { if (!dict_struct.id) - throw Exception{"Simple key required for method", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Simple key required for method"); WriteBufferFromOwnString out; writeString("SELECT ", out); @@ -244,10 +244,10 @@ std::string ExternalQueryBuilder::composeLoadKeysQuery( const Columns & key_columns, const std::vector & requested_rows, LoadKeysMethod method, size_t partition_key_prefix) { if (!dict_struct.key) - throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Composite key required for method"); if (key_columns.size() != dict_struct.key->size()) - throw Exception{"The size of key_columns does not equal to the size of dictionary key", ErrorCodes::LOGICAL_ERROR}; + throw Exception(ErrorCodes::LOGICAL_ERROR, "The size of key_columns does not equal to the size of dictionary key"); WriteBufferFromOwnString out; writeString("SELECT ", out); @@ -386,7 +386,7 @@ void ExternalQueryBuilder::composeInWithTuples(const Columns & key_columns, cons void ExternalQueryBuilder::composeKeyTupleDefinition(WriteBuffer & out, size_t beg, size_t end) const { if (!dict_struct.key) - throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Composite key required for method"); writeChar('(', out); diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index addc2adef02..378c6f11857 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -1,8 +1,11 @@ #include "FileDictionarySource.h" + +#include +#include + #include #include #include -#include #include #include #include "DictionarySourceFactory.h" @@ -10,7 +13,6 @@ #include "registerDictionaries.h" #include "DictionarySourceHelpers.h" - namespace DB { static const UInt64 max_block_size = 8192; @@ -24,7 +26,7 @@ namespace ErrorCodes FileDictionarySource::FileDictionarySource( const std::string & filepath_, const std::string & format_, - Block & sample_block_, const Context & context_, bool check_config) + Block & sample_block_, ContextPtr context_, bool check_config) : filepath{filepath_} , format{format_} , sample_block{sample_block_} @@ -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); } } @@ -43,7 +55,7 @@ FileDictionarySource::FileDictionarySource(const FileDictionarySource & other) : filepath{other.filepath} , format{other.format} , sample_block{other.sample_block} - , context(other.context) + , context(Context::createCopy(other.context)) , last_modification{other.last_modification} { } @@ -53,7 +65,7 @@ BlockInputStreamPtr FileDictionarySource::loadAll() { LOG_TRACE(&Poco::Logger::get("FileDictionary"), "loadAll {}", toString()); auto in_ptr = std::make_unique(filepath); - auto stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size); + auto stream = context->getInputFormat(format, *in_ptr, sample_block, max_block_size); last_modification = getLastModification(); return std::make_shared>(stream, std::move(in_ptr)); @@ -77,17 +89,17 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - const Context & context, + ContextPtr context, const std::string & /* default_database */, bool check_config) -> DictionarySourcePtr { if (dict_struct.has_expressions) - throw Exception{"Dictionary source of type `file` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR}; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `file` does not support attribute expressions"); const auto filepath = config.getString(config_prefix + ".file.path"); const auto format = config.getString(config_prefix + ".file.format"); - Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config); return std::make_unique(filepath, format, sample_block, context_local_copy, check_config); }; diff --git a/src/Dictionaries/FileDictionarySource.h b/src/Dictionaries/FileDictionarySource.h index fa47b280911..6559503cccd 100644 --- a/src/Dictionaries/FileDictionarySource.h +++ b/src/Dictionaries/FileDictionarySource.h @@ -17,7 +17,7 @@ class FileDictionarySource final : public IDictionarySource { public: FileDictionarySource(const std::string & filepath_, const std::string & format_, - Block & sample_block_, const Context & context_, bool check_config); + Block & sample_block_, ContextPtr context_, bool check_config); FileDictionarySource(const FileDictionarySource & other); @@ -25,17 +25,17 @@ public: BlockInputStreamPtr loadUpdatedAll() override { - throw Exception{"Method loadUpdatedAll is unsupported for FileDictionarySource", ErrorCodes::NOT_IMPLEMENTED}; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for FileDictionarySource"); } BlockInputStreamPtr loadIds(const std::vector & /*ids*/) override { - throw Exception{"Method loadIds is unsupported for FileDictionarySource", ErrorCodes::NOT_IMPLEMENTED}; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadIds is unsupported for FileDictionarySource"); } BlockInputStreamPtr loadKeys(const Columns & /*key_columns*/, const std::vector & /*requested_rows*/) override { - throw Exception{"Method loadKeys is unsupported for FileDictionarySource", ErrorCodes::NOT_IMPLEMENTED}; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadKeys is unsupported for FileDictionarySource"); } bool isModified() const override @@ -61,7 +61,7 @@ private: const std::string filepath; const std::string format; Block sample_block; - const Context context; + ContextPtr context; Poco::Timestamp last_modification; }; diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 2d8d208d76b..1540a3a876b 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -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 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(hierarchical_attribute.null_values); - const ContainerType & parent_keys = std::get>(hierarchical_attribute.arrays); + const ContainerType & parent_keys = std::get>(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 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 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(hierarchical_attribute.null_values); - const ContainerType & parent_keys = std::get>(hierarchical_attribute.arrays); + const ContainerType & parent_keys = std::get>(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 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 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 & parent_keys = std::get>(hierarchical_attribute.arrays); + const ContainerType & parent_keys = std::get>(hierarchical_attribute.container); HashMap> 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(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 arena_holder; + DictionaryKeysExtractor keys_extractor({ keys_column }, arena_holder.getComplexKeyArena()); + auto keys = keys_extractor.extractAllKeys(); + + HashSet 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(), 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(block.cloneEmpty()); - for (const auto attribute_idx : ext::range(0, attributes.size() + 1)) + if (!previously_loaded_block) + previously_loaded_block = std::make_shared(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> 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>::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( + 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,26 +321,8 @@ void FlatDictionary::loadData() else updateData(); - if (require_nonempty && 0 == element_count) - throw Exception{full_name + ": dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY}; -} - - -template -void FlatDictionary::addAttributeSize(const Attribute & attribute) -{ - const auto & array_ref = std::get>(attribute.arrays); - bytes_allocated += sizeof(PaddedPODArray) + array_ref.allocated_bytes(); - bucket_count = array_ref.capacity(); -} - -template <> -void FlatDictionary::addAttributeSize(const Attribute & attribute) -{ - const auto & array_ref = std::get>(attribute.arrays); - bytes_allocated += sizeof(PaddedPODArray) + array_ref.allocated_bytes(); - bytes_allocated += sizeof(Arena) + attribute.string_arena->size(); - bucket_count = array_ref.capacity(); + if (configuration.require_nonempty && 0 == element_count) + throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, "{}: dictionary source is empty and 'require_nonempty' property is set.", full_name); } void FlatDictionary::calculateBytesAllocated() @@ -391,102 +335,108 @@ void FlatDictionary::calculateBytesAllocated() { using Type = std::decay_t; using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; - addAttributeSize(attribute); + const auto & container = std::get>(attribute.container); + bytes_allocated += sizeof(PaddedPODArray) + container.allocated_bytes(); + bucket_count = container.capacity(); + + if constexpr (std::is_same_v) + bytes_allocated += sizeof(Arena) + attribute.string_arena->size(); }; callOnDictionaryAttributeType(attribute.type, type_call); } } - -template -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()); - const auto & null_value_ref = std::get(attribute.null_values); - attribute.arrays.emplace>(initial_array_size, null_value_ref); -} + auto nullable_set = dictionary_attribute.is_nullable ? std::make_optional() : std::optional{}; + Attribute attribute{dictionary_attribute.underlying_type, std::move(nullable_set), {}, {}, {}}; -template <> -void FlatDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) -{ - attribute.string_arena = std::make_unique(); - const String & string = null_value.get(); - const char * string_in_arena = attribute.string_arena->insert(string.data(), string.size()); - attribute.null_values.emplace(string_in_arena, string.size()); - attribute.arrays.emplace>(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() : std::optional{}; - 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; using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; - createAttributeImpl(attr, null_value); + if constexpr (std::is_same_v) + { + attribute.string_arena = std::make_unique(); + const String & string = null_value.get(); + const char * string_in_arena = attribute.string_arena->insert(string.data(), string.size()); + attribute.null_values.emplace(string_in_arena, string.size()); + } + else + attribute.null_values = ValueType(null_value.get>()); + + const auto & null_value_ref = std::get(attribute.null_values); + attribute.container.emplace>(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 void FlatDictionary::getItemsImpl( const Attribute & attribute, - const PaddedPODArray & ids, + const PaddedPODArray & keys, ValueSetter && set_value, DefaultValueExtractor & default_value_extractor) const { - const auto & attr = std::get>(attribute.arrays); - const auto rows = ext::size(ids); + const auto & container = std::get>(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(attr[id]) : default_value_extractor[row]); + const auto key = keys[row]; + + if (key < loaded_keys.size() && loaded_keys[key]) + set_value(row, static_cast(container[key])); + else + set_value(row, default_value_extractor[row]); } query_count.fetch_add(rows, std::memory_order_relaxed); } template -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>(attribute.arrays); - if (id >= array.size()) + auto & container = std::get>(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(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(attribute.null_values)); } } template -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>(attribute.arrays); - array[id] = value; - loaded_ids[id] = true; + auto & array = std::get>(attribute.container); + array[key] = value; + loaded_keys[key] = true; } template <> -void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const UInt64 id, const String & value) +void FlatDictionary::setAttributeValueImpl(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, StringRef, AttributeType>; - resize(attribute, id); + resize(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(attribute, id, value.get()); + setAttributeValueImpl(attribute, key, value.get()); }; callOnDictionaryAttributeType(attribute.type, type_call); } -PaddedPODArray FlatDictionary::getIds() const -{ - const auto ids_count = ext::size(loaded_ids); - - PaddedPODArray 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(shared_from_this(), max_block_size, getIds(), column_names); + const auto keys_count = loaded_keys.size(); + + PaddedPODArray 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(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(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, std::move(configuration)); }; + factory.registerLayout("flat", create_layout, false); } diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index 09721bf1a99..0a5f88f270a 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -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 clone() const override { - return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, configuration, previously_loaded_block); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -113,6 +120,7 @@ private: Decimal32, Decimal64, Decimal128, + Decimal256, Float32, Float64, StringRef> @@ -130,10 +138,11 @@ private: ContainerType, ContainerType, ContainerType, + ContainerType, ContainerType, ContainerType, ContainerType> - arrays; + container; std::unique_ptr string_arena; }; @@ -143,54 +152,39 @@ private: void updateData(); void loadData(); - template - void addAttributeSize(const Attribute & attribute); - void calculateBytesAllocated(); - template - 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 void getItemsImpl( const Attribute & attribute, - const PaddedPODArray & ids, + const PaddedPODArray & keys, ValueSetter && set_value, DefaultValueExtractor & default_value_extractor) const; template - void resize(Attribute & attribute, const UInt64 id); + void resize(Attribute & attribute, UInt64 key); template - 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 - void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; - - PaddedPODArray 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 attribute_index_by_name; std::vector attributes; - std::vector loaded_ids; + std::vector loaded_keys; size_t bytes_allocated = 0; size_t element_count = 0; size_t bucket_count = 0; mutable std::atomic query_count{0}; - /// TODO: Remove - BlockPtr saved_block; + BlockPtr previously_loaded_block; }; } diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 62bf478afc4..b674d593444 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -30,7 +30,7 @@ HTTPDictionarySource::HTTPDictionarySource( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block_, - const Context & context_, + ContextPtr context_, bool check_config) : log(&Poco::Logger::get("HTTPDictionarySource")) , update_time{std::chrono::system_clock::from_time_t(0)} @@ -44,7 +44,7 @@ HTTPDictionarySource::HTTPDictionarySource( { if (check_config) - context.getRemoteHostFilter().checkURL(Poco::URI(url)); + context->getRemoteHostFilter().checkURL(Poco::URI(url)); const auto & credentials_prefix = config_prefix + ".credentials"; @@ -80,7 +80,7 @@ HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) , update_field{other.update_field} , format{other.format} , sample_block{other.sample_block} - , context(other.context) + , context(Context::createCopy(other.context)) , timeouts(ConnectionTimeouts::getHTTPTimeouts(context)) { credentials.setUsername(other.credentials.getUsername()); @@ -111,7 +111,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadAll() auto in_ptr = std::make_unique( uri, Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts, 0, credentials, DBMS_DEFAULT_BUFFER_SIZE, header_entries); - auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size); + auto input_stream = context->getInputFormat(format, *in_ptr, sample_block, max_block_size); return std::make_shared>(input_stream, std::move(in_ptr)); } @@ -123,7 +123,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll() auto in_ptr = std::make_unique( uri, Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts, 0, credentials, DBMS_DEFAULT_BUFFER_SIZE, header_entries); - auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size); + auto input_stream = context->getInputFormat(format, *in_ptr, sample_block, max_block_size); return std::make_shared>(input_stream, std::move(in_ptr)); } @@ -136,7 +136,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector & id ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [block, this](std::ostream & ostr) { WriteBufferFromOStream out_buffer(ostr); - auto output_stream = context.getOutputStreamParallelIfPossible(format, out_buffer, sample_block); + auto output_stream = context->getOutputStreamParallelIfPossible(format, out_buffer, sample_block); formatBlock(output_stream, block); }; @@ -144,7 +144,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector & id auto in_ptr = std::make_unique( uri, Poco::Net::HTTPRequest::HTTP_POST, out_stream_callback, timeouts, 0, credentials, DBMS_DEFAULT_BUFFER_SIZE, header_entries); - auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size); + auto input_stream = context->getInputFormat(format, *in_ptr, sample_block, max_block_size); return std::make_shared>(input_stream, std::move(in_ptr)); } @@ -157,7 +157,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadKeys(const Columns & key_columns, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [block, this](std::ostream & ostr) { WriteBufferFromOStream out_buffer(ostr); - auto output_stream = context.getOutputStreamParallelIfPossible(format, out_buffer, sample_block); + auto output_stream = context->getOutputStreamParallelIfPossible(format, out_buffer, sample_block); formatBlock(output_stream, block); }; @@ -165,7 +165,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadKeys(const Columns & key_columns, auto in_ptr = std::make_unique( uri, Poco::Net::HTTPRequest::HTTP_POST, out_stream_callback, timeouts, 0, credentials, DBMS_DEFAULT_BUFFER_SIZE, header_entries); - auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size); + auto input_stream = context->getInputFormat(format, *in_ptr, sample_block, max_block_size); return std::make_shared>(input_stream, std::move(in_ptr)); } @@ -201,14 +201,14 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - const Context & context, + ContextPtr context, const std::string & /* default_database */, bool check_config) -> DictionarySourcePtr { if (dict_struct.has_expressions) - throw Exception{"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR}; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `http` does not support attribute expressions"); - Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config); return std::make_unique( dict_struct, config, config_prefix + ".http", diff --git a/src/Dictionaries/HTTPDictionarySource.h b/src/Dictionaries/HTTPDictionarySource.h index e7920132e83..c42c67ec8c9 100644 --- a/src/Dictionaries/HTTPDictionarySource.h +++ b/src/Dictionaries/HTTPDictionarySource.h @@ -26,7 +26,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block_, - const Context & context_, + ContextPtr context_, bool check_config); HTTPDictionarySource(const HTTPDictionarySource & other); @@ -65,7 +65,7 @@ private: std::string update_field; const std::string format; Block sample_block; - Context context; + ContextPtr context; ConnectionTimeouts timeouts; }; diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 3fbab4c3e39..2a403554a80 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -1,9 +1,5 @@ #include "HashedDictionary.h" -#include - -#include - #include #include #include @@ -46,13 +42,13 @@ HashedDictionary::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(); @@ -127,7 +123,7 @@ ColumnPtr HashedDictionary::getColumn( [&](const size_t row, const auto value) { return out[row] = value; }, [&](const size_t row) { - out[row] = 0; + out[row] = ValueType(); (*vec_null_map_to)[row] = true; }, default_value_extractor); @@ -347,7 +343,7 @@ void HashedDictionary::createAttributes() template void HashedDictionary::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::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(block.cloneEmpty()); + if (!previously_loaded_block) + previously_loaded_block = std::make_shared(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::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 arena_holder; - DictionaryKeysExtractor 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 block_keys_extractor(saved_block_key_columns, arena_holder.getComplexKeyArena()); - auto keys_extracted_from_block = block_keys_extractor.extractAllKeys(); - - absl::flat_hash_map, DefaultHash> 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( + 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()); } } @@ -604,7 +547,7 @@ void HashedDictionary::loadData() if (require_nonempty && 0 == element_count) throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, - "({}): dictionary source is empty and 'require_nonempty' property is set.", + "{}: dictionary source is empty and 'require_nonempty' property is set.", full_name); } @@ -624,7 +567,11 @@ void HashedDictionary::calculateBytesAllocated() if constexpr (sparse || std::is_same_v) { - bytes_allocated += container.max_size() * (sizeof(KeyType) + sizeof(AttributeValueType)); + /// bucket_count() - Returns table size, that includes empty and deleted + /// size() - Returns table size, w/o empty and deleted + /// and since this is sparsehash, empty cells should not be significant, + /// and since items cannot be removed from the dictionary, deleted is also not important. + bytes_allocated += container.size() * (sizeof(KeyType) + sizeof(AttributeValueType)); bucket_count = container.bucket_count(); } else @@ -729,10 +676,10 @@ void registerDictionaryHashed(DictionaryFactory & factory) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for complex key hashed dictionary"); 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); const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 3882b669324..0d0ab8dcb43 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -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 clone() const override { - return std::make_shared>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); + return std::make_shared>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, previously_loaded_block); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -152,6 +152,7 @@ private: Decimal32, Decimal64, Decimal128, + Decimal256, Float32, Float64, StringRef> @@ -170,13 +171,13 @@ private: CollectionType, CollectionType, CollectionType, + CollectionType, CollectionType, CollectionType, CollectionType> container; std::unique_ptr string_arena; - }; void createAttributes(); @@ -219,8 +220,7 @@ private: size_t bucket_count = 0; mutable std::atomic query_count{0}; - /// TODO: Remove - BlockPtr saved_block; + BlockPtr previously_loaded_block; Arena complex_key_arena; }; diff --git a/src/Dictionaries/HierarchyDictionariesUtils.cpp b/src/Dictionaries/HierarchyDictionariesUtils.cpp index fffe0d30e0e..5bca6a5ac1a 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.cpp +++ b/src/Dictionaries/HierarchyDictionariesUtils.cpp @@ -52,7 +52,7 @@ namespace const auto * parent_key_column_typed = checkAndGetColumn>(*parent_key_column); if (!parent_key_column_typed) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Parent key column should be UInt64. Actual ({})", + "Parent key column should be UInt64. Actual {}", hierarchical_attribute.type->getName()); const auto & parent_keys = parent_key_column_typed->getData(); diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index d66c285bc42..5c5f71e0b1d 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -124,20 +124,20 @@ static size_t formatIPWithPrefix(const unsigned char * src, UInt8 prefix_len, bo static void validateKeyTypes(const DataTypes & key_types) { if (key_types.empty() || key_types.size() > 2) - throw Exception{"Expected a single IP address or IP with mask", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a single IP address or IP with mask"); const auto * key_ipv4type = typeid_cast(key_types[0].get()); const auto * key_ipv6type = typeid_cast(key_types[0].get()); if (key_ipv4type == nullptr && (key_ipv6type == nullptr || key_ipv6type->getN() != 16)) - throw Exception{"Key does not match, expected either `IPv4` (`UInt32`) or `IPv6` (`FixedString(16)`)", - ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Key does not match, expected either `IPv4` (`UInt32`) or `IPv6` (`FixedString(16)`)"); if (key_types.size() > 1) { const auto * mask_col_type = typeid_cast(key_types[1].get()); if (mask_col_type == nullptr) - throw Exception{"Mask do not match, expected UInt8", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Mask do not match, expected UInt8"); } } @@ -293,7 +293,7 @@ ColumnUInt8::Ptr IPAddressDictionary::hasKeys(const Columns & key_columns, const { auto addr = first_column->getDataAt(i); if (unlikely(addr.size != IPV6_BINARY_LENGTH)) - throw Exception("Expected key to be FixedString(16)", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected key to be FixedString(16)"); auto found = tryLookupIPv6(reinterpret_cast(addr.data)); out[i] = (found != ipNotFound()); @@ -316,8 +316,10 @@ void IPAddressDictionary::createAttributes() attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value)); if (attribute.hierarchical) - throw Exception{full_name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(), - ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, + "{}: hierarchical attributes not supported for dictionary of type {}", + full_name, + getTypeName()); } }; @@ -492,7 +494,7 @@ void IPAddressDictionary::loadData() LOG_TRACE(logger, "{} ip records are read", ip_records.size()); if (require_nonempty && 0 == element_count) - throw Exception{full_name + ": dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY}; + throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, "{}: dictionary source is empty and 'require_nonempty' property is set.", full_name); } template @@ -591,7 +593,7 @@ void IPAddressDictionary::getItemsByTwoKeyColumnsImpl( { const auto * key_ip_column_ptr = typeid_cast *>(&*key_columns.front()); if (key_ip_column_ptr == nullptr) - throw Exception{"Expected a UInt32 IP column", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a UInt32 IP column"); const auto & key_mask_column = assert_cast &>(*key_columns.back()); @@ -625,7 +627,7 @@ void IPAddressDictionary::getItemsByTwoKeyColumnsImpl( const auto * key_ip_column_ptr = typeid_cast(&*key_columns.front()); if (key_ip_column_ptr == nullptr || key_ip_column_ptr->getN() != IPV6_BINARY_LENGTH) - throw Exception{"Expected a FixedString(16) IP column", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a FixedString(16) IP column"); const auto & key_mask_column = assert_cast &>(*key_columns.back()); @@ -698,7 +700,7 @@ void IPAddressDictionary::getItemsImpl( { auto addr = first_column->getDataAt(i); if (addr.size != IPV6_BINARY_LENGTH) - throw Exception("Expected key to be FixedString(16)", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected key to be FixedString(16)"); auto found = tryLookupIPv6(reinterpret_cast(addr.data)); if (found != ipNotFound()) @@ -744,7 +746,7 @@ const IPAddressDictionary::Attribute & IPAddressDictionary::getAttribute(const s { const auto it = attribute_index_by_name.find(attribute_name); if (it == std::end(attribute_index_by_name)) - throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}: no such attribute '{}'", full_name, attribute_name); return attributes[it->second]; } @@ -921,7 +923,7 @@ void registerDictionaryTrie(DictionaryFactory & factory) DictionarySourcePtr source_ptr) -> DictionaryPtr { if (!dict_struct.key || dict_struct.key->size() != 1) - throw Exception{"Dictionary of layout 'ip_trie' has to have one 'key'", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary of layout 'ip_trie' has to have one 'key'"); const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index cf79caa75fc..619d1579e4e 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -101,6 +101,7 @@ private: Decimal32, Decimal64, Decimal128, + Decimal256, Float32, Float64, String> @@ -118,6 +119,7 @@ private: ContainerType, ContainerType, ContainerType, + ContainerType, ContainerType, ContainerType, ContainerType> diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 0632dd3e30f..f09869b5a30 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -1,4 +1,5 @@ #include "LibraryDictionarySource.h" + #include #include #include @@ -6,307 +7,186 @@ #include #include #include -#include #include "DictionarySourceFactory.h" +#include "DictionarySourceHelpers.h" #include "DictionaryStructure.h" #include "LibraryDictionarySourceExternal.h" #include "registerDictionaries.h" +#include +#include + namespace DB { + namespace ErrorCodes { - extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int FILE_DOESNT_EXIST; extern const int EXTERNAL_LIBRARY_ERROR; extern const int PATH_ACCESS_DENIED; } -class CStringsHolder -{ -public: - using Container = std::vector; - explicit CStringsHolder(const Container & strings_pass) - { - strings_holder = strings_pass; - strings.size = strings_holder.size(); - ptr_holder = std::make_unique(strings.size); - strings.data = ptr_holder.get(); - size_t i = 0; - for (auto & str : strings_holder) - { - strings.data[i] = str.c_str(); - ++i; - } - } - - ClickHouseLibrary::CStrings strings; // will pass pointer to lib - -private: - std::unique_ptr ptr_holder = nullptr; - Container strings_holder; -}; - - -namespace -{ - constexpr auto lib_config_settings = ".settings"; - - - CStringsHolder getLibSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_root) - { - Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys(config_root, config_keys); - CStringsHolder::Container strings; - for (const auto & key : config_keys) - { - std::string key_name = key; - auto bracket_pos = key.find('['); - if (bracket_pos != std::string::npos && bracket_pos > 0) - key_name = key.substr(0, bracket_pos); - strings.emplace_back(key_name); - strings.emplace_back(config.getString(config_root + "." + key)); - } - return CStringsHolder(strings); - } - - - Block dataToBlock(const Block & sample_block, const ClickHouseLibrary::RawClickHouseLibraryTable data) - { - if (!data) - throw Exception("LibraryDictionarySource: No data returned", ErrorCodes::EXTERNAL_LIBRARY_ERROR); - - const auto * columns_received = static_cast(data); - if (columns_received->error_code) - throw Exception( - "LibraryDictionarySource: Returned error: " + std::to_string(columns_received->error_code) + " " - + (columns_received->error_string ? columns_received->error_string : ""), - ErrorCodes::EXTERNAL_LIBRARY_ERROR); - - MutableColumns columns = sample_block.cloneEmptyColumns(); - - for (size_t col_n = 0; col_n < columns_received->size; ++col_n) - { - if (columns.size() != columns_received->data[col_n].size) - throw Exception( - "LibraryDictionarySource: Returned unexpected number of columns: " + std::to_string(columns_received->data[col_n].size) - + ", must be " + std::to_string(columns.size()), - ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - - for (size_t row_n = 0; row_n < columns_received->data[col_n].size; ++row_n) - { - const auto & field = columns_received->data[col_n].data[row_n]; - if (!field.data) - { - /// sample_block contains null_value (from config) inside corresponding column - const auto & col = sample_block.getByPosition(row_n); - columns[row_n]->insertFrom(*(col.column), 0); - } - else - { - const auto & size = field.size; - columns[row_n]->insertData(static_cast(field.data), size); - } - } - } - - return sample_block.cloneWithColumns(std::move(columns)); - } -} - - LibraryDictionarySource::LibraryDictionarySource( const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix_, Block & sample_block_, - const Context & context, + ContextPtr context_, bool check_config) : log(&Poco::Logger::get("LibraryDictionarySource")) , dict_struct{dict_struct_} , config_prefix{config_prefix_} , path{config.getString(config_prefix + ".path", "")} + , dictionary_id(getDictID()) , sample_block{sample_block_} + , context(Context::createCopy(context_)) { if (check_config) { - const String dictionaries_lib_path = context.getDictionariesLibPath(); + const String dictionaries_lib_path = context->getDictionariesLibPath(); if (!startsWith(path, dictionaries_lib_path)) - throw Exception("LibraryDictionarySource: Library path " + path + " is not inside " + dictionaries_lib_path, ErrorCodes::PATH_ACCESS_DENIED); + throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "LibraryDictionarySource: Library path {} is not inside {}", path, dictionaries_lib_path); } if (!Poco::File(path).exists()) - throw Exception( - "LibraryDictionarySource: Can't load library " + Poco::File(path).path() + ": file doesn't exist", - ErrorCodes::FILE_DOESNT_EXIST); + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "LibraryDictionarySource: Can't load library {}: file doesn't exist", Poco::File(path).path()); description.init(sample_block); - library = std::make_shared(path, RTLD_LAZY -#if defined(RTLD_DEEPBIND) && !defined(ADDRESS_SANITIZER) // Does not exists in FreeBSD. Cannot work with Address Sanitizer. - | RTLD_DEEPBIND -#endif - ); - settings = std::make_shared(getLibSettings(config, config_prefix + lib_config_settings)); + bridge_helper = std::make_shared(context, description.sample_block, dictionary_id); + auto res = bridge_helper->initLibrary(path, getLibrarySettingsString(config, config_prefix + ".settings"), getDictAttributesString()); - if (auto lib_new = library->tryGet(ClickHouseLibrary::LIBRARY_CREATE_NEW_FUNC_NAME)) - lib_data = lib_new(&settings->strings, ClickHouseLibrary::log); + if (!res) + throw Exception(ErrorCodes::EXTERNAL_LIBRARY_ERROR, "Failed to create shared library from path: {}", path); } + +LibraryDictionarySource::~LibraryDictionarySource() +{ + bridge_helper->removeLibrary(); +} + + LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other) : log(&Poco::Logger::get("LibraryDictionarySource")) , dict_struct{other.dict_struct} , config_prefix{other.config_prefix} , path{other.path} + , dictionary_id{getDictID()} , sample_block{other.sample_block} - , library{other.library} + , context(other.context) , description{other.description} - , settings{other.settings} { - if (auto lib_clone = library->tryGet(ClickHouseLibrary::LIBRARY_CLONE_FUNC_NAME)) - lib_data = lib_clone(other.lib_data); - else if (auto lib_new = library->tryGet(ClickHouseLibrary::LIBRARY_CREATE_NEW_FUNC_NAME)) - lib_data = lib_new(&settings->strings, ClickHouseLibrary::log); + bridge_helper = std::make_shared(context, description.sample_block, dictionary_id); + bridge_helper->cloneLibrary(other.dictionary_id); } -LibraryDictionarySource::~LibraryDictionarySource() + +bool LibraryDictionarySource::isModified() const { - if (auto lib_delete = library->tryGet(ClickHouseLibrary::LIBRARY_DELETE_FUNC_NAME)) - lib_delete(lib_data); + return bridge_helper->isModified(); } + +bool LibraryDictionarySource::supportsSelectiveLoad() const +{ + return bridge_helper->supportsSelectiveLoad(); +} + + BlockInputStreamPtr LibraryDictionarySource::loadAll() { LOG_TRACE(log, "loadAll {}", toString()); - - auto columns_holder = std::make_unique(dict_struct.attributes.size()); - ClickHouseLibrary::CStrings columns{static_cast(columns_holder.get()), - dict_struct.attributes.size()}; - size_t i = 0; - for (const auto & a : dict_struct.attributes) - { - columns.data[i] = a.name.c_str(); - ++i; - } - - auto load_all_func = library->get(ClickHouseLibrary::LIBRARY_LOAD_ALL_FUNC_NAME); - auto data_new_func = library->get(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME); - auto data_delete_func = library->get(ClickHouseLibrary::LIBRARY_DATA_DELETE_FUNC_NAME); - - ClickHouseLibrary::LibraryData data_ptr = data_new_func(lib_data); - SCOPE_EXIT(data_delete_func(lib_data, data_ptr)); - - ClickHouseLibrary::RawClickHouseLibraryTable data = load_all_func(data_ptr, &settings->strings, &columns); - auto block = dataToBlock(description.sample_block, data); - - return std::make_shared(block); + return bridge_helper->loadAll(); } + BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); - - const ClickHouseLibrary::VectorUInt64 ids_data{ext::bit_cast(ids.data()), ids.size()}; - auto columns_holder = std::make_unique(dict_struct.attributes.size()); - ClickHouseLibrary::CStrings columns_pass{static_cast(columns_holder.get()), - dict_struct.attributes.size()}; - size_t i = 0; - for (const auto & a : dict_struct.attributes) - { - columns_pass.data[i] = a.name.c_str(); - ++i; - } - - auto load_ids_func = library->get(ClickHouseLibrary::LIBRARY_LOAD_IDS_FUNC_NAME); - auto data_new_func = library->get(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME); - auto data_delete_func = library->get(ClickHouseLibrary::LIBRARY_DATA_DELETE_FUNC_NAME); - - ClickHouseLibrary::LibraryData data_ptr = data_new_func(lib_data); - SCOPE_EXIT(data_delete_func(lib_data, data_ptr)); - - ClickHouseLibrary::RawClickHouseLibraryTable data = load_ids_func(data_ptr, &settings->strings, &columns_pass, &ids_data); - auto block = dataToBlock(description.sample_block, data); - - return std::make_shared(block); + return bridge_helper->loadIds(getDictIdsString(ids)); } + BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); - - auto holder = std::make_unique(key_columns.size()); - std::vector> column_data_holders; - for (size_t i = 0; i < key_columns.size(); ++i) - { - auto cell_holder = std::make_unique(requested_rows.size()); - for (size_t j = 0; j < requested_rows.size(); ++j) - { - auto data_ref = key_columns[i]->getDataAt(requested_rows[j]); - cell_holder[j] = ClickHouseLibrary::Field{.data = static_cast(data_ref.data), .size = data_ref.size}; - } - holder[i] - = ClickHouseLibrary::Row{.data = static_cast(cell_holder.get()), .size = requested_rows.size()}; - - column_data_holders.push_back(std::move(cell_holder)); - } - - ClickHouseLibrary::Table request_cols{.data = static_cast(holder.get()), .size = key_columns.size()}; - - auto load_keys_func = library->get(ClickHouseLibrary::LIBRARY_LOAD_KEYS_FUNC_NAME); - auto data_new_func = library->get(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME); - auto data_delete_func = library->get(ClickHouseLibrary::LIBRARY_DATA_DELETE_FUNC_NAME); - - ClickHouseLibrary::LibraryData data_ptr = data_new_func(lib_data); - SCOPE_EXIT(data_delete_func(lib_data, data_ptr)); - - ClickHouseLibrary::RawClickHouseLibraryTable data = load_keys_func(data_ptr, &settings->strings, &request_cols); - auto block = dataToBlock(description.sample_block, data); - - return std::make_shared(block); + auto block = blockForKeys(dict_struct, key_columns, requested_rows); + return bridge_helper->loadKeys(block); } -bool LibraryDictionarySource::isModified() const -{ - if (auto func_is_modified = library->tryGet( - ClickHouseLibrary::LIBRARY_IS_MODIFIED_FUNC_NAME)) - return func_is_modified(lib_data, &settings->strings); - - return true; -} - -bool LibraryDictionarySource::supportsSelectiveLoad() const -{ - if (auto func_supports_selective_load = library->tryGet( - ClickHouseLibrary::LIBRARY_SUPPORTS_SELECTIVE_LOAD_FUNC_NAME)) - return func_supports_selective_load(lib_data, &settings->strings); - - return true; -} DictionarySourcePtr LibraryDictionarySource::clone() const { return std::make_unique(*this); } + std::string LibraryDictionarySource::toString() const { return path; } + +String LibraryDictionarySource::getLibrarySettingsString(const Poco::Util::AbstractConfiguration & config, const std::string & config_root) +{ + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(config_root, config_keys); + WriteBufferFromOwnString out; + std::vector settings; + + for (const auto & key : config_keys) + { + std::string key_name = key; + auto bracket_pos = key.find('['); + + if (bracket_pos != std::string::npos && bracket_pos > 0) + key_name = key.substr(0, bracket_pos); + + settings.push_back(key_name); + settings.push_back(config.getString(config_root + "." + key)); + } + + writeVectorBinary(settings, out); + return out.str(); +} + + +String LibraryDictionarySource::getDictIdsString(const std::vector & ids) +{ + WriteBufferFromOwnString out; + writeVectorBinary(ids, out); + return out.str(); +} + + +String LibraryDictionarySource::getDictAttributesString() +{ + std::vector attributes_names(dict_struct.attributes.size()); + for (size_t i = 0; i < dict_struct.attributes.size(); ++i) + attributes_names[i] = dict_struct.attributes[i].name; + WriteBufferFromOwnString out; + writeVectorBinary(attributes_names, out); + return out.str(); +} + + void registerDictionarySourceLibrary(DictionarySourceFactory & factory) { auto create_table_source = [=](const DictionaryStructure & dict_struct, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - const Context & context, + ContextPtr context, const std::string & /* default_database */, bool check_config) -> DictionarySourcePtr { return std::make_unique(dict_struct, config, config_prefix + ".library", sample_block, context, check_config); }; + factory.registerSource("library", create_table_source); } + } diff --git a/src/Dictionaries/LibraryDictionarySource.h b/src/Dictionaries/LibraryDictionarySource.h index 4d73b3f97d4..1ab47c5a06f 100644 --- a/src/Dictionaries/LibraryDictionarySource.h +++ b/src/Dictionaries/LibraryDictionarySource.h @@ -1,10 +1,13 @@ #pragma once #include +#include #include +#include #include "DictionaryStructure.h" #include #include "IDictionarySource.h" +#include namespace Poco @@ -17,18 +20,17 @@ namespace Util } } - namespace DB { + namespace ErrorCodes { extern const int NOT_IMPLEMENTED; } -class CStringsHolder; -/// Allows loading dictionaries from dynamic libraries (.so) -/// Experimental version -/// Example: tests/external_dictionaries/dictionary_library/dictionary_library.cpp +class CStringsHolder; +using LibraryBridgeHelperPtr = std::shared_ptr; + class LibraryDictionarySource final : public IDictionarySource { public: @@ -37,7 +39,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix_, Block & sample_block_, - const Context & context, + ContextPtr context_, bool check_config); LibraryDictionarySource(const LibraryDictionarySource & other); @@ -49,7 +51,7 @@ public: BlockInputStreamPtr loadUpdatedAll() override { - throw Exception{"Method loadUpdatedAll is unsupported for LibraryDictionarySource", ErrorCodes::NOT_IMPLEMENTED}; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for LibraryDictionarySource"); } BlockInputStreamPtr loadIds(const std::vector & ids) override; @@ -68,18 +70,26 @@ public: std::string toString() const override; private: - Poco::Logger * log; + static String getDictIdsString(const std::vector & ids); - LocalDateTime getLastModification() const; + String getDictAttributesString(); + + static String getLibrarySettingsString(const Poco::Util::AbstractConfiguration & config, const std::string & config_root); + + static Field getDictID() { return UUIDHelpers::generateV4(); } + + Poco::Logger * log; const DictionaryStructure dict_struct; const std::string config_prefix; const std::string path; + const Field dictionary_id; + Block sample_block; - SharedLibraryPtr library; + ContextPtr context; + + LibraryBridgeHelperPtr bridge_helper; ExternalResultDescription description; - std::shared_ptr settings; - void * lib_data = nullptr; }; } diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index 5b5d0c4d20f..0ab45dc4593 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -13,7 +13,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & root_config_prefix, Block & sample_block, - const Context &, + ContextPtr, const std::string & /* default_database */, bool /* check_config */) { @@ -126,7 +126,7 @@ MongoDBDictionarySource::MongoDBDictionarySource( #if POCO_VERSION >= 0x01070800 Poco::MongoDB::Database poco_db(db); if (!poco_db.authenticate(*connection, user, password, method.empty() ? Poco::MongoDB::Database::AUTH_SCRAM_SHA1 : method)) - throw Exception("Cannot authenticate in MongoDB, incorrect user or password", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE); + throw Exception(ErrorCodes::MONGODB_CANNOT_AUTHENTICATE, "Cannot authenticate in MongoDB, incorrect user or password"); #else authenticate(*connection, db, user, password); #endif @@ -151,7 +151,7 @@ BlockInputStreamPtr MongoDBDictionarySource::loadAll() BlockInputStreamPtr MongoDBDictionarySource::loadIds(const std::vector & ids) { if (!dict_struct.id) - throw Exception{"'id' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is required for selective loading"); auto cursor = createCursor(db, collection, sample_block); @@ -172,7 +172,7 @@ BlockInputStreamPtr MongoDBDictionarySource::loadIds(const std::vector & BlockInputStreamPtr MongoDBDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { if (!dict_struct.key) - throw Exception{"'key' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is required for selective loading"); auto cursor = createCursor(db, collection, sample_block); @@ -198,6 +198,7 @@ BlockInputStreamPtr MongoDBDictionarySource::loadKeys(const Columns & key_column case AttributeUnderlyingType::utDecimal32: case AttributeUnderlyingType::utDecimal64: case AttributeUnderlyingType::utDecimal128: + case AttributeUnderlyingType::utDecimal256: key.add(attr.second.name, Int32(key_columns[attr.first]->get64(row_idx))); break; diff --git a/src/Dictionaries/MongoDBDictionarySource.h b/src/Dictionaries/MongoDBDictionarySource.h index ad7b66fe1a7..fef5749190f 100644 --- a/src/Dictionaries/MongoDBDictionarySource.h +++ b/src/Dictionaries/MongoDBDictionarySource.h @@ -50,7 +50,7 @@ public: BlockInputStreamPtr loadUpdatedAll() override { - throw Exception{"Method loadUpdatedAll is unsupported for MongoDBDictionarySource", ErrorCodes::NOT_IMPLEMENTED}; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for MongoDBDictionarySource"); } bool supportsSelectiveLoad() const override { return true; } diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index a1d0cbe7ffc..a78da1c7e74 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -18,7 +18,7 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - const Context & /* context */, + ContextPtr /* context */, const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { #if USE_MYSQL @@ -28,8 +28,8 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) (void)config; (void)config_prefix; (void)sample_block; - throw Exception{"Dictionary source of type `mysql` is disabled because ClickHouse was built without mysql support.", - ErrorCodes::SUPPORT_IS_DISABLED}; + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Dictionary source of type `mysql` is disabled because ClickHouse was built without mysql support."); #endif }; factory.registerSource("mysql", create_table_source); @@ -70,7 +70,7 @@ MySQLDictionarySource::MySQLDictionarySource( , update_field{config.getString(config_prefix + ".update_field", "")} , dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)} , sample_block{sample_block_} - , pool{mysqlxx::PoolFactory::instance().get(config, config_prefix)} + , pool{std::make_shared(mysqlxx::PoolFactory::instance().get(config, config_prefix))} , query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks} , load_all_query{query_builder.composeLoadAllQuery()} , invalidate_query{config.getString(config_prefix + ".invalidate_query", "")} @@ -119,67 +119,43 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate() } } -BlockInputStreamPtr MySQLDictionarySource::retriedCreateMySqlBIStream(const std::string & data_fetch_query_str, const size_t max_tries) +BlockInputStreamPtr MySQLDictionarySource::loadFromQuery(const String & query) { - size_t count_connection_lost = 0; - - while (true) - { - auto connection = pool.get(); - - try - { - return std::make_shared( - connection, data_fetch_query_str, sample_block, max_block_size, close_connection); - } - catch (const mysqlxx::ConnectionLost & ecl) /// There are two retriable failures: CR_SERVER_GONE_ERROR, CR_SERVER_LOST - { - if (++count_connection_lost < max_tries) - { - LOG_WARNING(log, ecl.displayText()); - LOG_WARNING(log, "Lost connection ({}/{}). Trying to reconnect...", count_connection_lost, max_tries); - continue; - } - - LOG_ERROR(log, "Failed ({}/{}) to create BlockInputStream for MySQL dictionary source.", count_connection_lost, max_tries); - throw; - } - } + return std::make_shared( + pool, query, sample_block, max_block_size, close_connection, false, max_tries_for_mysql_block_input_stream); } BlockInputStreamPtr MySQLDictionarySource::loadAll() { - auto connection = pool.get(); + auto connection = pool->get(); last_modification = getLastModification(connection, false); LOG_TRACE(log, load_all_query); - return retriedCreateMySqlBIStream(load_all_query, max_tries_for_mysql_block_input_stream); + return loadFromQuery(load_all_query); } BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll() { - auto connection = pool.get(); + auto connection = pool->get(); last_modification = getLastModification(connection, false); std::string load_update_query = getUpdateFieldAndDate(); LOG_TRACE(log, load_update_query); - return retriedCreateMySqlBIStream(load_update_query, max_tries_for_mysql_block_input_stream); + return loadFromQuery(load_update_query); } BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector & ids) { /// We do not log in here and do not update the modification time, as the request can be large, and often called. - const auto query = query_builder.composeLoadIdsQuery(ids); - return retriedCreateMySqlBIStream(query, max_tries_for_mysql_block_input_stream); + return loadFromQuery(query); } BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { /// We do not log in here and do not update the modification time, as the request can be large, and often called. - const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); - return retriedCreateMySqlBIStream(query, max_tries_for_mysql_block_input_stream); + return loadFromQuery(query); } bool MySQLDictionarySource::isModified() const @@ -195,7 +171,7 @@ bool MySQLDictionarySource::isModified() const if (dont_check_update_time) return true; - auto connection = pool.get(); + auto connection = pool->get(); return getLastModification(connection, true) > last_modification; } @@ -293,7 +269,7 @@ std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request Block invalidate_sample_block; ColumnPtr column(ColumnString::create()); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - MySQLBlockInputStream block_input_stream(pool.get(), request, invalidate_sample_block, 1, close_connection); + MySQLBlockInputStream block_input_stream(pool->get(), request, invalidate_sample_block, 1, close_connection); return readInvalidateQuery(block_input_stream); } diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index 54bc2da6a56..dcd911146aa 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -34,7 +34,7 @@ public: MySQLDictionarySource( const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, + const String & config_prefix, const Block & sample_block_); /// copy-constructor is provided in order to support cloneability @@ -60,6 +60,8 @@ public: std::string toString() const override; private: + BlockInputStreamPtr loadFromQuery(const String & query); + std::string getUpdateFieldAndDate(); static std::string quoteForLike(const std::string s); @@ -69,9 +71,6 @@ private: // execute invalidate_query. expects single cell in result std::string doInvalidateQuery(const std::string & request) const; - /// A helper method for recovering from "Lost connection to MySQL server during query" errors - BlockInputStreamPtr retriedCreateMySqlBIStream(const std::string & query_str, const size_t max_tries); - Poco::Logger * log; std::chrono::time_point update_time; @@ -82,7 +81,7 @@ private: const std::string update_field; const bool dont_check_update_time; Block sample_block; - mutable mysqlxx::PoolWithFailover pool; + mutable mysqlxx::PoolWithFailoverPtr pool; ExternalQueryBuilder query_builder; const std::string load_all_query; LocalDateTime last_modification; diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 7046741b3a0..64d435cf632 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -151,7 +151,7 @@ BlockInputStreamPtr IPolygonDictionary::getBlockInputStream(const Names &, size_ { // TODO: In order for this to work one would first have to support retrieving arrays from dictionaries. // I believe this is a separate task done by some other people. - throw Exception{"Reading the dictionary is not allowed", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Reading the dictionary is not allowed"); } void IPolygonDictionary::setup() @@ -164,9 +164,9 @@ void IPolygonDictionary::setup() attributes.emplace_back(std::move(column)); if (attribute.hierarchical) - throw Exception{ErrorCodes::TYPE_MISMATCH, + throw Exception(ErrorCodes::TYPE_MISMATCH, "{}: hierarchical attributes not supported for dictionary of polygonal type", - getDictionaryID().getNameForLogs()}; + getDictionaryID().getNameForLogs()); } } @@ -248,13 +248,13 @@ void IPolygonDictionary::calculateBytesAllocated() std::vector IPolygonDictionary::extractPoints(const Columns & key_columns) { if (key_columns.size() != 2) - throw Exception{"Expected two columns of coordinates with type Float64", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected two columns of coordinates with type Float64"); const auto * column_x = typeid_cast*>(key_columns[0].get()); const auto * column_y = typeid_cast*>(key_columns[1].get()); if (!column_x || !column_y) - throw Exception{"Expected columns of Float64", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected columns of Float64"); const auto rows = key_columns.front()->size(); @@ -392,17 +392,17 @@ const IColumn * unrollMultiPolygons(const ColumnPtr & column, Offset & offset) { const auto * ptr_multi_polygons = typeid_cast(column.get()); if (!ptr_multi_polygons) - throw Exception{"Expected a column containing arrays of polygons", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column containing arrays of polygons"); offset.multi_polygon_offsets.assign(ptr_multi_polygons->getOffsets()); const auto * ptr_polygons = typeid_cast(&ptr_multi_polygons->getData()); if (!ptr_polygons) - throw Exception{"Expected a column containing arrays of rings when reading polygons", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column containing arrays of rings when reading polygons"); offset.polygon_offsets.assign(ptr_polygons->getOffsets()); const auto * ptr_rings = typeid_cast(&ptr_polygons->getData()); if (!ptr_rings) - throw Exception{"Expected a column containing arrays of points when reading rings", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column containing arrays of points when reading rings"); offset.ring_offsets.assign(ptr_rings->getOffsets()); return ptr_rings->getDataPtr().get(); @@ -412,7 +412,7 @@ const IColumn * unrollSimplePolygons(const ColumnPtr & column, Offset & offset) { const auto * ptr_polygons = typeid_cast(column.get()); if (!ptr_polygons) - throw Exception{"Expected a column containing arrays of points", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column containing arrays of points"); offset.ring_offsets.assign(ptr_polygons->getOffsets()); std::iota(offset.polygon_offsets.begin(), offset.polygon_offsets.end(), 1); offset.multi_polygon_offsets.assign(offset.polygon_offsets); @@ -425,13 +425,13 @@ void handlePointsReprByArrays(const IColumn * column, Data & data, Offset & offs const auto * ptr_points = typeid_cast(column); const auto * ptr_coord = typeid_cast*>(&ptr_points->getData()); if (!ptr_coord) - throw Exception{"Expected coordinates to be of type Float64", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected coordinates to be of type Float64"); const auto & offsets = ptr_points->getOffsets(); IColumn::Offset prev_offset = 0; for (size_t i = 0; i < offsets.size(); ++i) { if (offsets[i] - prev_offset != 2) - throw Exception{"All points should be two-dimensional", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "All points should be two-dimensional"); prev_offset = offsets[i]; addNewPoint(ptr_coord->getElement(2 * i), ptr_coord->getElement(2 * i + 1), data, offset); } @@ -441,13 +441,13 @@ void handlePointsReprByTuples(const IColumn * column, Data & data, Offset & offs { const auto * ptr_points = typeid_cast(column); if (!ptr_points) - throw Exception{"Expected a column of tuples representing points", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column of tuples representing points"); if (ptr_points->tupleSize() != 2) - throw Exception{"Points should be two-dimensional", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Points should be two-dimensional"); const auto * column_x = typeid_cast*>(&ptr_points->getColumn(0)); const auto * column_y = typeid_cast*>(&ptr_points->getColumn(1)); if (!column_x || !column_y) - throw Exception{"Expected coordinates to be of type Float64", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected coordinates to be of type Float64"); for (size_t i = 0; i < column_x->size(); ++i) { addNewPoint(column_x->getElement(i), column_y->getElement(i), data, offset); @@ -473,8 +473,8 @@ void IPolygonDictionary::extractPolygons(const ColumnPtr & column) } if (!offset.allRingsHaveAPositiveArea()) - throw Exception{"Every ring included in a polygon or excluded from it should contain at least 3 points", - ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Every ring included in a polygon or excluded from it should contain at least 3 points"); /** Adding the first empty polygon */ data.addPolygon(true); diff --git a/src/Dictionaries/PolygonDictionaryImplementations.cpp b/src/Dictionaries/PolygonDictionaryImplementations.cpp index cb4bd05fd83..fb4fb605465 100644 --- a/src/Dictionaries/PolygonDictionaryImplementations.cpp +++ b/src/Dictionaries/PolygonDictionaryImplementations.cpp @@ -172,10 +172,10 @@ DictionaryPtr createLayout(const std::string & , const String name = config.getString(config_prefix + ".name"); if (!dict_struct.key) - throw Exception{"'key' is required for a polygon dictionary", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'key' is required for a polygon dictionary"); if (dict_struct.key->size() != 1) - throw Exception{"The 'key' should consist of a single attribute for a polygon dictionary", - ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "The 'key' should consist of a single attribute for a polygon dictionary"); IPolygonDictionary::InputType input_type; IPolygonDictionary::PointType point_type; @@ -206,19 +206,19 @@ DictionaryPtr createLayout(const std::string & , point_type = IPolygonDictionary::PointType::Tuple; } else - throw Exception{"The key type " + key_type->getName() + - " is not one of the following allowed types for a polygon dictionary: " + - multi_polygon_array.getName() + " " + - multi_polygon_tuple.getName() + " " + - simple_polygon_array.getName() + " " + - simple_polygon_tuple.getName() + " ", - ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "The key type {} is not one of the following allowed types for a polygon dictionary: {} {} {} {} ", + key_type->getName(), + multi_polygon_array.getName(), + multi_polygon_tuple.getName(), + simple_polygon_array.getName(), + simple_polygon_tuple.getName()); if (dict_struct.range_min || dict_struct.range_max) - throw Exception{name - + ": elements range_min and range_max should be defined only " - "for a dictionary of layout 'range_hashed'", - ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "{}: elements range_min and range_max should be defined only " + "for a dictionary of layout 'range_hashed'", + name); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 79298fc2fb3..54022dfd5cb 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -31,7 +31,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource( const Block & sample_block_) : dict_struct{dict_struct_} , sample_block(sample_block_) - , connection(std::make_shared(config_, config_prefix)) + , connection(std::make_shared(config_, config_prefix)) , log(&Poco::Logger::get("PostgreSQLDictionarySource")) , db(config_.getString(fmt::format("{}.db", config_prefix), "")) , table(config_.getString(fmt::format("{}.table", config_prefix), "")) @@ -167,7 +167,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & root_config_prefix, Block & sample_block, - const Context & /* context */, + ContextPtr /* context */, const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { @@ -180,8 +180,8 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) (void)config; (void)root_config_prefix; (void)sample_block; - throw Exception{"Dictionary source of type `postgresql` is disabled because ClickHouse was built without postgresql support.", - ErrorCodes::SUPPORT_IS_DISABLED}; + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Dictionary source of type `postgresql` is disabled because ClickHouse was built without postgresql support."); #endif }; factory.registerSource("postgresql", create_table_source); diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index dd2d35db83a..f1520a37a79 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include @@ -51,7 +51,7 @@ private: const DictionaryStructure dict_struct; Block sample_block; - PostgreSQLReplicaConnectionPtr connection; + postgres::PoolWithFailoverPtr connection; Poco::Logger * log; const std::string db; diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index 30395114a8e..1bdcd942b4a 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -260,8 +260,8 @@ void RangeHashedDictionary::createAttributes() attributes.push_back(createAttribute(attribute, attribute.null_value)); if (attribute.hierarchical) - throw Exception{ErrorCodes::BAD_ARGUMENTS, "Hierarchical attributes not supported by {} dictionary.", - getDictionaryID().getNameForLogs()}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Hierarchical attributes not supported by {} dictionary.", + getDictionaryID().getNameForLogs()); } } @@ -311,8 +311,8 @@ void RangeHashedDictionary::loadData() stream->readSuffix(); if (require_nonempty && 0 == element_count) - throw Exception{full_name + ": dictionary source is empty and 'require_nonempty' property is set.", - ErrorCodes::DICTIONARY_IS_EMPTY}; + throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, + "{}: dictionary source is empty and 'require_nonempty' property is set."); } template @@ -497,7 +497,7 @@ const RangeHashedDictionary::Attribute & RangeHashedDictionary::getAttribute(con { const auto it = attribute_index_by_name.find(attribute_name); if (it == std::end(attribute_index_by_name)) - throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}: no such attribute '{}'", full_name, attribute_name); return attributes[it->second]; } @@ -507,8 +507,9 @@ RangeHashedDictionary::getAttributeWithType(const std::string & attribute_name, { const auto & attribute = getAttribute(attribute_name); if (attribute.type != type) - throw Exception{attribute_name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), - ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "attribute {} has type {}", + attribute_name, + toString(attribute.type)); return attribute; } @@ -613,8 +614,9 @@ BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & col ListType::forEach(callable); if (!callable.stream) - throw Exception( - "Unexpected range type for RangeHashed dictionary: " + dict_struct.range_min->type->getName(), ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected range type for RangeHashed dictionary: {}", + dict_struct.range_min->type->getName()); return callable.stream; } @@ -629,11 +631,12 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) DictionarySourcePtr source_ptr) -> DictionaryPtr { if (dict_struct.key) - throw Exception{"'key' is not supported for dictionary of layout 'range_hashed'", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for dictionary of layout 'range_hashed'"); if (!dict_struct.range_min || !dict_struct.range_max) - throw Exception{full_name + ": dictionary of layout 'range_hashed' requires .structure.range_min and .structure.range_max", - ErrorCodes::BAD_ARGUMENTS}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "{}: dictionary of layout 'range_hashed' requires .structure.range_min and .structure.range_max", + full_name); const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index ca2a925df5e..8a286f530ba 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -112,6 +112,7 @@ private: Decimal32, Decimal64, Decimal128, + Decimal256, Float32, Float64, StringRef> @@ -129,6 +130,7 @@ private: Ptr, Ptr, Ptr, + Ptr, Ptr, Ptr, Ptr> diff --git a/src/Dictionaries/RedisBlockInputStream.cpp b/src/Dictionaries/RedisBlockInputStream.cpp index 39cc2d610df..f74ac348a99 100644 --- a/src/Dictionaries/RedisBlockInputStream.cpp +++ b/src/Dictionaries/RedisBlockInputStream.cpp @@ -57,7 +57,7 @@ namespace DB void insertValue(IColumn & column, const ValueType type, const Poco::Redis::BulkString & bulk_string) { if (bulk_string.isNull()) - throw Exception{"Type mismatch, expected not Null String", ErrorCodes::TYPE_MISMATCH}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected not Null String"); const String & string_value = bulk_string.value(); switch (type) @@ -112,7 +112,9 @@ namespace DB assert_cast(column).insertValue(parse(string_value)); break; default: - throw Exception("Value of unsupported type:" + column.getName(), ErrorCodes::UNKNOWN_TYPE); + throw Exception(ErrorCodes::UNKNOWN_TYPE, + "Value of unsupported type: {}", + column.getName()); } } } @@ -152,8 +154,9 @@ namespace DB const auto & keys_array = keys.get(cursor); if (keys_array.size() < 2) { - throw Exception{"Too low keys in request to source: " + DB::toString(keys_array.size()) - + ", expected 2 or more", ErrorCodes::LOGICAL_ERROR}; + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Too low keys in request to source: {}, expected 2 or more", + DB::toString(keys_array.size())); } if (num_rows + keys_array.size() - 1 > max_block_size) @@ -166,8 +169,8 @@ namespace DB auto values = client->execute(command_for_values); if (keys_array.size() != values.size() + 1) // 'HMGET' primary_key secondary_keys - throw Exception{"Inconsistent sizes of keys and values in Redis request", - ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH}; + throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, + "Inconsistent sizes of keys and values in Redis request"); const auto & primary_key = keys_array.get(0); for (size_t i = 0; i < values.size(); ++i) @@ -196,7 +199,8 @@ namespace DB auto values = client->execute(command_for_values); if (values.size() != need_values) - throw Exception{"Inconsistent sizes of keys and values in Redis request", ErrorCodes::INTERNAL_REDIS_ERROR}; + throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, + "Inconsistent sizes of keys and values in Redis request"); for (size_t i = 0; i < values.size(); ++i) { diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index f1d0c0c5d3c..8144b37e63d 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -12,7 +12,7 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const String & config_prefix, Block & sample_block, - const Context & /* context */, + ContextPtr /* context */, const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { return std::make_unique(dict_struct, config, config_prefix + ".redis", sample_block); @@ -66,25 +66,27 @@ namespace DB , client{std::make_shared(host, port)} { if (dict_struct.attributes.size() != 1) - throw Exception{"Invalid number of non key columns for Redis source: " + - DB::toString(dict_struct.attributes.size()) + ", expected 1", - ErrorCodes::INVALID_CONFIG_PARAMETER}; + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, + "Invalid number of non key columns for Redis source: {}, expected 1", + DB::toString(dict_struct.attributes.size())); if (storage_type == RedisStorageType::HASH_MAP) { if (!dict_struct.key) - throw Exception{"Redis source with storage type \'hash_map\' must have key", - ErrorCodes::INVALID_CONFIG_PARAMETER}; + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, + "Redis source with storage type \'hash_map\' must have key"); if (dict_struct.key->size() != 2) - throw Exception{"Redis source with storage type \'hash_map\' requires 2 keys", - ErrorCodes::INVALID_CONFIG_PARAMETER}; + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, + "Redis source with storage type \'hash_map\' requires 2 keys"); // suppose key[0] is primary key, key[1] is secondary key for (const auto & key : *dict_struct.key) if (!isInteger(key.type) && !isString(key.type)) throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, - "Redis source supports only integer or string key, but key '{}' of type {} given", key.name, key.type->getName()); + "Redis source supports only integer or string key, but key '{}' of type {} given", + key.name, + key.type->getName()); } if (!password.empty()) @@ -93,8 +95,9 @@ namespace DB command << password; String reply = client->execute(command); if (reply != "OK") - throw Exception{"Authentication failed with reason " - + reply, ErrorCodes::INTERNAL_REDIS_ERROR}; + throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, + "Authentication failed with reason {}", + reply); } if (db_index != 0) @@ -103,8 +106,10 @@ namespace DB command << std::to_string(db_index); String reply = client->execute(command); if (reply != "OK") - throw Exception{"Selecting database with index " + DB::toString(db_index) - + " failed with reason " + reply, ErrorCodes::INTERNAL_REDIS_ERROR}; + throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, + "Selecting database with index {} failed with reason {}", + DB::toString(db_index), + reply); } } @@ -215,10 +220,10 @@ namespace DB client->connect(host, port); if (storage_type == RedisStorageType::HASH_MAP) - throw Exception{"Cannot use loadIds with 'hash_map' storage type", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use loadIds with 'hash_map' storage type"); if (!dict_struct.id) - throw Exception{"'id' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is required for selective loading"); RedisArray keys; @@ -234,7 +239,7 @@ namespace DB client->connect(host, port); if (key_columns.size() != dict_struct.key->size()) - throw Exception{"The size of key_columns does not equal to the size of dictionary key", ErrorCodes::LOGICAL_ERROR}; + throw Exception(ErrorCodes::LOGICAL_ERROR, "The size of key_columns does not equal to the size of dictionary key"); RedisArray keys; for (auto row : requested_rows) @@ -268,7 +273,7 @@ namespace DB if (storage_type_str == "hash_map") return RedisStorageType::HASH_MAP; else if (!storage_type_str.empty() && storage_type_str != "simple") - throw Exception("Unknown storage type " + storage_type_str + " for Redis dictionary", ErrorCodes::INVALID_CONFIG_PARAMETER); + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Unknown storage type {} for Redis dictionary", storage_type_str); return RedisStorageType::SIMPLE; } diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index 71d2a7e724a..b2c5859decd 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -63,7 +63,7 @@ namespace ErrorCodes BlockInputStreamPtr loadUpdatedAll() override { - throw Exception{"Method loadUpdatedAll is unsupported for RedisDictionarySource", ErrorCodes::NOT_IMPLEMENTED}; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for RedisDictionarySource"); } bool supportsSelectiveLoad() const override { return true; } diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index 67f0465a2c7..7d72beca35e 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -465,7 +465,7 @@ public: /// If cache file is in directory that does not exists create it if (!std::filesystem::exists(parent_path_directory)) if (!std::filesystem::create_directories(parent_path_directory)) - throw Exception{"Failed to create directories.", ErrorCodes::CANNOT_CREATE_DIRECTORY}; + throw Exception(ErrorCodes::CANNOT_CREATE_DIRECTORY, "Failed to create directories."); ProfileEvents::increment(ProfileEvents::FileOpen); @@ -514,7 +514,7 @@ public: while (io_submit(aio_context.ctx, 1, &write_request_ptr) < 0) { if (errno != EINTR) - throw Exception("Cannot submit request for asynchronous IO on file " + file_path, ErrorCodes::CANNOT_IO_SUBMIT); + throw Exception(ErrorCodes::CANNOT_IO_SUBMIT, "Cannot submit request for asynchronous IO on file {}", file_path); } // CurrentMetrics::Increment metric_increment_write{CurrentMetrics::Write}; @@ -524,7 +524,7 @@ public: while (io_getevents(aio_context.ctx, 1, 1, &event, nullptr) < 0) { if (errno != EINTR) - throw Exception("Failed to wait for asynchronous IO completion on file " + file_path, ErrorCodes::CANNOT_IO_GETEVENTS); + throw Exception(ErrorCodes::CANNOT_IO_GETEVENTS, "Failed to wait for asynchronous IO completion on file {}", file_path); } // Unpoison the memory returned from an uninstrumented system function. @@ -536,7 +536,10 @@ public: ProfileEvents::increment(ProfileEvents::WriteBufferAIOWriteBytes, bytes_written); if (bytes_written != static_cast(block_size * buffer_size_in_blocks)) - throw Exception("Not all data was written for asynchronous IO on file " + file_path + ". returned: " + std::to_string(bytes_written), ErrorCodes::AIO_WRITE_ERROR); + throw Exception(ErrorCodes::AIO_WRITE_ERROR, + "Not all data was written for asynchronous IO on file {}. returned: {}", + file_path, + std::to_string(bytes_written)); if (::fsync(file.fd) < 0) throwFromErrnoWithPath("Cannot fsync " + file_path, file_path, ErrorCodes::CANNOT_FSYNC); @@ -593,7 +596,10 @@ public: if (read_bytes != static_cast(buffer_size_in_bytes)) throw Exception(ErrorCodes::AIO_READ_ERROR, - "GC: AIO failed to read file ({}). Expected bytes ({}). Actual bytes ({})", file_path, buffer_size_in_bytes, read_bytes); + "GC: AIO failed to read file {}. Expected bytes {}. Actual bytes {}", + file_path, + buffer_size_in_bytes, + read_bytes); SSDCacheBlock block(block_size); @@ -694,7 +700,10 @@ public: { std::string calculated_check_sum = std::to_string(block.calculateCheckSum()); std::string check_sum = std::to_string(block.getCheckSum()); - throw Exception("Cache data corrupted. Checksum validation failed. Calculated " + calculated_check_sum + " in block " + check_sum, ErrorCodes::CORRUPTED_DATA); + throw Exception(ErrorCodes::CORRUPTED_DATA, + "Cache data corrupted. Checksum validation failed. Calculated {} in block {}", + calculated_check_sum, + check_sum); } std::forward(func)(blocks_to_fetch[block_to_fetch_index], block.getBlockData()); @@ -835,7 +844,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::simple) return fetchColumnsForKeysImpl(keys, fetch_request); else - throw Exception("Method insertColumnsForKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage"); } void insertColumnsForKeys(const PaddedPODArray & keys, Columns columns) override @@ -843,7 +852,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::simple) insertColumnsForKeysImpl(keys, columns); else - throw Exception("Method insertColumnsForKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage"); } void insertDefaultKeys(const PaddedPODArray & keys) override @@ -851,7 +860,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::simple) insertDefaultKeysImpl(keys); else - throw Exception("Method insertDefaultKeysImpl is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage"); } PaddedPODArray getCachedSimpleKeys() const override @@ -859,7 +868,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::simple) return getCachedKeysImpl(); else - throw Exception("Method getCachedSimpleKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage"); } bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; } @@ -871,7 +880,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::complex) return fetchColumnsForKeysImpl(keys, fetch_request); else - throw Exception("Method fetchColumnsForKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage"); } void insertColumnsForKeys(const PaddedPODArray & keys, Columns columns) override @@ -879,7 +888,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::complex) insertColumnsForKeysImpl(keys, columns); else - throw Exception("Method insertColumnsForKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage"); } void insertDefaultKeys(const PaddedPODArray & keys) override @@ -887,7 +896,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::complex) insertDefaultKeysImpl(keys); else - throw Exception("Method insertDefaultKeysImpl is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage"); } PaddedPODArray getCachedComplexKeys() const override @@ -895,7 +904,7 @@ public: if constexpr (dictionary_key_type == DictionaryKeyType::complex) return getCachedKeysImpl(); else - throw Exception("Method getCachedSimpleKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for simple key storage"); } size_t getSize() const override { return index.size(); } @@ -1108,7 +1117,7 @@ private: SSDCacheKeyType ssd_cache_key { key, allocated_size_for_columns, block_start }; if (!SSDCacheBlock::canBeWrittenInEmptyBlock(ssd_cache_key, configuration.block_size)) - throw Exception("Serialized columns size is greater than allowed block size and metadata", ErrorCodes::UNSUPPORTED_METHOD); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Serialized columns size is greater than allowed block size and metadata"); /// We cannot reuse place that is already allocated in file or memory cache so we erase key from index eraseKeyFromIndex(key); diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 37d54f3549c..ac7b9111a11 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include "DictionarySourceFactory.h" @@ -41,7 +40,7 @@ namespace const Poco::URI & uri, std::function callback, const Block & sample_block, - const Context & context, + ContextPtr context, UInt64 max_block_size, const ConnectionTimeouts & timeouts, const String name_) @@ -89,9 +88,9 @@ namespace else { if (!schema.empty()) - throw Exception{"Dictionary source of type " + bridge_.getName() + " specifies a schema but schema is not supported by " - + bridge_.getName() + "-driver", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Dictionary source of type {0} specifies a schema but schema is not supported by {0}-driver", + bridge_.getName()); } return {dict_struct_, db_, schema, table, where_, bridge_.getIdentifierQuotingStyle()}; @@ -106,9 +105,10 @@ XDBCDictionarySource::XDBCDictionarySource( const Poco::Util::AbstractConfiguration & config_, const std::string & config_prefix_, const Block & sample_block_, - const Context & context_, + ContextPtr context_, const BridgeHelperPtr bridge_) - : log(&Poco::Logger::get(bridge_->getName() + "DictionarySource")) + : WithContext(context_->getGlobalContext()) + , log(&Poco::Logger::get(bridge_->getName() + "DictionarySource")) , update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , db{config_.getString(config_prefix_ + ".db", "")} @@ -122,7 +122,6 @@ XDBCDictionarySource::XDBCDictionarySource( , invalidate_query{config_.getString(config_prefix_ + ".invalidate_query", "")} , bridge_helper{bridge_} , timeouts{ConnectionTimeouts::getHTTPTimeouts(context_)} - , global_context(context_.getGlobalContext()) { bridge_url = bridge_helper->getMainURI(); @@ -133,7 +132,8 @@ XDBCDictionarySource::XDBCDictionarySource( /// copy-constructor is provided in order to support cloneability XDBCDictionarySource::XDBCDictionarySource(const XDBCDictionarySource & other) - : log(&Poco::Logger::get(other.bridge_helper->getName() + "DictionarySource")) + : WithContext(other.getContext()) + , log(&Poco::Logger::get(other.bridge_helper->getName() + "DictionarySource")) , update_time{other.update_time} , dict_struct{other.dict_struct} , db{other.db} @@ -148,7 +148,6 @@ XDBCDictionarySource::XDBCDictionarySource(const XDBCDictionarySource & other) , bridge_helper{other.bridge_helper} , bridge_url{other.bridge_url} , timeouts{other.timeouts} - , global_context{other.global_context} { } @@ -244,7 +243,7 @@ std::string XDBCDictionarySource::doInvalidateQuery(const std::string & request) invalidate_url, [request](std::ostream & os) { os << "query=" << request; }, invalidate_sample_block, - global_context, + getContext(), max_block_size, timeouts, bridge_helper->getName() + "BlockInputStream"); @@ -259,7 +258,7 @@ BlockInputStreamPtr XDBCDictionarySource::loadBase(const std::string & query) co bridge_url, [query](std::ostream & os) { os << "query=" << query; }, sample_block, - global_context, + getContext(), max_block_size, timeouts, bridge_helper->getName() + "BlockInputStream"); @@ -275,12 +274,12 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - const Context & context, + ContextPtr context, const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { #if USE_ODBC BridgeHelperPtr bridge = std::make_shared>( - context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string")); + context, context->getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string")); return std::make_unique(dict_struct, config, config_prefix + ".odbc", sample_block, context, bridge); #else (void)dict_struct; @@ -288,8 +287,8 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) (void)config_prefix; (void)sample_block; (void)context; - throw Exception{"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.", - ErrorCodes::SUPPORT_IS_DISABLED}; + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Dictionary source of type `odbc` is disabled because poco library was built without ODBC support."); #endif }; factory.registerSource("odbc", create_table_source); @@ -301,11 +300,11 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & /* config */, const std::string & /* config_prefix */, Block & /* sample_block */, - const Context & /* context */, + ContextPtr /* context */, const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { - throw Exception{"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.", - ErrorCodes::SUPPORT_IS_DISABLED}; + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Dictionary source of type `jdbc` is disabled until consistent support for nullable fields."); // BridgeHelperPtr bridge = std::make_shared>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string")); // return std::make_unique(dict_struct, config, config_prefix + ".jdbc", sample_block, context, bridge); }; diff --git a/src/Dictionaries/XDBCDictionarySource.h b/src/Dictionaries/XDBCDictionarySource.h index 87bc42c76ab..438f48027e5 100644 --- a/src/Dictionaries/XDBCDictionarySource.h +++ b/src/Dictionaries/XDBCDictionarySource.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include "DictionaryStructure.h" #include "ExternalQueryBuilder.h" #include "IDictionarySource.h" @@ -23,7 +23,7 @@ class Logger; namespace DB { /// Allows loading dictionaries from a XDBC source via bridges -class XDBCDictionarySource final : public IDictionarySource +class XDBCDictionarySource final : public IDictionarySource, WithContext { public: XDBCDictionarySource( @@ -31,7 +31,7 @@ public: const Poco::Util::AbstractConfiguration & config_, const std::string & config_prefix_, const Block & sample_block_, - const Context & context_, + ContextPtr context_, BridgeHelperPtr bridge); /// copy-constructor is provided in order to support cloneability @@ -82,7 +82,6 @@ private: BridgeHelperPtr bridge_helper; Poco::URI bridge_url; ConnectionTimeouts timeouts; - const Context & global_context; }; } diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 04ba1db09fc..55a965394a1 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -299,8 +299,8 @@ void buildPrimaryKeyConfiguration( if (!complex) { if (key_names.size() != 1) - throw Exception("Primary key for simple dictionary must contain exactly one element", - ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, + "Primary key for simple dictionary must contain exactly one element"); AutoPtr id_element(doc->createElement("id")); root->appendChild(id_element); @@ -317,8 +317,8 @@ void buildPrimaryKeyConfiguration( else { if (children.size() < key_names.size()) - throw Exception( - "Primary key fields count is more, than dictionary attributes count.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, + "Primary key fields count is more, than dictionary attributes count."); AutoPtr key_element(doc->createElement("key")); root->appendChild(key_element); @@ -336,8 +336,9 @@ void buildPrimaryKeyConfiguration( } } if (!found) - throw Exception( - "Primary key field '" + key_name + "' not found among attributes.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, + "Primary key field '{}' not found among attributes.", + key_name); } } } @@ -358,7 +359,7 @@ NamesToTypeNames buildDictionaryAttributesConfiguration( { const ASTDictionaryAttributeDeclaration * dict_attr = child->as(); if (!dict_attr->type) - throw Exception("Dictionary attribute must has type", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Dictionary attribute must has type"); attributes_names_and_types.emplace(dict_attr->name, queryToString(dict_attr->type)); if (std::find(key_columns.begin(), key_columns.end(), dict_attr->name) == key_columns.end()) @@ -374,7 +375,7 @@ void buildConfigurationFromFunctionWithKeyValueArguments( AutoPtr doc, AutoPtr root, const ASTExpressionList * ast_expr_list, - const Context & context) + ContextPtr context) { const auto & children = ast_expr_list->children; for (size_t i = 0; i != children.size(); ++i) @@ -416,9 +417,8 @@ void buildConfigurationFromFunctionWithKeyValueArguments( } else { - throw Exception( - "Incorrect ASTPair contains wrong value, should be literal, identifier or list", - ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, + "Incorrect ASTPair contains wrong value, should be literal, identifier or list"); } } } @@ -442,7 +442,7 @@ void buildSourceConfiguration( AutoPtr root, const ASTFunctionWithKeyValueArguments * source, const ASTDictionarySettings * settings, - const Context & context) + ContextPtr context) { AutoPtr outer_element(doc->createElement("source")); root->appendChild(outer_element); @@ -470,24 +470,24 @@ void buildSourceConfiguration( void checkAST(const ASTCreateQuery & query) { if (!query.is_dictionary || query.dictionary == nullptr) - throw Exception("Cannot convert dictionary to configuration from non-dictionary AST.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot convert dictionary to configuration from non-dictionary AST."); if (query.dictionary_attributes_list == nullptr || query.dictionary_attributes_list->children.empty()) - throw Exception("Cannot create dictionary with empty attributes list", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary with empty attributes list"); if (query.dictionary->layout == nullptr) - throw Exception("Cannot create dictionary with empty layout", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary with empty layout"); const auto is_direct_layout = !strcasecmp(query.dictionary->layout->layout_type.data(), "direct") || !strcasecmp(query.dictionary->layout->layout_type.data(), "complex_key_direct"); if (query.dictionary->lifetime == nullptr && !is_direct_layout) - throw Exception("Cannot create dictionary with empty lifetime", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary with empty lifetime"); if (query.dictionary->primary_key == nullptr) - throw Exception("Cannot create dictionary without primary key", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary without primary key"); if (query.dictionary->source == nullptr) - throw Exception("Cannot create dictionary with empty source", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary with empty source"); /// Range can be empty } @@ -496,14 +496,14 @@ void checkPrimaryKey(const NamesToTypeNames & all_attrs, const Names & key_attrs { for (const auto & key_attr : key_attrs) if (all_attrs.count(key_attr) == 0) - throw Exception("Unknown key attribute '" + key_attr + "'", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Unknown key attribute '{}'", key_attr); } } DictionaryConfigurationPtr -getDictionaryConfigurationFromAST(const ASTCreateQuery & query, const Context & context, const std::string & database_) +getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr context, const std::string & database_) { checkAST(query); diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.h b/src/Dictionaries/getDictionaryConfigurationFromAST.h index 5132e3c77e0..de8659e4d7b 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.h +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -11,5 +12,5 @@ using DictionaryConfigurationPtr = Poco::AutoPtr 1) - throw Exception("Expected single column in resultset, got " + std::to_string(columns), ErrorCodes::TOO_MANY_COLUMNS); + throw Exception(ErrorCodes::TOO_MANY_COLUMNS, "Expected single column in resultset, got {}", std::to_string(columns)); auto rows = block.rows(); if (rows == 0) - throw Exception("Expected single row in resultset, got 0", ErrorCodes::RECEIVED_EMPTY_DATA); + throw Exception(ErrorCodes::RECEIVED_EMPTY_DATA, "Expected single row in resultset, got 0"); if (rows > 1) - throw Exception("Expected single row in resultset, got at least " + std::to_string(rows), ErrorCodes::TOO_MANY_ROWS); + throw Exception(ErrorCodes::TOO_MANY_ROWS, "Expected single row in resultset, got at least {}", std::to_string(rows)); WriteBufferFromOwnString out; auto & column_type = block.getByPosition(0); @@ -38,7 +38,7 @@ std::string readInvalidateQuery(IBlockInputStream & block_input_stream) while ((block = block_input_stream.read())) if (block.rows() > 0) - throw Exception("Expected single row in resultset, got at least " + std::to_string(rows + 1), ErrorCodes::TOO_MANY_ROWS); + throw Exception(ErrorCodes::TOO_MANY_ROWS, "Expected single row in resultset, got at least {}", std::to_string(rows + 1)); block_input_stream.readSuffix(); return out.str(); diff --git a/src/Dictionaries/registerCacheDictionaries.cpp b/src/Dictionaries/registerCacheDictionaries.cpp index b93a08acb76..9cc90c44418 100644 --- a/src/Dictionaries/registerCacheDictionaries.cpp +++ b/src/Dictionaries/registerCacheDictionaries.cpp @@ -26,7 +26,7 @@ CacheDictionaryStorageConfiguration parseCacheStorageConfiguration( const size_t size = config.getUInt64(dictionary_configuration_prefix + "size_in_cells"); if (size == 0) throw Exception(ErrorCodes::TOO_SMALL_BUFFER_SIZE, - "({}): cache dictionary cannot have 0 cells", + "{}: cache dictionary cannot have 0 cells", full_name); size_t dict_lifetime_seconds = static_cast(dict_lifetime.max_sec); @@ -68,26 +68,26 @@ SSDCacheDictionaryStorageConfiguration parseSSDCacheStorageConfiguration( const size_t file_size = config.getInt64(dictionary_configuration_prefix + "file_size", DEFAULT_FILE_SIZE_BYTES); if (file_size % block_size != 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): file_size must be a multiple of block_size", + "{}: file_size must be a multiple of block_size", full_name); const size_t read_buffer_size = config.getInt64(dictionary_configuration_prefix + "read_buffer_size", DEFAULT_READ_BUFFER_SIZE_BYTES); if (read_buffer_size % block_size != 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): read_buffer_size must be a multiple of block_size", + "{}: read_buffer_size must be a multiple of block_size", full_name); const size_t write_buffer_size = config.getInt64(dictionary_configuration_prefix + "write_buffer_size", DEFAULT_WRITE_BUFFER_SIZE_BYTES); if (write_buffer_size % block_size != 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): write_buffer_size must be a multiple of block_size", + "{}: write_buffer_size must be a multiple of block_size", full_name); auto directory_path = config.getString(dictionary_configuration_prefix + "path"); if (directory_path.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): ssd cache dictionary cannot have empty path", + "{}: ssd cache dictionary cannot have empty path", full_name); if (directory_path.at(0) != '/') @@ -119,7 +119,7 @@ CacheDictionaryUpdateQueueConfiguration parseCacheDictionaryUpdateQueueConfigura const size_t max_update_queue_size = config.getUInt64(layout_prefix + ".cache.max_update_queue_size", 100000); if (max_update_queue_size == 0) throw Exception(ErrorCodes::TOO_SMALL_BUFFER_SIZE, - "({}): dictionary of layout '({})' cannot have empty update queue of size 0", + "{}: dictionary of layout '{}' cannot have empty update queue of size 0", full_name, layout_type); @@ -127,7 +127,7 @@ CacheDictionaryUpdateQueueConfiguration parseCacheDictionaryUpdateQueueConfigura = config.getUInt64(layout_prefix + ".cache.update_queue_push_timeout_milliseconds", 10); if (update_queue_push_timeout_milliseconds < 10) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): dictionary of layout '({})' have too little update_queue_push_timeout", + "{}: dictionary of layout '{}' have too little update_queue_push_timeout", full_name, layout_type); @@ -136,7 +136,7 @@ CacheDictionaryUpdateQueueConfiguration parseCacheDictionaryUpdateQueueConfigura const size_t max_threads_for_updates = config.getUInt64(layout_prefix + ".max_threads_for_updates", 4); if (max_threads_for_updates == 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): dictionary of layout) '({})' cannot have zero threads for updates", + "{}: dictionary of layout) '{}' cannot have zero threads for updates", full_name, layout_type); @@ -169,14 +169,14 @@ DictionaryPtr createCacheDictionaryLayout( if (dict_struct.range_min || dict_struct.range_max) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): elements .structure.range_min and .structure.range_max should be defined only " - "for a dictionary of layout 'range_hashed'", + "{}: elements .structure.range_min and .structure.range_max should be defined only " + "for a dictionary of layout 'range_hashed'", full_name); const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); if (require_nonempty) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): cache dictionary of layout cannot have 'require_nonempty' attribute set", + "{}: cache dictionary of layout cannot have 'require_nonempty' attribute set", full_name); const auto & layout_prefix = config_prefix + ".layout"; @@ -222,14 +222,14 @@ DictionaryPtr createSSDCacheDictionaryLayout( if (dict_struct.range_min || dict_struct.range_max) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): elements .structure.range_min and .structure.range_max should be defined only " - "for a dictionary of layout 'range_hashed'", + "{}: elements .structure.range_min and .structure.range_max should be defined only " + "for a dictionary of layout 'range_hashed'", full_name); const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); if (require_nonempty) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): cache dictionary of layout cannot have 'require_nonempty' attribute set", + "{}: cache dictionary of layout cannot have 'require_nonempty' attribute set", full_name); const auto & layout_prefix = config_prefix + ".layout"; diff --git a/src/Disks/DiskFactory.cpp b/src/Disks/DiskFactory.cpp index b604269ae16..03412be9729 100644 --- a/src/Disks/DiskFactory.cpp +++ b/src/Disks/DiskFactory.cpp @@ -24,7 +24,7 @@ DiskPtr DiskFactory::create( const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - const Context & context) const + ContextConstPtr context) const { const auto disk_type = config.getString(config_prefix + ".type", "local"); diff --git a/src/Disks/DiskFactory.h b/src/Disks/DiskFactory.h index d41f14bd753..ba833d9e25f 100644 --- a/src/Disks/DiskFactory.h +++ b/src/Disks/DiskFactory.h @@ -1,17 +1,18 @@ #pragma once -#include #include +#include +#include + +#include +#include #include #include -#include -#include namespace DB { -class Context; /** * Disk factory. Responsible for creating new disk objects. @@ -23,7 +24,7 @@ public: const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - const Context & context)>; + ContextConstPtr context)>; static DiskFactory & instance(); @@ -33,7 +34,7 @@ public: const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - const Context & context) const; + ContextConstPtr context) const; private: using DiskTypeRegistry = std::unordered_map; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 69732bc271e..9d2bad0373d 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -363,7 +363,7 @@ void registerDiskLocal(DiskFactory & factory) auto creator = [](const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - const Context & context) -> DiskPtr { + ContextConstPtr context) -> DiskPtr { String path = config.getString(config_prefix + ".path", ""); if (name == "default") { @@ -371,7 +371,7 @@ void registerDiskLocal(DiskFactory & factory) throw Exception( "\"default\" disk path should be provided in not it ", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - path = context.getPath(); + path = context->getPath(); } else { @@ -402,7 +402,7 @@ void registerDiskLocal(DiskFactory & factory) throw Exception("'keep_free_space_ratio' have to be between 0 and 1", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); String tmp_path = path; if (tmp_path.empty()) - tmp_path = context.getPath(); + tmp_path = context->getPath(); // Create tmp disk for getting total disk space. keep_free_space_bytes = static_cast(DiskLocal("tmp", tmp_path, 0).getTotalSpace() * ratio); diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index d8b3c74d0d9..68257ec4948 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -451,7 +451,7 @@ void registerDiskMemory(DiskFactory & factory) auto creator = [](const String & name, const Poco::Util::AbstractConfiguration & /*config*/, const String & /*config_prefix*/, - const Context & /*context*/) -> DiskPtr { return std::make_shared(name); }; + ContextConstPtr /*context*/) -> DiskPtr { return std::make_shared(name); }; factory.registerDiskType("memory", creator); } diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index 0fb728a4f02..e317d8508da 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes extern const int UNKNOWN_DISK; } -DiskSelector::DiskSelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Context & context) +DiskSelector::DiskSelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextConstPtr context) { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); @@ -40,12 +40,12 @@ DiskSelector::DiskSelector(const Poco::Util::AbstractConfiguration & config, con disks.emplace(disk_name, factory.create(disk_name, config, disk_config_prefix, context)); } if (!has_default_disk) - disks.emplace(default_disk_name, std::make_shared(default_disk_name, context.getPath(), 0)); + disks.emplace(default_disk_name, std::make_shared(default_disk_name, context->getPath(), 0)); } DiskSelectorPtr DiskSelector::updateFromConfig( - const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Context & context) const + const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) const { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index 5d023fe1fbc..6b7e8174d3d 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -10,7 +10,6 @@ namespace DB { -class Context; class DiskSelector; using DiskSelectorPtr = std::shared_ptr; using DisksMap = std::map; @@ -20,13 +19,13 @@ using DisksMap = std::map; class DiskSelector { public: - DiskSelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Context & context); + DiskSelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextConstPtr context); DiskSelector(const DiskSelector & from) : disks(from.disks) { } DiskSelectorPtr updateFromConfig( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - const Context & context + ContextPtr context ) const; /// Get disk by name diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 3ce2f909760..a15b6bcf822 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -109,13 +109,13 @@ void registerDiskS3(DiskFactory & factory) auto creator = [](const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - const Context & context) -> DiskPtr { - Poco::File disk{context.getPath() + "disks/" + name}; + ContextConstPtr context) -> DiskPtr { + Poco::File disk{context->getPath() + "disks/" + name}; disk.createDirectories(); S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( - context.getRemoteHostFilter(), - context.getGlobalContext().getSettingsRef().s3_max_redirects); + context->getRemoteHostFilter(), + context->getGlobalContext()->getSettingsRef().s3_max_redirects); S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint"))); if (uri.key.back() != '/') @@ -143,7 +143,7 @@ void registerDiskS3(DiskFactory & factory) config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", false)) ); - String metadata_path = config.getString(config_prefix + ".metadata_path", context.getPath() + "disks/" + name + "/"); + String metadata_path = config.getString(config_prefix + ".metadata_path", context->getPath() + "disks/" + name + "/"); auto s3disk = std::make_shared( name, @@ -152,8 +152,8 @@ void registerDiskS3(DiskFactory & factory) uri.bucket, uri.key, metadata_path, - context.getSettingsRef().s3_min_upload_part_size, - context.getSettingsRef().s3_max_single_part_upload_size, + context->getSettingsRef().s3_min_upload_part_size, + context->getSettingsRef().s3_max_single_part_upload_size, config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getBool(config_prefix + ".send_metadata", false), config.getInt(config_prefix + ".thread_pool_size", 16), @@ -174,7 +174,7 @@ void registerDiskS3(DiskFactory & factory) if (cache_enabled) { - String cache_path = config.getString(config_prefix + ".cache_path", context.getPath() + "disks/" + name + "/cache/"); + String cache_path = config.getString(config_prefix + ".cache_path", context->getPath() + "disks/" + name + "/cache/"); if (metadata_path == cache_path) throw Exception("Metadata and cache path should be different: " + metadata_path, ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index f7f32cf9b6f..6774c5eed88 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -43,16 +43,15 @@ const FormatFactory::Creators & FormatFactory::getCreators(const String & name) throw Exception("Unknown format " + name, ErrorCodes::UNKNOWN_FORMAT); } -FormatSettings getFormatSettings(const Context & context) +FormatSettings getFormatSettings(ContextConstPtr context) { - const auto & settings = context.getSettingsRef(); + const auto & settings = context->getSettingsRef(); return getFormatSettings(context, settings); } template -FormatSettings getFormatSettings(const Context & context, - const Settings & settings) +FormatSettings getFormatSettings(ContextConstPtr context, const Settings & settings) { FormatSettings format_settings; @@ -99,8 +98,8 @@ FormatSettings getFormatSettings(const Context & context, format_settings.regexp.regexp = settings.format_regexp; format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched; format_settings.schema.format_schema = settings.format_schema; - format_settings.schema.format_schema_path = context.getFormatSchemaPath(); - format_settings.schema.is_server = context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER); + format_settings.schema.format_schema_path = context->getFormatSchemaPath(); + format_settings.schema.is_server = context->hasGlobalContext() && (context->getGlobalContext()->getApplicationType() == Context::ApplicationType::SERVER); format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields; format_settings.template_settings.resultset_format = settings.format_template_resultset; format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; @@ -120,26 +119,22 @@ FormatSettings getFormatSettings(const Context & context, { const Poco::URI & avro_schema_registry_url = settings.format_avro_schema_registry_url; if (!avro_schema_registry_url.empty()) - context.getRemoteHostFilter().checkURL(avro_schema_registry_url); + context->getRemoteHostFilter().checkURL(avro_schema_registry_url); } return format_settings; } -template -FormatSettings getFormatSettings(const Context & context, - const FormatFactorySettings & settings); +template FormatSettings getFormatSettings(ContextConstPtr context, const FormatFactorySettings & settings); -template -FormatSettings getFormatSettings(const Context & context, - const Settings & settings); +template FormatSettings getFormatSettings(ContextConstPtr context, const Settings & settings); InputFormatPtr FormatFactory::getInput( const String & name, ReadBuffer & buf, const Block & sample, - const Context & context, + ContextConstPtr context, UInt64 max_block_size, const std::optional & _format_settings) const { @@ -154,7 +149,7 @@ InputFormatPtr FormatFactory::getInput( throw Exception("Format " + name + " is not suitable for input (with processors)", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); } - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); const auto & file_segmentation_engine = getCreators(name).file_segmentation_engine; // Doesn't make sense to use parallel parsing with less than four threads @@ -204,13 +199,17 @@ InputFormatPtr FormatFactory::getInput( return format; } -BlockOutputStreamPtr FormatFactory::getOutputStreamParallelIfPossible(const String & name, - WriteBuffer & buf, const Block & sample, const Context & context, - WriteCallback callback, const std::optional & _format_settings) const +BlockOutputStreamPtr FormatFactory::getOutputStreamParallelIfPossible( + const String & name, + WriteBuffer & buf, + const Block & sample, + ContextConstPtr context, + WriteCallback callback, + const std::optional & _format_settings) const { const auto & output_getter = getCreators(name).output_processor_creator; - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); bool parallel_formatting = settings.output_format_parallel_formatting; if (output_getter && parallel_formatting && getCreators(name).supports_parallel_formatting @@ -237,12 +236,15 @@ BlockOutputStreamPtr FormatFactory::getOutputStreamParallelIfPossible(const Stri } -BlockOutputStreamPtr FormatFactory::getOutputStream(const String & name, - WriteBuffer & buf, const Block & sample, const Context & context, - WriteCallback callback, const std::optional & _format_settings) const +BlockOutputStreamPtr FormatFactory::getOutputStream( + const String & name, + WriteBuffer & buf, + const Block & sample, + ContextConstPtr context, + WriteCallback callback, + const std::optional & _format_settings) const { - auto format_settings = _format_settings - ? *_format_settings : getFormatSettings(context); + auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); if (!getCreators(name).output_processor_creator) { @@ -267,7 +269,7 @@ InputFormatPtr FormatFactory::getInputFormat( const String & name, ReadBuffer & buf, const Block & sample, - const Context & context, + ContextConstPtr context, UInt64 max_block_size, const std::optional & _format_settings) const { @@ -275,13 +277,12 @@ InputFormatPtr FormatFactory::getInputFormat( if (!input_getter) throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); - if (context.hasQueryContext() && settings.log_queries) - context.getQueryContext().addQueryFactoriesInfo(Context::QueryLogFactories::Format, name); + if (context->hasQueryContext() && settings.log_queries) + context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Format, name); - auto format_settings = _format_settings - ? *_format_settings : getFormatSettings(context); + auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); RowInputFormatParams params; params.max_block_size = max_block_size; @@ -289,7 +290,6 @@ InputFormatPtr FormatFactory::getInputFormat( params.allow_errors_ratio = format_settings.input_allow_errors_ratio; params.max_execution_time = settings.max_execution_time; params.timeout_overflow_mode = settings.timeout_overflow_mode; - auto format = input_getter(buf, sample, params, format_settings); /// It's a kludge. Because I cannot remove context from values format. @@ -300,18 +300,20 @@ InputFormatPtr FormatFactory::getInputFormat( } OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( - const String & name, WriteBuffer & buf, const Block & sample, - const Context & context, WriteCallback callback, + const String & name, + WriteBuffer & buf, + const Block & sample, + ContextConstPtr context, + WriteCallback callback, const std::optional & _format_settings) const { const auto & output_getter = getCreators(name).output_processor_creator; if (!output_getter) throw Exception("Format " + name + " is not suitable for output (with processors)", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT); - auto format_settings = _format_settings - ? *_format_settings : getFormatSettings(context); + auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); - const Settings & settings = context.getSettingsRef(); + const Settings & settings = context->getSettingsRef(); if (settings.output_format_parallel_formatting && getCreators(name).supports_parallel_formatting && !settings.output_format_json_array_of_rows) @@ -322,8 +324,8 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( ParallelFormattingOutputFormat::Params builder{buf, sample, formatter_creator, settings.max_threads}; - if (context.hasQueryContext() && settings.log_queries) - context.getQueryContext().addQueryFactoriesInfo(Context::QueryLogFactories::Format, name); + if (context->hasQueryContext() && settings.log_queries) + context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Format, name); return std::make_shared(builder); } @@ -333,16 +335,19 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( OutputFormatPtr FormatFactory::getOutputFormat( - const String & name, WriteBuffer & buf, const Block & sample, - const Context & context, WriteCallback callback, + const String & name, + WriteBuffer & buf, + const Block & sample, + ContextConstPtr context, + WriteCallback callback, const std::optional & _format_settings) const { const auto & output_getter = getCreators(name).output_processor_creator; if (!output_getter) throw Exception("Format " + name + " is not suitable for output (with processors)", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT); - if (context.hasQueryContext() && context.getSettingsRef().log_queries) - context.getQueryContext().addQueryFactoriesInfo(Context::QueryLogFactories::Format, name); + if (context->hasQueryContext() && context->getSettingsRef().log_queries) + context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Format, name); RowOutputFormatParams params; params.callback = std::move(callback); @@ -412,11 +417,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; diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 4fa7e9a0c01..d5e4b1b0341 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -1,21 +1,22 @@ #pragma once -#include #include #include #include +#include #include +#include + +#include #include #include #include -#include namespace DB { class Block; -class Context; struct Settings; struct FormatFactorySettings; @@ -34,11 +35,10 @@ struct RowOutputFormatParams; using InputFormatPtr = std::shared_ptr; using OutputFormatPtr = std::shared_ptr; -FormatSettings getFormatSettings(const Context & context); +FormatSettings getFormatSettings(ContextConstPtr context); template -FormatSettings getFormatSettings(const Context & context, - const T & settings); +FormatSettings getFormatSettings(ContextConstPtr context, const T & settings); /** Allows to create an IBlockInputStream or IBlockOutputStream by the name of the format. * Note: format and compression are independent things. @@ -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; @@ -112,38 +113,52 @@ public: const String & name, ReadBuffer & buf, const Block & sample, - const Context & context, + ContextConstPtr context, UInt64 max_block_size, const std::optional & format_settings = std::nullopt) const; /// Checks all preconditions. Returns ordinary stream if parallel formatting cannot be done. /// Currently used only in Client. Don't use it something else! Better look at getOutputFormatParallelIfPossible. - BlockOutputStreamPtr getOutputStreamParallelIfPossible(const String & name, WriteBuffer & buf, - const Block & sample, const Context & context, WriteCallback callback = {}, + BlockOutputStreamPtr getOutputStreamParallelIfPossible( + const String & name, + WriteBuffer & buf, + const Block & sample, + ContextConstPtr context, + WriteCallback callback = {}, const std::optional & format_settings = std::nullopt) const; /// Currently used only in Client. Don't use it something else! Better look at getOutputFormat. - BlockOutputStreamPtr getOutputStream(const String & name, WriteBuffer & buf, - const Block & sample, const Context & context, WriteCallback callback = {}, + BlockOutputStreamPtr getOutputStream( + const String & name, + WriteBuffer & buf, + const Block & sample, + ContextConstPtr context, + WriteCallback callback = {}, const std::optional & format_settings = std::nullopt) const; InputFormatPtr getInputFormat( const String & name, ReadBuffer & buf, const Block & sample, - const Context & context, + ContextConstPtr context, UInt64 max_block_size, const std::optional & format_settings = std::nullopt) const; /// Checks all preconditions. Returns ordinary format if parallel formatting cannot be done. OutputFormatPtr getOutputFormatParallelIfPossible( - const String & name, WriteBuffer & buf, const Block & sample, - const Context & context, WriteCallback callback = {}, + const String & name, + WriteBuffer & buf, + const Block & sample, + ContextConstPtr context, + WriteCallback callback = {}, const std::optional & format_settings = std::nullopt) const; OutputFormatPtr getOutputFormat( - const String & name, WriteBuffer & buf, const Block & sample, - const Context & context, WriteCallback callback = {}, + const String & name, + WriteBuffer & buf, + const Block & sample, + ContextConstPtr context, + WriteCallback callback = {}, const std::optional & format_settings = std::nullopt) const; /// Register format by its name. @@ -155,6 +170,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 { diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index da2a175e49f..e2620c71fbb 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -1,26 +1,29 @@ #if !defined(ARCADIA_BUILD) -# include "config_core.h" +#include "config_core.h" #endif #if USE_MYSQL -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include "MySQLBlockInputStream.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "MySQLBlockInputStream.h" + namespace DB { + namespace ErrorCodes { extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; @@ -36,6 +39,7 @@ MySQLBlockInputStream::Connection::Connection( { } +/// Used in MaterializeMySQL and in doInvalidateQuery for dictionary source. MySQLBlockInputStream::MySQLBlockInputStream( const mysqlxx::PoolWithFailover::Entry & entry, const std::string & query_str, @@ -43,7 +47,8 @@ MySQLBlockInputStream::MySQLBlockInputStream( const UInt64 max_block_size_, const bool auto_close_, const bool fetch_by_name_) - : connection{std::make_unique(entry, query_str)} + : log(&Poco::Logger::get("MySQLBlockInputStream")) + , connection{std::make_unique(entry, query_str)} , max_block_size{max_block_size_} , auto_close{auto_close_} , fetch_by_name(fetch_by_name_) @@ -52,6 +57,62 @@ MySQLBlockInputStream::MySQLBlockInputStream( initPositionMappingFromQueryResultStructure(); } +/// For descendant MySQLWithFailoverBlockInputStream +MySQLBlockInputStream::MySQLBlockInputStream( + const Block & sample_block_, + UInt64 max_block_size_, + bool auto_close_, + bool fetch_by_name_) + : log(&Poco::Logger::get("MySQLBlockInputStream")) + , max_block_size(max_block_size_) + , auto_close(auto_close_) + , fetch_by_name(fetch_by_name_) +{ + description.init(sample_block_); +} + +/// Used by MySQL storage / table function and dictionary source. +MySQLWithFailoverBlockInputStream::MySQLWithFailoverBlockInputStream( + mysqlxx::PoolWithFailoverPtr pool_, + const std::string & query_str_, + const Block & sample_block_, + const UInt64 max_block_size_, + const bool auto_close_, + const bool fetch_by_name_, + const size_t max_tries_) + : MySQLBlockInputStream(sample_block_, max_block_size_, auto_close_, fetch_by_name_) + , pool(pool_) + , query_str(query_str_) + , max_tries(max_tries_) +{ +} + +void MySQLWithFailoverBlockInputStream::readPrefix() +{ + size_t count_connect_attempts = 0; + + /// For recovering from "Lost connection to MySQL server during query" errors + while (true) + { + try + { + connection = std::make_unique(pool->get(), query_str); + break; + } + catch (const mysqlxx::ConnectionLost & ecl) /// There are two retriable failures: CR_SERVER_GONE_ERROR, CR_SERVER_LOST + { + LOG_WARNING(log, "Failed connection ({}/{}). Trying to reconnect... (Info: {})", count_connect_attempts, max_tries, ecl.displayText()); + } + + if (++count_connect_attempts > max_tries) + { + LOG_ERROR(log, "Failed to create connection to MySQL. ({}/{})", count_connect_attempts, max_tries); + throw; + } + } + + initPositionMappingFromQueryResultStructure(); +} namespace { @@ -139,6 +200,7 @@ Block MySQLBlockInputStream::readImpl() { if (auto_close) connection->entry.disconnect(); + return {}; } @@ -191,18 +253,6 @@ Block MySQLBlockInputStream::readImpl() return description.sample_block.cloneWithColumns(std::move(columns)); } -MySQLBlockInputStream::MySQLBlockInputStream( - const Block & sample_block_, - UInt64 max_block_size_, - bool auto_close_, - bool fetch_by_name_) - : max_block_size(max_block_size_) - , auto_close(auto_close_) - , fetch_by_name(fetch_by_name_) -{ - description.init(sample_block_); -} - void MySQLBlockInputStream::initPositionMappingFromQueryResultStructure() { position_mapping.resize(description.sample_block.columns()); @@ -250,25 +300,6 @@ void MySQLBlockInputStream::initPositionMappingFromQueryResultStructure() } } -MySQLLazyBlockInputStream::MySQLLazyBlockInputStream( - mysqlxx::Pool & pool_, - const std::string & query_str_, - const Block & sample_block_, - const UInt64 max_block_size_, - const bool auto_close_, - const bool fetch_by_name_) - : MySQLBlockInputStream(sample_block_, max_block_size_, auto_close_, fetch_by_name_) - , pool(pool_) - , query_str(query_str_) -{ -} - -void MySQLLazyBlockInputStream::readPrefix() -{ - connection = std::make_unique(pool.get(), query_str); - initPositionMappingFromQueryResultStructure(); -} - } #endif diff --git a/src/Formats/MySQLBlockInputStream.h b/src/Formats/MySQLBlockInputStream.h index 269b630fcc7..86380c1597a 100644 --- a/src/Formats/MySQLBlockInputStream.h +++ b/src/Formats/MySQLBlockInputStream.h @@ -41,6 +41,7 @@ protected: mysqlxx::UseQueryResult result; }; + Poco::Logger * log; std::unique_ptr connection; const UInt64 max_block_size; @@ -52,22 +53,27 @@ protected: /// Like MySQLBlockInputStream, but allocates connection only when reading is starting. /// It allows to create a lot of stream objects without occupation of all connection pool. -class MySQLLazyBlockInputStream final : public MySQLBlockInputStream +/// Also makes attempts to reconnect in case of connection failures. +class MySQLWithFailoverBlockInputStream final : public MySQLBlockInputStream { public: - MySQLLazyBlockInputStream( - mysqlxx::Pool & pool_, + static constexpr inline auto MAX_TRIES_MYSQL_CONNECT = 5; + + MySQLWithFailoverBlockInputStream( + mysqlxx::PoolWithFailoverPtr pool_, const std::string & query_str_, const Block & sample_block_, const UInt64 max_block_size_, const bool auto_close_ = false, - const bool fetch_by_name_ = false); + const bool fetch_by_name_ = false, + const size_t max_tries_ = MAX_TRIES_MYSQL_CONNECT); private: void readPrefix() override; - mysqlxx::Pool & pool; + mysqlxx::PoolWithFailoverPtr pool; std::string query_str; + size_t max_tries; }; } diff --git a/src/Functions/FunctionBase64Conversion.h b/src/Functions/FunctionBase64Conversion.h index 4887e9fe8cc..d060b86c54b 100644 --- a/src/Functions/FunctionBase64Conversion.h +++ b/src/Functions/FunctionBase64Conversion.h @@ -61,7 +61,7 @@ class FunctionBase64Conversion : public IFunction public: static constexpr auto name = Func::name; - static FunctionPtr create(const Context &) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index bb85ae32622..ef9d05b4b5e 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -512,7 +512,7 @@ class FunctionBinaryArithmetic : public IFunction static constexpr const bool is_multiply = IsOperation::multiply; static constexpr const bool is_division = IsOperation::division; - const Context & context; + ContextPtr context; bool check_decimal_overflow = true; template @@ -593,7 +593,7 @@ class FunctionBinaryArithmetic : public IFunction } static FunctionOverloadResolverPtr - getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, const Context & context) + getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { bool first_is_date_or_datetime = isDateOrDateTime(type0); bool second_is_date_or_datetime = isDateOrDateTime(type1); @@ -939,9 +939,9 @@ class FunctionBinaryArithmetic : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context & context) { return std::make_shared(context); } + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionBinaryArithmetic(const Context & context_) + explicit FunctionBinaryArithmetic(ContextPtr context_) : context(context_), check_decimal_overflow(decimalCheckArithmeticOverflow(context)) {} @@ -955,7 +955,7 @@ public: return getReturnTypeImplStatic(arguments, context); } - static DataTypePtr getReturnTypeImplStatic(const DataTypes & arguments, const Context & context) + static DataTypePtr getReturnTypeImplStatic(const DataTypes & arguments, ContextPtr context) { /// Special case when multiply aggregate function state if (isAggregateMultiply(arguments[0], arguments[1])) @@ -1367,7 +1367,7 @@ public: const ColumnWithTypeAndName & left_, const ColumnWithTypeAndName & right_, const DataTypePtr & return_type_, - const Context & context) + ContextPtr context) { return std::make_shared(left_, right_, return_type_, context); } @@ -1376,7 +1376,7 @@ public: const ColumnWithTypeAndName & left_, const ColumnWithTypeAndName & right_, const DataTypePtr & return_type_, - const Context & context_) + ContextPtr context_) : Base(context_), left(left_), right(right_), return_type(return_type_) { } @@ -1530,12 +1530,12 @@ class BinaryArithmeticOverloadResolver : public IFunctionOverloadResolverImpl { public: static constexpr auto name = Name::name; - static FunctionOverloadResolverImplPtr create(const Context & context) + static FunctionOverloadResolverImplPtr create(ContextPtr context) { return std::make_unique(context); } - explicit BinaryArithmeticOverloadResolver(const Context & context_) : context(context_) {} + explicit BinaryArithmeticOverloadResolver(ContextPtr context_) : context(context_) {} String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } @@ -1571,6 +1571,6 @@ public: } private: - const Context & context; + ContextPtr context; }; } diff --git a/src/Functions/FunctionBitTestMany.h b/src/Functions/FunctionBitTestMany.h index 6d527c66390..a2be56c42b0 100644 --- a/src/Functions/FunctionBitTestMany.h +++ b/src/Functions/FunctionBitTestMany.h @@ -24,7 +24,7 @@ struct FunctionBitTestMany : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionCustomWeekToSomething.h b/src/Functions/FunctionCustomWeekToSomething.h index 5634ea11584..4050bb512bd 100644 --- a/src/Functions/FunctionCustomWeekToSomething.h +++ b/src/Functions/FunctionCustomWeekToSomething.h @@ -24,7 +24,7 @@ class FunctionCustomWeekToSomething : public IFunction { public: static constexpr auto name = Transform::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionDateOrDateTimeAddInterval.h b/src/Functions/FunctionDateOrDateTimeAddInterval.h index 95b9e25ead1..8a9d454028c 100644 --- a/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -352,7 +352,7 @@ class FunctionDateOrDateTimeAddInterval : public IFunction { public: static constexpr auto name = Transform::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/FunctionDateOrDateTimeToSomething.h b/src/Functions/FunctionDateOrDateTimeToSomething.h index abe859e2f29..b9d0a7f45fe 100644 --- a/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -24,7 +24,7 @@ class FunctionDateOrDateTimeToSomething : public IFunction { public: static constexpr auto name = Transform::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/FunctionFQDN.cpp b/src/Functions/FunctionFQDN.cpp index 7b3b89eb511..9cad9011cd4 100644 --- a/src/Functions/FunctionFQDN.cpp +++ b/src/Functions/FunctionFQDN.cpp @@ -12,7 +12,7 @@ class FunctionFQDN : public IFunction { public: static constexpr auto name = "FQDN"; - static FunctionPtr create(const Context &) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index e13f310de09..35ac9ab647b 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -52,7 +52,7 @@ void FunctionFactory::registerFunction(const FunctionOverloadResolverImplPtr FunctionFactory::getImpl( const std::string & name, - const Context & context) const + ContextPtr context) const { auto res = tryGetImpl(name, context); if (!res) @@ -82,14 +82,14 @@ std::vector FunctionFactory::getAllNames() const FunctionOverloadResolverPtr FunctionFactory::get( const std::string & name, - const Context & context) const + ContextPtr context) const { return std::make_shared(getImpl(name, context)); } FunctionOverloadResolverImplPtr FunctionFactory::tryGetImpl( const std::string & name_param, - const Context & context) const + ContextPtr context) const { String name = getAliasToOrName(name_param); FunctionOverloadResolverImplPtr res; @@ -110,7 +110,7 @@ FunctionOverloadResolverImplPtr FunctionFactory::tryGetImpl( if (CurrentThread::isInitialized()) { - const auto * query_context = CurrentThread::get().getQueryContext(); + auto query_context = CurrentThread::get().getQueryContext(); if (query_context && query_context->getSettingsRef().log_queries) query_context->addQueryFactoriesInfo(Context::QueryLogFactories::Function, name); } @@ -120,7 +120,7 @@ FunctionOverloadResolverImplPtr FunctionFactory::tryGetImpl( FunctionOverloadResolverPtr FunctionFactory::tryGet( const std::string & name, - const Context & context) const + ContextPtr context) const { auto impl = tryGetImpl(name, context); return impl ? std::make_shared(std::move(impl)) diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index 7990e78daf8..96238a88420 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -12,14 +13,12 @@ namespace DB { -class Context; - - /** Creates function by name. * Function could use for initialization (take ownership of shared_ptr, for example) * some dictionaries from Context. */ -class FunctionFactory : private boost::noncopyable, public IFactoryWithAliases> +class FunctionFactory : private boost::noncopyable, + public IFactoryWithAliases> { public: static FunctionFactory & instance(); @@ -43,14 +42,14 @@ public: std::vector getAllNames() const; /// Throws an exception if not found. - FunctionOverloadResolverPtr get(const std::string & name, const Context & context) const; + FunctionOverloadResolverPtr get(const std::string & name, ContextPtr context) const; /// Returns nullptr if not found. - FunctionOverloadResolverPtr tryGet(const std::string & name, const Context & context) const; + FunctionOverloadResolverPtr tryGet(const std::string & name, ContextPtr context) const; /// The same methods to get developer interface implementation. - FunctionOverloadResolverImplPtr getImpl(const std::string & name, const Context & context) const; - FunctionOverloadResolverImplPtr tryGetImpl(const std::string & name, const Context & context) const; + FunctionOverloadResolverImplPtr getImpl(const std::string & name, ContextPtr context) const; + FunctionOverloadResolverImplPtr tryGetImpl(const std::string & name, ContextPtr context) const; private: using Functions = std::unordered_map; @@ -59,7 +58,7 @@ private: Functions case_insensitive_functions; template - static FunctionOverloadResolverImplPtr createDefaultFunction(const Context & context) + static FunctionOverloadResolverImplPtr createDefaultFunction(ContextPtr context) { return std::make_unique(Function::create(context)); } diff --git a/src/Functions/FunctionFile.cpp b/src/Functions/FunctionFile.cpp index f477f6123c3..9247152367b 100644 --- a/src/Functions/FunctionFile.cpp +++ b/src/Functions/FunctionFile.cpp @@ -21,12 +21,12 @@ namespace ErrorCodes } /// A function to read file as a string. -class FunctionFile : public IFunction +class FunctionFile : public IFunction, WithContext { public: static constexpr auto name = "file"; - static FunctionPtr create(const Context &context) { return std::make_shared(context); } - explicit FunctionFile(const Context &context_) : context(context_) {} + static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } + explicit FunctionFile(ContextPtr context_) : WithContext(context_) {} String getName() const override { return name; } @@ -68,7 +68,7 @@ public: { const char * filename = reinterpret_cast(&chars[source_offset]); - const String user_files_path = context.getUserFilesPath(); + const String user_files_path = getContext()->getUserFilesPath(); String user_files_absolute_path = Poco::Path(user_files_path).makeAbsolute().makeDirectory().toString(); Poco::Path poco_filepath = Poco::Path(filename); if (poco_filepath.isRelative()) @@ -113,7 +113,7 @@ private: void checkReadIsAllowedOrThrow(const std::string & user_files_absolute_path, const std::string & file_absolute_path) const { // If run in Local mode, no need for path checking. - if (context.getApplicationType() != Context::ApplicationType::LOCAL) + if (getContext()->getApplicationType() != Context::ApplicationType::LOCAL) if (file_absolute_path.find(user_files_absolute_path) != 0) throw Exception("File is not inside " + user_files_absolute_path, ErrorCodes::DATABASE_ACCESS_DENIED); @@ -121,8 +121,6 @@ private: if (path_poco_file.exists() && path_poco_file.isDirectory()) throw Exception("File can't be a directory", ErrorCodes::INCORRECT_FILE_NAME); } - - const Context & context; }; diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index 49f155c687b..2ad9985d44c 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -36,7 +36,7 @@ ExecutableFunctionImplPtr FunctionJoinGet::prepare(const ColumnsWithTyp } static std::pair, String> -getJoin(const ColumnsWithTypeAndName & arguments, const Context & context) +getJoin(const ColumnsWithTypeAndName & arguments, ContextPtr context) { String join_name; if (const auto * name_col = checkAndGetColumnConst(arguments[0].column.get())) @@ -52,7 +52,7 @@ getJoin(const ColumnsWithTypeAndName & arguments, const Context & context) String database_name; if (dot == String::npos) { - database_name = context.getCurrentDatabase(); + database_name = context->getCurrentDatabase(); dot = 0; } else @@ -88,12 +88,12 @@ FunctionBaseImplPtr JoinGetOverloadResolver::build(const ColumnsWithTyp "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be greater or equal to 3", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - auto [storage_join, attr_name] = getJoin(arguments, context); + auto [storage_join, attr_name] = getJoin(arguments, getContext()); DataTypes data_types(arguments.size() - 2); for (size_t i = 2; i < arguments.size(); ++i) data_types[i - 2] = arguments[i].type; auto return_type = storage_join->joinGetCheckAndGetReturnType(data_types, attr_name, or_null); - auto table_lock = storage_join->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); + auto table_lock = storage_join->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); return std::make_unique>(table_lock, storage_join, attr_name, data_types, return_type); } diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index f88b58e961a..016ef15f2d5 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -1,5 +1,7 @@ #pragma once + #include +#include #include #include #include @@ -7,7 +9,6 @@ namespace DB { -class Context; class HashJoin; class StorageJoin; using StorageJoinPtr = std::shared_ptr; @@ -73,13 +74,13 @@ private: }; template -class JoinGetOverloadResolver final : public IFunctionOverloadResolverImpl +class JoinGetOverloadResolver final : public IFunctionOverloadResolverImpl, WithContext { public: static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; - static FunctionOverloadResolverImplPtr create(const Context & context) { return std::make_unique(context); } + static FunctionOverloadResolverImplPtr create(ContextPtr context_) { return std::make_unique(context_); } - explicit JoinGetOverloadResolver(const Context & context_) : context(context_) {} + explicit JoinGetOverloadResolver(ContextPtr context_) : WithContext(context_) {} String getName() const override { return name; } @@ -92,9 +93,6 @@ public: bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } - -private: - const Context & context; }; } diff --git a/src/Functions/FunctionMathBinaryFloat64.h b/src/Functions/FunctionMathBinaryFloat64.h index 0a0688dc75c..00f6909f65c 100644 --- a/src/Functions/FunctionMathBinaryFloat64.h +++ b/src/Functions/FunctionMathBinaryFloat64.h @@ -28,7 +28,7 @@ class FunctionMathBinaryFloat64 : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } static_assert(Impl::rows_per_iteration > 0, "Impl must process at least one row per iteration"); bool useDefaultImplementationForConstants() const override { return true; } diff --git a/src/Functions/FunctionMathConstFloat64.h b/src/Functions/FunctionMathConstFloat64.h index f03f469bc35..04ade5b4637 100644 --- a/src/Functions/FunctionMathConstFloat64.h +++ b/src/Functions/FunctionMathConstFloat64.h @@ -13,7 +13,7 @@ class FunctionMathConstFloat64 : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: String getName() const override { return name; } diff --git a/src/Functions/FunctionMathUnary.h b/src/Functions/FunctionMathUnary.h index 49b0428811a..e24d19e5a9b 100644 --- a/src/Functions/FunctionMathUnary.h +++ b/src/Functions/FunctionMathUnary.h @@ -35,7 +35,7 @@ class FunctionMathUnary : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: String getName() const override { return name; } diff --git a/src/Functions/FunctionNumericPredicate.h b/src/Functions/FunctionNumericPredicate.h index 72a17adac4c..9bb0f700fe0 100644 --- a/src/Functions/FunctionNumericPredicate.h +++ b/src/Functions/FunctionNumericPredicate.h @@ -1,8 +1,10 @@ #pragma once + #include #include #include #include +#include #include @@ -21,7 +23,7 @@ class FunctionNumericPredicate : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 2899bc259d5..87c07c5efd2 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -142,7 +142,7 @@ template class FunctionStartsEndsWith : public TargetSpecific::Default::FunctionStartsEndsWith { public: - explicit FunctionStartsEndsWith(const Context & context) : selector(context) + explicit FunctionStartsEndsWith(ContextPtr context) : selector(context) { selector.registerImplementation>(); @@ -164,7 +164,7 @@ public: return selector.selectAndExecute(arguments, result_type, input_rows_count); } - static FunctionPtr create(const Context & context) + static FunctionPtr create(ContextPtr context) { return std::make_shared>(context); } diff --git a/src/Functions/FunctionStringOrArrayToT.h b/src/Functions/FunctionStringOrArrayToT.h index e00acc006cf..40ce62a5666 100644 --- a/src/Functions/FunctionStringOrArrayToT.h +++ b/src/Functions/FunctionStringOrArrayToT.h @@ -25,7 +25,7 @@ class FunctionStringOrArrayToT : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/FunctionStringReplace.h b/src/Functions/FunctionStringReplace.h index bd8edbf9202..2e99f58531a 100644 --- a/src/Functions/FunctionStringReplace.h +++ b/src/Functions/FunctionStringReplace.h @@ -23,7 +23,7 @@ class FunctionStringReplace : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionStringToString.h b/src/Functions/FunctionStringToString.h index 4123b41c547..36009dc30c0 100644 --- a/src/Functions/FunctionStringToString.h +++ b/src/Functions/FunctionStringToString.h @@ -1,9 +1,11 @@ #pragma once + #include #include #include #include #include +#include namespace DB @@ -21,7 +23,7 @@ class FunctionStringToString : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) + static FunctionPtr create(ContextPtr) { return std::make_shared(); } diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index 06469a83e3e..9145c4a2ec5 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -110,7 +110,7 @@ class FunctionUnaryArithmetic : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { diff --git a/src/Functions/FunctionUnixTimestamp64.h b/src/Functions/FunctionUnixTimestamp64.h index 20e225990bd..18f9c7a8b02 100644 --- a/src/Functions/FunctionUnixTimestamp64.h +++ b/src/Functions/FunctionUnixTimestamp64.h @@ -33,7 +33,7 @@ public: static_assert(std::is_same_v || std::is_same_v); - static auto create(const Context &) + static auto create(ContextPtr) { return std::make_shared>(); } diff --git a/src/Functions/FunctionsAES.h b/src/Functions/FunctionsAES.h index 8af4a27ecc9..b76b454fd77 100644 --- a/src/Functions/FunctionsAES.h +++ b/src/Functions/FunctionsAES.h @@ -141,7 +141,7 @@ class FunctionEncrypt : public IFunction public: static constexpr OpenSSLDetails::CompatibilityMode compatibility_mode = Impl::compatibility_mode; static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: using CipherMode = OpenSSLDetails::CipherMode; @@ -416,7 +416,7 @@ class FunctionDecrypt : public IFunction public: static constexpr OpenSSLDetails::CompatibilityMode compatibility_mode = Impl::compatibility_mode; static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: using CipherMode = OpenSSLDetails::CipherMode; diff --git a/src/Functions/FunctionsBitmap.h b/src/Functions/FunctionsBitmap.h index a79365ec16c..aa4a6b50ea3 100644 --- a/src/Functions/FunctionsBitmap.h +++ b/src/Functions/FunctionsBitmap.h @@ -93,7 +93,7 @@ class FunctionBitmapBuildImpl : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -221,7 +221,7 @@ class FunctionBitmapToArrayImpl : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -311,7 +311,7 @@ class FunctionBitmapSubset : public IFunction public: static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared>(); } + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } String getName() const override { return name; } @@ -469,7 +469,7 @@ class FunctionBitmapTransform : public IFunction public: static constexpr auto name = "bitmapTransform"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -635,7 +635,7 @@ class FunctionBitmapSelfCardinalityImpl : public IFunction public: static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared>(); } + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } String getName() const override { return name; } @@ -807,7 +807,7 @@ class FunctionBitmapContains : public IFunction public: static constexpr auto name = "bitmapContains"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -911,7 +911,7 @@ class FunctionBitmapCardinality : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -1054,7 +1054,7 @@ class FunctionBitmap : public IFunction public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionsCoding.h b/src/Functions/FunctionsCoding.h index ee15561f133..933d4f01b92 100644 --- a/src/Functions/FunctionsCoding.h +++ b/src/Functions/FunctionsCoding.h @@ -1,28 +1,29 @@ #pragma once -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include +#include +#include #include #include #include -#include +#include +#include +#include #include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -68,7 +69,7 @@ class FunctionIPv6NumToString : public IFunction { public: static constexpr auto name = "IPv6NumToString"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -138,7 +139,7 @@ class FunctionCutIPv6 : public IFunction { public: static constexpr auto name = "cutIPv6"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -261,7 +262,7 @@ class FunctionIPv6StringToNum : public IFunction { public: static constexpr auto name = "IPv6StringToNum"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } static inline bool tryParseIPv4(const char * pos) { @@ -339,7 +340,7 @@ class FunctionIPv4NumToString : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared>(); } + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } String getName() const override { @@ -400,7 +401,7 @@ class FunctionIPv4StringToNum : public IFunction { public: static constexpr auto name = "IPv4StringToNum"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -463,7 +464,7 @@ class FunctionIPv4ToIPv6 : public IFunction { public: static constexpr auto name = "IPv4ToIPv6"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -518,7 +519,7 @@ class FunctionToIPv4 : public FunctionIPv4StringToNum { public: static constexpr auto name = "toIPv4"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -541,7 +542,7 @@ class FunctionToIPv6 : public FunctionIPv6StringToNum { public: static constexpr auto name = "toIPv6"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -559,7 +560,7 @@ class FunctionMACNumToString : public IFunction { public: static constexpr auto name = "MACNumToString"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -689,7 +690,7 @@ class FunctionMACStringTo : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared>(); } + static FunctionPtr create(ContextPtr) { return std::make_shared>(); } String getName() const override { @@ -752,7 +753,7 @@ class FunctionUUIDNumToString : public IFunction public: static constexpr auto name = "UUIDNumToString"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -850,7 +851,7 @@ private: public: static constexpr auto name = "UUIDStringToNum"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -954,7 +955,7 @@ class FunctionHex : public IFunction { public: static constexpr auto name = "hex"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -1237,7 +1238,7 @@ class FunctionUnhex : public IFunction { public: static constexpr auto name = "unhex"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -1326,7 +1327,7 @@ class FunctionChar : public IFunction { public: static constexpr auto name = "char"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -1421,7 +1422,7 @@ class FunctionBitmaskToArray : public IFunction { public: static constexpr auto name = "bitmaskToArray"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -1507,7 +1508,7 @@ class FunctionToStringCutToZero : public IFunction { public: static constexpr auto name = "toStringCutToZero"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -1672,7 +1673,7 @@ private: public: static constexpr auto name = "IPv6CIDRToRange"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } @@ -1786,7 +1787,7 @@ private: public: static constexpr auto name = "IPv4CIDRToRange"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } @@ -1867,7 +1868,7 @@ class FunctionIsIPv4String : public FunctionIPv4StringToNum public: static constexpr auto name = "isIPv4String"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -1913,7 +1914,7 @@ class FunctionIsIPv6String : public FunctionIPv6StringToNum public: static constexpr auto name = "isIPv6String"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index bbfa1ea20d8..025b7c67b21 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -552,22 +552,18 @@ struct NameLessOrEquals { static constexpr auto name = "lessOrEquals"; }; struct NameGreaterOrEquals { static constexpr auto name = "greaterOrEquals"; }; -template < - template class Op, - typename Name> +template