SQLUserDefinedFunctions support CREATE OR REPLACE, CREATE IF NOT EXISTS

This commit is contained in:
Maksim Kita 2021-10-20 16:04:02 +03:00
parent 73e6110b2a
commit ff48017f4a
13 changed files with 102 additions and 19 deletions

View File

@ -31,20 +31,32 @@ BlockIO InterpreterCreateFunctionQuery::execute()
if (!create_function_query)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected CREATE FUNCTION query");
auto & user_defined_function_factory = UserDefinedSQLFunctionFactory::instance();
auto & function_name = create_function_query->function_name;
bool if_not_exists = create_function_query->if_not_exists;
bool replace = create_function_query->or_replace;
create_function_query->if_not_exists = false;
create_function_query->or_replace = false;
if (if_not_exists && user_defined_function_factory.tryGet(function_name) != nullptr)
return {};
validateFunction(create_function_query->function_core, function_name);
UserDefinedSQLFunctionFactory::instance().registerFunction(function_name, query_ptr);
user_defined_function_factory.registerFunction(function_name, query_ptr, replace);
if (!persist_function)
if (persist_function)
{
try
{
UserDefinedSQLObjectsLoader::instance().storeObject(current_context, UserDefinedSQLObjectType::Function, function_name, *query_ptr);
UserDefinedSQLObjectsLoader::instance().storeObject(current_context, UserDefinedSQLObjectType::Function, function_name, *query_ptr, replace);
}
catch (Exception & exception)
{
UserDefinedSQLFunctionFactory::instance().unregisterFunction(function_name);
user_defined_function_factory.unregisterFunction(function_name);
exception.addMessage(fmt::format("while storing user defined function {} on disk", backQuote(function_name)));
throw;
}

View File

@ -278,7 +278,7 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
}
else if (query->as<ASTCreateFunctionQuery>())
{
return std::make_unique<InterpreterCreateFunctionQuery>(query, context, false /*is_internal*/);
return std::make_unique<InterpreterCreateFunctionQuery>(query, context, true /*persist_function*/);
}
else if (query->as<ASTDropFunctionQuery>())
{

View File

@ -19,7 +19,7 @@ UserDefinedSQLFunctionFactory & UserDefinedSQLFunctionFactory::instance()
return result;
}
void UserDefinedSQLFunctionFactory::registerFunction(const String & function_name, ASTPtr create_function_query)
void UserDefinedSQLFunctionFactory::registerFunction(const String & function_name, ASTPtr create_function_query, bool replace)
{
if (FunctionFactory::instance().hasNameOrAlias(function_name))
throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "The function '{}' already exists", function_name);
@ -29,11 +29,17 @@ void UserDefinedSQLFunctionFactory::registerFunction(const String & function_nam
std::lock_guard lock(mutex);
auto [_, inserted] = function_name_to_create_query.emplace(function_name, std::move(create_function_query));
auto [it, inserted] = function_name_to_create_query.emplace(function_name, create_function_query);
if (!inserted)
throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS,
"The function name '{}' is not unique",
function_name);
{
if (replace)
it->second = std::move(create_function_query);
else
throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS,
"The function name '{}' is not unique",
function_name);
}
}
void UserDefinedSQLFunctionFactory::unregisterFunction(const String & function_name)

View File

