Skip to content

Commit 6b2e35c

Browse files
pdabre12Pratik Joseph Dabre
authored andcommitted
Introduce getSqlInvokedFunctions SPI and BuiltInPluginFunctionNamespaceManager for registering sql invoked functions
1 parent d95fca4 commit 6b2e35c

File tree

14 files changed

+399
-76
lines changed

14 files changed

+399
-76
lines changed

presto-main-base/src/main/java/com/facebook/presto/metadata/BuiltInFunctionHandle.java

Lines changed: 15 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -30,12 +30,16 @@ public class BuiltInFunctionHandle
3030
implements FunctionHandle
3131
{
3232
private final Signature signature;
33+
private final boolean isBuiltInPluginFunction;
3334

3435
@JsonCreator
35-
public BuiltInFunctionHandle(@JsonProperty("signature") Signature signature)
36+
public BuiltInFunctionHandle(
37+
@JsonProperty("signature") Signature signature,
38+
@JsonProperty("isBuiltInPluginFunction") boolean isBuiltInPluginFunction)
3639
{
3740
this.signature = requireNonNull(signature, "signature is null");
3841
checkArgument(signature.getTypeVariableConstraints().isEmpty(), "%s has unbound type parameters", signature);
42+
this.isBuiltInPluginFunction = isBuiltInPluginFunction;
3943
}
4044

4145
@JsonProperty
@@ -62,6 +66,13 @@ public List<TypeSignature> getArgumentTypes()
6266
return signature.getArgumentTypes();
6367
}
6468

69+
@JsonProperty
70+
@Override
71+
public boolean isBuiltInPluginFunction()
72+
{
73+
return isBuiltInPluginFunction;
74+
}
75+
6576
@Override
6677
public CatalogSchemaName getCatalogSchemaName()
6778
{
@@ -78,13 +89,14 @@ public boolean equals(Object o)
7889
return false;
7990
}
8091
BuiltInFunctionHandle that = (BuiltInFunctionHandle) o;
81-
return Objects.equals(signature, that.signature);
92+
return Objects.equals(signature, that.signature)
93+
&& Objects.equals(isBuiltInPluginFunction, that.isBuiltInPluginFunction);
8294
}
8395

8496
@Override
8597
public int hashCode()
8698
{
87-
return Objects.hash(signature);
99+
return Objects.hash(signature, isBuiltInPluginFunction);
88100
}
89101

