Skip to content

Commit

Permalink
Remove TypeManager from APIs when FunctionAndTypeManager is available
Browse files Browse the repository at this point in the history
  • Loading branch information
rongrong authored and Rongrong Zhong committed Oct 13, 2020
1 parent 63af935 commit ca07a42
Show file tree
Hide file tree
Showing 114 changed files with 234 additions and 357 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -202,7 +202,7 @@ protected RowExpression toRowExpression(Expression expression, Session session)
expression,
ImmutableList.of(),
WarningCollector.NOOP);
return SqlToRowExpressionTranslator.translate(expression, expressionTypes, ImmutableMap.of(), functionMetadataManager, typeManager, session);
return SqlToRowExpressionTranslator.translate(expression, expressionTypes, ImmutableMap.of(), functionMetadataManager, session);
}

protected LimitNode limit(PlanBuilder pb, long count, PlanNode source)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -429,15 +429,15 @@ public BuiltInFunctionNamespaceManager(
"Unsupported scalar function class: %s",
key.getFunction().getClass());
return key.getFunction() instanceof SqlScalarFunction
? ((SqlScalarFunction) key.getFunction()).specialize(key.getBoundVariables(), key.getArity(), typeManager, functionAndTypeManager)
? ((SqlScalarFunction) key.getFunction()).specialize(key.getBoundVariables(), key.getArity(), functionAndTypeManager)
: new SqlInvokedScalarFunctionImplementation(((SqlInvokedFunction) key.getFunction()).getBody());
}));

specializedAggregationCache = CacheBuilder.newBuilder()
.maximumSize(1000)
.expireAfterWrite(1, HOURS)
.build(CacheLoader.from(key -> ((SqlAggregationFunction) key.getFunction())
.specialize(key.getBoundVariables(), key.getArity(), typeManager, functionAndTypeManager)));
.specialize(key.getBoundVariables(), key.getArity(), functionAndTypeManager)));

specializedWindowCache = CacheBuilder.newBuilder()
.maximumSize(1000)
Expand All @@ -448,7 +448,7 @@ public BuiltInFunctionNamespaceManager(
return supplier(key.getFunction().getSignature(), specializedAggregationCache.getUnchecked(key));
}
return ((SqlWindowFunction) key.getFunction())
.specialize(key.getBoundVariables(), key.getArity(), typeManager, functionAndTypeManager);
.specialize(key.getBoundVariables(), key.getArity(), functionAndTypeManager);
}));

FunctionListBuilder builder = new FunctionListBuilder()
Expand Down Expand Up @@ -1063,7 +1063,7 @@ public String getDescription()
}

@Override
public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
Type parameterType = boundVariables.getTypeVariable("T");
Type type = boundVariables.getTypeVariable("R");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
package com.facebook.presto.metadata;

import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.metadata.PolymorphicScalarFunctionBuilder.MethodAndNativeContainerTypes;
import com.facebook.presto.metadata.PolymorphicScalarFunctionBuilder.MethodsGroup;
import com.facebook.presto.metadata.PolymorphicScalarFunctionBuilder.SpecializeContext;
Expand Down Expand Up @@ -93,26 +92,25 @@ public String getDescription()
}

@Override
public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
public BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
ImmutableList.Builder<ScalarImplementationChoice> implementationChoices = ImmutableList.builder();

for (PolymorphicScalarFunctionChoice choice : choices) {
implementationChoices.add(getScalarFunctionImplementationChoice(boundVariables, typeManager, functionAndTypeManager, choice));
implementationChoices.add(getScalarFunctionImplementationChoice(boundVariables, functionAndTypeManager, choice));
}

return new BuiltInScalarFunctionImplementation(implementationChoices.build());
}

