From d7f4bc4c72f238d4de1ac9dfda786d3a678c29c5 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 15 Mar 2023 17:01:19 +0100 Subject: [PATCH 01/57] ActionsDAG: remove wrong optimization Replacing AND with a casto to UInt8 can change the condition. e.g. 0x100 cast to UInt8 is 0, but it should be 1. 0.1 cast to UInt8 is 0, but it should be 1. Basically we can't cast anything to UInt8 so we completely remove this optimization. Closes #47317 --- src/Interpreters/ActionsDAG.cpp | 62 ++++++------------- .../02568_and_consistency.reference | 5 ++ .../0_stateless/02568_and_consistency.sql | 42 +++++++++++++ 3 files changed, 66 insertions(+), 43 deletions(-) create mode 100644 tests/queries/0_stateless/02568_and_consistency.reference create mode 100644 tests/queries/0_stateless/02568_and_consistency.sql diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 46b5a93b28c..ac11862d375 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1966,8 +1966,12 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown( } auto conjunction = getConjunctionNodes(predicate, allowed_nodes); - if (conjunction.rejected.size() == 1 && WhichDataType{removeNullable(conjunction.rejected.front()->result_type)}.isFloat()) + if (conjunction.rejected.size() == 1 && !conjunction.rejected.front()->result_type->equals(*predicate->result_type) + && conjunction.allowed.front()->type == ActionType::COLUMN) + { + // No further optimization can be done return nullptr; + } auto actions = cloneActionsForConjunction(conjunction.allowed, all_inputs); if (!actions) @@ -2017,55 +2021,26 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown( else { /// Predicate is conjunction, where both allowed and rejected sets are not empty. - /// Replace this node to conjunction of rejected predicates. NodeRawConstPtrs new_children = std::move(conjunction.rejected); - if (new_children.size() == 1) + if (new_children.size() == 1 && new_children.front()->result_type->equals(*predicate->result_type)) { - /// Rejected set has only one predicate. - if (new_children.front()->result_type->equals(*predicate->result_type)) - { - /// If it's type is same, just add alias. - Node node; - node.type = ActionType::ALIAS; - node.result_name = predicate->result_name; - node.result_type = predicate->result_type; - node.children.swap(new_children); - *predicate = std::move(node); - } - else if (!WhichDataType{removeNullable(new_children.front()->result_type)}.isFloat()) - { - /// If type is different, cast column. - /// This case is possible, cause AND can use any numeric type as argument. - /// But casting floats to UInt8 or Bool produces different results. - /// so we can't apply this optimization to them. - Node node; - node.type = ActionType::COLUMN; - node.result_name = predicate->result_type->getName(); - node.column = DataTypeString().createColumnConst(0, node.result_name); - node.result_type = std::make_shared(); - - const auto * right_arg = &nodes.emplace_back(std::move(node)); - const auto * left_arg = new_children.front(); - - predicate->children = {left_arg, right_arg}; - auto arguments = prepareFunctionArguments(predicate->children); - - FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); - - predicate->function_base = func_builder_cast->build(arguments); - predicate->function = predicate->function_base->prepare(arguments); - } + /// Rejected set has only one predicate. And the type is the same as the result_type. + /// Just add alias. + Node node; + node.type = ActionType::ALIAS; + node.result_name = predicate->result_name; + node.result_type = predicate->result_type; + node.children.swap(new_children); + *predicate = std::move(node); } else { - /// Predicate is function AND, which still have more then one argument. - /// Or there is only one argument that is a float and we can't just - /// remove the AND. + /// Predicate is function AND, which still have more then one argument + /// or it has one argument of the wrong type. /// Just update children and rebuild it. - predicate->children.swap(new_children); - if (WhichDataType{removeNullable(predicate->children.front()->result_type)}.isFloat()) + if (new_children.size() == 1) { Node node; node.type = ActionType::COLUMN; @@ -2073,8 +2048,9 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown( node.column = DataTypeUInt8().createColumnConst(0, 1u); node.result_type = std::make_shared(); const auto * const_col = &nodes.emplace_back(std::move(node)); - predicate->children.emplace_back(const_col); + new_children.emplace_back(const_col); } + predicate->children.swap(new_children); auto arguments = prepareFunctionArguments(predicate->children); FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); diff --git a/tests/queries/0_stateless/02568_and_consistency.reference b/tests/queries/0_stateless/02568_and_consistency.reference new file mode 100644 index 00000000000..07a8041d0ee --- /dev/null +++ b/tests/queries/0_stateless/02568_and_consistency.reference @@ -0,0 +1,5 @@ += +1554690688 += +1554690688 += diff --git a/tests/queries/0_stateless/02568_and_consistency.sql b/tests/queries/0_stateless/02568_and_consistency.sql new file mode 100644 index 00000000000..4e76da78427 --- /dev/null +++ b/tests/queries/0_stateless/02568_and_consistency.sql @@ -0,0 +1,42 @@ +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (c0 Int32, PRIMARY KEY (c0)) ENGINE=MergeTree; +INSERT INTO t1 VALUES (1554690688); + +select '='; + +SELECT MIN(t1.c0) +FROM t1 +GROUP BY + (-sign(cos(t1.c0))) * (-max2(t1.c0, t1.c0 / t1.c0)), + t1.c0 * t1.c0, + sign(-exp(-t1.c0)) +HAVING -(-(MIN(t1.c0) + MIN(t1.c0))) AND (pow('{b' > '-657301241', log(-1004522121)) IS NOT NULL) +UNION ALL +SELECT MIN(t1.c0) +FROM t1 +GROUP BY + (-sign(cos(t1.c0))) * (-max2(t1.c0, t1.c0 / t1.c0)), + t1.c0 * t1.c0, + sign(-exp(-t1.c0)) +HAVING NOT (-(-(MIN(t1.c0) + MIN(t1.c0))) AND (pow('{b' > '-657301241', log(-1004522121)) IS NOT NULL)) +UNION ALL +SELECT MIN(t1.c0) +FROM t1 +GROUP BY + (-sign(cos(t1.c0))) * (-max2(t1.c0, t1.c0 / t1.c0)), + t1.c0 * t1.c0, + sign(-exp(-t1.c0)) +HAVING (-(-(MIN(t1.c0) + MIN(t1.c0))) AND (pow('{b' > '-657301241', log(-1004522121)) IS NOT NULL)) IS NULL +SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0; + +select '='; + +SELECT MIN(t1.c0) +FROM t1 +GROUP BY t1.c0 +HAVING and(MIN(t1.c0) + MIN(t1.c0), 1) +SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0; + +select '='; + +DROP TABLE IF EXISTS t1; From e7d19cc45f5e9b42d2b6b6a063835c6e9614e3c1 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 15 Mar 2023 17:15:22 +0100 Subject: [PATCH 02/57] Fix test that expected CH to apply a wrong optimization The result of minus is an int64. Casting that result to uint8 instead of bool can lead to wrong results e.g. 256 cast to uint8 is 0, but it should be `true` --- tests/queries/0_stateless/01655_plan_optimizations.reference | 2 +- tests/queries/0_stateless/01655_plan_optimizations.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index f870a52284c..48d99647b43 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -53,7 +53,7 @@ Filter column: notEquals(y, 0) 9 10 > one condition of filter should be pushed down after aggregating, other condition is casted Filter column -FUNCTION _CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4)) +FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2 Aggregating Filter column: notEquals(y, 0) 0 1 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index aaecdc390cb..ec856c9bf27 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -56,7 +56,7 @@ $CLICKHOUSE_CLIENT -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION _CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y From 8cc425cd2946243f3756eed577d128705e781ceb Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 18 Apr 2023 00:14:35 -0700 Subject: [PATCH 03/57] INTO OUTFILE enhancements --- src/Client/ClientBase.cpp | 14 +++++++++++++- src/Parsers/ASTQueryWithOutput.h | 1 + src/Parsers/ParserQueryWithOutput.cpp | 6 ++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 64a18479ca9..91c8c7e6e79 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -571,6 +571,12 @@ try CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); UInt64 compression_level = 3; + if (query_with_output->is_outfile_append && compression_method != CompressionMethod::None) { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot append to compressed file. Please use uncompressed file or remove APPEND keyword."); + } + if (query_with_output->compression_level) { const auto & compression_level_node = query_with_output->compression_level->as(); @@ -585,8 +591,14 @@ try range.second); } + auto flags = O_WRONLY | O_EXCL; + if (query_with_output->is_outfile_append) + flags |= O_APPEND; + else + flags |= O_CREAT; + out_file_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), + std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, flags), compression_method, static_cast(compression_level) ); diff --git a/src/Parsers/ASTQueryWithOutput.h b/src/Parsers/ASTQueryWithOutput.h index 892d911e2e2..09f08772468 100644 --- a/src/Parsers/ASTQueryWithOutput.h +++ b/src/Parsers/ASTQueryWithOutput.h @@ -16,6 +16,7 @@ class ASTQueryWithOutput : public IAST public: ASTPtr out_file; bool is_into_outfile_with_stdout; + bool is_outfile_append; ASTPtr format; ASTPtr settings_ast; ASTPtr compression; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 2fb7c406d74..1ba44fc9939 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -100,6 +100,12 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!out_file_p.parse(pos, query_with_output.out_file, expected)) return false; + ParserKeyword s_append("APPEND"); + if (s_append.ignore(pos, expected)) + { + query_with_output.is_outfile_append = true; + } + ParserKeyword s_stdout("AND STDOUT"); if (s_stdout.ignore(pos, expected)) { From 2efc7492270a7583654d88cb4a5391d01dccab92 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 18 Apr 2023 00:30:00 -0700 Subject: [PATCH 04/57] Add test for INTO FILE ... APPEND. --- .../02001_append_output_file.reference | 2 ++ .../0_stateless/02001_append_output_file.sh | 15 +++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02001_append_output_file.reference create mode 100755 tests/queries/0_stateless/02001_append_output_file.sh diff --git a/tests/queries/0_stateless/02001_append_output_file.reference b/tests/queries/0_stateless/02001_append_output_file.reference new file mode 100644 index 00000000000..6f51dfc24e1 --- /dev/null +++ b/tests/queries/0_stateless/02001_append_output_file.reference @@ -0,0 +1,2 @@ +Hello, World! From client. +Hello, World! From local. diff --git a/tests/queries/0_stateless/02001_append_output_file.sh b/tests/queries/0_stateless/02001_append_output_file.sh new file mode 100755 index 00000000000..5cbae5b8cb3 --- /dev/null +++ b/tests/queries/0_stateless/02001_append_output_file.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e + +[ -e "${CLICKHOUSE_TMP}"/test_append_to_output_file] && rm "${CLICKHOUSE_TMP}"/test_append_to_output_file + +${CLICKHOUSE_CLIENT} --query "SELECT * FROM (SELECT 'Hello, World! From client.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_append_to_output_file'" +${CLICKHOUSE_LOCAL} --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_append_to_output_file' APPEND" +cat ${CLICKHOUSE_TMP}/test_append_to_output_file + +rm -f "${CLICKHOUSE_TMP}/test_append_to_output_file" From 8874ede98a05a5e41670e5c9a666c2eaa6faced0 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 18 Apr 2023 00:56:14 -0700 Subject: [PATCH 05/57] Fix style in test script. --- tests/queries/0_stateless/02001_append_output_file.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02001_append_output_file.sh b/tests/queries/0_stateless/02001_append_output_file.sh index 5cbae5b8cb3..47ac0183d91 100755 --- a/tests/queries/0_stateless/02001_append_output_file.sh +++ b/tests/queries/0_stateless/02001_append_output_file.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -[ -e "${CLICKHOUSE_TMP}"/test_append_to_output_file] && rm "${CLICKHOUSE_TMP}"/test_append_to_output_file +[ -e "${CLICKHOUSE_TMP}"/test_append_to_output_file ] && rm "${CLICKHOUSE_TMP}"/test_append_to_output_file ${CLICKHOUSE_CLIENT} --query "SELECT * FROM (SELECT 'Hello, World! From client.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_append_to_output_file'" ${CLICKHOUSE_LOCAL} --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_append_to_output_file' APPEND" From 966902fdfda744bfb386cd555c21ae168af02d53 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 16 Feb 2023 16:25:07 +0800 Subject: [PATCH 06/57] extend firt_value/last_value to accept null --- .../AggregateFunctionAny.cpp | 14 +- .../AggregateFunctionFactory.cpp | 2 +- .../AggregateFunctionMinMaxAny.h | 165 +++++++++++++++--- src/AggregateFunctions/HelpersMinMaxAny.h | 48 ++++- 4 files changed, 199 insertions(+), 30 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAny.cpp b/src/AggregateFunctions/AggregateFunctionAny.cpp index 9bc6e6af14f..219d530c58d 100644 --- a/src/AggregateFunctions/AggregateFunctionAny.cpp +++ b/src/AggregateFunctions/AggregateFunctionAny.cpp @@ -14,11 +14,21 @@ AggregateFunctionPtr createAggregateFunctionAny(const std::string & name, const return AggregateFunctionPtr(createAggregateFunctionSingleValue(name, argument_types, parameters, settings)); } +AggregateFunctionPtr createAggregateFunctionNullableAny(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings) +{ + return AggregateFunctionPtr(createAggregateFunctionSingleNullableValue(name, argument_types, parameters, settings)); +} + AggregateFunctionPtr createAggregateFunctionAnyLast(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings) { return AggregateFunctionPtr(createAggregateFunctionSingleValue(name, argument_types, parameters, settings)); } +AggregateFunctionPtr createAggregateFunctionNullableAnyLast(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings) +{ + return AggregateFunctionPtr(createAggregateFunctionSingleNullableValue(name, argument_types, parameters, settings)); +} + AggregateFunctionPtr createAggregateFunctionAnyHeavy(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings) { return AggregateFunctionPtr(createAggregateFunctionSingleValue(name, argument_types, parameters, settings)); @@ -36,10 +46,10 @@ void registerAggregateFunctionsAny(AggregateFunctionFactory & factory) // Synonyms for use as window functions. factory.registerFunction("first_value", - { createAggregateFunctionAny, properties }, + { createAggregateFunctionNullableAny, properties }, AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("last_value", - { createAggregateFunctionAnyLast, properties }, + { createAggregateFunctionNullableAnyLast, properties }, AggregateFunctionFactory::CaseInsensitive); } diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index 6cacf66500f..dc8b410dd9a 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -106,7 +106,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get( // nullability themselves. Another special case is functions from Nothing // that are rewritten to AggregateFunctionNothing, in this case // nested_function is nullptr. - if (!nested_function || !nested_function->isOnlyWindowFunction()) + if (!nested_function->getResultType()->isNullable() && (!nested_function || !nested_function->isOnlyWindowFunction())) return combinator->transformAggregateFunction(nested_function, out_properties, types_without_low_cardinality, parameters); } diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index b984772c8ea..76cdd5690a5 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -766,19 +766,24 @@ static_assert( /// For any other value types. +template struct SingleValueDataGeneric { private: using Self = SingleValueDataGeneric; Field value; + bool has_value = false; public: - static constexpr bool is_nullable = false; + static constexpr bool is_nullable = IS_NULLABLE; static constexpr bool is_any = false; + static constexpr bool is_null_greater = IS_NULL_GREATER; bool has() const { + if constexpr (is_nullable) + return has_value; return !value.isNull(); } @@ -813,11 +818,15 @@ public: void change(const IColumn & column, size_t row_num, Arena *) { column.get(row_num, value); + if constexpr (is_nullable) + has_value = true; } void change(const Self & to, Arena *) { value = to.value; + if constexpr (is_nullable) + has_value = true; } bool changeFirstTime(const IColumn & column, size_t row_num, Arena * arena) @@ -833,7 +842,7 @@ public: bool changeFirstTime(const Self & to, Arena * arena) { - if (!has() && to.has()) + if (!has() && (is_nullable || to.has())) { change(to, arena); return true; @@ -868,27 +877,86 @@ public: } else { - Field new_value; - column.get(row_num, new_value); - if (new_value < value) + if constexpr (is_nullable) { - value = new_value; - return true; + Field new_value; + column.get(row_num, new_value); + if constexpr (!is_null_greater) + { + if (!value.isNull() && (new_value.isNull() || new_value < value)) + { + value = new_value; + return true; + } + else + return false; + } + else + { + if ((value.isNull() && !new_value.isNull()) || (!new_value.isNull() && new_value < value)) + { + value = new_value; + return true; + } + + return false; + } } else - return false; + { + Field new_value; + column.get(row_num, new_value); + if (new_value < value) + { + value = new_value; + return true; + } + else + return false; + } } } bool changeIfLess(const Self & to, Arena * arena) { - if (to.has() && (!has() || to.value < value)) + if (!to.has()) + return false; + if constexpr (is_nullable) { - change(to, arena); - return true; + if (!has()) + { + change(to, arena); + return true; + } + if constexpr (!is_null_greater) + { + if (to.value.isNull() || (!value.isNull() && to.value < value)) + { + value = to.value; + return true; + } + return false; + } + else + { + if ((value.isNull() && !to.value.isNull()) || (!to.value.isNull() || to.value < value)) + { + value = to.value; + return true; + } + return false; + } } else - return false; + { + if (!has() || to.value < value) + { + change(to, arena); + return true; + } + else + return false; + } } bool changeIfGreater(const IColumn & column, size_t row_num, Arena * arena) @@ -900,27 +968,80 @@ public: } else { - Field new_value; - column.get(row_num, new_value); - if (new_value > value) + if constexpr (is_nullable) { - value = new_value; - return true; + Field new_value; + column.get(row_num, new_value); + if constexpr (is_null_greater) + { + if (!value.isNull() && (new_value.isNull() || value < new_value)) + { + value = new_value; + return true; + } + return false; + } + else + { + if ((value.isNull() && !new_value.isNull()) || (!new_value.isNull() && value < new_value)) + { + value = new_value; + return true; + } + return false; + } } else - return false; + { + Field new_value; + column.get(row_num, new_value); + if (new_value > value) + { + value = new_value; + return true; + } + else + return false; + } } } bool changeIfGreater(const Self & to, Arena * arena) { - if (to.has() && (!has() || to.value > value)) + if (!to.has()) + return false; + if constexpr (is_nullable) { - change(to, arena); - return true; + if constexpr (is_null_greater) + { + if (!value.isNull() && (to.value.isNull() || value < to.value)) + { + value = to.value; + return true; + } + return false; + } + else + { + if ((value.isNull() && !to.value.isNull()) || (!to.value.isNull() && value < to.value)) + { + value = to.value; + return true; + } + return false; + + } } else - return false; + { + if (!has() || to.value > value) + { + change(to, arena); + return true; + } + else + return false; + } } bool isEqualTo(const IColumn & column, size_t row_num) const diff --git a/src/AggregateFunctions/HelpersMinMaxAny.h b/src/AggregateFunctions/HelpersMinMaxAny.h index 026a206b109..f8fac616d2c 100644 --- a/src/AggregateFunctions/HelpersMinMaxAny.h +++ b/src/AggregateFunctions/HelpersMinMaxAny.h @@ -9,9 +9,12 @@ #include #include - namespace DB { +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} struct Settings; /// min, max, any, anyLast, anyHeavy, etc... @@ -22,7 +25,6 @@ static IAggregateFunction * createAggregateFunctionSingleValue(const String & na assertUnary(name, argument_types); const DataTypePtr & argument_type = argument_types[0]; - WhichDataType which(argument_type); #define DISPATCH(TYPE) \ if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate>>(argument_type); /// NOLINT @@ -46,7 +48,43 @@ static IAggregateFunction * createAggregateFunctionSingleValue(const String & na if (which.idx == TypeIndex::String) return new AggregateFunctionTemplate>(argument_type); - return new AggregateFunctionTemplate>(argument_type); + return new AggregateFunctionTemplate>>(argument_type); +} + +template