90102
@Override
Lines changed: 183 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,183 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* http://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
package com.facebook.presto.metadata;
15+
16+
import com.facebook.presto.common.QualifiedObjectName;
17+
import com.facebook.presto.spi.PrestoException;
18+
import com.facebook.presto.spi.function.FunctionHandle;
19+
import com.facebook.presto.spi.function.FunctionMetadata;
20+
import com.facebook.presto.spi.function.FunctionNamespaceManager;
21+
import com.facebook.presto.spi.function.Parameter;
22+
import com.facebook.presto.spi.function.ScalarFunctionImplementation;
23+
import com.facebook.presto.spi.function.Signature;
24+
import com.facebook.presto.spi.function.SqlFunction;
25+
import com.facebook.presto.spi.function.SqlInvokedFunction;
26+
import com.facebook.presto.spi.function.SqlInvokedScalarFunctionImplementation;
27+
import com.google.common.base.Supplier;
28+
import com.google.common.base.Suppliers;
29+
import com.google.common.cache.CacheBuilder;
30+
import com.google.common.cache.CacheLoader;
31+
import com.google.common.cache.LoadingCache;
32+
import com.google.common.util.concurrent.UncheckedExecutionException;
33+
34+
import java.util.Collection;
35+
import java.util.List;
36+
import java.util.Optional;
37+
38+
import static com.facebook.presto.spi.function.FunctionImplementationType.SQL;
39+
import static com.facebook.presto.spi.function.FunctionKind.SCALAR;
40+
import static com.google.common.base.Preconditions.checkArgument;
41+
import static com.google.common.base.Throwables.throwIfInstanceOf;
42+
import static com.google.common.collect.ImmutableList.toImmutableList;
43+
import static java.util.Collections.emptyList;
44+
import static java.util.Objects.requireNonNull;
45+
import static java.util.concurrent.TimeUnit.HOURS;
46+
47+
public class BuiltInPluginFunctionNamespaceManager
48+
{
49+
private volatile FunctionMap functions = new FunctionMap();
50+
private final FunctionAndTypeManager functionAndTypeManager;
51+
private final Supplier<FunctionMap> cachedFunctions =
52+
Suppliers.memoize(this::checkForNamingConflicts);
53+
private final LoadingCache<Signature, SpecializedFunctionKey> specializedFunctionKeyCache;
54+
private final LoadingCache<SpecializedFunctionKey, ScalarFunctionImplementation> specializedScalarCache;
55+
56+
public BuiltInPluginFunctionNamespaceManager(FunctionAndTypeManager functionAndTypeManager)
57+
{
58+
this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionAndTypeManager is null");
59+
specializedFunctionKeyCache = CacheBuilder.newBuilder()
60+
.maximumSize(1000)
61+
.expireAfterWrite(1, HOURS)
62+
.build(CacheLoader.from(this::doGetSpecializedFunctionKey));
63+
specializedScalarCache = CacheBuilder.newBuilder()
64+
.maximumSize(1000)
65+
.expireAfterWrite(1, HOURS)
66+
.build(CacheLoader.from(key -> {
67+
checkArgument(
68+
key.getFunction() instanceof SqlInvokedFunction,
69+
"Unsupported scalar function class: %s",
70+
key.getFunction().getClass());
71+
return new SqlInvokedScalarFunctionImplementation(((SqlInvokedFunction) key.getFunction()).getBody());
72+
}));
73+
}
74+
75+
public synchronized void registerPluginFunctions(List<? extends SqlFunction> functions)
76+
{
77+
checkForNamingConflicts(functions);
78+
this.functions = new FunctionMap(this.functions, functions);
79+
}
80+
81+
public Collection<? extends SqlFunction> getFunctions(QualifiedObjectName functionName)
82+
{
83+
if (functions.list().isEmpty() ||
84+
(!functionName.getCatalogSchemaName().equals(functionAndTypeManager.getDefaultNamespace()))) {
85+
return emptyList();
86+
}
87+
return cachedFunctions.get().get(functionName);
88+
}
89+
90+
public List<SqlFunction> listFunctions()
91+
{
92+
return cachedFunctions.get().list();
93+
}
94+
95+
public FunctionHandle getFunctionHandle(Signature signature)
96+
{
97+
return new BuiltInFunctionHandle(signature, true);
98+
}
99+
100+
public FunctionMetadata getFunctionMetadata(FunctionHandle functionHandle)
101+
{
102+
checkArgument(functionHandle instanceof BuiltInFunctionHandle, "Expect BuiltInFunctionHandle");
103+
Signature signature = ((BuiltInFunctionHandle) functionHandle).getSignature();
104+
SpecializedFunctionKey functionKey;
105+
try {
106+
functionKey = specializedFunctionKeyCache.getUnchecked(signature);
107+
}
108+
catch (UncheckedExecutionException e) {
109+
throwIfInstanceOf(e.getCause(), PrestoException.class);
110+
throw e;
111+
}
112+
SqlFunction function = functionKey.getFunction();
113+
checkArgument(function instanceof SqlInvokedFunction, "BuiltInPluginFunctionNamespaceManager only support SqlInvokedFunctions");
114+
SqlInvokedFunction sqlFunction = (SqlInvokedFunction) function;
115+
List<String> argumentNames = sqlFunction.getParameters().stream().map(Parameter::getName).collect(toImmutableList());
116+
return new FunctionMetadata(
117+
signature.getName(),
118+
signature.getArgumentTypes(),
119+
argumentNames,
120+
signature.getReturnType(),
121+
signature.getKind(),
122+
sqlFunction.getRoutineCharacteristics().getLanguage(),
123+
SQL,
124+
function.isDeterministic(),
125+
function.isCalledOnNullInput(),
126+
sqlFunction.getVersion(),
127+
sqlFunction.getComplexTypeFunctionDescriptor());
128+
}
129+
130+
public ScalarFunctionImplementation getScalarFunctionImplementation(FunctionHandle functionHandle)
131+
{
132+
checkArgument(functionHandle instanceof BuiltInFunctionHandle, "Expect BuiltInFunctionHandle");
133+
return getScalarFunctionImplementation(((BuiltInFunctionHandle) functionHandle).getSignature());
134+
}
135+
136+
private ScalarFunctionImplementation getScalarFunctionImplementation(Signature signature)
137+
{
138+
checkArgument(signature.getKind() == SCALAR, "%s is not a scalar function", signature);
139+
checkArgument(signature.getTypeVariableConstraints().isEmpty(), "%s has unbound type parameters", signature);
140+
141+
try {
142+
return specializedScalarCache.getUnchecked(getSpecializedFunctionKey(signature));
143+
}
144+
catch (UncheckedExecutionException e) {
145+
throwIfInstanceOf(e.getCause(), PrestoException.class);
146+
throw e;
147+
}
148+
}
149+
150+
private synchronized FunctionMap checkForNamingConflicts()
151+
{
152+
Optional<FunctionNamespaceManager<?>> functionNamespaceManager =
153+
functionAndTypeManager.getServingFunctionNamespaceManager(functionAndTypeManager.getDefaultNamespace());
154+
checkArgument(functionNamespaceManager.isPresent(), "Cannot find function namespace for catalog '%s'", functionAndTypeManager.getDefaultNamespace().getCatalogName());
155+
checkForNamingConflicts(functionNamespaceManager.get().listFunctions(Optional.empty(), Optional.empty()));
156+
return functions;
157+
}
158+
159+
private synchronized void checkForNamingConflicts(Collection<? extends SqlFunction> functions)
160+
{
161+
for (SqlFunction function : functions) {
162+
for (SqlFunction existingFunction : this.functions.list()) {
163+
checkArgument(!function.getSignature().equals(existingFunction.getSignature()), "Function already registered: %s", function.getSignature());
164+
}
165+
}
166+
}
167+
168+
private SpecializedFunctionKey doGetSpecializedFunctionKey(Signature signature)
169+
{
170+
return functionAndTypeManager.getSpecializedFunctionKey(signature, getFunctions(signature.getName()));
171+
}
172+
173+
private SpecializedFunctionKey getSpecializedFunctionKey(Signature signature)
174+
{
175+
try {
176+
return specializedFunctionKeyCache.getUnchecked(signature);
177+
}
178+
catch (UncheckedExecutionException e) {
179+
throwIfInstanceOf(e.getCause(), PrestoException.class);
180+
throw e;
181+
}
182+
}
183+
}

