Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into support-apple-m1

This commit is contained in:
changvvb 2021-04-08 18:11:16 +08:00
commit b070090ca7
265 changed files with 7112 additions and 2293 deletions

View File

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

View File

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

View File

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

View File

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

238
base/bridge/IBridge.cpp Normal file
View File

@ -0,0 +1,238 @@
#include "IBridge.h"
#include <IO/ReadHelpers.h>
#include <boost/program_options.hpp>
#include <Poco/Net/NetException.h>
#include <Poco/Util/HelpFormatter.h>
#include <Common/StringUtils/StringUtils.h>
#include <Formats/registerFormats.h>
#include <common/logger_useful.h>
#include <Common/SensitiveDataMasker.h>
#include <Server/HTTP/HTTPServer.h>
#if USE_ODBC
# include <Poco/Data/ODBC/Connector.h>
#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 <listen_host> element of configuration file. Example: <listen_host>0.0.0.0</listen_host>", 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 <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<decltype(*this)>;
options.addOption(
Poco::Util::Option("help", "", "produce this help message").binding("help").callback(Poco::Util::OptionCallback<Me>(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<std::string> & /*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();
Context context(Context::createGlobal(shared_context.get()));
context.makeGlobalContext();
if (config().has("query_masking_rules"))
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(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;
}
}

50
base/bridge/IBridge.h Normal file
View File

@ -0,0 +1,50 @@
#pragma once
#include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <Poco/Util/ServerApplication.h>
#include <Poco/Logger.h>
#include <daemon/BaseDaemon.h>
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<HTTPRequestHandlerFactory>;
void initialize(Application & self) override;
void uninitialize() override;
int main(const std::vector<std::string> & args) override;
virtual const std::string bridgeName() const = 0;
virtual HandlerFactoryPtr getHandlerFactoryPtr(Context & 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;
};
}

View File

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

View File

@ -2,7 +2,6 @@
#include <ctime>
#include <random>
#include <thread>
#include <mysqlxx/PoolWithFailover.h>
@ -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<Pool>(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)
{

View File

@ -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<Pool>;
using Replicas = std::vector<PoolPtr>;
/// [priority][index] -> replica.
/// [priority][index] -> replica. Highest priority is 0.
using ReplicasByPriority = std::map<int, Replicas>;
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<std::pair<std::string, uint16_t>>;
/**
* 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<PoolWithFailover>;
}

View File

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

View File

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

2
contrib/NuRaft vendored

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -11,10 +11,3 @@ services:
default:
aliases:
- postgre-sql.local
postgres2:
image: postgres
restart: always
environment:
POSTGRES_PASSWORD: mysecretpassword
ports:
- 5441:5432

View File

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

View File

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

View File

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

View File

@ -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/) <!--hide-->

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -15,3 +15,4 @@
#cmakedefine01 ENABLE_CLICKHOUSE_GIT_IMPORT
#cmakedefine01 ENABLE_CLICKHOUSE_INSTALL
#cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE
#cmakedefine01 ENABLE_CLICKHOUSE_LIBRARY_BRIDGE

View File

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

View File

@ -0,0 +1,26 @@
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
clickhouse_parsers
bridge
)
set_target_properties(clickhouse-library-bridge PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..)
install(TARGETS clickhouse-library-bridge RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)

View File

@ -0,0 +1,23 @@
#include "HandlerFactory.h"
#include <Poco/Net/HTTPServerRequest.h>
#include <Server/HTTP/HTMLForm.h>
#include "Handlers.h"
namespace DB
{
std::unique_ptr<HTTPRequestHandler> 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<PingHandler>(keep_alive_timeout);
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST)
return std::make_unique<LibraryRequestHandler>(keep_alive_timeout, context);
return nullptr;
}
}

View File

@ -0,0 +1,38 @@
#pragma once
#include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <common/logger_useful.h>
namespace DB
{
class SharedLibraryHandler;
using SharedLibraryHandlerPtr = std::shared_ptr<SharedLibraryHandler>;
/// Factory for '/ping', '/' handlers.
class LibraryBridgeHandlerFactory : public HTTPRequestHandlerFactory
{
public:
LibraryBridgeHandlerFactory(
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_)
{
}
std::unique_ptr<HTTPRequestHandler> createRequestHandler(const HTTPServerRequest & request) override;
private:
Poco::Logger * log;
std::string name;
size_t keep_alive_timeout;
Context & context;
};
}

View File

@ -0,0 +1,288 @@
#include "Handlers.h"
#include "SharedLibraryHandlerFactory.h"
#include <DataStreams/copyData.h>
#include <Formats/FormatFactory.h>
#include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
#include <Poco/Net/HTMLForm.h>
#include <Poco/ThreadPool.h>
#include <Processors/Formats/InputStreamFromInputFormat.h>
#include <Server/HTTP/HTMLForm.h>
#include <IO/ReadBufferFromString.h>
namespace DB
{
namespace
{
std::shared_ptr<Block> parseColumns(std::string && column_string)
{
auto sample_block = std::make_shared<Block>();
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<uint64_t> parseIdsFromBinary(const std::string & ids_string)
{
ReadBufferFromString buf(ids_string);
std::vector<uint64_t> ids;
readVectorBinary(ids, buf);
return ids;
}
std::vector<std::string> parseNamesFromBinary(const std::string & names_string)
{
ReadBufferFromString buf(names_string);
std::vector<std::string> 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<std::string> 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<std::string> 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<Block> 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, context, DEFAULT_BLOCK_SIZE);
auto reader = std::make_shared<InputStreamFromInputFormat>(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, context);
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<uint64_t> 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, context);
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<Block> 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, context, DEFAULT_BLOCK_SIZE);
auto reader = std::make_shared<InputStreamFromInputFormat>(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, context);
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");
}
}
}

View File

@ -0,0 +1,60 @@
#pragma once
#include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandler.h>
#include <common/logger_useful.h>
#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
{
public:
LibraryRequestHandler(
size_t keep_alive_timeout_,
Context & context_)
: log(&Poco::Logger::get("LibraryRequestHandler"))
, keep_alive_timeout(keep_alive_timeout_)
, context(context_)
{
}
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;
Context & context;
};
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;
};
}

View File

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

View File

@ -0,0 +1,26 @@
#pragma once
#include <Interpreters/Context.h>
#include <bridge/IBridge.h>
#include "HandlerFactory.h"
namespace DB
{
class LibraryBridge : public IBridge
{
protected:
const std::string bridgeName() const override
{
return "LibraryBridge";
}
HandlerFactoryPtr getHandlerFactoryPtr(Context & context) const override
{
return std::make_shared<LibraryBridgeHandlerFactory>("LibraryRequestHandlerFactory-factory", keep_alive_timeout, context);
}
};
}

View File

@ -0,0 +1,43 @@
#pragma once
#include <Common/StringUtils/StringUtils.h>
#include <Dictionaries/LibraryDictionarySourceExternal.h>
#include <Core/Block.h>
#include <ext/bit_cast.h>
#include <ext/range.h>
namespace DB
{
class CStringsHolder
{
public:
using Container = std::vector<std::string>;
explicit CStringsHolder(const Container & strings_pass)
{
strings_holder = strings_pass;
strings.size = strings_holder.size();
ptr_holder = std::make_unique<ClickHouseLibrary::CString[]>(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<ClickHouseLibrary::CString[]> ptr_holder = nullptr;
Container strings_holder;
};
}

View File

@ -0,0 +1,219 @@
#include "SharedLibraryHandler.h"
#include <ext/scope_guard.h>
#include <IO/ReadHelpers.h>
#include <common/find_symbols.h>
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<std::string> & library_settings,
const Block & sample_block_,
const std::vector<std::string> & attributes_names_)
: library_path(library_path_)
, sample_block(sample_block_)
, attributes_names(attributes_names_)
{
library = std::make_shared<SharedLibrary>(library_path, RTLD_LAZY);
settings_holder = std::make_shared<CStringsHolder>(CStringsHolder(library_settings));
auto lib_new = library->tryGet<ClickHouseLibrary::LibraryNewFunc>(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::LibraryCloneFunc>(ClickHouseLibrary::LIBRARY_CLONE_FUNC_NAME);
if (lib_clone)
{
lib_data = lib_clone(other.lib_data);
}
else
{
auto lib_new = library->tryGet<ClickHouseLibrary::LibraryNewFunc>(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::LibraryDeleteFunc>(ClickHouseLibrary::LIBRARY_DELETE_FUNC_NAME);
if (lib_delete)
lib_delete(lib_data);
}
bool SharedLibraryHandler::isModified()
{
auto func_is_modified = library->tryGet<ClickHouseLibrary::LibraryIsModifiedFunc>(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::LibrarySupportsSelectiveLoadFunc>(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<ClickHouseLibrary::CString[]>(attributes_names.size());
ClickHouseLibrary::CStrings columns{static_cast<decltype(ClickHouseLibrary::CStrings::data)>(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::LibraryLoadAllFunc>(ClickHouseLibrary::LIBRARY_LOAD_ALL_FUNC_NAME);
auto data_new_func = library->get<ClickHouseLibrary::LibraryDataNewFunc>(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME);
auto data_delete_func = library->get<ClickHouseLibrary::LibraryDataDeleteFunc>(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<OneBlockInputStream>(block);
}
BlockInputStreamPtr SharedLibraryHandler::loadIds(const std::vector<uint64_t> & ids)
{
const ClickHouseLibrary::VectorUInt64 ids_data{ext::bit_cast<decltype(ClickHouseLibrary::VectorUInt64::data)>(ids.data()), ids.size()};
auto columns_holder = std::make_unique<ClickHouseLibrary::CString[]>(attributes_names.size());
ClickHouseLibrary::CStrings columns_pass{static_cast<decltype(ClickHouseLibrary::CStrings::data)>(columns_holder.get()), attributes_names.size()};
auto load_ids_func = library->get<ClickHouseLibrary::LibraryLoadIdsFunc>(ClickHouseLibrary::LIBRARY_LOAD_IDS_FUNC_NAME);
auto data_new_func = library->get<ClickHouseLibrary::LibraryDataNewFunc>(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME);
auto data_delete_func = library->get<ClickHouseLibrary::LibraryDataDeleteFunc>(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<OneBlockInputStream>(block);
}
BlockInputStreamPtr SharedLibraryHandler::loadKeys(const Columns & key_columns)
{
auto holder = std::make_unique<ClickHouseLibrary::Row[]>(key_columns.size());
std::vector<std::unique_ptr<ClickHouseLibrary::Field[]>> column_data_holders;
for (size_t i = 0; i < key_columns.size(); ++i)
{
auto cell_holder = std::make_unique<ClickHouseLibrary::Field[]>(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<const void *>(data_ref.data),
.size = data_ref.size};
}
holder[i] = ClickHouseLibrary::Row{
.data = static_cast<ClickHouseLibrary::Field *>(cell_holder.get()),
.size = key_columns[i]->size()};
column_data_holders.push_back(std::move(cell_holder));
}
ClickHouseLibrary::Table request_cols{
.data = static_cast<ClickHouseLibrary::Row *>(holder.get()),
.size = key_columns.size()};
auto load_keys_func = library->get<ClickHouseLibrary::LibraryLoadKeysFunc>(ClickHouseLibrary::LIBRARY_LOAD_KEYS_FUNC_NAME);
auto data_new_func = library->get<ClickHouseLibrary::LibraryDataNewFunc>(ClickHouseLibrary::LIBRARY_DATA_NEW_FUNC_NAME);
auto data_delete_func = library->get<ClickHouseLibrary::LibraryDataDeleteFunc>(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<OneBlockInputStream>(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<const ClickHouseLibrary::Table *>(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<const char *>(field.data), size);
}
}
}
return sample_block.cloneWithColumns(std::move(columns));
}
}

View File

@ -0,0 +1,54 @@
#pragma once
#include <Common/SharedLibrary.h>
#include <common/logger_useful.h>
#include <DataStreams/OneBlockInputStream.h>
#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<std::string> & library_settings,
const Block & sample_block_,
const std::vector<std::string> & attributes_names_);
SharedLibraryHandler(const SharedLibraryHandler & other);
~SharedLibraryHandler();
BlockInputStreamPtr loadAll();
BlockInputStreamPtr loadIds(const std::vector<uint64_t> & 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<std::string> attributes_names;
SharedLibraryPtr library;
std::shared_ptr<CStringsHolder> settings_holder;
void * lib_data;
};
using SharedLibraryHandlerPtr = std::shared_ptr<SharedLibraryHandler>;
}

View File

@ -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<std::string> & library_settings,
const Block & sample_block,
const std::vector<std::string> & attributes_names)
{
std::lock_guard lock(mutex);
library_handlers[dictionary_id] = std::make_shared<SharedLibraryHandler>(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<SharedLibraryHandler>(*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;
}
}

View File

@ -0,0 +1,37 @@
#pragma once
#include "SharedLibraryHandler.h"
#include <unordered_map>
#include <mutex>
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<std::string> & library_settings,
const Block & sample_block,
const std::vector<std::string> & 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<std::string, SharedLibraryHandlerPtr> library_handlers;
std::mutex mutex;
};
}

View File

@ -0,0 +1,3 @@
int mainEntryClickHouseLibraryBridge(int argc, char ** argv);
int main(int argc_, char ** argv_) { return mainEntryClickHouseLibraryBridge(argc_, argv_); }

View File

@ -0,0 +1,66 @@
#include <Dictionaries/LibraryDictionarySourceExternal.h>
#include <common/logger_useful.h>
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;
}
}
}

View File

@ -24,6 +24,7 @@ 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

View File

@ -8,7 +8,7 @@
namespace DB
{
std::unique_ptr<HTTPRequestHandler> HandlerFactory::createRequestHandler(const HTTPServerRequest & request)
std::unique_ptr<HTTPRequestHandler> ODBCBridgeHandlerFactory::createRequestHandler(const HTTPServerRequest & request)
{
Poco::URI uri{request.getURI()};
LOG_TRACE(log, "Request URI: {}", uri.toString());

View File

@ -20,10 +20,10 @@ 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
{
public:
HandlerFactory(const std::string & name_, size_t keep_alive_timeout_, Context & context_)
ODBCBridgeHandlerFactory(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_)
{
pool_map = std::make_shared<ODBCHandler::PoolMap>();
@ -38,4 +38,5 @@ private:
Context & context;
std::shared_ptr<ODBCHandler::PoolMap> pool_map;
};
}

View File

@ -1,244 +1,4 @@
#include "ODBCBridge.h"
#include "HandlerFactory.h"
#include <string>
#include <errno.h>
#include <IO/ReadHelpers.h>
#include <boost/program_options.hpp>
#if USE_ODBC
// It doesn't make much sense to build this bridge without ODBC, but we still do this.
# include <Poco/Data/ODBC/Connector.h>
#endif
#include <Poco/Net/NetException.h>
#include <Poco/String.h>
#include <Poco/Util/HelpFormatter.h>
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/config.h>
#include <Formats/registerFormats.h>
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include <ext/range.h>
#include <Common/SensitiveDataMasker.h>
#include <Server/HTTP/HTTPServer.h>
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 <listen_host> element of configuration file. Example: <listen_host>0.0.0.0</listen_host>", 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 <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<decltype(*this)>;
options.addOption(Poco::Util::Option("help", "", "produce this help message")
.binding("help")
.callback(Poco::Util::OptionCallback<Me>(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<std::string> & /*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<SensitiveDataMasker>(config(), "query_masking_rules"));
}
auto server = HTTPServer(
context,
std::make_shared<HandlerFactory>("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)

View File

@ -2,38 +2,25 @@
#include <Interpreters/Context.h>
#include <Poco/Logger.h>
#include <daemon/BaseDaemon.h>
#include <bridge/IBridge.h>
#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;
const std::string bridgeName() const override
{
return "ODBCBridge";
}
void uninitialize() override;
int main(const std::vector<std::string> & 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(Context & context) const override
{
return std::make_shared<ODBCBridgeHandlerFactory>("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context);
}
};
}

View File

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

View File

@ -0,0 +1,132 @@
#include "IBridgeHelper.h"
#include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/ReadHelpers.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Path.h>
#include <Poco/URI.h>
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<ShellCommand> 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<std::string> 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));
}
}

View File

@ -0,0 +1,70 @@
#pragma once
#include <Interpreters/Context.h>
#include <Poco/Logger.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Net/HTTPRequest.h>
#include <Common/ShellCommand.h>
#include <IO/ConnectionTimeoutsContext.h>
#include <common/logger_useful.h>
namespace DB
{
/// Common base class for XDBC and Library bridge helpers.
/// Contains helper methods to check/start bridge sync.
class IBridgeHelper
{
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;
virtual ~IBridgeHelper() = default;
void startBridgeSync() const;
Poco::URI getMainURI() const;
Poco::URI getPingURI() const;
protected:
/// clickhouse-odbc-bridge, clickhouse-library-bridge
virtual const String serviceAlias() const = 0;
virtual const String serviceFileName() const = 0;
virtual size_t getDefaultPort() const = 0;
virtual bool startBridgeManually() const = 0;
virtual void startBridge(std::unique_ptr<ShellCommand> cmd) const = 0;
virtual const String configPrefix() const = 0;
virtual const Context & getContext() 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<ShellCommand> startBridgeCommand() const;
};
}

View File

@ -0,0 +1,183 @@
#include "LibraryBridgeHelper.h"
#include <IO/ReadHelpers.h>
#include <DataStreams/OneBlockInputStream.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <Dictionaries/DictionarySourceHelpers.h>
#include <Processors/Formats/InputStreamFromInputFormat.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteBufferFromString.h>
#include <Formats/FormatFactory.h>
#include <Poco/Path.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/ShellCommand.h>
#include <common/logger_useful.h>
#include <ext/range.h>
#include <Core/Field.h>
#include <Common/escapeForFileName.h>
namespace DB
{
LibraryBridgeHelper::LibraryBridgeHelper(
const Context & context_,
const Block & sample_block_,
const Field & dictionary_id_)
: log(&Poco::Logger::get("LibraryBridgeHelper"))
, context(context_)
, 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<ShellCommand> cmd) const
{
context.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 = context.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 = context.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(context));
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<ReadWriteBufferFromHTTP>(
uri,
Poco::Net::HTTPRequest::HTTP_POST,
std::move(out_stream_callback),
ConnectionTimeouts::getHTTPTimeouts(context),
0,
Poco::Net::HTTPBasicCredentials{},
DBMS_DEFAULT_BUFFER_SIZE,
ReadWriteBufferFromHTTP::HTTPHeaderEntries{});
auto input_stream = context.getInputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, DEFAULT_BLOCK_SIZE);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(read_buf_ptr));
}
}

View File

@ -0,0 +1,88 @@
#pragma once
#include <Interpreters/Context.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Poco/Logger.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/URI.h>
#include <Bridge/IBridgeHelper.h>
namespace DB
{
class LibraryBridgeHelper : public IBridgeHelper
{
public:
static constexpr inline size_t DEFAULT_PORT = 9012;
LibraryBridgeHelper(const Context & context_, const Block & sample_block, const Field & dictionary_id_);
bool initLibrary(const std::string & library_path, const std::string library_settings, const std::string attributes_names);
bool cloneLibrary(const Field & other_dictionary_id);
bool removeLibrary();
bool isModified();
bool supportsSelectiveLoad();
BlockInputStreamPtr loadAll();
BlockInputStreamPtr loadIds(const 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<ShellCommand> cmd) const override;
const String serviceAlias() const override { return "clickhouse-library-bridge"; }
const String serviceFileName() const override { return serviceAlias(); }
size_t getDefaultPort() const override { return DEFAULT_PORT; }
bool startBridgeManually() const override { return false; }
const String configPrefix() const override { return "library_bridge"; }
const Context & getContext() const override { return context; }
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 Context & context;
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;
};
}

View File

@ -0,0 +1,266 @@
#pragma once
#include <IO/ReadHelpers.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Interpreters/Context.h>
#include <Access/AccessType.h>
#include <Parsers/IdentifierQuotingStyle.h>
#include <Poco/File.h>
#include <Poco/Logger.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Path.h>
#include <Poco/URI.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/ShellCommand.h>
#include <IO/ConnectionTimeoutsContext.h>
#include <common/logger_useful.h>
#include <ext/range.h>
#include <Bridge/IBridgeHelper.h>
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
#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:
virtual std::vector<std::pair<std::string, std::string>> 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 const String getName() const = 0;
};
using BridgeHelperPtr = std::shared_ptr<IXDBCBridgeHelper>;
template <typename BridgeHelperMixin>
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(
const Context & global_context_,
const Poco::Timespan & http_timeout_,
const std::string & connection_string_)
: log(&Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper"))
, connection_string(connection_string_)
, http_timeout(http_timeout_)
, context(global_context_)
, config(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; }
const String getName() const override { return BridgeHelperMixin::getName(); }
size_t getDefaultPort() const override { return DEFAULT_PORT; }
const String serviceAlias() const override { return BridgeHelperMixin::serviceAlias(); }
/// Same for odbc and jdbc
const String serviceFileName() const override { return "clickhouse-odbc-bridge"; }
const String configPrefix() const override { return BridgeHelperMixin::configPrefix(); }
const Context & getContext() const override { return context; }
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<ShellCommand> cmd) const override
{
context.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 Context & context;
const Configuration & config;
std::optional<IdentifierQuotingStyle> quote_style;
std::optional<bool> is_schema_allowed;
protected:
using URLParams = std::vector<std::pair<std::string, std::string>>;
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<std::pair<std::string, std::string>> 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(context));
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(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;
}
};
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 bool startBridgeManually()
{
return true;
}
};
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 bool startBridgeManually()
{
return false;
}
};
}

17
src/Bridge/ya.make Normal file
View File

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

14
src/Bridge/ya.make.in Normal file
View File

@ -0,0 +1,14 @@
OWNER(g:clickhouse)
LIBRARY()
PEERDIR(
clickhouse/src/Common
)
SRCS(
<? find . -name '*.cpp' | grep -v -F tests | sed 's/^\.\// /' | sort ?>
)
END()

View File

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

View File

@ -590,7 +590,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).

View File

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

View File

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

View File

@ -1,351 +0,0 @@
#pragma once
#include <IO/ReadHelpers.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Interpreters/Context.h>
#include <Access/AccessType.h>
#include <Parsers/IdentifierQuotingStyle.h>
#include <Poco/File.h>
#include <Poco/Logger.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Path.h>
#include <Poco/URI.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/ShellCommand.h>
#include <IO/ConnectionTimeoutsContext.h>
#include <common/logger_useful.h>
#include <ext/range.h>
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
#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<std::pair<std::string, std::string>> 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<IXDBCBridgeHelper>;
template <typename BridgeHelperMixin>
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<IdentifierQuotingStyle> quote_style;
std::optional<bool> 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<std::pair<std::string, std::string>> getURLParams(const std::string & cols, UInt64 max_block_size) const override
{
std::vector<std::pair<std::string, std::string>> 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<ShellCommand> 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<ShellCommand> 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<std::string> 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<ODBCBridgeMixin>::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));
}
};
}

View File

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

View File

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

View File

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

View File

@ -1,6 +1,9 @@
#include "parseRemoteDescription.h"
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <common/logger_useful.h>
namespace DB
{
@ -167,4 +170,27 @@ std::vector<String> parseRemoteDescription(const String & description, size_t l,
return res;
}
std::vector<std::pair<String, uint16_t>> parseRemoteDescriptionForExternalDatabase(const String & description, size_t max_addresses, UInt16 default_port)
{
auto addresses = parseRemoteDescription(description, 0, description.size(), '|', max_addresses);
std::vector<std::pair<String, uint16_t>> 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<UInt16>(address.substr(colon + 1))));
}
}
return result;
}
}

View File

@ -17,4 +17,7 @@ namespace DB
*/
std::vector<String> 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<std::pair<String, uint16_t>> parseRemoteDescriptionForExternalDatabase(const String & description, size_t max_addresses, UInt16 default_port);
}

View File

@ -357,10 +357,6 @@ void Changelog::readChangelogAndInitWriter(size_t last_commited_log_index, size_
void Changelog::rotate(size_t new_start_log_index)
{
//// doesn't exist on init
if (current_writer)
current_writer->flush();
ChangelogFileDescription new_description;
new_description.prefix = DEFAULT_PREFIX;
new_description.from_log_index = new_start_log_index;

View File

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

View File

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

View File

@ -371,6 +371,7 @@ 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(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) \

View File

@ -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:
{
if (value == "t")
assert_cast<ColumnUInt8 &>(column).insertValue(1);
else if (value == "f")
assert_cast<ColumnUInt8 &>(column).insertValue(0);
else
assert_cast<ColumnUInt8 &>(column).insertValue(pqxx::from_string<uint16_t>(value));
break;
}
case ValueType::vtUInt16:
assert_cast<ColumnUInt16 &>(column).insertValue(pqxx::from_string<uint16_t>(value));
break;

View File

@ -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<pqxx::read_transaction> tx;
std::unique_ptr<pqxx::stream_from> stream;

View File

@ -30,6 +30,7 @@
#endif
#if USE_MYSQL || USE_LIBPQXX
#include <Common/parseRemoteDescription.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Common/parseAddress.h>
#endif
@ -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<String>(arguments[0], engine_name);
const auto & host_port = safeGetLiteralValue<String>(arguments[0], engine_name);
const auto & mysql_database_name = safeGetLiteralValue<String>(arguments[1], engine_name);
const auto & mysql_user_name = safeGetLiteralValue<String>(arguments[2], engine_name);
const auto & mysql_user_password = safeGetLiteralValue<String>(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<ConnectionMySQLSettings>();
/// 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<MaterializeMySQLSettings>();
@ -243,12 +248,14 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
if (engine->arguments->children.size() == 5)
use_table_cache = safeGetLiteralValue<UInt64>(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<PostgreSQLConnectionPool>(
auto connection_pool = std::make_shared<postgres::PoolWithFailover>(
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);

View File

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

View File

@ -45,8 +45,14 @@ constexpr static const auto suffix = ".remove_flag";
static constexpr const std::chrono::seconds cleaner_sleep_time{30};
static const std::chrono::seconds lock_acquire_timeout{10};
DatabaseConnectionMySQL::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<ConnectionMySQLSettings> settings_, mysqlxx::Pool && pool)
DatabaseConnectionMySQL::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<ConnectionMySQLSettings> settings_,
mysqlxx::PoolWithFailover && pool)
: IDatabase(database_name_)
, global_context(context.getGlobalContext())
, metadata_path(metadata_path_)

View File

@ -10,6 +10,7 @@
#include <Databases/DatabasesCommon.h>
#include <Databases/MySQL/ConnectionMySQLSettings.h>
#include <Parsers/ASTCreateQuery.h>
#include <mysqlxx/PoolWithFailover.h>
#include <atomic>
#include <condition_variable>
@ -36,9 +37,13 @@ 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<ConnectionMySQLSettings> settings_,
mysqlxx::Pool && pool);
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<ConnectionMySQLSettings> settings_,
mysqlxx::PoolWithFailover && pool);
String getEngineName() const override { return "MySQL"; }
@ -91,7 +96,7 @@ private:
std::atomic<bool> quit{false};
std::condition_variable cond;
using MySQLPool = mysqlxx::Pool;
using MySQLPool = mysqlxx::PoolWithFailover;
using ModifyTimeAndStorage = std::pair<UInt64, StoragePtr>;
mutable MySQLPool mysql_pool;

View File

@ -41,7 +41,7 @@ namespace DB
{
std::map<String, NamesAndTypesList> fetchTablesColumnsList(
mysqlxx::Pool & pool,
mysqlxx::PoolWithFailover & pool,
const String & database_name,
const std::vector<String> & tables_name,
bool external_table_functions_use_nulls,

View File

@ -3,7 +3,7 @@
#include "config_core.h"
#if USE_MYSQL
#include <mysqlxx/Pool.h>
#include <mysqlxx/PoolWithFailover.h>
#include <common/types.h>
#include <Core/MultiEnum.h>
@ -17,7 +17,7 @@ namespace DB
{
std::map<String, NamesAndTypesList> fetchTablesColumnsList(
mysqlxx::Pool & pool,
mysqlxx::PoolWithFailover & pool,
const String & database_name,
const std::vector<String> & tables_name,
bool external_table_functions_use_nulls,

View File

@ -40,7 +40,7 @@ DatabasePostgreSQL::DatabasePostgreSQL(
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())
@ -171,7 +171,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, const Conte
return StoragePtr{};
auto storage = StoragePostgreSQL::create(
StorageID(database_name, table_name), table_name, std::make_shared<PostgreSQLConnectionPool>(*connection_pool),
StorageID(database_name, table_name), *connection_pool, table_name,
ColumnsDescription{*columns}, ConstraintsDescription{}, context);
if (cache_tables)

View File

@ -9,14 +9,13 @@
#include <Databases/DatabasesCommon.h>
#include <Core/BackgroundSchedulePool.h>
#include <Parsers/ASTCreateQuery.h>
#include <Storages/PostgreSQL/PostgreSQLPoolWithFailover.h>
namespace DB
{
class Context;
class PostgreSQLConnectionPool;
using PostgreSQLConnectionPoolPtr = std::shared_ptr<PostgreSQLConnectionPool>;
/** Real-time access to table list and table structure from remote PostgreSQL.
@ -34,7 +33,7 @@ public:
const ASTStorage * database_engine_define,
const String & dbname_,
const String & postgres_dbname,
PostgreSQLConnectionPoolPtr connection_pool_,
postgres::PoolWithFailoverPtr connection_pool_,
const bool cache_tables_);
String getEngineName() const override { return "PostgreSQL"; }
@ -72,7 +71,7 @@ private:
String metadata_path;
ASTPtr database_engine_define;
String dbname;
PostgreSQLConnectionPoolPtr connection_pool;
postgres::PoolWithFailoverPtr connection_pool;
const bool cache_tables;
mutable Tables cached_tables;

View File

@ -40,6 +40,8 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl
res = std::make_shared<DataTypeInt32>();
else if (type == "bigint")
res = std::make_shared<DataTypeInt64>();
else if (type == "boolean")
res = std::make_shared<DataTypeUInt8>();
else if (type == "real")
res = std::make_shared<DataTypeFloat32>();
else if (type == "double precision")
@ -94,7 +96,7 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl
std::shared_ptr<NamesAndTypesList> 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();

View File

@ -12,7 +12,7 @@ namespace DB
{
std::shared_ptr<NamesAndTypesList> fetchPostgreSQLTableStructure(
PostgreSQLConnectionHolderPtr connection, const String & postgres_table_name, bool use_nulls);
postgres::ConnectionHolderPtr connection, const String & postgres_table_name, bool use_nulls);
}

View File

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

View File

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

View File

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

View File

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

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