Skip to content

Commit

Permalink
Rename StaticFunctionNamespace to BuiltInFunctionNamespace
Browse files Browse the repository at this point in the history
  • Loading branch information
rongrong committed Sep 4, 2019
1 parent 85d30a6 commit 3052863
Show file tree
Hide file tree
Showing 60 changed files with 94 additions and 119 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,14 +23,14 @@

import static java.util.Objects.requireNonNull;

public class StaticFunctionHandle
public class BuiltInFunctionHandle
implements FunctionHandle
{
private static final CatalogSchemaName STATIC_FUNCTION_NAMESPACE_CATALOG_SCHEMA_NAME = new CatalogSchemaName("static", "system");
private final Signature signature;

@JsonCreator
public StaticFunctionHandle(@JsonProperty("signature") Signature signature)
public BuiltInFunctionHandle(@JsonProperty("signature") Signature signature)
{
this.signature = requireNonNull(signature, "signature is null");
checkArgument(signature.getTypeVariableConstraints().isEmpty(), "%s has unbound type parameters", signature);
Expand All @@ -51,7 +51,7 @@ public boolean equals(Object o)
if (o == null || getClass() != o.getClass()) {
return false;
}
StaticFunctionHandle that = (StaticFunctionHandle) o;
BuiltInFunctionHandle that = (BuiltInFunctionHandle) o;
return Objects.equals(signature, that.signature);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,12 @@
import com.facebook.presto.spi.function.FunctionHandle;
import com.facebook.presto.spi.function.FunctionHandleResolver;

public class StaticFunctionNamespaceHandleResolver
public class BuiltInFunctionNamespaceHandleResolver
implements FunctionHandleResolver
{
@Override
public Class<? extends FunctionHandle> getFunctionHandleClass()
{
return StaticFunctionHandle.class;
return BuiltInFunctionHandle.class;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -345,7 +345,7 @@
import static java.util.concurrent.TimeUnit.HOURS;

@ThreadSafe
public class StaticFunctionNamespaceManager
public class BuiltInFunctionNamespaceManager
implements FunctionNamespaceManager
{
public static final FullyQualifiedName.Prefix DEFAULT_NAMESPACE = FullyQualifiedName.of("presto.default.foo").getPrefix();
Expand All @@ -358,7 +358,7 @@ public class StaticFunctionNamespaceManager
private final MagicLiteralFunction magicLiteralFunction;
private volatile FunctionMap functions = new FunctionMap();

public StaticFunctionNamespaceManager(
public BuiltInFunctionNamespaceManager(
TypeManager typeManager,
BlockEncodingSerde blockEncodingSerde,
FeaturesConfig featuresConfig,
Expand Down Expand Up @@ -686,14 +686,14 @@ public Collection<SqlFunction> getCandidates(QueryId queryId, FullyQualifiedName
@Override
public FunctionHandle getFunctionHandle(QueryId queryId, Signature signature)
{
return new StaticFunctionHandle(signature);
return new BuiltInFunctionHandle(signature);
}

@Override
public FunctionMetadata getFunctionMetadata(FunctionHandle functionHandle)
{
checkArgument(functionHandle instanceof StaticFunctionHandle, "Expect StaticFunctionHandle");
Signature signature = ((StaticFunctionHandle) functionHandle).getSignature();
checkArgument(functionHandle instanceof BuiltInFunctionHandle, "Expect BuiltInFunctionHandle");
Signature signature = ((BuiltInFunctionHandle) functionHandle).getSignature();
SpecializedFunctionKey functionKey;
try {
functionKey = specializedFunctionKeyCache.getUnchecked(signature);
Expand Down Expand Up @@ -726,8 +726,8 @@ public FunctionMetadata getFunctionMetadata(FunctionHandle functionHandle)

public WindowFunctionSupplier getWindowFunctionImplementation(FunctionHandle functionHandle)
{
checkArgument(functionHandle instanceof StaticFunctionHandle, "Expect StaticFunctionHandle");
Signature signature = ((StaticFunctionHandle) functionHandle).getSignature();
checkArgument(functionHandle instanceof BuiltInFunctionHandle, "Expect BuiltInFunctionHandle");
Signature signature = ((BuiltInFunctionHandle) functionHandle).getSignature();
checkArgument(signature.getKind() == WINDOW || signature.getKind() == AGGREGATE, "%s is not a window function", signature);
checkArgument(signature.getTypeVariableConstraints().isEmpty(), "%s has unbound type parameters", signature);

Expand All @@ -742,8 +742,8 @@ public WindowFunctionSupplier getWindowFunctionImplementation(FunctionHandle fun

public InternalAggregationFunction getAggregateFunctionImplementation(FunctionHandle functionHandle)
{
checkArgument(functionHandle instanceof StaticFunctionHandle, "Expect StaticFunctionHandle");
Signature signature = ((StaticFunctionHandle) functionHandle).getSignature();
checkArgument(functionHandle instanceof BuiltInFunctionHandle, "Expect BuiltInFunctionHandle");
Signature signature = ((BuiltInFunctionHandle) functionHandle).getSignature();
checkArgument(signature.getKind() == AGGREGATE, "%s is not an aggregate function", signature);
checkArgument(signature.getTypeVariableConstraints().isEmpty(), "%s has unbound type parameters", signature);

Expand All @@ -758,8 +758,8 @@ public InternalAggregationFunction getAggregateFunctionImplementation(FunctionHa

public ScalarFunctionImplementation getScalarFunctionImplementation(FunctionHandle functionHandle)
{
checkArgument(functionHandle instanceof StaticFunctionHandle, "Expect StaticFunctionHandle");
return getScalarFunctionImplementation(((StaticFunctionHandle) functionHandle).getSignature());
checkArgument(functionHandle instanceof BuiltInFunctionHandle, "Expect BuiltInFunctionHandle");
return getScalarFunctionImplementation(((BuiltInFunctionHandle) functionHandle).getSignature());
}

public ScalarFunctionImplementation getScalarFunctionImplementation(Signature signature)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
import com.facebook.presto.spi.function.OperatorType;
import com.facebook.presto.spi.relation.FullyQualifiedName;

import static com.facebook.presto.metadata.StaticFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.metadata.BuiltInFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.operator.scalar.JsonStringToArrayCast.JSON_STRING_TO_ARRAY_NAME;
import static com.facebook.presto.operator.scalar.JsonStringToMapCast.JSON_STRING_TO_MAP_NAME;
import static com.facebook.presto.operator.scalar.JsonStringToRowCast.JSON_STRING_TO_ROW_NAME;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,8 +49,8 @@
import java.util.Set;
import java.util.stream.Collectors;

import static com.facebook.presto.metadata.BuiltInFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.metadata.CastType.toOperatorType;
import static com.facebook.presto.metadata.StaticFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.spi.StandardErrorCode.AMBIGUOUS_FUNCTION_CALL;
import static com.facebook.presto.spi.StandardErrorCode.FUNCTION_IMPLEMENTATION_MISSING;
import static com.facebook.presto.spi.StandardErrorCode.FUNCTION_NOT_FOUND;
Expand All @@ -77,13 +77,13 @@ public class FunctionManager
implements FunctionMetadataManager
{
private final TypeManager typeManager;
private final StaticFunctionNamespaceManager staticFunctionNamespace;
private final BuiltInFunctionNamespaceManager builtInFunctionNamespace;
private final FunctionInvokerProvider functionInvokerProvider;

public FunctionManager(TypeManager typeManager, BlockEncodingSerde blockEncodingSerde, FeaturesConfig featuresConfig)
{
this.typeManager = requireNonNull(typeManager, "typeManager is null");
this.staticFunctionNamespace = new StaticFunctionNamespaceManager(typeManager, blockEncodingSerde, featuresConfig, this);
this.builtInFunctionNamespace = new BuiltInFunctionNamespaceManager(typeManager, blockEncodingSerde, featuresConfig, this);
this.functionInvokerProvider = new FunctionInvokerProvider(this);
if (typeManager instanceof TypeRegistry) {
((TypeRegistry) typeManager).setFunctionManager(this);
Expand All @@ -97,12 +97,12 @@ public FunctionInvokerProvider getFunctionInvokerProvider()

public void addFunctions(List<? extends SqlFunction> functions)
{
staticFunctionNamespace.addFunctions(functions);
builtInFunctionNamespace.addFunctions(functions);
}

public List<SqlFunction> listFunctions()
{
return staticFunctionNamespace.listFunctions().stream()
return builtInFunctionNamespace.listFunctions().stream()
.filter(function -> !function.isHidden())
.collect(toImmutableList());
}
Expand All @@ -128,10 +128,10 @@ public FunctionHandle resolveFunction(Session session, QualifiedName name, List<
}
}
FullyQualifiedName fullyQualifiedName = FullyQualifiedName.of(DEFAULT_NAMESPACE, name.getSuffix());
Collection<SqlFunction> allCandidates = staticFunctionNamespace.getCandidates(null, fullyQualifiedName);
Collection<SqlFunction> allCandidates = builtInFunctionNamespace.getCandidates(null, fullyQualifiedName);
Optional<Signature> match = matchFunctionWithCoercion(allCandidates, parameterTypes);
if (match.isPresent()) {
return new StaticFunctionHandle(match.get());
return new BuiltInFunctionHandle(match.get());
}

if (name.getSuffix().startsWith(MAGIC_LITERAL_FUNCTION_PREFIX)) {
Expand All @@ -144,7 +144,7 @@ public FunctionHandle resolveFunction(Session session, QualifiedName name, List<
// verify we have one parameter of the proper type
checkArgument(parameterTypes.size() == 1, "Expected one argument to literal function, but got %s", parameterTypes);

return new StaticFunctionHandle(getMagicLiteralFunctionSignature(type));
return new BuiltInFunctionHandle(getMagicLiteralFunctionSignature(type));
}

throw new PrestoException(FUNCTION_NOT_FOUND, constructFunctionNotFoundErrorMessage(name.getSuffix(), parameterTypes, allCandidates));
Expand All @@ -153,22 +153,22 @@ public FunctionHandle resolveFunction(Session session, QualifiedName name, List<
@Override
public FunctionMetadata getFunctionMetadata(FunctionHandle functionHandle)
{
return staticFunctionNamespace.getFunctionMetadata(functionHandle);
return builtInFunctionNamespace.getFunctionMetadata(functionHandle);
}

public WindowFunctionSupplier getWindowFunctionImplementation(FunctionHandle functionHandle)
{
return staticFunctionNamespace.getWindowFunctionImplementation(functionHandle);
return builtInFunctionNamespace.getWindowFunctionImplementation(functionHandle);
}

public InternalAggregationFunction getAggregateFunctionImplementation(FunctionHandle functionHandle)
{
return staticFunctionNamespace.getAggregateFunctionImplementation(functionHandle);
return builtInFunctionNamespace.getAggregateFunctionImplementation(functionHandle);
}

public ScalarFunctionImplementation getScalarFunctionImplementation(FunctionHandle functionHandle)
{
return staticFunctionNamespace.getScalarFunctionImplementation(functionHandle);
return builtInFunctionNamespace.getScalarFunctionImplementation(functionHandle);
}

@VisibleForTesting
Expand All @@ -178,7 +178,7 @@ public List<SqlFunction> listOperators()
.map(OperatorType::getFunctionName)
.collect(toImmutableSet());

return staticFunctionNamespace.listFunctions().stream()
return builtInFunctionNamespace.listFunctions().stream()
.filter(function -> operatorNames.contains(function.getSignature().getName()))
.collect(toImmutableList());
}
Expand Down Expand Up @@ -211,14 +211,14 @@ public FunctionHandle resolveOperator(OperatorType operatorType, List<TypeSignat
public FunctionHandle lookupFunction(String name, List<TypeSignatureProvider> parameterTypes)
{
FullyQualifiedName fullyQualifiedName = FullyQualifiedName.of(DEFAULT_NAMESPACE, name);
Collection<SqlFunction> allCandidates = staticFunctionNamespace.getCandidates(null, fullyQualifiedName);
Collection<SqlFunction> allCandidates = builtInFunctionNamespace.getCandidates(null, fullyQualifiedName);
List<SqlFunction> exactCandidates = allCandidates.stream()
.filter(function -> function.getSignature().getTypeVariableConstraints().isEmpty())
.collect(Collectors.toList());

Optional<Signature> match = matchFunctionExact(exactCandidates, parameterTypes);
if (match.isPresent()) {
return new StaticFunctionHandle(match.get());
return new BuiltInFunctionHandle(match.get());
}

List<SqlFunction> genericCandidates = allCandidates.stream()
Expand All @@ -227,7 +227,7 @@ public FunctionHandle lookupFunction(String name, List<TypeSignatureProvider> pa

match = matchFunctionExact(genericCandidates, parameterTypes);
if (match.isPresent()) {
return new StaticFunctionHandle(match.get());
return new BuiltInFunctionHandle(match.get());
}

throw new PrestoException(FUNCTION_NOT_FOUND, constructFunctionNotFoundErrorMessage(name, parameterTypes, allCandidates));
Expand All @@ -238,15 +238,15 @@ public FunctionHandle lookupCast(CastType castType, TypeSignature fromType, Type
Signature signature = new Signature(castType.getCastName(), SCALAR, emptyList(), emptyList(), toType, singletonList(fromType), false);

try {
staticFunctionNamespace.getScalarFunctionImplementation(signature);
builtInFunctionNamespace.getScalarFunctionImplementation(signature);
}
catch (PrestoException e) {
if (castType.isOperatorType() && e.getErrorCode().getCode() == FUNCTION_IMPLEMENTATION_MISSING.toErrorCode().getCode()) {
throw new OperatorNotFoundException(toOperatorType(castType), ImmutableList.of(fromType), toType);
}
throw e;
}
return staticFunctionNamespace.getFunctionHandle(null, signature);
return builtInFunctionNamespace.getFunctionHandle(null, signature);
}

private String constructFunctionNotFoundErrorMessage(String name, List<TypeSignatureProvider> parameterTypes, Collection<SqlFunction> candidates)
Expand Down Expand Up @@ -437,7 +437,7 @@ private boolean returnsNullOnGivenInputTypes(ApplicableFunction applicableFuncti
Type parameterType = parameterTypes.get(i);
if (parameterType.equals(UNKNOWN)) {
// TODO: This still doesn't feel right. Need to understand function resolution logic better to know what's the right way.
StaticFunctionHandle functionHandle = new StaticFunctionHandle(boundSignature);
BuiltInFunctionHandle functionHandle = new BuiltInFunctionHandle(boundSignature);
if (getFunctionMetadata(functionHandle).isCalledOnNullInput()) {
return false;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ public HandleResolver()
handleResolvers.put("$info_schema", new MaterializedHandleResolver(new InformationSchemaHandleResolver()));
handleResolvers.put("$empty", new MaterializedHandleResolver(new EmptySplitHandleResolver()));

functionHandleResolvers.put("$static", new MaterializedFunctionHandleResolver(new StaticFunctionNamespaceHandleResolver()));
functionHandleResolvers.put("$static", new MaterializedFunctionHandleResolver(new BuiltInFunctionNamespaceHandleResolver()));
}

public void addConnectorName(String name, ConnectorHandleResolver resolver)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@

import java.util.List;

import static com.facebook.presto.metadata.StaticFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.metadata.BuiltInFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.spi.function.FunctionKind.SCALAR;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@

import java.util.List;

import static com.facebook.presto.metadata.StaticFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.metadata.BuiltInFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.spi.function.FunctionKind.AGGREGATE;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.collect.ImmutableList.toImmutableList;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@
import java.util.Set;
import java.util.stream.Stream;

import static com.facebook.presto.metadata.StaticFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.metadata.BuiltInFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.operator.aggregation.AggregationMetadata.ParameterMetadata.ParameterType.BLOCK_INDEX;
import static com.facebook.presto.operator.aggregation.AggregationMetadata.ParameterMetadata.ParameterType.STATE;
import static com.facebook.presto.operator.aggregation.AggregationMetadata.ParameterMetadata.ParameterType.inputChannelParameterType;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@

import java.lang.invoke.MethodHandle;

import static com.facebook.presto.metadata.StaticFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.metadata.BuiltInFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementation.ArgumentProperty.functionTypeArgumentProperty;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementation.ArgumentProperty.valueTypeArgumentProperty;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementation.NullConvention.USE_BOXED_TYPE;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@
import java.lang.invoke.MethodHandle;
import java.util.Optional;

import static com.facebook.presto.metadata.StaticFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.metadata.BuiltInFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementation.ArgumentProperty.valueTypeArgumentProperty;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementation.NullConvention.RETURN_NULL_ON_NULL;
import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@
import static com.facebook.presto.bytecode.expression.BytecodeExpressions.constantInt;
import static com.facebook.presto.bytecode.expression.BytecodeExpressions.constantNull;
import static com.facebook.presto.bytecode.expression.BytecodeExpressions.equal;
import static com.facebook.presto.metadata.StaticFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.metadata.BuiltInFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementation.ArgumentProperty.valueTypeArgumentProperty;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementation.NullConvention.USE_BOXED_TYPE;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@

import java.lang.invoke.MethodHandle;

import static com.facebook.presto.metadata.StaticFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.metadata.BuiltInFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementation.ArgumentProperty.valueTypeArgumentProperty;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementation.NullConvention.RETURN_NULL_ON_NULL;
import static com.facebook.presto.spi.function.Signature.typeVariable;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,8 @@
import java.util.Map;
import java.util.Optional;

import static com.facebook.presto.metadata.BuiltInFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.metadata.CastType.CAST;
import static com.facebook.presto.metadata.StaticFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementation.ArgumentProperty.valueTypeArgumentProperty;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementation.NullConvention.RETURN_NULL_ON_NULL;
import static com.facebook.presto.operator.scalar.ScalarFunctionImplementation.NullConvention.USE_BOXED_TYPE;
Expand Down
Loading

0 comments on commit 3052863

Please sign in to comment.