diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index e28c486afca..d384ed639eb 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1203,12 +1203,14 @@ SELECT * FROM json_each_row_nested - [input_format_json_read_bools_as_numbers](/docs/en/operations/settings/settings.md/#input_format_json_read_bools_as_numbers) - allow to parse bools as numbers in JSON input formats. Default value - `true`. - [input_format_json_read_numbers_as_strings](/docs/en/operations/settings/settings.md/#input_format_json_read_numbers_as_strings) - allow to parse numbers as strings in JSON input formats. Default value - `false`. - [input_format_json_read_objects_as_strings](/docs/en/operations/settings/settings.md/#input_format_json_read_objects_as_strings) - allow to parse JSON objects as strings in JSON input formats. Default value - `false`. +- [input_format_json_named_tuples_as_objects](/docs/en/operations/settings/settings.md/#input_format_json_named_tuples_as_objects) - parse named tuple columns as JSON objects. Default value - `true`. +- [input_format_json_defaults_for_missing_elements_in_named_tuple](/docs/en/operations/settings/settings.md/#input_format_json_defaults_for_missing_elements_in_named_tuple) - insert default values for missing elements in JSON object while parsing named tuple. Default value - `true`. - [output_format_json_quote_64bit_integers](/docs/en/operations/settings/settings.md/#output_format_json_quote_64bit_integers) - controls quoting of 64-bit integers in JSON output format. Default value - `true`. - [output_format_json_quote_64bit_floats](/docs/en/operations/settings/settings.md/#output_format_json_quote_64bit_floats) - controls quoting of 64-bit floats in JSON output format. Default value - `false`. - [output_format_json_quote_denormals](/docs/en/operations/settings/settings.md/#output_format_json_quote_denormals) - enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format. Default value - `false`. - [output_format_json_quote_decimals](/docs/en/operations/settings/settings.md/#output_format_json_quote_decimals) - controls quoting of decimals in JSON output format. Default value - `false`. - [output_format_json_escape_forward_slashes](/docs/en/operations/settings/settings.md/#output_format_json_escape_forward_slashes) - controls escaping forward slashes for string outputs in JSON output format. Default value - `true`. -- [output_format_json_named_tuples_as_objects](/docs/en/operations/settings/settings.md/#output_format_json_named_tuples_as_objects) - serialize named tuple columns as JSON objects. Default value - `false`. +- [output_format_json_named_tuples_as_objects](/docs/en/operations/settings/settings.md/#output_format_json_named_tuples_as_objects) - serialize named tuple columns as JSON objects. Default value - `true`. - [output_format_json_array_of_rows](/docs/en/operations/settings/settings.md/#output_format_json_array_of_rows) - output a JSON array of all rows in JSONEachRow(Compact) format. Default value - `false`. - [output_format_json_validate_utf8](/docs/en/operations/settings/settings.md/#output_format_json_validate_utf8) - enables validation of UTF-8 sequences in JSON output formats (note that it doesn't impact formats JSON/JSONCompact/JSONColumnsWithMetadata, they always validate utf8). Default value - `false`. diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index 376b7480358..7a6b2340d29 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -266,7 +266,7 @@ Default value: 0. Limits the size in bytes of the hash table used when joining tables. -This settings applies to [SELECT … JOIN](../../sql-reference/statements/select/join.md#select-join) operations and [Join table engine](../../engines/table-engines/special/join.md). +This setting applies to [SELECT … JOIN](../../sql-reference/statements/select/join.md#select-join) operations and [Join table engine](../../engines/table-engines/special/join.md). If the query contains joins, ClickHouse checks this setting for every intermediate result. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4ffe2bbc7c4..9def33debbd 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -402,40 +402,62 @@ Default value: `ALL`. ## join_algorithm {#settings-join_algorithm} -Specifies [JOIN](../../sql-reference/statements/select/join.md) algorithm. +Specifies which [JOIN](../../sql-reference/statements/select/join.md) algorithm is used. Several algorithms can be specified, and an available one would be chosen for a particular query based on kind/strictness and table engine. Possible values: -- `default` — `hash` or `direct`, if possible (same as `direct,hash`) +### `default` -- `hash` — [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. +This is the equivalent of `hash` or `direct`, if possible (same as `direct,hash`) -- `parallel_hash` - a variation of `hash` join that splits the data into buckets and builds several hashtables instead of one concurrently to speed up this process. +### `grace_hash` + +[Grace hash join](https://en.wikipedia.org/wiki/Hash_join#Grace_hash_join) is used. Grace hash provides an algorithm option that provides performant complex joins while limiting memory use. + +The first phase of a grace join reads the right table and splits it into N buckets depending on the hash value of key columns (initially, N is `grace_hash_join_initial_buckets`). This is done in a way to ensure that each bucket can be processed independently. Rows from the first bucket are added to an in-memory hash table while the others are saved to disk. If the hash table grows beyond the memory limit (e.g., as set by [`max_bytes_in_join`](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join)), the number of buckets is increased and the assigned bucket for each row. Any rows which don’t belong to the current bucket are flushed and reassigned. + +### `hash` + +[Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. + +### `parallel_hash` + +A variation of `hash` join that splits the data into buckets and builds several hashtables instead of one concurrently to speed up this process. When using the `hash` algorithm, the right part of `JOIN` is uploaded into RAM. -- `partial_merge` — a variation of the [sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join), where only the right table is fully sorted. +### `partial_merge` + +A variation of the [sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join), where only the right table is fully sorted. The `RIGHT JOIN` and `FULL JOIN` are supported only with `ALL` strictness (`SEMI`, `ANTI`, `ANY`, and `ASOF` are not supported). -When using `partial_merge` algorithm, ClickHouse sorts the data and dumps it to the disk. The `partial_merge` algorithm in ClickHouse differs slightly from the classic realization. First, ClickHouse sorts the right table by joining keys in blocks and creates a min-max index for sorted blocks. Then it sorts parts of the left table by `join key` and joins them over the right table. The min-max index is also used to skip unneeded right table blocks. +When using the `partial_merge` algorithm, ClickHouse sorts the data and dumps it to the disk. The `partial_merge` algorithm in ClickHouse differs slightly from the classic realization. First, ClickHouse sorts the right table by joining keys in blocks and creates a min-max index for sorted blocks. Then it sorts parts of the left table by the `join key` and joins them over the right table. The min-max index is also used to skip unneeded right table blocks. -- `direct` - can be applied when the right storage supports key-value requests. +### `direct` + +This algorithm can be applied when the storage for the right table supports key-value requests. The `direct` algorithm performs a lookup in the right table using rows from the left table as keys. It's supported only by special storage such as [Dictionary](../../engines/table-engines/special/dictionary.md/#dictionary) or [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) and only the `LEFT` and `INNER` JOINs. -- `auto` — try `hash` join and switch on the fly to another algorithm if the memory limit is violated. +### `auto` -- `full_sorting_merge` — [Sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join) with full sorting joined tables before joining. +When set to `auto`, `hash` join is tried first, and the algorithm is switched on the fly to another algorithm if the memory limit is violated. -- `prefer_partial_merge` — ClickHouse always tries to use `partial_merge` join if possible, otherwise, it uses `hash`. *Deprecated*, same as `partial_merge,hash`. +### `full_sorting_merge` + +[Sort-merge algorithm](https://en.wikipedia.org/wiki/Sort-merge_join) with full sorting joined tables before joining. + +### `prefer_partial_merge` + +ClickHouse always tries to use `partial_merge` join if possible, otherwise, it uses `hash`. *Deprecated*, same as `partial_merge,hash`. ## join_any_take_last_row {#settings-join_any_take_last_row} -Changes behaviour of join operations with `ANY` strictness. +Changes the behaviour of join operations with `ANY` strictness. :::warning This setting applies only for `JOIN` operations with [Join](../../engines/table-engines/special/join.md) engine tables. @@ -498,7 +520,7 @@ Default value: 65536. Limits the number of files allowed for parallel sorting in MergeJoin operations when they are executed on disk. -The bigger the value of the setting, the more RAM used and the less disk I/O needed. +The bigger the value of the setting, the more RAM is used and the less disk I/O is needed. Possible values: @@ -514,12 +536,12 @@ Enables legacy ClickHouse server behaviour in `ANY INNER|LEFT JOIN` operations. Use this setting only for backward compatibility if your use cases depend on legacy `JOIN` behaviour. ::: -When the legacy behaviour enabled: +When the legacy behaviour is enabled: - Results of `t1 ANY LEFT JOIN t2` and `t2 ANY RIGHT JOIN t1` operations are not equal because ClickHouse uses the logic with many-to-one left-to-right table keys mapping. - Results of `ANY INNER JOIN` operations contain all rows from the left table like the `SEMI LEFT JOIN` operations do. -When the legacy behaviour disabled: +When the legacy behaviour is disabled: - Results of `t1 ANY LEFT JOIN t2` and `t2 ANY RIGHT JOIN t1` operations are equal because ClickHouse uses the logic which provides one-to-many keys mapping in `ANY RIGHT JOIN` operations. - Results of `ANY INNER JOIN` operations contain one row per key from both the left and right tables. @@ -572,7 +594,7 @@ Default value: `163840`. ## merge_tree_min_rows_for_concurrent_read_for_remote_filesystem {#merge-tree-min-rows-for-concurrent-read-for-remote-filesystem} -The minimum number of lines to read from one file before [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can parallelize reading, when reading from remote filesystem. +The minimum number of lines to read from one file before the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can parallelize reading, when reading from remote filesystem. Possible values: @@ -706,7 +728,7 @@ log_queries=1 ## log_queries_min_query_duration_ms {#settings-log-queries-min-query-duration-ms} -If enabled (non-zero), queries faster then the value of this setting will not be logged (you can think about this as a `long_query_time` for [MySQL Slow Query Log](https://dev.mysql.com/doc/refman/5.7/en/slow-query-log.html)), and this basically means that you will not find them in the following tables: +If enabled (non-zero), queries faster than the value of this setting will not be logged (you can think about this as a `long_query_time` for [MySQL Slow Query Log](https://dev.mysql.com/doc/refman/5.7/en/slow-query-log.html)), and this basically means that you will not find them in the following tables: - `system.query_log` - `system.query_thread_log` @@ -741,7 +763,7 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' Setting up query threads logging. -Query threads log into [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting have effect only when [log_queries](#settings-log-queries) is true. Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_thread_log) server configuration parameter. +Query threads log into the [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting has effect only when [log_queries](#settings-log-queries) is true. Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_thread_log) server configuration parameter. Possible values: @@ -760,7 +782,7 @@ log_query_threads=1 Setting up query views logging. -When a query run by ClickHouse with this setup on has associated views (materialized or live views), they are logged in the [query_views_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_views_log) server configuration parameter. +When a query run by ClickHouse with this setting enabled has associated views (materialized or live views), they are logged in the [query_views_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_views_log) server configuration parameter. Example: @@ -787,7 +809,7 @@ It can be used to improve the readability of server logs. Additionally, it helps Possible values: -- Any string no longer than [max_query_size](#settings-max_query_size). If length is exceeded, the server throws an exception. +- Any string no longer than [max_query_size](#settings-max_query_size). If the max_query_size is exceeded, the server throws an exception. Default value: empty string. @@ -821,11 +843,11 @@ The setting also does not have a purpose when using INSERT SELECT, since data is Default value: 1,048,576. -The default is slightly more than `max_block_size`. The reason for this is because certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allow sorting more data in RAM. +The default is slightly more than `max_block_size`. The reason for this is that certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allow sorting more data in RAM. ## min_insert_block_size_rows {#min-insert-block-size-rows} -Sets the minimum number of rows in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. +Sets the minimum number of rows in the block that can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. Possible values: @@ -891,7 +913,7 @@ Higher values will lead to higher memory usage. ## max_compress_block_size {#max-compress-block-size} -The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). Specifying smaller block size generally leads to slightly reduced compression ratio, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced. +The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). Specifying a smaller block size generally leads to slightly reduced compression ratio, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced. :::warning This is an expert-level setting, and you shouldn't change it if you're just getting started with ClickHouse. @@ -935,7 +957,7 @@ Default value: 1000. ## interactive_delay {#interactive-delay} -The interval in microseconds for checking whether request execution has been cancelled and sending the progress. +The interval in microseconds for checking whether request execution has been canceled and sending the progress. Default value: 100,000 (checks for cancelling and sends the progress ten times per second). @@ -4122,7 +4144,20 @@ Enabled by default. Serialize named tuple columns as JSON objects. -Disabled by default. +Enabled by default. + +### input_format_json_named_tuples_as_objects {#input_format_json_named_tuples_as_objects} + +Parse named tuple columns as JSON objects. + +Enabled by default. + +### input_format_json_defaults_for_missing_elements_in_named_tuple {#input_format_json_defaults_for_missing_elements_in_named_tuple} + +Insert default values for missing elements in JSON object while parsing named tuple. +This setting works only when setting `input_format_json_named_tuples_as_objects` is enabled. + +Enabled by default. ### output_format_json_array_of_rows {#output_format_json_array_of_rows} diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantileinterpolatedweighted.md b/docs/en/sql-reference/aggregate-functions/reference/quantileinterpolatedweighted.md new file mode 100644 index 00000000000..07fcd187217 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/quantileinterpolatedweighted.md @@ -0,0 +1,68 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/quantileInterpolatedWeighted +sidebar_position: 203 +--- + +# quantileInterpolatedWeighted + +Computes [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence using linear interpolation, taking into account the weight of each element. + +To get the interpolated value, all the passed values are combined into an array, which are then sorted by their corresponding weights. Quantile interpolation is then performed using the [weighted percentile method](https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method) by building a cumulative distribution based on weights and then a linear interpolation is performed using the weights and the values to compute the quantiles. + +When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function. + +**Syntax** + +``` sql +quantileInterpolatedWeighted(level)(expr, weight) +``` + +Alias: `medianInterpolatedWeighted`. + +**Arguments** + +- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median). +- `expr` — Expression over the column values resulting in numeric [data types](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md). +- `weight` — Column with weights of sequence members. Weight is a number of value occurrences. + +**Returned value** + +- Quantile of the specified level. + +Type: + +- [Float64](../../../sql-reference/data-types/float.md) for numeric data type input. +- [Date](../../../sql-reference/data-types/date.md) if input values have the `Date` type. +- [DateTime](../../../sql-reference/data-types/datetime.md) if input values have the `DateTime` type. + +**Example** + +Input table: + +``` text +┌─n─┬─val─┐ +│ 0 │ 3 │ +│ 1 │ 2 │ +│ 2 │ 1 │ +│ 5 │ 4 │ +└───┴─────┘ +``` + +Query: + +``` sql +SELECT quantileInterpolatedWeighted(n, val) FROM t +``` + +Result: + +``` text +┌─quantileInterpolatedWeighted(n, val)─┐ +│ 1 │ +└──────────────────────────────────────┘ +``` + +**See Also** + +- [median](../../../sql-reference/aggregate-functions/reference/median.md#median) +- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiles.md b/docs/en/sql-reference/aggregate-functions/reference/quantiles.md index 5c9120fb8f4..57151915336 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiles.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiles.md @@ -9,7 +9,7 @@ sidebar_position: 201 Syntax: `quantiles(level1, level2, …)(x)` -All the quantile functions also have corresponding quantiles functions: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantilesTDigest`, `quantilesBFloat16`. These functions calculate all the quantiles of the listed levels in one pass, and return an array of the resulting values. +All the quantile functions also have corresponding quantiles functions: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantileInterpolatedWeighted`, `quantilesTDigest`, `quantilesBFloat16`. These functions calculate all the quantiles of the listed levels in one pass, and return an array of the resulting values. ## quantilesExactExclusive diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 742838d6433..419b80ccff2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -140,6 +140,7 @@ namespace CurrentMetrics namespace ProfileEvents { extern const Event MainConfigLoads; + extern const Event ServerStartupMilliseconds; } namespace fs = std::filesystem; @@ -652,6 +653,8 @@ static void sanityChecks(Server & server) int Server::main(const std::vector & /*args*/) try { + Stopwatch startup_watch; + Poco::Logger * log = &logger(); UseSSL use_ssl; @@ -1822,6 +1825,9 @@ try LOG_INFO(log, "Ready for connections."); } + startup_watch.stop(); + ProfileEvents::increment(ProfileEvents::ServerStartupMilliseconds, startup_watch.elapsedMilliseconds()); + try { global_context->startClusterDiscovery(); diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 366667410d5..f1f99fc9166 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -167,6 +167,7 @@ enum class AccessType M(SYSTEM_SYNC_REPLICA, "SYNC REPLICA", TABLE, SYSTEM) \ M(SYSTEM_RESTART_REPLICA, "RESTART REPLICA", TABLE, SYSTEM) \ M(SYSTEM_RESTORE_REPLICA, "RESTORE REPLICA", TABLE, SYSTEM) \ + M(SYSTEM_WAIT_LOADING_PARTS, "WAIT LOADING PARTS", TABLE, SYSTEM) \ M(SYSTEM_SYNC_DATABASE_REPLICA, "SYNC DATABASE REPLICA", DATABASE, SYSTEM) \ M(SYSTEM_SYNC_TRANSACTION_LOG, "SYNC TRANSACTION LOG", GLOBAL, SYSTEM) \ M(SYSTEM_FLUSH_DISTRIBUTED, "FLUSH DISTRIBUTED", TABLE, SYSTEM_FLUSH) \ diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 02aafb7415b..e21ebda2a31 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -53,7 +53,7 @@ TEST(AccessRights, Union) "SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " "SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, " "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " - "SYSTEM RESTORE REPLICA, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*"); + "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*"); } diff --git a/src/AggregateFunctions/AggregateFunctionHistogram.h b/src/AggregateFunctions/AggregateFunctionHistogram.h index c559b3f115f..ac81f7466fa 100644 --- a/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -207,7 +207,7 @@ private: { // Fuse points if their text representations differ only in last digit auto min_diff = 10 * (points[left].mean + points[right].mean) * std::numeric_limits::epsilon(); - if (points[left].mean + min_diff >= points[right].mean) + if (points[left].mean + std::fabs(min_diff) >= points[right].mean) { points[left] = points[left] + points[right]; } diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.h b/src/AggregateFunctions/AggregateFunctionQuantile.h index 6427d03f089..49157acf690 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -232,6 +232,9 @@ struct NameQuantilesExactInclusive { static constexpr auto name = "quantilesExac struct NameQuantileExactWeighted { static constexpr auto name = "quantileExactWeighted"; }; struct NameQuantilesExactWeighted { static constexpr auto name = "quantilesExactWeighted"; }; +struct NameQuantileInterpolatedWeighted { static constexpr auto name = "quantileInterpolatedWeighted"; }; +struct NameQuantilesInterpolatedWeighted { static constexpr auto name = "quantilesInterpolatedWeighted"; }; + struct NameQuantileTiming { static constexpr auto name = "quantileTiming"; }; struct NameQuantileTimingWeighted { static constexpr auto name = "quantileTimingWeighted"; }; struct NameQuantilesTiming { static constexpr auto name = "quantilesTiming"; }; diff --git a/src/AggregateFunctions/AggregateFunctionQuantileInterpolatedWeighted.cpp b/src/AggregateFunctions/AggregateFunctionQuantileInterpolatedWeighted.cpp new file mode 100644 index 00000000000..68b42376df7 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionQuantileInterpolatedWeighted.cpp @@ -0,0 +1,70 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +struct Settings; + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + + template using FuncQuantileInterpolatedWeighted = AggregateFunctionQuantile, NameQuantileInterpolatedWeighted, true, void, false>; + template using FuncQuantilesInterpolatedWeighted = AggregateFunctionQuantile, NameQuantilesInterpolatedWeighted, true, void, true>; + + template