Skip to content

Commit

Permalink
Remove ExpressionAnalyzer dependency on Metadata
Browse files Browse the repository at this point in the history
  • Loading branch information
electrum committed Apr 16, 2015
1 parent 850fab2 commit b59f90e
Show file tree
Hide file tree
Showing 2 changed files with 34 additions and 27 deletions.
Expand Up @@ -15,10 +15,12 @@

import com.facebook.presto.Session;
import com.facebook.presto.metadata.FunctionInfo;
import com.facebook.presto.metadata.FunctionRegistry;
import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.metadata.OperatorNotFoundException;
import com.facebook.presto.metadata.OperatorType;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.spi.type.TypeManager;
import com.facebook.presto.spi.type.TypeSignature;
import com.facebook.presto.sql.parser.SqlParser;
import com.facebook.presto.sql.planner.DependencyExtractor;
Expand Down Expand Up @@ -80,6 +82,7 @@
import java.util.Map.Entry;
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;

import static com.facebook.presto.metadata.FunctionRegistry.canCoerce;
import static com.facebook.presto.metadata.FunctionRegistry.getCommonSuperType;
Expand Down Expand Up @@ -116,25 +119,21 @@

public class ExpressionAnalyzer
{
private final Analysis analysis;
private final Metadata metadata;
private final SqlParser sqlParser;
private final boolean experimentalSyntaxEnabled;
private final Session session;
private final FunctionRegistry functionRegistry;
private final TypeManager typeManager;
private final Function<Node, StatementAnalyzer> statementAnalyzerFactory;
private final Map<QualifiedName, Integer> resolvedNames = new HashMap<>();
private final IdentityHashMap<FunctionCall, FunctionInfo> resolvedFunctions = new IdentityHashMap<>();
private final IdentityHashMap<Expression, Type> expressionTypes = new IdentityHashMap<>();
private final IdentityHashMap<Expression, Type> expressionCoercions = new IdentityHashMap<>();
private final IdentityHashMap<Expression, Boolean> rowFieldReferences = new IdentityHashMap<>();
private final Set<InPredicate> subqueryInPredicates = newIdentityHashSet();

public ExpressionAnalyzer(Analysis analysis, Session session, Metadata metadata, SqlParser sqlParser, boolean experimentalSyntaxEnabled)
public ExpressionAnalyzer(FunctionRegistry functionRegistry, TypeManager typeManager, Function<Node, StatementAnalyzer> statementAnalyzerFactory)
{
this.analysis = checkNotNull(analysis, "analysis is null");
this.session = checkNotNull(session, "session is null");
this.metadata = checkNotNull(metadata, "metadata is null");
this.sqlParser = checkNotNull(sqlParser, "sqlParser is null");
this.experimentalSyntaxEnabled = experimentalSyntaxEnabled;
this.functionRegistry = checkNotNull(functionRegistry, "functionRegistry is null");
this.typeManager = checkNotNull(typeManager, "typeManager is null");
this.statementAnalyzerFactory = checkNotNull(statementAnalyzerFactory, "statementAnalyzerFactory is null");
}

public Map<QualifiedName, Integer> getResolvedNames()
Expand Down Expand Up @@ -524,7 +523,7 @@ protected Type visitSubscriptExpression(SubscriptExpression node, AnalysisContex
protected Type visitArrayConstructor(ArrayConstructor node, AnalysisContext context)
{
Type type = coerceToSingleType(context, "All ARRAY elements must be the same type: %s", node.getValues());
Type arrayType = metadata.getTypeManager().getParameterizedType(ARRAY.getName(), ImmutableList.of(type.getTypeSignature()), ImmutableList.of());
Type arrayType = typeManager.getParameterizedType(ARRAY.getName(), ImmutableList.of(type.getTypeSignature()), ImmutableList.of());
expressionTypes.put(node, arrayType);
return arrayType;
}
Expand Down Expand Up @@ -560,13 +559,13 @@ protected Type visitBooleanLiteral(BooleanLiteral node, AnalysisContext context)
@Override
protected Type visitGenericLiteral(GenericLiteral node, AnalysisContext context)
{
Type type = metadata.getType(parseTypeSignature(node.getType()));
Type type = typeManager.getType(parseTypeSignature(node.getType()));
if (type == null) {
throw new SemanticException(TYPE_MISMATCH, node, "Unknown type: " + node.getType());
}

try {
metadata.getFunctionRegistry().getCoercion(VARCHAR, type);
functionRegistry.getCoercion(VARCHAR, type);
}
catch (IllegalArgumentException e) {
throw new SemanticException(TYPE_MISMATCH, node, "No literal form for type %s", type);
Expand Down Expand Up @@ -669,10 +668,10 @@ protected Type visitFunctionCall(FunctionCall node, AnalysisContext context)
argumentTypes.add(process(expression, context).getTypeSignature());
}

FunctionInfo function = metadata.resolveFunction(node.getName(), argumentTypes.build(), context.isApproximate());
FunctionInfo function = functionRegistry.resolveFunction(node.getName(), argumentTypes.build(), context.isApproximate());
for (int i = 0; i < node.getArguments().size(); i++) {
Expression expression = node.getArguments().get(i);
Type type = metadata.getType(function.getArgumentTypes().get(i));
Type type = typeManager.getType(function.getArgumentTypes().get(i));
checkNotNull(type, "Type %s not found", function.getArgumentTypes().get(i));
if (node.isDistinct() && !type.isComparable()) {
throw new SemanticException(TYPE_MISMATCH, node, "DISTINCT can only be applied to comparable types (actual: %s)", type);
Expand All @@ -681,7 +680,7 @@ protected Type visitFunctionCall(FunctionCall node, AnalysisContext context)
}
resolvedFunctions.put(node, function);

Type type = metadata.getType(function.getReturnType());
Type type = typeManager.getType(function.getReturnType());
expressionTypes.put(node, type);

return type;
Expand Down Expand Up @@ -723,7 +722,7 @@ protected Type visitBetweenPredicate(BetweenPredicate node, AnalysisContext cont
@Override
public Type visitCast(Cast node, AnalysisContext context)
{
Type type = metadata.getType(parseTypeSignature(node.getType()));
Type type = typeManager.getType(parseTypeSignature(node.getType()));
if (type == null) {
throw new SemanticException(TYPE_MISMATCH, node, "Unknown type: " + node.getType());
}
Expand All @@ -735,7 +734,7 @@ public Type visitCast(Cast node, AnalysisContext context)
Type value = process(node.getExpression(), context);
if (!value.equals(UNKNOWN)) {
try {
metadata.getFunctionRegistry().getCoercion(value, type);
functionRegistry.getCoercion(value, type);
}
catch (OperatorNotFoundException e) {
throw new SemanticException(TYPE_MISMATCH, node, "Cannot cast %s to %s", value, type);
Expand Down Expand Up @@ -783,7 +782,7 @@ protected Type visitInListExpression(InListExpression node, AnalysisContext cont
@Override
protected Type visitSubqueryExpression(SubqueryExpression node, AnalysisContext context)
{
StatementAnalyzer analyzer = new StatementAnalyzer(analysis, metadata, sqlParser, session, experimentalSyntaxEnabled, Optional.empty());
StatementAnalyzer analyzer = statementAnalyzerFactory.apply(node);
TupleDescriptor descriptor = analyzer.process(node.getQuery(), context);

// Scalar subqueries should only produce one column
Expand Down Expand Up @@ -823,19 +822,19 @@ private Type getOperator(AnalysisContext context, Expression node, OperatorType

FunctionInfo operatorInfo;
try {
operatorInfo = metadata.resolveOperator(operatorType, argumentTypes.build());
operatorInfo = functionRegistry.resolveOperator(operatorType, argumentTypes.build());
}
catch (OperatorNotFoundException e) {
throw new SemanticException(TYPE_MISMATCH, node, e.getMessage());
}

for (int i = 0; i < arguments.length; i++) {
Expression expression = arguments[i];
Type type = metadata.getType(operatorInfo.getArgumentTypes().get(i));
Type type = typeManager.getType(operatorInfo.getArgumentTypes().get(i));
coerceType(context, expression, type, String.format("Operator %s argument %d", operatorInfo, i));
}

Type type = metadata.getType(operatorInfo.getReturnType());
Type type = typeManager.getType(operatorInfo.getReturnType());
expressionTypes.put(node, type);

return type;
Expand Down Expand Up @@ -993,7 +992,7 @@ private static ExpressionAnalysis analyzeExpressions(
TupleDescriptor tupleDescriptor,
Iterable<? extends Expression> expressions)
{
ExpressionAnalyzer analyzer = new ExpressionAnalyzer(new Analysis(), session, metadata, sqlParser, false);
ExpressionAnalyzer analyzer = create(new Analysis(), session, metadata, sqlParser, false);
for (Expression expression : expressions) {
analyzer.analyze(expression, tupleDescriptor, new AnalysisContext());
}
Expand All @@ -1014,7 +1013,7 @@ public static ExpressionAnalysis analyzeExpression(
AnalysisContext context,
Expression expression)
{
ExpressionAnalyzer analyzer = new ExpressionAnalyzer(analysis, session, metadata, sqlParser, approximateQueriesEnabled);
ExpressionAnalyzer analyzer = create(analysis, session, metadata, sqlParser, approximateQueriesEnabled);
analyzer.analyze(expression, tupleDescriptor, context);

IdentityHashMap<Expression, Type> expressionTypes = analyzer.getExpressionTypes();
Expand All @@ -1034,4 +1033,12 @@ public static ExpressionAnalysis analyzeExpression(

return new ExpressionAnalysis(expressionTypes, expressionCoercions, subqueryInPredicates);
}

public static ExpressionAnalyzer create(Analysis analysis, Session session, Metadata metadata, SqlParser sqlParser, boolean experimentalSyntaxEnabled)
{
return new ExpressionAnalyzer(
metadata.getFunctionRegistry(),
metadata.getTypeManager(),
node -> new StatementAnalyzer(analysis, metadata, sqlParser, session, experimentalSyntaxEnabled, Optional.empty()));
}
}
Expand Up @@ -464,7 +464,7 @@ else if (criteria instanceof JoinOn) {

// ensure all names can be resolved, types match, etc (we don't need to record resolved names, subexpression types, etc. because
// we do it further down when after we determine which subexpressions apply to left vs right tuple)
ExpressionAnalyzer analyzer = new ExpressionAnalyzer(analysis, session, metadata, sqlParser, experimentalSyntaxEnabled);
ExpressionAnalyzer analyzer = ExpressionAnalyzer.create(analysis, session, metadata, sqlParser, experimentalSyntaxEnabled);
analyzer.analyze(expression, output, context);

Analyzer.verifyNoAggregatesOrWindowFunctions(metadata, expression, "JOIN");
Expand All @@ -490,7 +490,7 @@ else if (criteria instanceof JoinOn) {
}
// The optimization above may have rewritten the expression tree which breaks all the identity maps, so redo the analysis
// to re-analyze coercions that might be necessary
analyzer = new ExpressionAnalyzer(analysis, session, metadata, sqlParser, experimentalSyntaxEnabled);
analyzer = ExpressionAnalyzer.create(analysis, session, metadata, sqlParser, experimentalSyntaxEnabled);
analyzer.analyze((Expression) optimizedExpression, output, context);
analysis.addCoercions(analyzer.getExpressionCoercions());

Expand Down

0 comments on commit b59f90e

Please sign in to comment.