@ -10,21 +10,31 @@
namespace DB
{
/// Factory for SQLUserDefinedFunctions
class UserDefinedSQLFunctionFactory : public IHints<1, UserDefinedSQLFunctionFactory>
{
public:
static UserDefinedSQLFunctionFactory & instance();
void registerFunction(const String & function_name, ASTPtr create_function_query);
/** Register function for function_name in factory for specified create_function_query.
* If replace = true and function with function_name already exists replace it with create_function_query.
* Otherwise throws exception.
*/
void registerFunction(const String & function_name, ASTPtr create_function_query, bool replace);
/// Unregister function for function_name
void unregisterFunction(const String & function_name);
/// Get function create query for function_name. If no function registered with function_name throws exception.
ASTPtr get(const String & function_name) const;
/// Get function create query for function_name. If no function registered with function_name return nullptr.
ASTPtr tryGet(const String & function_name) const;
/// Check if function with function_name registered.
bool has(const String & function_name) const;
/// Get all user defined functions registered names.
std::vector<String> getAllRegisteredNames() const override;
private:

View File

@ -69,7 +69,7 @@ void UserDefinedSQLObjectsLoader::loadUserDefinedObject(ContextPtr context, User
0,
context->getSettingsRef().max_parser_depth);
InterpreterCreateFunctionQuery interpreter(ast, context, true /*is internal*/);
InterpreterCreateFunctionQuery interpreter(ast, context, false /*persist_function*/);
interpreter.execute();
}
}
@ -111,7 +111,7 @@ void UserDefinedSQLObjectsLoader::loadObjects(ContextPtr context)
}
}
void UserDefinedSQLObjectsLoader::storeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name, const IAST & ast)
void UserDefinedSQLObjectsLoader::storeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name, const IAST & ast, bool replace)
{
if (unlikely(!enable_persistence))
return;
@ -127,7 +127,7 @@ void UserDefinedSQLObjectsLoader::storeObject(ContextPtr context, UserDefinedSQL
}
}
if (std::filesystem::exists(file_path))
if (!replace && std::filesystem::exists(file_path))
throw Exception(ErrorCodes::OBJECT_ALREADY_STORED_ON_DISK, "User defined object {} already stored on disk", backQuote(file_path));
LOG_DEBUG(log, "Storing object {} to file {}", backQuote(object_name), file_path);
@ -135,9 +135,9 @@ void UserDefinedSQLObjectsLoader::storeObject(ContextPtr context, UserDefinedSQL
WriteBufferFromOwnString create_statement_buf;
formatAST(ast, create_statement_buf, false);
writeChar('\n', create_statement_buf);
String create_statement = create_statement_buf.str();
WriteBufferFromFile out(file_path, create_statement.size(), O_WRONLY | O_CREAT | O_EXCL);
WriteBufferFromFile out(file_path, create_statement.size());
writeString(create_statement, out);
out.next();
if (context->getSettingsRef().fsync_metadata)

View File

@ -21,7 +21,7 @@ public:
UserDefinedSQLObjectsLoader();
void loadObjects(ContextPtr context);
void storeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name, const IAST & ast);
void storeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name, const IAST & ast, bool replace);
void removeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name);
/// For ClickHouse local if path is not set we can disable loader.

View File

@ -12,7 +12,18 @@ ASTPtr ASTCreateFunctionQuery::clone() const
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_keyword : "") << "CREATE ";
if (or_replace)
settings.ostr << "OR REPLACE ";
settings.ostr << "FUNCTION ";
if (if_not_exists)
settings.ostr << "IF NOT EXISTS ";
settings.ostr << (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);

View File

@ -12,6 +12,9 @@ public:
String function_name;
ASTPtr function_core;
bool or_replace = false;
bool if_not_exists = false;
String getID(char) const override { return "CreateFunctionQuery"; }
ASTPtr clone() const override;

View File

@ -1,10 +1,12 @@
#include <Parsers/ParserCreateFunctionQuery.h>
#include <Parsers/ASTCreateFunctionQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserCreateFunctionQuery.h>
namespace DB
{
@ -13,6 +15,8 @@ bool ParserCreateFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp
{
ParserKeyword s_create("CREATE");
ParserKeyword s_function("FUNCTION");
ParserKeyword s_or_replace("OR REPLACE");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserIdentifier function_name_p;
ParserKeyword s_as("AS");
ParserLambdaExpression lambda_p;
@ -20,12 +24,21 @@ bool ParserCreateFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp
ASTPtr function_name;
ASTPtr function_core;
bool or_replace = false;
bool if_not_exists = false;
if (!s_create.ignore(pos, expected))
return false;
if (s_or_replace.ignore(pos, expected))
or_replace = true;
if (!s_function.ignore(pos, expected))
return false;
if (!or_replace && s_if_not_exists.ignore(pos, expected))
if_not_exists = true;
if (!function_name_p.parse(pos, function_name, expected))
return false;
@ -40,6 +53,8 @@ bool ParserCreateFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp
create_function_query->function_name = function_name->as<ASTIdentifier &>().name();
create_function_query->function_core = function_core;
create_function_query->or_replace = or_replace;
create_function_query->if_not_exists = if_not_exists;
return true;
}

View File

@ -0,0 +1,4 @@
CREATE FUNCTION `02101_test_function` AS x -> (x + 1)
2
CREATE FUNCTION `02101_test_function` AS x -> (x + 2)
3

View File

@ -0,0 +1,13 @@
-- Tags: no-parallel
CREATE OR REPLACE FUNCTION 02101_test_function AS x -> x + 1;
SELECT create_query FROM system.functions WHERE name = '02101_test_function';
SELECT 02101_test_function(1);
CREATE OR REPLACE FUNCTION 02101_test_function AS x -> x + 2;
SELECT create_query FROM system.functions WHERE name = '02101_test_function';
SELECT 02101_test_function(1);
DROP FUNCTION 02101_test_function;

View File

@ -0,0 +1,8 @@
-- Tags: no-parallel
CREATE FUNCTION IF NOT EXISTS 02102_test_function AS x -> x + 1;
SELECT 02102_test_function(1);
CREATE FUNCTION 02102_test_function AS x -> x + 1; --{serverError 609}
CREATE FUNCTION IF NOT EXISTS 02102_test_function AS x -> x + 1;
DROP FUNCTION 02102_test_function;