From 13a71696692b19dd6210b9d0d1ae58dbc92f7762 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Sun, 9 May 2021 12:47:29 +0300 Subject: [PATCH 001/224] Add feature: create user defined function as lambda --- src/Functions/FunctionFactory.cpp | 16 ++++ src/Functions/FunctionFactory.h | 8 ++ src/Functions/UserDefinedFunction.cpp | 90 +++++++++++++++++++ src/Functions/UserDefinedFunction.h | 37 ++++++++ src/Functions/ya.make | 1 + .../InterpreterCreateFunctionQuery.cpp | 18 ++++ .../InterpreterCreateFunctionQuery.h | 22 +++++ src/Interpreters/InterpreterFactory.cpp | 5 ++ src/Interpreters/ya.make | 3 +- src/Parsers/ASTCreateFunctionQuery.cpp | 21 +++++ src/Parsers/ASTCreateFunctionQuery.h | 22 +++++ src/Parsers/ParserCreateFunctionQuery.cpp | 46 ++++++++++ src/Parsers/ParserCreateFunctionQuery.h | 15 ++++ src/Parsers/ParserQuery.cpp | 3 + src/Parsers/ya.make | 2 + 15 files changed, 308 insertions(+), 1 deletion(-) create mode 100644 src/Functions/UserDefinedFunction.cpp create mode 100644 src/Functions/UserDefinedFunction.h create mode 100644 src/Interpreters/InterpreterCreateFunctionQuery.cpp create mode 100644 src/Interpreters/InterpreterCreateFunctionQuery.h create mode 100644 src/Parsers/ASTCreateFunctionQuery.cpp create mode 100644 src/Parsers/ASTCreateFunctionQuery.h create mode 100644 src/Parsers/ParserCreateFunctionQuery.cpp create mode 100644 src/Parsers/ParserCreateFunctionQuery.h diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index 35ac9ab647b..7f330d45c37 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -1,4 +1,5 @@ #include +#include #include @@ -133,4 +134,19 @@ FunctionFactory & FunctionFactory::instance() return ret; } +void FunctionFactory::registerUserDefinedFunction( + const ASTCreateFunctionQuery & create_function_query, + CaseSensitiveness case_sensitiveness) +{ + registerFunction(create_function_query.function_name, [create_function_query](ContextPtr context) + { + auto function = UserDefinedFunction::create(context); + function->setName(create_function_query.function_name); + function->setFunctionCore(create_function_query.function_core); + + FunctionOverloadResolverImplPtr res = std::make_unique(function); + return res; + }, case_sensitiveness); +} + } diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index 96238a88420..176178f7593 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -1,8 +1,10 @@ #pragma once #include +#include #include #include +#include #include #include @@ -13,6 +15,8 @@ namespace DB { +class UserDefinedFunction; + /** Creates function by name. * Function could use for initialization (take ownership of shared_ptr, for example) * some dictionaries from Context. @@ -38,6 +42,10 @@ public: registerFunction(name, &Function::create, case_sensitiveness); } + void registerUserDefinedFunction( + const ASTCreateFunctionQuery & create_function_query, + CaseSensitiveness case_sensitiveness = CaseSensitive); + /// This function is used by YQL - internal Yandex product that depends on ClickHouse by source code. std::vector getAllNames() const; diff --git a/src/Functions/UserDefinedFunction.cpp b/src/Functions/UserDefinedFunction.cpp new file mode 100644 index 00000000000..b7b4ff8de3e --- /dev/null +++ b/src/Functions/UserDefinedFunction.cpp @@ -0,0 +1,90 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; +} + +UserDefinedFunction::UserDefinedFunction(ContextPtr context_) + : function_core(nullptr) + , context(context_) +{} + +UserDefinedFunctionPtr UserDefinedFunction::create(ContextPtr context) +{ + return std::make_shared(context); +} + +String UserDefinedFunction::getName() const +{ + return name; +} + +ColumnPtr UserDefinedFunction::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const +{ + Block block = executeCore(arguments); + + String result_name = function_core->as()->arguments->children.at(1)->getColumnName(); + + // result of function executing was inserted in the end + return block.getColumns().back(); +} + +size_t UserDefinedFunction::getNumberOfArguments() const +{ + return function_core->as()->arguments->children[0]->size() - 2; +} + +void UserDefinedFunction::setName(const String & name_) +{ + name = name_; +} + +void UserDefinedFunction::setFunctionCore(ASTPtr function_core_) +{ + function_core = function_core_; +} + +DataTypePtr UserDefinedFunction::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const +{ + Block block = executeCore(arguments); + return block.getDataTypes().back(); +} + +Block UserDefinedFunction::executeCore(const ColumnsWithTypeAndName & arguments) const +{ + const auto * lambda_args_tuple = function_core->as()->arguments->children.at(0)->as(); + const ASTs & lambda_arg_asts = lambda_args_tuple->arguments->children; + + NamesAndTypesList lambda_arguments; + Block block; + + for (size_t j = 0; j < lambda_arg_asts.size(); ++j) + { + auto opt_arg_name = tryGetIdentifierName(lambda_arg_asts[j]); + if (!opt_arg_name) + throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); + + lambda_arguments.emplace_back(*opt_arg_name, arguments[j].type); + block.insert({arguments[j].column, arguments[j].type, *opt_arg_name}); + } + + ASTPtr lambda_body = function_core->as()->children.at(0)->children.at(1); + auto syntax_result = TreeRewriter(context).analyze(lambda_body, lambda_arguments); + ExpressionAnalyzer analyzer(lambda_body, syntax_result, context); + ExpressionActionsPtr actions = analyzer.getActions(false); + + actions->execute(block); + return block; +} + +} diff --git a/src/Functions/UserDefinedFunction.h b/src/Functions/UserDefinedFunction.h new file mode 100644 index 00000000000..2b519740204 --- /dev/null +++ b/src/Functions/UserDefinedFunction.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class UserDefinedFunction; +using UserDefinedFunctionPtr = std::shared_ptr; + +class UserDefinedFunction : public IFunction +{ +public: + explicit UserDefinedFunction(ContextPtr context_); + static UserDefinedFunctionPtr create(ContextPtr context); + + String getName() const override; + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override; + size_t getNumberOfArguments() const override; + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; + + void setName(const String & name_); + void setFunctionCore(ASTPtr function_core_); + +private: + Block executeCore(const ColumnsWithTypeAndName & arguments) const; + +private: + String name; + ASTPtr function_core; + ContextPtr context; +}; + +} diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 2a541369ff4..fef9bae3685 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -104,6 +104,7 @@ SRCS( URL/registerFunctionsURL.cpp URL/tldLookup.generated.cpp URL/topLevelDomain.cpp + UserDefinedFunction.cpp abs.cpp acos.cpp acosh.cpp diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp new file mode 100644 index 00000000000..4fa524534f3 --- /dev/null +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -0,0 +1,18 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +BlockIO InterpreterCreateFunctionQuery::execute() +{ + FunctionNameNormalizer().visit(query_ptr.get()); + auto & create_function_query = query_ptr->as(); + FunctionFactory::instance().registerUserDefinedFunction(create_function_query); + return {}; +} + +} diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.h b/src/Interpreters/InterpreterCreateFunctionQuery.h new file mode 100644 index 00000000000..81347bcc711 --- /dev/null +++ b/src/Interpreters/InterpreterCreateFunctionQuery.h @@ -0,0 +1,22 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ASTCreateFunctionQuery; + +class InterpreterCreateFunctionQuery : public IInterpreter, WithContext +{ +public: + InterpreterCreateFunctionQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 4af8b6ffa7d..54122292589 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #include #include @@ -264,6 +265,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextPtr { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else { throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY); diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index 105e1e11365..5c49c7e8946 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -54,7 +54,7 @@ SRCS( ExpressionAnalyzer.cpp ExternalDictionariesLoader.cpp ExternalLoader.cpp - ExternalLoaderDictionaryStorageConfigRepository.cpp + ExternalLoaderDatabaseConfigRepository.cpp ExternalLoaderTempConfigRepository.cpp ExternalLoaderXMLConfigRepository.cpp ExternalModelsLoader.cpp @@ -70,6 +70,7 @@ SRCS( InternalTextLogsQueue.cpp InterpreterAlterQuery.cpp InterpreterCheckQuery.cpp + InterpreterCreateFunctionQuery.cpp InterpreterCreateQuery.cpp InterpreterCreateQuotaQuery.cpp InterpreterCreateRoleQuery.cpp diff --git a/src/Parsers/ASTCreateFunctionQuery.cpp b/src/Parsers/ASTCreateFunctionQuery.cpp new file mode 100644 index 00000000000..0b3991ddc44 --- /dev/null +++ b/src/Parsers/ASTCreateFunctionQuery.cpp @@ -0,0 +1,21 @@ +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTCreateFunctionQuery::clone() const +{ + return std::make_shared(*this); +} + +void ASTCreateFunctionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE FUNCTION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(function_name) << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); + function_core->formatImpl(settings, state, frame); +} + +} diff --git a/src/Parsers/ASTCreateFunctionQuery.h b/src/Parsers/ASTCreateFunctionQuery.h new file mode 100644 index 00000000000..3adddad8fbd --- /dev/null +++ b/src/Parsers/ASTCreateFunctionQuery.h @@ -0,0 +1,22 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ASTCreateFunctionQuery : public IAST +{ +public: + String function_name; + ASTPtr function_core; + + String getID(char) const override { return "CreateFunctionQuery"; } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; +}; + +} diff --git a/src/Parsers/ParserCreateFunctionQuery.cpp b/src/Parsers/ParserCreateFunctionQuery.cpp new file mode 100644 index 00000000000..1fcce6cbf45 --- /dev/null +++ b/src/Parsers/ParserCreateFunctionQuery.cpp @@ -0,0 +1,46 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +bool ParserCreateFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_create("CREATE"); + ParserKeyword s_function("FUNCTION"); + ParserIdentifier function_name_p; + ParserKeyword s_as("AS"); + ParserLambdaExpression lambda_p; + + ASTPtr function_name; + ASTPtr function_core; + + if (!s_create.ignore(pos, expected)) + return false; + + if (!s_function.ignore(pos, expected)) + return false; + + if (!function_name_p.parse(pos, function_name, expected)) + return false; + + if (!s_as.ignore(pos, expected)) + return false; + + if (!lambda_p.parse(pos, function_core, expected)) + return false; + + auto create_function_query = std::make_shared(); + node = create_function_query; + + create_function_query->function_name = function_name->as().name(); + create_function_query->function_core = function_core; + + return true; +} + +} diff --git a/src/Parsers/ParserCreateFunctionQuery.h b/src/Parsers/ParserCreateFunctionQuery.h new file mode 100644 index 00000000000..a48bbdeb563 --- /dev/null +++ b/src/Parsers/ParserCreateFunctionQuery.h @@ -0,0 +1,15 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ +/// CREATE FUNCTION test AS x -> x || '1' +class ParserCreateFunctionQuery : public IParserBase +{ +protected: + const char * getName() const override { return "CREATE FUNCTION query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 4550bdc8a75..274dc0201b3 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -36,6 +37,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateQuotaQuery create_quota_p; ParserCreateRowPolicyQuery create_row_policy_p; ParserCreateSettingsProfileQuery create_settings_profile_p; + ParserCreateFunctionQuery create_function_p; ParserDropAccessEntityQuery drop_access_entity_p; ParserGrantQuery grant_p; ParserSetRoleQuery set_role_p; @@ -52,6 +54,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_quota_p.parse(pos, node, expected) || create_row_policy_p.parse(pos, node, expected) || create_settings_profile_p.parse(pos, node, expected) + || create_function_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected) || external_ddl_p.parse(pos, node, expected); diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index 4bd31cb79de..3b9fcb33e0f 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -15,6 +15,7 @@ SRCS( ASTColumnsMatcher.cpp ASTColumnsTransformers.cpp ASTConstraintDeclaration.cpp + ASTCreateFunctionQuery.cpp ASTCreateQuery.cpp ASTCreateQuotaQuery.cpp ASTCreateRoleQuery.cpp @@ -86,6 +87,7 @@ SRCS( ParserAlterQuery.cpp ParserCase.cpp ParserCheckQuery.cpp + ParserCreateFunctionQuery.cpp ParserCreateQuery.cpp ParserCreateQuotaQuery.cpp ParserCreateRoleQuery.cpp From ca5ce926885dedda64feb37fda1a2557fdf5a8b7 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Sun, 9 May 2021 12:48:23 +0300 Subject: [PATCH 002/224] Add tests --- .../queries/0_stateless/01855_create_simple_function.reference | 1 + tests/queries/0_stateless/01855_create_simple_function.sql | 2 ++ ...1856_create_function_with_unknown_variable_in_body.reference | 0 .../01856_create_function_with_unknown_variable_in_body.sql | 2 ++ .../01857_create_function_that_already_exists.reference | 0 .../0_stateless/01857_create_function_that_already_exists.sql | 2 ++ 6 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/01855_create_simple_function.reference create mode 100644 tests/queries/0_stateless/01855_create_simple_function.sql create mode 100644 tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.reference create mode 100644 tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.sql create mode 100644 tests/queries/0_stateless/01857_create_function_that_already_exists.reference create mode 100644 tests/queries/0_stateless/01857_create_function_that_already_exists.sql diff --git a/tests/queries/0_stateless/01855_create_simple_function.reference b/tests/queries/0_stateless/01855_create_simple_function.reference new file mode 100644 index 00000000000..a45fd52cc58 --- /dev/null +++ b/tests/queries/0_stateless/01855_create_simple_function.reference @@ -0,0 +1 @@ +24 diff --git a/tests/queries/0_stateless/01855_create_simple_function.sql b/tests/queries/0_stateless/01855_create_simple_function.sql new file mode 100644 index 00000000000..8fb0117fd3c --- /dev/null +++ b/tests/queries/0_stateless/01855_create_simple_function.sql @@ -0,0 +1,2 @@ +create function MyFunc as (a, b, c) -> a * b * c; +select MyFunc(2, 3, 4); diff --git a/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.reference b/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.sql b/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.sql new file mode 100644 index 00000000000..a563d1c4ab2 --- /dev/null +++ b/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.sql @@ -0,0 +1,2 @@ +create function MyFunc2 as (a, b) -> a || b || c; +select MyFunc2('1', '2'); -- { serverError 47 } diff --git a/tests/queries/0_stateless/01857_create_function_that_already_exists.reference b/tests/queries/0_stateless/01857_create_function_that_already_exists.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01857_create_function_that_already_exists.sql b/tests/queries/0_stateless/01857_create_function_that_already_exists.sql new file mode 100644 index 00000000000..444b722a272 --- /dev/null +++ b/tests/queries/0_stateless/01857_create_function_that_already_exists.sql @@ -0,0 +1,2 @@ +create function MyFunc3 as (a, b) -> a + b; +create function MyFunc3 as (a) -> a || '!!!'; -- { serverError 49 } From ad4594c29829bf03293fa2da13d938b0aea7edb9 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Tue, 11 May 2021 01:35:22 +0300 Subject: [PATCH 003/224] Add validation of unknown identifiers in function --- src/Functions/FunctionFactory.cpp | 2 + src/Functions/FunctionFactory.h | 3 -- .../InterpreterCreateFunctionQuery.cpp | 41 +++++++++++++++++++ .../InterpreterCreateFunctionQuery.h | 4 ++ src/Parsers/ParserCreateFunctionQuery.cpp | 1 + src/Parsers/ParserCreateFunctionQuery.h | 1 + ...function_with_unknown_variable_in_body.sql | 3 +- 7 files changed, 50 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index 7f330d45c37..69a34d4d030 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -16,6 +16,8 @@ namespace DB { +class UserDefinedFunction; + namespace ErrorCodes { extern const int UNKNOWN_FUNCTION; diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index 176178f7593..850323fa2df 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include @@ -15,8 +14,6 @@ namespace DB { -class UserDefinedFunction; - /** Creates function by name. * Function could use for initialization (take ownership of shared_ptr, for example) * some dictionaries from Context. diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index 4fa524534f3..9de6a4aaeff 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -7,12 +8,52 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_IDENTIFIER; +} + BlockIO InterpreterCreateFunctionQuery::execute() { FunctionNameNormalizer().visit(query_ptr.get()); auto & create_function_query = query_ptr->as(); + validateFunction(create_function_query.function_core); FunctionFactory::instance().registerUserDefinedFunction(create_function_query); return {}; } +void InterpreterCreateFunctionQuery::validateFunction(ASTPtr function) +{ + const auto * args_tuple = function->as()->arguments->children.at(0)->as(); + std::unordered_set arguments; + for (const auto & argument : args_tuple->arguments->children) + arguments.insert(argument->as()->name()); + + std::vector identifiers_in_body; + ASTPtr function_body = function->as()->children.at(0)->children.at(1); + getIdentifiers(function_body, identifiers_in_body); + + for (const auto & identifier : identifiers_in_body) + { + if (!arguments.contains(identifier)) + { + std::stringstream s; + s << "Identifier '" << identifier << "' does not exist in arguments"; + throw Exception(s.str(), ErrorCodes::UNKNOWN_IDENTIFIER); + } + } +} + +void InterpreterCreateFunctionQuery::getIdentifiers(ASTPtr node, std::vector & identifiers) +{ + for (const auto & child : node->children) + { + auto identifier_name_opt = tryGetIdentifierName(child); + if (identifier_name_opt) + identifiers.push_back(identifier_name_opt.value()); + + getIdentifiers(child, identifiers); + } +} + } diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.h b/src/Interpreters/InterpreterCreateFunctionQuery.h index 81347bcc711..79b7839116a 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.h +++ b/src/Interpreters/InterpreterCreateFunctionQuery.h @@ -15,6 +15,10 @@ public: BlockIO execute() override; +private: + static void validateFunction(ASTPtr function); + static void getIdentifiers(ASTPtr node, std::vector & identifiers); + private: ASTPtr query_ptr; }; diff --git a/src/Parsers/ParserCreateFunctionQuery.cpp b/src/Parsers/ParserCreateFunctionQuery.cpp index 1fcce6cbf45..fbfd02415e7 100644 --- a/src/Parsers/ParserCreateFunctionQuery.cpp +++ b/src/Parsers/ParserCreateFunctionQuery.cpp @@ -8,6 +8,7 @@ namespace DB { + bool ParserCreateFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_create("CREATE"); diff --git a/src/Parsers/ParserCreateFunctionQuery.h b/src/Parsers/ParserCreateFunctionQuery.h index a48bbdeb563..aac643b995d 100644 --- a/src/Parsers/ParserCreateFunctionQuery.h +++ b/src/Parsers/ParserCreateFunctionQuery.h @@ -4,6 +4,7 @@ namespace DB { + /// CREATE FUNCTION test AS x -> x || '1' class ParserCreateFunctionQuery : public IParserBase { diff --git a/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.sql b/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.sql index a563d1c4ab2..1fed9fe9103 100644 --- a/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.sql +++ b/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.sql @@ -1,2 +1 @@ -create function MyFunc2 as (a, b) -> a || b || c; -select MyFunc2('1', '2'); -- { serverError 47 } +create function MyFunc2 as (a, b) -> a || b || c; -- { serverError 47 } From ed8ecc987e7919eaed67211ced2dc58bbb1f9f60 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Tue, 11 May 2021 02:20:43 +0300 Subject: [PATCH 004/224] Fix style error --- src/Interpreters/InterpreterCreateFunctionQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index 9de6a4aaeff..83fdc8ad29c 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -37,7 +37,7 @@ void InterpreterCreateFunctionQuery::validateFunction(ASTPtr function) { if (!arguments.contains(identifier)) { - std::stringstream s; + WriteBufferFromOwnString s; s << "Identifier '" << identifier << "' does not exist in arguments"; throw Exception(s.str(), ErrorCodes::UNKNOWN_IDENTIFIER); } From 7f3e5b9166f11912c645830552fa59fe43eaa3f6 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Fri, 14 May 2021 00:14:58 +0300 Subject: [PATCH 005/224] Move UserDefinedFunction.cpp/h files to same library with where FunctionFactory --- src/CMakeLists.txt | 35 +++++++++++++++++------------------ 1 file changed, 17 insertions(+), 18 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 69a84dbeb2c..4c7c9e23d24 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -18,17 +18,17 @@ else() endif() include(../cmake/limit_jobs.cmake) -set (CONFIG_VERSION "${CMAKE_CURRENT_BINARY_DIR}/Common/config_version.h") -set (CONFIG_COMMON "${CMAKE_CURRENT_BINARY_DIR}/Common/config.h") +set (CONFIG_VERSION ${CMAKE_CURRENT_BINARY_DIR}/Common/config_version.h) +set (CONFIG_COMMON ${CMAKE_CURRENT_BINARY_DIR}/Common/config.h) include (../cmake/version.cmake) message (STATUS "Will build ${VERSION_FULL} revision ${VERSION_REVISION} ${VERSION_OFFICIAL}") configure_file (Common/config.h.in ${CONFIG_COMMON}) configure_file (Common/config_version.h.in ${CONFIG_VERSION}) -configure_file (Core/config_core.h.in "${CMAKE_CURRENT_BINARY_DIR}/Core/include/config_core.h") +configure_file (Core/config_core.h.in ${CMAKE_CURRENT_BINARY_DIR}/Core/include/config_core.h) if (USE_DEBUG_HELPERS) - set (INCLUDE_DEBUG_HELPERS "-I\"${ClickHouse_SOURCE_DIR}/base\" -include \"${ClickHouse_SOURCE_DIR}/src/Core/iostream_debug_helpers.h\"") + set (INCLUDE_DEBUG_HELPERS "-I${ClickHouse_SOURCE_DIR}/base -include ${ClickHouse_SOURCE_DIR}/src/Core/iostream_debug_helpers.h") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${INCLUDE_DEBUG_HELPERS}") endif () @@ -106,8 +106,8 @@ endif() list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) -list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp Functions/replicate.cpp Functions/FunctionsLogical.cpp) -list (APPEND dbms_headers Functions/IFunctionImpl.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/replicate.h Functions/FunctionsLogical.h) +list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp Functions/replicate.cpp Functions/FunctionsLogical.cpp Functions/UserDefinedFunction.cpp) +list (APPEND dbms_headers Functions/IFunctionImpl.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/replicate.h Functions/FunctionsLogical.h Functions/UserDefinedFunction.h) list (APPEND dbms_sources AggregateFunctions/AggregateFunctionFactory.cpp @@ -245,8 +245,8 @@ macro (dbms_target_link_libraries) endforeach () endmacro () -dbms_target_include_directories (PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src") -target_include_directories (clickhouse_common_io PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src") +dbms_target_include_directories (PUBLIC ${ClickHouse_SOURCE_DIR}/src ${ClickHouse_BINARY_DIR}/src) +target_include_directories (clickhouse_common_io PUBLIC ${ClickHouse_SOURCE_DIR}/src ${ClickHouse_BINARY_DIR}/src) if (USE_EMBEDDED_COMPILER) dbms_target_link_libraries (PRIVATE ${REQUIRED_LLVM_LIBRARIES}) @@ -322,6 +322,10 @@ if (USE_RDKAFKA) endif() endif() +if (USE_AMQPCPP) + dbms_target_link_libraries(PUBLIC amqp-cpp) +endif() + if (USE_CYRUS_SASL) dbms_target_link_libraries(PRIVATE ${CYRUS_SASL_LIBRARY}) endif() @@ -357,8 +361,8 @@ dbms_target_link_libraries ( clickhouse_common_io ) -target_include_directories(clickhouse_common_io PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/Core/include") # uses some includes from core -dbms_target_include_directories(PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/Core/include") +target_include_directories(clickhouse_common_io PUBLIC ${CMAKE_CURRENT_BINARY_DIR}/Core/include) # uses some includes from core +dbms_target_include_directories(PUBLIC ${CMAKE_CURRENT_BINARY_DIR}/Core/include) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${PDQSORT_INCLUDE_DIR}) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${MINISELECT_INCLUDE_DIR}) @@ -437,11 +441,6 @@ if (USE_BROTLI) target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${BROTLI_INCLUDE_DIR}) endif() -if (USE_AMQPCPP) - dbms_target_link_libraries(PUBLIC ${AMQPCPP_LIBRARY}) - dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${AMQPCPP_INCLUDE_DIR}) -endif() - if (USE_CASSANDRA) dbms_target_link_libraries(PUBLIC ${CASSANDRA_LIBRARY}) dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${CASS_INCLUDE_DIR}) @@ -458,7 +457,7 @@ target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${FAST_FLO if (USE_ORC) dbms_target_link_libraries(PUBLIC ${ORC_LIBRARIES}) - dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR} "${CMAKE_BINARY_DIR}/contrib/orc/c++/include") + dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR} ${CMAKE_BINARY_DIR}/contrib/orc/c++/include) endif () if (USE_ROCKSDB) @@ -477,7 +476,7 @@ endif () dbms_target_link_libraries(PRIVATE _boost_context) -include ("${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake") +include (${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake) if (ENABLE_TESTS AND USE_GTEST) macro (grep_gtest_sources BASE_DIR DST_VAR) @@ -486,7 +485,7 @@ if (ENABLE_TESTS AND USE_GTEST) endmacro() # attach all dbms gtest sources - grep_gtest_sources("${ClickHouse_SOURCE_DIR}/src" dbms_gtest_sources) + grep_gtest_sources(${ClickHouse_SOURCE_DIR}/src dbms_gtest_sources) add_executable(unit_tests_dbms ${dbms_gtest_sources}) # gtest framework has substandard code From 2643b71393a2e310d9184fceab0a940650af7c5b Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Fri, 14 May 2021 00:41:35 +0300 Subject: [PATCH 006/224] Add new error code for function that already exists and use while trying create user-defined functions --- src/Common/ErrorCodes.cpp | 1 + src/Functions/FunctionFactory.cpp | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index ad0463db889..b6523d772f2 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -549,6 +549,7 @@ M(579, INCORRECT_PART_TYPE) \ M(580, CANNOT_SET_ROUNDING_MODE) \ M(581, TOO_LARGE_DISTRIBUTED_DEPTH) \ + M(582, FUNCTION_ALREADY_EXISTS) \ \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index 69a34d4d030..8f71db3df7f 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes { extern const int UNKNOWN_FUNCTION; extern const int LOGICAL_ERROR; + extern const int FUNCTION_ALREADY_EXISTS; } const String & getFunctionCanonicalNameIfAny(const String & name) @@ -140,6 +141,9 @@ void FunctionFactory::registerUserDefinedFunction( const ASTCreateFunctionQuery & create_function_query, CaseSensitiveness case_sensitiveness) { + if (hasNameOrAlias(create_function_query.function_name)) + throw Exception("The function '" + create_function_query.function_name + "' already exists", ErrorCodes::FUNCTION_ALREADY_EXISTS); + registerFunction(create_function_query.function_name, [create_function_query](ContextPtr context) { auto function = UserDefinedFunction::create(context); From ee3d488337fae3c42435cb86ab24066d82f012eb Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Fri, 14 May 2021 00:43:02 +0300 Subject: [PATCH 007/224] Change error code in test 01857_create_function_that_already_exists.sql --- .../0_stateless/01857_create_function_that_already_exists.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01857_create_function_that_already_exists.sql b/tests/queries/0_stateless/01857_create_function_that_already_exists.sql index 444b722a272..e00f85d8552 100644 --- a/tests/queries/0_stateless/01857_create_function_that_already_exists.sql +++ b/tests/queries/0_stateless/01857_create_function_that_already_exists.sql @@ -1,2 +1,3 @@ create function MyFunc3 as (a, b) -> a + b; -create function MyFunc3 as (a) -> a || '!!!'; -- { serverError 49 } +create function MyFunc3 as (a) -> a || '!!!'; -- { serverError 585 } +create function cast as x -> x + 1; -- { serverError 585 } From 49f0af1d7c69effb539b7ab84e0796122be1fe66 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Fri, 14 May 2021 00:52:29 +0300 Subject: [PATCH 008/224] Revert wrong changes in CMake --- src/CMakeLists.txt | 31 ++++++++++++++++--------------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 4c7c9e23d24..985f96459ad 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -18,17 +18,17 @@ else() endif() include(../cmake/limit_jobs.cmake) -set (CONFIG_VERSION ${CMAKE_CURRENT_BINARY_DIR}/Common/config_version.h) -set (CONFIG_COMMON ${CMAKE_CURRENT_BINARY_DIR}/Common/config.h) +set (CONFIG_VERSION "${CMAKE_CURRENT_BINARY_DIR}/Common/config_version.h") +set (CONFIG_COMMON "${CMAKE_CURRENT_BINARY_DIR}/Common/config.h") include (../cmake/version.cmake) message (STATUS "Will build ${VERSION_FULL} revision ${VERSION_REVISION} ${VERSION_OFFICIAL}") configure_file (Common/config.h.in ${CONFIG_COMMON}) configure_file (Common/config_version.h.in ${CONFIG_VERSION}) -configure_file (Core/config_core.h.in ${CMAKE_CURRENT_BINARY_DIR}/Core/include/config_core.h) +configure_file (Core/config_core.h.in "${CMAKE_CURRENT_BINARY_DIR}/Core/include/config_core.h") if (USE_DEBUG_HELPERS) - set (INCLUDE_DEBUG_HELPERS "-I${ClickHouse_SOURCE_DIR}/base -include ${ClickHouse_SOURCE_DIR}/src/Core/iostream_debug_helpers.h") + set (INCLUDE_DEBUG_HELPERS "-I\"${ClickHouse_SOURCE_DIR}/base\" -include \"${ClickHouse_SOURCE_DIR}/src/Core/iostream_debug_helpers.h\"") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${INCLUDE_DEBUG_HELPERS}") endif () @@ -245,8 +245,8 @@ macro (dbms_target_link_libraries) endforeach () endmacro () -dbms_target_include_directories (PUBLIC ${ClickHouse_SOURCE_DIR}/src ${ClickHouse_BINARY_DIR}/src) -target_include_directories (clickhouse_common_io PUBLIC ${ClickHouse_SOURCE_DIR}/src ${ClickHouse_BINARY_DIR}/src) +dbms_target_include_directories (PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src") +target_include_directories (clickhouse_common_io PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src") if (USE_EMBEDDED_COMPILER) dbms_target_link_libraries (PRIVATE ${REQUIRED_LLVM_LIBRARIES}) @@ -322,10 +322,6 @@ if (USE_RDKAFKA) endif() endif() -if (USE_AMQPCPP) - dbms_target_link_libraries(PUBLIC amqp-cpp) -endif() - if (USE_CYRUS_SASL) dbms_target_link_libraries(PRIVATE ${CYRUS_SASL_LIBRARY}) endif() @@ -361,8 +357,8 @@ dbms_target_link_libraries ( clickhouse_common_io ) -target_include_directories(clickhouse_common_io PUBLIC ${CMAKE_CURRENT_BINARY_DIR}/Core/include) # uses some includes from core -dbms_target_include_directories(PUBLIC ${CMAKE_CURRENT_BINARY_DIR}/Core/include) +target_include_directories(clickhouse_common_io PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/Core/include") # uses some includes from core +dbms_target_include_directories(PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/Core/include") dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${PDQSORT_INCLUDE_DIR}) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${MINISELECT_INCLUDE_DIR}) @@ -441,6 +437,11 @@ if (USE_BROTLI) target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${BROTLI_INCLUDE_DIR}) endif() +if (USE_AMQPCPP) + dbms_target_link_libraries(PUBLIC ${AMQPCPP_LIBRARY}) + dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${AMQPCPP_INCLUDE_DIR}) +endif() + if (USE_CASSANDRA) dbms_target_link_libraries(PUBLIC ${CASSANDRA_LIBRARY}) dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${CASS_INCLUDE_DIR}) @@ -457,7 +458,7 @@ target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${FAST_FLO if (USE_ORC) dbms_target_link_libraries(PUBLIC ${ORC_LIBRARIES}) - dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR} ${CMAKE_BINARY_DIR}/contrib/orc/c++/include) + dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR} "${CMAKE_BINARY_DIR}/contrib/orc/c++/include") endif () if (USE_ROCKSDB) @@ -476,7 +477,7 @@ endif () dbms_target_link_libraries(PRIVATE _boost_context) -include (${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake) +include ("${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake") if (ENABLE_TESTS AND USE_GTEST) macro (grep_gtest_sources BASE_DIR DST_VAR) @@ -485,7 +486,7 @@ if (ENABLE_TESTS AND USE_GTEST) endmacro() # attach all dbms gtest sources - grep_gtest_sources(${ClickHouse_SOURCE_DIR}/src dbms_gtest_sources) + grep_gtest_sources("${ClickHouse_SOURCE_DIR}/src dbms_gtest_sources") add_executable(unit_tests_dbms ${dbms_gtest_sources}) # gtest framework has substandard code From 0d8233b809ba8b01de4eb7ade1a49e871e6a6bb7 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Fri, 14 May 2021 00:53:36 +0300 Subject: [PATCH 009/224] Revert wrong changes in CMake --- src/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 985f96459ad..1ee46d9ef78 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -486,7 +486,7 @@ if (ENABLE_TESTS AND USE_GTEST) endmacro() # attach all dbms gtest sources - grep_gtest_sources("${ClickHouse_SOURCE_DIR}/src dbms_gtest_sources") + grep_gtest_sources("${ClickHouse_SOURCE_DIR}/src" dbms_gtest_sources) add_executable(unit_tests_dbms ${dbms_gtest_sources}) # gtest framework has substandard code From 5059d2ee9afe9ffafe624c7602b414b6b66d946e Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Sat, 15 May 2021 01:43:04 +0300 Subject: [PATCH 010/224] Add ability to drop functions --- src/Common/ErrorCodes.cpp | 1 + src/Functions/FunctionFactory.cpp | 30 +++++++++++++--- src/Functions/FunctionFactory.h | 7 ++-- .../InterpreterDropFunctionQuery.cpp | 17 +++++++++ .../InterpreterDropFunctionQuery.h | 21 +++++++++++ src/Interpreters/InterpreterFactory.cpp | 6 ++++ src/Interpreters/ya.make | 1 + src/Parsers/ASTDropFunctionQuery.cpp | 19 ++++++++++ src/Parsers/ASTDropFunctionQuery.h | 20 +++++++++++ src/Parsers/ParserDropFunctionQuery.cpp | 35 +++++++++++++++++++ src/Parsers/ParserDropFunctionQuery.h | 14 ++++++++ src/Parsers/ParserQuery.cpp | 3 ++ src/Parsers/ya.make | 2 ++ 13 files changed, 168 insertions(+), 8 deletions(-) create mode 100644 src/Interpreters/InterpreterDropFunctionQuery.cpp create mode 100644 src/Interpreters/InterpreterDropFunctionQuery.h create mode 100644 src/Parsers/ASTDropFunctionQuery.cpp create mode 100644 src/Parsers/ASTDropFunctionQuery.h create mode 100644 src/Parsers/ParserDropFunctionQuery.cpp create mode 100644 src/Parsers/ParserDropFunctionQuery.h diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 3e802266c8b..cfd73a6bb28 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -553,6 +553,7 @@ M(583, ILLEGAL_PROJECTION) \ M(584, PROJECTION_NOT_USED) \ M(585, FUNCTION_ALREADY_EXISTS) \ + M(586, CANNOT_DROP_SYSTEM_FUNCTION) \ \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index 8f71db3df7f..cc52d8bd5e9 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes extern const int UNKNOWN_FUNCTION; extern const int LOGICAL_ERROR; extern const int FUNCTION_ALREADY_EXISTS; + extern const int CANNOT_DROP_SYSTEM_FUNCTION; } const String & getFunctionCanonicalNameIfAny(const String & name) @@ -137,12 +138,10 @@ FunctionFactory & FunctionFactory::instance() return ret; } -void FunctionFactory::registerUserDefinedFunction( - const ASTCreateFunctionQuery & create_function_query, - CaseSensitiveness case_sensitiveness) +void FunctionFactory::registerUserDefinedFunction(const ASTCreateFunctionQuery & create_function_query) { if (hasNameOrAlias(create_function_query.function_name)) - throw Exception("The function '" + create_function_query.function_name + "' already exists", ErrorCodes::FUNCTION_ALREADY_EXISTS); + throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "The function {} already exists", create_function_query.function_name); registerFunction(create_function_query.function_name, [create_function_query](ContextPtr context) { @@ -152,7 +151,28 @@ void FunctionFactory::registerUserDefinedFunction( FunctionOverloadResolverImplPtr res = std::make_unique(function); return res; - }, case_sensitiveness); + }, CaseSensitiveness::CaseSensitive); + user_defined_functions.insert(create_function_query.function_name); +} + +void FunctionFactory::unregisterUserDefinedFunction(const String & name) +{ + if (functions.contains(name)) + { + if (user_defined_functions.contains(name)) + { + functions.erase(name); + user_defined_functions.erase(name); + return; + } else + throw Exception("System functions cannot be dropped", ErrorCodes::CANNOT_DROP_SYSTEM_FUNCTION); + } + + auto hints = this->getHints(name); + if (!hints.empty()) + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown function {}. Maybe you meant: {}", name, toString(hints)); + else + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown function {}", name); } } diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index 9a93e7cd614..0bf8f562359 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -39,9 +39,9 @@ public: registerFunction(name, &Function::create, case_sensitiveness); } - void registerUserDefinedFunction( - const ASTCreateFunctionQuery & create_function_query, - CaseSensitiveness case_sensitiveness = CaseSensitive); + void registerUserDefinedFunction(const ASTCreateFunctionQuery & create_function_query); + + void unregisterUserDefinedFunction(const String & function_name); /// This function is used by YQL - internal Yandex product that depends on ClickHouse by source code. std::vector getAllNames() const; @@ -67,6 +67,7 @@ private: using Functions = std::unordered_map; Functions functions; + std::unordered_set user_defined_functions; Functions case_insensitive_functions; template diff --git a/src/Interpreters/InterpreterDropFunctionQuery.cpp b/src/Interpreters/InterpreterDropFunctionQuery.cpp new file mode 100644 index 00000000000..d3d07d99e37 --- /dev/null +++ b/src/Interpreters/InterpreterDropFunctionQuery.cpp @@ -0,0 +1,17 @@ +#include +#include +#include +#include + +namespace DB +{ + +BlockIO InterpreterDropFunctionQuery::execute() +{ + FunctionNameNormalizer().visit(query_ptr.get()); + auto & drop_function_query = query_ptr->as(); + FunctionFactory::instance().unregisterUserDefinedFunction(drop_function_query.function_name); + return {}; +} + +} diff --git a/src/Interpreters/InterpreterDropFunctionQuery.h b/src/Interpreters/InterpreterDropFunctionQuery.h new file mode 100644 index 00000000000..3d16b8c9ef3 --- /dev/null +++ b/src/Interpreters/InterpreterDropFunctionQuery.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +class ASTDropFunctionQuery; + +class InterpreterDropFunctionQuery : public IInterpreter, WithContext +{ +public: + InterpreterDropFunctionQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 54122292589..e5f88a80ddb 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -42,6 +43,7 @@ #include #include #include +#include #include #include #include @@ -269,6 +271,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextPtr { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else { throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY); diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index d101144725e..d5d1af50319 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -80,6 +80,7 @@ SRCS( InterpreterCreateUserQuery.cpp InterpreterDescribeQuery.cpp InterpreterDropAccessEntityQuery.cpp + InterpreterDropFunctionQuery.cpp InterpreterDropQuery.cpp InterpreterExistsQuery.cpp InterpreterExplainQuery.cpp diff --git a/src/Parsers/ASTDropFunctionQuery.cpp b/src/Parsers/ASTDropFunctionQuery.cpp new file mode 100644 index 00000000000..5800a7ba9cb --- /dev/null +++ b/src/Parsers/ASTDropFunctionQuery.cpp @@ -0,0 +1,19 @@ +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTDropFunctionQuery::clone() const +{ + return std::make_shared(*this); +} + +void ASTDropFunctionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP FUNCTION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(function_name) << (settings.hilite ? hilite_none : ""); +} + +} diff --git a/src/Parsers/ASTDropFunctionQuery.h b/src/Parsers/ASTDropFunctionQuery.h new file mode 100644 index 00000000000..e32bf93a64d --- /dev/null +++ b/src/Parsers/ASTDropFunctionQuery.h @@ -0,0 +1,20 @@ +#pragma once + +#include "IAST.h" + +namespace DB +{ + +class ASTDropFunctionQuery : public IAST +{ +public: + String function_name; + + String getID(char) const override { return "DropFunctionQuery"; } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; +}; + +} diff --git a/src/Parsers/ParserDropFunctionQuery.cpp b/src/Parsers/ParserDropFunctionQuery.cpp new file mode 100644 index 00000000000..04d26109836 --- /dev/null +++ b/src/Parsers/ParserDropFunctionQuery.cpp @@ -0,0 +1,35 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserDropFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_drop("DROP"); + ParserKeyword s_function("FUNCTION"); + ParserIdentifier function_name_p; + + ASTPtr function_name; + + if (!s_drop.ignore(pos, expected)) + return false; + + if (!s_function.ignore(pos, expected)) + return false; + + if (!function_name_p.parse(pos, function_name, expected)) + return false; + + auto drop_function_query = std::make_shared(); + node = drop_function_query; + + drop_function_query->function_name = function_name->as().name(); + + return true; +} + +} diff --git a/src/Parsers/ParserDropFunctionQuery.h b/src/Parsers/ParserDropFunctionQuery.h new file mode 100644 index 00000000000..03602c7ae96 --- /dev/null +++ b/src/Parsers/ParserDropFunctionQuery.h @@ -0,0 +1,14 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ +/// DROP FUNCTION function1 +class ParserDropFunctionQuery : public IParserBase +{ +protected: + const char * getName() const override { return "DROP FUNCTION query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 274dc0201b3..82106b2c1fb 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -38,6 +39,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateRowPolicyQuery create_row_policy_p; ParserCreateSettingsProfileQuery create_settings_profile_p; ParserCreateFunctionQuery create_function_p; + ParserDropFunctionQuery drop_function_p; ParserDropAccessEntityQuery drop_access_entity_p; ParserGrantQuery grant_p; ParserSetRoleQuery set_role_p; @@ -55,6 +57,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_row_policy_p.parse(pos, node, expected) || create_settings_profile_p.parse(pos, node, expected) || create_function_p.parse(pos, node, expected) + || drop_function_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected) || external_ddl_p.parse(pos, node, expected); diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index 4b6c27e7075..17499d0ca0a 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -25,6 +25,7 @@ SRCS( ASTDictionary.cpp ASTDictionaryAttributeDeclaration.cpp ASTDropAccessEntityQuery.cpp + ASTDropFunctionQuery.cpp ASTDropQuery.cpp ASTExpressionList.cpp ASTFunction.cpp @@ -101,6 +102,7 @@ SRCS( ParserDictionary.cpp ParserDictionaryAttributeDeclaration.cpp ParserDropAccessEntityQuery.cpp + ParserDropFunctionQuery.cpp ParserDropQuery.cpp ParserExplainQuery.cpp ParserExternalDDLQuery.cpp From 1077253cd638a54d2ce7f104b6639e1a9c5a7a97 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Sat, 15 May 2021 01:43:25 +0300 Subject: [PATCH 011/224] Add drop in tests --- tests/queries/0_stateless/01855_create_simple_function.sql | 1 + .../0_stateless/01857_create_function_that_already_exists.sql | 1 + .../01858_drop_unknown_and_system_function.reference | 0 .../0_stateless/01858_drop_unknown_and_system_function.sql | 2 ++ 4 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/01858_drop_unknown_and_system_function.reference create mode 100644 tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql diff --git a/tests/queries/0_stateless/01855_create_simple_function.sql b/tests/queries/0_stateless/01855_create_simple_function.sql index 8fb0117fd3c..d0f7400dce1 100644 --- a/tests/queries/0_stateless/01855_create_simple_function.sql +++ b/tests/queries/0_stateless/01855_create_simple_function.sql @@ -1,2 +1,3 @@ create function MyFunc as (a, b, c) -> a * b * c; select MyFunc(2, 3, 4); +drop function MyFunc; diff --git a/tests/queries/0_stateless/01857_create_function_that_already_exists.sql b/tests/queries/0_stateless/01857_create_function_that_already_exists.sql index e00f85d8552..040ed2ef7f2 100644 --- a/tests/queries/0_stateless/01857_create_function_that_already_exists.sql +++ b/tests/queries/0_stateless/01857_create_function_that_already_exists.sql @@ -1,3 +1,4 @@ create function MyFunc3 as (a, b) -> a + b; create function MyFunc3 as (a) -> a || '!!!'; -- { serverError 585 } create function cast as x -> x + 1; -- { serverError 585 } +drop function MyFunc3; diff --git a/tests/queries/0_stateless/01858_drop_unknown_and_system_function.reference b/tests/queries/0_stateless/01858_drop_unknown_and_system_function.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql b/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql new file mode 100644 index 00000000000..a87fee5b044 --- /dev/null +++ b/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql @@ -0,0 +1,2 @@ +drop function unknownFunc; -- { serverError 46 } +drop function CAST; -- { serverError 586 } From 15046adba3c815b4ff0d830b3435741d2039b7f6 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Sat, 15 May 2021 15:48:27 +0300 Subject: [PATCH 012/224] Add tests in excluding list of parallel runs --- src/Interpreters/ya.make | 2 +- tests/queries/skip_list.json | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index d5d1af50319..0d1a4e01a04 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -55,7 +55,7 @@ SRCS( ExpressionAnalyzer.cpp ExternalDictionariesLoader.cpp ExternalLoader.cpp - ExternalLoaderDatabaseConfigRepository.cpp + ExternalLoaderDictionaryStorageConfigRepository.cpp ExternalLoaderTempConfigRepository.cpp ExternalLoaderXMLConfigRepository.cpp ExternalModelsLoader.cpp diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index bc12fed0c92..526505316b5 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -719,6 +719,8 @@ "01802_test_postgresql_protocol_with_row_policy", /// Creates database and users "01804_dictionary_decimal256_type", "01850_dist_INSERT_preserve_error", // uses cluster with different static databases shard_0/shard_1 - "01710_projection_fetch" + "01710_projection_fetch", + "01855_create_simple_function", + "01857_create_function_that_already_exists" ] } From f4ef8a80d7d4d43bb3fd9c4a7d61274beb827b9d Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Sun, 16 May 2021 21:02:59 +0300 Subject: [PATCH 013/224] Create default column if it has null value --- src/Functions/UserDefinedFunction.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Functions/UserDefinedFunction.cpp b/src/Functions/UserDefinedFunction.cpp index b7b4ff8de3e..78d407f9a1b 100644 --- a/src/Functions/UserDefinedFunction.cpp +++ b/src/Functions/UserDefinedFunction.cpp @@ -75,7 +75,11 @@ Block UserDefinedFunction::executeCore(const ColumnsWithTypeAndName & arguments) throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); lambda_arguments.emplace_back(*opt_arg_name, arguments[j].type); - block.insert({arguments[j].column, arguments[j].type, *opt_arg_name}); + std::cerr << "***Arguments: " << *opt_arg_name << " " << arguments[j].type << " " << arguments[j].column << std::endl; + auto column_ptr = arguments[j].column; + if (!column_ptr) + column_ptr = arguments[j].type->createColumnConstWithDefaultValue(1); + block.insert({column_ptr, arguments[j].type, *opt_arg_name}); } ASTPtr lambda_body = function_core->as()->children.at(0)->children.at(1); From 45a44ff4b396484d0c75721ea395f22e1d637e9e Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Sun, 16 May 2021 21:10:32 +0300 Subject: [PATCH 014/224] Remove debug prints --- src/Functions/UserDefinedFunction.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/UserDefinedFunction.cpp b/src/Functions/UserDefinedFunction.cpp index 78d407f9a1b..ad49fc1b0bb 100644 --- a/src/Functions/UserDefinedFunction.cpp +++ b/src/Functions/UserDefinedFunction.cpp @@ -75,7 +75,6 @@ Block UserDefinedFunction::executeCore(const ColumnsWithTypeAndName & arguments) throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); lambda_arguments.emplace_back(*opt_arg_name, arguments[j].type); - std::cerr << "***Arguments: " << *opt_arg_name << " " << arguments[j].type << " " << arguments[j].column << std::endl; auto column_ptr = arguments[j].column; if (!column_ptr) column_ptr = arguments[j].type->createColumnConstWithDefaultValue(1); From 5f71d65344be4a3a29d498645d4fda93a734d8f0 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Wed, 9 Jun 2021 19:35:21 +0300 Subject: [PATCH 015/224] Fix build errors --- src/Functions/UserDefinedFunction.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/UserDefinedFunction.h b/src/Functions/UserDefinedFunction.h index 2b519740204..2781bae9ef8 100644 --- a/src/Functions/UserDefinedFunction.h +++ b/src/Functions/UserDefinedFunction.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include From 3c2a5fcdf55832c7877d4811120b289634a5da07 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Wed, 9 Jun 2021 19:37:42 +0300 Subject: [PATCH 016/224] Fix tests --- .../0_stateless/01857_create_function_that_already_exists.sql | 4 ++-- .../0_stateless/01858_drop_unknown_and_system_function.sql | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01857_create_function_that_already_exists.sql b/tests/queries/0_stateless/01857_create_function_that_already_exists.sql index 040ed2ef7f2..28dbf2c950b 100644 --- a/tests/queries/0_stateless/01857_create_function_that_already_exists.sql +++ b/tests/queries/0_stateless/01857_create_function_that_already_exists.sql @@ -1,4 +1,4 @@ create function MyFunc3 as (a, b) -> a + b; -create function MyFunc3 as (a) -> a || '!!!'; -- { serverError 585 } -create function cast as x -> x + 1; -- { serverError 585 } +create function MyFunc3 as (a) -> a || '!!!'; -- { serverError 587 } +create function cast as x -> x + 1; -- { serverError 587 } drop function MyFunc3; diff --git a/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql b/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql index a87fee5b044..c882df0d0c5 100644 --- a/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql +++ b/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql @@ -1,2 +1,2 @@ drop function unknownFunc; -- { serverError 46 } -drop function CAST; -- { serverError 586 } +drop function CAST; -- { serverError 588 } From e2b82d3a3ad0077c00360f43e7003926b5718797 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Tue, 15 Jun 2021 02:07:20 +0300 Subject: [PATCH 017/224] Fix build error --- src/Functions/FunctionFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index 917cbedfb3f..68b6954b901 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -148,7 +148,7 @@ void FunctionFactory::registerUserDefinedFunction(const ASTCreateFunctionQuery & function->setName(create_function_query.function_name); function->setFunctionCore(create_function_query.function_core); - FunctionOverloadResolverImplPtr res = std::make_unique(function); + FunctionOverloadResolverPtr res = std::make_unique(function); return res; }, CaseSensitiveness::CaseSensitive); user_defined_functions.insert(create_function_query.function_name); From fa4d2614cc4bc6d616762579f9f3495183c480f2 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Mon, 21 Jun 2021 22:45:17 +0300 Subject: [PATCH 018/224] Update tests after merge conflicts --- .../0_stateless/01857_create_function_that_already_exists.sql | 4 ++-- .../0_stateless/01858_drop_unknown_and_system_function.sql | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01857_create_function_that_already_exists.sql b/tests/queries/0_stateless/01857_create_function_that_already_exists.sql index 28dbf2c950b..8fb6615744d 100644 --- a/tests/queries/0_stateless/01857_create_function_that_already_exists.sql +++ b/tests/queries/0_stateless/01857_create_function_that_already_exists.sql @@ -1,4 +1,4 @@ create function MyFunc3 as (a, b) -> a + b; -create function MyFunc3 as (a) -> a || '!!!'; -- { serverError 587 } -create function cast as x -> x + 1; -- { serverError 587 } +create function MyFunc3 as (a) -> a || '!!!'; -- { serverError 588 } +create function cast as x -> x + 1; -- { serverError 588 } drop function MyFunc3; diff --git a/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql b/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql index c882df0d0c5..f9f347a805f 100644 --- a/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql +++ b/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql @@ -1,2 +1,2 @@ drop function unknownFunc; -- { serverError 46 } -drop function CAST; -- { serverError 588 } +drop function CAST; -- { serverError 589 } From 3bf0db519eb2b682fadd8a6412bcd4762bbad090 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Sat, 3 Jul 2021 23:17:02 +0300 Subject: [PATCH 019/224] Improve tests and validation of new functions --- src/Common/ErrorCodes.cpp | 1 + src/Functions/FunctionFactory.cpp | 12 ++++--- .../InterpreterCreateFunctionQuery.cpp | 28 +++++++++++++---- .../InterpreterCreateFunctionQuery.h | 5 +-- src/Parsers/ASTFunction.cpp | 31 +++++++++++++++++++ src/Parsers/ASTFunction.h | 10 ++++++ .../01855_create_simple_function.reference | 1 + .../01855_create_simple_function.sql | 1 + ...=> 01856_create_function_errors.reference} | 0 .../01856_create_function_errors.sql | 13 ++++++++ ...function_with_unknown_variable_in_body.sql | 1 - ..._function_and_check_jit_compiled.reference | 3 ++ ...create_function_and_check_jit_compiled.sql | 5 +++ ...ate_function_that_already_exists.reference | 0 ...57_create_function_that_already_exists.sql | 4 --- ...01858_drop_unknown_and_system_function.sql | 2 +- 16 files changed, 99 insertions(+), 18 deletions(-) rename tests/queries/0_stateless/{01856_create_function_with_unknown_variable_in_body.reference => 01856_create_function_errors.reference} (100%) create mode 100644 tests/queries/0_stateless/01856_create_function_errors.sql delete mode 100644 tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.sql create mode 100644 tests/queries/0_stateless/01857_create_function_and_check_jit_compiled.reference create mode 100644 tests/queries/0_stateless/01857_create_function_and_check_jit_compiled.sql delete mode 100644 tests/queries/0_stateless/01857_create_function_that_already_exists.reference delete mode 100644 tests/queries/0_stateless/01857_create_function_that_already_exists.sql diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f79d1ba1003..335ef6fbf53 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -559,6 +559,7 @@ M(589, DISTRIBUTED_BROKEN_BATCH_FILES) \ M(590, FUNCTION_ALREADY_EXISTS) \ M(591, CANNOT_DROP_SYSTEM_FUNCTION) \ + M(592, CANNOT_CREATE_RECURSIVE_FUNCTION) \ \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index b267727372c..75d045a24ed 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -57,7 +58,7 @@ void FunctionFactory::registerFunction(const FunctionOverloadResolverPtr FunctionFactory::getImpl( const std::string & name, - ContextPtr context) const + ContextConstPtr context) const { auto res = tryGetImpl(name, context); if (!res) @@ -87,14 +88,14 @@ std::vector FunctionFactory::getAllNames() const FunctionOverloadResolverPtr FunctionFactory::get( const std::string & name, - ContextPtr context) const + ContextConstPtr context) const { return getImpl(name, context); } FunctionOverloadResolverPtr FunctionFactory::tryGetImpl( const std::string & name_param, - ContextPtr context) const + ContextConstPtr context) const { String name = getAliasToOrName(name_param); FunctionOverloadResolverPtr res; @@ -125,7 +126,7 @@ FunctionOverloadResolverPtr FunctionFactory::tryGetImpl( FunctionOverloadResolverPtr FunctionFactory::tryGet( const std::string & name, - ContextPtr context) const + ContextConstPtr context) const { auto impl = tryGetImpl(name, context); return impl ? std::move(impl) : nullptr; @@ -142,6 +143,9 @@ void FunctionFactory::registerUserDefinedFunction(const ASTCreateFunctionQuery & if (hasNameOrAlias(create_function_query.function_name)) throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "The function {} already exists", create_function_query.function_name); + if (AggregateFunctionFactory::instance().isAggregateFunctionName(create_function_query.function_name)) + throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "The aggregate function {} already exists", create_function_query.function_name); + registerFunction(create_function_query.function_name, [create_function_query](ContextPtr context) { auto function = UserDefinedFunction::create(context); diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index 83fdc8ad29c..5ea6070dd3c 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -1,9 +1,9 @@ -#include #include #include #include #include #include +#include namespace DB { @@ -11,25 +11,26 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_IDENTIFIER; + extern const int CANNOT_CREATE_RECURSIVE_FUNCTION; } BlockIO InterpreterCreateFunctionQuery::execute() { FunctionNameNormalizer().visit(query_ptr.get()); auto & create_function_query = query_ptr->as(); - validateFunction(create_function_query.function_core); + validateFunction(create_function_query.function_core, create_function_query.function_name); FunctionFactory::instance().registerUserDefinedFunction(create_function_query); return {}; } -void InterpreterCreateFunctionQuery::validateFunction(ASTPtr function) +void InterpreterCreateFunctionQuery::validateFunction(ASTPtr function, const String & name) { const auto * args_tuple = function->as()->arguments->children.at(0)->as(); std::unordered_set arguments; for (const auto & argument : args_tuple->arguments->children) arguments.insert(argument->as()->name()); - std::vector identifiers_in_body; + std::set identifiers_in_body; ASTPtr function_body = function->as()->children.at(0)->children.at(1); getIdentifiers(function_body, identifiers_in_body); @@ -42,18 +43,33 @@ void InterpreterCreateFunctionQuery::validateFunction(ASTPtr function) throw Exception(s.str(), ErrorCodes::UNKNOWN_IDENTIFIER); } } + + validateFunctionRecursiveness(function_body, name); } -void InterpreterCreateFunctionQuery::getIdentifiers(ASTPtr node, std::vector & identifiers) +void InterpreterCreateFunctionQuery::getIdentifiers(ASTPtr node, std::set & identifiers) { for (const auto & child : node->children) { auto identifier_name_opt = tryGetIdentifierName(child); if (identifier_name_opt) - identifiers.push_back(identifier_name_opt.value()); + identifiers.insert(identifier_name_opt.value()); getIdentifiers(child, identifiers); } } +void InterpreterCreateFunctionQuery::validateFunctionRecursiveness(ASTPtr node, const String & function_to_create) +{ + for (const auto & child : node->children) + { + auto function_name_opt = tryGetFunctionName(child); + if (function_name_opt && function_name_opt.value() == function_to_create) + throw Exception("You cannot create recursive function", ErrorCodes::CANNOT_CREATE_RECURSIVE_FUNCTION); + + validateFunctionRecursiveness(child, function_to_create); + } +} + + } diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.h b/src/Interpreters/InterpreterCreateFunctionQuery.h index 79b7839116a..dff01fb013b 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.h +++ b/src/Interpreters/InterpreterCreateFunctionQuery.h @@ -16,8 +16,9 @@ public: BlockIO execute() override; private: - static void validateFunction(ASTPtr function); - static void getIdentifiers(ASTPtr node, std::vector & identifiers); + static void validateFunction(ASTPtr function, const String & name); + static void getIdentifiers(ASTPtr node, std::set & identifiers); + static void validateFunctionRecursiveness(ASTPtr node, const String & function_to_create); private: ASTPtr query_ptr; diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index dfbf2532f1f..e7d2bb3b52c 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -21,6 +22,7 @@ namespace DB namespace ErrorCodes { extern const int UNEXPECTED_EXPRESSION; + extern const int UNEXPECTED_AST_STRUCTURE; } void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const @@ -573,4 +575,33 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format } } +String getFunctionName(const IAST * ast) +{ + String res; + if (tryGetFunctionNameInto(ast, res)) + return res; + throw Exception(ast ? queryToString(*ast) + " is not an function" : "AST node is nullptr", ErrorCodes::UNEXPECTED_AST_STRUCTURE); +} + +std::optional tryGetFunctionName(const IAST * ast) +{ + String res; + if (tryGetFunctionNameInto(ast, res)) + return res; + return {}; +} + +bool tryGetFunctionNameInto(const IAST * ast, String & name) +{ + if (ast) + { + if (const auto * node = ast->as()) + { + name = node->name; + return true; + } + } + return false; +} + } diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 8e657afbf6e..6f73fc4b852 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -75,4 +75,14 @@ std::shared_ptr makeASTFunction(const String & name, Args &&... arg return function; } +/// ASTFunction Helpers: hide casts and semantic. + +String getFunctionName(const IAST * ast); +std::optional tryGetFunctionName(const IAST * ast); +bool tryGetFunctionNameInto(const IAST * ast, String & name); + +inline String getFunctionName(const ASTPtr & ast) { return getFunctionName(ast.get()); } +inline std::optional tryGetFunctionName(const ASTPtr & ast) { return tryGetFunctionName(ast.get()); } +inline bool tryGetFunctionNameInto(const ASTPtr & ast, String & name) { return tryGetFunctionNameInto(ast.get(), name); } + } diff --git a/tests/queries/0_stateless/01855_create_simple_function.reference b/tests/queries/0_stateless/01855_create_simple_function.reference index a45fd52cc58..a211b2318a0 100644 --- a/tests/queries/0_stateless/01855_create_simple_function.reference +++ b/tests/queries/0_stateless/01855_create_simple_function.reference @@ -1 +1,2 @@ 24 +1 diff --git a/tests/queries/0_stateless/01855_create_simple_function.sql b/tests/queries/0_stateless/01855_create_simple_function.sql index d0f7400dce1..36e3f8cfd94 100644 --- a/tests/queries/0_stateless/01855_create_simple_function.sql +++ b/tests/queries/0_stateless/01855_create_simple_function.sql @@ -1,3 +1,4 @@ create function MyFunc as (a, b, c) -> a * b * c; select MyFunc(2, 3, 4); +select isConstant(MyFunc(1, 2, 3)); drop function MyFunc; diff --git a/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.reference b/tests/queries/0_stateless/01856_create_function_errors.reference similarity index 100% rename from tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.reference rename to tests/queries/0_stateless/01856_create_function_errors.reference diff --git a/tests/queries/0_stateless/01856_create_function_errors.sql b/tests/queries/0_stateless/01856_create_function_errors.sql new file mode 100644 index 00000000000..abed91e677b --- /dev/null +++ b/tests/queries/0_stateless/01856_create_function_errors.sql @@ -0,0 +1,13 @@ +create function MyFunc2 as (a, b) -> a || b || c; -- { serverError 47 } + +create function MyFunc2 as (a, b) -> MyFunc2(a, b) + MyFunc2(a, b); -- { serverError 592 } recursive function + +create function cast as a -> a + 1; -- { serverError 590 } function already exist + +create function sum as (a, b) -> a + b; -- { serverError 590 } aggregate function already exist + +create function MyFunc3 as (a, b) -> a + b; + +create function MyFunc3 as (a) -> a || '!!!'; -- { serverError 590 } function already exist + +drop function MyFunc3; diff --git a/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.sql b/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.sql deleted file mode 100644 index 1fed9fe9103..00000000000 --- a/tests/queries/0_stateless/01856_create_function_with_unknown_variable_in_body.sql +++ /dev/null @@ -1 +0,0 @@ -create function MyFunc2 as (a, b) -> a || b || c; -- { serverError 47 } diff --git a/tests/queries/0_stateless/01857_create_function_and_check_jit_compiled.reference b/tests/queries/0_stateless/01857_create_function_and_check_jit_compiled.reference new file mode 100644 index 00000000000..7938dcdde86 --- /dev/null +++ b/tests/queries/0_stateless/01857_create_function_and_check_jit_compiled.reference @@ -0,0 +1,3 @@ +0 +1 +0 diff --git a/tests/queries/0_stateless/01857_create_function_and_check_jit_compiled.sql b/tests/queries/0_stateless/01857_create_function_and_check_jit_compiled.sql new file mode 100644 index 00000000000..058c701bd4c --- /dev/null +++ b/tests/queries/0_stateless/01857_create_function_and_check_jit_compiled.sql @@ -0,0 +1,5 @@ +create function MyFunc as (a, b, c) -> a + b > c AND c < 10; +select MyFunc(1, 2, 3); +select MyFunc(2, 2, 3); +select MyFunc(20, 20, 11); +drop function MyFunc; diff --git a/tests/queries/0_stateless/01857_create_function_that_already_exists.reference b/tests/queries/0_stateless/01857_create_function_that_already_exists.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/01857_create_function_that_already_exists.sql b/tests/queries/0_stateless/01857_create_function_that_already_exists.sql deleted file mode 100644 index 8fb6615744d..00000000000 --- a/tests/queries/0_stateless/01857_create_function_that_already_exists.sql +++ /dev/null @@ -1,4 +0,0 @@ -create function MyFunc3 as (a, b) -> a + b; -create function MyFunc3 as (a) -> a || '!!!'; -- { serverError 588 } -create function cast as x -> x + 1; -- { serverError 588 } -drop function MyFunc3; diff --git a/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql b/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql index f9f347a805f..8debd575edc 100644 --- a/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql +++ b/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql @@ -1,2 +1,2 @@ drop function unknownFunc; -- { serverError 46 } -drop function CAST; -- { serverError 589 } +drop function CAST; -- { serverError 591 } From a640ea1f2c854be4d636d465422602c7e42e088e Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Sun, 4 Jul 2021 00:47:26 +0300 Subject: [PATCH 020/224] Add test with jit compiled checking in skip list --- tests/queries/skip_list.json | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 266a98e5be2..dd81fb02316 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -847,7 +847,8 @@ "01821_table_comment", "01710_projection_fetch", "01855_create_simple_function", - "01857_create_function_that_already_exists", + "01856_create_function_errors.sql", + "01857_create_function_and_check_jit_compiled.sql", "01824_prefer_global_in_and_join", "01870_modulo_partition_key", "01870_buffer_flush", // creates database From b277d4db91c4785a4260c97051e9349623fb1baa Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Sun, 4 Jul 2021 00:47:43 +0300 Subject: [PATCH 021/224] Add mutex in FunctionFactory --- src/Functions/FunctionFactory.cpp | 14 ++++++++++---- src/Functions/FunctionFactory.h | 14 ++++++++------ 2 files changed, 18 insertions(+), 10 deletions(-) diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index 75d045a24ed..26cefc45df0 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -1,4 +1,3 @@ -#include #include #include @@ -32,11 +31,12 @@ const String & getFunctionCanonicalNameIfAny(const String & name) return FunctionFactory::instance().getCanonicalNameIfAny(name); } -void FunctionFactory::registerFunction(const - std::string & name, +void FunctionFactory::registerFunction( + const std::string & name, Value creator, CaseSensitiveness case_sensitiveness) { + std::lock_guard guard(mutex); if (!functions.emplace(name, creator).second) throw Exception("FunctionFactory: the function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); @@ -79,6 +79,7 @@ FunctionOverloadResolverPtr FunctionFactory::getImpl( std::vector FunctionFactory::getAllNames() const { + std::lock_guard guard(mutex); std::vector res; res.reserve(functions.size()); for (const auto & func : functions) @@ -97,6 +98,7 @@ FunctionOverloadResolverPtr FunctionFactory::tryGetImpl( const std::string & name_param, ContextConstPtr context) const { + std::lock_guard guard(mutex); String name = getAliasToOrName(name_param); FunctionOverloadResolverPtr res; @@ -146,6 +148,10 @@ void FunctionFactory::registerUserDefinedFunction(const ASTCreateFunctionQuery & if (AggregateFunctionFactory::instance().isAggregateFunctionName(create_function_query.function_name)) throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "The aggregate function {} already exists", create_function_query.function_name); + { + std::lock_guard guard(mutex); + user_defined_functions.insert(create_function_query.function_name); + } registerFunction(create_function_query.function_name, [create_function_query](ContextPtr context) { auto function = UserDefinedFunction::create(context); @@ -155,11 +161,11 @@ void FunctionFactory::registerUserDefinedFunction(const ASTCreateFunctionQuery & FunctionOverloadResolverPtr res = std::make_unique(function); return res; }, CaseSensitiveness::CaseSensitive); - user_defined_functions.insert(create_function_query.function_name); } void FunctionFactory::unregisterUserDefinedFunction(const String & name) { + std::lock_guard guard(mutex); if (functions.contains(name)) { if (user_defined_functions.contains(name)) diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index ade597d63bb..e62f8eb97da 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -8,6 +8,7 @@ #include #include +#include #include #include @@ -20,7 +21,7 @@ namespace DB * some dictionaries from Context. */ class FunctionFactory : private boost::noncopyable, - public IFactoryWithAliases> + public IFactoryWithAliases> { public: static FunctionFactory & instance(); @@ -49,14 +50,14 @@ public: std::vector getAllNames() const; /// Throws an exception if not found. - FunctionOverloadResolverPtr get(const std::string & name, ContextPtr context) const; + FunctionOverloadResolverPtr get(const std::string & name, ContextConstPtr context) const; /// Returns nullptr if not found. - FunctionOverloadResolverPtr tryGet(const std::string & name, ContextPtr context) const; + FunctionOverloadResolverPtr tryGet(const std::string & name, ContextConstPtr context) const; /// The same methods to get developer interface implementation. - FunctionOverloadResolverPtr getImpl(const std::string & name, ContextPtr context) const; - FunctionOverloadResolverPtr tryGetImpl(const std::string & name, ContextPtr context) const; + FunctionOverloadResolverPtr getImpl(const std::string & name, ContextConstPtr context) const; + FunctionOverloadResolverPtr tryGetImpl(const std::string & name, ContextConstPtr context) const; /// Register a function by its name. /// No locking, you must register all functions before usage of get. @@ -71,9 +72,10 @@ private: Functions functions; std::unordered_set user_defined_functions; Functions case_insensitive_functions; + mutable std::mutex mutex; template - static FunctionOverloadResolverPtr adaptFunctionToOverloadResolver(ContextPtr context) + static FunctionOverloadResolverPtr adaptFunctionToOverloadResolver(ContextConstPtr context) { return std::make_unique(Function::create(context)); } From ed162094aa2b71617d673eca41ba7bb0b9be077f Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Fri, 9 Jul 2021 00:18:44 +0300 Subject: [PATCH 022/224] Update tests after merging --- .../queries/0_stateless/01856_create_function_errors.sql | 8 ++++---- .../01858_drop_unknown_and_system_function.sql | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01856_create_function_errors.sql b/tests/queries/0_stateless/01856_create_function_errors.sql index abed91e677b..d185dacc7f1 100644 --- a/tests/queries/0_stateless/01856_create_function_errors.sql +++ b/tests/queries/0_stateless/01856_create_function_errors.sql @@ -1,13 +1,13 @@ create function MyFunc2 as (a, b) -> a || b || c; -- { serverError 47 } -create function MyFunc2 as (a, b) -> MyFunc2(a, b) + MyFunc2(a, b); -- { serverError 592 } recursive function +create function MyFunc2 as (a, b) -> MyFunc2(a, b) + MyFunc2(a, b); -- { serverError 593 } recursive function -create function cast as a -> a + 1; -- { serverError 590 } function already exist +create function cast as a -> a + 1; -- { serverError 591 } function already exist -create function sum as (a, b) -> a + b; -- { serverError 590 } aggregate function already exist +create function sum as (a, b) -> a + b; -- { serverError 591 } aggregate function already exist create function MyFunc3 as (a, b) -> a + b; -create function MyFunc3 as (a) -> a || '!!!'; -- { serverError 590 } function already exist +create function MyFunc3 as (a) -> a || '!!!'; -- { serverError 591 } function already exist drop function MyFunc3; diff --git a/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql b/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql index 8debd575edc..06d9b30f9f6 100644 --- a/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql +++ b/tests/queries/0_stateless/01858_drop_unknown_and_system_function.sql @@ -1,2 +1,2 @@ drop function unknownFunc; -- { serverError 46 } -drop function CAST; -- { serverError 591 } +drop function CAST; -- { serverError 592 } From 71b3e771b112dd5c1cf3ebaffed34ad564bcbca3 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Thu, 15 Jul 2021 00:20:13 +0300 Subject: [PATCH 023/224] Fix renaming ContextPtr -> ContextConstPtr from previous commit --- src/Functions/FunctionFactory.cpp | 10 +++++----- src/Functions/FunctionFactory.h | 12 ++++++------ 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index 26cefc45df0..f691e6e7e41 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -58,7 +58,7 @@ void FunctionFactory::registerFunction( FunctionOverloadResolverPtr FunctionFactory::getImpl( const std::string & name, - ContextConstPtr context) const + ContextPtr context) const { auto res = tryGetImpl(name, context); if (!res) @@ -89,14 +89,14 @@ std::vector FunctionFactory::getAllNames() const FunctionOverloadResolverPtr FunctionFactory::get( const std::string & name, - ContextConstPtr context) const + ContextPtr context) const { return getImpl(name, context); } FunctionOverloadResolverPtr FunctionFactory::tryGetImpl( const std::string & name_param, - ContextConstPtr context) const + ContextPtr context) const { std::lock_guard guard(mutex); String name = getAliasToOrName(name_param); @@ -127,8 +127,8 @@ FunctionOverloadResolverPtr FunctionFactory::tryGetImpl( } FunctionOverloadResolverPtr FunctionFactory::tryGet( - const std::string & name, - ContextConstPtr context) const + const std::string & name, + ContextPtr context) const { auto impl = tryGetImpl(name, context); return impl ? std::move(impl) : nullptr; diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index e62f8eb97da..b652ff46f5a 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -21,7 +21,7 @@ namespace DB * some dictionaries from Context. */ class FunctionFactory : private boost::noncopyable, - public IFactoryWithAliases> + public IFactoryWithAliases> { public: static FunctionFactory & instance(); @@ -50,14 +50,14 @@ public: std::vector getAllNames() const; /// Throws an exception if not found. - FunctionOverloadResolverPtr get(const std::string & name, ContextConstPtr context) const; + FunctionOverloadResolverPtr get(const std::string & name, ContextPtr context) const; /// Returns nullptr if not found. - FunctionOverloadResolverPtr tryGet(const std::string & name, ContextConstPtr context) const; + FunctionOverloadResolverPtr tryGet(const std::string & name, ContextPtr context) const; /// The same methods to get developer interface implementation. - FunctionOverloadResolverPtr getImpl(const std::string & name, ContextConstPtr context) const; - FunctionOverloadResolverPtr tryGetImpl(const std::string & name, ContextConstPtr context) const; + FunctionOverloadResolverPtr getImpl(const std::string & name, ContextPtr context) const; + FunctionOverloadResolverPtr tryGetImpl(const std::string & name, ContextPtr context) const; /// Register a function by its name. /// No locking, you must register all functions before usage of get. @@ -75,7 +75,7 @@ private: mutable std::mutex mutex; template - static FunctionOverloadResolverPtr adaptFunctionToOverloadResolver(ContextConstPtr context) + static FunctionOverloadResolverPtr adaptFunctionToOverloadResolver(ContextPtr context) { return std::make_unique(Function::create(context)); } From 6ce203504b04aaa5ca6856d453369fbc6c96879a Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Tue, 20 Jul 2021 02:34:04 +0300 Subject: [PATCH 024/224] Add access rights --- src/Access/AccessType.h | 2 + .../InterpreterCreateFunctionQuery.cpp | 3 ++ .../InterpreterCreateFunctionQuery.h | 5 +- .../InterpreterDropFunctionQuery.cpp | 4 ++ .../InterpreterDropFunctionQuery.h | 5 +- .../test_access_for_functions/__main__.py | 0 .../test_access_for_functions/test.py | 47 +++++++++++++++++++ 7 files changed, 62 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_access_for_functions/__main__.py create mode 100644 tests/integration/test_access_for_functions/test.py diff --git a/src/Access/AccessType.h b/src/Access/AccessType.h index 47153b5ab63..e7bb5f7ce23 100644 --- a/src/Access/AccessType.h +++ b/src/Access/AccessType.h @@ -87,6 +87,7 @@ enum class AccessType M(CREATE_DICTIONARY, "", DICTIONARY, CREATE) /* allows to execute {CREATE|ATTACH} DICTIONARY */\ M(CREATE_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables; implicitly enabled by the grant CREATE_TABLE on any table */ \ + M(CREATE_FUNCTION, "", DATABASE, CREATE) /* allows to execute CREATE FUNCTION */ \ M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \ \ M(DROP_DATABASE, "", DATABASE, DROP) /* allows to execute {DROP|DETACH} DATABASE */\ @@ -94,6 +95,7 @@ enum class AccessType M(DROP_VIEW, "", VIEW, DROP) /* allows to execute {DROP|DETACH} TABLE for views; implicitly enabled by the grant DROP_TABLE */\ M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\ + M(DROP_FUNCTION, "", DATABASE, DROP) /* allows to execute DROP FUNCTION */\ M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\ \ M(TRUNCATE, "TRUNCATE TABLE", TABLE, ALL) \ diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index 5ea6070dd3c..5204d767863 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -1,3 +1,5 @@ +#include +#include #include #include #include @@ -16,6 +18,7 @@ namespace ErrorCodes BlockIO InterpreterCreateFunctionQuery::execute() { + getContext()->checkAccess(AccessType::CREATE_FUNCTION); FunctionNameNormalizer().visit(query_ptr.get()); auto & create_function_query = query_ptr->as(); validateFunction(create_function_query.function_core, create_function_query.function_name); diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.h b/src/Interpreters/InterpreterCreateFunctionQuery.h index dff01fb013b..e657c8fff31 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.h +++ b/src/Interpreters/InterpreterCreateFunctionQuery.h @@ -7,11 +7,12 @@ namespace DB { class ASTCreateFunctionQuery; +class Context; -class InterpreterCreateFunctionQuery : public IInterpreter, WithContext +class InterpreterCreateFunctionQuery : public IInterpreter, WithMutableContext { public: - InterpreterCreateFunctionQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + InterpreterCreateFunctionQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} BlockIO execute() override; diff --git a/src/Interpreters/InterpreterDropFunctionQuery.cpp b/src/Interpreters/InterpreterDropFunctionQuery.cpp index d3d07d99e37..d2d02b38d6c 100644 --- a/src/Interpreters/InterpreterDropFunctionQuery.cpp +++ b/src/Interpreters/InterpreterDropFunctionQuery.cpp @@ -1,13 +1,17 @@ +#include +#include #include #include #include #include + namespace DB { BlockIO InterpreterDropFunctionQuery::execute() { + getContext()->checkAccess(AccessType::DROP_FUNCTION); FunctionNameNormalizer().visit(query_ptr.get()); auto & drop_function_query = query_ptr->as(); FunctionFactory::instance().unregisterUserDefinedFunction(drop_function_query.function_name); diff --git a/src/Interpreters/InterpreterDropFunctionQuery.h b/src/Interpreters/InterpreterDropFunctionQuery.h index 3d16b8c9ef3..31d914948ed 100644 --- a/src/Interpreters/InterpreterDropFunctionQuery.h +++ b/src/Interpreters/InterpreterDropFunctionQuery.h @@ -6,11 +6,12 @@ namespace DB { class ASTDropFunctionQuery; +class Context; -class InterpreterDropFunctionQuery : public IInterpreter, WithContext +class InterpreterDropFunctionQuery : public IInterpreter, WithMutableContext { public: - InterpreterDropFunctionQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} + InterpreterDropFunctionQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} BlockIO execute() override; diff --git a/tests/integration/test_access_for_functions/__main__.py b/tests/integration/test_access_for_functions/__main__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_access_for_functions/test.py b/tests/integration/test_access_for_functions/test.py new file mode 100644 index 00000000000..4252dd24cf2 --- /dev/null +++ b/tests/integration/test_access_for_functions/test.py @@ -0,0 +1,47 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance') + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def cleanup_after_test(): + try: + yield + finally: + instance.query("DROP USER IF EXISTS A") + instance.query("DROP USER IF EXISTS B") + + +def test_merge(): + create_function_query = "CREATE FUNCTION MySum AS (a, b) -> a + b" + + instance.query("CREATE USER A") + instance.query("CREATE USER B") + assert "it's necessary to have grant CREATE FUNCTION ON *.*" in instance.query_and_get_error(create_function_query, user = 'A') + + instance.query("GRANT CREATE FUNCTION on *.* TO A") + + instance.query(create_function_query.format, user = 'A') + assert instance.query("SELECT MySum(1, 2)") == "3" + + assert "it's necessary to have grant DROP FUNCTION ON *.*" in instance.query("DROP FUNCTION MySum", user = 'B') + + instance.query("GRANT DROP FUNCTION ON *.* TO B") + instance.query("DROP FUNCTION MySum", user = 'B') + assert "Unknown function MySum" in instance.query("SELECT MySum(1, 2)") + + instance.query("REVOKE CREATE FUNCTION ON default FROM A") + assert "it's necessary to have grant CREATE FUNCTION ON *.*" in instance.query_and_get_error(create_function_query, user = 'A') From 7966bded6cca9f1d68a36b9dbe2677025bb39161 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Tue, 20 Jul 2021 23:20:23 +0300 Subject: [PATCH 025/224] Add storing on disk for created functions --- programs/local/LocalServer.cpp | 7 + programs/server/Server.cpp | 14 ++ src/Common/ErrorCodes.cpp | 2 + .../InterpreterCreateFunctionQuery.cpp | 18 ++ .../InterpreterCreateFunctionQuery.h | 5 + src/Interpreters/UserDefinedObjectsOnDisk.cpp | 176 ++++++++++++++++++ src/Interpreters/UserDefinedObjectsOnDisk.h | 30 +++ src/Interpreters/ya.make | 1 + 8 files changed, 253 insertions(+) create mode 100644 src/Interpreters/UserDefinedObjectsOnDisk.cpp create mode 100644 src/Interpreters/UserDefinedObjectsOnDisk.h diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 2633f0e9426..3aa20437cda 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -286,6 +287,12 @@ try /// Lock path directory before read status.emplace(path + "status", StatusFile::write_full_info); + fs::create_directories(fs::path(path) / "user_defined/"); + LOG_DEBUG(log, "Loading user defined objects from {}", path); + Poco::File(path + "user_defined/").createDirectories(); + UserDefinedObjectsOnDisk::instance().loadUserDefinedObjects(global_context); + LOG_DEBUG(log, "Loaded user defined objects."); + LOG_DEBUG(log, "Loading metadata from {}", path); fs::create_directories(fs::path(path) / "data/"); fs::create_directories(fs::path(path) / "metadata/"); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 313523d19dc..3f007bb32f4 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -50,6 +50,7 @@ #include #include #include +#include #include #include #include @@ -717,6 +718,7 @@ int Server::main(const std::vector & /*args*/) { fs::create_directories(fs::path(path) / "data/"); fs::create_directories(fs::path(path) / "metadata/"); + fs::create_directories(fs::path(path) / "user_defined/"); /// Directory with metadata of tables, which was marked as dropped by Atomic database fs::create_directories(fs::path(path) / "metadata_dropped/"); @@ -1034,6 +1036,18 @@ int Server::main(const std::vector & /*args*/) /// system logs may copy global context. global_context->setCurrentDatabaseNameInGlobalContext(default_database); + LOG_INFO(log, "Loading user defined objects from {}", path); + try + { + UserDefinedObjectsOnDisk::instance().loadUserDefinedObjects(global_context); + } + catch (...) + { + tryLogCurrentException(log, "Caught exception while loading user defined objects"); + throw; + } + LOG_DEBUG(log, "Loaded user defined objects"); + LOG_INFO(log, "Loading metadata from {}", path); try diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 4cab28c8d53..da1c5dab89d 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -562,6 +562,8 @@ M(591, FUNCTION_ALREADY_EXISTS) \ M(592, CANNOT_DROP_SYSTEM_FUNCTION) \ M(593, CANNOT_CREATE_RECURSIVE_FUNCTION) \ + M(594, FUNCTION_ALREADY_STORED_ON_DISK) \ + M(595, FUNCTION_WAS_NOT_STORED_ON_DISK) \ \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index 5204d767863..1416fbf9001 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -23,6 +24,19 @@ BlockIO InterpreterCreateFunctionQuery::execute() auto & create_function_query = query_ptr->as(); validateFunction(create_function_query.function_core, create_function_query.function_name); FunctionFactory::instance().registerUserDefinedFunction(create_function_query); + if (!internal) + { + try + { + UserDefinedObjectsOnDisk::instance().storeUserDefinedFunction(getContext(), create_function_query); + } + catch (Exception & e) + { + FunctionFactory::instance().unregisterUserDefinedFunction(create_function_query.function_name); + e.addMessage(fmt::format("while storing user defined function {} on disk", backQuote(create_function_query.function_name))); + throw; + } + } return {}; } @@ -74,5 +88,9 @@ void InterpreterCreateFunctionQuery::validateFunctionRecursiveness(ASTPtr node, } } +void InterpreterCreateFunctionQuery::setInternal(bool internal_) +{ + internal = internal_; +} } diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.h b/src/Interpreters/InterpreterCreateFunctionQuery.h index e657c8fff31..5325683fbf0 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.h +++ b/src/Interpreters/InterpreterCreateFunctionQuery.h @@ -16,6 +16,8 @@ public: BlockIO execute() override; + void setInternal(bool internal_); + private: static void validateFunction(ASTPtr function, const String & name); static void getIdentifiers(ASTPtr node, std::set & identifiers); @@ -23,6 +25,9 @@ private: private: ASTPtr query_ptr; + + /// Is this an internal query - not from the user. + bool internal = false; }; } diff --git a/src/Interpreters/UserDefinedObjectsOnDisk.cpp b/src/Interpreters/UserDefinedObjectsOnDisk.cpp new file mode 100644 index 00000000000..6d701980925 --- /dev/null +++ b/src/Interpreters/UserDefinedObjectsOnDisk.cpp @@ -0,0 +1,176 @@ +#include +#include +#include + +#include +#include +#include +#include + +#include +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FUNCTION_ALREADY_STORED_ON_DISK; + extern const int FUNCTION_WAS_NOT_STORED_ON_DISK; +} + +UserDefinedObjectsOnDisk & UserDefinedObjectsOnDisk::instance() +{ + static UserDefinedObjectsOnDisk ret; + return ret; +} + + +void UserDefinedObjectsOnDisk::executeCreateFunctionQuery( + const String & query, + ContextMutablePtr context, + const String & file_name) +{ + ParserCreateFunctionQuery parser; + ASTPtr ast = parseQuery( + parser, query.data(), query.data() + query.size(), "in file " + file_name, 0, context->getSettingsRef().max_parser_depth); + + InterpreterCreateFunctionQuery interpreter(ast, context); + interpreter.setInternal(true); + interpreter.execute(); +} + +void UserDefinedObjectsOnDisk::loadUserDefinedObject(ContextMutablePtr context, const String & name, const String & path) +{ + Poco::Logger * log = &Poco::Logger::get("LoadUserDefinedObject"); + String object_create_query; + + LOG_DEBUG(log, "Loading function {} from file {}", backQuote(name), path); + if (Poco::File(path).exists()) + { + /// There is .sql file with user defined object creation statement. + ReadBufferFromFile in(path, 1024); + readStringUntilEOF(object_create_query, in); + } + try + { + executeCreateFunctionQuery(object_create_query, context, path); + LOG_DEBUG(log, "Loaded function {}", backQuote(name)); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while loading user defined function {} from path {}", backQuote(name), path)); + throw; + } +} + +void UserDefinedObjectsOnDisk::loadUserDefinedObjects(ContextMutablePtr context) +{ + String dir_path = context->getPath() + "user_defined/"; + std::vector> user_defined_objects_with_priority; + Poco::DirectoryIterator dir_end; + for (Poco::DirectoryIterator it(dir_path); it != dir_end; ++it) + { + if (it->isLink()) + continue; + + if (!it->isDirectory() && endsWith(it.name(), ".sql")) + { + int priority = std::stoi(it.name().substr(0, it.name().find('_'))); + user_defined_objects_with_priority.emplace_back(priority, it.name()); + + continue; + } + + /// For '.svn', '.gitignore' directory and similar. + if (it.name().at(0) == '.') + continue; + } + std::sort(user_defined_objects_with_priority.begin(), user_defined_objects_with_priority.end()); + + for (const auto & [priority, file_name] : user_defined_objects_with_priority) + { + int name_start_index = file_name.find('_') + 1; + String name = file_name.substr(name_start_index, file_name.size() - 4 - name_start_index); + loadUserDefinedObject(context, name, dir_path + file_name); + } + if (user_defined_objects_with_priority.empty()) + user_defined_objects_count.store(0); + else + user_defined_objects_count.store(user_defined_objects_with_priority.back().first); +} + +void UserDefinedObjectsOnDisk::storeUserDefinedFunction(ContextPtr context, const ASTCreateFunctionQuery & ast) +{ + Poco::Logger * log = &Poco::Logger::get("StoreUserDefinedFunction"); + + String dir_path = context->getPath() + "user_defined/"; + Poco::DirectoryIterator dir_end; + for (Poco::DirectoryIterator it(dir_path); it != dir_end; ++it) + { + re2::StringPiece input(it.name()); + re2::RE2 re("[0-9]+_" + escapeForFileName(ast.function_name) + "\\.sql"); + + if (re2::RE2::FullMatch(input, re)) + { + throw Exception("User defined function " + backQuote(it.name()) + " already stored on disk", ErrorCodes::FUNCTION_ALREADY_STORED_ON_DISK); + } + } + + int object_priority = ++user_defined_objects_count; + String new_file_path = dir_path + toString(object_priority) + "_" + escapeForFileName(ast.function_name) + ".sql"; + LOG_DEBUG(log, "Storing function {} to file {}", backQuote(ast.function_name), new_file_path); + + WriteBufferFromOwnString create_statement_buf; + formatAST(ast, create_statement_buf, false); + writeChar('\n', create_statement_buf); + String create_statement = create_statement_buf.str(); + + WriteBufferFromFile out(new_file_path, create_statement.size(), O_WRONLY | O_CREAT | O_EXCL); + writeString(create_statement, out); + out.next(); + if (context->getSettingsRef().fsync_metadata) + out.sync(); + out.close(); + LOG_DEBUG(log, "Stored function {}", backQuote(ast.function_name)); +} + +void UserDefinedObjectsOnDisk::removeUserDefinedFunction(ContextPtr context, const String & name) +{ + Poco::Logger * log = &Poco::Logger::get("RemoveUserDefinedFunction"); + + String dir_path = context->getPath() + "user_defined/"; + + LOG_DEBUG(log, "Removing file for function {} from {}", backQuote(name), dir_path); + + Poco::DirectoryIterator dir_end; + for (Poco::DirectoryIterator it(dir_path); it != dir_end; ++it) + { + String file_name = it.name(); + re2::StringPiece input(file_name); + re2::RE2 re("[0-9]+_" + escapeForFileName(name) + "\\.sql"); + + if (re2::RE2::FullMatch(input, re)) + { + it->remove(); + LOG_DEBUG(log, "Removed file {}", dir_path + file_name); + return; + } + } + + throw Exception("Stored file for user defined function " + backQuote(name) + " was not found", ErrorCodes::FUNCTION_WAS_NOT_STORED_ON_DISK); +} + +} diff --git a/src/Interpreters/UserDefinedObjectsOnDisk.h b/src/Interpreters/UserDefinedObjectsOnDisk.h new file mode 100644 index 00000000000..f0ecf294093 --- /dev/null +++ b/src/Interpreters/UserDefinedObjectsOnDisk.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + +#include +#include + + +namespace DB +{ + +class UserDefinedObjectsOnDisk : private boost::noncopyable +{ +public: + static UserDefinedObjectsOnDisk & instance(); + + void loadUserDefinedObjects(ContextMutablePtr context); + void storeUserDefinedFunction(ContextPtr context, const ASTCreateFunctionQuery & ast); + static void removeUserDefinedFunction(ContextPtr context, const String & name); + +private: + static void loadUserDefinedObject(ContextMutablePtr context, const String & name, const String & path); + static void executeCreateFunctionQuery(const String & query, ContextMutablePtr context, const String & file_name); + +private: + std::atomic_int user_defined_objects_count = 0; +}; + +} diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index 098bb01de17..dd7bf8de016 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -156,6 +156,7 @@ SRCS( TranslateQualifiedNamesVisitor.cpp TreeOptimizer.cpp TreeRewriter.cpp + UserDefinedObjectsOnDisk.cpp WindowDescription.cpp addMissingDefaults.cpp addTypeConversionToAST.cpp From f4e2e552eb91fc36fd9e82d7019ff0a833266693 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Tue, 20 Jul 2021 23:42:23 +0300 Subject: [PATCH 026/224] Add integration test for persistence --- .../__init__.py | 0 .../test.py | 43 +++++++++++++++++++ 2 files changed, 43 insertions(+) create mode 100644 tests/integration/test_user_defined_object_persistence/__init__.py create mode 100644 tests/integration/test_user_defined_object_persistence/test.py diff --git a/tests/integration/test_user_defined_object_persistence/__init__.py b/tests/integration/test_user_defined_object_persistence/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_user_defined_object_persistence/test.py b/tests/integration/test_user_defined_object_persistence/test.py new file mode 100644 index 00000000000..eb07ce0a95a --- /dev/null +++ b/tests/integration/test_user_defined_object_persistence/test.py @@ -0,0 +1,43 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance') + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_persistence(): + create_function_query1 = "CREATE FUNCTION MySum1 AS (a, b) -> a + b" + create_function_query2 = "CREATE FUNCTION MySum2 AS (a, b) -> MySum1(a, b) + b" + + instance.query(create_function_query1) + instance.query(create_function_query2) + + assert instance.query("SELECT MySum1(1,2)") == "3" + assert instance.query("SELECT MySum2(1,2)") == "5" + + cluster.shutdown() + cluster.start() + instance = cluster.add_instance('instance') + + assert instance.query("SELECT MySum1(1,2)") == "3" + assert instance.query("SELECT MySum2(1,2)") == "5" + + instance.query("DROP FUNCTION MySum2") + instance.query("DROP FUNCTION MySum1") + + cluster.shutdown() + cluster.start() + instance = cluster.add_instance('instance') + + assert "Unknown function MySum1" in instance.query("SELECT MySum1(1, 2)") + assert "Unknown function MySum2" in instance.query("SELECT MySum2(1, 2)") From 44e1c2bcf5b8f830238b9c48c1d02d5920ee072f Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Tue, 20 Jul 2021 23:42:48 +0300 Subject: [PATCH 027/224] Fix test for check access rights for functions --- tests/integration/test_access_for_functions/test.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/test_access_for_functions/test.py b/tests/integration/test_access_for_functions/test.py index 4252dd24cf2..9719f381530 100644 --- a/tests/integration/test_access_for_functions/test.py +++ b/tests/integration/test_access_for_functions/test.py @@ -1,6 +1,5 @@ import pytest from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance') @@ -34,7 +33,7 @@ def test_merge(): instance.query("GRANT CREATE FUNCTION on *.* TO A") - instance.query(create_function_query.format, user = 'A') + instance.query(create_function_query, user = 'A') assert instance.query("SELECT MySum(1, 2)") == "3" assert "it's necessary to have grant DROP FUNCTION ON *.*" in instance.query("DROP FUNCTION MySum", user = 'B') From c03a75a289265fd658d47fc53e7fbda57993ece8 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Wed, 21 Jul 2021 01:02:18 +0300 Subject: [PATCH 028/224] Fix tests --- src/Interpreters/InterpreterDropFunctionQuery.cpp | 2 ++ tests/queries/0_stateless/01271_show_privileges.reference | 2 ++ 2 files changed, 4 insertions(+) diff --git a/src/Interpreters/InterpreterDropFunctionQuery.cpp b/src/Interpreters/InterpreterDropFunctionQuery.cpp index d2d02b38d6c..c1e3e15f116 100644 --- a/src/Interpreters/InterpreterDropFunctionQuery.cpp +++ b/src/Interpreters/InterpreterDropFunctionQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -15,6 +16,7 @@ BlockIO InterpreterDropFunctionQuery::execute() FunctionNameNormalizer().visit(query_ptr.get()); auto & drop_function_query = query_ptr->as(); FunctionFactory::instance().unregisterUserDefinedFunction(drop_function_query.function_name); + UserDefinedObjectsOnDisk::instance().removeUserDefinedFunction(getContext(), drop_function_query.function_name); return {}; } diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 035cb902bff..b8b1656d867 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -45,11 +45,13 @@ CREATE TABLE [] TABLE CREATE CREATE VIEW [] VIEW CREATE CREATE DICTIONARY [] DICTIONARY CREATE CREATE TEMPORARY TABLE [] GLOBAL CREATE +CREATE FUNCTION [] DATABASE CREATE CREATE [] \N ALL DROP DATABASE [] DATABASE DROP DROP TABLE [] TABLE DROP DROP VIEW [] VIEW DROP DROP DICTIONARY [] DICTIONARY DROP +DROP FUNCTION [] DATABASE DROP DROP [] \N ALL TRUNCATE ['TRUNCATE TABLE'] TABLE ALL OPTIMIZE ['OPTIMIZE TABLE'] TABLE ALL From 31f4fe9a4d83d39ed8d6d0a96e42614c01b61c34 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Wed, 21 Jul 2021 10:29:26 +0300 Subject: [PATCH 029/224] Fix tests --- .../test_access_for_functions/test.py | 4 ++-- .../test.py | 20 +++++++++---------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_access_for_functions/test.py b/tests/integration/test_access_for_functions/test.py index 9719f381530..4118a244662 100644 --- a/tests/integration/test_access_for_functions/test.py +++ b/tests/integration/test_access_for_functions/test.py @@ -24,7 +24,7 @@ def cleanup_after_test(): instance.query("DROP USER IF EXISTS B") -def test_merge(): +def test_access_rights_for_funtion(): create_function_query = "CREATE FUNCTION MySum AS (a, b) -> a + b" instance.query("CREATE USER A") @@ -34,7 +34,7 @@ def test_merge(): instance.query("GRANT CREATE FUNCTION on *.* TO A") instance.query(create_function_query, user = 'A') - assert instance.query("SELECT MySum(1, 2)") == "3" + assert instance.query("SELECT MySum(1, 2)") == "3\n" assert "it's necessary to have grant DROP FUNCTION ON *.*" in instance.query("DROP FUNCTION MySum", user = 'B') diff --git a/tests/integration/test_user_defined_object_persistence/test.py b/tests/integration/test_user_defined_object_persistence/test.py index eb07ce0a95a..1646c9e8188 100644 --- a/tests/integration/test_user_defined_object_persistence/test.py +++ b/tests/integration/test_user_defined_object_persistence/test.py @@ -22,22 +22,22 @@ def test_persistence(): instance.query(create_function_query1) instance.query(create_function_query2) - assert instance.query("SELECT MySum1(1,2)") == "3" - assert instance.query("SELECT MySum2(1,2)") == "5" + assert instance.query("SELECT MySum1(1,2)") == "3\n" + assert instance.query("SELECT MySum2(1,2)") == "5\n" cluster.shutdown() cluster.start() - instance = cluster.add_instance('instance') + instance1 = cluster.add_instance('instance') - assert instance.query("SELECT MySum1(1,2)") == "3" - assert instance.query("SELECT MySum2(1,2)") == "5" + assert instance1.query("SELECT MySum1(1,2)") == "3\n" + assert instance1.query("SELECT MySum2(1,2)") == "5\n" - instance.query("DROP FUNCTION MySum2") - instance.query("DROP FUNCTION MySum1") + instance1.query("DROP FUNCTION MySum2") + instance1.query("DROP FUNCTION MySum1") cluster.shutdown() cluster.start() - instance = cluster.add_instance('instance') + instance2 = cluster.add_instance('instance') - assert "Unknown function MySum1" in instance.query("SELECT MySum1(1, 2)") - assert "Unknown function MySum2" in instance.query("SELECT MySum2(1, 2)") + assert "Unknown function MySum1" in instance2.query("SELECT MySum1(1, 2)") + assert "Unknown function MySum2" in instance2.query("SELECT MySum2(1, 2)") From b5b69b0255aa4567f5deb02ebb25127f85e53d90 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Wed, 21 Jul 2021 10:33:04 +0300 Subject: [PATCH 030/224] Fix unit tests --- src/Access/tests/gtest_access_rights_ops.cpp | 94 ++++++++++++++++++++ 1 file changed, 94 insertions(+) create mode 100644 src/Access/tests/gtest_access_rights_ops.cpp diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp new file mode 100644 index 00000000000..2881825dd17 --- /dev/null +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -0,0 +1,94 @@ +#include +#include + +using namespace DB; + + +TEST(AccessRights, Union) +{ + AccessRights lhs, rhs; + lhs.grant(AccessType::CREATE_TABLE, "db1", "tb1"); + rhs.grant(AccessType::SELECT, "db2"); + lhs.makeUnion(rhs); + ASSERT_EQ(lhs.toString(), "GRANT CREATE TABLE ON db1.tb1, GRANT SELECT ON db2.*"); + + lhs.clear(); + rhs.clear(); + rhs.grant(AccessType::SELECT, "db2"); + lhs.grant(AccessType::CREATE_TABLE, "db1", "tb1"); + lhs.makeUnion(rhs); + ASSERT_EQ(lhs.toString(), "GRANT CREATE TABLE ON db1.tb1, GRANT SELECT ON db2.*"); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT); + rhs.grant(AccessType::SELECT, "db1", "tb1"); + lhs.makeUnion(rhs); + ASSERT_EQ(lhs.toString(), "GRANT SELECT ON *.*"); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT, "db1", "tb1", Strings{"col1", "col2"}); + rhs.grant(AccessType::SELECT, "db1", "tb1", Strings{"col2", "col3"}); + lhs.makeUnion(rhs); + ASSERT_EQ(lhs.toString(), "GRANT SELECT(col1, col2, col3) ON db1.tb1"); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT, "db1", "tb1", Strings{"col1", "col2"}); + rhs.grantWithGrantOption(AccessType::SELECT, "db1", "tb1", Strings{"col2", "col3"}); + lhs.makeUnion(rhs); + ASSERT_EQ(lhs.toString(), "GRANT SELECT(col1) ON db1.tb1, GRANT SELECT(col2, col3) ON db1.tb1 WITH GRANT OPTION"); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::INSERT); + rhs.grant(AccessType::ALL, "db1"); + lhs.makeUnion(rhs); + ASSERT_EQ(lhs.toString(), "GRANT INSERT ON *.*, GRANT SHOW, SELECT, ALTER, CREATE DATABASE, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, CREATE FUNCTION, DROP, TRUNCATE, OPTIMIZE, 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 FLUSH DISTRIBUTED, dictGet ON db1.*"); +} + + +TEST(AccessRights, Intersection) +{ + AccessRights lhs, rhs; + lhs.grant(AccessType::CREATE_TABLE, "db1", "tb1"); + rhs.grant(AccessType::SELECT, "db2"); + lhs.makeIntersection(rhs); + ASSERT_EQ(lhs.toString(), "GRANT USAGE ON *.*"); + + lhs.clear(); + rhs.clear(); + lhs.grant(AccessType::SELECT, "db2"); + rhs.grant(AccessType::CREATE_TABLE, "db1", "tb1"); + lhs.makeIntersection(rhs); + ASSERT_EQ(lhs.toString(), "GRANT USAGE ON *.*"); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT); + rhs.grant(AccessType::SELECT, "db1", "tb1"); + lhs.makeIntersection(rhs); + ASSERT_EQ(lhs.toString(), "GRANT SELECT ON db1.tb1"); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT, "db1", "tb1", Strings{"col1", "col2"}); + rhs.grant(AccessType::SELECT, "db1", "tb1", Strings{"col2", "col3"}); + lhs.makeIntersection(rhs); + ASSERT_EQ(lhs.toString(), "GRANT SELECT(col2) ON db1.tb1"); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT, "db1", "tb1", Strings{"col1", "col2"}); + rhs.grantWithGrantOption(AccessType::SELECT, "db1", "tb1", Strings{"col2", "col3"}); + lhs.makeIntersection(rhs); + ASSERT_EQ(lhs.toString(), "GRANT SELECT(col2) ON db1.tb1"); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::INSERT); + rhs.grant(AccessType::ALL, "db1"); + lhs.makeIntersection(rhs); + ASSERT_EQ(lhs.toString(), "GRANT INSERT ON db1.*"); +} From 503d62cce1f419bf4bc48f0d271fd5df9b7909b3 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Wed, 21 Jul 2021 12:41:17 +0300 Subject: [PATCH 031/224] Fix integration tests --- .../integration/test_access_for_functions/test.py | 2 +- .../test_user_defined_object_persistence/test.py | 14 ++++++-------- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_access_for_functions/test.py b/tests/integration/test_access_for_functions/test.py index 4118a244662..0d1f20f3ad3 100644 --- a/tests/integration/test_access_for_functions/test.py +++ b/tests/integration/test_access_for_functions/test.py @@ -36,7 +36,7 @@ def test_access_rights_for_funtion(): instance.query(create_function_query, user = 'A') assert instance.query("SELECT MySum(1, 2)") == "3\n" - assert "it's necessary to have grant DROP FUNCTION ON *.*" in instance.query("DROP FUNCTION MySum", user = 'B') + assert "it's necessary to have grant DROP FUNCTION ON *.*" in instance.query_and_get_error("DROP FUNCTION MySum", user = 'B') instance.query("GRANT DROP FUNCTION ON *.* TO B") instance.query("DROP FUNCTION MySum", user = 'B') diff --git a/tests/integration/test_user_defined_object_persistence/test.py b/tests/integration/test_user_defined_object_persistence/test.py index 1646c9e8188..f755edd1073 100644 --- a/tests/integration/test_user_defined_object_persistence/test.py +++ b/tests/integration/test_user_defined_object_persistence/test.py @@ -27,17 +27,15 @@ def test_persistence(): cluster.shutdown() cluster.start() - instance1 = cluster.add_instance('instance') - assert instance1.query("SELECT MySum1(1,2)") == "3\n" - assert instance1.query("SELECT MySum2(1,2)") == "5\n" + assert instance.query("SELECT MySum1(1,2)") == "3\n" + assert instance.query("SELECT MySum2(1,2)") == "5\n" - instance1.query("DROP FUNCTION MySum2") - instance1.query("DROP FUNCTION MySum1") + instance.query("DROP FUNCTION MySum2") + instance.query("DROP FUNCTION MySum1") cluster.shutdown() cluster.start() - instance2 = cluster.add_instance('instance') - assert "Unknown function MySum1" in instance2.query("SELECT MySum1(1, 2)") - assert "Unknown function MySum2" in instance2.query("SELECT MySum2(1, 2)") + assert "Unknown function MySum1" in instance.query_and_get_error("SELECT MySum1(1, 2)") + assert "Unknown function MySum2" in instance.query_and_get_error("SELECT MySum2(1, 2)") From 1c99253c6acb1734602029993474d4e3cf71b26e Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Wed, 21 Jul 2021 14:10:37 +0300 Subject: [PATCH 032/224] Fix integration tests --- tests/integration/test_access_for_functions/test.py | 2 +- .../test_user_defined_object_persistence/test.py | 6 ++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_access_for_functions/test.py b/tests/integration/test_access_for_functions/test.py index 0d1f20f3ad3..c93531c0a77 100644 --- a/tests/integration/test_access_for_functions/test.py +++ b/tests/integration/test_access_for_functions/test.py @@ -40,7 +40,7 @@ def test_access_rights_for_funtion(): instance.query("GRANT DROP FUNCTION ON *.* TO B") instance.query("DROP FUNCTION MySum", user = 'B') - assert "Unknown function MySum" in instance.query("SELECT MySum(1, 2)") + assert "Unknown function MySum" in instance.query_and_get_error("SELECT MySum(1, 2)") instance.query("REVOKE CREATE FUNCTION ON default FROM A") assert "it's necessary to have grant CREATE FUNCTION ON *.*" in instance.query_and_get_error(create_function_query, user = 'A') diff --git a/tests/integration/test_user_defined_object_persistence/test.py b/tests/integration/test_user_defined_object_persistence/test.py index f755edd1073..153e9c11773 100644 --- a/tests/integration/test_user_defined_object_persistence/test.py +++ b/tests/integration/test_user_defined_object_persistence/test.py @@ -25,8 +25,7 @@ def test_persistence(): assert instance.query("SELECT MySum1(1,2)") == "3\n" assert instance.query("SELECT MySum2(1,2)") == "5\n" - cluster.shutdown() - cluster.start() + instance.restart_clickhouse() assert instance.query("SELECT MySum1(1,2)") == "3\n" assert instance.query("SELECT MySum2(1,2)") == "5\n" @@ -34,8 +33,7 @@ def test_persistence(): instance.query("DROP FUNCTION MySum2") instance.query("DROP FUNCTION MySum1") - cluster.shutdown() - cluster.start() + instance.restart_clickhouse() assert "Unknown function MySum1" in instance.query_and_get_error("SELECT MySum1(1, 2)") assert "Unknown function MySum2" in instance.query_and_get_error("SELECT MySum2(1, 2)") From c3461ce99af391283ebb8c65243449e1e2618c63 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Wed, 21 Jul 2021 16:04:52 +0300 Subject: [PATCH 033/224] Fix integration tests --- tests/integration/test_access_for_functions/test.py | 2 +- tests/integration/test_user_defined_object_persistence/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_access_for_functions/test.py b/tests/integration/test_access_for_functions/test.py index c93531c0a77..6eacd906079 100644 --- a/tests/integration/test_access_for_functions/test.py +++ b/tests/integration/test_access_for_functions/test.py @@ -42,5 +42,5 @@ def test_access_rights_for_funtion(): instance.query("DROP FUNCTION MySum", user = 'B') assert "Unknown function MySum" in instance.query_and_get_error("SELECT MySum(1, 2)") - instance.query("REVOKE CREATE FUNCTION ON default FROM A") + instance.query("REVOKE CREATE FUNCTION ON *.* FROM A") assert "it's necessary to have grant CREATE FUNCTION ON *.*" in instance.query_and_get_error(create_function_query, user = 'A') diff --git a/tests/integration/test_user_defined_object_persistence/test.py b/tests/integration/test_user_defined_object_persistence/test.py index 153e9c11773..6993bc13615 100644 --- a/tests/integration/test_user_defined_object_persistence/test.py +++ b/tests/integration/test_user_defined_object_persistence/test.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance') +instance = cluster.add_instance('instance', stay_alive=True) @pytest.fixture(scope="module", autouse=True) From 21c577d337f6ccb46703053cdfb1e1ecae199f81 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Thu, 22 Jul 2021 20:13:19 +0300 Subject: [PATCH 034/224] Rename __main__ to __init__ in tests --- tests/integration/test_access_for_functions/__main__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 tests/integration/test_access_for_functions/__main__.py diff --git a/tests/integration/test_access_for_functions/__main__.py b/tests/integration/test_access_for_functions/__main__.py deleted file mode 100644 index e69de29bb2d..00000000000 From 263c3f9ff328f019899fa88b36118550fa6da486 Mon Sep 17 00:00:00 2001 From: ANDREI STAROVEROV Date: Fri, 23 Jul 2021 10:27:35 +0300 Subject: [PATCH 035/224] Add __init__.py for test_access_for_functions --- tests/integration/test_access_for_functions/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_access_for_functions/__init__.py diff --git a/tests/integration/test_access_for_functions/__init__.py b/tests/integration/test_access_for_functions/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From 36aaabfd3ab1a67235419a311e954c38e8e9b0ba Mon Sep 17 00:00:00 2001 From: Nickita Taranov Date: Sun, 25 Jul 2021 21:43:00 +0300 Subject: [PATCH 036/224] impl --- src/Parsers/ExpressionElementParsers.cpp | 69 +++++++++++++++++-- src/Parsers/ExpressionElementParsers.h | 7 ++ .../02000_select_with_filter.reference | 2 + .../0_stateless/02000_select_with_filter.sql | 4 ++ 4 files changed, 76 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02000_select_with_filter.reference create mode 100644 tests/queries/0_stateless/02000_select_with_filter.sql diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index ca563ddea41..dd9fc738094 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -3,10 +3,10 @@ #include -#include #include -#include +#include #include +#include #include #include @@ -49,6 +49,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int SYNTAX_ERROR; extern const int LOGICAL_ERROR; + extern const int ILLEGAL_AGGREGATION; } @@ -268,7 +269,6 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex return true; } - bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserIdentifier id_parser; @@ -276,6 +276,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword all("ALL"); ParserExpressionList contents(false, is_table_function); ParserSelectWithUnionQuery select; + ParserKeyword filter("FILTER"); ParserKeyword over("OVER"); bool has_all = false; @@ -440,16 +441,27 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) function_node->children.push_back(function_node->parameters); } - if (over.ignore(pos, expected)) + if (filter.ignore(pos, expected)) { - function_node->is_window_function = true; - // We are slightly breaking the parser interface by parsing the window // definition into an existing ASTFunction. Normally it would take a // reference to ASTPtr and assign it the new node. We only have a pointer // of a different type, hence this workaround with a temporary pointer. ASTPtr function_node_as_iast = function_node; + ParserFilterClause filter_parser; + if (!filter_parser.parse(pos, function_node_as_iast, expected)) + { + return false; + } + } + + if (over.ignore(pos, expected)) + { + function_node->is_window_function = true; + + ASTPtr function_node_as_iast = function_node; + ParserWindowReference window_reference; if (!window_reference.parse(pos, function_node_as_iast, expected)) { @@ -504,6 +516,51 @@ bool ParserTableFunctionView::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return true; } +bool ParserFilterClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + assert(node); + ASTFunction & function = dynamic_cast(*node); + + ParserToken parser_openging_bracket(TokenType::OpeningRoundBracket); + if (!parser_openging_bracket.ignore(pos, expected)) + { + return false; + } + + ParserKeyword parser_where("WHERE"); + if (!parser_where.ignore(pos, expected)) + { + return false; + } + ParserExpressionList parser_condition(false); + ASTPtr condition; + if (!parser_condition.parse(pos, condition, expected)) + { + return false; + } + + ParserToken parser_closing_bracket(TokenType::ClosingRoundBracket); + if (!parser_closing_bracket.ignore(pos, expected)) + { + return false; + } + + if (function.name.find("If") != String::npos) + { + throw Exception( + ErrorCodes::ILLEGAL_AGGREGATION, + "Filter clause provided for an aggregating function (" + function.name + ") already containing If suffix"); + } + if (condition->children.empty()) + { + throw Exception(ErrorCodes::SYNTAX_ERROR, "Empty condition for WHERE"); + } + + function.name += "If"; + function.arguments->children.push_back(condition->children[0]); + return true; +} + bool ParserWindowReference::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { assert(node); diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index c4ddb056a4d..7d1cfc196d6 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -171,6 +171,13 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +// Allows to make queries like SELECT SUM() FILTER(WHERE ) FROM ... +class ParserFilterClause : public IParserBase +{ + const char * getName() const override { return "filter"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + // Window reference (the thing that goes after OVER) for window function. // Can be either window name or window definition. class ParserWindowReference : public IParserBase diff --git a/tests/queries/0_stateless/02000_select_with_filter.reference b/tests/queries/0_stateless/02000_select_with_filter.reference new file mode 100644 index 00000000000..9d104af5e8c --- /dev/null +++ b/tests/queries/0_stateless/02000_select_with_filter.reference @@ -0,0 +1,2 @@ +98 +2450 diff --git a/tests/queries/0_stateless/02000_select_with_filter.sql b/tests/queries/0_stateless/02000_select_with_filter.sql new file mode 100644 index 00000000000..eb83b6478a1 --- /dev/null +++ b/tests/queries/0_stateless/02000_select_with_filter.sql @@ -0,0 +1,4 @@ +SELECT argMax(number, number + 1) FILTER(WHERE number != 99) FROM numbers(100) ; +SELECT sum(number) FILTER(WHERE number % 2 == 0) FROM numbers(100); +SELECT sumIfOrNull(number, number % 2 == 1) FILTER(WHERE number % 2 == 0) FROM numbers(100); -- { clientError 184 } +SELECT sum(number) FILTER(WHERE) FROM numbers(100); -- { clientError 62 } From be8d87029c6204f49e13d21bffad80b08de41b8a Mon Sep 17 00:00:00 2001 From: adevyatova Date: Sun, 15 Aug 2021 13:53:49 +0000 Subject: [PATCH 037/224] Add templates --- docs/en/engines/database-engines/index.md | 2 + docs/en/engines/database-engines/sqlite.md | 63 +++++++++++++++++++ .../table-engines/integrations/index.md | 1 + .../table-engines/integrations/sqlite.md | 63 +++++++++++++++++++ .../en/sql-reference/table-functions/index.md | 1 + .../sql-reference/table-functions/sqlite.md | 51 +++++++++++++++ 6 files changed, 181 insertions(+) create mode 100644 docs/en/engines/database-engines/sqlite.md create mode 100644 docs/en/engines/table-engines/integrations/sqlite.md create mode 100644 docs/en/sql-reference/table-functions/sqlite.md diff --git a/docs/en/engines/database-engines/index.md b/docs/en/engines/database-engines/index.md index 1d1028cbeb5..dd8959d2700 100644 --- a/docs/en/engines/database-engines/index.md +++ b/docs/en/engines/database-engines/index.md @@ -23,3 +23,5 @@ You can also use the following database engines: - [PostgreSQL](../../engines/database-engines/postgresql.md) - [Replicated](../../engines/database-engines/replicated.md) + +- [SQLite](../../engines/database-engines/sqlite.md) diff --git a/docs/en/engines/database-engines/sqlite.md b/docs/en/engines/database-engines/sqlite.md new file mode 100644 index 00000000000..5ebab54580c --- /dev/null +++ b/docs/en/engines/database-engines/sqlite.md @@ -0,0 +1,63 @@ +# EngineName {#enginename} + +- What the Database/Table engine does. +- Relations with other engines if they exist. + +## Creating a Database {#creating-a-database} +``` sql + CREATE DATABASE ... +``` +or + +## Creating a Table {#creating-a-table} +``` sql + CREATE TABLE ... +``` + +**Engine Parameters** + +**Query Clauses** (for Table engines only) + +## Virtual columns {#virtual-columns} (for Table engines only) + +List and virtual columns with description, if they exist. + +## Data Types Support {#data_types-support} (for Database engines only) + +| EngineName | ClickHouse | +|-----------------------|------------------------------------| +| NativeDataTypeName | [ClickHouseDataTypeName](link#) | + + +## Specifics and recommendations {#specifics-and-recommendations} + +Algorithms +Specifics of read and write processes +Examples of tasks +Recommendations for usage +Specifics of data storage + +## Usage Example {#usage-example} + +The example must show usage and use cases. The following text contains the recommended parts of this section. + +Input table: + +``` text +``` + +Query: + +``` sql +``` + +Result: + +``` text +``` + +Follow up with any text to clarify the example. + +**See Also** + +- [link](#) \ No newline at end of file diff --git a/docs/en/engines/table-engines/integrations/index.md b/docs/en/engines/table-engines/integrations/index.md index eb1c5411e18..743d25ad616 100644 --- a/docs/en/engines/table-engines/integrations/index.md +++ b/docs/en/engines/table-engines/integrations/index.md @@ -19,3 +19,4 @@ List of supported integrations: - [EmbeddedRocksDB](../../../engines/table-engines/integrations/embedded-rocksdb.md) - [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) - [PostgreSQL](../../../engines/table-engines/integrations/postgresql.md) +- [SQLite](../../../engines/table-engines/integrations/sqlite.md) diff --git a/docs/en/engines/table-engines/integrations/sqlite.md b/docs/en/engines/table-engines/integrations/sqlite.md new file mode 100644 index 00000000000..5ebab54580c --- /dev/null +++ b/docs/en/engines/table-engines/integrations/sqlite.md @@ -0,0 +1,63 @@ +# EngineName {#enginename} + +- What the Database/Table engine does. +- Relations with other engines if they exist. + +## Creating a Database {#creating-a-database} +``` sql + CREATE DATABASE ... +``` +or + +## Creating a Table {#creating-a-table} +``` sql + CREATE TABLE ... +``` + +**Engine Parameters** + +**Query Clauses** (for Table engines only) + +## Virtual columns {#virtual-columns} (for Table engines only) + +List and virtual columns with description, if they exist. + +## Data Types Support {#data_types-support} (for Database engines only) + +| EngineName | ClickHouse | +|-----------------------|------------------------------------| +| NativeDataTypeName | [ClickHouseDataTypeName](link#) | + + +## Specifics and recommendations {#specifics-and-recommendations} + +Algorithms +Specifics of read and write processes +Examples of tasks +Recommendations for usage +Specifics of data storage + +## Usage Example {#usage-example} + +The example must show usage and use cases. The following text contains the recommended parts of this section. + +Input table: + +``` text +``` + +Query: + +``` sql +``` + +Result: + +``` text +``` + +Follow up with any text to clarify the example. + +**See Also** + +- [link](#) \ No newline at end of file diff --git a/docs/en/sql-reference/table-functions/index.md b/docs/en/sql-reference/table-functions/index.md index d65a18ab985..5d4e36ece6f 100644 --- a/docs/en/sql-reference/table-functions/index.md +++ b/docs/en/sql-reference/table-functions/index.md @@ -34,5 +34,6 @@ You can use table functions in: | [odbc](../../sql-reference/table-functions/odbc.md) | Creates a [ODBC](../../engines/table-engines/integrations/odbc.md)-engine table. | | [hdfs](../../sql-reference/table-functions/hdfs.md) | Creates a [HDFS](../../engines/table-engines/integrations/hdfs.md)-engine table. | | [s3](../../sql-reference/table-functions/s3.md) | Creates a [S3](../../engines/table-engines/integrations/s3.md)-engine table. | +| [sqlite](../../sql-reference/table-functions/sqlite.md) | Creates a [sqlite](../../engines/table-engines/integrations/sqlite.md)-engine table. | [Original article](https://clickhouse.tech/docs/en/sql-reference/table-functions/) diff --git a/docs/en/sql-reference/table-functions/sqlite.md b/docs/en/sql-reference/table-functions/sqlite.md new file mode 100644 index 00000000000..6b9ee2d9223 --- /dev/null +++ b/docs/en/sql-reference/table-functions/sqlite.md @@ -0,0 +1,51 @@ +## functionName {#functionname-in-lower-case} + +Short description. + +**Syntax** (without SELECT) + +``` sql + +``` + +Alias: ``. (Optional) + +More text (Optional). + +**Arguments** (Optional) + +- `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). +- `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). + +**Parameters** (Optional, only for parametric aggregate functions) + +- `z` — Description. Optional (only for optional parameters). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). + +**Returned value(s)** + +- Returned values list. + +Type: [Type name](relative/path/to/type/dscr.md#type). + +**Example** + +The example must show usage and/or a use cases. The following text contains recommended parts of an example. + +Input table (Optional): + +``` text +``` + +Query: + +``` sql +``` + +Result: + +``` text +``` + +**See Also** (Optional) + +- [link](#) \ No newline at end of file From 346def3666053fb324653071fbce853a87b65c4b Mon Sep 17 00:00:00 2001 From: adevyatova Date: Sun, 15 Aug 2021 14:11:37 +0000 Subject: [PATCH 038/224] Update --- docs/en/engines/database-engines/sqlite.md | 19 +++++---------- .../table-engines/integrations/sqlite.md | 24 +++++++------------ .../sql-reference/table-functions/sqlite.md | 13 +++++----- 3 files changed, 21 insertions(+), 35 deletions(-) diff --git a/docs/en/engines/database-engines/sqlite.md b/docs/en/engines/database-engines/sqlite.md index 5ebab54580c..cfe58288d46 100644 --- a/docs/en/engines/database-engines/sqlite.md +++ b/docs/en/engines/database-engines/sqlite.md @@ -1,4 +1,9 @@ -# EngineName {#enginename} +--- +toc_priority: 32 +toc_title: SQLite +--- + +# SQLite {#sqlite} - What the Database/Table engine does. - Relations with other engines if they exist. @@ -7,21 +12,9 @@ ``` sql CREATE DATABASE ... ``` -or - -## Creating a Table {#creating-a-table} -``` sql - CREATE TABLE ... -``` **Engine Parameters** -**Query Clauses** (for Table engines only) - -## Virtual columns {#virtual-columns} (for Table engines only) - -List and virtual columns with description, if they exist. - ## Data Types Support {#data_types-support} (for Database engines only) | EngineName | ClickHouse | diff --git a/docs/en/engines/table-engines/integrations/sqlite.md b/docs/en/engines/table-engines/integrations/sqlite.md index 5ebab54580c..0360b2717c8 100644 --- a/docs/en/engines/table-engines/integrations/sqlite.md +++ b/docs/en/engines/table-engines/integrations/sqlite.md @@ -1,14 +1,13 @@ -# EngineName {#enginename} +--- +toc_priority: 7 +toc_title: SQLite +--- + +# SQLite {#sqlite} - What the Database/Table engine does. - Relations with other engines if they exist. -## Creating a Database {#creating-a-database} -``` sql - CREATE DATABASE ... -``` -or - ## Creating a Table {#creating-a-table} ``` sql CREATE TABLE ... @@ -16,19 +15,12 @@ or **Engine Parameters** -**Query Clauses** (for Table engines only) +**Query Clauses** -## Virtual columns {#virtual-columns} (for Table engines only) +## Virtual columns {#virtual-columns} List and virtual columns with description, if they exist. -## Data Types Support {#data_types-support} (for Database engines only) - -| EngineName | ClickHouse | -|-----------------------|------------------------------------| -| NativeDataTypeName | [ClickHouseDataTypeName](link#) | - - ## Specifics and recommendations {#specifics-and-recommendations} Algorithms diff --git a/docs/en/sql-reference/table-functions/sqlite.md b/docs/en/sql-reference/table-functions/sqlite.md index 6b9ee2d9223..28cc03071c6 100644 --- a/docs/en/sql-reference/table-functions/sqlite.md +++ b/docs/en/sql-reference/table-functions/sqlite.md @@ -1,8 +1,13 @@ -## functionName {#functionname-in-lower-case} +--- +toc_priority: 55 +toc_title: sqlite +--- + +## sqlite {#sqlite} Short description. -**Syntax** (without SELECT) +**Syntax** ``` sql @@ -17,10 +22,6 @@ More text (Optional). - `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). - `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). -**Parameters** (Optional, only for parametric aggregate functions) - -- `z` — Description. Optional (only for optional parameters). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). - **Returned value(s)** - Returned values list. From 6177a9c454fb1aae1ee9f86c7a6ad192d13e112a Mon Sep 17 00:00:00 2001 From: Alexey Date: Sun, 15 Aug 2021 15:36:24 +0000 Subject: [PATCH 039/224] en draft --- .../sql-reference/statements/select/join.md | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 0002e6db313..e3621db2b24 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -52,6 +52,40 @@ The behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_ - [join_on_disk_max_files_to_merge](../../../operations/settings/settings.md#join_on_disk_max_files_to_merge) - [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) +## Conditions in ON Section {conditions-in-on-section} + +In addition to join keys an `ON` section can contain conditions concatenated by `AND` and `OR`. Any condition can be applied either to the left or to the right table of a query. Rows are joined if the whole complex condition is met including matching join keys. +If the condition is not met, still rows may be included in the result depending on the `JOIN` type. If the same conditions are placed in a `WHERE` section and they are not met, then rows are always filtered out from the result. + +**Example** + +Consider `table_1` and `table_2`: + +``` +┌─Id─┬─name─┐ ┌─Id─┬─text───────────┐ +│ 1 │ A │ │ 1 │ Text A │ +│ 2 │ B │ │ 1 │ Another text A │ +│ 3 │ C │ │ 2 │ Text B │ +└────┴──────┘ └────┴────────────────┘ +``` + +Query: + +``` sql +SELECT name, text FROM table_1 LEFT JOIN table_2 + ON table_1.Id = table_2.Id AND startsWith(table_2.text, 'Text'); +``` + +Note that the result contains the row with the name `C` and an empty text column. It is there to satisfy the `OUTER` type of a join. + +``` +┌─name─┬─text───┐ +│ A │ Text A │ +│ B │ Text B │ +│ C │ │ +└──────┴────────┘ +``` + ## ASOF JOIN Usage {#asof-join-usage} `ASOF JOIN` is useful when you need to join records that have no exact match. From a54060dd023a460f4e1483d8648f4ebfe48aeac2 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Sun, 15 Aug 2021 16:34:10 +0000 Subject: [PATCH 040/224] Add en description --- docs/en/engines/database-engines/sqlite.md | 22 +++++++---------- .../table-engines/integrations/sqlite.md | 23 +++++++++++------- .../sql-reference/table-functions/sqlite.md | 24 +++++++------------ 3 files changed, 32 insertions(+), 37 deletions(-) diff --git a/docs/en/engines/database-engines/sqlite.md b/docs/en/engines/database-engines/sqlite.md index cfe58288d46..3ecbfc7fc07 100644 --- a/docs/en/engines/database-engines/sqlite.md +++ b/docs/en/engines/database-engines/sqlite.md @@ -5,17 +5,20 @@ toc_title: SQLite # SQLite {#sqlite} -- What the Database/Table engine does. -- Relations with other engines if they exist. +The engine works with [SQLite](https://www.sqlite.org/index.html). ## Creating a Database {#creating-a-database} + ``` sql - CREATE DATABASE ... + CREATE DATABASE sqlite_database + ENGINE = SQLite('db_path') ``` **Engine Parameters** -## Data Types Support {#data_types-support} (for Database engines only) +- `db_path` — Path to SQLite file with the database. + +## Data Types Support {#data_types-support} | EngineName | ClickHouse | |-----------------------|------------------------------------| @@ -24,11 +27,8 @@ toc_title: SQLite ## Specifics and recommendations {#specifics-and-recommendations} -Algorithms -Specifics of read and write processes -Examples of tasks -Recommendations for usage -Specifics of data storage +SQLite stores the entire database (definitions, tables, indices, and the data itself) as a single cross-platform file on a host machine. It is locking the entire database file during writing. SQLite read operations can be multitasked, though writes can only be performed sequentially. +SQLite does not require service management (such as startup scripts) or access control based on `GRANT` and passwords. Access control is handled by means of file-system permissions given to the database file itself. ## Usage Example {#usage-example} @@ -50,7 +50,3 @@ Result: ``` Follow up with any text to clarify the example. - -**See Also** - -- [link](#) \ No newline at end of file diff --git a/docs/en/engines/table-engines/integrations/sqlite.md b/docs/en/engines/table-engines/integrations/sqlite.md index 0360b2717c8..6808b4f3a8e 100644 --- a/docs/en/engines/table-engines/integrations/sqlite.md +++ b/docs/en/engines/table-engines/integrations/sqlite.md @@ -5,21 +5,26 @@ toc_title: SQLite # SQLite {#sqlite} -- What the Database/Table engine does. -- Relations with other engines if they exist. +The engine provide to import and export data to SQLite and query SQLite tables directly in ClickHouse. ## Creating a Table {#creating-a-table} + ``` sql - CREATE TABLE ... + CREATE TABLE [IF NOT EXISTS] [db.]table_name + ( + name1 [type1], + name2 [type2], + ... + ) + ENGINE = SQLite('db_path', 'table') ``` **Engine Parameters** -**Query Clauses** - -## Virtual columns {#virtual-columns} - -List and virtual columns with description, if they exist. +- `name1, name2, ...` — The column names. +- `type1, type2, ...` — The column types. +- `db_path` — Path to SQLite file with the database. +- `table` — The SQLite table name. ## Specifics and recommendations {#specifics-and-recommendations} @@ -52,4 +57,4 @@ Follow up with any text to clarify the example. **See Also** -- [link](#) \ No newline at end of file +- [The `sqlite` table function](../../../sql-reference/table-functions/sqlite.md) \ No newline at end of file diff --git a/docs/en/sql-reference/table-functions/sqlite.md b/docs/en/sql-reference/table-functions/sqlite.md index 28cc03071c6..972c16f4dd6 100644 --- a/docs/en/sql-reference/table-functions/sqlite.md +++ b/docs/en/sql-reference/table-functions/sqlite.md @@ -5,28 +5,22 @@ toc_title: sqlite ## sqlite {#sqlite} -Short description. +Allows to performed queries on data that is stored in the `SQLite` database. **Syntax** ``` sql - + sqlite('db_path', 'table_name') ``` -Alias: ``. (Optional) +**Arguments** -More text (Optional). +- `db_path` — Path to SQLite file with the database. [String](../../sql-reference/data-types/string.md). +- `table_name` — The SQLite table name. [String](../../sql-reference/data-types/string.md). -**Arguments** (Optional) +**Returned value** -- `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). -- `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). - -**Returned value(s)** - -- Returned values list. - -Type: [Type name](relative/path/to/type/dscr.md#type). +- A table object with the same columns as the original `SQLite` table. **Example** @@ -47,6 +41,6 @@ Result: ``` text ``` -**See Also** (Optional) +**See Also** -- [link](#) \ No newline at end of file +- [`SQLite` table engine](../../engines/table-engines/integrations/sqlite.md) \ No newline at end of file From c667a94978f52f2753c84408294b07d103394fd7 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sun, 15 Aug 2021 19:33:04 +0000 Subject: [PATCH 041/224] Several format fixes in other places --- docs/en/sql-reference/statements/select/join.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index e3621db2b24..8694916a599 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -6,7 +6,7 @@ toc_title: JOIN Join produces a new table by combining columns from one or multiple tables by using values common to each. It is a common operation in databases with SQL support, which corresponds to [relational algebra](https://en.wikipedia.org/wiki/Relational_algebra#Joins_and_join-like_operators) join. The special case of one table join is often referred to as “self-join”. -Syntax: +**Syntax** ``` sql SELECT @@ -93,7 +93,7 @@ Note that the result contains the row with the name `C` and an empty text column Algorithm requires the special column in tables. This column: - Must contain an ordered sequence. -- Can be one of the following types: [Int*, UInt*](../../../sql-reference/data-types/int-uint.md), [Float\*](../../../sql-reference/data-types/float.md), [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md), [Decimal\*](../../../sql-reference/data-types/decimal.md). +- Can be one of the following types: [Int\*, UInt\*](../../../sql-reference/data-types/int-uint.md), [Float\*](../../../sql-reference/data-types/float.md), [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md), [Decimal\*](../../../sql-reference/data-types/decimal.md). - Can’t be the only column in the `JOIN` clause. Syntax `ASOF JOIN ... ON`: @@ -118,7 +118,7 @@ ASOF JOIN table_2 USING (equi_column1, ... equi_columnN, asof_column) ``` -`ASOF JOIN` uses `equi_columnX` for joining on equality and `asof_column` for joining on the closest match with the `table_1.asof_column >= table_2.asof_column` condition. The `asof_column` column always the last one in the `USING` clause. +`ASOF JOIN` uses `equi_columnX` for joining on equality and `asof_column` for joining on the closest match with the `table_1.asof_column >= table_2.asof_column` condition. The `asof_column` column is always the last one in the `USING` clause. For example, consider the following tables: From 3b13b79d650f9a48dde59c5325d84f4df0468327 Mon Sep 17 00:00:00 2001 From: Nickita Taranov Date: Sun, 8 Aug 2021 22:31:48 +0300 Subject: [PATCH 042/224] throw in AggregateFunctionCombinatorIf instead --- src/AggregateFunctions/AggregateFunctionIf.cpp | 7 ++++++- src/Parsers/ExpressionElementParsers.cpp | 18 +++--------------- .../00545_weird_aggregate_functions.sql | 2 +- .../0_stateless/02000_select_with_filter.sql | 3 +-- 4 files changed, 11 insertions(+), 19 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index d841fe8c06d..89688ce1ffd 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -1,5 +1,5 @@ -#include #include +#include #include "AggregateFunctionNull.h" @@ -11,6 +11,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_AGGREGATION; } class AggregateFunctionCombinatorIf final : public IAggregateFunctionCombinator @@ -37,6 +38,10 @@ public: const DataTypes & arguments, const Array & params) const override { + if (nested_function->getName().find(getName()) != String::npos) + { + throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, "nested function for {0}-combinator must not have {0}-combinator", getName()); + } return std::make_shared(nested_function, arguments, params); } }; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index dd9fc738094..5190e805922 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -49,7 +49,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int SYNTAX_ERROR; extern const int LOGICAL_ERROR; - extern const int ILLEGAL_AGGREGATION; } @@ -521,8 +520,8 @@ bool ParserFilterClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expected assert(node); ASTFunction & function = dynamic_cast(*node); - ParserToken parser_openging_bracket(TokenType::OpeningRoundBracket); - if (!parser_openging_bracket.ignore(pos, expected)) + ParserToken parser_opening_bracket(TokenType::OpeningRoundBracket); + if (!parser_opening_bracket.ignore(pos, expected)) { return false; } @@ -534,7 +533,7 @@ bool ParserFilterClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } ParserExpressionList parser_condition(false); ASTPtr condition; - if (!parser_condition.parse(pos, condition, expected)) + if (!parser_condition.parse(pos, condition, expected) || condition->children.size() != 1) { return false; } @@ -545,17 +544,6 @@ bool ParserFilterClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; } - if (function.name.find("If") != String::npos) - { - throw Exception( - ErrorCodes::ILLEGAL_AGGREGATION, - "Filter clause provided for an aggregating function (" + function.name + ") already containing If suffix"); - } - if (condition->children.empty()) - { - throw Exception(ErrorCodes::SYNTAX_ERROR, "Empty condition for WHERE"); - } - function.name += "If"; function.arguments->children.push_back(condition->children[0]); return true; diff --git a/tests/queries/0_stateless/00545_weird_aggregate_functions.sql b/tests/queries/0_stateless/00545_weird_aggregate_functions.sql index 1f662850d05..c728dfcc534 100644 --- a/tests/queries/0_stateless/00545_weird_aggregate_functions.sql +++ b/tests/queries/0_stateless/00545_weird_aggregate_functions.sql @@ -1 +1 @@ -SELECT sumForEachMergeArray(y) FROM (SELECT sumForEachStateForEachIfArrayIfMerge(x) AS y FROM (SELECT sumForEachStateForEachIfArrayIfState([[[1, 2, 3], [4, 5, 6], [7, 8, 9]]], [1], 1) AS x)); +SELECT sumForEachMergeArray(y) FROM (SELECT sumForEachStateForEachIfArrayMerge(x) AS y FROM (SELECT sumForEachStateForEachIfArrayState([[[1, 2, 3], [4, 5, 6], [7, 8, 9]]], [1]) AS x)); diff --git a/tests/queries/0_stateless/02000_select_with_filter.sql b/tests/queries/0_stateless/02000_select_with_filter.sql index eb83b6478a1..4d10f86ed96 100644 --- a/tests/queries/0_stateless/02000_select_with_filter.sql +++ b/tests/queries/0_stateless/02000_select_with_filter.sql @@ -1,4 +1,3 @@ SELECT argMax(number, number + 1) FILTER(WHERE number != 99) FROM numbers(100) ; SELECT sum(number) FILTER(WHERE number % 2 == 0) FROM numbers(100); -SELECT sumIfOrNull(number, number % 2 == 1) FILTER(WHERE number % 2 == 0) FROM numbers(100); -- { clientError 184 } -SELECT sum(number) FILTER(WHERE) FROM numbers(100); -- { clientError 62 } +SELECT sumIfOrNull(number, number % 2 == 1) FILTER(WHERE number % 2 == 0) FROM numbers(100); -- { serverError 184 } From 48d590627aabea885faa074041c4cc4911aae085 Mon Sep 17 00:00:00 2001 From: RedClusive Date: Mon, 16 Aug 2021 19:08:17 +0000 Subject: [PATCH 043/224] fix weighted --- src/AggregateFunctions/AggregateFunctionQuantile.cpp | 6 ++++++ src/AggregateFunctions/AggregateFunctionQuantile.h | 2 ++ 2 files changed, 8 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.cpp b/src/AggregateFunctions/AggregateFunctionQuantile.cpp index 62c8a41b614..788f27ab254 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantile.cpp @@ -55,6 +55,8 @@ template using FuncQuantilesTDigestWeighted template using FuncQuantileBFloat16 = AggregateFunctionQuantile, NameQuantileBFloat16, false, std::conditional_t, false>; template using FuncQuantilesBFloat16 = AggregateFunctionQuantile, NameQuantilesBFloat16, false, std::conditional_t, true>; +template using FuncQuantileBFloat16Weighted = AggregateFunctionQuantile, NameQuantileBFloat16Weighted, true, std::conditional_t, false>; +template using FuncQuantilesBFloat16Weighted = AggregateFunctionQuantile, NameQuantilesBFloat16Weighted, true, std::conditional_t, true>; template