diff --git a/presto-main-base/src/main/java/com/facebook/presto/metadata/FunctionAndTypeManager.java b/presto-main-base/src/main/java/com/facebook/presto/metadata/FunctionAndTypeManager.java index 9f469711cd8d4..985692e155d45 100644 --- a/presto-main-base/src/main/java/com/facebook/presto/metadata/FunctionAndTypeManager.java +++ b/presto-main-base/src/main/java/com/facebook/presto/metadata/FunctionAndTypeManager.java @@ -344,9 +344,8 @@ public FunctionMetadata getFunctionMetadata(FunctionHandle functionHandle) if (functionHandle.getCatalogSchemaName().equals(SESSION_NAMESPACE)) { return ((SessionFunctionHandle) functionHandle).getFunctionMetadata(); } - Optional> 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 @@ -526,18 +525,16 @@ public FunctionHandle resolveFunction( QualifiedObjectName functionName, List parameterTypes) { - if (functionName.getCatalogSchemaName().equals(JAVA_BUILTIN_NAMESPACE)) { - if (sessionFunctions.isPresent()) { - Collection candidates = SessionFunctionUtils.getFunctions(sessionFunctions.get(), functionName); - Optional match = functionSignatureMatcher.match(candidates, parameterTypes, true); - if (match.isPresent()) { - return SessionFunctionUtils.getFunctionHandle(sessionFunctions.get(), match.get()); - } + if (sessionFunctions.isPresent()) { + Collection candidates = SessionFunctionUtils.getFunctions(sessionFunctions.get(), functionName); + Optional 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); @@ -595,9 +592,8 @@ public ScalarFunctionImplementation getScalarFunctionImplementation(FunctionHand if (functionHandle.getCatalogSchemaName().equals(SESSION_NAMESPACE)) { return ((SessionFunctionHandle) functionHandle).getScalarFunctionImplementation(); } - Optional> 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) @@ -761,7 +757,7 @@ private Type getUserDefinedType(TypeSignature signature) private FunctionHandle resolveFunctionInternal(Optional transactionId, QualifiedObjectName functionName, List 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())); } @@ -798,8 +794,6 @@ private FunctionHandle resolveFunctionInternal(Optional transacti private FunctionHandle resolveBuiltInFunction(QualifiedObjectName functionName, List 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); } @@ -832,12 +826,9 @@ private Optional> getServingFunc public SpecializedFunctionKey getSpecializedFunctionKey(Signature signature) { QualifiedObjectName functionName = signature.getName(); - Optional> 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 candidates = (Collection) functionNamespaceManager.get().getFunctions(Optional.empty(), functionName); + Collection candidates = (Collection) functionNamespaceManager.getFunctions(Optional.empty(), functionName); // search for exact match Type returnType = getType(signature.getReturnType()); diff --git a/presto-main-base/src/main/java/com/facebook/presto/operator/scalar/annotations/ScalarImplementationHeader.java b/presto-main-base/src/main/java/com/facebook/presto/operator/scalar/annotations/ScalarImplementationHeader.java index 2a3045bed71b1..4df12b0d56fc2 100644 --- a/presto-main-base/src/main/java/com/facebook/presto/operator/scalar/annotations/ScalarImplementationHeader.java +++ b/presto-main-base/src/main/java/com/facebook/presto/operator/scalar/annotations/ScalarImplementationHeader.java @@ -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; @@ -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 @@ -42,13 +44,18 @@ public class ScalarImplementationHeader private final Optional 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(); @@ -84,10 +91,18 @@ public static List 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))); } } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/function/FunctionNamespace.java b/presto-spi/src/main/java/com/facebook/presto/spi/function/FunctionNamespace.java new file mode 100644 index 0000000000000..75aa59e39616b --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/function/FunctionNamespace.java @@ -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 ""; +} diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/function/TestFunctions.java b/presto-tests/src/main/java/com/facebook/presto/tests/function/TestFunctions.java new file mode 100644 index 0000000000000..f5c91aa7d2783 --- /dev/null +++ b/presto-tests/src/main/java/com/facebook/presto/tests/function/TestFunctions.java @@ -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; + } +} diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/function/TestScalarFunctions.java b/presto-tests/src/main/java/com/facebook/presto/tests/function/TestScalarFunctions.java new file mode 100644 index 0000000000000..237ef5b8c65bd --- /dev/null +++ b/presto-tests/src/main/java/com/facebook/presto/tests/function/TestScalarFunctions.java @@ -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) + { + 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> getFunctions() + { + return ImmutableSet.>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"); + } +}