Fixed integration tests

This commit is contained in:
Maksim Kita 2021-08-23 17:31:58 +03:00
parent 2a6aa50d49
commit 010d72593f
15 changed files with 36 additions and 54 deletions

View File

@ -15,8 +15,6 @@
namespace DB
{
class UserDefinedFunction;
namespace ErrorCodes
{
extern const int UNKNOWN_FUNCTION;

View File

@ -4,11 +4,9 @@
#include <Common/IFactoryWithAliases.h>
#include <Functions/IFunction.h>
#include <Functions/IFunctionAdaptors.h>
#include <Parsers/ASTCreateFunctionQuery.h>
#include <functional>
#include <memory>
#include <mutex>
#include <string>
#include <unordered_map>
@ -42,10 +40,6 @@ public:
registerFunction(name, &Function::create, case_sensitiveness);
}
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<std::string> getAllNames() const;

View File

@ -1,4 +1,6 @@
#include <Access/ContextAccess.h>
#include <Parsers/ASTCreateFunctionQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
@ -6,7 +8,7 @@
#include <Interpreters/FunctionNameNormalizer.h>
#include <Interpreters/UserDefinedObjectsLoader.h>
#include <Interpreters/UserDefinedFunctionFactory.h>
#include <Parsers/ASTIdentifier.h>
namespace DB
{
@ -20,7 +22,9 @@ namespace ErrorCodes
BlockIO InterpreterCreateFunctionQuery::execute()
{
getContext()->checkAccess(AccessType::CREATE_FUNCTION);
auto context = getContext();
context->checkAccess(AccessType::CREATE_FUNCTION);
FunctionNameNormalizer().visit(query_ptr.get());
auto * create_function_query = query_ptr->as<ASTCreateFunctionQuery>();
@ -34,15 +38,14 @@ BlockIO InterpreterCreateFunctionQuery::execute()
if (!is_internal)
{
try
{
UserDefinedObjectsLoader::instance().storeObject(getContext(), UserDefinedObjectType::Function, function_name, *query_ptr);
UserDefinedObjectsLoader::instance().storeObject(context, UserDefinedObjectType::Function, function_name, *query_ptr);
}
catch (Exception & e)
catch (Exception & exception)
{
UserDefinedFunctionFactory::instance().unregisterFunction(function_name);
e.addMessage(fmt::format("while storing user defined function {} on disk", backQuote(function_name)));
exception.addMessage(fmt::format("while storing user defined function {} on disk", backQuote(function_name)));
throw;
}
}

View File

@ -1,12 +1,11 @@
#pragma once
#include <Interpreters/IInterpreter.h>
#include <Parsers/ASTCreateFunctionQuery.h>
namespace DB
{
class ASTCreateFunctionQuery;
class Context;
class InterpreterCreateFunctionQuery : public IInterpreter, WithContext

View File

@ -12,11 +12,14 @@ namespace DB
BlockIO InterpreterDropFunctionQuery::execute()
{
getContext()->checkAccess(AccessType::DROP_FUNCTION);
auto context = getContext();
context->checkAccess(AccessType::DROP_FUNCTION);
FunctionNameNormalizer().visit(query_ptr.get());
auto & drop_function_query = query_ptr->as<ASTDropFunctionQuery &>();
UserDefinedFunctionFactory::instance().unregisterFunction(drop_function_query.function_name);
UserDefinedObjectsLoader::instance().removeObject(getContext(), UserDefinedObjectType::Function, drop_function_query.function_name);
UserDefinedObjectsLoader::instance().removeObject(context, UserDefinedObjectType::Function, drop_function_query.function_name);
return {};
}

View File

@ -5,7 +5,6 @@
namespace DB
{
class ASTDropFunctionQuery;
class Context;
class InterpreterDropFunctionQuery : public IInterpreter, WithMutableContext

View File

@ -6,6 +6,7 @@
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTCreateSettingsProfileQuery.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTCreateFunctionQuery.h>
#include <Parsers/ASTDropAccessEntityQuery.h>
#include <Parsers/ASTDropFunctionQuery.h>
#include <Parsers/ASTDropQuery.h>

View File

@ -40,12 +40,13 @@ void UserDefinedFunctionFactory::unregisterFunction(const String & function_name
AggregateFunctionFactory::instance().hasNameOrAlias(function_name))
throw Exception(ErrorCodes::CANNOT_DROP_SYSTEM_FUNCTION, "Cannot drop system function '{}'", function_name);
auto it = function_name_to_create_query.find(function_name);
if (it == function_name_to_create_query.end())
throw Exception(ErrorCodes::UNKNOWN_FUNCTION,
"The function name '{}' is not registered",
function_name);
function_name_to_create_query.erase(it);
}
ASTPtr UserDefinedFunctionFactory::get(const String & function_name) const

View File

@ -14,16 +14,6 @@ def started_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_access_rights_for_funtion():
create_function_query = "CREATE FUNCTION MySum AS (a, b) -> a + b"
@ -44,3 +34,6 @@ def test_access_rights_for_funtion():
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')
instance.query("DROP USER IF EXISTS A")
instance.query("DROP USER IF EXISTS B")

View File

@ -1,4 +0,0 @@
create function MyFunc as (a, b, c) -> a * b * c;
select MyFunc(2, 3, 4);
select isConstant(MyFunc(1, 2, 3));
drop function MyFunc;

View File

@ -0,0 +1,13 @@
CREATE FUNCTION 01856_test_function_0 AS (a, b, c) -> a * b * c;
SELECT 01856_test_function_0(2, 3, 4);
SELECT isConstant(01856_test_function_0(1, 2, 3));
DROP FUNCTION 01856_test_function_0;
CREATE FUNCTION 01856_test_function_1 AS (a, b) -> a || b || c; --{serverError 47}
CREATE FUNCTION 01856_test_function_1 AS (a, b) -> 01856_test_function_1(a, b) + 01856_test_function_1(a, b); --{serverError 600}
CREATE FUNCTION cast AS a -> a + 1; --{serverError 598}
CREATE FUNCTION sum AS (a, b) -> a + b; --{serverError 598}
CREATE FUNCTION 01856_test_function_2 AS (a, b) -> a + b;
CREATE FUNCTION 01856_test_function_2 AS (a) -> a || '!!!'; --{serverError 598}
DROP FUNCTION 01856_test_function_2;
DROP FUNCTION unknown_function; -- {serverError 46}
DROP FUNCTION CAST; -- {serverError 599}

View File

@ -1,17 +0,0 @@
-- CREATE FUNCTION MyFunc2 AS (a, b) -> a || b || c; --{serverError 47}
-- CREATE FUNCTION MyFunc2 AS (a, b) -> MyFunc2(a, b) + MyFunc2(a, b); --{serverError 600}
-- CREATE FUNCTION cast AS a -> a + 1; --{serverError 598}
-- CREATE FUNCTION sum AS (a, b) -> a + b; --{serverError 598}
-- CREATE FUNCTION MyFunc3 AS (a, b) -> a + b;
-- CREATE FUNCTION MyFunc3 AS (a) -> a || '!!!'; --{serverError 598}
-- DROP FUNCTION MyFunc3;
-- DROP FUNCTION unknownFunc; -- {serverError 46}
DROP FUNCTION CAST; -- {serverError 599}

View File

@ -483,8 +483,7 @@
"01804_dictionary_decimal256_type",
"01850_dist_INSERT_preserve_error", // uses cluster with different static databases shard_0/shard_1
"01821_table_comment",
"01855_create_simple_function",
"01856_create_function_errors",
"01856_create_function",
"01857_create_function_and_check_jit_compiled",
"01824_prefer_global_in_and_join",
"01870_modulo_partition_key",