Skip to content

Commit

Permalink
Refactor Signature to take TypeSignature instead of String
Browse files Browse the repository at this point in the history
This is second step of refactoring focusing on avoiding use of
combination of String and LiteralParameters after first function
processing.

The main advantages of this changes are:
 - Use of semanticaly descriptive structure of TypeSignature instead of general type String.
 - Avoidance of processing String and LiteralParameters in code that does only need TypeSignature.
 - Creation of TypeSignature once instead of creating it on demand from string
   (with LiteralParameters required to proceed).
 - Moving away from Strings ASAP!
  • Loading branch information
fiedukow authored and martint committed May 18, 2016
1 parent af3c997 commit 4a26683
Show file tree
Hide file tree
Showing 48 changed files with 617 additions and 371 deletions.
Expand Up @@ -43,6 +43,7 @@
import javax.annotation.Nullable;

import java.lang.annotation.Annotation;

import java.lang.invoke.MethodHandle;
import java.lang.reflect.AnnotatedElement;
import java.lang.reflect.Method;
Expand All @@ -60,6 +61,7 @@
import static com.facebook.presto.spi.StandardErrorCode.FUNCTION_IMPLEMENTATION_ERROR;
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.util.ImmutableCollectors.toImmutableList;
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;
Expand Down Expand Up @@ -102,10 +104,9 @@ public FunctionListBuilder window(String name, Class<? extends ValueWindowFuncti
WINDOW,
ImmutableList.of(typeVariable(typeVariable)),
ImmutableList.of(),
typeVariable,
ImmutableList.copyOf(argumentTypes),
false,
ImmutableSet.of());
parseTypeSignature(typeVariable),
Arrays.asList(argumentTypes).stream().map(TypeSignature::parseTypeSignature).collect(toImmutableList()),
false);
functions.add(new SqlWindowFunction(new ReflectionWindowFunctionSupplier<>(signature, clazz)));
return this;
}
Expand Down Expand Up @@ -301,8 +302,8 @@ private static Type type(TypeManager typeManager, SqlType explicitType)
private static List<TypeSignature> parameterTypeSignatures(Method method, Set<String> literalParameters)
{
Annotation[][] parameterAnnotations = method.getParameterAnnotations();
ImmutableList.Builder<TypeSignature> parameters = ImmutableList.builder();

ImmutableList.Builder<TypeSignature> types = ImmutableList.builder();
for (int i = 0; i < method.getParameterTypes().length; i++) {
Class<?> clazz = method.getParameterTypes()[i];
// skip session parameters
Expand All @@ -319,9 +320,9 @@ private static List<TypeSignature> parameterTypeSignatures(Method method, Set<St
}
}
checkArgument(explicitType != null, "Method %s argument %s does not have a @SqlType annotation", method, i);
types.add(parseTypeSignature(explicitType.value(), literalParameters));
parameters.add(parseTypeSignature(explicitType.value(), literalParameters));
}
return types.build();
return parameters.build();
}

private static void verifyMethodSignature(Method method, TypeSignature returnTypeName, List<TypeSignature> argumentTypeNames, TypeManager typeManager)
Expand Down
Expand Up @@ -19,16 +19,13 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;

import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;

import static com.facebook.presto.metadata.FunctionKind.SCALAR;
import static com.facebook.presto.metadata.FunctionRegistry.mangleOperatorName;
import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.util.ImmutableCollectors.toImmutableList;
import static java.util.Objects.requireNonNull;
import static java.util.stream.Stream.concat;
Expand Down Expand Up @@ -66,51 +63,7 @@ public Signature(
this.variableArity = variableArity;
}

public Signature(
String name,
FunctionKind kind,
List<TypeVariableConstraint> typeVariableConstraints,
List<LongVariableConstraint> longVariableConstraints,
String returnType,
List<String> argumentTypes,
boolean variableArity)
{
this(name, kind, typeVariableConstraints, longVariableConstraints, returnType, argumentTypes, variableArity, ImmutableSet.of());
}