private ScalarImplementationChoice getScalarFunctionImplementationChoice(
BoundVariables boundVariables,
TypeManager typeManager,
FunctionAndTypeManager functionAndTypeManager,
PolymorphicScalarFunctionChoice choice)
{
List<Type> resolvedParameterTypes = applyBoundVariables(typeManager, getSignature().getArgumentTypes(), boundVariables);
Type resolvedReturnType = applyBoundVariables(typeManager, getSignature().getReturnType(), boundVariables);
SpecializeContext context = new SpecializeContext(boundVariables, resolvedParameterTypes, resolvedReturnType, typeManager, functionAndTypeManager);
List<Type> resolvedParameterTypes = applyBoundVariables(functionAndTypeManager, getSignature().getArgumentTypes(), boundVariables);
Type resolvedReturnType = applyBoundVariables(functionAndTypeManager, getSignature().getReturnType(), boundVariables);
SpecializeContext context = new SpecializeContext(boundVariables, resolvedParameterTypes, resolvedReturnType, functionAndTypeManager);
Optional<MethodAndNativeContainerTypes> matchingMethod = Optional.empty();

Optional<MethodsGroup> matchingMethodsGroup = Optional.empty();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@

import com.facebook.presto.common.function.OperatorType;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.metadata.PolymorphicScalarFunction.PolymorphicScalarFunctionChoice;
import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ArgumentProperty;
import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ReturnPlaceConvention;
Expand Down Expand Up @@ -151,14 +150,12 @@ public static final class SpecializeContext
private final BoundVariables boundVariables;
private final List<Type> parameterTypes;
private final Type returnType;
private final TypeManager typeManager;
private final FunctionAndTypeManager functionAndTypeManager;

SpecializeContext(BoundVariables boundVariables, List<Type> parameterTypes, Type returnType, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
SpecializeContext(BoundVariables boundVariables, List<Type> parameterTypes, Type returnType, FunctionAndTypeManager functionAndTypeManager)
{
this.boundVariables = requireNonNull(boundVariables, "boundVariables is null");
this.parameterTypes = requireNonNull(parameterTypes, "parameterTypes is null");
this.typeManager = requireNonNull(typeManager, "typeManager is null");
this.returnType = requireNonNull(returnType, "returnType is null");
this.functionAndTypeManager = requireNonNull(functionAndTypeManager, "functionManager is null");
}
Expand All @@ -183,11 +180,6 @@ public Type getReturnType()
return returnType;
}

public TypeManager getTypeManager()
{
return typeManager;
}

public FunctionAndTypeManager getFunctionManager()
{
return functionAndTypeManager;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
package com.facebook.presto.metadata;

import com.facebook.presto.common.function.QualifiedFunctionName;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.common.type.TypeSignature;
import com.facebook.presto.operator.aggregation.AggregationFromAnnotationsParser;
import com.facebook.presto.operator.aggregation.InternalAggregationFunction;
Expand Down Expand Up @@ -122,5 +121,5 @@ public boolean isDeterministic()
return true;
}

public abstract InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager);
public abstract InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager);
}
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
package com.facebook.presto.metadata;

import com.facebook.presto.common.function.OperatorType;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation;
import com.facebook.presto.spi.function.Signature;

Expand All @@ -39,7 +38,7 @@ public final Signature getSignature()
return signature;
}

public abstract BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager);
public abstract BuiltInScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager);

public static PolymorphicScalarFunctionBuilder builder(Class<?> clazz, OperatorType operatorType)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@
*/
package com.facebook.presto.operator;

import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
import com.facebook.presto.operator.annotations.ImplementationDependency;
Expand All @@ -26,10 +25,10 @@ public class ParametricFunctionHelpers
{
private ParametricFunctionHelpers() {}

public static MethodHandle bindDependencies(MethodHandle handle, List<ImplementationDependency> dependencies, BoundVariables variables, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
public static MethodHandle bindDependencies(MethodHandle handle, List<ImplementationDependency> dependencies, BoundVariables variables, FunctionAndTypeManager functionAndTypeManager)
{
for (ImplementationDependency dependency : dependencies) {
handle = MethodHandles.insertArguments(handle, 0, dependency.resolve(variables, typeManager, functionAndTypeManager));
handle = MethodHandles.insertArguments(handle, 0, dependency.resolve(variables, functionAndTypeManager));
}
return handle;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
import com.facebook.presto.common.block.BlockBuilder;
import com.facebook.presto.common.function.OperatorType;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
import com.facebook.presto.metadata.SqlAggregationFunction;
Expand Down Expand Up @@ -87,7 +86,7 @@ protected AbstractMinMaxAggregationFunction(String name, boolean min)
}

@Override
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
Type type = boundVariables.getTypeVariable("E");
MethodHandle compareMethodHandle = functionAndTypeManager.getBuiltInScalarFunctionImplementation(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
import com.facebook.presto.common.type.ArrayType;
import com.facebook.presto.common.type.StandardTypes;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
import com.facebook.presto.metadata.SqlAggregationFunction;
Expand Down Expand Up @@ -72,7 +71,7 @@ protected AbstractMinMaxNAggregationFunction(String name, Function<Type, BlockCo
}

@Override
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
Type type = boundVariables.getTypeVariable("E");
return generateAggregation(type);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@
import com.facebook.presto.common.block.Block;
import com.facebook.presto.common.block.BlockBuilder;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
import com.facebook.presto.metadata.SqlAggregationFunction;
Expand Down Expand Up @@ -82,7 +81,7 @@ public String getDescription()
}

@Override
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
Type valueType = boundVariables.getTypeVariable("T");
return generateAggregation(valueType);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
import com.facebook.presto.common.block.BlockBuilder;
import com.facebook.presto.common.type.StandardTypes;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
import com.facebook.presto.metadata.SqlAggregationFunction;
Expand Down Expand Up @@ -71,7 +70,7 @@ public String getDescription()
}

@Override
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
Type valueType = boundVariables.getTypeVariable("T");
return generateAggregation(valueType);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
import com.facebook.presto.common.block.BlockBuilder;
import com.facebook.presto.common.type.StandardTypes;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
import com.facebook.presto.metadata.SqlAggregationFunction;
Expand Down Expand Up @@ -68,7 +67,7 @@ public String getDescription()
}

