Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -344,9 +344,8 @@ public FunctionMetadata getFunctionMetadata(FunctionHandle functionHandle)
if (functionHandle.getCatalogSchemaName().equals(SESSION_NAMESPACE)) {
return ((SessionFunctionHandle) functionHandle).getFunctionMetadata();
}
Optional<FunctionNamespaceManager<?>> functionNamespaceManager = getServingFunctionNamespaceManager(functionHandle.getCatalogSchemaName());
checkArgument(functionNamespaceManager.isPresent(), "Cannot find function namespace for '%s'", functionHandle.getCatalogSchemaName());
return functionNamespaceManager.get().getFunctionMetadata(functionHandle);
FunctionNamespaceManager<?> functionNamespaceManager = getServingFunctionNamespaceManager(functionHandle.getCatalogSchemaName()).orElse(builtInTypeAndFunctionNamespaceManager);
return functionNamespaceManager.getFunctionMetadata(functionHandle);
}

@Override
Expand Down Expand Up @@ -526,18 +525,16 @@ public FunctionHandle resolveFunction(
QualifiedObjectName functionName,
List<TypeSignatureProvider> parameterTypes)
{
if (functionName.getCatalogSchemaName().equals(JAVA_BUILTIN_NAMESPACE)) {
if (sessionFunctions.isPresent()) {
Collection<SqlFunction> candidates = SessionFunctionUtils.getFunctions(sessionFunctions.get(), functionName);
Optional<Signature> match = functionSignatureMatcher.match(candidates, parameterTypes, true);
if (match.isPresent()) {
return SessionFunctionUtils.getFunctionHandle(sessionFunctions.get(), match.get());
}
if (sessionFunctions.isPresent()) {
Collection<SqlFunction> candidates = SessionFunctionUtils.getFunctions(sessionFunctions.get(), functionName);
Optional<Signature> match = functionSignatureMatcher.match(candidates, parameterTypes, true);
if (match.isPresent()) {
return SessionFunctionUtils.getFunctionHandle(sessionFunctions.get(), match.get());
}
}

if (parameterTypes.stream().noneMatch(TypeSignatureProvider::hasDependency)) {
return lookupCachedFunction(functionName, parameterTypes);
}
if (parameterTypes.stream().noneMatch(TypeSignatureProvider::hasDependency)) {
return lookupCachedFunction(functionName, parameterTypes);
}

return resolveFunctionInternal(transactionId, functionName, parameterTypes);
Expand Down Expand Up @@ -595,9 +592,8 @@ public ScalarFunctionImplementation getScalarFunctionImplementation(FunctionHand
if (functionHandle.getCatalogSchemaName().equals(SESSION_NAMESPACE)) {
return ((SessionFunctionHandle) functionHandle).getScalarFunctionImplementation();
}
Optional<FunctionNamespaceManager<?>> functionNamespaceManager = getServingFunctionNamespaceManager(functionHandle.getCatalogSchemaName());
checkArgument(functionNamespaceManager.isPresent(), "Cannot find function namespace for '%s'", functionHandle.getCatalogSchemaName());
return functionNamespaceManager.get().getScalarFunctionImplementation(functionHandle);
FunctionNamespaceManager<?> functionNamespaceManager = getServingFunctionNamespaceManager(functionHandle.getCatalogSchemaName()).orElse(builtInTypeAndFunctionNamespaceManager);
return functionNamespaceManager.getScalarFunctionImplementation(functionHandle);
}

public AggregationFunctionImplementation getAggregateFunctionImplementation(FunctionHandle functionHandle)
Expand Down Expand Up @@ -761,7 +757,7 @@ private Type getUserDefinedType(TypeSignature signature)

private FunctionHandle resolveFunctionInternal(Optional<TransactionId> transactionId, QualifiedObjectName functionName, List<TypeSignatureProvider> parameterTypes)
{
FunctionNamespaceManager<?> functionNamespaceManager = getServingFunctionNamespaceManager(functionName.getCatalogSchemaName()).orElse(null);
FunctionNamespaceManager<?> functionNamespaceManager = getServingFunctionNamespaceManager(functionName.getCatalogSchemaName()).orElse(builtInTypeAndFunctionNamespaceManager);
if (functionNamespaceManager == null) {
throw new PrestoException(FUNCTION_NOT_FOUND, constructFunctionNotFoundErrorMessage(functionName, parameterTypes, ImmutableList.of()));
}
Expand Down Expand Up @@ -798,8 +794,6 @@ private FunctionHandle resolveFunctionInternal(Optional<TransactionId> transacti

private FunctionHandle resolveBuiltInFunction(QualifiedObjectName functionName, List<TypeSignatureProvider> parameterTypes)
{
checkArgument(functionName.getCatalogSchemaName().equals(defaultNamespace) ||
functionName.getCatalogSchemaName().equals(JAVA_BUILTIN_NAMESPACE), "Expect built-in/default namespace functions");
checkArgument(parameterTypes.stream().noneMatch(TypeSignatureProvider::hasDependency), "Expect parameter types not to have dependency");
return resolveFunctionInternal(Optional.empty(), functionName, parameterTypes);
}
Expand Down Expand Up @@ -832,12 +826,9 @@ private Optional<FunctionNamespaceManager<? extends SqlFunction>> getServingFunc
public SpecializedFunctionKey getSpecializedFunctionKey(Signature signature)
{
QualifiedObjectName functionName = signature.getName();
Optional<FunctionNamespaceManager<?>> functionNamespaceManager = getServingFunctionNamespaceManager(functionName.getCatalogSchemaName());
if (!functionNamespaceManager.isPresent()) {
throw new PrestoException(FUNCTION_NOT_FOUND, format("Cannot find function namespace for signature '%s'", functionName));
}
FunctionNamespaceManager<?> functionNamespaceManager = getServingFunctionNamespaceManager(functionName.getCatalogSchemaName()).orElse(builtInTypeAndFunctionNamespaceManager);

Collection<SqlFunction> candidates = (Collection<SqlFunction>) functionNamespaceManager.get().getFunctions(Optional.empty(), functionName);
Collection<SqlFunction> candidates = (Collection<SqlFunction>) functionNamespaceManager.getFunctions(Optional.empty(), functionName);

// search for exact match
Type returnType = getType(signature.getReturnType());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import com.facebook.presto.common.function.OperatorType;
import com.facebook.presto.operator.scalar.ScalarHeader;
import com.facebook.presto.spi.function.ComplexTypeFunctionDescriptor;
import com.facebook.presto.spi.function.FunctionNamespace;
import com.facebook.presto.spi.function.ScalarFunction;
import com.facebook.presto.spi.function.ScalarOperator;
import com.facebook.presto.spi.function.SqlFunctionVisibility;
Expand All @@ -34,6 +35,7 @@
import static com.google.common.base.CaseFormat.LOWER_CAMEL;
import static com.google.common.base.CaseFormat.LOWER_UNDERSCORE;
import static com.google.common.base.Preconditions.checkArgument;
import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;

public class ScalarImplementationHeader
Expand All @@ -42,13 +44,18 @@ public class ScalarImplementationHeader
private final Optional<OperatorType> operatorType;
private final ScalarHeader header;

private ScalarImplementationHeader(String name, ScalarHeader header)
private ScalarImplementationHeader(QualifiedObjectName qualifiedObjectName, ScalarHeader header)
{
this.name = QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, requireNonNull(name));
this.name = qualifiedObjectName;
this.operatorType = Optional.empty();
this.header = requireNonNull(header);
}

private ScalarImplementationHeader(String name, ScalarHeader header)
{
this(QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, name), header);
}

private ScalarImplementationHeader(OperatorType operatorType, ScalarHeader header)
{
this.name = operatorType.getFunctionName();
Expand Down Expand Up @@ -84,10 +91,18 @@ public static List<ScalarImplementationHeader> fromAnnotatedElement(AnnotatedEle

if (scalarFunction != null) {
String baseName = scalarFunction.value().isEmpty() ? camelToSnake(annotatedName(annotated)) : scalarFunction.value();
builder.add(new ScalarImplementationHeader(baseName, new ScalarHeader(description, scalarFunction.visibility(), scalarFunction.deterministic(), scalarFunction.calledOnNullInput(), descriptor)));
FunctionNamespace functionNamespaceAnnotation = null;
if (annotated instanceof Method) {
functionNamespaceAnnotation = ((Method) annotated).getDeclaringClass().getAnnotation(FunctionNamespace.class);
}
String functionNamespace;
functionNamespace = functionNamespaceAnnotation != null ? functionNamespaceAnnotation.value() : JAVA_BUILTIN_NAMESPACE.toString();
QualifiedObjectName qualifiedObjectName = QualifiedObjectName.valueOf(functionNamespace + "." + baseName.toLowerCase(ENGLISH));

builder.add(new ScalarImplementationHeader(qualifiedObjectName, new ScalarHeader(description, scalarFunction.visibility(), scalarFunction.deterministic(), scalarFunction.calledOnNullInput(), descriptor)));
for (String alias : scalarFunction.alias()) {
builder.add(new ScalarImplementationHeader(alias, new ScalarHeader(description, scalarFunction.visibility(), scalarFunction.deterministic(), scalarFunction.calledOnNullInput(), descriptor)));
QualifiedObjectName aliasObjectName = QualifiedObjectName.valueOf(functionNamespace + "." + alias.toLowerCase(ENGLISH));
builder.add(new ScalarImplementationHeader(aliasObjectName, new ScalarHeader(description, scalarFunction.visibility(), scalarFunction.deterministic(), scalarFunction.calledOnNullInput(), descriptor)));
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.spi.function;

import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;

@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.TYPE)
public @interface FunctionNamespace
{
String value() default "";
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.tests.function;

import com.facebook.presto.common.type.StandardTypes;
import com.facebook.presto.spi.function.Description;
import com.facebook.presto.spi.function.FunctionNamespace;
import com.facebook.presto.spi.function.ScalarFunction;
import com.facebook.presto.spi.function.SqlType;
import io.airlift.slice.Slice;

import static com.facebook.presto.tests.function.TestScalarFunctions.TEST_FUNCTION_NAMESPACE;

@FunctionNamespace(TEST_FUNCTION_NAMESPACE)
public final class TestFunctions
{
private TestFunctions()
{}

@Description("Returns modulo of value by numberOfBuckets")
@ScalarFunction
@SqlType(StandardTypes.BIGINT)
public static long modulo(
@SqlType(StandardTypes.BIGINT) long value,
@SqlType(StandardTypes.BIGINT) long numberOfBuckets)
{
return value % numberOfBuckets;
}

@Description(("Return the input string"))
@ScalarFunction
@SqlType(StandardTypes.VARCHAR)
public static Slice identity(@SqlType(StandardTypes.VARCHAR) Slice slice)
{
return slice;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.tests.function;

import com.facebook.presto.common.type.TimeZoneKey;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.server.testing.TestingPrestoServer;
import com.facebook.presto.spi.Plugin;
import com.facebook.presto.testing.MaterializedResult;
import com.facebook.presto.tests.TestingPrestoClient;
import com.google.common.collect.ImmutableSet;
import com.google.inject.Key;
import org.intellij.lang.annotations.Language;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

import java.util.Set;

import static com.facebook.presto.common.type.BigintType.BIGINT;
import static com.facebook.presto.common.type.VarcharType.VARCHAR;
import static com.facebook.presto.testing.TestingSession.testSessionBuilder;
import static java.lang.String.format;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.fail;

public class TestScalarFunctions
{
public static final String TEST_FUNCTION_NAMESPACE = "test.namespace";

protected TestingPrestoServer server;
protected TestingPrestoClient client;
protected TypeManager typeManager;

@BeforeClass
public void setup()
throws Exception
{
server = new TestingPrestoServer();
server.installPlugin(new TestFunctionPlugin());
client = new TestingPrestoClient(server, testSessionBuilder()
.setTimeZoneKey(TimeZoneKey.getTimeZoneKey("America/Bahia_Banderas"))
.build());
typeManager = server.getInstance(Key.get(TypeManager.class));
}

public void assertInvalidFunction(String expr, String exceptionPattern)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not used. Can you please add some tests with invalide function name and namespace?

{
try {
client.execute("SELECT " + expr);
fail("Function expected to fail but not");
}
catch (Exception e) {
if (!(e.getMessage().matches(exceptionPattern))) {
fail(format("Expected exception message '%s' to match '%s' but not",
e.getMessage(), exceptionPattern));
}
}
}

private class TestFunctionPlugin
implements Plugin
{
@Override
public Set<Class<?>> getFunctions()
{
return ImmutableSet.<Class<?>>builder()
.add(TestFunctions.class)
.build();
}
}

public void check(@Language("SQL") String query, Type expectedType, Object expectedValue)
{
MaterializedResult result = client.execute(query).getResult();
assertEquals(result.getRowCount(), 1);
assertEquals(result.getTypes().get(0), expectedType);
Object actual = result.getMaterializedRows().get(0).getField(0);
assertEquals(actual, expectedValue);
}

@Test
public void testNewFunctionNamespaceFunction()
{
check("SELECT test.namespace.modulo(10,3)", BIGINT, 1L);
check("SELECT test.namespace.identity('test-functions')", VARCHAR, "test-functions");
}

@Test
public void testInvalidFunctionAndNamespace()
{
assertInvalidFunction("invalid.namespace.modulo(10,3)", "line 1:8: Function invalid.namespace.modulo not registered");
assertInvalidFunction("test.namespace.dummy(10)", "line 1:8: Function test.namespace.dummy not registered");
}
}
Loading