public Signature(
String name,
FunctionKind kind,
List<TypeVariableConstraint> typeVariableConstraints,
List<LongVariableConstraint> longVariableConstraints,
String returnType,
List<String> argumentTypes,
boolean variableArity,
Set<String> literalParameters)
{
this(name,
kind,
typeVariableConstraints,
longVariableConstraints,
parseTypeSignature(returnType, literalParameters),
argumentTypes.stream().map(argument -> parseTypeSignature(argument, literalParameters)).collect(toImmutableList()),
variableArity
);
}

public Signature(String name, FunctionKind kind, String returnType, List<String> argumentTypes)
{
this(name,
kind,
ImmutableList.<TypeVariableConstraint>of(),
ImmutableList.<LongVariableConstraint>of(),
parseTypeSignature(returnType),
argumentTypes.stream().map(TypeSignature::parseTypeSignature).collect(toImmutableList()),
false
);
}

public Signature(String name, FunctionKind kind, String returnType, String... argumentTypes)
public Signature(String name, FunctionKind kind, TypeSignature returnType, TypeSignature... argumentTypes)
{
this(name, kind, returnType, ImmutableList.copyOf(argumentTypes));
}
Expand All @@ -120,17 +73,12 @@ public Signature(String name, FunctionKind kind, TypeSignature returnType, List<
this(name, kind, ImmutableList.<TypeVariableConstraint>of(), ImmutableList.<LongVariableConstraint>of(), returnType, argumentTypes, false);
}

public Signature(String name, FunctionKind kind, TypeSignature returnType, TypeSignature... argumentTypes)
{
this(name, kind, returnType, ImmutableList.copyOf(argumentTypes));
}

public static Signature internalOperator(OperatorType operator, Type returnType, List<? extends Type> argumentTypes)
{
return internalScalarFunction(mangleOperatorName(operator.name()), returnType.getTypeSignature(), argumentTypes.stream().map(Type::getTypeSignature).collect(toImmutableList()));
}

public static Signature internalOperator(OperatorType operator, String returnType, List<String> argumentTypes)
public static Signature internalOperator(OperatorType operator, TypeSignature returnType, List<TypeSignature> argumentTypes)
{
return internalScalarFunction(mangleOperatorName(operator.name()), returnType, argumentTypes);
}
Expand All @@ -145,16 +93,6 @@ public static Signature internalOperator(String name, TypeSignature returnType,
return internalScalarFunction(mangleOperatorName(name), returnType, ImmutableList.copyOf(argumentTypes));
}

public static Signature internalScalarFunction(String name, String returnType, String... argumentTypes)
{
return internalScalarFunction(name, returnType, ImmutableList.copyOf(argumentTypes));
}

public static Signature internalScalarFunction(String name, String returnType, List<String> argumentTypes)
{
return new Signature(name, SCALAR, ImmutableList.<TypeVariableConstraint>of(), ImmutableList.<LongVariableConstraint>of(), returnType, argumentTypes, false, ImmutableSet.of());
}

