Skip to content

Commit

Permalink
Rename TupleDescriptor to RelationType
Browse files Browse the repository at this point in the history
TupleDescriptor is a legacy name. It represents the type of each
element in a relation, so the new name is more accurate.

At some point, when the type system is unified, it should be merged
with RowType
  • Loading branch information
martint committed Nov 5, 2015
1 parent f3ff49c commit 847a2bb
Show file tree
Hide file tree
Showing 12 changed files with 133 additions and 130 deletions.
Expand Up @@ -81,9 +81,9 @@ class AggregationAnalyzer
private final Metadata metadata;
private final Set<Expression> columnReferences;

private final TupleDescriptor tupleDescriptor;
private final RelationType tupleDescriptor;

public AggregationAnalyzer(List<FieldOrExpression> groupByExpressions, Metadata metadata, TupleDescriptor tupleDescriptor, Set<Expression> columnReferences)
public AggregationAnalyzer(List<FieldOrExpression> groupByExpressions, Metadata metadata, RelationType tupleDescriptor, Set<Expression> columnReferences)
{
requireNonNull(groupByExpressions, "groupByExpressions is null");
requireNonNull(metadata, "metadata is null");
Expand Down
Expand Up @@ -51,8 +51,8 @@ public class Analysis

private final IdentityHashMap<Table, Query> namedQueries = new IdentityHashMap<>();

private TupleDescriptor outputDescriptor;
private final IdentityHashMap<Node, TupleDescriptor> outputDescriptors = new IdentityHashMap<>();
private RelationType outputDescriptor;
private final IdentityHashMap<Node, RelationType> outputDescriptors = new IdentityHashMap<>();
private final IdentityHashMap<Expression, Integer> resolvedNames = new IdentityHashMap<>();

private final IdentityHashMap<QuerySpecification, List<FunctionCall>> aggregates = new IdentityHashMap<>();
Expand Down Expand Up @@ -260,22 +260,22 @@ public List<FunctionCall> getWindowFunctions(QuerySpecification query)
return windowFunctions.get(query);
}

public void setOutputDescriptor(TupleDescriptor descriptor)
public void setOutputDescriptor(RelationType descriptor)
{
outputDescriptor = descriptor;
}

public TupleDescriptor getOutputDescriptor()
public RelationType getOutputDescriptor()
{
return outputDescriptor;
}

public void setOutputDescriptor(Node node, TupleDescriptor descriptor)
public void setOutputDescriptor(Node node, RelationType descriptor)
{
outputDescriptors.put(node, descriptor);
}

