Skip to content

Commit

Permalink
Add syntax support for CREATE/DROP TEMPORARY FUNCTION
Browse files Browse the repository at this point in the history
  • Loading branch information
prithvip authored and caithagoras0 committed Jan 28, 2021
1 parent 5a294dc commit 574d21d
Show file tree
Hide file tree
Showing 13 changed files with 147 additions and 39 deletions.
19 changes: 16 additions & 3 deletions presto-docs/src/main/sphinx/sql/create-function.rst
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@ Synopsis

.. code-block:: none
CREATE [ OR REPLACE ] FUNCTION
CREATE [ OR REPLACE ] [TEMPORARY] FUNCTION
qualified_function_name (
parameter_name parameter_type
[, ...]
Expand All @@ -25,11 +25,18 @@ Description

Create a new function with the specified definition.

Each function is uniquely identified by its qualified function name
When ``TEMPORARY`` is specified, the created function is valid and visible
within the current session, but no persistent entry is made.

Each permanent function is uniquely identified by its qualified function name
and its parameter type list. ``qualified_function_name`` needs to be in
the format of ``catalog.schema.function_name``.

In order to create a function, the corresponding function namespace
Each temporary functions is uniquely identified by the function name.
The name cannot be qualified, or collide with the name of an existing built-in
function.

In order to create a permanent function, the corresponding function namespace
(in the format ``catalog.schema``) must first be managed by a function
namespace manager (See :doc:`/admin/function-namespace-managers`).

Expand Down Expand Up @@ -78,6 +85,12 @@ routine characteristics::
RETURNS NULL ON NULL INPUT
RETURN sin(x) / cos(x)

Create a new temporary function ``square``::

CREATE TEMPORARY FUNCTION square(x int)
RETURNS int
RETURN x * x

See Also
--------

Expand Down
4 changes: 3 additions & 1 deletion presto-docs/src/main/sphinx/sql/drop-function.rst
Original file line number Diff line number Diff line change
Expand Up @@ -7,14 +7,16 @@ Synopsis

.. code-block:: none
DROP FUNCTION [ IF EXISTS ] qualified_function_name [ ( parameter_type[, ...] ) ]
DROP [TEMPORARY] FUNCTION [ IF EXISTS ] qualified_function_name [ ( parameter_type[, ...] ) ]
Description
-----------

Drop an existing function.

When ``TEMPORARY`` is specified, a temporary function with the given signature is dropped.

The optional ``IF EXISTS`` clause causes the ``NOT_FOUND`` error
to be suppressed if the function does not exists.

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -246,6 +246,11 @@ public List<SqlFunction> listFunctions(Session session)
.collect(toImmutableList());
}

public Collection<SqlFunction> listBuiltInFunctions()
{
return builtInTypeAndFunctionNamespaceManager.listFunctions();
}