public static Signature internalScalarFunction(String name, TypeSignature returnType, TypeSignature... argumentTypes)
{
return internalScalarFunction(name, returnType, ImmutableList.copyOf(argumentTypes));
Expand Down
Expand Up @@ -13,16 +13,15 @@
*/
package com.facebook.presto.metadata;

import java.util.HashSet;
import com.facebook.presto.spi.type.TypeSignature;

import java.util.List;
import java.util.Set;

import static com.facebook.presto.metadata.FunctionKind.SCALAR;
import static com.facebook.presto.metadata.FunctionRegistry.mangleOperatorName;
import static com.google.common.collect.ImmutableList.copyOf;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptySet;
import static java.util.Objects.requireNonNull;

public final class SignatureBuilder
Expand All @@ -31,10 +30,9 @@ public final class SignatureBuilder
private FunctionKind kind;
private List<TypeVariableConstraint> typeVariableConstraints = emptyList();
private List<LongVariableConstraint> longVariableConstraints = emptyList();
private String returnType;
private List<String> argumentTypes;
private TypeSignature returnType;
private List<TypeSignature> argumentTypes = emptyList();
private boolean variableArity;
private Set<String> literalParameters = emptySet();

public SignatureBuilder() {}

Expand Down Expand Up @@ -68,6 +66,12 @@ public SignatureBuilder typeVariableConstraints(List<TypeVariableConstraint> typ
return this;
}

public SignatureBuilder returnType(TypeSignature returnType)
{
this.returnType = requireNonNull(returnType, "returnType is null");
return this;
}

public SignatureBuilder longVariableConstraints(LongVariableConstraint... longVariableConstraints)
{
return longVariableConstraints(asList(requireNonNull(longVariableConstraints, "longVariableConstraints is null")));
Expand All @@ -79,18 +83,12 @@ public SignatureBuilder longVariableConstraints(List<LongVariableConstraint> lon
return this;
}

public SignatureBuilder returnType(String returnType)
{
this.returnType = requireNonNull(returnType, "returnType is null");
return this;
}

public SignatureBuilder argumentTypes(String... argumentTypes)
public SignatureBuilder argumentTypes(TypeSignature... argumentTypes)
{
return argumentTypes(asList(requireNonNull(argumentTypes, "argumentTypes is Null")));
}

public SignatureBuilder argumentTypes(List<String> argumentTypes)
public SignatureBuilder argumentTypes(List<TypeSignature> argumentTypes)
{
this.argumentTypes = copyOf(requireNonNull(argumentTypes, "argumentTypes is null"));
return this;
Expand All @@ -102,19 +100,8 @@ public SignatureBuilder setVariableArity(boolean variableArity)
return this;
}

public SignatureBuilder literalParameters(String... literalParameters)
{
return literalParameters(new HashSet<>(asList(literalParameters)));
}

public SignatureBuilder literalParameters(Set<String> literalParameters)
{
this.literalParameters = literalParameters;
return this;
}

public Signature build()
{
return new Signature(name, kind, typeVariableConstraints, longVariableConstraints, returnType, argumentTypes, variableArity, literalParameters);
return new Signature(name, kind, typeVariableConstraints, longVariableConstraints, returnType, argumentTypes, variableArity);
}
}
Expand Up @@ -19,10 +19,8 @@
import com.facebook.presto.spi.type.TypeSignature;
import com.facebook.presto.util.ImmutableCollectors;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;

import java.util.List;
import java.util.Set;

import static com.facebook.presto.metadata.FunctionKind.AGGREGATE;
import static com.facebook.presto.metadata.FunctionKind.APPROXIMATE_AGGREGATE;
Expand All @@ -43,27 +41,25 @@ protected SqlAggregationFunction(
String name,
List<TypeVariableConstraint> typeVariableConstraints,
List<LongVariableConstraint> longVariableConstraints,
String returnType,
List<String> argumentTypes)
TypeSignature returnType,
List<TypeSignature> argumentTypes)
{
this(name, typeVariableConstraints, longVariableConstraints, returnType, argumentTypes, AGGREGATE, ImmutableSet.of());
this(name, typeVariableConstraints, longVariableConstraints, returnType, argumentTypes, AGGREGATE);
}

protected SqlAggregationFunction(
String name,
List<TypeVariableConstraint> typeVariableConstraints,
List<LongVariableConstraint> longVariableConstraints,
String returnType,
List<String> argumentTypes,
FunctionKind kind,
Set<String> literalParameters)
TypeSignature returnType,
List<TypeSignature> argumentTypes,
FunctionKind kind)
{
requireNonNull(name, "name is null");
requireNonNull(typeVariableConstraints, "typeVariableConstraints is null");
requireNonNull(longVariableConstraints, "longVariableConstraints is null");
requireNonNull(returnType, "returnType is null");
requireNonNull(argumentTypes, "argumentTypes is null");
requireNonNull(literalParameters, "argumentTypes is null");
checkArgument(kind == AGGREGATE || kind == APPROXIMATE_AGGREGATE, "kind must be an aggregate");
this.signature = new Signature(
name,
Expand All @@ -72,8 +68,7 @@ protected SqlAggregationFunction(
ImmutableList.copyOf(longVariableConstraints),
returnType,
ImmutableList.copyOf(argumentTypes),
false,
literalParameters);
false);
}