public TupleDescriptor getOutputDescriptor(Node node)
public RelationType getOutputDescriptor(Node node)
{
Preconditions.checkState(outputDescriptors.containsKey(node), "Output descriptor missing for %s. Broken analysis?", node);
return outputDescriptors.get(node);
Expand Down
Expand Up @@ -23,7 +23,7 @@ public class AnalysisContext
{
private final AnalysisContext parent;
private final Map<String, Query> namedQueries = new HashMap<>();
private TupleDescriptor lateralTupleDescriptor = new TupleDescriptor();
private RelationType lateralTupleDescriptor = new RelationType();
private boolean approximate;

public AnalysisContext(AnalysisContext parent)
Expand All @@ -37,12 +37,12 @@ public AnalysisContext()
parent = null;
}

public void setLateralTupleDescriptor(TupleDescriptor lateralTupleDescriptor)
public void setLateralTupleDescriptor(RelationType lateralTupleDescriptor)
{
this.lateralTupleDescriptor = lateralTupleDescriptor;
}

public TupleDescriptor getLateralTupleDescriptor()
public RelationType getLateralTupleDescriptor()
{
return lateralTupleDescriptor;
}
Expand Down
Expand Up @@ -57,7 +57,7 @@ public Analysis analyze(Statement statement)
{
Analysis analysis = new Analysis();
StatementAnalyzer analyzer = new StatementAnalyzer(analysis, metadata, sqlParser, accessControl, session, experimentalSyntaxEnabled, queryExplainer);
TupleDescriptor outputDescriptor = analyzer.process(statement, new AnalysisContext());
RelationType outputDescriptor = analyzer.process(statement, new AnalysisContext());
analysis.setOutputDescriptor(outputDescriptor);
return analysis;
}
Expand Down
Expand Up @@ -178,7 +178,7 @@ public Set<Expression> getColumnReferences()
* @param tupleDescriptor the tuple descriptor to use to resolve QualifiedNames
* @param context the namespace context of the surrounding query
*/
public Type analyze(Expression expression, TupleDescriptor tupleDescriptor, AnalysisContext context)
public Type analyze(Expression expression, RelationType tupleDescriptor, AnalysisContext context)
{
ScalarSubqueryDetector scalarSubqueryDetector = new ScalarSubqueryDetector();
expression.accept(scalarSubqueryDetector, null);
Expand Down Expand Up @@ -214,9 +214,9 @@ protected Void visitSubqueryExpression(SubqueryExpression node, Void context)
private class Visitor
extends AstVisitor<Type, AnalysisContext>
{
private final TupleDescriptor tupleDescriptor;
private final RelationType tupleDescriptor;

private Visitor(TupleDescriptor tupleDescriptor)
private Visitor(RelationType tupleDescriptor)
{
this.tupleDescriptor = requireNonNull(tupleDescriptor, "tupleDescriptor is null");
}
Expand Down Expand Up @@ -806,7 +806,7 @@ protected Type visitInListExpression(InListExpression node, AnalysisContext cont
protected Type visitSubqueryExpression(SubqueryExpression node, AnalysisContext context)
{
StatementAnalyzer analyzer = statementAnalyzerFactory.apply(node);
TupleDescriptor descriptor = analyzer.process(node.getQuery(), context);
RelationType descriptor = analyzer.process(node.getQuery(), context);

// Scalar subqueries should only produce one column
if (descriptor.getVisibleFieldCount() != 1) {
Expand Down Expand Up @@ -989,7 +989,7 @@ public static ExpressionAnalysis analyzeExpressionsWithSymbols(
})
.collect(toImmutableList());

return analyzeExpressions(session, metadata, sqlParser, new TupleDescriptor(fields), expressions);
return analyzeExpressions(session, metadata, sqlParser, new RelationType(fields), expressions);
}

private static ExpressionAnalysis analyzeExpressionsWithInputs(
Expand All @@ -1003,7 +1003,7 @@ private static ExpressionAnalysis analyzeExpressionsWithInputs(
for (Entry<Integer, Type> entry : types.entrySet()) {
fields[entry.getKey()] = Field.newUnqualified(Optional.empty(), entry.getValue());
}
TupleDescriptor tupleDescriptor = new TupleDescriptor(fields);
RelationType tupleDescriptor = new RelationType(fields);

return analyzeExpressions(session, metadata, sqlParser, tupleDescriptor, expressions);
}
Expand All @@ -1012,7 +1012,7 @@ private static ExpressionAnalysis analyzeExpressions(
Session session,
Metadata metadata,
SqlParser sqlParser,
TupleDescriptor tupleDescriptor,
RelationType tupleDescriptor,
Iterable<? extends Expression> expressions)
{
// expressions at this point can not have sub queries so deny all access checks
Expand All @@ -1034,7 +1034,7 @@ public static ExpressionAnalysis analyzeExpression(
Metadata metadata,
AccessControl accessControl,
SqlParser sqlParser,
TupleDescriptor tupleDescriptor,
RelationType tupleDescriptor,
Analysis analysis,
boolean approximateQueriesEnabled,
AnalysisContext context,
Expand Down
Expand Up @@ -35,20 +35,23 @@
import static com.google.common.base.Predicates.not;
import static java.util.Objects.requireNonNull;

/**
* TODO: this needs to be merged with RowType at some point (when the type system is unified)
*/
@Immutable
public class TupleDescriptor
public class RelationType
{
private final List<Field> visibleFields;
private final List<Field> allFields;

private final Map<Field, Integer> fieldIndexes;

public TupleDescriptor(Field... fields)
public RelationType(Field... fields)
{
this(ImmutableList.copyOf(fields));
}

public TupleDescriptor(List<Field> fields)
public RelationType(List<Field> fields)
{
requireNonNull(fields, "fields is null");
this.allFields = ImmutableList.copyOf(fields);
Expand Down Expand Up @@ -154,20 +157,20 @@ public Predicate<QualifiedName> canResolvePredicate()
* Creates a new tuple descriptor containing all fields from this tuple descriptor
* and all fields from the specified tuple descriptor.
*/
public TupleDescriptor joinWith(TupleDescriptor other)
public RelationType joinWith(RelationType other)
{
List<Field> fields = ImmutableList.<Field>builder()
.addAll(this.allFields)
.addAll(other.allFields)
.build();

return new TupleDescriptor(fields);
return new RelationType(fields);
}

/**
* Creates a new tuple descriptor with the relation, and, optionally, the columns aliased.
*/
public TupleDescriptor withAlias(String relationAlias, List<String> columnAliases)
public RelationType withAlias(String relationAlias, List<String> columnAliases)
{
if (columnAliases != null) {
checkArgument(columnAliases.size() == visibleFields.size(),
Expand All @@ -191,15 +194,15 @@ else if (!field.isHidden()) {
}
}

return new TupleDescriptor(fieldsBuilder.build());
return new RelationType(fieldsBuilder.build());
}

/**
* Creates a new tuple descriptor containing only the visible fields.
*/
public TupleDescriptor withOnlyVisibleFields()
public RelationType withOnlyVisibleFields()
{
return new TupleDescriptor(visibleFields);
return new RelationType(visibleFields);
}

@Override
Expand Down

0 comments on commit 847a2bb

Please sign in to comment.