mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge branch 'master' into vdimir/test_alter_moving_garbage_51342
This commit is contained in:
commit
068b2b6ab9
102
CHANGELOG.md
102
CHANGELOG.md
@ -1,4 +1,5 @@
|
||||
### Table of Contents
|
||||
**[ClickHouse release v23.6, 2023-06-30](#236)**<br/>
|
||||
**[ClickHouse release v23.5, 2023-06-08](#235)**<br/>
|
||||
**[ClickHouse release v23.4, 2023-04-26](#234)**<br/>
|
||||
**[ClickHouse release v23.3 LTS, 2023-03-30](#233)**<br/>
|
||||
@ -8,6 +9,107 @@
|
||||
|
||||
# 2023 Changelog
|
||||
|
||||
### <a id="236"></a> ClickHouse release 23.6, 2023-06-29
|
||||
|
||||
#### Backward Incompatible Change
|
||||
* Delete feature `do_not_evict_index_and_mark_files` in the fs cache. This feature was only making things worse. [#51253](https://github.com/ClickHouse/ClickHouse/pull/51253) ([Kseniia Sumarokova](https://github.com/kssenii)).
|
||||
* Remove ALTER support for experimental LIVE VIEW. [#51287](https://github.com/ClickHouse/ClickHouse/pull/51287) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* Decrease the default values for `http_max_field_value_size` and `http_max_field_name_size` to 128 KiB. [#51163](https://github.com/ClickHouse/ClickHouse/pull/51163) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
* CGroups metrics related to CPU are replaced with one metric, `CGroupMaxCPU` for better usability. The `Normalized` CPU usage metrics will be normalized to CGroups limits instead of the total number of CPUs when they are set. This closes [#50836](https://github.com/ClickHouse/ClickHouse/issues/50836). [#50835](https://github.com/ClickHouse/ClickHouse/pull/50835) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
|
||||
#### New Feature
|
||||
* Added `Overlay` database engine to combine multiple databases into one. Added `Filesystem` database engine to represent a directory in the filesystem as a set of implicitly available tables with auto-detected formats and structures. A new `S3` database engine allows to read-only interact with s3 storage by representing a prefix as a set of tables. A new `HDFS` database engine allows to interact with HDFS storage in the same way. [#48821](https://github.com/ClickHouse/ClickHouse/pull/48821) ([alekseygolub](https://github.com/alekseygolub)).
|
||||
* The function `transform` as well as `CASE` with value matching started to support all data types. This closes [#29730](https://github.com/ClickHouse/ClickHouse/issues/29730). This closes [#32387](https://github.com/ClickHouse/ClickHouse/issues/32387). This closes [#50827](https://github.com/ClickHouse/ClickHouse/issues/50827). This closes [#31336](https://github.com/ClickHouse/ClickHouse/issues/31336). This closes [#40493](https://github.com/ClickHouse/ClickHouse/issues/40493). [#51351](https://github.com/ClickHouse/ClickHouse/pull/51351) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* Added option `--rename_files_after_processing <pattern>`. This closes [#34207](https://github.com/ClickHouse/ClickHouse/issues/34207). [#49626](https://github.com/ClickHouse/ClickHouse/pull/49626) ([alekseygolub](https://github.com/alekseygolub)).
|
||||
* Add support for `TRUNCATE` modifier in `INTO OUTFILE` clause. Suggest using `APPEND` or `TRUNCATE` for `INTO OUTFILE` when file exists. [#50950](https://github.com/ClickHouse/ClickHouse/pull/50950) ([alekar](https://github.com/alekar)).
|
||||
* Add table engine `Redis` and table function `redis`. It allows querying external Redis servers. [#50150](https://github.com/ClickHouse/ClickHouse/pull/50150) ([JackyWoo](https://github.com/JackyWoo)).
|
||||
* Allow to skip empty files in file/s3/url/hdfs table functions using settings `s3_skip_empty_files`, `hdfs_skip_empty_files`, `engine_file_skip_empty_files`, `engine_url_skip_empty_files`. [#50364](https://github.com/ClickHouse/ClickHouse/pull/50364) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Add a new setting named `use_mysql_types_in_show_columns` to alter the `SHOW COLUMNS` SQL statement to display MySQL equivalent types when a client is connected via the MySQL compatibility port. [#49577](https://github.com/ClickHouse/ClickHouse/pull/49577) ([Thomas Panetti](https://github.com/tpanetti)).
|
||||
* Clickhouse-client can now be called with a connection string instead of "--host", "--port", "--user" etc. [#50689](https://github.com/ClickHouse/ClickHouse/pull/50689) ([Alexey Gerasimchuck](https://github.com/Demilivor)).
|
||||
* Add setting `session_timezone`, it is used as default timezone for session when not explicitly specified. [#44149](https://github.com/ClickHouse/ClickHouse/pull/44149) ([Andrey Zvonov](https://github.com/zvonand)).
|
||||
* Codec DEFLATE_QPL is now controlled via server setting "enable_deflate_qpl_codec" (default: false) instead of setting "allow_experimental_codecs". This marks DEFLATE_QPL non-experimental. [#50775](https://github.com/ClickHouse/ClickHouse/pull/50775) ([Robert Schulze](https://github.com/rschu1ze)).
|
||||
|
||||
#### Performance Improvement
|
||||
* Improved scheduling of merge selecting and cleanup tasks in `ReplicatedMergeTree`. The tasks will not be executed too frequently when there's nothing to merge or cleanup. Added settings `max_merge_selecting_sleep_ms`, `merge_selecting_sleep_slowdown_factor`, `max_cleanup_delay_period` and `cleanup_thread_preferred_points_per_iteration`. It should close [#31919](https://github.com/ClickHouse/ClickHouse/issues/31919). [#50107](https://github.com/ClickHouse/ClickHouse/pull/50107) ([Alexander Tokmakov](https://github.com/tavplubix)).
|
||||
* Make filter push down through cross join. [#50605](https://github.com/ClickHouse/ClickHouse/pull/50605) ([Han Fei](https://github.com/hanfei1991)).
|
||||
* Improve performance with enabled QueryProfiler using thread-local timer_id instead of global object. [#48778](https://github.com/ClickHouse/ClickHouse/pull/48778) ([Jiebin Sun](https://github.com/jiebinn)).
|
||||
* Rewrite CapnProto input/output format to improve its performance. Map column names and CapnProto fields case insensitive, fix reading/writing of nested structure fields. [#49752](https://github.com/ClickHouse/ClickHouse/pull/49752) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Optimize parquet write performance for parallel threads. [#50102](https://github.com/ClickHouse/ClickHouse/pull/50102) ([Hongbin Ma](https://github.com/binmahone)).
|
||||
* Disable `parallelize_output_from_storages` for processing MATERIALIZED VIEWs and storages with one block only. [#50214](https://github.com/ClickHouse/ClickHouse/pull/50214) ([Azat Khuzhin](https://github.com/azat)).
|
||||
* Merge PR [#46558](https://github.com/ClickHouse/ClickHouse/pull/46558). Avoid block permutation during sort if the block is already sorted. [#50697](https://github.com/ClickHouse/ClickHouse/pull/50697) ([Alexey Milovidov](https://github.com/alexey-milovidov), [Maksim Kita](https://github.com/kitaisreal)).
|
||||
* Make multiple list requests to ZooKeeper in parallel to speed up reading from system.zookeeper table. [#51042](https://github.com/ClickHouse/ClickHouse/pull/51042) ([Alexander Gololobov](https://github.com/davenger)).
|
||||
* Speedup initialization of DateTime lookup tables for time zones. This should reduce startup/connect time of clickhouse-client especially in debug build as it is rather heavy. [#51347](https://github.com/ClickHouse/ClickHouse/pull/51347) ([Alexander Gololobov](https://github.com/davenger)).
|
||||
* Fix data lakes slowness because of synchronous head requests. (Related to Iceberg/Deltalake/Hudi being slow with a lot of files). [#50976](https://github.com/ClickHouse/ClickHouse/pull/50976) ([Kseniia Sumarokova](https://github.com/kssenii)).
|
||||
* Do not read all the columns from right GLOBAL JOIN table. [#50721](https://github.com/ClickHouse/ClickHouse/pull/50721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
|
||||
|
||||
#### Experimental Feature
|
||||
* Support parallel replicas with the analyzer. [#50441](https://github.com/ClickHouse/ClickHouse/pull/50441) ([Raúl Marín](https://github.com/Algunenano)).
|
||||
* Add random sleep before large merges/mutations execution to split load more evenly between replicas in case of zero-copy replication. [#51282](https://github.com/ClickHouse/ClickHouse/pull/51282) ([alesapin](https://github.com/alesapin)).
|
||||
* Do not replicate `ALTER PARTITION` queries and mutations through `Replicated` database if it has only one shard and the underlying table is `ReplicatedMergeTree`. [#51049](https://github.com/ClickHouse/ClickHouse/pull/51049) ([Alexander Tokmakov](https://github.com/tavplubix)).
|
||||
|
||||
#### Improvement
|
||||
* Relax the thresholds for "too many parts" to be more modern. Return the backpressure during long-running insert queries. [#50856](https://github.com/ClickHouse/ClickHouse/pull/50856) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* Allow to cast IPv6 to IPv4 address for CIDR ::ffff:0:0/96 (IPv4-mapped addresses). [#49759](https://github.com/ClickHouse/ClickHouse/pull/49759) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
|
||||
* Update MongoDB protocol to support MongoDB 5.1 version and newer. Support for the versions with the old protocol (<3.6) is preserved. Closes [#45621](https://github.com/ClickHouse/ClickHouse/issues/45621), [#49879](https://github.com/ClickHouse/ClickHouse/issues/49879). [#50061](https://github.com/ClickHouse/ClickHouse/pull/50061) ([Nikolay Degterinsky](https://github.com/evillique)).
|
||||
* Add setting `input_format_max_bytes_to_read_for_schema_inference` to limit the number of bytes to read in schema inference. Closes [#50577](https://github.com/ClickHouse/ClickHouse/issues/50577). [#50592](https://github.com/ClickHouse/ClickHouse/pull/50592) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Respect setting `input_format_null_as_default` in schema inference. [#50602](https://github.com/ClickHouse/ClickHouse/pull/50602) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Allow to skip trailing empty lines in CSV/TSV/CustomSeparated formats via settings `input_format_csv_skip_trailing_empty_lines`, `input_format_tsv_skip_trailing_empty_lines` and `input_format_custom_skip_trailing_empty_lines` (disabled by default). Closes [#49315](https://github.com/ClickHouse/ClickHouse/issues/49315). [#50635](https://github.com/ClickHouse/ClickHouse/pull/50635) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Functions "toDateOrDefault|OrNull" and "accuateCast[OrDefault|OrNull]" now correctly parse numeric arguments. [#50709](https://github.com/ClickHouse/ClickHouse/pull/50709) ([Dmitry Kardymon](https://github.com/kardymonds)).
|
||||
* Support CSV with whitespace or `\t` field delimiters, and these delimiters are supported in Spark. [#50712](https://github.com/ClickHouse/ClickHouse/pull/50712) ([KevinyhZou](https://github.com/KevinyhZou)).
|
||||
* Settings `number_of_mutations_to_delay` and `number_of_mutations_to_throw` are enabled by default now with values 500 and 1000 respectively. [#50726](https://github.com/ClickHouse/ClickHouse/pull/50726) ([Anton Popov](https://github.com/CurtizJ)).
|
||||
* The dashboard correctly shows missing values. This closes [#50831](https://github.com/ClickHouse/ClickHouse/issues/50831). [#50832](https://github.com/ClickHouse/ClickHouse/pull/50832) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* Added the possibility to use date and time arguments in the syslog timestamp format in functions `parseDateTimeBestEffort*` and `parseDateTime64BestEffort*`. [#50925](https://github.com/ClickHouse/ClickHouse/pull/50925) ([Victor Krasnov](https://github.com/sirvickr)).
|
||||
* Command line parameter "--password" in clickhouse-client can now be specified only once. [#50966](https://github.com/ClickHouse/ClickHouse/pull/50966) ([Alexey Gerasimchuck](https://github.com/Demilivor)).
|
||||
* Use `hash_of_all_files` from `system.parts` to check identity of parts during on-cluster backups. [#50997](https://github.com/ClickHouse/ClickHouse/pull/50997) ([Vitaly Baranov](https://github.com/vitlibar)).
|
||||
* The system table zookeeper_connection connected_time identifies the time when the connection is established (standard format), and session_uptime_elapsed_seconds is added, which labels the duration of the established connection session (in seconds). [#51026](https://github.com/ClickHouse/ClickHouse/pull/51026) ([郭小龙](https://github.com/guoxiaolongzte)).
|
||||
* Improve the progress bar for file/s3/hdfs/url table functions by using chunk size from source data and using incremental total size counting in each thread. Fix the progress bar for *Cluster functions. This closes [#47250](https://github.com/ClickHouse/ClickHouse/issues/47250). [#51088](https://github.com/ClickHouse/ClickHouse/pull/51088) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Add total_bytes_to_read to the Progress packet in TCP protocol for better Progress bar. [#51158](https://github.com/ClickHouse/ClickHouse/pull/51158) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Better checking of data parts on disks with filesystem cache. [#51164](https://github.com/ClickHouse/ClickHouse/pull/51164) ([Anton Popov](https://github.com/CurtizJ)).
|
||||
* Fix sometimes not correct current_elements_num in fs cache. [#51242](https://github.com/ClickHouse/ClickHouse/pull/51242) ([Kseniia Sumarokova](https://github.com/kssenii)).
|
||||
|
||||
#### Build/Testing/Packaging Improvement
|
||||
* Add embedded keeper-client to standalone keeper binary. [#50964](https://github.com/ClickHouse/ClickHouse/pull/50964) ([pufit](https://github.com/pufit)).
|
||||
* Actual LZ4 version is used now. [#50621](https://github.com/ClickHouse/ClickHouse/pull/50621) ([Nikita Taranov](https://github.com/nickitat)).
|
||||
* ClickHouse server will print the list of changed settings on fatal errors. This closes [#51137](https://github.com/ClickHouse/ClickHouse/issues/51137). [#51138](https://github.com/ClickHouse/ClickHouse/pull/51138) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* Allow building ClickHouse with clang-17. [#51300](https://github.com/ClickHouse/ClickHouse/pull/51300) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* [SQLancer](https://github.com/sqlancer/sqlancer) check is considered stable as bugs that were triggered by it are fixed. Now failures of SQLancer check will be reported as failed check status. [#51340](https://github.com/ClickHouse/ClickHouse/pull/51340) ([Ilya Yatsishin](https://github.com/qoega)).
|
||||
* Split huge `RUN` in Dockerfile into smaller conditional. Install the necessary tools on demand in the same `RUN` layer, and remove them after that. Upgrade the OS only once at the beginning. Use a modern way to check the signed repository. Downgrade the base repo to ubuntu:20.04 to address the issues on older docker versions. Upgrade golang version to address golang vulnerabilities. [#51504](https://github.com/ClickHouse/ClickHouse/pull/51504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
|
||||
#### Bug Fix (user-visible misbehavior in an official stable release)
|
||||
|
||||
* Report loading status for executable dictionaries correctly [#48775](https://github.com/ClickHouse/ClickHouse/pull/48775) ([Anton Kozlov](https://github.com/tonickkozlov)).
|
||||
* Proper mutation of skip indices and projections [#50104](https://github.com/ClickHouse/ClickHouse/pull/50104) ([Amos Bird](https://github.com/amosbird)).
|
||||
* Cleanup moving parts [#50489](https://github.com/ClickHouse/ClickHouse/pull/50489) ([vdimir](https://github.com/vdimir)).
|
||||
* Fix backward compatibility for IP types hashing in aggregate functions [#50551](https://github.com/ClickHouse/ClickHouse/pull/50551) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
|
||||
* Fix Log family table return wrong rows count after truncate [#50585](https://github.com/ClickHouse/ClickHouse/pull/50585) ([flynn](https://github.com/ucasfl)).
|
||||
* Fix bug in `uniqExact` parallel merging [#50590](https://github.com/ClickHouse/ClickHouse/pull/50590) ([Nikita Taranov](https://github.com/nickitat)).
|
||||
* Revert recent grace hash join changes [#50699](https://github.com/ClickHouse/ClickHouse/pull/50699) ([vdimir](https://github.com/vdimir)).
|
||||
* Query Cache: Try to fix bad cast from `ColumnConst` to `ColumnVector<char8_t>` [#50704](https://github.com/ClickHouse/ClickHouse/pull/50704) ([Robert Schulze](https://github.com/rschu1ze)).
|
||||
* Avoid storing logs in Keeper containing unknown operation [#50751](https://github.com/ClickHouse/ClickHouse/pull/50751) ([Antonio Andelic](https://github.com/antonio2368)).
|
||||
* SummingMergeTree support for DateTime64 [#50797](https://github.com/ClickHouse/ClickHouse/pull/50797) ([Jordi Villar](https://github.com/jrdi)).
|
||||
* Add compatibility setting for non-const timezones [#50834](https://github.com/ClickHouse/ClickHouse/pull/50834) ([Robert Schulze](https://github.com/rschu1ze)).
|
||||
* Fix hashing of LDAP params in the cache entries [#50865](https://github.com/ClickHouse/ClickHouse/pull/50865) ([Julian Maicher](https://github.com/jmaicher)).
|
||||
* Fallback to parsing big integer from String instead of exception in Parquet format [#50873](https://github.com/ClickHouse/ClickHouse/pull/50873) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Fix checking the lock file too often while writing a backup [#50889](https://github.com/ClickHouse/ClickHouse/pull/50889) ([Vitaly Baranov](https://github.com/vitlibar)).
|
||||
* Do not apply projection if read-in-order was enabled. [#50923](https://github.com/ClickHouse/ClickHouse/pull/50923) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
|
||||
* Fix race in the Azure blob storage iterator [#50936](https://github.com/ClickHouse/ClickHouse/pull/50936) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
|
||||
* Fix erroneous `sort_description` propagation in `CreatingSets` [#50955](https://github.com/ClickHouse/ClickHouse/pull/50955) ([Nikita Taranov](https://github.com/nickitat)).
|
||||
* Fix Iceberg v2 optional metadata parsing [#50974](https://github.com/ClickHouse/ClickHouse/pull/50974) ([Kseniia Sumarokova](https://github.com/kssenii)).
|
||||
* MaterializedMySQL: Keep parentheses for empty table overrides [#50977](https://github.com/ClickHouse/ClickHouse/pull/50977) ([Val Doroshchuk](https://github.com/valbok)).
|
||||
* Fix crash in BackupCoordinationStageSync::setError() [#51012](https://github.com/ClickHouse/ClickHouse/pull/51012) ([Vitaly Baranov](https://github.com/vitlibar)).
|
||||
* Fix subtly broken copy-on-write of ColumnLowCardinality dictionary [#51064](https://github.com/ClickHouse/ClickHouse/pull/51064) ([Michael Kolupaev](https://github.com/al13n321)).
|
||||
* Generate safe IVs [#51086](https://github.com/ClickHouse/ClickHouse/pull/51086) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
|
||||
* Fix ineffective query cache for SELECTs with subqueries [#51132](https://github.com/ClickHouse/ClickHouse/pull/51132) ([Robert Schulze](https://github.com/rschu1ze)).
|
||||
* Fix Set index with constant nullable comparison. [#51205](https://github.com/ClickHouse/ClickHouse/pull/51205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
|
||||
* Fix a crash in s3 and s3Cluster functions [#51209](https://github.com/ClickHouse/ClickHouse/pull/51209) ([Nikolay Degterinsky](https://github.com/evillique)).
|
||||
* Fix a crash with compiled expressions [#51231](https://github.com/ClickHouse/ClickHouse/pull/51231) ([LiuNeng](https://github.com/liuneng1994)).
|
||||
* Fix use-after-free in StorageURL when switching URLs [#51260](https://github.com/ClickHouse/ClickHouse/pull/51260) ([Michael Kolupaev](https://github.com/al13n321)).
|
||||
* Updated check for parameterized view [#51272](https://github.com/ClickHouse/ClickHouse/pull/51272) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
|
||||
* Fix multiple writing of same file to backup [#51299](https://github.com/ClickHouse/ClickHouse/pull/51299) ([Vitaly Baranov](https://github.com/vitlibar)).
|
||||
* Fix fuzzer failure in ActionsDAG [#51301](https://github.com/ClickHouse/ClickHouse/pull/51301) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* Remove garbage from function `transform` [#51350](https://github.com/ClickHouse/ClickHouse/pull/51350) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
|
||||
|
||||
### <a id="235"></a> ClickHouse release 23.5, 2023-06-08
|
||||
|
||||
#### Upgrade Notes
|
||||
|
@ -27,6 +27,7 @@ curl https://clickhouse.com/ | sh
|
||||
* [**ClickHouse Meetup in Boston**](https://www.meetup.com/clickhouse-boston-user-group/events/293913596) - Jul 18
|
||||
* [**ClickHouse Meetup in NYC**](https://www.meetup.com/clickhouse-new-york-user-group/events/293913441) - Jul 19
|
||||
* [**ClickHouse Meetup in Toronto**](https://www.meetup.com/clickhouse-toronto-user-group/events/294183127) - Jul 20
|
||||
* [**ClickHouse Meetup in Singapore**](https://www.meetup.com/clickhouse-singapore-meetup-group/events/294428050/) - Jul 27
|
||||
|
||||
Also, keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler <at> clickhouse <dot> com.
|
||||
|
||||
|
@ -2,21 +2,23 @@
|
||||
|
||||
#include <base/strong_typedef.h>
|
||||
#include <base/extended_types.h>
|
||||
#include <Common/formatIPv6.h>
|
||||
#include <Common/memcmpSmall.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using IPv4 = StrongTypedef<UInt32, struct IPv4Tag>;
|
||||
struct IPv4 : StrongTypedef<UInt32, struct IPv4Tag>
|
||||
{
|
||||
using StrongTypedef::StrongTypedef;
|
||||
using StrongTypedef::operator=;
|
||||
constexpr explicit IPv4(UInt64 value): StrongTypedef(static_cast<UnderlyingType>(value)) {}
|
||||
};
|
||||
|
||||
struct IPv6 : StrongTypedef<UInt128, struct IPv6Tag>
|
||||
{
|
||||
constexpr IPv6() = default;
|
||||
constexpr explicit IPv6(const UInt128 & x) : StrongTypedef(x) {}
|
||||
constexpr explicit IPv6(UInt128 && x) : StrongTypedef(std::move(x)) {}
|
||||
|
||||
IPv6 & operator=(const UInt128 & rhs) { StrongTypedef::operator=(rhs); return *this; }
|
||||
IPv6 & operator=(UInt128 && rhs) { StrongTypedef::operator=(std::move(rhs)); return *this; }
|
||||
using StrongTypedef::StrongTypedef;
|
||||
using StrongTypedef::operator=;
|
||||
|
||||
bool operator<(const IPv6 & rhs) const
|
||||
{
|
||||
@ -54,12 +56,22 @@ namespace DB
|
||||
|
||||
namespace std
|
||||
{
|
||||
/// For historical reasons we hash IPv6 as a FixedString(16)
|
||||
template <>
|
||||
struct hash<DB::IPv6>
|
||||
{
|
||||
size_t operator()(const DB::IPv6 & x) const
|
||||
{
|
||||
return std::hash<DB::IPv6::UnderlyingType>()(x.toUnderType());
|
||||
return std::hash<std::string_view>{}(std::string_view(reinterpret_cast<const char*>(&x.toUnderType()), IPV6_BINARY_LENGTH));
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct hash<DB::IPv4>
|
||||
{
|
||||
size_t operator()(const DB::IPv4 & x) const
|
||||
{
|
||||
return std::hash<DB::IPv4::UnderlyingType>()(x.toUnderType());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -7,7 +7,13 @@
|
||||
|
||||
/** Returns value `from` converted to type `To` while retaining bit representation.
|
||||
* `To` and `From` must satisfy `CopyConstructible`.
|
||||
*
|
||||
* In contrast to std::bit_cast can cast types of different width.
|
||||
*
|
||||
* Note: for signed types of narrower size, the casted result is zero-extended
|
||||
* instead of sign-extended as with regular static_cast.
|
||||
* For example, -1 Int8 (represented as 0xFF) bit_casted to UInt64
|
||||
* gives 255 (represented as 0x00000000000000FF) instead of 0xFFFFFFFFFFFFFFFF
|
||||
*/
|
||||
template <typename To, typename From>
|
||||
std::decay_t<To> bit_cast(const From & from)
|
||||
|
@ -27,6 +27,8 @@ using FromDoubleIntermediateType = long double;
|
||||
using FromDoubleIntermediateType = boost::multiprecision::cpp_bin_float_double_extended;
|
||||
#endif
|
||||
|
||||
namespace CityHash_v1_0_2 { struct uint128; }
|
||||
|
||||
namespace wide
|
||||
{
|
||||
|
||||
@ -281,6 +283,17 @@ struct integer<Bits, Signed>::_impl
|
||||
}
|
||||
}
|
||||
|
||||
template <typename CityHashUInt128 = CityHash_v1_0_2::uint128>
|
||||
constexpr static void wide_integer_from_cityhash_uint128(integer<Bits, Signed> & self, const CityHashUInt128 & value) noexcept
|
||||
{
|
||||
static_assert(sizeof(item_count) >= 2);
|
||||
|
||||
if constexpr (std::endian::native == std::endian::little)
|
||||
wide_integer_from_tuple_like(self, std::make_pair(value.low64, value.high64));
|
||||
else
|
||||
wide_integer_from_tuple_like(self, std::make_pair(value.high64, value.low64));
|
||||
}
|
||||
|
||||
/**
|
||||
* N.B. t is constructed from double, so max(t) = max(double) ~ 2^310
|
||||
* the recursive call happens when t / 2^64 > 2^64, so there won't be more than 5 of them.
|
||||
@ -1036,6 +1049,8 @@ constexpr integer<Bits, Signed>::integer(T rhs) noexcept
|
||||
_impl::wide_integer_from_wide_integer(*this, rhs);
|
||||
else if constexpr (IsTupleLike<T>::value)
|
||||
_impl::wide_integer_from_tuple_like(*this, rhs);
|
||||
else if constexpr (std::is_same_v<std::remove_cvref_t<T>, CityHash_v1_0_2::uint128>)
|
||||
_impl::wide_integer_from_cityhash_uint128(*this, rhs);
|
||||
else
|
||||
_impl::wide_integer_from_builtin(*this, rhs);
|
||||
}
|
||||
@ -1051,6 +1066,8 @@ constexpr integer<Bits, Signed>::integer(std::initializer_list<T> il) noexcept
|
||||
_impl::wide_integer_from_wide_integer(*this, *il.begin());
|
||||
else if constexpr (IsTupleLike<T>::value)
|
||||
_impl::wide_integer_from_tuple_like(*this, *il.begin());
|
||||
else if constexpr (std::is_same_v<std::remove_cvref_t<T>, CityHash_v1_0_2::uint128>)
|
||||
_impl::wide_integer_from_cityhash_uint128(*this, *il.begin());
|
||||
else
|
||||
_impl::wide_integer_from_builtin(*this, *il.begin());
|
||||
}
|
||||
@ -1088,6 +1105,8 @@ constexpr integer<Bits, Signed> & integer<Bits, Signed>::operator=(T rhs) noexce
|
||||
{
|
||||
if constexpr (IsTupleLike<T>::value)
|
||||
_impl::wide_integer_from_tuple_like(*this, rhs);
|
||||
else if constexpr (std::is_same_v<std::remove_cvref_t<T>, CityHash_v1_0_2::uint128>)
|
||||
_impl::wide_integer_from_cityhash_uint128(*this, rhs);
|
||||
else
|
||||
_impl::wide_integer_from_builtin(*this, rhs);
|
||||
return *this;
|
||||
|
@ -1,53 +0,0 @@
|
||||
//
|
||||
// ConsoleCertificateHandler.h
|
||||
//
|
||||
// Library: NetSSL_OpenSSL
|
||||
// Package: SSLCore
|
||||
// Module: ConsoleCertificateHandler
|
||||
//
|
||||
// Definition of the ConsoleCertificateHandler class.
|
||||
//
|
||||
// Copyright (c) 2006-2009, Applied Informatics Software Engineering GmbH.
|
||||
// and Contributors.
|
||||
//
|
||||
// SPDX-License-Identifier: BSL-1.0
|
||||
//
|
||||
|
||||
|
||||
#ifndef NetSSL_ConsoleCertificateHandler_INCLUDED
|
||||
#define NetSSL_ConsoleCertificateHandler_INCLUDED
|
||||
|
||||
|
||||
#include "Poco/Net/InvalidCertificateHandler.h"
|
||||
#include "Poco/Net/NetSSL.h"
|
||||
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
namespace Net
|
||||
{
|
||||
|
||||
|
||||
class NetSSL_API ConsoleCertificateHandler : public InvalidCertificateHandler
|
||||
/// A ConsoleCertificateHandler is invoked whenever an error occurs verifying the certificate.
|
||||
///
|
||||
/// The certificate is printed to stdout and the user is asked via console if he wants to accept it.
|
||||
{
|
||||
public:
|
||||
ConsoleCertificateHandler(bool handleErrorsOnServerSide);
|
||||
/// Creates the ConsoleCertificateHandler.
|
||||
|
||||
virtual ~ConsoleCertificateHandler();
|
||||
/// Destroys the ConsoleCertificateHandler.
|
||||
|
||||
void onInvalidCertificate(const void * pSender, VerificationErrorArgs & errorCert);
|
||||
/// Prints the certificate to stdout and waits for user input on the console
|
||||
/// to decide if a certificate should be accepted/rejected.
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
} // namespace Poco::Net
|
||||
|
||||
|
||||
#endif // NetSSL_ConsoleCertificateHandler_INCLUDED
|
@ -85,7 +85,7 @@ namespace Net
|
||||
/// </options>
|
||||
/// </privateKeyPassphraseHandler>
|
||||
/// <invalidCertificateHandler>
|
||||
/// <name>ConsoleCertificateHandler</name>
|
||||
/// <name>RejectCertificateHandler</name>
|
||||
/// </invalidCertificateHandler>
|
||||
/// <cacheSessions>true|false</cacheSessions>
|
||||
/// <sessionIdContext>someString</sessionIdContext> <!-- server only -->
|
||||
@ -186,7 +186,7 @@ namespace Net
|
||||
///
|
||||
/// Valid initialization code would be:
|
||||
/// SharedPtr<PrivateKeyPassphraseHandler> pConsoleHandler = new KeyConsoleHandler;
|
||||
/// SharedPtr<InvalidCertificateHandler> pInvalidCertHandler = new ConsoleCertificateHandler;
|
||||
/// SharedPtr<InvalidCertificateHandler> pInvalidCertHandler = new RejectCertificateHandler;
|
||||
/// Context::Ptr pContext = new Context(Context::SERVER_USE, "any.pem", "any.pem", "rootcert.pem", Context::VERIFY_RELAXED, 9, false, "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH");
|
||||
/// SSLManager::instance().initializeServer(pConsoleHandler, pInvalidCertHandler, pContext);
|
||||
|
||||
@ -203,7 +203,7 @@ namespace Net
|
||||
///
|
||||
/// Valid initialization code would be:
|
||||
/// SharedPtr<PrivateKeyPassphraseHandler> pConsoleHandler = new KeyConsoleHandler;
|
||||
/// SharedPtr<InvalidCertificateHandler> pInvalidCertHandler = new ConsoleCertificateHandler;
|
||||
/// SharedPtr<InvalidCertificateHandler> pInvalidCertHandler = new RejectCertificateHandler;
|
||||
/// Context::Ptr pContext = new Context(Context::CLIENT_USE, "", "", "rootcert.pem", Context::VERIFY_RELAXED, 9, false, "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH");
|
||||
/// SSLManager::instance().initializeClient(pConsoleHandler, pInvalidCertHandler, pContext);
|
||||
|
||||
|
@ -13,7 +13,6 @@
|
||||
|
||||
|
||||
#include "Poco/Net/CertificateHandlerFactoryMgr.h"
|
||||
#include "Poco/Net/ConsoleCertificateHandler.h"
|
||||
#include "Poco/Net/AcceptCertificateHandler.h"
|
||||
#include "Poco/Net/RejectCertificateHandler.h"
|
||||
|
||||
@ -24,7 +23,6 @@ namespace Net {
|
||||
|
||||
CertificateHandlerFactoryMgr::CertificateHandlerFactoryMgr()
|
||||
{
|
||||
setFactory("ConsoleCertificateHandler", new CertificateHandlerFactoryImpl<ConsoleCertificateHandler>());
|
||||
setFactory("AcceptCertificateHandler", new CertificateHandlerFactoryImpl<AcceptCertificateHandler>());
|
||||
setFactory("RejectCertificateHandler", new CertificateHandlerFactoryImpl<RejectCertificateHandler>());
|
||||
}
|
||||
|
@ -1,53 +0,0 @@
|
||||
//
|
||||
// ConsoleCertificateHandler.cpp
|
||||
//
|
||||
// Library: NetSSL_OpenSSL
|
||||
// Package: SSLCore
|
||||
// Module: ConsoleCertificateHandler
|
||||
//
|
||||
// Copyright (c) 2006-2009, Applied Informatics Software Engineering GmbH.
|
||||
// and Contributors.
|
||||
//
|
||||
// SPDX-License-Identifier: BSL-1.0
|
||||
//
|
||||
|
||||
|
||||
#include "Poco/Net/ConsoleCertificateHandler.h"
|
||||
#include <iostream>
|
||||
|
||||
|
||||
namespace Poco {
|
||||
namespace Net {
|
||||
|
||||
|
||||
ConsoleCertificateHandler::ConsoleCertificateHandler(bool server): InvalidCertificateHandler(server)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
ConsoleCertificateHandler::~ConsoleCertificateHandler()
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
void ConsoleCertificateHandler::onInvalidCertificate(const void*, VerificationErrorArgs& errorCert)
|
||||
{
|
||||
const X509Certificate& aCert = errorCert.certificate();
|
||||
std::cout << "\n";
|
||||
std::cout << "WARNING: Certificate verification failed\n";
|
||||
std::cout << "----------------------------------------\n";
|
||||
std::cout << "Issuer Name: " << aCert.issuerName() << "\n";
|
||||
std::cout << "Subject Name: " << aCert.subjectName() << "\n\n";
|
||||
std::cout << "The certificate yielded the error: " << errorCert.errorMessage() << "\n\n";
|
||||
std::cout << "The error occurred in the certificate chain at position " << errorCert.errorDepth() << "\n";
|
||||
std::cout << "Accept the certificate (y,n)? ";
|
||||
char c = 0;
|
||||
std::cin >> c;
|
||||
if (c == 'y' || c == 'Y')
|
||||
errorCert.setIgnoreError(true);
|
||||
else
|
||||
errorCert.setIgnoreError(false);
|
||||
}
|
||||
|
||||
|
||||
} } // namespace Poco::Net
|
@ -46,7 +46,7 @@ const std::string SSLManager::CFG_PREFER_SERVER_CIPHERS("preferServerCiphers");
|
||||
const std::string SSLManager::CFG_DELEGATE_HANDLER("privateKeyPassphraseHandler.name");
|
||||
const std::string SSLManager::VAL_DELEGATE_HANDLER("KeyConsoleHandler");
|
||||
const std::string SSLManager::CFG_CERTIFICATE_HANDLER("invalidCertificateHandler.name");
|
||||
const std::string SSLManager::VAL_CERTIFICATE_HANDLER("ConsoleCertificateHandler");
|
||||
const std::string SSLManager::VAL_CERTIFICATE_HANDLER("RejectCertificateHandler");
|
||||
const std::string SSLManager::CFG_SERVER_PREFIX("openSSL.server.");
|
||||
const std::string SSLManager::CFG_CLIENT_PREFIX("openSSL.client.");
|
||||
const std::string SSLManager::CFG_CACHE_SESSIONS("cacheSessions");
|
||||
|
@ -31,12 +31,12 @@ endif()
|
||||
|
||||
set (CMAKE_CXX_STANDARD 17)
|
||||
|
||||
set(ARROW_VERSION "6.0.1")
|
||||
set(ARROW_VERSION "11.0.0")
|
||||
string(REGEX MATCH "^[0-9]+\\.[0-9]+\\.[0-9]+" ARROW_BASE_VERSION "${ARROW_VERSION}")
|
||||
|
||||
set(ARROW_VERSION_MAJOR "6")
|
||||
set(ARROW_VERSION_MAJOR "11")
|
||||
set(ARROW_VERSION_MINOR "0")
|
||||
set(ARROW_VERSION_PATCH "1")
|
||||
set(ARROW_VERSION_PATCH "0")
|
||||
|
||||
if(ARROW_VERSION_MAJOR STREQUAL "0")
|
||||
# Arrow 0.x.y => SO version is "x", full SO version is "x.y.0"
|
||||
@ -514,6 +514,10 @@ if (SANITIZE STREQUAL "undefined")
|
||||
target_compile_options(_arrow PRIVATE -fno-sanitize=undefined)
|
||||
endif ()
|
||||
|
||||
# Define Thrift version for parquet (we use 0.16.0)
|
||||
add_definitions(-DPARQUET_THRIFT_VERSION_MAJOR=0)
|
||||
add_definitions(-DPARQUET_THRIFT_VERSION_MINOR=16)
|
||||
|
||||
# === tools
|
||||
|
||||
set(TOOLS_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/tools/parquet")
|
||||
|
@ -61,11 +61,24 @@ namespace CityHash_v1_0_2
|
||||
typedef uint8_t uint8;
|
||||
typedef uint32_t uint32;
|
||||
typedef uint64_t uint64;
|
||||
typedef std::pair<uint64, uint64> uint128;
|
||||
|
||||
/// Represent an unsigned integer of 128 bits as it's used in CityHash.
|
||||
/// Originally CityHash used `std::pair<uint64, uint64>` instead of this struct,
|
||||
/// however the members `first` and `second` could be easily confused so they were renamed to `low64` and `high64`:
|
||||
/// `first` -> `low64`, `second` -> `high64`.
|
||||
struct uint128
|
||||
{
|
||||
uint64 low64 = 0;
|
||||
uint64 high64 = 0;
|
||||
|
||||
inline uint64 Uint128Low64(const uint128& x) { return x.first; }
|
||||
inline uint64 Uint128High64(const uint128& x) { return x.second; }
|
||||
uint128() = default;
|
||||
uint128(uint64 low64_, uint64 high64_) : low64(low64_), high64(high64_) {}
|
||||
friend bool operator ==(const uint128 & x, const uint128 & y) { return (x.low64 == y.low64) && (x.high64 == y.high64); }
|
||||
friend bool operator !=(const uint128 & x, const uint128 & y) { return !(x == y); }
|
||||
};
|
||||
|
||||
inline uint64 Uint128Low64(const uint128 & x) { return x.low64; }
|
||||
inline uint64 Uint128High64(const uint128 & x) { return x.high64; }
|
||||
|
||||
// Hash function for a byte array.
|
||||
uint64 CityHash64(const char *buf, size_t len);
|
||||
|
@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \
|
||||
esac
|
||||
|
||||
ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release"
|
||||
ARG VERSION="23.5.3.24"
|
||||
ARG VERSION="23.5.4.25"
|
||||
ARG PACKAGES="clickhouse-keeper"
|
||||
|
||||
# user/group precreated explicitly with fixed uid/gid on purpose.
|
||||
|
@ -89,7 +89,7 @@ RUN arch=${TARGETARCH:-amd64} \
|
||||
&& dpkg -i /tmp/nfpm.deb \
|
||||
&& rm /tmp/nfpm.deb
|
||||
|
||||
ARG GO_VERSION=1.19.5
|
||||
ARG GO_VERSION=1.19.10
|
||||
# We need go for clickhouse-diagnostics
|
||||
RUN arch=${TARGETARCH:-amd64} \
|
||||
&& curl -Lo /tmp/go.tgz "https://go.dev/dl/go${GO_VERSION}.linux-${arch}.tar.gz" \
|
||||
|
@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \
|
||||
# lts / testing / prestable / etc
|
||||
ARG REPO_CHANNEL="stable"
|
||||
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
|
||||
ARG VERSION="23.5.3.24"
|
||||
ARG VERSION="23.5.4.25"
|
||||
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
|
||||
|
||||
# user/group precreated explicitly with fixed uid/gid on purpose.
|
||||
|
@ -1,4 +1,4 @@
|
||||
FROM ubuntu:22.04
|
||||
FROM ubuntu:20.04
|
||||
|
||||
# see https://github.com/moby/moby/issues/4032#issuecomment-192327844
|
||||
ARG DEBIAN_FRONTEND=noninteractive
|
||||
@ -11,18 +11,19 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
|
||||
&& apt-get update \
|
||||
&& apt-get upgrade -yq \
|
||||
&& apt-get install --yes --no-install-recommends \
|
||||
apt-transport-https \
|
||||
ca-certificates \
|
||||
dirmngr \
|
||||
gnupg2 \
|
||||
wget \
|
||||
locales \
|
||||
tzdata \
|
||||
&& apt-get clean
|
||||
wget \
|
||||
&& apt-get clean \
|
||||
&& rm -rf \
|
||||
/var/lib/apt/lists/* \
|
||||
/var/cache/debconf \
|
||||
/tmp/*
|
||||
|
||||
ARG REPO_CHANNEL="stable"
|
||||
ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
|
||||
ARG VERSION="23.5.3.24"
|
||||
ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
|
||||
ARG VERSION="23.5.4.25"
|
||||
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
|
||||
|
||||
# set non-empty deb_location_url url to create a docker image
|
||||
@ -43,49 +44,68 @@ ARG single_binary_location_url=""
|
||||
|
||||
ARG TARGETARCH
|
||||
|
||||
RUN arch=${TARGETARCH:-amd64} \
|
||||
# install from a web location with deb packages
|
||||
RUN arch="${TARGETARCH:-amd64}" \
|
||||
&& if [ -n "${deb_location_url}" ]; then \
|
||||
echo "installing from custom url with deb packages: ${deb_location_url}" \
|
||||
rm -rf /tmp/clickhouse_debs \
|
||||
&& rm -rf /tmp/clickhouse_debs \
|
||||
&& mkdir -p /tmp/clickhouse_debs \
|
||||
&& for package in ${PACKAGES}; do \
|
||||
{ wget --progress=bar:force:noscroll "${deb_location_url}/${package}_${VERSION}_${arch}.deb" -P /tmp/clickhouse_debs || \
|
||||
wget --progress=bar:force:noscroll "${deb_location_url}/${package}_${VERSION}_all.deb" -P /tmp/clickhouse_debs ; } \
|
||||
|| exit 1 \
|
||||
; done \
|
||||
&& dpkg -i /tmp/clickhouse_debs/*.deb ; \
|
||||
elif [ -n "${single_binary_location_url}" ]; then \
|
||||
&& dpkg -i /tmp/clickhouse_debs/*.deb \
|
||||
&& rm -rf /tmp/* ; \
|
||||
fi
|
||||
|
||||
# install from a single binary
|
||||
RUN if [ -n "${single_binary_location_url}" ]; then \
|
||||
echo "installing from single binary url: ${single_binary_location_url}" \
|
||||
&& rm -rf /tmp/clickhouse_binary \
|
||||
&& mkdir -p /tmp/clickhouse_binary \
|
||||
&& wget --progress=bar:force:noscroll "${single_binary_location_url}" -O /tmp/clickhouse_binary/clickhouse \
|
||||
&& chmod +x /tmp/clickhouse_binary/clickhouse \
|
||||
&& /tmp/clickhouse_binary/clickhouse install --user "clickhouse" --group "clickhouse" ; \
|
||||
else \
|
||||
mkdir -p /etc/apt/sources.list.d \
|
||||
&& apt-key adv --keyserver keyserver.ubuntu.com --recv 8919F6BD2B48D754 \
|
||||
&& echo ${REPOSITORY} > /etc/apt/sources.list.d/clickhouse.list \
|
||||
&& /tmp/clickhouse_binary/clickhouse install --user "clickhouse" --group "clickhouse" \
|
||||
&& rm -rf /tmp/* ; \
|
||||
fi
|
||||
|
||||
# A fallback to installation from ClickHouse repository
|
||||
RUN if ! clickhouse local -q "SELECT ''" > /dev/null 2>&1; then \
|
||||
apt-get update \
|
||||
&& apt-get install --yes --no-install-recommends \
|
||||
apt-transport-https \
|
||||
ca-certificates \
|
||||
dirmngr \
|
||||
gnupg2 \
|
||||
&& mkdir -p /etc/apt/sources.list.d \
|
||||
&& GNUPGHOME=$(mktemp -d) \
|
||||
&& GNUPGHOME="$GNUPGHOME" gpg --no-default-keyring \
|
||||
--keyring /usr/share/keyrings/clickhouse-keyring.gpg \
|
||||
--keyserver hkp://keyserver.ubuntu.com:80 --recv-keys 8919F6BD2B48D754 \
|
||||
&& rm -r "$GNUPGHOME" \
|
||||
&& chmod +r /usr/share/keyrings/clickhouse-keyring.gpg \
|
||||
&& echo "${REPOSITORY}" > /etc/apt/sources.list.d/clickhouse.list \
|
||||
&& echo "installing from repository: ${REPOSITORY}" \
|
||||
&& apt-get update \
|
||||
&& apt-get --yes -o "Dpkg::Options::=--force-confdef" -o "Dpkg::Options::=--force-confold" upgrade \
|
||||
&& for package in ${PACKAGES}; do \
|
||||
packages="${packages} ${package}=${VERSION}" \
|
||||
; done \
|
||||
&& apt-get install --allow-unauthenticated --yes --no-install-recommends ${packages} || exit 1 \
|
||||
; fi \
|
||||
&& clickhouse-local -q 'SELECT * FROM system.build_options' \
|
||||
&& rm -rf \
|
||||
/var/lib/apt/lists/* \
|
||||
/var/cache/debconf \
|
||||
/tmp/* \
|
||||
&& mkdir -p /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client \
|
||||
&& chmod ugo+Xrw -R /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client
|
||||
|
||||
RUN apt-get autoremove --purge -yq libksba8 && \
|
||||
apt-get autoremove -yq
|
||||
&& rm -rf \
|
||||
/var/lib/apt/lists/* \
|
||||
/var/cache/debconf \
|
||||
/tmp/* \
|
||||
&& apt-get autoremove --purge -yq libksba8 \
|
||||
&& apt-get autoremove -yq \
|
||||
; fi
|
||||
|
||||
# post install
|
||||
# we need to allow "others" access to clickhouse folder, because docker container
|
||||
# can be started with arbitrary uid (openshift usecase)
|
||||
RUN clickhouse-local -q 'SELECT * FROM system.build_options' \
|
||||
&& mkdir -p /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client \
|
||||
&& chmod ugo+Xrw -R /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client
|
||||
|
||||
RUN locale-gen en_US.UTF-8
|
||||
ENV LANG en_US.UTF-8
|
||||
|
@ -20,7 +20,6 @@ For more information and documentation see https://clickhouse.com/.
|
||||
|
||||
- The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3.
|
||||
- The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A). Most ARM CPUs after 2017 support ARMv8.2-A. A notable exception is Raspberry Pi 4 from 2019 whose CPU only supports ARMv8.0-A.
|
||||
- Since the Clickhouse 23.3 Ubuntu image started using `ubuntu:22.04` as its base image, it requires docker version >= `20.10.10`, or use `docker run -- privileged` instead. Alternatively, try the Clickhouse Alpine image.
|
||||
|
||||
## How to use this image
|
||||
|
||||
|
@ -52,6 +52,8 @@ 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_BASE_TAG=${DOCKER_BASE_TAG:=latest}
|
||||
export DOCKER_HELPER_TAG=${DOCKER_HELPER_TAG:=latest}
|
||||
export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest}
|
||||
export DOCKER_DOTNET_CLIENT_TAG=${DOCKER_DOTNET_CLIENT_TAG:=latest}
|
||||
export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest}
|
||||
|
@ -14,6 +14,13 @@ LEFT_SERVER_PORT=9001
|
||||
# patched version
|
||||
RIGHT_SERVER_PORT=9002
|
||||
|
||||
# abort_conf -- abort if some options is not recognized
|
||||
# abort -- abort if something is not right in the env (i.e. per-cpu arenas does not work)
|
||||
# narenas -- set them explicitly to avoid disabling per-cpu arena in env
|
||||
# that returns different number of CPUs for some of the following
|
||||
# _SC_NPROCESSORS_ONLN/_SC_NPROCESSORS_CONF/sched_getaffinity
|
||||
export MALLOC_CONF="abort_conf:true,abort:true,narenas:$(nproc --all)"
|
||||
|
||||
function wait_for_server # port, pid
|
||||
{
|
||||
for _ in {1..60}
|
||||
@ -109,10 +116,6 @@ function restart
|
||||
while pkill -f clickhouse-serv ; do echo . ; sleep 1 ; done
|
||||
echo all killed
|
||||
|
||||
# Change the jemalloc settings here.
|
||||
# https://github.com/jemalloc/jemalloc/wiki/Getting-Started
|
||||
export MALLOC_CONF="confirm_conf:true"
|
||||
|
||||
set -m # Spawn servers in their own process groups
|
||||
|
||||
local left_server_opts=(
|
||||
@ -147,8 +150,6 @@ function restart
|
||||
|
||||
set +m
|
||||
|
||||
unset MALLOC_CONF
|
||||
|
||||
wait_for_server $LEFT_SERVER_PORT $left_pid
|
||||
echo left ok
|
||||
|
||||
|
@ -86,7 +86,7 @@ def process_test_log(log_path, broken_tests):
|
||||
test_name,
|
||||
"NOT_FAILED",
|
||||
test_time,
|
||||
["This test passed. Update broken_tests.txt.\n"],
|
||||
["This test passed. Update analyzer_tech_debt.txt.\n"],
|
||||
)
|
||||
)
|
||||
else:
|
||||
@ -205,7 +205,7 @@ if __name__ == "__main__":
|
||||
parser.add_argument("--in-results-dir", default="/test_output/")
|
||||
parser.add_argument("--out-results-file", default="/test_output/test_results.tsv")
|
||||
parser.add_argument("--out-status-file", default="/test_output/check_status.tsv")
|
||||
parser.add_argument("--broken-tests", default="/broken_tests.txt")
|
||||
parser.add_argument("--broken-tests", default="/analyzer_tech_debt.txt")
|
||||
args = parser.parse_args()
|
||||
|
||||
broken_tests = list()
|
||||
|
19
docs/changelogs/v23.3.6.7-lts.md
Normal file
19
docs/changelogs/v23.3.6.7-lts.md
Normal file
@ -0,0 +1,19 @@
|
||||
---
|
||||
sidebar_position: 1
|
||||
sidebar_label: 2023
|
||||
---
|
||||
|
||||
# 2023 Changelog
|
||||
|
||||
### ClickHouse release v23.3.6.7-lts (7e3f0a271b7) FIXME as compared to v23.3.5.9-lts (f5fbc2fd2b3)
|
||||
|
||||
#### Improvement
|
||||
* Backported in [#51240](https://github.com/ClickHouse/ClickHouse/issues/51240): Improve the progress bar for file/s3/hdfs/url table functions by using chunk size from source data and using incremental total size counting in each thread. Fix the progress bar for *Cluster functions. This closes [#47250](https://github.com/ClickHouse/ClickHouse/issues/47250). [#51088](https://github.com/ClickHouse/ClickHouse/pull/51088) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
|
||||
#### Build/Testing/Packaging Improvement
|
||||
* Backported in [#51529](https://github.com/ClickHouse/ClickHouse/issues/51529): Split huge `RUN` in Dockerfile into smaller conditional. Install the necessary tools on demand in the same `RUN` layer, and remove them after that. Upgrade the OS only once at the beginning. Use a modern way to check the signed repository. Downgrade the base repo to ubuntu:20.04 to address the issues on older docker versions. Upgrade golang version to address golang vulnerabilities. [#51504](https://github.com/ClickHouse/ClickHouse/pull/51504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
|
||||
#### Bug Fix (user-visible misbehavior in an official stable release)
|
||||
|
||||
* Fix type of LDAP server params hash in cache entry [#50865](https://github.com/ClickHouse/ClickHouse/pull/50865) ([Julian Maicher](https://github.com/jmaicher)).
|
||||
|
31
docs/changelogs/v23.5.4.25-stable.md
Normal file
31
docs/changelogs/v23.5.4.25-stable.md
Normal file
@ -0,0 +1,31 @@
|
||||
---
|
||||
sidebar_position: 1
|
||||
sidebar_label: 2023
|
||||
---
|
||||
|
||||
# 2023 Changelog
|
||||
|
||||
### ClickHouse release v23.5.4.25-stable (190f962abcf) FIXME as compared to v23.5.3.24-stable (76f54616d3b)
|
||||
|
||||
#### Improvement
|
||||
* Backported in [#51235](https://github.com/ClickHouse/ClickHouse/issues/51235): Improve the progress bar for file/s3/hdfs/url table functions by using chunk size from source data and using incremental total size counting in each thread. Fix the progress bar for *Cluster functions. This closes [#47250](https://github.com/ClickHouse/ClickHouse/issues/47250). [#51088](https://github.com/ClickHouse/ClickHouse/pull/51088) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Backported in [#51255](https://github.com/ClickHouse/ClickHouse/issues/51255): Disable cache setting `do_not_evict_index_and_mark_files` (Was enabled in `23.5`). [#51222](https://github.com/ClickHouse/ClickHouse/pull/51222) ([Kseniia Sumarokova](https://github.com/kssenii)).
|
||||
|
||||
#### Build/Testing/Packaging Improvement
|
||||
* Backported in [#51531](https://github.com/ClickHouse/ClickHouse/issues/51531): Split huge `RUN` in Dockerfile into smaller conditional. Install the necessary tools on demand in the same `RUN` layer, and remove them after that. Upgrade the OS only once at the beginning. Use a modern way to check the signed repository. Downgrade the base repo to ubuntu:20.04 to address the issues on older docker versions. Upgrade golang version to address golang vulnerabilities. [#51504](https://github.com/ClickHouse/ClickHouse/pull/51504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
* Backported in [#51572](https://github.com/ClickHouse/ClickHouse/issues/51572): This a follow-up for [#51504](https://github.com/ClickHouse/ClickHouse/issues/51504), the cleanup was lost during refactoring. [#51564](https://github.com/ClickHouse/ClickHouse/pull/51564) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
|
||||
|
||||
#### Bug Fix (user-visible misbehavior in an official stable release)
|
||||
|
||||
* Query Cache: Try to fix bad cast from ColumnConst to ColumnVector<char8_t> [#50704](https://github.com/ClickHouse/ClickHouse/pull/50704) ([Robert Schulze](https://github.com/rschu1ze)).
|
||||
* Fix type of LDAP server params hash in cache entry [#50865](https://github.com/ClickHouse/ClickHouse/pull/50865) ([Julian Maicher](https://github.com/jmaicher)).
|
||||
* Fallback to parsing big integer from String instead of exception in Parquet format [#50873](https://github.com/ClickHouse/ClickHouse/pull/50873) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Do not apply projection if read-in-order was enabled. [#50923](https://github.com/ClickHouse/ClickHouse/pull/50923) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
|
||||
* Fix race azure blob storage iterator [#50936](https://github.com/ClickHouse/ClickHouse/pull/50936) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)).
|
||||
* Fix ineffective query cache for SELECTs with subqueries [#51132](https://github.com/ClickHouse/ClickHouse/pull/51132) ([Robert Schulze](https://github.com/rschu1ze)).
|
||||
* Fix fuzzer failure in ActionsDAG [#51301](https://github.com/ClickHouse/ClickHouse/pull/51301) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
|
||||
#### NOT FOR CHANGELOG / INSIGNIFICANT
|
||||
|
||||
* Fix ParallelReadBuffer seek [#50820](https://github.com/ClickHouse/ClickHouse/pull/50820) ([Michael Kolupaev](https://github.com/al13n321)).
|
||||
|
@ -44,11 +44,12 @@ Create a table in ClickHouse which allows to read data from Redis:
|
||||
``` sql
|
||||
CREATE TABLE redis_table
|
||||
(
|
||||
`k` String,
|
||||
`m` String,
|
||||
`n` UInt32
|
||||
`key` String,
|
||||
`v1` UInt32,
|
||||
`v2` String,
|
||||
`v3` Float32
|
||||
)
|
||||
ENGINE = Redis('redis1:6379') PRIMARY KEY(k);
|
||||
ENGINE = Redis('redis1:6379') PRIMARY KEY(key);
|
||||
```
|
||||
|
||||
Insert:
|
||||
@ -111,9 +112,16 @@ Flush Redis db asynchronously. Also `Truncate` support SYNC mode.
|
||||
TRUNCATE TABLE redis_table SYNC;
|
||||
```
|
||||
|
||||
Join:
|
||||
|
||||
Join with other tables.
|
||||
|
||||
```
|
||||
SELECT * FROM redis_table JOIN merge_tree_table ON merge_tree_table.key=redis_table.key;
|
||||
```
|
||||
|
||||
## Limitations {#limitations}
|
||||
|
||||
Redis engine also supports scanning queries, such as `where k > xx`, but it has some limitations:
|
||||
1. Scanning query may produce some duplicated keys in a very rare case when it is rehashing. See details in [Redis Scan](https://github.com/redis/redis/blob/e4d183afd33e0b2e6e8d1c79a832f678a04a7886/src/dict.c#L1186-L1269)
|
||||
1. Scanning query may produce some duplicated keys in a very rare case when it is rehashing. See details in [Redis Scan](https://github.com/redis/redis/blob/e4d183afd33e0b2e6e8d1c79a832f678a04a7886/src/dict.c#L1186-L1269).
|
||||
2. During the scanning, keys could be created and deleted, so the resulting dataset can not represent a valid point in time.
|
||||
|
@ -756,6 +756,17 @@ If you perform the `SELECT` query between merges, you may get expired data. To a
|
||||
- [ttl_only_drop_parts](/docs/en/operations/settings/settings.md/#ttl_only_drop_parts) setting
|
||||
|
||||
|
||||
## Disk types
|
||||
|
||||
In addition to local block devices, ClickHouse supports these storage types:
|
||||
- [`s3` for S3 and MinIO](#table_engine-mergetree-s3)
|
||||
- [`gcs` for GCS](/docs/en/integrations/data-ingestion/gcs/index.md/#creating-a-disk)
|
||||
- [`blob_storage_disk` for Azure Blob Storage](#table_engine-mergetree-azure-blob-storage)
|
||||
- [`hdfs` for HDFS](#hdfs-storage)
|
||||
- [`web` for read-only from web](#web-storage)
|
||||
- [`cache` for local caching](/docs/en/operations/storing-data.md/#using-local-cache)
|
||||
- [`s3_plain` for backups to S3](/docs/en/operations/backup#backuprestore-using-an-s3-disk)
|
||||
|
||||
## Using Multiple Block Devices for Data Storage {#table_engine-mergetree-multiple-volumes}
|
||||
|
||||
### Introduction {#introduction}
|
||||
@ -936,6 +947,8 @@ configuration files; all the settings are in the CREATE/ATTACH query.
|
||||
The example uses `type=web`, but any disk type can be configured as dynamic, even Local disk. Local disks require a path argument to be inside the server config parameter `custom_local_disks_base_directory`, which has no default, so set that also when using local disk.
|
||||
:::
|
||||
|
||||
#### Example dynamic web storage
|
||||
|
||||
```sql
|
||||
ATTACH TABLE uk_price_paid UUID 'cf712b4f-2ca8-435c-ac23-c4393efe52f7'
|
||||
(
|
||||
@ -1238,6 +1251,93 @@ Examples of working configurations can be found in integration tests directory (
|
||||
Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use.
|
||||
:::
|
||||
|
||||
## HDFS storage {#hdfs-storage}
|
||||
|
||||
In this sample configuration:
|
||||
- the disk is of type `hdfs`
|
||||
- the data is hosted at `hdfs://hdfs1:9000/clickhouse/`
|
||||
|
||||
```xml
|
||||
<clickhouse>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<hdfs>
|
||||
<type>hdfs</type>
|
||||
<endpoint>hdfs://hdfs1:9000/clickhouse/</endpoint>
|
||||
<skip_access_check>true</skip_access_check>
|
||||
</hdfs>
|
||||
<hdd>
|
||||
<type>local</type>
|
||||
<path>/</path>
|
||||
</hdd>
|
||||
</disks>
|
||||
<policies>
|
||||
<hdfs>
|
||||
<volumes>
|
||||
<main>
|
||||
<disk>hdfs</disk>
|
||||
</main>
|
||||
<external>
|
||||
<disk>hdd</disk>
|
||||
</external>
|
||||
</volumes>
|
||||
</hdfs>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
</clickhouse>
|
||||
```
|
||||
|
||||
## Web storage (read-only) {#web-storage}
|
||||
|
||||
Web storage can be used for read-only purposes. An example use is for hosting sample
|
||||
data, or for migrating data.
|
||||
|
||||
:::tip
|
||||
Storage can also be configured temporarily within a query, if a web dataset is not expected
|
||||
to be used routinely, see [dynamic storage](#dynamic-storage) and skip editing the
|
||||
configuration file.
|
||||
:::
|
||||
|
||||
In this sample configuration:
|
||||
- the disk is of type `web`
|
||||
- the data is hosted at `http://nginx:80/test1/`
|
||||
- a cache on local storage is used
|
||||
|
||||
```xml
|
||||
<clickhouse>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<web>
|
||||
<type>web</type>
|
||||
<endpoint>http://nginx:80/test1/</endpoint>
|
||||
</web>
|
||||
<cached_web>
|
||||
<type>cache</type>
|
||||
<disk>web</disk>
|
||||
<path>cached_web_cache/</path>
|
||||
<max_size>100000000</max_size>
|
||||
</cached_web>
|
||||
</disks>
|
||||
<policies>
|
||||
<web>
|
||||
<volumes>
|
||||
<main>
|
||||
<disk>web</disk>
|
||||
</main>
|
||||
</volumes>
|
||||
</web>
|
||||
<cached_web>
|
||||
<volumes>
|
||||
<main>
|
||||
<disk>cached_web</disk>
|
||||
</main>
|
||||
</volumes>
|
||||
</cached_web>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
</clickhouse>
|
||||
```
|
||||
|
||||
## Virtual Columns {#virtual-columns}
|
||||
|
||||
- `_part` — Name of a part.
|
||||
|
@ -2454,18 +2454,22 @@ In this format, all input data is read to a single value. It is possible to pars
|
||||
The result is output in binary format without delimiters and escaping. If more than one value is output, the format is ambiguous, and it will be impossible to read the data back.
|
||||
|
||||
Below is a comparison of the formats `RawBLOB` and [TabSeparatedRaw](#tabseparatedraw).
|
||||
|
||||
`RawBLOB`:
|
||||
- data is output in binary format, no escaping;
|
||||
- there are no delimiters between values;
|
||||
- no newline at the end of each value.
|
||||
[TabSeparatedRaw] (#tabseparatedraw):
|
||||
|
||||
`TabSeparatedRaw`:
|
||||
- data is output without escaping;
|
||||
- the rows contain values separated by tabs;
|
||||
- there is a line feed after the last value in every row.
|
||||
|
||||
The following is a comparison of the `RawBLOB` and [RowBinary](#rowbinary) formats.
|
||||
|
||||
`RawBLOB`:
|
||||
- String fields are output without being prefixed by length.
|
||||
|
||||
`RowBinary`:
|
||||
- String fields are represented as length in varint format (unsigned [LEB128] (https://en.wikipedia.org/wiki/LEB128)), followed by the bytes of the string.
|
||||
|
||||
|
@ -1602,7 +1602,7 @@ Keys for server/client settings:
|
||||
- requireTLSv1_2 (default: false) – Require a TLSv1.2 connection. Acceptable values: `true`, `false`.
|
||||
- fips (default: false) – Activates OpenSSL FIPS mode. Supported if the library’s OpenSSL version supports FIPS.
|
||||
- privateKeyPassphraseHandler (default: `KeyConsoleHandler`)– Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: `<privateKeyPassphraseHandler>`, `<name>KeyFileHandler</name>`, `<options><password>test</password></options>`, `</privateKeyPassphraseHandler>`.
|
||||
- invalidCertificateHandler (default: `ConsoleCertificateHandler`) – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: `<invalidCertificateHandler> <name>ConsoleCertificateHandler</name> </invalidCertificateHandler>` .
|
||||
- invalidCertificateHandler (default: `RejectCertificateHandler`) – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: `<invalidCertificateHandler> <name>RejectCertificateHandler</name> </invalidCertificateHandler>` .
|
||||
- disableProtocols (default: "") – Protocols that are not allowed to use.
|
||||
- preferServerCiphers (default: false) – Preferred server ciphers on the client.
|
||||
|
||||
|
@ -1322,7 +1322,7 @@ Connection pool size for PostgreSQL table engine and database engine.
|
||||
|
||||
Default value: 16
|
||||
|
||||
## postgresql_connection_pool_size {#postgresql-connection-pool-size}
|
||||
## postgresql_connection_pool_wait_timeout {#postgresql-connection-pool-wait-timeout}
|
||||
|
||||
Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.
|
||||
|
||||
|
@ -97,6 +97,10 @@ Result:
|
||||
|
||||
If you apply this combinator, the aggregate function does not return the resulting value (such as the number of unique values for the [uniq](../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) function), but an intermediate state of the aggregation (for `uniq`, this is the hash table for calculating the number of unique values). This is an `AggregateFunction(...)` that can be used for further processing or stored in a table to finish aggregating later.
|
||||
|
||||
:::note
|
||||
Please notice, that -MapState is not an invariant for the same data due to the fact that order of data in intermediate state can change, though it doesn't impact ingestion of this data.
|
||||
:::
|
||||
|
||||
To work with these states, use:
|
||||
|
||||
- [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) table engine.
|
||||
|
@ -19,8 +19,19 @@ Standard aggregate functions:
|
||||
- [stddevSamp](/docs/en/sql-reference/aggregate-functions/reference/stddevsamp.md)
|
||||
- [varPop](/docs/en/sql-reference/aggregate-functions/reference/varpop.md)
|
||||
- [varSamp](/docs/en/sql-reference/aggregate-functions/reference/varsamp.md)
|
||||
- [corr](./corr.md)
|
||||
- [covarPop](/docs/en/sql-reference/aggregate-functions/reference/covarpop.md)
|
||||
- [covarSamp](/docs/en/sql-reference/aggregate-functions/reference/covarsamp.md)
|
||||
- [entropy](./entropy.md)
|
||||
- [exponentialMovingAverage](./exponentialmovingaverage.md)
|
||||
- [intervalLengthSum](./intervalLengthSum.md)
|
||||
- [kolmogorovSmirnovTest](./kolmogorovsmirnovtest.md)
|
||||
- [mannwhitneyutest](./mannwhitneyutest.md)
|
||||
- [median](./median.md)
|
||||
- [rankCorr](./rankCorr.md)
|
||||
- [sumKahan](./sumkahan.md)
|
||||
- [studentTTest](./studentttest.md)
|
||||
- [welchTTest](./welchttest.md)
|
||||
|
||||
ClickHouse-specific aggregate functions:
|
||||
|
||||
@ -34,12 +45,15 @@ ClickHouse-specific aggregate functions:
|
||||
- [avgWeighted](/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md)
|
||||
- [topK](/docs/en/sql-reference/aggregate-functions/reference/topk.md)
|
||||
- [topKWeighted](/docs/en/sql-reference/aggregate-functions/reference/topkweighted.md)
|
||||
- [deltaSum](./deltasum.md)
|
||||
- [deltaSumTimestamp](./deltasumtimestamp.md)
|
||||
- [groupArray](/docs/en/sql-reference/aggregate-functions/reference/grouparray.md)
|
||||
- [groupArrayLast](/docs/en/sql-reference/aggregate-functions/reference/grouparraylast.md)
|
||||
- [groupUniqArray](/docs/en/sql-reference/aggregate-functions/reference/groupuniqarray.md)
|
||||
- [groupArrayInsertAt](/docs/en/sql-reference/aggregate-functions/reference/grouparrayinsertat.md)
|
||||
- [groupArrayMovingAvg](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingavg.md)
|
||||
- [groupArrayMovingSum](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingsum.md)
|
||||
- [groupArraySample](./grouparraysample.md)
|
||||
- [groupBitAnd](/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md)
|
||||
- [groupBitOr](/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md)
|
||||
- [groupBitXor](/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md)
|
||||
@ -84,3 +98,9 @@ ClickHouse-specific aggregate functions:
|
||||
- [theilsU](./theilsu.md)
|
||||
- [maxIntersections](./maxintersections.md)
|
||||
- [maxIntersectionsPosition](./maxintersectionsposition.md)
|
||||
- [meanZTest](./meanztest.md)
|
||||
- [quantileGK](./quantileGK.md)
|
||||
- [quantileInterpolatedWeighted](./quantileinterpolatedweighted.md)
|
||||
- [sparkBar](./sparkbar.md)
|
||||
- [sumCount](./sumcount.md)
|
||||
|
||||
|
@ -230,13 +230,15 @@ hasAll(set, subset)
|
||||
**Arguments**
|
||||
|
||||
- `set` – Array of any type with a set of elements.
|
||||
- `subset` – Array of any type with elements that should be tested to be a subset of `set`.
|
||||
- `subset` – Array of any type that shares a common supertype with `set` containing elements that should be tested to be a subset of `set`.
|
||||
|
||||
**Return values**
|
||||
|
||||
- `1`, if `set` contains all of the elements from `subset`.
|
||||
- `0`, otherwise.
|
||||
|
||||
Raises an exception `NO_COMMON_TYPE` if the set and subset elements do not share a common supertype.
|
||||
|
||||
**Peculiar properties**
|
||||
|
||||
- An empty array is a subset of any array.
|
||||
@ -253,7 +255,7 @@ hasAll(set, subset)
|
||||
|
||||
`SELECT hasAll(['a', 'b'], ['a'])` returns 1.
|
||||
|
||||
`SELECT hasAll([1], ['a'])` returns 0.
|
||||
`SELECT hasAll([1], ['a'])` raises a `NO_COMMON_TYPE` exception.
|
||||
|
||||
`SELECT hasAll([[1, 2], [3, 4]], [[1, 2], [3, 5]])` returns 0.
|
||||
|
||||
@ -268,13 +270,15 @@ hasAny(array1, array2)
|
||||
**Arguments**
|
||||
|
||||
- `array1` – Array of any type with a set of elements.
|
||||
- `array2` – Array of any type with a set of elements.
|
||||
- `array2` – Array of any type that shares a common supertype with `array1`.
|
||||
|
||||
**Return values**
|
||||
|
||||
- `1`, if `array1` and `array2` have one similar element at least.
|
||||
- `0`, otherwise.
|
||||
|
||||
Raises an exception `NO_COMMON_TYPE` if the array1 and array2 elements do not share a common supertype.
|
||||
|
||||
**Peculiar properties**
|
||||
|
||||
- `Null` processed as a value.
|
||||
@ -288,7 +292,7 @@ hasAny(array1, array2)
|
||||
|
||||
`SELECT hasAny([-128, 1., 512], [1])` returns `1`.
|
||||
|
||||
`SELECT hasAny([[1, 2], [3, 4]], ['a', 'c'])` returns `0`.
|
||||
`SELECT hasAny([[1, 2], [3, 4]], ['a', 'c'])` raises a `NO_COMMON_TYPE` exception.
|
||||
|
||||
`SELECT hasAll([[1, 2], [3, 4]], [[1, 2], [1, 2]])` returns `1`.
|
||||
|
||||
@ -318,6 +322,8 @@ For Example:
|
||||
- `1`, if `array1` contains `array2`.
|
||||
- `0`, otherwise.
|
||||
|
||||
Raises an exception `NO_COMMON_TYPE` if the array1 and array2 elements do not share a common supertype.
|
||||
|
||||
**Peculiar properties**
|
||||
|
||||
- The function will return `1` if `array2` is empty.
|
||||
@ -339,6 +345,9 @@ For Example:
|
||||
`SELECT hasSubstr(['a', 'b' , 'c'], ['a', 'c'])` returns 0.
|
||||
|
||||
`SELECT hasSubstr([[1, 2], [3, 4], [5, 6]], [[1, 2], [3, 4]])` returns 1.
|
||||
i
|
||||
`SELECT hasSubstr([1, 2, NULL, 3, 4], ['a'])` raises a `NO_COMMON_TYPE` exception.
|
||||
|
||||
|
||||
## indexOf(arr, x)
|
||||
|
||||
|
@ -8,7 +8,7 @@ sidebar_label: Nullable
|
||||
|
||||
## isNull
|
||||
|
||||
Returns whether the argument is [NULL](../../sql-reference/syntax.md#null-literal).
|
||||
Returns whether the argument is [NULL](../../sql-reference/syntax.md#null).
|
||||
|
||||
``` sql
|
||||
isNull(x)
|
||||
|
@ -21,6 +21,9 @@ Expressions from `ON` clause and columns from `USING` clause are called “join
|
||||
## Related Content
|
||||
|
||||
- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Part 1](https://clickhouse.com/blog/clickhouse-fully-supports-joins)
|
||||
- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Under the Hood - Part 2](https://clickhouse.com/blog/clickhouse-fully-supports-joins-hash-joins-part2)
|
||||
- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Under the Hood - Part 3](https://clickhouse.com/blog/clickhouse-fully-supports-joins-full-sort-partial-merge-part3)
|
||||
- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Under the Hood - Part 4](https://clickhouse.com/blog/clickhouse-fully-supports-joins-direct-join-part4)
|
||||
|
||||
## Supported Types of JOIN
|
||||
|
||||
|
@ -1067,7 +1067,7 @@ ClickHouse использует потоки из глобального пул
|
||||
- requireTLSv1_2 - Требование соединения TLSv1.2. Допустимые значения: `true`, `false`.
|
||||
- fips - Активация режима OpenSSL FIPS. Поддерживается, если версия OpenSSL, с которой собрана библиотека поддерживает fips.
|
||||
- privateKeyPassphraseHandler - Класс (подкласс PrivateKeyPassphraseHandler)запрашивающий кодовую фразу доступа к секретному ключу. Например, `<privateKeyPassphraseHandler>`, `<name>KeyFileHandler</name>`, `<options><password>test</password></options>`, `</privateKeyPassphraseHandler>`.
|
||||
- invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтверждения не валидных сертификатов. Например, `<invalidCertificateHandler> <name>ConsoleCertificateHandler</name> </invalidCertificateHandler>`.
|
||||
- invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтверждения не валидных сертификатов. Например, `<invalidCertificateHandler> <name>RejectCertificateHandler</name> </invalidCertificateHandler>`.
|
||||
- disableProtocols - Запрещенные к использованию протоколы.
|
||||
- preferServerCiphers - Предпочтение серверных шифров на клиенте.
|
||||
|
||||
|
@ -66,6 +66,10 @@ WITH anySimpleState(number) AS c SELECT toTypeName(c), c FROM numbers(1);
|
||||
|
||||
В случае применения этого комбинатора, агрегатная функция возвращает не готовое значение (например, в случае функции [uniq](reference/uniq.md#agg_function-uniq) — количество уникальных значений), а промежуточное состояние агрегации (например, в случае функции `uniq` — хэш-таблицу для расчёта количества уникальных значений), которое имеет тип `AggregateFunction(...)` и может использоваться для дальнейшей обработки или может быть сохранено в таблицу для последующей доагрегации.
|
||||
|
||||
:::note
|
||||
Промежуточное состояние для -MapState не является инвариантом для одних и тех же исходных данные т.к. порядок данных может меняться. Это не влияет, тем не менее, на загрузку таких данных.
|
||||
:::
|
||||
|
||||
Для работы с промежуточными состояниями предназначены:
|
||||
|
||||
- Движок таблиц [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md).
|
||||
|
@ -466,7 +466,7 @@ SSL客户端/服务器配置。
|
||||
- requireTLSv1_2 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`.
|
||||
- fips – Activates OpenSSL FIPS mode. Supported if the library’s OpenSSL version supports FIPS.
|
||||
- privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: `<privateKeyPassphraseHandler>`, `<name>KeyFileHandler</name>`, `<options><password>test</password></options>`, `</privateKeyPassphraseHandler>`.
|
||||
- invalidCertificateHandler – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: `<invalidCertificateHandler> <name>ConsoleCertificateHandler</name> </invalidCertificateHandler>` .
|
||||
- invalidCertificateHandler – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: `<invalidCertificateHandler> <name>RejectCertificateHandler</name> </invalidCertificateHandler>` .
|
||||
- disableProtocols – Protocols that are not allowed to use.
|
||||
- preferServerCiphers – Preferred server ciphers on the client.
|
||||
|
||||
|
@ -282,6 +282,13 @@
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<disableProtocols>sslv2,sslv3</disableProtocols>
|
||||
<preferServerCiphers>true</preferServerCiphers>
|
||||
|
||||
<invalidCertificateHandler>
|
||||
<!-- The server, in contrast to the client, cannot ask about the certificate interactively.
|
||||
The only reasonable option is to reject.
|
||||
-->
|
||||
<name>RejectCertificateHandler</name>
|
||||
</invalidCertificateHandler>
|
||||
</server>
|
||||
|
||||
<client> <!-- Used for connecting to https dictionary source and secured Zookeeper communication -->
|
||||
|
@ -25,6 +25,7 @@ IAggregateFunction * createWithNumericOrTimeType(const IDataType & argument_type
|
||||
WhichDataType which(argument_type);
|
||||
if (which.idx == TypeIndex::Date) return new AggregateFunctionTemplate<UInt16, Data>(std::forward<TArgs>(args)...);
|
||||
if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTemplate<UInt32, Data>(std::forward<TArgs>(args)...);
|
||||
if (which.idx == TypeIndex::IPv4) return new AggregateFunctionTemplate<IPv4, Data>(std::forward<TArgs>(args)...);
|
||||
return createWithNumericType<AggregateFunctionTemplate, Data, TArgs...>(argument_type, std::forward<TArgs>(args)...);
|
||||
}
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -39,12 +40,22 @@ public:
|
||||
static DataTypePtr createResultType() { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>()); }
|
||||
};
|
||||
|
||||
template <typename HasLimit>
|
||||
class AggregateFunctionGroupUniqArrayIPv4 : public AggregateFunctionGroupUniqArray<DataTypeIPv4::FieldType, HasLimit>
|
||||
{
|
||||
public:
|
||||
explicit AggregateFunctionGroupUniqArrayIPv4(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
|
||||
: AggregateFunctionGroupUniqArray<DataTypeIPv4::FieldType, HasLimit>(argument_type, parameters_, createResultType(), max_elems_) {}
|
||||
static DataTypePtr createResultType() { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeIPv4>()); }
|
||||
};
|
||||
|
||||
template <typename HasLimit, typename ... TArgs>
|
||||
IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, TArgs && ... args)
|
||||
{
|
||||
WhichDataType which(argument_type);
|
||||
if (which.idx == TypeIndex::Date) return new AggregateFunctionGroupUniqArrayDate<HasLimit>(argument_type, std::forward<TArgs>(args)...);
|
||||
else if (which.idx == TypeIndex::DateTime) return new AggregateFunctionGroupUniqArrayDateTime<HasLimit>(argument_type, std::forward<TArgs>(args)...);
|
||||
else if (which.idx == TypeIndex::IPv4) return new AggregateFunctionGroupUniqArrayIPv4<HasLimit>(argument_type, std::forward<TArgs>(args)...);
|
||||
else
|
||||
{
|
||||
/// Check that we can use plain version of AggregateFunctionGroupUniqArrayGeneric
|
||||
|
@ -100,6 +100,10 @@ public:
|
||||
return std::make_shared<AggregateFunctionMap<UInt256>>(nested_function, arguments);
|
||||
case TypeIndex::UUID:
|
||||
return std::make_shared<AggregateFunctionMap<UUID>>(nested_function, arguments);
|
||||
case TypeIndex::IPv4:
|
||||
return std::make_shared<AggregateFunctionMap<IPv4>>(nested_function, arguments);
|
||||
case TypeIndex::IPv6:
|
||||
return std::make_shared<AggregateFunctionMap<IPv6>>(nested_function, arguments);
|
||||
case TypeIndex::FixedString:
|
||||
case TypeIndex::String:
|
||||
return std::make_shared<AggregateFunctionMap<String>>(nested_function, arguments);
|
||||
|
@ -19,7 +19,9 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include "DataTypes/Serializations/ISerialization.h"
|
||||
#include <base/IPv4andIPv6.h>
|
||||
#include "base/types.h"
|
||||
#include <Common/formatIPv6.h>
|
||||
#include <Common/Arena.h>
|
||||
#include "AggregateFunctions/AggregateFunctionFactory.h"
|
||||
|
||||
@ -69,6 +71,31 @@ struct AggregateFunctionMapCombinatorData<String>
|
||||
}
|
||||
};
|
||||
|
||||
/// Specialization for IPv6 - for historical reasons it should be stored as FixedString(16)
|
||||
template <>
|
||||
struct AggregateFunctionMapCombinatorData<IPv6>
|
||||
{
|
||||
struct IPv6Hash
|
||||
{
|
||||
using hash_type = std::hash<IPv6>;
|
||||
using is_transparent = void;
|
||||
|
||||
size_t operator()(const IPv6 & ip) const { return hash_type{}(ip); }
|
||||
};
|
||||
|
||||
using SearchType = IPv6;
|
||||
std::unordered_map<IPv6, AggregateDataPtr, IPv6Hash, std::equal_to<>> merged_maps;
|
||||
|
||||
static void writeKey(const IPv6 & key, WriteBuffer & buf)
|
||||
{
|
||||
writeIPv6Binary(key, buf);
|
||||
}
|
||||
static void readKey(IPv6 & key, ReadBuffer & buf)
|
||||
{
|
||||
readIPv6Binary(key, buf);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename KeyType>
|
||||
class AggregateFunctionMap final
|
||||
: public IAggregateFunctionDataHelper<AggregateFunctionMapCombinatorData<KeyType>, AggregateFunctionMap<KeyType>>
|
||||
@ -147,6 +174,8 @@ public:
|
||||
StringRef key_ref;
|
||||
if (key_type->getTypeId() == TypeIndex::FixedString)
|
||||
key_ref = assert_cast<const ColumnFixedString &>(key_column).getDataAt(offset + i);
|
||||
else if (key_type->getTypeId() == TypeIndex::IPv6)
|
||||
key_ref = assert_cast<const ColumnIPv6 &>(key_column).getDataAt(offset + i);
|
||||
else
|
||||
key_ref = assert_cast<const ColumnString &>(key_column).getDataAt(offset + i);
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
|
||||
|
||||
static inline constexpr UInt64 TOP_K_MAX_SIZE = 0xFFFFFF;
|
||||
@ -60,6 +61,22 @@ public:
|
||||
{}
|
||||
};
|
||||
|
||||
template <bool is_weighted>
|
||||
class AggregateFunctionTopKIPv4 : public AggregateFunctionTopK<DataTypeIPv4::FieldType, is_weighted>
|
||||
{
|
||||
public:
|
||||
using AggregateFunctionTopK<DataTypeIPv4::FieldType, is_weighted>::AggregateFunctionTopK;
|
||||
|
||||
AggregateFunctionTopKIPv4(UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params)
|
||||
: AggregateFunctionTopK<DataTypeIPv4::FieldType, is_weighted>(
|
||||
threshold_,
|
||||
load_factor,
|
||||
argument_types_,
|
||||
params,
|
||||
std::make_shared<DataTypeArray>(std::make_shared<DataTypeIPv4>()))
|
||||
{}
|
||||
};
|
||||
|
||||
|
||||
template <bool is_weighted>
|
||||
IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt64 threshold, UInt64 load_factor, const Array & params)
|
||||
@ -72,6 +89,8 @@ IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt
|
||||
return new AggregateFunctionTopKDate<is_weighted>(threshold, load_factor, argument_types, params);
|
||||
if (which.idx == TypeIndex::DateTime)
|
||||
return new AggregateFunctionTopKDateTime<is_weighted>(threshold, load_factor, argument_types, params);
|
||||
if (which.idx == TypeIndex::IPv4)
|
||||
return new AggregateFunctionTopKIPv4<is_weighted>(threshold, load_factor, argument_types, params);
|
||||
|
||||
/// Check that we can use plain version of AggregateFunctionTopKGeneric
|
||||
if (argument_types[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
@ -60,6 +61,10 @@ createAggregateFunctionUniq(const std::string & name, const DataTypes & argument
|
||||
return std::make_shared<AggregateFunctionUniq<String, Data>>(argument_types);
|
||||
else if (which.isUUID())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeUUID::FieldType, Data>>(argument_types);
|
||||
else if (which.isIPv4())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeIPv4::FieldType, Data>>(argument_types);
|
||||
else if (which.isIPv6())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeIPv6::FieldType, Data>>(argument_types);
|
||||
else if (which.isTuple())
|
||||
{
|
||||
if (use_exact_hash_function)
|
||||
@ -109,6 +114,10 @@ createAggregateFunctionUniq(const std::string & name, const DataTypes & argument
|
||||
return std::make_shared<AggregateFunctionUniq<String, Data<String, is_able_to_parallelize_merge>>>(argument_types);
|
||||
else if (which.isUUID())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeUUID::FieldType, Data<DataTypeUUID::FieldType, is_able_to_parallelize_merge>>>(argument_types);
|
||||
else if (which.isIPv4())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeIPv4::FieldType, Data<DataTypeIPv4::FieldType, is_able_to_parallelize_merge>>>(argument_types);
|
||||
else if (which.isIPv6())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeIPv6::FieldType, Data<DataTypeIPv6::FieldType, is_able_to_parallelize_merge>>>(argument_types);
|
||||
else if (which.isTuple())
|
||||
{
|
||||
if (use_exact_hash_function)
|
||||
|
@ -101,6 +101,18 @@ struct AggregateFunctionUniqHLL12Data<UUID, false>
|
||||
static String getName() { return "uniqHLL12"; }
|
||||
};
|
||||
|
||||
template <>
|
||||
struct AggregateFunctionUniqHLL12Data<IPv6, false>
|
||||
{
|
||||
using Set = HyperLogLogWithSmallSetOptimization<UInt64, 16, 12>;
|
||||
Set set;
|
||||
|
||||
constexpr static bool is_able_to_parallelize_merge = false;
|
||||
constexpr static bool is_variadic = false;
|
||||
|
||||
static String getName() { return "uniqHLL12"; }
|
||||
};
|
||||
|
||||
template <bool is_exact_, bool argument_is_tuple_, bool is_able_to_parallelize_merge_>
|
||||
struct AggregateFunctionUniqHLL12DataForVariadic
|
||||
{
|
||||
@ -155,6 +167,25 @@ struct AggregateFunctionUniqExactData<String, is_able_to_parallelize_merge_>
|
||||
static String getName() { return "uniqExact"; }
|
||||
};
|
||||
|
||||
/// For historical reasons IPv6 is treated as FixedString(16)
|
||||
template <bool is_able_to_parallelize_merge_>
|
||||
struct AggregateFunctionUniqExactData<IPv6, is_able_to_parallelize_merge_>
|
||||
{
|
||||
using Key = UInt128;
|
||||
|
||||
/// When creating, the hash table must be small.
|
||||
using SingleLevelSet = HashSet<Key, UInt128TrivialHash, HashTableGrower<3>, HashTableAllocatorWithStackMemory<sizeof(Key) * (1 << 3)>>;
|
||||
using TwoLevelSet = TwoLevelHashSet<Key, UInt128TrivialHash>;
|
||||
using Set = UniqExactSet<SingleLevelSet, TwoLevelSet>;
|
||||
|
||||
Set set;
|
||||
|
||||
constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_;
|
||||
constexpr static bool is_variadic = false;
|
||||
|
||||
static String getName() { return "uniqExact"; }
|
||||
};
|
||||
|
||||
template <bool is_exact_, bool argument_is_tuple_, bool is_able_to_parallelize_merge_>
|
||||
struct AggregateFunctionUniqExactDataForVariadic : AggregateFunctionUniqExactData<String, is_able_to_parallelize_merge_>
|
||||
{
|
||||
@ -248,27 +279,22 @@ struct Adder
|
||||
AggregateFunctionUniqUniquesHashSetData> || std::is_same_v<Data, AggregateFunctionUniqHLL12Data<T, Data::is_able_to_parallelize_merge>>)
|
||||
{
|
||||
const auto & column = *columns[0];
|
||||
if constexpr (!std::is_same_v<T, String>)
|
||||
if constexpr (std::is_same_v<T, String> || std::is_same_v<T, IPv6>)
|
||||
{
|
||||
StringRef value = column.getDataAt(row_num);
|
||||
data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size));
|
||||
}
|
||||
else
|
||||
{
|
||||
using ValueType = typename decltype(data.set)::value_type;
|
||||
const auto & value = assert_cast<const ColumnVector<T> &>(column).getElement(row_num);
|
||||
data.set.insert(static_cast<ValueType>(AggregateFunctionUniqTraits<T>::hash(value)));
|
||||
}
|
||||
else
|
||||
{
|
||||
StringRef value = column.getDataAt(row_num);
|
||||
data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size));
|
||||
}
|
||||
}
|
||||
else if constexpr (std::is_same_v<Data, AggregateFunctionUniqExactData<T, Data::is_able_to_parallelize_merge>>)
|
||||
{
|
||||
const auto & column = *columns[0];
|
||||
if constexpr (!std::is_same_v<T, String>)
|
||||
{
|
||||
data.set.template insert<const T &, use_single_level_hash_table>(
|
||||
assert_cast<const ColumnVector<T> &>(column).getData()[row_num]);
|
||||
}
|
||||
else
|
||||
if constexpr (std::is_same_v<T, String> || std::is_same_v<T, IPv6>)
|
||||
{
|
||||
StringRef value = column.getDataAt(row_num);
|
||||
|
||||
@ -279,6 +305,11 @@ struct Adder
|
||||
|
||||
data.set.template insert<const UInt128 &, use_single_level_hash_table>(key);
|
||||
}
|
||||
else
|
||||
{
|
||||
data.set.template insert<const T &, use_single_level_hash_table>(
|
||||
assert_cast<const ColumnVector<T> &>(column).getData()[row_num]);
|
||||
}
|
||||
}
|
||||
#if USE_DATASKETCHES
|
||||
else if constexpr (std::is_same_v<Data, AggregateFunctionUniqThetaData>)
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeIPv4andIPv6.h>
|
||||
|
||||
#include <functional>
|
||||
|
||||
@ -60,6 +61,10 @@ namespace
|
||||
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<String>>(argument_types, params);
|
||||
else if (which.isUUID())
|
||||
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeUUID::FieldType>>(argument_types, params);
|
||||
else if (which.isIPv4())
|
||||
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeIPv4::FieldType>>(argument_types, params);
|
||||
else if (which.isIPv6())
|
||||
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeIPv6::FieldType>>(argument_types, params);
|
||||
else if (which.isTuple())
|
||||
{
|
||||
if (use_exact_hash_function)
|
||||
|
@ -119,6 +119,10 @@ struct AggregateFunctionUniqCombinedData<String, K, HashValueType> : public Aggr
|
||||
{
|
||||
};
|
||||
|
||||
template <UInt8 K, typename HashValueType>
|
||||
struct AggregateFunctionUniqCombinedData<IPv6, K, HashValueType> : public AggregateFunctionUniqCombinedDataWithKey<UInt64 /*always*/, K>
|
||||
{
|
||||
};
|
||||
|
||||
template <typename T, UInt8 K, typename HashValueType>
|
||||
class AggregateFunctionUniqCombined final
|
||||
@ -141,16 +145,16 @@ public:
|
||||
|
||||
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
|
||||
{
|
||||
if constexpr (!std::is_same_v<T, String>)
|
||||
{
|
||||
const auto & value = assert_cast<const ColumnVector<T> &>(*columns[0]).getElement(row_num);
|
||||
this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits<T, HashValueType>::hash(value));
|
||||
}
|
||||
else
|
||||
if constexpr (std::is_same_v<T, String> || std::is_same_v<T, IPv6>)
|
||||
{
|
||||
StringRef value = columns[0]->getDataAt(row_num);
|
||||
this->data(place).set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size));
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & value = assert_cast<const ColumnVector<T> &>(*columns[0]).getElement(row_num);
|
||||
this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits<T, HashValueType>::hash(value));
|
||||
}
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
|
||||
|
@ -2383,7 +2383,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con
|
||||
|
||||
auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
|
||||
auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context);
|
||||
|
||||
auto result = std::make_shared<TableNode>(std::move(storage), std::move(storage_lock), std::move(storage_snapshot));
|
||||
if (is_temporary_table)
|
||||
result->setTemporaryTableName(table_name);
|
||||
@ -5210,14 +5209,26 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
||||
const auto & second_argument_constant_literal = second_argument_constant_node->getValue();
|
||||
const auto & second_argument_constant_type = second_argument_constant_node->getResultType();
|
||||
|
||||
auto set = makeSetForConstantValue(first_argument_constant_type,
|
||||
const auto & settings = scope.context->getSettingsRef();
|
||||
|
||||
auto result_block = getSetElementsForConstantValue(first_argument_constant_type,
|
||||
second_argument_constant_literal,
|
||||
second_argument_constant_type,
|
||||
scope.context->getSettingsRef());
|
||||
settings.transform_null_in);
|
||||
|
||||
SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode};
|
||||
|
||||
auto set = std::make_shared<Set>(size_limits_for_set, 0, settings.transform_null_in);
|
||||
|
||||
set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName());
|
||||
set->insertFromBlock(result_block.getColumnsWithTypeAndName());
|
||||
set->finishInsert();
|
||||
|
||||
auto future_set = std::make_shared<FutureSetFromStorage>(std::move(set));
|
||||
|
||||
/// Create constant set column for constant folding
|
||||
|
||||
auto column_set = ColumnSet::create(1, FutureSet(std::move(set)));
|
||||
auto column_set = ColumnSet::create(1, std::move(future_set));
|
||||
argument_columns[1].column = ColumnConst::create(std::move(column_set), 1);
|
||||
}
|
||||
|
||||
|
@ -118,7 +118,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes &
|
||||
|
||||
}
|
||||
|
||||
SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, const Settings & settings)
|
||||
Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in)
|
||||
{
|
||||
DataTypes set_element_types = {expression_type};
|
||||
const auto * lhs_tuple_type = typeid_cast<const DataTypeTuple *>(expression_type.get());
|
||||
@ -135,9 +135,6 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field
|
||||
size_t lhs_type_depth = getCompoundTypeDepth(*expression_type);
|
||||
size_t rhs_type_depth = getCompoundTypeDepth(*value_type);
|
||||
|
||||
SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode};
|
||||
bool tranform_null_in = settings.transform_null_in;
|
||||
|
||||
Block result_block;
|
||||
|
||||
if (lhs_type_depth == rhs_type_depth)
|
||||
@ -145,7 +142,7 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field
|
||||
/// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc.
|
||||
|
||||
Array array{value};
|
||||
result_block = createBlockFromCollection(array, set_element_types, tranform_null_in);
|
||||
result_block = createBlockFromCollection(array, set_element_types, transform_null_in);
|
||||
}
|
||||
else if (lhs_type_depth + 1 == rhs_type_depth)
|
||||
{
|
||||
@ -154,9 +151,9 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field
|
||||
WhichDataType rhs_which_type(value_type);
|
||||
|
||||
if (rhs_which_type.isArray())
|
||||
result_block = createBlockFromCollection(value.get<const Array &>(), set_element_types, tranform_null_in);
|
||||
result_block = createBlockFromCollection(value.get<const Array &>(), set_element_types, transform_null_in);
|
||||
else if (rhs_which_type.isTuple())
|
||||
result_block = createBlockFromCollection(value.get<const Tuple &>(), set_element_types, tranform_null_in);
|
||||
result_block = createBlockFromCollection(value.get<const Tuple &>(), set_element_types, transform_null_in);
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Unsupported type at the right-side of IN. Expected Array or Tuple. Actual {}",
|
||||
@ -170,13 +167,7 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field
|
||||
value_type->getName());
|
||||
}
|
||||
|
||||
auto set = std::make_shared<Set>(size_limits_for_set, true /*fill_set_elements*/, tranform_null_in);
|
||||
|
||||
set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName());
|
||||
set->insertFromBlock(result_block.getColumnsWithTypeAndName());
|
||||
set->finishInsert();
|
||||
|
||||
return set;
|
||||
return result_block;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,19 +12,12 @@ namespace DB
|
||||
class Set;
|
||||
using SetPtr = std::shared_ptr<Set>;
|
||||
|
||||
/** Make set for constant part of IN subquery.
|
||||
/** Get set elements for constant part of IN subquery.
|
||||
* Throws exception if parameters are not valid for IN function.
|
||||
*
|
||||
* Example: SELECT id FROM test_table WHERE id IN (1, 2, 3, 4);
|
||||
* Example: SELECT id FROM test_table WHERE id IN ((1, 2), (3, 4));
|
||||
*
|
||||
* @param expression_type - type of first argument of function IN.
|
||||
* @param value - constant value of second argument of function IN.
|
||||
* @param value_type - type of second argument of function IN.
|
||||
* @param settings - query settings.
|
||||
*
|
||||
* @return SetPtr for constant value.
|
||||
*/
|
||||
SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, const Settings & settings);
|
||||
Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in);
|
||||
|
||||
}
|
||||
|
@ -1175,16 +1175,12 @@ ProfileInfo Connection::receiveProfileInfo() const
|
||||
|
||||
ParallelReadRequest Connection::receiveParallelReadRequest() const
|
||||
{
|
||||
ParallelReadRequest request;
|
||||
request.deserialize(*in);
|
||||
return request;
|
||||
return ParallelReadRequest::deserialize(*in);
|
||||
}
|
||||
|
||||
InitialAllRangesAnnouncement Connection::receiveInitialParallelReadAnnounecement() const
|
||||
{
|
||||
InitialAllRangesAnnouncement announcement;
|
||||
announcement.deserialize(*in);
|
||||
return announcement;
|
||||
return InitialAllRangesAnnouncement::deserialize(*in);
|
||||
}
|
||||
|
||||
|
||||
|
@ -16,6 +16,10 @@
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
#include <optional>
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -34,9 +38,9 @@ struct Packet
|
||||
ProfileInfo profile_info;
|
||||
std::vector<UUID> part_uuids;
|
||||
|
||||
InitialAllRangesAnnouncement announcement;
|
||||
ParallelReadRequest request;
|
||||
ParallelReadResponse response;
|
||||
/// The part of parallel replicas protocol
|
||||
std::optional<InitialAllRangesAnnouncement> announcement;
|
||||
std::optional<ParallelReadRequest> request;
|
||||
|
||||
std::string server_timezone;
|
||||
|
||||
|
@ -21,7 +21,7 @@ class ColumnSet final : public COWHelper<IColumnDummy, ColumnSet>
|
||||
private:
|
||||
friend class COWHelper<IColumnDummy, ColumnSet>;
|
||||
|
||||
ColumnSet(size_t s_, FutureSet data_) : data(std::move(data_)) { s = s_; }
|
||||
ColumnSet(size_t s_, FutureSetPtr data_) : data(std::move(data_)) { s = s_; }
|
||||
ColumnSet(const ColumnSet &) = default;
|
||||
|
||||
public:
|
||||
@ -29,13 +29,13 @@ public:
|
||||
TypeIndex getDataType() const override { return TypeIndex::Set; }
|
||||
MutableColumnPtr cloneDummy(size_t s_) const override { return ColumnSet::create(s_, data); }
|
||||
|
||||
ConstSetPtr getData() const { if (!data.isReady()) return nullptr; return data.get(); }
|
||||
FutureSetPtr getData() const { return data; }
|
||||
|
||||
// Used only for debugging, making it DUMPABLE
|
||||
Field operator[](size_t) const override { return {}; }
|
||||
|
||||
private:
|
||||
FutureSet data;
|
||||
FutureSetPtr data;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -21,12 +21,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
|
||||
/** Variants for searching a substring in a string.
|
||||
* In most cases, performance is less than Volnitsky (see Volnitsky.h).
|
||||
*/
|
||||
@ -811,15 +805,22 @@ class TokenSearcher : public StringSearcherBase
|
||||
size_t needle_size;
|
||||
|
||||
public:
|
||||
|
||||
template <typename CharT>
|
||||
requires (sizeof(CharT) == 1)
|
||||
static bool isValidNeedle(const CharT * needle_, size_t needle_size_)
|
||||
{
|
||||
return std::none_of(needle_, needle_ + needle_size_, isTokenSeparator);
|
||||
}
|
||||
|
||||
template <typename CharT>
|
||||
requires (sizeof(CharT) == 1)
|
||||
TokenSearcher(const CharT * needle_, size_t needle_size_)
|
||||
: searcher(needle_, needle_size_)
|
||||
, needle_size(needle_size_)
|
||||
{
|
||||
if (std::any_of(needle_, needle_ + needle_size_, isTokenSeparator))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle must not contain whitespace or separator characters");
|
||||
|
||||
/// The caller is responsible for calling isValidNeedle()
|
||||
chassert(isValidNeedle(needle_, needle_size_));
|
||||
}
|
||||
|
||||
template <typename CharT>
|
||||
@ -880,6 +881,7 @@ using ASCIICaseSensitiveStringSearcher = impl::StringSearcher<true, true>;
|
||||
using ASCIICaseInsensitiveStringSearcher = impl::StringSearcher<false, true>;
|
||||
using UTF8CaseSensitiveStringSearcher = impl::StringSearcher<true, false>;
|
||||
using UTF8CaseInsensitiveStringSearcher = impl::StringSearcher<false, false>;
|
||||
|
||||
using ASCIICaseSensitiveTokenSearcher = impl::TokenSearcher<ASCIICaseSensitiveStringSearcher>;
|
||||
using ASCIICaseInsensitiveTokenSearcher = impl::TokenSearcher<ASCIICaseInsensitiveStringSearcher>;
|
||||
|
||||
|
@ -406,8 +406,7 @@ public:
|
||||
/// And also adding from the end guarantees that we will find first occurrence because we will lookup bigger offsets first.
|
||||
for (auto i = static_cast<ssize_t>(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i)
|
||||
{
|
||||
bool ok = VolnitskyTraits::putNGram<CaseSensitive, ASCII>(
|
||||
needle + i, static_cast<int>(i + 1), needle, needle_size, callback);
|
||||
bool ok = VolnitskyTraits::putNGram<CaseSensitive, ASCII>(needle + i, static_cast<int>(i + 1), needle, needle_size, callback);
|
||||
|
||||
/** `putNGramUTF8CaseInsensitive` does not work if characters with lower and upper cases
|
||||
* are represented by different number of bytes or code points.
|
||||
|
@ -49,8 +49,8 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
|
||||
|
||||
/// TODO mess up of endianness in error message.
|
||||
message << "Checksum doesn't match: corrupted data."
|
||||
" Reference: " + getHexUIntLowercase(expected_checksum.first) + getHexUIntLowercase(expected_checksum.second)
|
||||
+ ". Actual: " + getHexUIntLowercase(calculated_checksum.first) + getHexUIntLowercase(calculated_checksum.second)
|
||||
" Reference: " + getHexUIntLowercase(expected_checksum.high64) + getHexUIntLowercase(expected_checksum.low64)
|
||||
+ ". Actual: " + getHexUIntLowercase(calculated_checksum.high64) + getHexUIntLowercase(calculated_checksum.low64)
|
||||
+ ". Size of compressed block: " + toString(size);
|
||||
|
||||
const char * message_hardware_failure = "This is most likely due to hardware failure. "
|
||||
@ -95,8 +95,8 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
|
||||
}
|
||||
|
||||
/// Check if the difference caused by single bit flip in stored checksum.
|
||||
size_t difference = std::popcount(expected_checksum.first ^ calculated_checksum.first)
|
||||
+ std::popcount(expected_checksum.second ^ calculated_checksum.second);
|
||||
size_t difference = std::popcount(expected_checksum.low64 ^ calculated_checksum.low64)
|
||||
+ std::popcount(expected_checksum.high64 ^ calculated_checksum.high64);
|
||||
|
||||
if (difference == 1)
|
||||
{
|
||||
@ -194,8 +194,8 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed,
|
||||
{
|
||||
Checksum checksum;
|
||||
ReadBufferFromMemory checksum_in(own_compressed_buffer.data(), sizeof(checksum));
|
||||
readBinaryLittleEndian(checksum.first, checksum_in);
|
||||
readBinaryLittleEndian(checksum.second, checksum_in);
|
||||
readBinaryLittleEndian(checksum.low64, checksum_in);
|
||||
readBinaryLittleEndian(checksum.high64, checksum_in);
|
||||
|
||||
validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum);
|
||||
}
|
||||
@ -238,8 +238,8 @@ size_t CompressedReadBufferBase::readCompressedDataBlockForAsynchronous(size_t &
|
||||
{
|
||||
Checksum checksum;
|
||||
ReadBufferFromMemory checksum_in(own_compressed_buffer.data(), sizeof(checksum));
|
||||
readBinaryLittleEndian(checksum.first, checksum_in);
|
||||
readBinaryLittleEndian(checksum.second, checksum_in);
|
||||
readBinaryLittleEndian(checksum.low64, checksum_in);
|
||||
readBinaryLittleEndian(checksum.high64, checksum_in);
|
||||
|
||||
validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum);
|
||||
}
|
||||
|
@ -38,8 +38,8 @@ void CompressedWriteBuffer::nextImpl()
|
||||
|
||||
CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(out_compressed_ptr, compressed_size);
|
||||
|
||||
writeBinaryLittleEndian(checksum.first, out);
|
||||
writeBinaryLittleEndian(checksum.second, out);
|
||||
writeBinaryLittleEndian(checksum.low64, out);
|
||||
writeBinaryLittleEndian(checksum.high64, out);
|
||||
|
||||
out.position() += compressed_size;
|
||||
}
|
||||
@ -50,8 +50,8 @@ void CompressedWriteBuffer::nextImpl()
|
||||
|
||||
CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer.data(), compressed_size);
|
||||
|
||||
writeBinaryLittleEndian(checksum.first, out);
|
||||
writeBinaryLittleEndian(checksum.second, out);
|
||||
writeBinaryLittleEndian(checksum.low64, out);
|
||||
writeBinaryLittleEndian(checksum.high64, out);
|
||||
|
||||
out.write(compressed_buffer.data(), compressed_size);
|
||||
}
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Common/logger_useful.h>
|
||||
#include "libaccel_config.h"
|
||||
#include <Common/MemorySanitizer.h>
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -34,6 +35,7 @@ DeflateQplJobHWPool::DeflateQplJobHWPool()
|
||||
// loop all configured workqueue size to get maximum job number.
|
||||
accfg_ctx * ctx_ptr = nullptr;
|
||||
auto ctx_status = accfg_new(&ctx_ptr);
|
||||
SCOPE_EXIT({ accfg_unref(ctx_ptr); });
|
||||
if (ctx_status == 0)
|
||||
{
|
||||
auto * dev_ptr = accfg_device_get_first(ctx_ptr);
|
||||
|
@ -141,7 +141,7 @@ class IColumn;
|
||||
M(Bool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \
|
||||
M(Bool, allow_suspicious_fixed_string_types, false, "In CREATE TABLE statement allows creating columns of type FixedString(n) with n > 256. FixedString with length >= 256 is suspicious and most likely indicates misusage", 0) \
|
||||
M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \
|
||||
M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \
|
||||
M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \
|
||||
M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
|
||||
M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code. This feature has a bug and should not be used.", 0) \
|
||||
M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \
|
||||
|
@ -27,7 +27,7 @@ namespace DB
|
||||
|
||||
using UUID = StrongTypedef<UInt128, struct UUIDTag>;
|
||||
|
||||
using IPv4 = StrongTypedef<UInt32, struct IPv4Tag>;
|
||||
struct IPv4;
|
||||
|
||||
struct IPv6;
|
||||
|
||||
|
@ -69,7 +69,7 @@ void DataTypeMap::assertKeyType() const
|
||||
if (!checkKeyType(key_type))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Type of Map key must be a type, that can be represented by integer "
|
||||
"or String or FixedString (possibly LowCardinality) or UUID,"
|
||||
"or String or FixedString (possibly LowCardinality) or UUID or IPv6,"
|
||||
" but {} given", key_type->getName());
|
||||
}
|
||||
|
||||
@ -120,6 +120,7 @@ bool DataTypeMap::checkKeyType(DataTypePtr key_type)
|
||||
else if (!key_type->isValueRepresentedByInteger()
|
||||
&& !isStringOrFixedString(*key_type)
|
||||
&& !WhichDataType(key_type).isNothing()
|
||||
&& !WhichDataType(key_type).isIPv6()
|
||||
&& !WhichDataType(key_type).isUUID())
|
||||
{
|
||||
return false;
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
bool isParametric() const override { return true; }
|
||||
|
||||
// Used for expressions analysis.
|
||||
MutableColumnPtr createColumn() const override { return ColumnSet::create(0, FutureSet{}); }
|
||||
MutableColumnPtr createColumn() const override { return ColumnSet::create(0, nullptr); }
|
||||
|
||||
// Used only for debugging, making it DUMPABLE
|
||||
Field getDefault() const override { return Tuple(); }
|
||||
|
@ -507,9 +507,6 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext()
|
||||
current_file_segment->use();
|
||||
implementation_buffer = getImplementationBuffer(*current_file_segment);
|
||||
|
||||
if (read_type == ReadType::CACHED)
|
||||
current_file_segment->incrementHitsCount();
|
||||
|
||||
LOG_TEST(
|
||||
log, "New segment range: {}, old range: {}",
|
||||
current_file_segment->range().toString(), completed_range.toString());
|
||||
@ -852,9 +849,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
else
|
||||
{
|
||||
implementation_buffer = getImplementationBuffer(file_segments->front());
|
||||
|
||||
if (read_type == ReadType::CACHED)
|
||||
file_segments->front().incrementHitsCount();
|
||||
file_segments->front().use();
|
||||
}
|
||||
|
||||
chassert(!internal_buffer.empty());
|
||||
@ -875,10 +870,11 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
|
||||
LOG_TEST(
|
||||
log,
|
||||
"Current read type: {}, read offset: {}, impl offset: {}, file segment: {}",
|
||||
"Current read type: {}, read offset: {}, impl offset: {}, impl position: {}, file segment: {}",
|
||||
toString(read_type),
|
||||
file_offset_of_buffer_end,
|
||||
implementation_buffer->getFileOffsetOfBufferEnd(),
|
||||
implementation_buffer->getPosition(),
|
||||
file_segment.getInfoForLog());
|
||||
|
||||
chassert(current_read_range.left <= file_offset_of_buffer_end);
|
||||
@ -937,7 +933,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
// We don't support implementation_buffer implementations that use nextimpl_working_buffer_offset.
|
||||
chassert(implementation_buffer->position() == implementation_buffer->buffer().begin());
|
||||
|
||||
size = implementation_buffer->buffer().size();
|
||||
if (result)
|
||||
size = implementation_buffer->buffer().size();
|
||||
|
||||
LOG_TEST(
|
||||
log,
|
||||
@ -951,15 +948,21 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size);
|
||||
ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed);
|
||||
|
||||
const size_t new_file_offset = file_offset_of_buffer_end + size;
|
||||
const size_t file_segment_write_offset = file_segment.getCurrentWriteOffset(true);
|
||||
if (new_file_offset > file_segment.range().right + 1 || new_file_offset > file_segment_write_offset)
|
||||
if (result)
|
||||
{
|
||||
auto file_segment_path = file_segment.getPathInLocalCache();
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Read unexpected size. File size: {}, file path: {}, file segment info: {}",
|
||||
fs::file_size(file_segment_path), file_segment_path, file_segment.getInfoForLog());
|
||||
const size_t new_file_offset = file_offset_of_buffer_end + size;
|
||||
const size_t file_segment_write_offset = file_segment.getCurrentWriteOffset(true);
|
||||
if (new_file_offset > file_segment.range().right + 1 || new_file_offset > file_segment_write_offset)
|
||||
{
|
||||
auto file_segment_path = file_segment.getPathInLocalCache();
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR, "Read unexpected size. "
|
||||
"File size: {}, file segment path: {}, impl size: {}, impl path: {}"
|
||||
"file segment info: {}",
|
||||
fs::file_size(file_segment_path), file_segment_path,
|
||||
implementation_buffer->getFileSize(), implementation_buffer->getFileName(),
|
||||
file_segment.getInfoForLog());
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
|
@ -14,6 +14,7 @@ public:
|
||||
virtual bool isValid() = 0;
|
||||
virtual RelativePathWithMetadata current() = 0;
|
||||
virtual RelativePathsWithMetadata currentBatch() = 0;
|
||||
virtual std::optional<RelativePathsWithMetadata> getCurrrentBatchAndScheduleNext() = 0;
|
||||
virtual size_t getAccumulatedSize() const = 0;
|
||||
|
||||
virtual ~IObjectStorageIterator() = default;
|
||||
@ -53,6 +54,11 @@ public:
|
||||
return batch;
|
||||
}
|
||||
|
||||
virtual std::optional<RelativePathsWithMetadata> getCurrrentBatchAndScheduleNext() override
|
||||
{
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
size_t getAccumulatedSize() const override
|
||||
{
|
||||
return batch.size();
|
||||
|
@ -100,6 +100,22 @@ RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch()
|
||||
return current_batch;
|
||||
}
|
||||
|
||||
std::optional<RelativePathsWithMetadata> IObjectStorageIteratorAsync::getCurrrentBatchAndScheduleNext()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (!is_initialized)
|
||||
nextBatch();
|
||||
|
||||
if (current_batch_iterator != current_batch.end())
|
||||
{
|
||||
auto temp_current_batch = current_batch;
|
||||
nextBatch();
|
||||
return temp_current_batch;
|
||||
}
|
||||
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
size_t IObjectStorageIteratorAsync::getAccumulatedSize() const
|
||||
{
|
||||
return accumulated_size.load(std::memory_order_relaxed);
|
||||
|
@ -27,6 +27,7 @@ public:
|
||||
RelativePathWithMetadata current() override;
|
||||
RelativePathsWithMetadata currentBatch() override;
|
||||
size_t getAccumulatedSize() const override;
|
||||
std::optional<RelativePathsWithMetadata> getCurrrentBatchAndScheduleNext() override;
|
||||
|
||||
~IObjectStorageIteratorAsync() override
|
||||
{
|
||||
@ -48,7 +49,7 @@ protected:
|
||||
bool is_initialized{false};
|
||||
bool is_finished{false};
|
||||
|
||||
mutable std::mutex mutex;
|
||||
mutable std::recursive_mutex mutex;
|
||||
ThreadPool list_objects_pool;
|
||||
ThreadPoolCallbackRunner<BatchAndHasNext> list_objects_scheduler;
|
||||
std::future<BatchAndHasNext> outcome_future;
|
||||
|
@ -322,7 +322,6 @@ struct ToTimeImpl
|
||||
{
|
||||
throwDateTimeIsNotSupported(name);
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ToDateImpl;
|
||||
};
|
||||
@ -394,7 +393,6 @@ struct ToStartOfSecondImpl
|
||||
{
|
||||
throwDateTimeIsNotSupported(name);
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -442,7 +440,6 @@ struct ToStartOfMillisecondImpl
|
||||
{
|
||||
throwDateTimeIsNotSupported(name);
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -486,7 +483,6 @@ struct ToStartOfMicrosecondImpl
|
||||
{
|
||||
throwDateTimeIsNotSupported(name);
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -524,7 +520,6 @@ struct ToStartOfNanosecondImpl
|
||||
{
|
||||
throwDateTimeIsNotSupported(name);
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -723,28 +718,6 @@ struct ToYearImpl
|
||||
return time_zone.toYear(DayNum(d));
|
||||
}
|
||||
|
||||
static inline constexpr bool hasPreimage() { return true; }
|
||||
|
||||
static inline RangeOrNull getPreimage(const IDataType & type, const Field & point)
|
||||
{
|
||||
if (point.getType() != Field::Types::UInt64) return std::nullopt;
|
||||
|
||||
auto year = point.get<UInt64>();
|
||||
if (year < DATE_LUT_MIN_YEAR || year >= DATE_LUT_MAX_YEAR) return std::nullopt;
|
||||
|
||||
const DateLUTImpl & date_lut = DateLUT::instance();
|
||||
|
||||
auto start_time = date_lut.makeDateTime(year, 1, 1, 0, 0, 0);
|
||||
auto end_time = date_lut.addYears(start_time, 1);
|
||||
|
||||
if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type))
|
||||
return {std::make_pair(Field(start_time), Field(end_time))};
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64",
|
||||
type.getName(), name);
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
@ -818,7 +791,6 @@ struct ToQuarterImpl
|
||||
{
|
||||
return time_zone.toQuarter(DayNum(d));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ToStartOfYearImpl;
|
||||
};
|
||||
@ -843,7 +815,6 @@ struct ToMonthImpl
|
||||
{
|
||||
return time_zone.toMonth(DayNum(d));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ToStartOfYearImpl;
|
||||
};
|
||||
@ -869,7 +840,6 @@ struct ToDayOfMonthImpl
|
||||
return time_zone.toDayOfMonth(DayNum(d));
|
||||
}
|
||||
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
using FactorTransform = ToStartOfMonthImpl;
|
||||
};
|
||||
|
||||
@ -917,7 +887,6 @@ struct ToDayOfYearImpl
|
||||
{
|
||||
return time_zone.toDayOfYear(DayNum(d));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ToStartOfYearImpl;
|
||||
};
|
||||
@ -942,7 +911,6 @@ struct ToHourImpl
|
||||
{
|
||||
throwDateTimeIsNotSupported(name);
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ToDateImpl;
|
||||
};
|
||||
@ -971,7 +939,6 @@ struct TimezoneOffsetImpl
|
||||
throwDateTimeIsNotSupported(name);
|
||||
}
|
||||
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
using FactorTransform = ToTimeImpl;
|
||||
};
|
||||
|
||||
@ -995,7 +962,6 @@ struct ToMinuteImpl
|
||||
{
|
||||
throwDateTimeIsNotSupported(name);
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ToStartOfHourImpl;
|
||||
};
|
||||
@ -1020,7 +986,6 @@ struct ToSecondImpl
|
||||
{
|
||||
throwDateTimeIsNotSupported(name);
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ToStartOfMinuteImpl;
|
||||
};
|
||||
@ -1045,7 +1010,6 @@ struct ToISOYearImpl
|
||||
{
|
||||
return time_zone.toISOYear(DayNum(d));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -1102,7 +1066,6 @@ struct ToISOWeekImpl
|
||||
{
|
||||
return time_zone.toISOWeek(DayNum(d));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ToISOYearImpl;
|
||||
};
|
||||
@ -1145,7 +1108,6 @@ struct ToRelativeYearNumImpl
|
||||
{
|
||||
return time_zone.toYear(DayNum(d));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -1177,7 +1139,6 @@ struct ToRelativeQuarterNumImpl
|
||||
{
|
||||
return time_zone.toRelativeQuarterNum(DayNum(d));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -1209,7 +1170,6 @@ struct ToRelativeMonthNumImpl
|
||||
{
|
||||
return time_zone.toRelativeMonthNum(DayNum(d));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -1241,7 +1201,6 @@ struct ToRelativeWeekNumImpl
|
||||
{
|
||||
return time_zone.toRelativeWeekNum(DayNum(d));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -1273,7 +1232,6 @@ struct ToRelativeDayNumImpl
|
||||
{
|
||||
return static_cast<DayNum>(d);
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -1311,7 +1269,6 @@ struct ToRelativeHourNumImpl
|
||||
else
|
||||
return static_cast<UInt32>(time_zone.toRelativeHourNum(DayNum(d)));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -1343,7 +1300,6 @@ struct ToRelativeMinuteNumImpl
|
||||
{
|
||||
return static_cast<UInt32>(time_zone.toRelativeMinuteNum(DayNum(d)));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -1372,7 +1328,6 @@ struct ToRelativeSecondNumImpl
|
||||
{
|
||||
return static_cast<UInt32>(time_zone.fromDayNum(DayNum(d)));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -1397,31 +1352,6 @@ struct ToYYYYMMImpl
|
||||
{
|
||||
return time_zone.toNumYYYYMM(DayNum(d));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return true; }
|
||||
|
||||
static inline RangeOrNull getPreimage(const IDataType & type, const Field & point)
|
||||
{
|
||||
if (point.getType() != Field::Types::UInt64) return std::nullopt;
|
||||
|
||||
auto year_month = point.get<UInt64>();
|
||||
auto year = year_month / 100;
|
||||
auto month = year_month % 100;
|
||||
|
||||
if (year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || (year == DATE_LUT_MAX_YEAR && month == 12))
|
||||
return std::nullopt;
|
||||
|
||||
const DateLUTImpl & date_lut = DateLUT::instance();
|
||||
|
||||
auto start_time = date_lut.makeDateTime(year, month, 1, 0, 0, 0);
|
||||
auto end_time = date_lut.addMonths(start_time, 1);
|
||||
|
||||
if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type))
|
||||
return {std::make_pair(Field(start_time), Field(end_time))};
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64",
|
||||
type.getName(), name);
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -1446,7 +1376,6 @@ struct ToYYYYMMDDImpl
|
||||
{
|
||||
return time_zone.toNumYYYYMMDD(DayNum(d));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
@ -1471,7 +1400,6 @@ struct ToYYYYMMDDhhmmssImpl
|
||||
{
|
||||
return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(DayNum(d)));
|
||||
}
|
||||
static inline constexpr bool hasPreimage() { return false; }
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
@ -7,7 +7,6 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
/// See DateTimeTransforms.h
|
||||
@ -84,18 +83,6 @@ public:
|
||||
arguments[0].type->getName(), this->getName());
|
||||
}
|
||||
|
||||
bool hasInformationAboutPreimage() const override { return Transform::hasPreimage(); }
|
||||
|
||||
RangeOrNull getPreimage(const IDataType & type, const Field & point) const override
|
||||
{
|
||||
if constexpr (Transform::hasPreimage())
|
||||
return Transform::getPreimage(type, point);
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Function {} has no information about its preimage",
|
||||
Transform::name);
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -154,6 +154,8 @@ private:
|
||||
using ColVecType = ColumnVectorOrDecimal<Type>;
|
||||
|
||||
const auto col_vec = checkAndGetColumn<ColVecType>(col.column.get());
|
||||
if (col_vec == nullptr)
|
||||
return false;
|
||||
return (res = execute<Type, ReturnType>(col_vec)) != nullptr;
|
||||
};
|
||||
|
||||
|
@ -402,34 +402,6 @@ struct SipHash128ReferenceImpl
|
||||
static constexpr bool use_int_hash_for_pods = false;
|
||||
};
|
||||
|
||||
struct SipHash128ReferenceKeyedImpl
|
||||
{
|
||||
static constexpr auto name = "sipHash128ReferenceKeyed";
|
||||
using ReturnType = UInt128;
|
||||
using Key = impl::SipHashKey;
|
||||
|
||||
static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); }
|
||||
|
||||
static UInt128 applyKeyed(const Key & key, const char * begin, size_t size)
|
||||
{
|
||||
return sipHash128ReferenceKeyed(key.key0, key.key1, begin, size);
|
||||
}
|
||||
|
||||
static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2)
|
||||
{
|
||||
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
|
||||
UInt128 tmp;
|
||||
reverseMemcpy(&tmp, &h1, sizeof(UInt128));
|
||||
h1 = tmp;
|
||||
reverseMemcpy(&tmp, &h2, sizeof(UInt128));
|
||||
h2 = tmp;
|
||||
#endif
|
||||
UInt128 hashes[] = {h1, h2};
|
||||
return applyKeyed(key, reinterpret_cast<const char *>(hashes), 2 * sizeof(UInt128));
|
||||
}
|
||||
|
||||
static constexpr bool use_int_hash_for_pods = false;
|
||||
};
|
||||
|
||||
/** Why we need MurmurHash2?
|
||||
* MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash.
|
||||
@ -1737,7 +1709,6 @@ using FunctionSHA512 = FunctionStringHashFixedString<SHA512Impl>;
|
||||
using FunctionSipHash128 = FunctionAnyHash<SipHash128Impl>;
|
||||
using FunctionSipHash128Keyed = FunctionAnyHash<SipHash128KeyedImpl, true, SipHash128KeyedImpl::Key>;
|
||||
using FunctionSipHash128Reference = FunctionAnyHash<SipHash128ReferenceImpl>;
|
||||
using FunctionSipHash128ReferenceKeyed = FunctionAnyHash<SipHash128ReferenceKeyedImpl, true, SipHash128ReferenceKeyedImpl::Key>;
|
||||
using FunctionCityHash64 = FunctionAnyHash<ImplCityHash64>;
|
||||
using FunctionFarmFingerprint64 = FunctionAnyHash<ImplFarmFingerprint64>;
|
||||
using FunctionFarmHash64 = FunctionAnyHash<ImplFarmHash64>;
|
||||
|
@ -20,11 +20,6 @@ REGISTER_FUNCTION(Hashing)
|
||||
.examples{{"hash", "SELECT hex(sipHash128Reference('foo', '\\x01', 3))", ""}},
|
||||
.categories{"Hash"}
|
||||
});
|
||||
factory.registerFunction<FunctionSipHash128ReferenceKeyed>(FunctionDocumentation{
|
||||
.description="Same as [sipHash128Reference](#hash_functions-siphash128reference) but additionally takes an explicit key argument instead of using a fixed key.",
|
||||
.examples{{"hash", "SELECT hex(sipHash128ReferenceKeyed((506097522914230528, 1084818905618843912),'foo', '\\x01', 3));", ""}},
|
||||
.categories{"Hash"}
|
||||
});
|
||||
factory.registerFunction<FunctionCityHash64>();
|
||||
factory.registerFunction<FunctionFarmFingerprint64>();
|
||||
factory.registerFunction<FunctionFarmHash64>();
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/StringSearcher.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
|
||||
|
||||
@ -9,6 +10,7 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
@ -44,51 +46,47 @@ struct HasTokenImpl
|
||||
const UInt8 * const end = haystack_data.data() + haystack_data.size();
|
||||
const UInt8 * pos = begin;
|
||||
|
||||
try
|
||||
if (!ASCIICaseSensitiveTokenSearcher::isValidNeedle(pattern.data(), pattern.size()))
|
||||
{
|
||||
/// Parameter `pattern` is supposed to be a literal of letters and/or numbers.
|
||||
/// Otherwise, an exception from the constructor of `TokenSearcher` is thrown.
|
||||
/// If no exception is thrown at that point, then no further error cases may occur.
|
||||
TokenSearcher searcher(pattern.data(), pattern.size(), end - pos);
|
||||
if (res_null)
|
||||
std::ranges::fill(res_null->getData(), false);
|
||||
|
||||
/// The current index in the array of strings.
|
||||
size_t i = 0;
|
||||
/// We will search for the next occurrence in all rows at once.
|
||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
||||
{
|
||||
/// Let's determine which index it refers to.
|
||||
while (begin + haystack_offsets[i] <= pos)
|
||||
{
|
||||
res[i] = negate;
|
||||
++i;
|
||||
}
|
||||
|
||||
/// We check that the entry does not pass through the boundaries of strings.
|
||||
if (pos + pattern.size() < begin + haystack_offsets[i])
|
||||
res[i] = !negate;
|
||||
else
|
||||
res[i] = negate;
|
||||
|
||||
pos = begin + haystack_offsets[i];
|
||||
++i;
|
||||
}
|
||||
|
||||
/// Tail, in which there can be no substring.
|
||||
if (i < res.size())
|
||||
memset(&res[i], negate, (res.size() - i) * sizeof(res[0]));
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (!res_null)
|
||||
throw;
|
||||
else
|
||||
{
|
||||
std::ranges::fill(res, 0);
|
||||
std::ranges::fill(res_null->getData(), true);
|
||||
return;
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle must not contain whitespace or separator characters");
|
||||
}
|
||||
|
||||
TokenSearcher searcher(pattern.data(), pattern.size(), end - pos);
|
||||
if (res_null)
|
||||
std::ranges::fill(res_null->getData(), false);
|
||||
|
||||
/// The current index in the array of strings.
|
||||
size_t i = 0;
|
||||
/// We will search for the next occurrence in all rows at once.
|
||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
||||
{
|
||||
/// Let's determine which index it refers to.
|
||||
while (begin + haystack_offsets[i] <= pos)
|
||||
{
|
||||
res[i] = negate;
|
||||
++i;
|
||||
}
|
||||
|
||||
/// We check that the entry does not pass through the boundaries of strings.
|
||||
if (pos + pattern.size() < begin + haystack_offsets[i])
|
||||
res[i] = !negate;
|
||||
else
|
||||
res[i] = negate;
|
||||
|
||||
pos = begin + haystack_offsets[i];
|
||||
++i;
|
||||
}
|
||||
|
||||
/// Tail, in which there can be no substring.
|
||||
if (i < res.size())
|
||||
memset(&res[i], negate, (res.size() - i) * sizeof(res[0]));
|
||||
}
|
||||
|
||||
template <typename... Args>
|
||||
|
@ -2,8 +2,6 @@
|
||||
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/ColumnsWithTypeAndName.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/ValuesWithType.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Core/IResolvedFunction.h>
|
||||
#include <Common/Exception.h>
|
||||
@ -13,6 +11,11 @@
|
||||
|
||||
#include <memory>
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# include <Core/ValuesWithType.h>
|
||||
#endif
|
||||
|
||||
|
||||
/// This file contains user interface for functions.
|
||||
|
||||
namespace llvm
|
||||
@ -32,8 +35,7 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
/// A left-closed and right-open interval representing the preimage of a function.
|
||||
using RangeOrNull = std::optional<std::pair<Field, Field>>;
|
||||
class Field;
|
||||
|
||||
/// The simplest executable object.
|
||||
/// Motivation:
|
||||
@ -231,12 +233,6 @@ public:
|
||||
*/
|
||||
virtual bool hasInformationAboutMonotonicity() const { return false; }
|
||||
|
||||
/** Lets you know if the function has its definition of preimage.
|
||||
* This is used to work with predicate optimizations, where the comparison between
|
||||
* f(x) and a constant c could be converted to the comparison between x and f's preimage [b, e).
|
||||
*/
|
||||
virtual bool hasInformationAboutPreimage() const { return false; }
|
||||
|
||||
struct ShortCircuitSettings
|
||||
{
|
||||
/// Should we enable lazy execution for the first argument of short-circuit function?
|
||||
@ -290,14 +286,6 @@ public:
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName());
|
||||
}
|
||||
|
||||
/** Get the preimage of a function in the form of a left-closed and right-open interval. Call only if hasInformationAboutPreimage.
|
||||
* std::nullopt might be returned if the point (a single value) is invalid for this function.
|
||||
*/
|
||||
virtual RangeOrNull getPreimage(const IDataType & /*type*/, const Field & /*point*/) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its preimage", getName());
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
|
||||
@ -487,17 +475,12 @@ public:
|
||||
virtual bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const = 0;
|
||||
|
||||
virtual bool hasInformationAboutMonotonicity() const { return false; }
|
||||
virtual bool hasInformationAboutPreimage() const { return false; }
|
||||
|
||||
using Monotonicity = IFunctionBase::Monotonicity;
|
||||
virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName());
|
||||
}
|
||||
virtual RangeOrNull getPreimage(const IDataType & /*type*/, const Field & /*point*/) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its preimage", getName());
|
||||
}
|
||||
|
||||
/// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored).
|
||||
virtual size_t getNumberOfArguments() const = 0;
|
||||
|
@ -90,17 +90,10 @@ public:
|
||||
|
||||
bool hasInformationAboutMonotonicity() const override { return function->hasInformationAboutMonotonicity(); }
|
||||
|
||||
bool hasInformationAboutPreimage() const override { return function->hasInformationAboutPreimage(); }
|
||||
|
||||
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
|
||||
{
|
||||
return function->getMonotonicityForRange(type, left, right);
|
||||
}
|
||||
|
||||
RangeOrNull getPreimage(const IDataType & type, const Field & point) const override
|
||||
{
|
||||
return function->getPreimage(type, point);
|
||||
}
|
||||
private:
|
||||
std::shared_ptr<IFunction> function;
|
||||
DataTypes arguments;
|
||||
|
@ -139,7 +139,7 @@ struct LowerUpperUTF8Impl
|
||||
/// In case partial buffer was passed (due to SSE optimization)
|
||||
/// we cannot convert it with current src_end, but we may have more
|
||||
/// bytes to convert and eventually got correct symbol.
|
||||
if (partial && src_sequence_length > static_cast<size_t>(src_end-src))
|
||||
if (partial && src_sequence_length > static_cast<size_t>(src_end - src))
|
||||
return false;
|
||||
|
||||
auto src_code_point = UTF8::convertUTF8ToCodePoint(src, src_end - src);
|
||||
@ -181,7 +181,9 @@ private:
|
||||
|
||||
#ifdef __SSE2__
|
||||
static constexpr auto bytes_sse = sizeof(__m128i);
|
||||
const auto * src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse;
|
||||
|
||||
/// If we are before this position, we can still read at least bytes_sse.
|
||||
const auto * src_end_sse = src_end - bytes_sse + 1;
|
||||
|
||||
/// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f)
|
||||
const auto v_zero = _mm_setzero_si128();
|
||||
@ -227,9 +229,11 @@ private:
|
||||
{
|
||||
/// UTF-8
|
||||
|
||||
/// Find the offset of the next string after src
|
||||
size_t offset_from_begin = src - begin;
|
||||
while (offset_from_begin >= *offset_it)
|
||||
++offset_it;
|
||||
|
||||
/// Do not allow one row influence another (since row may have invalid sequence, and break the next)
|
||||
const UInt8 * row_end = begin + *offset_it;
|
||||
chassert(row_end >= src);
|
||||
@ -247,8 +251,9 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
/// Find which offset src has now
|
||||
while (offset_it != offsets.end() && static_cast<size_t>(src - begin) >= *offset_it)
|
||||
/// Find the offset of the next string after src
|
||||
size_t offset_from_begin = src - begin;
|
||||
while (offset_it != offsets.end() && offset_from_begin >= *offset_it)
|
||||
++offset_it;
|
||||
#endif
|
||||
|
||||
|
@ -59,14 +59,13 @@ protected:
|
||||
|
||||
void setResult(StringRefs & result, const Dwarf::LocationInfo & location, const std::vector<Dwarf::SymbolizedFrame> & inline_frames) const override
|
||||
{
|
||||
|
||||
appendLocationToResult(result, location, nullptr);
|
||||
for (const auto & inline_frame : inline_frames)
|
||||
appendLocationToResult(result, inline_frame.location, &inline_frame);
|
||||
}
|
||||
private:
|
||||
|
||||
inline ALWAYS_INLINE void appendLocationToResult(StringRefs & result, const Dwarf::LocationInfo & location, const Dwarf::SymbolizedFrame * frame) const
|
||||
private:
|
||||
void appendLocationToResult(StringRefs & result, const Dwarf::LocationInfo & location, const Dwarf::SymbolizedFrame * frame) const
|
||||
{
|
||||
const char * arena_begin = nullptr;
|
||||
WriteBufferFromArena out(cache.arena, arena_begin);
|
||||
@ -83,6 +82,7 @@ private:
|
||||
}
|
||||
|
||||
result.emplace_back(out.complete());
|
||||
out.finalize();
|
||||
}
|
||||
|
||||
};
|
||||
|
@ -16,18 +16,18 @@ struct NameHasTokenOrNull
|
||||
static constexpr auto name = "hasTokenOrNull";
|
||||
};
|
||||
|
||||
using FunctionHasToken = DB::FunctionsStringSearch<DB::HasTokenImpl<NameHasToken, DB::VolnitskyCaseSensitiveToken, false>>;
|
||||
using FunctionHasTokenOrNull = DB::
|
||||
FunctionsStringSearch<DB::HasTokenImpl<NameHasTokenOrNull, DB::VolnitskyCaseSensitiveToken, false>, DB::ExecutionErrorPolicy::Null>;
|
||||
using FunctionHasToken
|
||||
= FunctionsStringSearch<HasTokenImpl<NameHasToken, VolnitskyCaseSensitiveToken, false>>;
|
||||
using FunctionHasTokenOrNull
|
||||
= FunctionsStringSearch<HasTokenImpl<NameHasTokenOrNull, VolnitskyCaseSensitiveToken, false>, ExecutionErrorPolicy::Null>;
|
||||
|
||||
REGISTER_FUNCTION(HasToken)
|
||||
{
|
||||
factory.registerFunction<FunctionHasToken>(FunctionDocumentation
|
||||
{.description="Performs lookup of needle in haystack using tokenbf_v1 index."}, DB::FunctionFactory::CaseSensitive);
|
||||
{.description="Performs lookup of needle in haystack using tokenbf_v1 index."}, FunctionFactory::CaseSensitive);
|
||||
|
||||
factory.registerFunction<FunctionHasTokenOrNull>(FunctionDocumentation
|
||||
{.description="Performs lookup of needle in haystack using tokenbf_v1 index. Returns null if needle is ill-formed."},
|
||||
DB::FunctionFactory::CaseSensitive);
|
||||
{.description="Performs lookup of needle in haystack using tokenbf_v1 index. Returns null if needle is ill-formed."}, FunctionFactory::CaseSensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -17,10 +17,9 @@ struct NameHasTokenCaseInsensitiveOrNull
|
||||
};
|
||||
|
||||
using FunctionHasTokenCaseInsensitive
|
||||
= DB::FunctionsStringSearch<DB::HasTokenImpl<NameHasTokenCaseInsensitive, DB::VolnitskyCaseInsensitiveToken, false>>;
|
||||
using FunctionHasTokenCaseInsensitiveOrNull = DB::FunctionsStringSearch<
|
||||
DB::HasTokenImpl<NameHasTokenCaseInsensitiveOrNull, DB::VolnitskyCaseInsensitiveToken, false>,
|
||||
DB::ExecutionErrorPolicy::Null>;
|
||||
= FunctionsStringSearch<HasTokenImpl<NameHasTokenCaseInsensitive, VolnitskyCaseInsensitiveToken, false>>;
|
||||
using FunctionHasTokenCaseInsensitiveOrNull
|
||||
= FunctionsStringSearch<HasTokenImpl<NameHasTokenCaseInsensitiveOrNull, VolnitskyCaseInsensitiveToken, false>, ExecutionErrorPolicy::Null>;
|
||||
|
||||
REGISTER_FUNCTION(HasTokenCaseInsensitive)
|
||||
{
|
||||
|
@ -122,9 +122,13 @@ public:
|
||||
tuple = typeid_cast<const ColumnTuple *>(materialized_tuple.get());
|
||||
}
|
||||
|
||||
auto set = column_set->getData();
|
||||
auto future_set = column_set->getData();
|
||||
if (!future_set)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No Set is passed as the second argument for function '{}'", getName());
|
||||
|
||||
auto set = future_set->get();
|
||||
if (!set)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set passed as the second argument for function '{}'", getName());
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set is passed as the second argument for function '{}'", getName());
|
||||
|
||||
auto set_types = set->getDataTypes();
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
@ -13,9 +14,10 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -25,32 +27,31 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
/** transform(x, from_array, to_array[, default]) - convert x according to an explicitly passed match.
|
||||
*/
|
||||
/** transform(x, [from...], [to...], default)
|
||||
* - converts the values according to the explicitly specified mapping.
|
||||
*
|
||||
* x - what to transform.
|
||||
* from - a constant array of values for the transformation.
|
||||
* to - a constant array of values into which values from `from` must be transformed.
|
||||
* default - what value to use if x is not equal to any of the values in `from`.
|
||||
* `from` and `to` - arrays of the same size.
|
||||
*
|
||||
* Types:
|
||||
* transform(T, Array(T), Array(U), U) -> U
|
||||
*
|
||||
* transform(x, [from...], [to...])
|
||||
* - if `default` is not specified, then for values of `x` for which there is no corresponding element in `from`, the unchanged value of `x` is returned.
|
||||
*
|
||||
* Types:
|
||||
* transform(T, Array(T), Array(T)) -> T
|
||||
*
|
||||
* Note: the implementation is rather cumbersome.
|
||||
*/
|
||||
* - converts the values according to the explicitly specified mapping.
|
||||
*
|
||||
* x - what to transform.
|
||||
* from - a constant array of values for the transformation.
|
||||
* to - a constant array of values into which values from `from` must be transformed.
|
||||
* default - what value to use if x is not equal to any of the values in `from`.
|
||||
* `from` and `to` - arrays of the same size.
|
||||
*
|
||||
* Types:
|
||||
* transform(T, Array(T), Array(U), U) -> U
|
||||
*
|
||||
* transform(x, [from...], [to...])
|
||||
* - if `default` is not specified, then for values of `x` for which there is no corresponding element in `from`, the unchanged value of `x` is returned.
|
||||
*
|
||||
* Types:
|
||||
* transform(T, Array(T), Array(T)) -> T
|
||||
*
|
||||
* Note: the implementation is rather cumbersome.
|
||||
*/
|
||||
class FunctionTransform : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -79,15 +80,6 @@ namespace
|
||||
args_size);
|
||||
|
||||
const DataTypePtr & type_x = arguments[0];
|
||||
const auto & type_x_nn = removeNullable(type_x);
|
||||
|
||||
if (!type_x_nn->isValueRepresentedByNumber() && !isString(type_x_nn) && !isNothing(type_x_nn))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Unsupported type {} of first argument "
|
||||
"of function {}, must be numeric type or Date/DateTime or String",
|
||||
type_x->getName(),
|
||||
getName());
|
||||
|
||||
const DataTypeArray * type_arr_from = checkAndGetDataType<DataTypeArray>(arguments[1].get());
|
||||
|
||||
@ -99,14 +91,16 @@ namespace
|
||||
|
||||
const auto type_arr_from_nested = type_arr_from->getNestedType();
|
||||
|
||||
if ((type_x->isValueRepresentedByNumber() != type_arr_from_nested->isValueRepresentedByNumber())
|
||||
|| (isString(type_x) != isString(type_arr_from_nested)))
|
||||
auto src = tryGetLeastSupertype(DataTypes{type_x, type_arr_from_nested});
|
||||
if (!src
|
||||
/// Compatibility with previous versions, that allowed even UInt64 with Int64,
|
||||
/// regardless of ambiguous conversions.
|
||||
&& !isNativeNumber(type_x) && !isNativeNumber(type_arr_from_nested))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"First argument and elements of array "
|
||||
"of second argument of function {} must have compatible types: "
|
||||
"both numeric or both strings.",
|
||||
"of the second argument of function {} must have compatible types",
|
||||
getName());
|
||||
}
|
||||
|
||||
@ -157,8 +151,8 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
ColumnPtr
|
||||
executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
ColumnPtr executeImpl(
|
||||
const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
initialize(arguments, result_type);
|
||||
|
||||
@ -172,22 +166,42 @@ namespace
|
||||
default_non_const = castColumn(arguments[3], result_type);
|
||||
|
||||
auto column_result = result_type->createColumn();
|
||||
if (!executeNum<ColumnVector<UInt8>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<UInt16>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<UInt32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<UInt64>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int8>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int16>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int64>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Float32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Float64>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnDecimal<Decimal32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnDecimal<Decimal64>>(in, *column_result, default_non_const)
|
||||
&& !executeString(in, *column_result, default_non_const))
|
||||
if (cache.is_empty)
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName());
|
||||
return default_non_const
|
||||
? default_non_const
|
||||
: castColumn(arguments[0], result_type);
|
||||
}
|
||||
else if (cache.table_num_to_idx)
|
||||
{
|
||||
if (!executeNum<ColumnVector<UInt8>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<UInt16>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<UInt32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<UInt64>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int8>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int16>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int64>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Float32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Float64>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnDecimal<Decimal32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnDecimal<Decimal64>>(in, *column_result, default_non_const))
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName());
|
||||
}
|
||||
}
|
||||
else if (cache.table_string_to_idx)
|
||||
{
|
||||
if (!executeString(in, *column_result, default_non_const))
|
||||
executeContiguous(in, *column_result, default_non_const);
|
||||
}
|
||||
else if (cache.table_anything_to_idx)
|
||||
{
|
||||
executeAnything(in, *column_result, default_non_const);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "State of the function `transform` is not initialized");
|
||||
|
||||
return column_result;
|
||||
}
|
||||
|
||||
@ -204,6 +218,47 @@ namespace
|
||||
return impl->execute(args, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const) const
|
||||
{
|
||||
const size_t size = in->size();
|
||||
const auto & table = *cache.table_anything_to_idx;
|
||||
column_result.reserve(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
SipHash hash;
|
||||
in->updateHashWithValue(i, hash);
|
||||
|
||||
const auto * it = table.find(hash.get128());
|
||||
if (it)
|
||||
column_result.insertFrom(*cache.to_column, it->getMapped());
|
||||
else if (cache.default_column)
|
||||
column_result.insertFrom(*cache.default_column, 0);
|
||||
else if (default_non_const)
|
||||
column_result.insertFrom(*default_non_const, i);
|
||||
else
|
||||
column_result.insertFrom(*in, i);
|
||||
}
|
||||
}
|
||||
|
||||
void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const) const
|
||||
{
|
||||
const size_t size = in->size();
|
||||
const auto & table = *cache.table_string_to_idx;
|
||||
column_result.reserve(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const auto * it = table.find(in->getDataAt(i));
|
||||
if (it)
|
||||
column_result.insertFrom(*cache.to_column, it->getMapped());
|
||||
else if (cache.default_column)
|
||||
column_result.insertFrom(*cache.default_column, 0);
|
||||
else if (default_non_const)
|
||||
column_result.insertFrom(*default_non_const, i);
|
||||
else
|
||||
column_result.insertFrom(*in, i);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const) const
|
||||
{
|
||||
@ -236,7 +291,7 @@ namespace
|
||||
{
|
||||
const auto * it = table.find(bit_cast<UInt64>(pod[i]));
|
||||
if (it)
|
||||
column_result.insertFrom(*cache.to_columns, it->getMapped());
|
||||
column_result.insertFrom(*cache.to_column, it->getMapped());
|
||||
else if (cache.default_column)
|
||||
column_result.insertFrom(*cache.default_column, 0);
|
||||
else if (default_non_const)
|
||||
@ -259,14 +314,14 @@ namespace
|
||||
out_offs.resize(size);
|
||||
auto & out_chars = out->getChars();
|
||||
|
||||
const auto * to_col = reinterpret_cast<const ColumnString *>(cache.to_columns.get());
|
||||
const auto * to_col = assert_cast<const ColumnString *>(cache.to_column.get());
|
||||
const auto & to_chars = to_col->getChars();
|
||||
const auto & to_offs = to_col->getOffsets();
|
||||
const auto & table = *cache.table_num_to_idx;
|
||||
|
||||
if (cache.default_column)
|
||||
{
|
||||
const auto * def = reinterpret_cast<const ColumnString *>(cache.default_column.get());
|
||||
const auto * def = assert_cast<const ColumnString *>(cache.default_column.get());
|
||||
const auto & def_chars = def->getChars();
|
||||
const auto & def_offs = def->getOffsets();
|
||||
const auto * def_data = def_chars.data();
|
||||
@ -275,7 +330,7 @@ namespace
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto * def = reinterpret_cast<const ColumnString *>(default_non_const.get());
|
||||
const auto * def = assert_cast<const ColumnString *>(default_non_const.get());
|
||||
const auto & def_chars = def->getChars();
|
||||
const auto & def_offs = def->getOffsets();
|
||||
executeNumToStringHelper(table, pod, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, size);
|
||||
@ -340,16 +395,16 @@ namespace
|
||||
if constexpr (std::is_same_v<ColumnDecimal<Decimal32>, T> || std::is_same_v<ColumnDecimal<Decimal64>, T>)
|
||||
out_scale = out->getScale();
|
||||
|
||||
const auto & to_pod = reinterpret_cast<const T *>(cache.to_columns.get())->getData();
|
||||
const auto & to_pod = assert_cast<const T *>(cache.to_column.get())->getData();
|
||||
const auto & table = *cache.table_num_to_idx;
|
||||
if (cache.default_column)
|
||||
{
|
||||
const auto const_def = reinterpret_cast<const T *>(cache.default_column.get())->getData()[0];
|
||||
const auto const_def = assert_cast<const T *>(cache.default_column.get())->getData()[0];
|
||||
executeNumToNumHelper(table, pod, out_pod, to_pod, const_def, size, out_scale, out_scale);
|
||||
}
|
||||
else if (default_non_const)
|
||||
{
|
||||
const auto & nconst_def = reinterpret_cast<const T *>(default_non_const.get())->getData();
|
||||
const auto & nconst_def = assert_cast<const T *>(default_non_const.get())->getData();
|
||||
executeNumToNumHelper(table, pod, out_pod, to_pod, nconst_def, size, out_scale, out_scale);
|
||||
}
|
||||
else
|
||||
@ -423,11 +478,11 @@ namespace
|
||||
ColumnString::Offset current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const StringRef ref{&data[current_offset], offsets[i] - current_offset};
|
||||
const StringRef ref{&data[current_offset], offsets[i] - current_offset - 1};
|
||||
current_offset = offsets[i];
|
||||
const auto * it = table.find(ref);
|
||||
if (it)
|
||||
column_result.insertFrom(*cache.to_columns, it->getMapped());
|
||||
column_result.insertFrom(*cache.to_column, it->getMapped());
|
||||
else if (cache.default_column)
|
||||
column_result.insertFrom(*cache.default_column, 0);
|
||||
else if (default_non_const)
|
||||
@ -453,14 +508,14 @@ namespace
|
||||
out_offs.resize(size);
|
||||
auto & out_chars = out->getChars();
|
||||
|
||||
const auto * to_col = reinterpret_cast<const ColumnString *>(cache.to_columns.get());
|
||||
const auto * to_col = assert_cast<const ColumnString *>(cache.to_column.get());
|
||||
const auto & to_chars = to_col->getChars();
|
||||
const auto & to_offs = to_col->getOffsets();
|
||||
|
||||
const auto & table = *cache.table_string_to_idx;
|
||||
if (cache.default_column)
|
||||
{
|
||||
const auto * def = reinterpret_cast<const ColumnString *>(cache.default_column.get());
|
||||
const auto * def = assert_cast<const ColumnString *>(cache.default_column.get());
|
||||
const auto & def_chars = def->getChars();
|
||||
const auto & def_offs = def->getOffsets();
|
||||
const auto * def_data = def_chars.data();
|
||||
@ -469,7 +524,7 @@ namespace
|
||||
}
|
||||
else if (default_non_const)
|
||||
{
|
||||
const auto * def = reinterpret_cast<const ColumnString *>(default_non_const.get());
|
||||
const auto * def = assert_cast<const ColumnString *>(default_non_const.get());
|
||||
const auto & def_chars = def->getChars();
|
||||
const auto & def_offs = def->getOffsets();
|
||||
executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, size);
|
||||
@ -500,7 +555,7 @@ namespace
|
||||
{
|
||||
const char8_t * to = nullptr;
|
||||
size_t to_size = 0;
|
||||
const StringRef ref{&data[current_offset], offsets[i] - current_offset};
|
||||
const StringRef ref{&data[current_offset], offsets[i] - current_offset - 1};
|
||||
current_offset = offsets[i];
|
||||
const auto * it = table.find(ref);
|
||||
if (it)
|
||||
@ -542,16 +597,16 @@ namespace
|
||||
const size_t size = offsets.size();
|
||||
out_pod.resize(size);
|
||||
|
||||
const auto & to_pod = reinterpret_cast<const T *>(cache.to_columns.get())->getData();
|
||||
const auto & to_pod = assert_cast<const T *>(cache.to_column.get())->getData();
|
||||
const auto & table = *cache.table_string_to_idx;
|
||||
if (cache.default_column)
|
||||
{
|
||||
const auto const_def = reinterpret_cast<const T *>(cache.default_column.get())->getData()[0];
|
||||
const auto const_def = assert_cast<const T *>(cache.default_column.get())->getData()[0];
|
||||
executeStringToNumHelper(table, data, offsets, out_pod, to_pod, const_def, size);
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & nconst_def = reinterpret_cast<const T *>(default_non_const.get())->getData();
|
||||
const auto & nconst_def = assert_cast<const T *>(default_non_const.get())->getData();
|
||||
executeStringToNumHelper(table, data, offsets, out_pod, to_pod, nconst_def, size);
|
||||
}
|
||||
return true;
|
||||
@ -570,7 +625,7 @@ namespace
|
||||
ColumnString::Offset current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const StringRef ref{&data[current_offset], offsets[i] - current_offset};
|
||||
const StringRef ref{&data[current_offset], offsets[i] - current_offset - 1};
|
||||
current_offset = offsets[i];
|
||||
const auto * it = table.find(ref);
|
||||
if (it)
|
||||
@ -593,15 +648,18 @@ namespace
|
||||
{
|
||||
using NumToIdx = HashMap<UInt64, size_t, HashCRC32<UInt64>>;
|
||||
using StringToIdx = HashMap<StringRef, size_t, StringRefHash>;
|
||||
using AnythingToIdx = HashMap<UInt128, size_t>;
|
||||
|
||||
std::unique_ptr<NumToIdx> table_num_to_idx;
|
||||
std::unique_ptr<StringToIdx> table_string_to_idx;
|
||||
std::unique_ptr<AnythingToIdx> table_anything_to_idx;
|
||||
|
||||
ColumnPtr to_columns;
|
||||
bool is_empty = false;
|
||||
|
||||
ColumnPtr from_column;
|
||||
ColumnPtr to_column;
|
||||
ColumnPtr default_column;
|
||||
|
||||
Arena string_pool;
|
||||
|
||||
std::atomic<bool> initialized{false};
|
||||
std::mutex mutex;
|
||||
};
|
||||
@ -609,27 +667,6 @@ namespace
|
||||
mutable Cache cache;
|
||||
|
||||
|
||||
static UInt64 bitCastToUInt64(const Field & x)
|
||||
{
|
||||
switch (x.getType())
|
||||
{
|
||||
case Field::Types::UInt64:
|
||||
return x.get<UInt64>();
|
||||
case Field::Types::Int64:
|
||||
return x.get<Int64>();
|
||||
case Field::Types::Float64:
|
||||
return std::bit_cast<UInt64>(x.get<Float64>());
|
||||
case Field::Types::Bool:
|
||||
return x.get<bool>();
|
||||
case Field::Types::Decimal32:
|
||||
return x.get<DecimalField<Decimal32>>().getValue();
|
||||
case Field::Types::Decimal64:
|
||||
return x.get<DecimalField<Decimal64>>().getValue();
|
||||
default:
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type in function 'transform'");
|
||||
}
|
||||
}
|
||||
|
||||
static void checkAllowedType(const DataTypePtr & type)
|
||||
{
|
||||
if (type->isNullable())
|
||||
@ -656,33 +693,56 @@ namespace
|
||||
/// Can be called from different threads. It works only on the first call.
|
||||
void initialize(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const
|
||||
{
|
||||
const ColumnConst * array_from = checkAndGetColumnConst<ColumnArray>(arguments[1].column.get());
|
||||
const ColumnConst * array_to = checkAndGetColumnConst<ColumnArray>(arguments[2].column.get());
|
||||
if (cache.initialized)
|
||||
return;
|
||||
|
||||
const DataTypePtr & from_type = arguments[0].type;
|
||||
|
||||
if (from_type->onlyNull())
|
||||
{
|
||||
cache.is_empty = true;
|
||||
return;
|
||||
}
|
||||
|
||||
const ColumnArray * array_from = checkAndGetColumnConstData<ColumnArray>(arguments[1].column.get());
|
||||
const ColumnArray * array_to = checkAndGetColumnConstData<ColumnArray>(arguments[2].column.get());
|
||||
|
||||
if (!array_from || !array_to)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN, "Second and third arguments of function {} must be constant arrays.", getName());
|
||||
|
||||
if (cache.initialized)
|
||||
return;
|
||||
|
||||
const auto & from = array_from->getValue<Array>();
|
||||
const size_t size = from.size();
|
||||
if (0 == size)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty arrays are illegal in function {}", getName());
|
||||
|
||||
std::lock_guard lock(cache.mutex);
|
||||
|
||||
if (cache.initialized)
|
||||
return;
|
||||
const ColumnPtr & from_column_uncasted = array_from->getDataPtr();
|
||||
|
||||
const auto & to = array_to->getValue<Array>();
|
||||
if (size != to.size())
|
||||
cache.from_column = castColumn(
|
||||
{
|
||||
from_column_uncasted,
|
||||
typeid_cast<const DataTypeArray &>(*arguments[1].type).getNestedType(),
|
||||
arguments[1].name
|
||||
},
|
||||
from_type);
|
||||
|
||||
cache.to_column = castColumn(
|
||||
{
|
||||
array_to->getDataPtr(),
|
||||
typeid_cast<const DataTypeArray &>(*arguments[2].type).getNestedType(),
|
||||
arguments[2].name
|
||||
},
|
||||
result_type);
|
||||
|
||||
const size_t size = cache.from_column->size();
|
||||
if (0 == size)
|
||||
{
|
||||
cache.is_empty = true;
|
||||
return;
|
||||
}
|
||||
|
||||
if (cache.to_column->size() != size)
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS, "Second and third arguments of function {} must be arrays of same size", getName());
|
||||
|
||||
/// Whether the default value is set.
|
||||
|
||||
if (arguments.size() == 4)
|
||||
{
|
||||
const IColumn * default_col = arguments[3].column.get();
|
||||
@ -702,45 +762,54 @@ namespace
|
||||
|
||||
/// Note: Doesn't check the duplicates in the `from` array.
|
||||
|
||||
const IDataType & from_type = *arguments[0].type;
|
||||
|
||||
if (from[0].getType() != Field::Types::String)
|
||||
WhichDataType which(from_type);
|
||||
if (isNativeNumber(which) || which.isDecimal32() || which.isDecimal64())
|
||||
{
|
||||
cache.table_num_to_idx = std::make_unique<Cache::NumToIdx>();
|
||||
auto & table = *cache.table_num_to_idx;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
Field key = convertFieldToType(from[i], from_type);
|
||||
if (key.isNull())
|
||||
continue;
|
||||
|
||||
/// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64
|
||||
table[bitCastToUInt64(key)] = i;
|
||||
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i]))
|
||||
{
|
||||
/// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64
|
||||
StringRef ref = cache.from_column->getDataAt(i);
|
||||
UInt64 key = 0;
|
||||
memcpy(&key, ref.data, ref.size);
|
||||
table[key] = i;
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
else if (from_type->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
||||
{
|
||||
cache.table_string_to_idx = std::make_unique<Cache::StringToIdx>();
|
||||
auto & table = *cache.table_string_to_idx;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const String & str_from = from[i].get<const String &>();
|
||||
StringRef ref{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
|
||||
table[ref] = i;
|
||||
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i]))
|
||||
{
|
||||
StringRef ref = cache.from_column->getDataAt(i);
|
||||
table[ref] = i;
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
cache.table_anything_to_idx = std::make_unique<Cache::AnythingToIdx>();
|
||||
auto & table = *cache.table_anything_to_idx;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i]))
|
||||
{
|
||||
SipHash hash;
|
||||
cache.from_column->updateHashWithValue(i, hash);
|
||||
table[hash.get128()] = i;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
auto to_columns = result_type->createColumn();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
Field to_value = convertFieldToType(to[i], *result_type);
|
||||
to_columns->insert(to_value);
|
||||
}
|
||||
cache.to_columns = std::move(to_columns);
|
||||
|
||||
cache.initialized = true;
|
||||
}
|
||||
};
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -63,6 +63,7 @@ namespace ErrorCodes
|
||||
extern const int INCORRECT_DATA;
|
||||
extern const int TOO_LARGE_STRING_SIZE;
|
||||
extern const int TOO_LARGE_ARRAY_SIZE;
|
||||
extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
/// Helper functions for formatted input.
|
||||
@ -138,6 +139,19 @@ inline void readStringBinary(std::string & s, ReadBuffer & buf, size_t max_strin
|
||||
buf.readStrict(s.data(), size);
|
||||
}
|
||||
|
||||
/// For historical reasons we store IPv6 as a String
|
||||
inline void readIPv6Binary(IPv6 & ip, ReadBuffer & buf)
|
||||
{
|
||||
size_t size = 0;
|
||||
readVarUInt(size, buf);
|
||||
|
||||
if (size != IPV6_BINARY_LENGTH)
|
||||
throw Exception(ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH,
|
||||
"Size of the string {} doesn't match size of binary IPv6 {}", size, IPV6_BINARY_LENGTH);
|
||||
|
||||
buf.readStrict(reinterpret_cast<char*>(&ip.toUnderType()), size);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void readVectorBinary(std::vector<T> & v, ReadBuffer & buf)
|
||||
{
|
||||
|
@ -10,6 +10,7 @@
|
||||
|
||||
#include <pcg-random/pcg_random.hpp>
|
||||
|
||||
#include "Common/formatIPv6.h"
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/LocalDate.h>
|
||||
#include <Common/LocalDateTime.h>
|
||||
@ -105,6 +106,13 @@ inline void writeStringBinary(const std::string & s, WriteBuffer & buf)
|
||||
buf.write(s.data(), s.size());
|
||||
}
|
||||
|
||||
/// For historical reasons we store IPv6 as a String
|
||||
inline void writeIPv6Binary(const IPv6 & ip, WriteBuffer & buf)
|
||||
{
|
||||
writeVarUInt(IPV6_BINARY_LENGTH, buf);
|
||||
buf.write(reinterpret_cast<const char *>(&ip.toUnderType()), IPV6_BINARY_LENGTH);
|
||||
}
|
||||
|
||||
inline void writeStringBinary(StringRef s, WriteBuffer & buf)
|
||||
{
|
||||
writeVarUInt(s.size, buf);
|
||||
|
@ -57,6 +57,9 @@
|
||||
#include <Functions/UserDefined/UserDefinedExecutableFunctionFactory.h>
|
||||
#include <Parsers/QueryParameterVisitor.h>
|
||||
|
||||
#include <Analyzer/QueryNode.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -422,9 +425,8 @@ Block createBlockForSet(
|
||||
}
|
||||
|
||||
|
||||
SetPtr makeExplicitSet(
|
||||
const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set,
|
||||
ContextPtr context, const SizeLimits & size_limits, PreparedSets & prepared_sets)
|
||||
FutureSetPtr makeExplicitSet(
|
||||
const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets)
|
||||
{
|
||||
const IAST & args = *node->arguments;
|
||||
|
||||
@ -443,14 +445,16 @@ SetPtr makeExplicitSet(
|
||||
if (left_tuple_type && left_tuple_type->getElements().size() != 1)
|
||||
set_element_types = left_tuple_type->getElements();
|
||||
|
||||
auto set_element_keys = Set::getElementTypes(set_element_types, context->getSettingsRef().transform_null_in);
|
||||
|
||||
auto set_key = right_arg->getTreeHash();
|
||||
if (auto set = prepared_sets.findTuple(set_key, set_element_keys))
|
||||
return set; /// Already prepared.
|
||||
|
||||
for (auto & element_type : set_element_types)
|
||||
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(element_type.get()))
|
||||
element_type = low_cardinality_type->getDictionaryType();
|
||||
|
||||
auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types);
|
||||
if (auto set = prepared_sets.get(set_key))
|
||||
return set; /// Already prepared.
|
||||
|
||||
Block block;
|
||||
const auto & right_arg_func = std::dynamic_pointer_cast<ASTFunction>(right_arg);
|
||||
if (right_arg_func && (right_arg_func->name == "tuple" || right_arg_func->name == "array"))
|
||||
@ -458,14 +462,7 @@ SetPtr makeExplicitSet(
|
||||
else
|
||||
block = createBlockForSet(left_arg_type, right_arg, set_element_types, context);
|
||||
|
||||
SetPtr set
|
||||
= std::make_shared<Set>(size_limits, create_ordered_set, context->getSettingsRef().transform_null_in);
|
||||
set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName());
|
||||
set->insertFromBlock(block.getColumnsWithTypeAndName());
|
||||
set->finishInsert();
|
||||
|
||||
prepared_sets.set(set_key, set);
|
||||
return set;
|
||||
return prepared_sets.addFromTuple(set_key, block, context->getSettings());
|
||||
}
|
||||
|
||||
class ScopeStack::Index
|
||||
@ -950,7 +947,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
return;
|
||||
}
|
||||
|
||||
FutureSet prepared_set;
|
||||
FutureSetPtr prepared_set;
|
||||
if (checkFunctionIsInOrGlobalInOperator(node))
|
||||
{
|
||||
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
|
||||
@ -959,7 +956,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
if (!data.no_makeset && !(data.is_create_parameterized_view && !analyzeReceiveQueryParams(ast).empty()))
|
||||
prepared_set = makeSet(node, data, data.no_subqueries);
|
||||
|
||||
if (prepared_set.isValid())
|
||||
if (prepared_set)
|
||||
{
|
||||
/// Transform tuple or subquery into a set.
|
||||
}
|
||||
@ -1171,14 +1168,14 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
num_arguments += columns.size() - 1;
|
||||
arg += columns.size() - 1;
|
||||
}
|
||||
else if (checkFunctionIsInOrGlobalInOperator(node) && arg == 1 && prepared_set.isValid())
|
||||
else if (checkFunctionIsInOrGlobalInOperator(node) && arg == 1 && prepared_set)
|
||||
{
|
||||
ColumnWithTypeAndName column;
|
||||
column.type = std::make_shared<DataTypeSet>();
|
||||
|
||||
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
|
||||
/// so that sets with the same literal representation do not fuse together (they can have different types).
|
||||
const bool is_constant_set = prepared_set.isCreated();
|
||||
const bool is_constant_set = typeid_cast<const FutureSetFromSubquery *>(prepared_set.get()) == nullptr;
|
||||
if (is_constant_set)
|
||||
column.name = data.getUniqueName("__set");
|
||||
else
|
||||
@ -1381,7 +1378,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */,
|
||||
data.addColumn(std::move(column));
|
||||
}
|
||||
|
||||
FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries)
|
||||
FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries)
|
||||
{
|
||||
if (!data.prepared_sets)
|
||||
return {};
|
||||
@ -1400,13 +1397,34 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no
|
||||
{
|
||||
if (no_subqueries)
|
||||
return {};
|
||||
auto set_key = PreparedSetKey::forSubquery(*right_in_operand);
|
||||
|
||||
PreparedSets::Hash set_key;
|
||||
if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier)
|
||||
{
|
||||
auto set = data.prepared_sets->getFuture(set_key);
|
||||
if (set.isValid())
|
||||
return set;
|
||||
/// Here we can be only from mutation interpreter. Normal selects with analyzed use other interpreter.
|
||||
/// This is a hacky way to allow reusing cache for prepared sets.
|
||||
///
|
||||
/// Mutation is executed in two stages:
|
||||
/// * first, query 'SELECT count() FROM table WHERE ...' is executed to get the set of affected parts (using analyzer)
|
||||
/// * second, every part is mutated separately, where plan is build "manually", using this code as well
|
||||
/// To share the Set in between first and second stage, we should use the same hash.
|
||||
/// New analyzer is uses a hash from query tree, so here we also build a query tree.
|
||||
///
|
||||
/// Note : this code can be safely removed, but the test 02581_share_big_sets will be too slow (and fail by timeout).
|
||||
/// Note : we should use new analyzer for mutations and remove this hack.
|
||||
InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery());
|
||||
const auto & query_tree = interpreter.getQueryTree();
|
||||
if (auto * query_node = query_tree->as<QueryNode>())
|
||||
query_node->setIsSubquery(true);
|
||||
set_key = query_tree->getTreeHash();
|
||||
}
|
||||
else
|
||||
set_key = right_in_operand->getTreeHash();
|
||||
|
||||
if (auto set = data.prepared_sets->findSubquery(set_key))
|
||||
return set;
|
||||
|
||||
FutureSetPtr external_table_set;
|
||||
|
||||
/// A special case is if the name of the table is specified on the right side of the IN statement,
|
||||
/// and the table has the type Set (a previously prepared set).
|
||||
@ -1417,20 +1435,25 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no
|
||||
|
||||
if (table)
|
||||
{
|
||||
StorageSet * storage_set = dynamic_cast<StorageSet *>(table.get());
|
||||
if (storage_set)
|
||||
{
|
||||
SetPtr set = storage_set->getSet();
|
||||
data.prepared_sets->set(set_key, set);
|
||||
return FutureSet(set);
|
||||
}
|
||||
if (auto set = data.prepared_sets->findStorage(set_key))
|
||||
return set;
|
||||
|
||||
if (StorageSet * storage_set = dynamic_cast<StorageSet *>(table.get()))
|
||||
return data.prepared_sets->addFromStorage(set_key, storage_set->getSet());
|
||||
}
|
||||
|
||||
if (!data.getContext()->isGlobalContext())
|
||||
{
|
||||
/// If we are reading from storage, it can be an external table which is used for GLOBAL IN.
|
||||
/// Here, we take FutureSet which is used to build external table.
|
||||
/// It will be used if set is useful for primary key. During PK analysis
|
||||
/// temporary table is not filled yet, so we need to fill it first.
|
||||
if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName()))
|
||||
external_table_set = tmp_table->future_set;
|
||||
}
|
||||
}
|
||||
|
||||
/// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery.
|
||||
String set_id = right_in_operand->getColumnName();
|
||||
bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in;
|
||||
SubqueryForSet & subquery_for_set = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in);
|
||||
std::unique_ptr<QueryPlan> source = std::make_unique<QueryPlan>();
|
||||
|
||||
/** The following happens for GLOBAL INs or INs:
|
||||
* - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1,
|
||||
@ -1440,13 +1463,12 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no
|
||||
* In case that we have HAVING with IN subquery, we have to force creating set for it.
|
||||
* Also it doesn't make sense if it is GLOBAL IN or ordinary IN.
|
||||
*/
|
||||
if (!subquery_for_set.hasSource())
|
||||
{
|
||||
auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {});
|
||||
subquery_for_set.createSource(*interpreter);
|
||||
interpreter->buildQueryPlan(*source);
|
||||
}
|
||||
|
||||
return subquery_for_set.set;
|
||||
return data.prepared_sets->addFromSubquery(set_key, std::move(source), nullptr, std::move(external_table_set), data.getContext()->getSettingsRef());
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1454,8 +1476,7 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no
|
||||
const auto & index = data.actions_stack.getLastActionsIndex();
|
||||
if (data.prepared_sets && index.contains(left_in_operand->getColumnName()))
|
||||
/// An explicit enumeration of values in parentheses.
|
||||
return FutureSet(
|
||||
makeExplicitSet(&node, last_actions, false, data.getContext(), data.set_size_limit, *data.prepared_sets));
|
||||
return makeExplicitSet(&node, last_actions, data.getContext(), *data.prepared_sets);
|
||||
else
|
||||
return {};
|
||||
}
|
||||
|
@ -25,9 +25,8 @@ class IFunctionOverloadResolver;
|
||||
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
|
||||
|
||||
/// The case of an explicit enumeration of values.
|
||||
SetPtr makeExplicitSet(
|
||||
const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set,
|
||||
ContextPtr context, const SizeLimits & limits, PreparedSets & prepared_sets);
|
||||
FutureSetPtr makeExplicitSet(
|
||||
const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets);
|
||||
|
||||
/** For ActionsVisitor
|
||||
* A stack of ExpressionActions corresponding to nested lambda expressions.
|
||||
@ -217,7 +216,7 @@ private:
|
||||
static void visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data);
|
||||
static void visit(ASTExpressionList & expression_list, const ASTPtr & ast, Data & data);
|
||||
|
||||
static FutureSet makeSet(const ASTFunction & node, Data & data, bool no_subqueries);
|
||||
static FutureSetPtr makeSet(const ASTFunction & node, Data & data, bool no_subqueries);
|
||||
static ASTs doUntuple(const ASTFunction * function, ActionsMatcher::Data & data);
|
||||
static std::optional<NameAndTypePair> getNameAndTypeFromAST(const ASTPtr & ast, Data & data);
|
||||
};
|
||||
|
@ -498,12 +498,14 @@ KeyMetadata::iterator FileCache::addFileSegment(
|
||||
chassert(size > 0); /// Empty file segments in cache are not allowed.
|
||||
|
||||
const auto & key = locked_key.getKey();
|
||||
if (locked_key.tryGetByOffset(offset))
|
||||
const FileSegment::Range range(offset, offset + size - 1);
|
||||
|
||||
if (auto intersecting_range = locked_key.hasIntersectingRange(range))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Cache entry already exists for key: `{}`, offset: {}, size: {}.",
|
||||
key, offset, size);
|
||||
"Attempt to add intersecting file segment in cache ({} intersects {})",
|
||||
range.toString(), intersecting_range->toString());
|
||||
}
|
||||
|
||||
FileSegment::State result_state;
|
||||
@ -958,8 +960,20 @@ void FileCache::loadMetadata()
|
||||
if ((main_priority->getSizeLimit() == 0 || main_priority->getSize(lock) + size <= main_priority->getSizeLimit())
|
||||
&& (main_priority->getElementsLimit() == 0 || main_priority->getElementsCount(lock) + 1 <= main_priority->getElementsLimit()))
|
||||
{
|
||||
auto file_segment_metadata_it = addFileSegment(
|
||||
*locked_key, offset, size, FileSegment::State::DOWNLOADED, CreateFileSegmentSettings(segment_kind), &lock);
|
||||
KeyMetadata::iterator file_segment_metadata_it;
|
||||
try
|
||||
{
|
||||
file_segment_metadata_it = addFileSegment(
|
||||
*locked_key, offset, size, FileSegment::State::DOWNLOADED, CreateFileSegmentSettings(segment_kind), &lock);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
chassert(false);
|
||||
|
||||
fs::remove(offset_it->path());
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto & file_segment_metadata = file_segment_metadata_it->second;
|
||||
chassert(file_segment_metadata->file_segment->assertCorrectness());
|
||||
|
@ -898,7 +898,7 @@ void FileSegment::use()
|
||||
if (it)
|
||||
{
|
||||
auto cache_lock = cache->lockCache();
|
||||
it->use(cache_lock);
|
||||
hits_count = it->use(cache_lock);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -129,6 +129,8 @@ public:
|
||||
|
||||
bool operator==(const Range & other) const { return left == other.left && right == other.right; }
|
||||
|
||||
bool operator<(const Range & other) const { return right < other.left; }
|
||||
|
||||
size_t size() const { return right - left + 1; }
|
||||
|
||||
String toString() const { return fmt::format("[{}, {}]", std::to_string(left), std::to_string(right)); }
|
||||
@ -173,8 +175,6 @@ public:
|
||||
|
||||
size_t getRefCount() const { return ref_count; }
|
||||
|
||||
void incrementHitsCount() { ++hits_count; }
|
||||
|
||||
size_t getCurrentWriteOffset(bool sync) const;
|
||||
|
||||
size_t getFirstNonDownloadedOffset(bool sync) const;
|
||||
|
@ -448,6 +448,29 @@ void LockedKey::shrinkFileSegmentToDownloadedSize(
|
||||
chassert(file_segment->assertCorrectnessUnlocked(segment_lock));
|
||||
}
|
||||
|
||||
std::optional<FileSegment::Range> LockedKey::hasIntersectingRange(const FileSegment::Range & range) const
|
||||
{
|
||||
if (key_metadata->empty())
|
||||
return {};
|
||||
|
||||
auto it = key_metadata->lower_bound(range.left);
|
||||
if (it != key_metadata->end()) /// has next range
|
||||
{
|
||||
auto next_range = it->second->file_segment->range();
|
||||
if (!(range < next_range))
|
||||
return next_range;
|
||||
|
||||
if (it == key_metadata->begin())
|
||||
return {};
|
||||
}
|
||||
|
||||
auto prev_range = std::prev(it)->second->file_segment->range();
|
||||
if (!(prev_range < range))
|
||||
return prev_range;
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
std::shared_ptr<const FileSegmentMetadata> LockedKey::getByOffset(size_t offset) const
|
||||
{
|
||||
auto it = key_metadata->find(offset);
|
||||
|
@ -164,6 +164,8 @@ struct LockedKey : private boost::noncopyable
|
||||
|
||||
bool isLastOwnerOfFileSegment(size_t offset) const;
|
||||
|
||||
std::optional<FileSegment::Range> hasIntersectingRange(const FileSegment::Range & range) const;
|
||||
|
||||
void removeFromCleanupQueue();
|
||||
|
||||
void markAsRemoved();
|
||||
|
@ -132,6 +132,11 @@ QueryCache::Key::Key(
|
||||
{
|
||||
}
|
||||
|
||||
QueryCache::Key::Key(ASTPtr ast_, const String & user_name_)
|
||||
: QueryCache::Key(ast_, {}, user_name_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST or user name
|
||||
{
|
||||
}
|
||||
|
||||
bool QueryCache::Key::operator==(const Key & other) const
|
||||
{
|
||||
return ast->getTreeHash() == other.ast->getTreeHash();
|
||||
@ -387,19 +392,22 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar
|
||||
return;
|
||||
}
|
||||
|
||||
if (!entry->key.is_shared && entry->key.user_name != key.user_name)
|
||||
const auto & entry_key = entry->key;
|
||||
const auto & entry_mapped = entry->mapped;
|
||||
|
||||
if (!entry_key.is_shared && entry_key.user_name != key.user_name)
|
||||
{
|
||||
LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.queryStringFromAst());
|
||||
return;
|
||||
}
|
||||
|
||||
if (IsStale()(entry->key))
|
||||
if (IsStale()(entry_key))
|
||||
{
|
||||
LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found for query {}", key.queryStringFromAst());
|
||||
return;
|
||||
}
|
||||
|
||||
if (!entry->key.is_compressed)
|
||||
if (!entry_key.is_compressed)
|
||||
{
|
||||
// Cloning chunks isn't exactly great. It could be avoided by another indirection, i.e. wrapping Entry's members chunks, totals and
|
||||
// extremes into shared_ptrs and assuming that the lifecycle of these shared_ptrs coincides with the lifecycle of the Entry
|
||||
@ -408,15 +416,15 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar
|
||||
// optimization.
|
||||
|
||||
Chunks cloned_chunks;
|
||||
for (const auto & chunk : entry->mapped->chunks)
|
||||
for (const auto & chunk : entry_mapped->chunks)
|
||||
cloned_chunks.push_back(chunk.clone());
|
||||
|
||||
buildSourceFromChunks(entry->key.header, std::move(cloned_chunks), entry->mapped->totals, entry->mapped->extremes);
|
||||
buildSourceFromChunks(entry_key.header, std::move(cloned_chunks), entry_mapped->totals, entry_mapped->extremes);
|
||||
}
|
||||
else
|
||||
{
|
||||
Chunks decompressed_chunks;
|
||||
const Chunks & chunks = entry->mapped->chunks;
|
||||
const Chunks & chunks = entry_mapped->chunks;
|
||||
for (const auto & chunk : chunks)
|
||||
{
|
||||
const Columns & columns = chunk.getColumns();
|
||||
@ -430,7 +438,7 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar
|
||||
decompressed_chunks.push_back(std::move(decompressed_chunk));
|
||||
}
|
||||
|
||||
buildSourceFromChunks(entry->key.header, std::move(decompressed_chunks), entry->mapped->totals, entry->mapped->extremes);
|
||||
buildSourceFromChunks(entry_key.header, std::move(decompressed_chunks), entry_mapped->totals, entry_mapped->extremes);
|
||||
}
|
||||
|
||||
LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.queryStringFromAst());
|
||||
|
@ -49,7 +49,7 @@ public:
|
||||
/// If the associated entry can be read by other users. In general, sharing is a bad idea: First, it is unlikely that different
|
||||
/// users pose the same queries. Second, sharing potentially breaches security. E.g. User A should not be able to bypass row
|
||||
/// policies on some table by running the same queries as user B for whom no row policies exist.
|
||||
bool is_shared;
|
||||
const bool is_shared;
|
||||
|
||||
/// When does the entry expire?
|
||||
const std::chrono::time_point<std::chrono::system_clock> expires_at;
|
||||
@ -58,12 +58,16 @@ public:
|
||||
/// (we could theoretically apply compression also to the totals and extremes but it's an obscure use case)
|
||||
const bool is_compressed;
|
||||
|
||||
/// Ctor to construct a Key for writing into query cache.
|
||||
Key(ASTPtr ast_,
|
||||
Block header_,
|
||||
const String & user_name_, bool is_shared_,
|
||||
std::chrono::time_point<std::chrono::system_clock> expires_at_,
|
||||
bool is_compressed);
|
||||
|
||||
/// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name).
|
||||
Key(ASTPtr ast_, const String & user_name_);
|
||||
|
||||
bool operator==(const Key & other) const;
|
||||
String queryStringFromAst() const;
|
||||
};
|
||||
|
@ -26,6 +26,8 @@ using QueryPlanPtr = std::unique_ptr<QueryPlan>;
|
||||
|
||||
struct StorageID;
|
||||
|
||||
class PreparedSets;
|
||||
using PreparedSetsPtr = std::shared_ptr<PreparedSets>;
|
||||
namespace ClusterProxy
|
||||
{
|
||||
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1319,6 +1319,21 @@ void Context::addExternalTable(const String & table_name, TemporaryTableHolder &
|
||||
external_tables_mapping.emplace(table_name, std::make_shared<TemporaryTableHolder>(std::move(temporary_table)));
|
||||
}
|
||||
|
||||
std::shared_ptr<TemporaryTableHolder> Context::findExternalTable(const String & table_name) const
|
||||
{
|
||||
if (isGlobalContext())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have external tables");
|
||||
|
||||
std::shared_ptr<TemporaryTableHolder> holder;
|
||||
{
|
||||
auto lock = getLock();
|
||||
auto iter = external_tables_mapping.find(table_name);
|
||||
if (iter == external_tables_mapping.end())
|
||||
return {};
|
||||
holder = iter->second;
|
||||
}
|
||||
return holder;
|
||||
}
|
||||
|
||||
std::shared_ptr<TemporaryTableHolder> Context::removeExternalTable(const String & table_name)
|
||||
{
|
||||
|
@ -615,6 +615,7 @@ public:
|
||||
|
||||
Tables getExternalTables() const;
|
||||
void addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table);
|
||||
std::shared_ptr<TemporaryTableHolder> findExternalTable(const String & table_name) const;
|
||||
std::shared_ptr<TemporaryTableHolder> removeExternalTable(const String & table_name);
|
||||
|
||||
const Scalars & getScalars() const;
|
||||
|
@ -110,7 +110,7 @@ TemporaryTableHolder::TemporaryTableHolder(
|
||||
}
|
||||
|
||||
TemporaryTableHolder::TemporaryTableHolder(TemporaryTableHolder && rhs) noexcept
|
||||
: WithContext(rhs.context), temporary_tables(rhs.temporary_tables), id(rhs.id)
|
||||
: WithContext(rhs.context), temporary_tables(rhs.temporary_tables), id(rhs.id), future_set(std::move(rhs.future_set))
|
||||
{
|
||||
rhs.id = UUIDHelpers::Nil;
|
||||
}
|
||||
@ -216,8 +216,24 @@ void DatabaseCatalog::shutdownImpl()
|
||||
|
||||
/// We still hold "databases" (instead of std::move) for Buffer tables to flush data correctly.
|
||||
|
||||
/// Delay shutdown of temporary and system databases. They will be shutdown last.
|
||||
/// Because some databases might use them until their shutdown is called, but calling shutdown
|
||||
/// on temporary database means clearing its set of tables, which will lead to unnecessary errors like "table not found".
|
||||
std::vector<DatabasePtr> databases_with_delayed_shutdown;
|
||||
for (auto & database : current_databases)
|
||||
{
|
||||
if (database.first == TEMPORARY_DATABASE || database.first == SYSTEM_DATABASE)
|
||||
{
|
||||
databases_with_delayed_shutdown.push_back(database.second);
|
||||
continue;
|
||||
}
|
||||
database.second->shutdown();
|
||||
}
|
||||
|
||||
for (auto & database : databases_with_delayed_shutdown)
|
||||
{
|
||||
database->shutdown();
|
||||
}
|
||||
|
||||
{
|
||||
std::lock_guard lock(tables_marked_dropped_mutex);
|
||||
|
@ -79,6 +79,8 @@ private:
|
||||
|
||||
using DDLGuardPtr = std::unique_ptr<DDLGuard>;
|
||||
|
||||
class FutureSet;
|
||||
using FutureSetPtr = std::shared_ptr<FutureSet>;
|
||||
|
||||
/// Creates temporary table in `_temporary_and_external_tables` with randomly generated unique StorageID.
|
||||
/// Such table can be accessed from everywhere by its ID.
|
||||
@ -111,6 +113,7 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext
|
||||
|
||||
IDatabase * temporary_tables = nullptr;
|
||||
UUID id = UUIDHelpers::Nil;
|
||||
FutureSetPtr future_set;
|
||||
};
|
||||
|
||||
///TODO maybe remove shared_ptr from here?
|
||||
|
@ -936,15 +936,12 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con
|
||||
for (const auto & action : actions)
|
||||
{
|
||||
if (action.node->type == ActionsDAG::ActionType::COLUMN && action.node->result_name == set_to_check)
|
||||
{
|
||||
// Constant ColumnSet cannot be empty, so we only need to check non-constant ones.
|
||||
if (const auto * column_set = checkAndGetColumn<const ColumnSet>(action.node->column.get()))
|
||||
{
|
||||
auto set = column_set->getData();
|
||||
if (set && set->isCreated() && set->getTotalRowCount() == 0)
|
||||
return true;
|
||||
}
|
||||
}
|
||||
if (auto future_set = column_set->getData())
|
||||
if (auto set = future_set->get())
|
||||
if (set->getTotalRowCount() == 0)
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -450,77 +450,6 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global, b
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options)
|
||||
{
|
||||
if (!prepared_sets)
|
||||
return;
|
||||
|
||||
auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name);
|
||||
|
||||
if (prepared_sets->getFuture(set_key).isValid())
|
||||
return; /// Already prepared.
|
||||
|
||||
if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name))
|
||||
{
|
||||
prepared_sets->set(set_key, set_ptr_from_storage_set);
|
||||
return;
|
||||
}
|
||||
|
||||
auto build_set = [&] () -> SetPtr
|
||||
{
|
||||
LOG_TRACE(getLogger(), "Building set, key: {}", set_key.toString());
|
||||
|
||||
auto interpreter_subquery = interpretSubquery(subquery_or_table_name, getContext(), {}, query_options);
|
||||
auto io = interpreter_subquery->execute();
|
||||
PullingAsyncPipelineExecutor executor(io.pipeline);
|
||||
|
||||
SetPtr set = std::make_shared<Set>(settings.size_limits_for_set_used_with_index, true, getContext()->getSettingsRef().transform_null_in);
|
||||
set->setHeader(executor.getHeader().getColumnsWithTypeAndName());
|
||||
|
||||
Block block;
|
||||
while (executor.pull(block))
|
||||
{
|
||||
if (block.rows() == 0)
|
||||
continue;
|
||||
|
||||
/// If the limits have been exceeded, give up and let the default subquery processing actions take place.
|
||||
if (!set->insertFromBlock(block.getColumnsWithTypeAndName()))
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
set->finishInsert();
|
||||
|
||||
return set;
|
||||
};
|
||||
|
||||
SetPtr set;
|
||||
|
||||
auto set_cache = getContext()->getPreparedSetsCache();
|
||||
if (set_cache)
|
||||
{
|
||||
auto from_cache = set_cache->findOrPromiseToBuild(set_key.toString());
|
||||
if (from_cache.index() == 0)
|
||||
{
|
||||
set = build_set();
|
||||
std::get<0>(from_cache).set_value(set);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_TRACE(getLogger(), "Waiting for set, key: {}", set_key.toString());
|
||||
set = std::get<1>(from_cache).get();
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
set = build_set();
|
||||
}
|
||||
|
||||
if (!set)
|
||||
return;
|
||||
|
||||
prepared_sets->set(set_key, std::move(set));
|
||||
}
|
||||
|
||||
SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name)
|
||||
{
|
||||
const auto * table = subquery_or_table_name->as<ASTTableIdentifier>();
|
||||
@ -534,54 +463,6 @@ SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_o
|
||||
return storage_set->getSet();
|
||||
}
|
||||
|
||||
|
||||
/// Performance optimization for IN() if storage supports it.
|
||||
void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
|
||||
{
|
||||
if (!node || !storage() || !storage()->supportsIndexForIn())
|
||||
return;
|
||||
|
||||
for (auto & child : node->children)
|
||||
{
|
||||
/// Don't descend into subqueries.
|
||||
if (child->as<ASTSubquery>())
|
||||
continue;
|
||||
|
||||
/// Don't descend into lambda functions
|
||||
const auto * func = child->as<ASTFunction>();
|
||||
if (func && func->name == "lambda")
|
||||
continue;
|
||||
|
||||
makeSetsForIndex(child);
|
||||
}
|
||||
|
||||
const auto * func = node->as<ASTFunction>();
|
||||
if (func && functionIsInOrGlobalInOperator(func->name))
|
||||
{
|
||||
const IAST & args = *func->arguments;
|
||||
const ASTPtr & left_in_operand = args.children.at(0);
|
||||
|
||||
if (storage()->mayBenefitFromIndexForIn(left_in_operand, getContext(), metadata_snapshot))
|
||||
{
|
||||
const ASTPtr & arg = args.children.at(1);
|
||||
if (arg->as<ASTSubquery>() || arg->as<ASTTableIdentifier>())
|
||||
{
|
||||
if (settings.use_index_for_in_with_subqueries)
|
||||
tryMakeSetForIndexFromSubquery(arg, query_options);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto temp_actions = std::make_shared<ActionsDAG>(columns_after_join);
|
||||
getRootActions(left_in_operand, true, temp_actions);
|
||||
|
||||
if (prepared_sets && temp_actions->tryFindInOutputs(left_in_operand->getColumnName()))
|
||||
makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, *prepared_sets);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts)
|
||||
{
|
||||
LogAST log;
|
||||
|
@ -141,11 +141,6 @@ public:
|
||||
void makeWindowDescriptionFromAST(const Context & context, const WindowDescriptions & existing_descriptions, WindowDescription & desc, const IAST * ast);
|
||||
void makeWindowDescriptions(ActionsDAGPtr actions);
|
||||
|
||||
/** Create Set from a subquery or a table expression in the query. The created set is suitable for using the index.
|
||||
* The set will not be created if its size hits the limit.
|
||||
*/
|
||||
void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options = {});
|
||||
|
||||
/** Checks if subquery is not a plain StorageSet.
|
||||
* Because while making set we will read data from StorageSet which is not allowed.
|
||||
* Returns valid SetPtr from StorageSet if the latter is used after IN or nullptr otherwise.
|
||||
@ -363,9 +358,6 @@ public:
|
||||
/// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases.
|
||||
ActionsDAGPtr appendProjectResult(ExpressionActionsChain & chain) const;
|
||||
|
||||
/// Create Set-s that we make from IN section to use index on them.
|
||||
void makeSetsForIndex(const ASTPtr & node);
|
||||
|
||||
private:
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
/// If non-empty, ignore all expressions not from this list.
|
||||
|
@ -31,6 +31,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int WRONG_GLOBAL_SUBQUERY;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
class GlobalSubqueriesMatcher
|
||||
@ -161,30 +162,20 @@ public:
|
||||
nullptr,
|
||||
/*create_for_global_subquery*/ true);
|
||||
StoragePtr external_storage = external_storage_holder->getTable();
|
||||
|
||||
external_tables.emplace(external_table_name, external_storage_holder);
|
||||
|
||||
/// We need to materialize external tables immediately because reading from distributed
|
||||
/// tables might generate local plans which can refer to external tables during index
|
||||
/// analysis. It's too late to populate the external table via CreatingSetsTransform.
|
||||
if (is_explain)
|
||||
auto set_key = database_and_table_name->getTreeHash();
|
||||
|
||||
if (!prepared_sets->findSubquery(set_key))
|
||||
{
|
||||
/// Do not materialize external tables if it's explain statement.
|
||||
}
|
||||
else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries)
|
||||
{
|
||||
auto external_table = external_storage_holder->getTable();
|
||||
auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext(), /*async_insert=*/false);
|
||||
auto io = interpreter->execute();
|
||||
io.pipeline.complete(std::move(table_out));
|
||||
CompletedPipelineExecutor executor(io.pipeline);
|
||||
executor.execute();
|
||||
std::unique_ptr<QueryPlan> source = std::make_unique<QueryPlan>();
|
||||
interpreter->buildQueryPlan(*source);
|
||||
|
||||
auto future_set = prepared_sets->addFromSubquery(set_key, std::move(source), std::move(external_storage), nullptr, getContext()->getSettingsRef());
|
||||
external_storage_holder->future_set = std::move(future_set);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto & subquery_for_set = prepared_sets->getSubquery(external_table_name);
|
||||
subquery_for_set.createSource(*interpreter, external_storage);
|
||||
}
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Set is already created for GLOBAL IN");
|
||||
|
||||
/** NOTE If it was written IN tmp_table - the existing temporary (but not external) table,
|
||||
* then a new temporary table will be created (for example, _data1),
|
||||
|
@ -953,10 +953,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
|
||||
if (storage && !options.only_analyze)
|
||||
{
|
||||
query_analyzer->makeSetsForIndex(select_query.where());
|
||||
query_analyzer->makeSetsForIndex(select_query.prewhere());
|
||||
query_info.prepared_sets = query_analyzer->getPreparedSets();
|
||||
|
||||
from_stage = storage->getQueryProcessingStage(context, options.to_stage, storage_snapshot, query_info);
|
||||
}
|
||||
|
||||
@ -3151,7 +3148,17 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan)
|
||||
|
||||
void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan)
|
||||
{
|
||||
addCreatingSetsStep(query_plan, prepared_sets, context);
|
||||
auto subqueries = prepared_sets->getSubqueries();
|
||||
|
||||
if (!subqueries.empty())
|
||||
{
|
||||
auto step = std::make_unique<DelayedCreatingSetsStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
std::move(subqueries),
|
||||
context);
|
||||
|
||||
query_plan.addStep(std::move(step));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user