@Override
Expand Down Expand Up @@ -110,13 +105,11 @@ public SimpleSqlAggregationFunction(
super(name,
ImmutableList.<TypeVariableConstraint>of(),
ImmutableList.<LongVariableConstraint>of(),
function.getFinalType().getTypeSignature().toString(),
function.getFinalType().getTypeSignature(),
function.getParameterTypes().stream()
.map(Type::getTypeSignature)
.map(TypeSignature::toString)
.collect(ImmutableCollectors.toImmutableList()),
function.isApproximate() ? APPROXIMATE_AGGREGATE : AGGREGATE,
ImmutableSet.of());
function.isApproximate() ? APPROXIMATE_AGGREGATE : AGGREGATE);
this.description = description;
this.function = requireNonNull(function, "function is null");
}
Expand Down
Expand Up @@ -15,7 +15,6 @@

import com.facebook.presto.operator.scalar.ScalarFunctionImplementation;
import com.facebook.presto.spi.type.TypeManager;
import com.google.common.collect.ImmutableList;

import java.lang.invoke.MethodHandle;
import java.util.List;
Expand Down Expand Up @@ -91,14 +90,7 @@ public SimpleSqlScalarFunction(
boolean nullable,
List<Boolean> nullableArguments)
{
super(new Signature(
signature.getName(),
FunctionKind.SCALAR,
ImmutableList.of(),
ImmutableList.of(),
signature.getReturnType(),
signature.getArgumentTypes(),
false));
super(signature);
checkArgument(signature.getTypeVariableConstraints().isEmpty(), "%s is parametric", signature);
this.description = description;
this.hidden = hidden;
Expand Down
Expand Up @@ -48,6 +48,7 @@
import static com.facebook.presto.operator.aggregation.AggregationUtils.generateAggregationName;
import static com.facebook.presto.spi.StandardErrorCode.INTERNAL_ERROR;
import static com.facebook.presto.spi.type.BooleanType.BOOLEAN;
import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.util.Reflection.methodHandle;
import static java.util.Objects.requireNonNull;

Expand All @@ -72,7 +73,11 @@ public abstract class AbstractMinMaxAggregationFunction

protected AbstractMinMaxAggregationFunction(String name, OperatorType operatorType)
{
super(name, ImmutableList.of(orderableTypeParameter("E")), ImmutableList.of(), "E", ImmutableList.of("E"));
super(name,
ImmutableList.of(orderableTypeParameter("E")),
ImmutableList.of(),
parseTypeSignature("E"),
ImmutableList.of(parseTypeSignature("E")));
requireNonNull(operatorType);
this.operatorType = operatorType;
}
Expand Down
Expand Up @@ -36,6 +36,7 @@
import static com.facebook.presto.operator.aggregation.AggregationMetadata.ParameterMetadata.ParameterType.NULLABLE_BLOCK_INPUT_CHANNEL;
import static com.facebook.presto.operator.aggregation.AggregationMetadata.ParameterMetadata.ParameterType.STATE;
import static com.facebook.presto.operator.aggregation.AggregationUtils.generateAggregationName;
import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.util.Reflection.methodHandle;
import static java.lang.invoke.MethodHandles.insertArguments;

Expand All @@ -50,7 +51,11 @@ public abstract class AbstractMinMaxBy

protected AbstractMinMaxBy(boolean min)
{
super((min ? "min" : "max") + "_by", ImmutableList.of(orderableTypeParameter("K"), typeVariable("V")), ImmutableList.of(), "V", ImmutableList.of("V", "K"));
super((min ? "min" : "max") + "_by",
ImmutableList.of(orderableTypeParameter("K"), typeVariable("V")),
ImmutableList.of(),
parseTypeSignature("V"),
ImmutableList.of(parseTypeSignature("V"), parseTypeSignature("K")));
this.min = min;
}

Expand Down

0 comments on commit 4a26683

Please sign in to comment.