presto-main-base/src/main/java/com/facebook/presto/metadata/BuiltInTypeAndFunctionNamespaceManager.java

Lines changed: 1 addition & 39 deletions
Original file line numberDiff line numberDiff line change
@@ -1133,7 +1133,7 @@ public Collection<SqlFunction> getFunctions(Optional<? extends FunctionNamespace
11331133
@Override
11341134
public FunctionHandle getFunctionHandle(Optional<? extends FunctionNamespaceTransactionHandle> transactionHandle, Signature signature)
11351135
{
1136-
return new BuiltInFunctionHandle(signature);
1136+
return new BuiltInFunctionHandle(signature, false);
11371137
}
11381138

11391139
@Override
@@ -1396,44 +1396,6 @@ private static class EmptyTransactionHandle
13961396
{
13971397
}
13981398

1399-
private static class FunctionMap
1400-
{
1401-
private final Multimap<QualifiedObjectName, SqlFunction> functions;
1402-
1403-
public FunctionMap()
1404-
{
1405-
functions = ImmutableListMultimap.of();
1406-
}
1407-
1408-
public FunctionMap(FunctionMap map, Iterable<? extends SqlFunction> functions)
1409-
{
1410-
this.functions = ImmutableListMultimap.<QualifiedObjectName, SqlFunction>builder()
1411-
.putAll(map.functions)
1412-
.putAll(Multimaps.index(functions, function -> function.getSignature().getName()))
1413-
.build();
1414-
1415-
// Make sure all functions with the same name are aggregations or none of them are
1416-
for (Map.Entry<QualifiedObjectName, Collection<SqlFunction>> entry : this.functions.asMap().entrySet()) {
1417-
Collection<SqlFunction> values = entry.getValue();
1418-
long aggregations = values.stream()
1419-
.map(function -> function.getSignature().getKind())
1420-
.filter(kind -> kind == AGGREGATE)
1421-
.count();
1422-
checkState(aggregations == 0 || aggregations == values.size(), "'%s' is both an aggregation and a scalar function", entry.getKey());
1423-
}
1424-
}
1425-
1426-
public List<SqlFunction> list()
1427-
{
1428-
return ImmutableList.copyOf(functions.values());
1429-
}
1430-
1431-
public Collection<SqlFunction> get(QualifiedObjectName name)
1432-
{
1433-
return functions.get(name);
1434-
}
1435-
}
1436-
14371399
/**
14381400
* TypeSignature but has overridden equals(). Here, we compare exact signature of any underlying distinct
14391401
* types. Some distinct types may have extra information on their lazily loaded parents, and same parent

0 commit comments

Comments
 (0)