public Collection<? extends SqlFunction> getFunctions(Optional<TransactionId> transactionId, QualifiedObjectName functionName)
{
Optional<FunctionNamespaceManager<? extends SqlFunction>> functionNamespaceManager = getServingFunctionNamespaceManager(functionName.getCatalogSchemaName());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,8 @@
import com.facebook.presto.spi.TableHandle;
import com.facebook.presto.spi.WarningCollector;
import com.facebook.presto.spi.function.FunctionKind;
import com.facebook.presto.spi.function.Signature;
import com.facebook.presto.spi.function.SqlFunction;
import com.facebook.presto.spi.security.AccessDeniedException;
import com.facebook.presto.spi.security.Identity;
import com.facebook.presto.sql.parser.ParsingException;
Expand Down Expand Up @@ -571,7 +573,12 @@ protected Scope visitCreateFunction(CreateFunction node, Optional<Scope> scope)
analysis.setUpdateType("CREATE FUNCTION");

// Check function name
checkFunctionName(node, node.getFunctionName());
checkFunctionName(node, node.getFunctionName(), node.isTemporary());

// Check no replace with temporary functions
if (node.isTemporary() && node.isReplace()) {
throw new SemanticException(NOT_SUPPORTED, node, "REPLACE is not supported for temporary functions");
}

// Check parameter
List<String> duplicateParameters = node.getParameters().stream()
Expand Down Expand Up @@ -614,14 +621,14 @@ protected Scope visitCreateFunction(CreateFunction node, Optional<Scope> scope)
@Override
protected Scope visitAlterFunction(AlterFunction node, Optional<Scope> scope)
{
checkFunctionName(node, node.getFunctionName());
checkFunctionName(node, node.getFunctionName(), false);
return createAndAssignScope(node, scope);
}

@Override
protected Scope visitDropFunction(DropFunction node, Optional<Scope> scope)
{
checkFunctionName(node, node.getFunctionName());
checkFunctionName(node, node.getFunctionName(), node.isTemporary());
return createAndAssignScope(node, scope);
}

Expand Down Expand Up @@ -1662,10 +1669,26 @@ else if (column.getExpression() instanceof Identifier) {
return assignments.build();
}

private void checkFunctionName(Statement node, QualifiedName functionName)
private void checkFunctionName(Statement node, QualifiedName functionName, boolean isTemporary)
{
if (functionName.getParts().size() != 3) {
throw new SemanticException(INVALID_FUNCTION_NAME, node, format("Function name should be in the form of catalog.schema.function_name, found: %s", functionName));
if (isTemporary) {
if (functionName.getParts().size() != 1) {
throw new SemanticException(INVALID_FUNCTION_NAME, node, "Temporary functions cannot be qualified.");
}

List<String> builtInFunctionNames = metadata.getFunctionAndTypeManager().listBuiltInFunctions().stream()
.map(SqlFunction::getSignature)
.map(Signature::getName)
.map(QualifiedObjectName::getObjectName)
.collect(Collectors.toList());
if (builtInFunctionNames.contains(functionName.toString())) {
throw new SemanticException(INVALID_FUNCTION_NAME, node, format("Function %s is already registered as a built-in function.", functionName));
}
}
else {
if (functionName.getParts().size() != 3) {
throw new SemanticException(INVALID_FUNCTION_NAME, node, format("Function name should be in the form of catalog.schema.function_name, found: %s", functionName));
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -520,6 +520,7 @@ protected Node visitShowCreateFunction(ShowCreateFunction node, Void context)
CreateFunction createFunction = new CreateFunction(
node.getName(),
false,
false,
sqlFunction.getParameters().stream()
.map(parameter -> new SqlParameterDeclaration(new Identifier(parameter.getName()), parameter.getType().toString()))
.collect(toImmutableList()),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.DUPLICATE_COLUMN_NAME;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.DUPLICATE_PROPERTY;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.DUPLICATE_RELATION;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.INVALID_FUNCTION_NAME;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.INVALID_LITERAL;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.INVALID_ORDINAL;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.INVALID_PARAMETER_USAGE;
Expand Down Expand Up @@ -1554,4 +1555,17 @@ public void testEmptySchemaName()
{
assertFails(MISSING_SCHEMA, "SELECT * FROM \"\".foo");
}

@Test
public void testReplaceTemporaryFunctionFails()
{
assertFails(NOT_SUPPORTED, "CREATE OR REPLACE TEMPORARY FUNCTION foo() RETURNS INT RETURN 1");
}

@Test
public void testInvalidTemporaryFunctionName()
{
assertFails(INVALID_FUNCTION_NAME, "CREATE TEMPORARY FUNCTION sum() RETURNS INT RETURN 1");
assertFails(INVALID_FUNCTION_NAME, "CREATE TEMPORARY FUNCTION dev.test.foo() RETURNS INT RETURN 1");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -64,14 +64,14 @@ statement
| CREATE MATERIALIZED VIEW (IF NOT EXISTS)? qualifiedName
(COMMENT string)?
(WITH properties)? AS (query | '('query')') #createMaterializedView
| CREATE (OR REPLACE)? FUNCTION functionName=qualifiedName
| CREATE (OR REPLACE)? TEMPORARY? FUNCTION functionName=qualifiedName
'(' (sqlParameterDeclaration (',' sqlParameterDeclaration)*)? ')'
RETURNS returnType=type
(COMMENT string)?
routineCharacteristics routineBody #createFunction
| ALTER FUNCTION qualifiedName types?
alterRoutineCharacteristics #alterFunction
| DROP FUNCTION (IF EXISTS)? qualifiedName types? #dropFunction
| DROP TEMPORARY? FUNCTION (IF EXISTS)? qualifiedName types? #dropFunction
| CALL qualifiedName '(' (callArgument (',' callArgument)*)? ')' #call
| CREATE ROLE name=identifier
(WITH ADMIN grantor)? #createRole
Expand Down Expand Up @@ -569,7 +569,7 @@ nonReserved
| RANGE | READ | RENAME | REPEATABLE | REPLACE | RESET | RESPECT | RESTRICT | RETURN | RETURNS | REVOKE | ROLE | ROLES | ROLLBACK | ROW | ROWS
| SCHEMA | SCHEMAS | SECOND | SECURITY | SERIALIZABLE | SESSION | SET | SETS | SQL
| SHOW | SOME | START | STATS | SUBSTRING | SYSTEM
| TABLES | TABLESAMPLE | TEXT | TIME | TIMESTAMP | TO | TRANSACTION | TRY_CAST | TYPE
| TABLES | TABLESAMPLE | TEMPORARY | TEXT | TIME | TIMESTAMP | TO | TRANSACTION | TRY_CAST | TYPE
| UNBOUNDED | UNCOMMITTED | USE | USER
| VALIDATE | VERBOSE | VIEW
| WORK | WRITE
Expand Down Expand Up @@ -749,6 +749,7 @@ SYSTEM: 'SYSTEM';
TABLE: 'TABLE';
TABLES: 'TABLES';
TABLESAMPLE: 'TABLESAMPLE';
TEMPORARY: 'TEMPORARY';
TEXT: 'TEXT';
THEN: 'THEN';
TIME: 'TIME';
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -584,7 +584,11 @@ protected Void visitCreateMaterializedView(CreateMaterializedView node, Integer
@Override
protected Void visitCreateFunction(CreateFunction node, Integer indent)
{
builder.append("CREATE FUNCTION ")
builder.append("CREATE ");
if (node.isTemporary()) {
builder.append("TEMPORARY ");
}
builder.append("FUNCTION ")
.append(formatName(node.getFunctionName()))
.append(" ")
.append(formatSqlParameterDeclarations(node.getParameters()))
Expand Down Expand Up @@ -618,7 +622,11 @@ protected Void visitAlterFunction(AlterFunction node, Integer indent)
@Override
protected Void visitDropFunction(DropFunction node, Integer indent)
{
builder.append("DROP FUNCTION ");
builder.append("DROP ");
if (node.isTemporary()) {
builder.append("TEMPORARY ");
}
builder.append("FUNCTION ");
if (node.isExists()) {
builder.append("IF EXISTS ");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -476,6 +476,7 @@ public Node visitCreateFunction(SqlBaseParser.CreateFunctionContext context)
return new CreateFunction(
getQualifiedName(context.functionName),
context.REPLACE() != null,
context.TEMPORARY() != null,
context.sqlParameterDeclaration().stream()
.map(this::getParameterDeclarations)
.collect(toImmutableList()),
Expand All @@ -500,7 +501,12 @@ public Node visitAlterFunction(SqlBaseParser.AlterFunctionContext context)
public Node visitDropFunction(SqlBaseParser.DropFunctionContext context)
{
Optional<List<String>> parameterTypes = context.types() == null ? Optional.empty() : Optional.of(getTypes(context.types()));
return new DropFunction(getLocation(context), getQualifiedName(context.qualifiedName()), parameterTypes, context.EXISTS() != null);
return new DropFunction(
getLocation(context),
getQualifiedName(context.qualifiedName()),
parameterTypes,
context.TEMPORARY() != null,
context.EXISTS() != null);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,27 +27,29 @@ public class CreateFunction
{
private final QualifiedName functionName;
private final boolean replace;
private final boolean temporary;
private final List<SqlParameterDeclaration> parameters;
private final String returnType;
private final Optional<String> comment;
private final RoutineCharacteristics characteristics;
private final RoutineBody body;

public CreateFunction(QualifiedName functionName, boolean replace, List<SqlParameterDeclaration> parameters, String returnType, Optional<String> comment, RoutineCharacteristics characteristics, RoutineBody body)
public CreateFunction(QualifiedName functionName, boolean replace, boolean temporary, List<SqlParameterDeclaration> parameters, String returnType, Optional<String> comment, RoutineCharacteristics characteristics, RoutineBody body)
{
this(Optional.empty(), replace, functionName, parameters, returnType, comment, characteristics, body);
this(Optional.empty(), replace, temporary, functionName, parameters, returnType, comment, characteristics, body);
}

public CreateFunction(NodeLocation location, boolean replace, QualifiedName functionName, List<SqlParameterDeclaration> parameters, String returnType, Optional<String> comment, RoutineCharacteristics characteristics, RoutineBody body)
public CreateFunction(NodeLocation location, boolean replace, boolean temporary, QualifiedName functionName, List<SqlParameterDeclaration> parameters, String returnType, Optional<String> comment, RoutineCharacteristics characteristics, RoutineBody body)
{
this(Optional.of(location), replace, functionName, parameters, returnType, comment, characteristics, body);
this(Optional.of(location), replace, temporary, functionName, parameters, returnType, comment, characteristics, body);
}

private CreateFunction(Optional<NodeLocation> location, boolean replace, QualifiedName functionName, List<SqlParameterDeclaration> parameters, String returnType, Optional<String> comment, RoutineCharacteristics characteristics, RoutineBody body)
private CreateFunction(Optional<NodeLocation> location, boolean replace, boolean temporary, QualifiedName functionName, List<SqlParameterDeclaration> parameters, String returnType, Optional<String> comment, RoutineCharacteristics characteristics, RoutineBody body)
{
super(location);
this.functionName = requireNonNull(functionName, "functionName is null");
this.replace = replace;
this.temporary = temporary;
this.parameters = ImmutableList.copyOf(requireNonNull(parameters, "parameters is null"));
this.returnType = requireNonNull(returnType, "returnType is null");
this.comment = requireNonNull(comment, "comment is null");
Expand All @@ -65,6 +67,11 @@ public boolean isReplace()
return replace;
}

public boolean isTemporary()
{
return temporary;
}

public List<SqlParameterDeclaration> getParameters()
{
return parameters;
Expand Down Expand Up @@ -107,7 +114,7 @@ public List<Node> getChildren()
@Override
public int hashCode()
{
return Objects.hash(functionName, parameters, returnType, comment, characteristics, body);
return Objects.hash(functionName, temporary, parameters, returnType, comment, characteristics, body);
}

@Override
Expand All @@ -121,6 +128,7 @@ public boolean equals(Object obj)
}
CreateFunction o = (CreateFunction) obj;
return Objects.equals(functionName, o.functionName) &&
Objects.equals(temporary, o.temporary) &&
Objects.equals(parameters, o.parameters) &&
Objects.equals(returnType, o.returnType) &&
Objects.equals(comment, o.comment) &&
Expand All @@ -133,6 +141,7 @@ public String toString()
{
return toStringHelper(this)
.add("functionName", functionName)
.add("temporary", temporary)
.add("parameters", parameters)
.add("returnType", returnType)
.add("comment", comment)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,23 +27,25 @@ public class DropFunction
{
private final QualifiedName functionName;
private final Optional<List<String>> parameterTypes;
private final boolean temporary;
private final boolean exists;

public DropFunction(QualifiedName functionName, Optional<List<String>> parameterTypes, boolean exists)
public DropFunction(QualifiedName functionName, Optional<List<String>> parameterTypes, boolean temporary, boolean exists)
{
this(Optional.empty(), functionName, parameterTypes, exists);
this(Optional.empty(), functionName, parameterTypes, temporary, exists);
}

public DropFunction(NodeLocation location, QualifiedName functionName, Optional<List<String>> parameterTypes, boolean exists)
public DropFunction(NodeLocation location, QualifiedName functionName, Optional<List<String>> parameterTypes, boolean temporary, boolean exists)
{
this(Optional.of(location), functionName, parameterTypes, exists);
this(Optional.of(location), functionName, parameterTypes, temporary, exists);
}

private DropFunction(Optional<NodeLocation> location, QualifiedName functionName, Optional<List<String>> parameterTypes, boolean exists)
private DropFunction(Optional<NodeLocation> location, QualifiedName functionName, Optional<List<String>> parameterTypes, boolean temporary, boolean exists)
{
super(location);
this.functionName = requireNonNull(functionName, "functionName is null");
this.parameterTypes = requireNonNull(parameterTypes, "parameterTypes is null").map(ImmutableList::copyOf);
this.temporary = temporary;
this.exists = exists;
}

Expand All @@ -57,6 +59,11 @@ public Optional<List<String>> getParameterTypes()
return parameterTypes;
}

public boolean isTemporary()
{
return temporary;
}

public boolean isExists()
{
return exists;
Expand All @@ -77,7 +84,7 @@ public List<Node> getChildren()
@Override
public int hashCode()
{
return Objects.hash(functionName, parameterTypes, exists);
return Objects.hash(functionName, parameterTypes, temporary, exists);
}

@Override
Expand All @@ -91,6 +98,7 @@ public boolean equals(Object obj)
}
DropFunction o = (DropFunction) obj;
return Objects.equals(functionName, o.functionName)
&& Objects.equals(temporary, o.temporary)
&& Objects.equals(parameterTypes, o.parameterTypes)
&& (exists == o.exists);
}
Expand All @@ -101,6 +109,7 @@ public String toString()
return toStringHelper(this)
.add("functionName", functionName)
.add("parameterTypes", parameterTypes)
.add("temporary", temporary)
.add("exists", exists)
.toString();
}
Expand Down
Loading

0 comments on commit 574d21d

Please sign in to comment.