Merge remote-tracking branch 'origin' into try-llvm-12

This commit is contained in:
Yatsishin Ilya 2021-05-27 17:12:16 +03:00
commit 438abac5fe
272 changed files with 6298 additions and 920 deletions

View File

@ -13,3 +13,6 @@ ClickHouse® is an open-source column-oriented database management system that a
* [Code Browser](https://clickhouse.tech/codebrowser/html_report/ClickHouse/index.html) with syntax highlight and navigation.
* [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any.
* You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person.
## Upcoming Events
* [SF Bay Area ClickHouse Community Meetup (online)](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/278144089/) on 16 June 2021.

View File

@ -552,6 +552,63 @@ create table query_metric_stats_denorm engine File(TSVWithNamesAndTypes,
order by test, query_index, metric_name
;
" 2> >(tee -a analyze/errors.log 1>&2)
# Fetch historical query variability thresholds from the CI database
clickhouse-local --query "
left join file('analyze/report-thresholds.tsv', TSV,
'test text, report_threshold float') thresholds
on query_metric_stats.test = thresholds.test
"
if [ -v CHPC_DATABASE_URL ]
then
set +x # Don't show password in the log
client=(clickhouse-client
# Surprisingly, clickhouse-client doesn't understand --host 127.0.0.1:9000
# so I have to extract host and port with clickhouse-local. I tried to use
# Poco URI parser to support this in the client, but it's broken and can't
# parse host:port.
$(clickhouse-local --query "with '${CHPC_DATABASE_URL}' as url select '--host ' || domain(url) || ' --port ' || toString(port(url)) format TSV")
--secure
--user "${CHPC_DATABASE_USER}"
--password "${CHPC_DATABASE_PASSWORD}"
--config "right/config/client_config.xml"
--database perftest
--date_time_input_format=best_effort)
# Precision is going to be 1.5 times worse for PRs. How do I know it? I ran this:
# SELECT quantilesExact(0., 0.1, 0.5, 0.75, 0.95, 1.)(p / m)
# FROM
# (
# SELECT
# quantileIf(0.95)(stat_threshold, pr_number = 0) AS m,
# quantileIf(0.95)(stat_threshold, (pr_number != 0) AND (abs(diff) < stat_threshold)) AS p
# FROM query_metrics_v2
# WHERE (event_date > (today() - toIntervalMonth(1))) AND (metric = 'client_time')
# GROUP BY
# test,
# query_index,
# query_display_name
# HAVING count(*) > 100
# )
# The file can be empty if the server is inaccessible, so we can't use TSVWithNamesAndTypes.
"${client[@]}" --query "
select test, query_index,
quantileExact(0.99)(abs(diff)) max_diff,
quantileExactIf(0.99)(stat_threshold, abs(diff) < stat_threshold) * 1.5 max_stat_threshold,
query_display_name
from query_metrics_v2
where event_date > now() - interval 1 month
and metric = 'client_time'
and pr_number = 0
group by test, query_index, query_display_name
having count(*) > 100
" > analyze/historical-thresholds.tsv
else
touch analyze/historical-thresholds.tsv
fi
}
# Analyze results
@ -596,6 +653,26 @@ create view query_metric_stats as
diff float, stat_threshold float')
;
create table report_thresholds engine File(TSVWithNamesAndTypes, 'report/thresholds.tsv')
as select
query_display_names.test test, query_display_names.query_index query_index,
ceil(greatest(0.1, historical_thresholds.max_diff,
test_thresholds.report_threshold), 2) changed_threshold,
ceil(greatest(0.2, historical_thresholds.max_stat_threshold,
test_thresholds.report_threshold + 0.1), 2) unstable_threshold,
query_display_names.query_display_name query_display_name
from query_display_names
left join file('analyze/historical-thresholds.tsv', TSV,
'test text, query_index int, max_diff float, max_stat_threshold float,
query_display_name text') historical_thresholds
on query_display_names.test = historical_thresholds.test
and query_display_names.query_index = historical_thresholds.query_index
and query_display_names.query_display_name = historical_thresholds.query_display_name
left join file('analyze/report-thresholds.tsv', TSV,
'test text, report_threshold float') test_thresholds
on query_display_names.test = test_thresholds.test
;
-- Main statistics for queries -- query time as reported in query log.
create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv')
as select
@ -610,23 +687,23 @@ create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv')
-- uncaught regressions, because for the default 7 runs we do for PRs,
-- the randomization distribution has only 16 values, so the max quantile
-- is actually 0.9375.
abs(diff) > report_threshold and abs(diff) >= stat_threshold as changed_fail,
abs(diff) > report_threshold - 0.05 and abs(diff) >= stat_threshold as changed_show,
abs(diff) > changed_threshold and abs(diff) >= stat_threshold as changed_fail,
abs(diff) > changed_threshold - 0.05 and abs(diff) >= stat_threshold as changed_show,
not changed_fail and stat_threshold > report_threshold + 0.10 as unstable_fail,
not changed_show and stat_threshold > report_threshold - 0.05 as unstable_show,
not changed_fail and stat_threshold > unstable_threshold as unstable_fail,
not changed_show and stat_threshold > unstable_threshold - 0.05 as unstable_show,
left, right, diff, stat_threshold,
if(report_threshold > 0, report_threshold, 0.10) as report_threshold,
query_metric_stats.test test, query_metric_stats.query_index query_index,
query_display_name
query_display_names.query_display_name query_display_name
from query_metric_stats
left join file('analyze/report-thresholds.tsv', TSV,
'test text, report_threshold float') thresholds
on query_metric_stats.test = thresholds.test
left join query_display_names
on query_metric_stats.test = query_display_names.test
and query_metric_stats.query_index = query_display_names.query_index
left join report_thresholds
on query_display_names.test = report_thresholds.test
and query_display_names.query_index = report_thresholds.query_index
and query_display_names.query_display_name = report_thresholds.query_display_name
-- 'server_time' is rounded down to ms, which might be bad for very short queries.
-- Use 'client_time' instead.
where metric_name = 'client_time'
@ -889,7 +966,6 @@ create table all_query_metrics_tsv engine File(TSV, 'report/all-query-metrics.ts
order by test, query_index;
" 2> >(tee -a report/errors.log 1>&2)
# Prepare source data for metrics and flamegraphs for queries that were profiled
# by perf.py.
for version in {right,left}

View File

@ -453,7 +453,10 @@ if args.report == 'main':
text += tableRow(r, attrs, anchor)
text += tableEnd()
tables.append(text)
# Don't add an empty table.
if very_unstable_queries:
tables.append(text)
add_unstable_queries()
@ -552,13 +555,13 @@ if args.report == 'main':
message_array.append(str(slower_queries) + ' slower')
if unstable_partial_queries:
unstable_queries += unstable_partial_queries
error_tests += unstable_partial_queries
very_unstable_queries += unstable_partial_queries
status = 'failure'
# Don't show mildly unstable queries, only the very unstable ones we
# treat as errors.
if very_unstable_queries:
error_tests += very_unstable_queries
status = 'failure'
message_array.append(str(very_unstable_queries) + ' unstable')

View File

@ -73,4 +73,4 @@ RUN set -x \
VOLUME /var/lib/docker
EXPOSE 2375
ENTRYPOINT ["dockerd-entrypoint.sh"]
CMD ["sh", "-c", "python3 regression.py --no-color -o classic --local --clickhouse-binary-path ${CLICKHOUSE_TESTS_SERVER_BIN_PATH} --log test.log ${TESTFLOWS_OPTS}; cat test.log | tfs report results --format json > results.json; /usr/local/bin/process_testflows_result.py || echo -e 'failure\tCannot parse results' > check_status.tsv"]
CMD ["sh", "-c", "python3 regression.py --no-color -o new-fails --local --clickhouse-binary-path ${CLICKHOUSE_TESTS_SERVER_BIN_PATH} --log test.log ${TESTFLOWS_OPTS}; cat test.log | tfs report results --format json > results.json; /usr/local/bin/process_testflows_result.py || echo -e 'failure\tCannot parse results' > check_status.tsv; find * -type f | grep _instances | grep clickhouse-server | xargs -n1 tar -rvf clickhouse_logs.tar; gzip -9 clickhouse_logs.tar"]

View File

@ -338,7 +338,7 @@ or
``` sql
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576
PATH /var/lib/clickhouse/clickhouse_dictionaries/test_dict))
PATH ./user_files/test_dict))
```
### complex_key_ssd_cache {#complex-key-ssd-cache}

View File

@ -53,7 +53,7 @@ optional settings are available:
or
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
SETTINGS(format_csv_allow_single_quotes = 0)
```
@ -70,7 +70,7 @@ Types of sources (`source_type`):
- [MongoDB](#dicts-external_dicts_dict_sources-mongodb)
- [Redis](#dicts-external_dicts_dict_sources-redis)
- [Cassandra](#dicts-external_dicts_dict_sources-cassandra)
- [PostgreSQL](#dicts-external_dicts_dict_sources-postgresql)
- [PostgreSQL](#dicts-external_dicts_dict_sources-postgresql)
## Local File {#dicts-external_dicts_dict_sources-local_file}
@ -88,7 +88,7 @@ Example of settings:
or
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
```
Setting fields:

View File

@ -55,7 +55,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
または
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
SETTINGS(format_csv_allow_single_quotes = 0)
```
@ -87,7 +87,7 @@ SETTINGS(format_csv_allow_single_quotes = 0)
または
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
```
フィールドの設定:

View File

@ -121,7 +121,7 @@ LAYOUT(HASHED(PREALLOCATE 0))
Аналогичен `hashed`, но при этом занимает меньше места в памяти и генерирует более высокую загрузку CPU.
Для этого типа размещения также можно задать `preallocate` в значении `true`. В данном случае это более важно, чем для типа `hashed`.
Для этого типа размещения также можно задать `preallocate` в значении `true`. В данном случае это более важно, чем для типа `hashed`.
Пример конфигурации:
@ -338,7 +338,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000))
``` sql
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576
PATH /var/lib/clickhouse/clickhouse_dictionaries/test_dict))
PATH ./user_files/test_dict))
```
### complex_key_ssd_cache {#complex-key-ssd-cache}

View File

@ -53,7 +53,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
или
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
SETTINGS(format_csv_allow_single_quotes = 0)
```
@ -69,7 +69,7 @@ SETTINGS(format_csv_allow_single_quotes = 0)
- [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse)
- [MongoDB](#dicts-external_dicts_dict_sources-mongodb)
- [Redis](#dicts-external_dicts_dict_sources-redis)
- [PostgreSQL](#dicts-external_dicts_dict_sources-postgresql)
- [PostgreSQL](#dicts-external_dicts_dict_sources-postgresql)
## Локальный файл {#dicts-external_dicts_dict_sources-local_file}
@ -87,7 +87,7 @@ SETTINGS(format_csv_allow_single_quotes = 0)
или
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
```
Поля настройки:

View File

@ -55,7 +55,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
SETTINGS(format_csv_allow_single_quotes = 0)
```
@ -87,7 +87,7 @@ SETTINGS(format_csv_allow_single_quotes = 0)
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
```
设置字段:

View File

@ -11,6 +11,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -105,7 +106,7 @@ public:
void registerAggregateFunctionAggThrow(AggregateFunctionFactory & factory)
{
factory.registerFunction("aggThrow", [](const std::string & name, const DataTypes & argument_types, const Array & parameters)
factory.registerFunction("aggThrow", [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
Float64 throw_probability = 1.0;
if (parameters.size() == 1)

View File

@ -1,28 +1,27 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/HelpersMinMaxAny.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace
{
AggregateFunctionPtr createAggregateFunctionAny(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionAny(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyData>(name, argument_types, parameters));
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyData>(name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionAnyLast(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionAnyLast(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyLastData>(name, argument_types, parameters));
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyLastData>(name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionAnyHeavy(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionAnyHeavy(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyHeavyData>(name, argument_types, parameters));
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyHeavyData>(name, argument_types, parameters, settings));
}
}

View File

@ -8,6 +8,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -5,6 +5,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -9,6 +9,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -3,10 +3,11 @@
#include <AggregateFunctions/AggregateFunctionAvg.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
@ -20,7 +21,7 @@ bool allowType(const DataTypePtr& type) noexcept
return t.isInt() || t.isUInt() || t.isFloat() || t.isDecimal();
}
AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);

View File

@ -12,6 +12,7 @@
namespace DB
{
struct Settings;
template <typename T>
using DecimalOrVectorCol = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;

View File

@ -4,10 +4,11 @@
#include <AggregateFunctions/AggregateFunctionAvgWeighted.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
@ -60,7 +61,8 @@ static IAggregateFunction * create(const IDataType & first_type, const IDataType
#undef LINE
}
AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr
createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertBinary(name, argument_types);

View File

@ -5,6 +5,8 @@
namespace DB
{
struct Settings;
template <typename T>
using AvgWeightedFieldType = std::conditional_t<IsDecimalNumber<T>,
std::conditional_t<std::is_same_v<T, Decimal256>, Decimal256, Decimal128>,

View File

@ -7,6 +7,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
@ -16,7 +18,7 @@ namespace
{
template <template <typename> class Data>
AggregateFunctionPtr createAggregateFunctionBitwise(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionBitwise(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);

View File

@ -12,6 +12,7 @@
namespace DB
{
struct Settings;
template <typename T>

View File

@ -6,6 +6,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -15,7 +16,7 @@ namespace ErrorCodes
namespace
{
AggregateFunctionPtr createAggregateFunctionRate(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionRate(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertBinary(name, argument_types);

View File

@ -11,6 +11,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -8,6 +8,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -21,8 +22,8 @@ namespace
AggregateFunctionPtr createAggregateFunctionCategoricalIV(
const std::string & name,
const DataTypes & arguments,
const Array & params
)
const Array & params,
const Settings *)
{
assertNoParameters(name, params);

View File

@ -13,6 +13,7 @@
namespace DB
{
struct Settings;
template <typename T = UInt64>
class AggregateFunctionCategoricalIV final : public IAggregateFunctionHelper<AggregateFunctionCategoricalIV<T>>

View File

@ -5,6 +5,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -9,6 +9,7 @@
namespace DB
{
struct Settings;
/** Create aggregate function combinator by matching suffix in aggregate function name.
*/

View File

@ -6,6 +6,7 @@
namespace DB
{
struct Settings;
AggregateFunctionPtr AggregateFunctionCount::getOwnNullAdapter(
const AggregateFunctionPtr &, const DataTypes & types, const Array & params, const AggregateFunctionProperties & /*properties*/) const
@ -16,7 +17,7 @@ AggregateFunctionPtr AggregateFunctionCount::getOwnNullAdapter(
namespace
{
AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertArityAtMost<1>(name, argument_types);

View File

@ -13,6 +13,7 @@
namespace DB
{
struct Settings;
struct AggregateFunctionCountData
{

View File

@ -7,6 +7,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -20,7 +21,8 @@ namespace
AggregateFunctionPtr createAggregateFunctionDeltaSum(
const String & name,
const DataTypes & arguments,
const Array & params)
const Array & params,
const Settings *)
{
assertNoParameters(name, params);

View File

@ -15,6 +15,7 @@
namespace DB
{
struct Settings;
template <typename T>
struct AggregationFunctionDeltaSumData

View File

@ -20,7 +20,8 @@ namespace
AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp(
const String & name,
const DataTypes & arguments,
const Array & params)
const Array & params,
const Settings *)
{
assertNoParameters(name, params);

View File

@ -6,6 +6,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;

View File

@ -13,6 +13,7 @@
namespace DB
{
struct Settings;
template <typename T>

View File

@ -7,6 +7,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -16,7 +17,7 @@ namespace ErrorCodes
namespace
{
AggregateFunctionPtr createAggregateFunctionEntropy(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionEntropy(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
if (argument_types.empty())

View File

@ -14,6 +14,7 @@
namespace DB
{
struct Settings;
/** Calculates Shannon Entropy, using HashMap and computing empirical distribution function.
* Entropy is measured in bits (base-2 logarithm is used).

View File

@ -23,6 +23,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -137,7 +138,8 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(
if (!out_properties.returns_default_when_only_null && has_null_arguments)
return nullptr;
return found.creator(name, argument_types, parameters);
const Settings * settings = query_context ? &query_context->getSettingsRef() : nullptr;
return found.creator(name, argument_types, parameters, settings);
}
/// Combinators of aggregate functions.

View File

@ -14,6 +14,7 @@
namespace DB
{
struct Settings;
class Context;
class IDataType;
@ -26,7 +27,7 @@ using DataTypes = std::vector<DataTypePtr>;
* Parameters are for "parametric" aggregate functions.
* For example, in quantileWeighted(0.9)(x, weight), 0.9 is "parameter" and x, weight are "arguments".
*/
using AggregateFunctionCreator = std::function<AggregateFunctionPtr(const String &, const DataTypes &, const Array &)>;
using AggregateFunctionCreator = std::function<AggregateFunctionPtr(const String &, const DataTypes &, const Array &, const Settings *)>;
struct AggregateFunctionWithProperties
{

View File

@ -6,6 +6,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -13,6 +13,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -9,6 +9,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -49,7 +50,7 @@ inline AggregateFunctionPtr createAggregateFunctionGroupArrayImpl(const DataType
}
AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);
@ -83,7 +84,7 @@ AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name,
return createAggregateFunctionGroupArrayImpl<GroupArrayTrait<true, Sampler::NONE>>(argument_types[0], max_elems);
}
AggregateFunctionPtr createAggregateFunctionGroupArraySample(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionGroupArraySample(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);

View File

@ -26,6 +26,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int TOO_LARGE_ARRAY_SIZE;

View File

@ -7,6 +7,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
@ -15,7 +17,7 @@ namespace ErrorCodes
namespace
{
AggregateFunctionPtr createAggregateFunctionGroupArrayInsertAt(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionGroupArrayInsertAt(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);

View File

@ -20,6 +20,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -10,6 +10,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -57,7 +58,7 @@ inline AggregateFunctionPtr createAggregateFunctionMovingImpl(const std::string
}
template <template <typename, typename> class Function>
AggregateFunctionPtr createAggregateFunctionMoving(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionMoving(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);

View File

@ -22,6 +22,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -9,6 +9,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
@ -33,7 +35,7 @@ namespace
}
template <template <typename> class Data>
AggregateFunctionPtr createAggregateFunctionBitmap(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionBitmap(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);
@ -57,7 +59,7 @@ namespace
// Additional aggregate functions to manipulate bitmaps.
template <template <typename, typename> class AggregateFunctionTemplate>
AggregateFunctionPtr
createAggregateFunctionBitmapL2(const std::string & name, const DataTypes & argument_types, const Array & parameters)
createAggregateFunctionBitmapL2(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);

View File

@ -11,6 +11,7 @@
namespace DB
{
struct Settings;
/// Counts bitmap operation on numbers.
template <typename T, typename Data>
class AggregateFunctionBitmap final : public IAggregateFunctionDataHelper<Data, AggregateFunctionBitmap<T, Data>>

View File

@ -16,6 +16,7 @@
namespace DB
{
struct Settings;
enum BitmapKind
{

View File

@ -9,6 +9,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -72,7 +73,7 @@ inline AggregateFunctionPtr createAggregateFunctionGroupUniqArrayImpl(const std:
}
AggregateFunctionPtr createAggregateFunctionGroupUniqArray(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionGroupUniqArray(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);

View File

@ -23,6 +23,7 @@
namespace DB
{
struct Settings;
template <typename T>

View File

@ -8,6 +8,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -22,7 +23,7 @@ namespace ErrorCodes
namespace
{
AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name, const DataTypes & arguments, const Array & params)
AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name, const DataTypes & arguments, const Array & params, const Settings *)
{
if (params.size() != 1)
throw Exception("Function " + name + " requires single parameter: bins count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

View File

@ -26,6 +26,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -6,6 +6,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -8,6 +8,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -16,6 +16,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
@ -29,7 +31,7 @@ namespace
using FuncLogisticRegression = AggregateFunctionMLMethod<LinearModelData, NameLogisticRegression>;
template <class Method>
AggregateFunctionPtr
createAggregateFunctionMLMethod(const std::string & name, const DataTypes & argument_types, const Array & parameters)
createAggregateFunctionMLMethod(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
if (parameters.size() > 4)
throw Exception(

View File

@ -11,6 +11,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;

View File

@ -12,11 +12,12 @@ extern const int NOT_IMPLEMENTED;
namespace DB
{
struct Settings;
namespace
{
AggregateFunctionPtr createAggregateFunctionMannWhitneyUTest(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionMannWhitneyUTest(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);

View File

@ -23,6 +23,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -6,18 +6,19 @@
namespace DB
{
struct Settings;
namespace
{
AggregateFunctionPtr createAggregateFunctionMax(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionMax(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types, parameters));
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionArgMax(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionArgMax(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMaxData>(name, argument_types, parameters));
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMaxData>(name, argument_types, parameters, settings));
}
}

View File

@ -7,6 +7,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
@ -32,11 +34,17 @@ namespace
void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory & factory)
{
factory.registerFunction("maxIntersections", [](const std::string & name, const DataTypes & argument_types, const Array & parameters)
{ return createAggregateFunctionMaxIntersections(AggregateFunctionIntersectionsKind::Count, name, argument_types, parameters); });
factory.registerFunction("maxIntersections",
[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
return createAggregateFunctionMaxIntersections(AggregateFunctionIntersectionsKind::Count, name, argument_types, parameters);
});
factory.registerFunction("maxIntersectionsPosition", [](const std::string & name, const DataTypes & argument_types, const Array & parameters)
{ return createAggregateFunctionMaxIntersections(AggregateFunctionIntersectionsKind::Position, name, argument_types, parameters); });
factory.registerFunction("maxIntersectionsPosition",
[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
return createAggregateFunctionMaxIntersections(AggregateFunctionIntersectionsKind::Position, name, argument_types, parameters);
});
}
}

View File

@ -19,6 +19,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -6,6 +6,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -9,6 +9,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;

View File

@ -6,18 +6,19 @@
namespace DB
{
struct Settings;
namespace
{
AggregateFunctionPtr createAggregateFunctionMin(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionMin(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMinData>(name, argument_types, parameters));
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMinData>(name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionArgMin(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionArgMin(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMinData>(name, argument_types, parameters));
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMinData>(name, argument_types, parameters, settings));
}
}

View File

@ -15,6 +15,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;

View File

@ -8,6 +8,7 @@
namespace DB
{
struct Settings;
/** Aggregate function that takes arbitrary number of arbitrary arguments and does nothing.

View File

@ -9,6 +9,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -12,6 +12,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -6,6 +6,7 @@
namespace DB
{
struct Settings;
namespace
{

View File

@ -11,6 +11,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -8,6 +8,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -83,7 +84,7 @@ static constexpr bool supportBigInt()
}
template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(const std::string & name, const DataTypes & argument_types, const Array & params)
AggregateFunctionPtr createAggregateFunctionQuantile(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);

View File

@ -29,6 +29,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -12,11 +12,12 @@ extern const int NOT_IMPLEMENTED;
namespace DB
{
struct Settings;
namespace
{
AggregateFunctionPtr createAggregateFunctionRankCorrelation(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionRankCorrelation(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);
assertNoParameters(name, parameters);

View File

@ -18,6 +18,7 @@
namespace DB
{
struct Settings;
struct RankCorrelationData : public StatisticalSample<Float64, Float64>

View File

@ -6,6 +6,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -9,6 +9,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -7,6 +7,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
@ -15,7 +17,7 @@ namespace ErrorCodes
namespace
{
AggregateFunctionPtr createAggregateFunctionRetention(const std::string & name, const DataTypes & arguments, const Array & params)
AggregateFunctionPtr createAggregateFunctionRetention(const std::string & name, const DataTypes & arguments, const Array & params, const Settings *)
{
assertNoParameters(name, params);

View File

@ -17,6 +17,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;

View File

@ -16,10 +16,13 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
struct Settings;
namespace
{
template <template <typename> class Data>
AggregateFunctionPtr createAggregateFunctionSegmentLengthSum(const std::string & name, const DataTypes & arguments, const Array &)
AggregateFunctionPtr
createAggregateFunctionSegmentLengthSum(const std::string & name, const DataTypes & arguments, const Array &, const Settings *)
{
if (arguments.size() != 2)
throw Exception(

View File

@ -10,6 +10,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -23,7 +24,7 @@ namespace
{
template <template <typename, typename> class AggregateFunction, template <typename> class Data>
AggregateFunctionPtr createAggregateFunctionSequenceBase(const std::string & name, const DataTypes & argument_types, const Array & params)
AggregateFunctionPtr createAggregateFunctionSequenceBase(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
if (params.size() != 1)
throw Exception{"Aggregate function " + name + " requires exactly one parameter.",

View File

@ -15,6 +15,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -4,10 +4,12 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <Core/TypeListNumber.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
@ -19,8 +21,8 @@ namespace
AggregateFunctionPtr createAggregateFunctionSimpleLinearRegression(
const String & name,
const DataTypes & arguments,
const Array & params
)
const Array & params,
const Settings *)
{
assertNoParameters(name, params);
assertBinary(name, arguments);

View File

@ -13,6 +13,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -3,6 +3,7 @@
namespace DB
{
struct Settings;
namespace
{
class AggregateFunctionCombinatorSimpleState final : public IAggregateFunctionCombinator

View File

@ -7,6 +7,7 @@
namespace DB
{
struct Settings;
/** Not an aggregate function, but an adapter of aggregate functions.
* Aggregate functions with the `SimpleState` suffix is almost identical to the corresponding ones,

View File

@ -7,6 +7,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -8,6 +8,7 @@
namespace DB
{
struct Settings;
/** Not an aggregate function, but an adapter of aggregate functions,

View File

@ -7,6 +7,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -17,7 +18,7 @@ namespace
{
template <template <typename> class FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);
@ -31,7 +32,7 @@ AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string &
}
template <template <typename, typename> class FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsBinary(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionStatisticsBinary(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertBinary(name, argument_types);

View File

@ -12,6 +12,7 @@
namespace DB
{
struct Settings;
namespace detail
{

View File

@ -7,6 +7,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -17,7 +18,7 @@ namespace
{
template <template <typename> class FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);
@ -36,7 +37,7 @@ AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string &
}
template <template <typename, typename> class FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsBinary(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionStatisticsBinary(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertBinary(name, argument_types);

View File

@ -30,6 +30,7 @@
namespace DB
{
struct Settings;
enum class StatisticsFunctionKind
{

View File

@ -14,6 +14,7 @@ namespace ErrorCodes
namespace DB
{
struct Settings;
namespace
{
@ -56,7 +57,7 @@ struct StudentTTestData : public TTestMoments<Float64>
}
};
AggregateFunctionPtr createAggregateFunctionStudentTTest(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionStudentTTest(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);
assertNoParameters(name, parameters);

View File

@ -7,6 +7,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
@ -50,7 +52,7 @@ template <typename T> using AggregateFunctionSumKahan =
template <template <typename> class Function>
AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);

View File

@ -15,6 +15,7 @@
namespace DB
{
struct Settings;
/// Uses addOverflow method (if available) to avoid UB for sumWithOverflow()
///

View File

@ -6,6 +6,9 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
@ -19,7 +22,8 @@ bool allowType(const DataTypePtr& type) noexcept
return t.isInt() || t.isUInt() || t.isFloat() || t.isDecimal();
}
AggregateFunctionPtr createAggregateFunctionSumCount(const std::string & name, const DataTypes & argument_types, const Array & parameters)
AggregateFunctionPtr
createAggregateFunctionSumCount(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);

View File

@ -9,6 +9,8 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
@ -75,7 +77,7 @@ auto parseArguments(const std::string & name, const DataTypes & arguments)
// function template that allows to choose the aggregate function variant that
// accepts either normal arguments or tuple argument.
template<template <bool tuple_argument> typename MappedFunction>
AggregateFunctionPtr createAggregateFunctionMap(const std::string & name, const DataTypes & arguments, const Array & params)
AggregateFunctionPtr createAggregateFunctionMap(const std::string & name, const DataTypes & arguments, const Array & params, const Settings *)
{
auto [keys_type, values_types, tuple_argument] = parseArguments(name, arguments);

View File

@ -21,6 +21,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -23,6 +23,7 @@ extern "C"
namespace DB
{
struct Settings;
class ReadBuffer;
class WriteBuffer;

View File

@ -12,6 +12,7 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
@ -58,7 +59,7 @@ static IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_ty
template <bool is_weighted>
AggregateFunctionPtr createAggregateFunctionTopK(const std::string & name, const DataTypes & argument_types, const Array & params)
AggregateFunctionPtr createAggregateFunctionTopK(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
if (!is_weighted)
{

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