@Override
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
Type type = boundVariables.getTypeVariable("T");
return generateAggregation(type);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
import com.facebook.presto.common.type.DecimalType;
import com.facebook.presto.common.type.Decimals;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.common.type.UnscaledDecimal128Arithmetic;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
Expand Down Expand Up @@ -88,9 +87,9 @@ public String getDescription()
}

@Override
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
Type type = getOnlyElement(applyBoundVariables(typeManager, getSignature().getArgumentTypes(), boundVariables));
Type type = getOnlyElement(applyBoundVariables(functionAndTypeManager, getSignature().getArgumentTypes(), boundVariables));
return generateAggregation(type);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
import com.facebook.presto.common.block.BlockBuilder;
import com.facebook.presto.common.type.DecimalType;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.common.type.UnscaledDecimal128Arithmetic;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
Expand Down Expand Up @@ -81,10 +80,10 @@ public String getDescription()
}

@Override
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
Type inputType = getOnlyElement(applyBoundVariables(typeManager, getSignature().getArgumentTypes(), boundVariables));
Type outputType = applyBoundVariables(typeManager, getSignature().getReturnType(), boundVariables);
Type inputType = getOnlyElement(applyBoundVariables(functionAndTypeManager, getSignature().getArgumentTypes(), boundVariables));
Type outputType = applyBoundVariables(functionAndTypeManager, getSignature().getReturnType(), boundVariables);
return generateAggregation(inputType, outputType);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
import com.facebook.presto.common.type.MapType;
import com.facebook.presto.common.type.StandardTypes;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.common.type.TypeSignatureParameter;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
Expand Down Expand Up @@ -70,11 +69,11 @@ public String getDescription()
}

@Override
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
Type keyType = boundVariables.getTypeVariable("K");
Type valueType = boundVariables.getTypeVariable("V");
MapType outputType = (MapType) typeManager.getParameterizedType(StandardTypes.MAP, ImmutableList.of(
MapType outputType = (MapType) functionAndTypeManager.getParameterizedType(StandardTypes.MAP, ImmutableList.of(
TypeSignatureParameter.of(keyType.getTypeSignature()),
TypeSignatureParameter.of(valueType.getTypeSignature())));
return generateAggregation(keyType, valueType, outputType);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
import com.facebook.presto.common.type.MapType;
import com.facebook.presto.common.type.StandardTypes;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.common.type.TypeSignatureParameter;
import com.facebook.presto.metadata.BoundVariables;
import com.facebook.presto.metadata.FunctionAndTypeManager;
Expand Down Expand Up @@ -64,11 +63,11 @@ public String getDescription()
}

@Override
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionAndTypeManager functionAndTypeManager)
public InternalAggregationFunction specialize(BoundVariables boundVariables, int arity, FunctionAndTypeManager functionAndTypeManager)
{
Type keyType = boundVariables.getTypeVariable("K");
Type valueType = boundVariables.getTypeVariable("V");
MapType outputType = (MapType) typeManager.getParameterizedType(StandardTypes.MAP, ImmutableList.of(
MapType outputType = (MapType) functionAndTypeManager.getParameterizedType(StandardTypes.MAP, ImmutableList.of(
TypeSignatureParameter.of(keyType.getTypeSignature()),
TypeSignatureParameter.of(valueType.getTypeSignature())));
return generateAggregation(keyType, valueType, outputType);
Expand Down
Loading

0 comments on commit ca07a42

Please sign in to comment.