/* * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package com.facebook.presto.sql.planner; import com.facebook.presto.Session; import com.facebook.presto.SystemSessionProperties; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.Signature; import com.facebook.presto.metadata.TableHandle; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.block.SortOrder; import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.type.Type; import com.facebook.presto.sql.analyzer.Analysis; import com.facebook.presto.sql.analyzer.Field; import com.facebook.presto.sql.analyzer.RelationId; import com.facebook.presto.sql.analyzer.RelationType; import com.facebook.presto.sql.analyzer.Scope; import com.facebook.presto.sql.planner.plan.AggregationNode; import com.facebook.presto.sql.planner.plan.Assignments; import com.facebook.presto.sql.planner.plan.DeleteNode; import com.facebook.presto.sql.planner.plan.FilterNode; import com.facebook.presto.sql.planner.plan.GroupIdNode; import com.facebook.presto.sql.planner.plan.LimitNode; import com.facebook.presto.sql.planner.plan.MarkDistinctNode; import com.facebook.presto.sql.planner.plan.PlanNode; import com.facebook.presto.sql.planner.plan.ProjectNode; import com.facebook.presto.sql.planner.plan.SortNode; import com.facebook.presto.sql.planner.plan.TableScanNode; import com.facebook.presto.sql.planner.plan.TableWriterNode.DeleteHandle; import com.facebook.presto.sql.planner.plan.TopNNode; import com.facebook.presto.sql.planner.plan.ValuesNode; import com.facebook.presto.sql.planner.plan.WindowNode; import com.facebook.presto.sql.tree.Cast; import com.facebook.presto.sql.tree.Delete; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.ExpressionTreeRewriter; import com.facebook.presto.sql.tree.FieldReference; import com.facebook.presto.sql.tree.FrameBound; import com.facebook.presto.sql.tree.FunctionCall; import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; import com.facebook.presto.sql.tree.Node; import com.facebook.presto.sql.tree.OrderBy; import com.facebook.presto.sql.tree.Query; import com.facebook.presto.sql.tree.QuerySpecification; import com.facebook.presto.sql.tree.SortItem; import com.facebook.presto.sql.tree.SortItem.NullOrdering; import com.facebook.presto.sql.tree.SortItem.Ordering; import com.facebook.presto.sql.tree.SymbolReference; import com.facebook.presto.sql.tree.Window; import com.facebook.presto.sql.tree.WindowFrame; import com.facebook.presto.util.maps.IdentityLinkedHashMap; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.StreamSupport; import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY; import static com.facebook.presto.sql.NodeUtils.getSortItemsFromOrderBy; import static com.google.common.base.MoreObjects.firstNonNull; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.Iterables.getOnlyElement; import static java.util.Objects.requireNonNull; class QueryPlanner { private final Analysis analysis; private final SymbolAllocator symbolAllocator; private final PlanNodeIdAllocator idAllocator; private final IdentityLinkedHashMap<LambdaArgumentDeclaration, Symbol> lambdaDeclarationToSymbolMap; private final Metadata metadata; private final Session session; private final SubqueryPlanner subqueryPlanner; QueryPlanner( Analysis analysis, SymbolAllocator symbolAllocator, PlanNodeIdAllocator idAllocator, IdentityLinkedHashMap<LambdaArgumentDeclaration, Symbol> lambdaDeclarationToSymbolMap, Metadata metadata, Session session) { requireNonNull(analysis, "analysis is null"); requireNonNull(symbolAllocator, "symbolAllocator is null"); requireNonNull(idAllocator, "idAllocator is null"); requireNonNull(lambdaDeclarationToSymbolMap, "lambdaDeclarationToSymbolMap is null"); requireNonNull(metadata, "metadata is null"); requireNonNull(session, "session is null"); this.analysis = analysis; this.symbolAllocator = symbolAllocator; this.idAllocator = idAllocator; this.lambdaDeclarationToSymbolMap = lambdaDeclarationToSymbolMap; this.metadata = metadata; this.session = session; this.subqueryPlanner = new SubqueryPlanner(analysis, symbolAllocator, idAllocator, lambdaDeclarationToSymbolMap, metadata, session, analysis.getParameters()); } public RelationPlan plan(Query query) { PlanBuilder builder = planQueryBody(query); List<Expression> orderBy = analysis.getOrderByExpressions(query); builder = handleSubqueries(builder, query, orderBy); List<Expression> outputs = analysis.getOutputExpressions(query); builder = handleSubqueries(builder, query, outputs); builder = project(builder, Iterables.concat(orderBy, outputs)); builder = sort(builder, query); builder = project(builder, analysis.getOutputExpressions(query)); builder = limit(builder, query); return new RelationPlan( builder.getRoot(), analysis.getScope(query), computeOutputs(builder, analysis.getOutputExpressions(query))); } public RelationPlan plan(QuerySpecification node) { PlanBuilder builder = planFrom(node); RelationPlan fromRelationPlan = builder.getRelationPlan(); builder = filter(builder, analysis.getWhere(node), node); builder = aggregate(builder, node); builder = filter(builder, analysis.getHaving(node), node); builder = window(builder, node); List<Expression> outputs = analysis.getOutputExpressions(node); builder = handleSubqueries(builder, node, outputs); if (node.getOrderBy().isPresent() && !SystemSessionProperties.isLegacyOrderByEnabled(session)) { if (analysis.getGroupingSets(node).isEmpty()) { // ORDER BY requires both output and source fields to be visible if there are no aggregations builder = project(builder, outputs, fromRelationPlan); outputs = toSymbolReferences(computeOutputs(builder, outputs)); builder = planBuilderFor(builder, analysis.getScope(node.getOrderBy().get())); } else { // ORDER BY requires output fields, groups and translated aggregations to be visible for queries with aggregation List<Expression> orderByAggregates = analysis.getOrderByAggregates(node.getOrderBy().get()); builder = project(builder, Iterables.concat(outputs, orderByAggregates)); outputs = toSymbolReferences(computeOutputs(builder, outputs)); builder = planBuilderFor(builder, analysis.getScope(node.getOrderBy().get()), orderByAggregates); } builder = window(builder, node.getOrderBy().get()); } List<Expression> orderBy = analysis.getOrderByExpressions(node); builder = handleSubqueries(builder, node, orderBy); builder = project(builder, Iterables.concat(orderBy, outputs)); builder = distinct(builder, node); builder = sort(builder, node); builder = project(builder, outputs); builder = limit(builder, node); return new RelationPlan( builder.getRoot(), analysis.getScope(node), computeOutputs(builder, outputs)); } public DeleteNode plan(Delete node) { RelationType descriptor = analysis.getOutputDescriptor(node.getTable()); TableHandle handle = analysis.getTableHandle(node.getTable()); ColumnHandle rowIdHandle = metadata.getUpdateRowIdColumnHandle(session, handle); Type rowIdType = metadata.getColumnMetadata(session, handle, rowIdHandle).getType(); // add table columns ImmutableList.Builder<Symbol> outputSymbols = ImmutableList.builder(); ImmutableMap.Builder<Symbol, ColumnHandle> columns = ImmutableMap.builder(); ImmutableList.Builder<Field> fields = ImmutableList.builder(); for (Field field : descriptor.getAllFields()) { Symbol symbol = symbolAllocator.newSymbol(field.getName().get(), field.getType()); outputSymbols.add(symbol); columns.put(symbol, analysis.getColumn(field)); fields.add(field); } // add rowId column Field rowIdField = Field.newUnqualified(Optional.empty(), rowIdType); Symbol rowIdSymbol = symbolAllocator.newSymbol("$rowId", rowIdField.getType()); outputSymbols.add(rowIdSymbol); columns.put(rowIdSymbol, rowIdHandle); fields.add(rowIdField); // create table scan PlanNode tableScan = new TableScanNode(idAllocator.getNextId(), handle, outputSymbols.build(), columns.build(), Optional.empty(), TupleDomain.all(), null); Scope scope = Scope.builder().withRelationType(RelationId.anonymous(), new RelationType(fields.build())).build(); RelationPlan relationPlan = new RelationPlan(tableScan, scope, outputSymbols.build()); TranslationMap translations = new TranslationMap(relationPlan, analysis, lambdaDeclarationToSymbolMap); translations.setFieldMappings(relationPlan.getFieldMappings()); PlanBuilder builder = new PlanBuilder(translations, relationPlan.getRoot(), analysis.getParameters()); if (node.getWhere().isPresent()) { builder = filter(builder, node.getWhere().get(), node); } // create delete node Symbol rowId = builder.translate(new FieldReference(relationPlan.getDescriptor().indexOf(rowIdField))); List<Symbol> outputs = ImmutableList.of( symbolAllocator.newSymbol("partialrows", BIGINT), symbolAllocator.newSymbol("fragment", VARBINARY)); return new DeleteNode(idAllocator.getNextId(), builder.getRoot(), new DeleteHandle(handle, metadata.getTableMetadata(session, handle).getTable()), rowId, outputs); } private static List<Symbol> computeOutputs(PlanBuilder builder, List<Expression> outputExpressions) { ImmutableList.Builder<Symbol> outputSymbols = ImmutableList.builder(); for (Expression expression : outputExpressions) { outputSymbols.add(builder.translate(expression)); } return outputSymbols.build(); } private PlanBuilder planQueryBody(Query query) { RelationPlan relationPlan = new RelationPlanner(analysis, symbolAllocator, idAllocator, lambdaDeclarationToSymbolMap, metadata, session) .process(query.getQueryBody(), null); return planBuilderFor(relationPlan); } private PlanBuilder planFrom(QuerySpecification node) { RelationPlan relationPlan; if (node.getFrom().isPresent()) { relationPlan = new RelationPlanner(analysis, symbolAllocator, idAllocator, lambdaDeclarationToSymbolMap, metadata, session) .process(node.getFrom().get(), null); } else { relationPlan = planImplicitTable(); } return planBuilderFor(relationPlan); } private PlanBuilder planBuilderFor(PlanBuilder builder, Scope scope, Iterable<? extends Expression> expressionsToRemap) { Map<Expression, Symbol> expressionsToSymbols = symbolsForExpressions(builder, expressionsToRemap); PlanBuilder newBuilder = planBuilderFor(builder, scope); expressionsToSymbols.entrySet() .forEach(entry -> newBuilder.getTranslations().put(entry.getKey(), entry.getValue())); return newBuilder; } private PlanBuilder planBuilderFor(PlanBuilder builder, Scope scope) { return planBuilderFor(new RelationPlan(builder.getRoot(), scope, builder.getRoot().getOutputSymbols())); } private PlanBuilder planBuilderFor(RelationPlan relationPlan) { TranslationMap translations = new TranslationMap(relationPlan, analysis, lambdaDeclarationToSymbolMap); // Make field->symbol mapping from underlying relation plan available for translations // This makes it possible to rewrite FieldOrExpressions that reference fields from the FROM clause directly translations.setFieldMappings(relationPlan.getFieldMappings()); return new PlanBuilder(translations, relationPlan.getRoot(), analysis.getParameters()); } private RelationPlan planImplicitTable() { List<Expression> emptyRow = ImmutableList.of(); Scope scope = Scope.create(); return new RelationPlan( new ValuesNode(idAllocator.getNextId(), ImmutableList.of(), ImmutableList.of(emptyRow)), scope, ImmutableList.of()); } private PlanBuilder filter(PlanBuilder subPlan, Expression predicate, Node node) { if (predicate == null) { return subPlan; } // rewrite expressions which contain already handled subqueries predicate = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters(), analysis), predicate); Expression rewrittenBeforeSubqueries = subPlan.rewrite(predicate); subPlan = subqueryPlanner.handleSubqueries(subPlan, rewrittenBeforeSubqueries, node); predicate = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters(), analysis), predicate); Expression rewrittenAfterSubqueries = subPlan.rewrite(predicate); return subPlan.withNewRoot(new FilterNode(idAllocator.getNextId(), subPlan.getRoot(), rewrittenAfterSubqueries)); } private PlanBuilder project(PlanBuilder subPlan, Iterable<Expression> expressions, RelationPlan parentRelationPlan) { return project(subPlan, Iterables.concat(expressions, toSymbolReferences(parentRelationPlan.getFieldMappings()))); } private PlanBuilder project(PlanBuilder subPlan, Iterable<Expression> expressions) { TranslationMap outputTranslations = new TranslationMap(subPlan.getRelationPlan(), analysis, lambdaDeclarationToSymbolMap); Assignments.Builder projections = Assignments.builder(); for (Expression expression : expressions) { if (expression instanceof SymbolReference) { Symbol symbol = Symbol.from(expression); projections.put(symbol, expression); outputTranslations.put(expression, symbol); continue; } Expression rewritten = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters(), analysis), expression); Symbol symbol = symbolAllocator.newSymbol(rewritten, analysis.getTypeWithCoercions(expression)); projections.put(symbol, subPlan.rewrite(rewritten)); outputTranslations.addIntermediateMapping(expression, rewritten); outputTranslations.put(rewritten, symbol); } return new PlanBuilder(outputTranslations, new ProjectNode( idAllocator.getNextId(), subPlan.getRoot(), projections.build()), analysis.getParameters()); } private Map<Symbol, Expression> coerce(Iterable<? extends Expression> expressions, PlanBuilder subPlan, TranslationMap translations) { ImmutableMap.Builder<Symbol, Expression> projections = ImmutableMap.builder(); for (Expression expression : expressions) { Type type = analysis.getType(expression); Type coercion = analysis.getCoercion(expression); Symbol symbol = symbolAllocator.newSymbol(expression, firstNonNull(coercion, type)); Expression parametersReplaced = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters(), analysis), expression); translations.addIntermediateMapping(expression, parametersReplaced); Expression rewritten = subPlan.rewrite(expression); if (coercion != null) { rewritten = new Cast( rewritten, coercion.getTypeSignature().toString(), false, metadata.getTypeManager().isTypeOnlyCoercion(type, coercion)); } projections.put(symbol, rewritten); translations.put(parametersReplaced, symbol); } return projections.build(); } private PlanBuilder explicitCoercionFields(PlanBuilder subPlan, Iterable<Expression> alreadyCoerced, Iterable<? extends Expression> uncoerced) { TranslationMap translations = new TranslationMap(subPlan.getRelationPlan(), analysis, lambdaDeclarationToSymbolMap); Assignments.Builder projections = Assignments.builder(); projections.putAll(coerce(uncoerced, subPlan, translations)); for (Expression expression : alreadyCoerced) { Symbol symbol = symbolAllocator.newSymbol(expression, analysis.getType(expression)); Expression parametersReplaced = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters(), analysis), expression); translations.addIntermediateMapping(expression, parametersReplaced); Expression rewritten = subPlan.rewrite(expression); projections.put(symbol, rewritten); translations.put(parametersReplaced, symbol); } return new PlanBuilder(translations, new ProjectNode( idAllocator.getNextId(), subPlan.getRoot(), projections.build()), analysis.getParameters()); } private PlanBuilder explicitCoercionSymbols(PlanBuilder subPlan, Iterable<Symbol> alreadyCoerced, Iterable<? extends Expression> uncoerced) { TranslationMap translations = subPlan.copyTranslations(); Assignments assignments = Assignments.builder().putAll(coerce(uncoerced, subPlan, translations)) .putAll(Assignments.identity(alreadyCoerced)) .build(); return new PlanBuilder(translations, new ProjectNode( idAllocator.getNextId(), subPlan.getRoot(), assignments), analysis.getParameters()); } private PlanBuilder aggregate(PlanBuilder subPlan, QuerySpecification node) { List<List<Expression>> groupingSets = analysis.getGroupingSets(node); if (groupingSets.isEmpty()) { return subPlan; } // 1. Pre-project all scalar inputs (arguments and non-trivial group by expressions) Set<Expression> distinctGroupingColumns = groupingSets.stream() .flatMap(Collection::stream) .collect(toImmutableSet()); ImmutableList.Builder<Expression> arguments = ImmutableList.builder(); analysis.getAggregates(node).stream() .map(FunctionCall::getArguments) .flatMap(List::stream) .forEach(arguments::add); // filter expressions need to be projected first analysis.getAggregates(node).stream() .map(FunctionCall::getFilter) .filter(Optional::isPresent) .map(Optional::get) .forEach(arguments::add); Iterable<Expression> inputs = Iterables.concat(distinctGroupingColumns, arguments.build()); subPlan = handleSubqueries(subPlan, node, inputs); if (!Iterables.isEmpty(inputs)) { // avoid an empty projection if the only aggregation is COUNT (which has no arguments) subPlan = project(subPlan, inputs); } // 2. Aggregate // 2.a. Rewrite aggregate arguments TranslationMap argumentTranslations = new TranslationMap(subPlan.getRelationPlan(), analysis, lambdaDeclarationToSymbolMap); ImmutableMap.Builder<Symbol, Symbol> argumentMappingBuilder = ImmutableMap.builder(); for (Expression argument : arguments.build()) { Expression parametersReplaced = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters(), analysis), argument); argumentTranslations.addIntermediateMapping(argument, parametersReplaced); Symbol input = subPlan.translate(parametersReplaced); if (!argumentTranslations.containsSymbol(parametersReplaced)) { Symbol output = symbolAllocator.newSymbol(parametersReplaced, analysis.getTypeWithCoercions(parametersReplaced), "arg"); argumentMappingBuilder.put(output, input); argumentTranslations.put(parametersReplaced, output); } } Map<Symbol, Symbol> argumentMappings = argumentMappingBuilder.build(); // 2.b. Rewrite grouping columns TranslationMap groupingTranslations = new TranslationMap(subPlan.getRelationPlan(), analysis, lambdaDeclarationToSymbolMap); Map<Symbol, Symbol> groupingSetMappings = new HashMap<>(); List<List<Symbol>> groupingSymbols = new ArrayList<>(); for (List<Expression> groupingSet : groupingSets) { ImmutableList.Builder<Symbol> symbols = ImmutableList.builder(); for (Expression expression : groupingSet) { Expression parametersReplaced = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters(), analysis), expression); groupingTranslations.addIntermediateMapping(expression, parametersReplaced); Symbol input = subPlan.translate(expression); Symbol output; if (!groupingTranslations.containsSymbol(parametersReplaced)) { output = symbolAllocator.newSymbol(parametersReplaced, analysis.getTypeWithCoercions(expression), "gid"); groupingTranslations.put(parametersReplaced, output); } else { output = groupingTranslations.get(parametersReplaced); } groupingSetMappings.put(output, input); symbols.add(output); } groupingSymbols.add(symbols.build()); } // 2.c. Generate GroupIdNode (multiple grouping sets) or ProjectNode (single grouping set) Optional<Symbol> groupIdSymbol = Optional.empty(); if (groupingSets.size() > 1) { groupIdSymbol = Optional.of(symbolAllocator.newSymbol("groupId", BIGINT)); GroupIdNode groupId = new GroupIdNode(idAllocator.getNextId(), subPlan.getRoot(), groupingSymbols, groupingSetMappings, argumentMappings, groupIdSymbol.get()); subPlan = new PlanBuilder(groupingTranslations, groupId, analysis.getParameters()); } else { Assignments.Builder assignments = Assignments.builder(); for (Symbol output : argumentMappings.keySet()) { assignments.put(output, argumentMappings.get(output).toSymbolReference()); } for (Symbol output : groupingSetMappings.keySet()) { assignments.put(output, groupingSetMappings.get(output).toSymbolReference()); } ProjectNode project = new ProjectNode(idAllocator.getNextId(), subPlan.getRoot(), assignments.build()); subPlan = new PlanBuilder(groupingTranslations, project, analysis.getParameters()); } TranslationMap aggregationTranslations = new TranslationMap(subPlan.getRelationPlan(), analysis, lambdaDeclarationToSymbolMap); aggregationTranslations.copyMappingsFrom(groupingTranslations); // 2.d. Rewrite aggregates ImmutableMap.Builder<Symbol, FunctionCall> aggregationAssignments = ImmutableMap.builder(); ImmutableMap.Builder<Symbol, Signature> functions = ImmutableMap.builder(); boolean needPostProjectionCoercion = false; for (FunctionCall aggregate : analysis.getAggregates(node)) { Expression parametersReplaced = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters(), analysis), aggregate); aggregationTranslations.addIntermediateMapping(aggregate, parametersReplaced); Expression rewritten = argumentTranslations.rewrite(parametersReplaced); Symbol newSymbol = symbolAllocator.newSymbol(rewritten, analysis.getType(aggregate)); // TODO: this is a hack, because we apply coercions to the output of expressions, rather than the arguments to expressions. // Therefore we can end up with this implicit cast, and have to move it into a post-projection if (rewritten instanceof Cast) { rewritten = ((Cast) rewritten).getExpression(); needPostProjectionCoercion = true; } aggregationAssignments.put(newSymbol, (FunctionCall) rewritten); aggregationTranslations.put(parametersReplaced, newSymbol); functions.put(newSymbol, analysis.getFunctionSignature(aggregate)); } // 2.e. Mark distinct rows for each aggregate that has DISTINCT // Map from aggregate function arguments to marker symbols, so that we can reuse the markers, if two aggregates have the same argument Map<Set<Expression>, Symbol> argumentMarkers = new HashMap<>(); // Map from aggregate functions to marker symbols Map<Symbol, Symbol> masks = new HashMap<>(); for (FunctionCall aggregate : Iterables.filter(analysis.getAggregates(node), FunctionCall::isDistinct)) { Set<Expression> args = ImmutableSet.copyOf(aggregate.getArguments()); Symbol marker = argumentMarkers.get(args); Symbol aggregateSymbol = aggregationTranslations.get(aggregate); if (marker == null) { if (args.size() == 1) { marker = symbolAllocator.newSymbol(getOnlyElement(args), BOOLEAN, "distinct"); } else { marker = symbolAllocator.newSymbol(aggregateSymbol.getName(), BOOLEAN, "distinct"); } argumentMarkers.put(args, marker); } masks.put(aggregateSymbol, marker); } for (Map.Entry<Set<Expression>, Symbol> entry : argumentMarkers.entrySet()) { ImmutableList.Builder<Symbol> builder = ImmutableList.builder(); builder.addAll(groupingSymbols.stream() .flatMap(Collection::stream) .distinct() .collect(Collectors.toList())); groupIdSymbol.ifPresent(builder::add); for (Expression expression : entry.getKey()) { builder.add(argumentTranslations.get(expression)); } subPlan = subPlan.withNewRoot( new MarkDistinctNode( idAllocator.getNextId(), subPlan.getRoot(), entry.getValue(), builder.build(), Optional.empty())); } AggregationNode aggregationNode = new AggregationNode( idAllocator.getNextId(), subPlan.getRoot(), aggregationAssignments.build(), functions.build(), masks, groupingSymbols, AggregationNode.Step.SINGLE, Optional.empty(), groupIdSymbol); subPlan = new PlanBuilder(aggregationTranslations, aggregationNode, analysis.getParameters()); // 3. Post-projection // Add back the implicit casts that we removed in 2.a // TODO: this is a hack, we should change type coercions to coerce the inputs to functions/operators instead of coercing the output if (needPostProjectionCoercion) { return explicitCoercionFields(subPlan, distinctGroupingColumns, analysis.getAggregates(node)); } return subPlan; } private PlanBuilder window(PlanBuilder subPlan, OrderBy node) { return window(subPlan, ImmutableList.copyOf(analysis.getOrderByWindowFunctions(node))); } private PlanBuilder window(PlanBuilder subPlan, QuerySpecification node) { return window(subPlan, ImmutableList.copyOf(analysis.getWindowFunctions(node))); } private PlanBuilder window(PlanBuilder subPlan, List<FunctionCall> windowFunctions) { if (windowFunctions.isEmpty()) { return subPlan; } for (FunctionCall windowFunction : windowFunctions) { Window window = windowFunction.getWindow().get(); // Extract frame WindowFrame.Type frameType = WindowFrame.Type.RANGE; FrameBound.Type frameStartType = FrameBound.Type.UNBOUNDED_PRECEDING; FrameBound.Type frameEndType = FrameBound.Type.CURRENT_ROW; Expression frameStart = null; Expression frameEnd = null; if (window.getFrame().isPresent()) { WindowFrame frame = window.getFrame().get(); frameType = frame.getType(); frameStartType = frame.getStart().getType(); frameStart = frame.getStart().getValue().orElse(null); if (frame.getEnd().isPresent()) { frameEndType = frame.getEnd().get().getType(); frameEnd = frame.getEnd().get().getValue().orElse(null); } } // Pre-project inputs ImmutableList.Builder<Expression> inputs = ImmutableList.<Expression>builder() .addAll(windowFunction.getArguments()) .addAll(window.getPartitionBy()) .addAll(Iterables.transform(getSortItemsFromOrderBy(window.getOrderBy()), SortItem::getSortKey)); if (frameStart != null) { inputs.add(frameStart); } if (frameEnd != null) { inputs.add(frameEnd); } subPlan = subPlan.appendProjections(inputs.build(), symbolAllocator, idAllocator); // Rewrite PARTITION BY in terms of pre-projected inputs ImmutableList.Builder<Symbol> partitionBySymbols = ImmutableList.builder(); for (Expression expression : window.getPartitionBy()) { partitionBySymbols.add(subPlan.translate(expression)); } // Rewrite ORDER BY in terms of pre-projected inputs Map<Symbol, SortOrder> orderings = new LinkedHashMap<>(); for (SortItem item : getSortItemsFromOrderBy(window.getOrderBy())) { Symbol symbol = subPlan.translate(item.getSortKey()); orderings.put(symbol, toSortOrder(item)); } // Rewrite frame bounds in terms of pre-projected inputs Optional<Symbol> frameStartSymbol = Optional.empty(); Optional<Symbol> frameEndSymbol = Optional.empty(); if (frameStart != null) { frameStartSymbol = Optional.of(subPlan.translate(frameStart)); } if (frameEnd != null) { frameEndSymbol = Optional.of(subPlan.translate(frameEnd)); } WindowNode.Frame frame = new WindowNode.Frame(frameType, frameStartType, frameStartSymbol, frameEndType, frameEndSymbol); TranslationMap outputTranslations = subPlan.copyTranslations(); // Rewrite function call in terms of pre-projected inputs Expression parametersReplaced = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters(), analysis), windowFunction); outputTranslations.addIntermediateMapping(windowFunction, parametersReplaced); Expression rewritten = subPlan.rewrite(parametersReplaced); boolean needCoercion = rewritten instanceof Cast; // Strip out the cast and add it back as a post-projection if (rewritten instanceof Cast) { rewritten = ((Cast) rewritten).getExpression(); } // If refers to existing symbol, don't create another PlanNode if (rewritten instanceof SymbolReference) { if (needCoercion) { subPlan = explicitCoercionSymbols(subPlan, subPlan.getRoot().getOutputSymbols(), ImmutableList.of(windowFunction)); } continue; } Symbol newSymbol = symbolAllocator.newSymbol(rewritten, analysis.getType(windowFunction)); outputTranslations.put(parametersReplaced, newSymbol); WindowNode.Function function = new WindowNode.Function( (FunctionCall) rewritten, analysis.getFunctionSignature(windowFunction), frame); List<Symbol> sourceSymbols = subPlan.getRoot().getOutputSymbols(); ImmutableList.Builder<Symbol> orderBySymbols = ImmutableList.builder(); orderBySymbols.addAll(orderings.keySet()); // create window node subPlan = new PlanBuilder(outputTranslations, new WindowNode( idAllocator.getNextId(), subPlan.getRoot(), new WindowNode.Specification( partitionBySymbols.build(), orderBySymbols.build(), orderings), ImmutableMap.of(newSymbol, function), Optional.empty(), ImmutableSet.of(), 0), analysis.getParameters()); if (needCoercion) { subPlan = explicitCoercionSymbols(subPlan, sourceSymbols, ImmutableList.of(windowFunction)); } } return subPlan; } private PlanBuilder handleSubqueries(PlanBuilder subPlan, Node node, Iterable<Expression> inputs) { for (Expression input : inputs) { input = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters(), analysis), input); Expression rewritten = subPlan.rewrite(input); subPlan = subqueryPlanner.handleSubqueries(subPlan, rewritten, node); } return subPlan; } private PlanBuilder distinct(PlanBuilder subPlan, QuerySpecification node) { if (node.getSelect().isDistinct()) { return subPlan.withNewRoot( new AggregationNode( idAllocator.getNextId(), subPlan.getRoot(), ImmutableMap.of(), ImmutableMap.of(), ImmutableMap.of(), ImmutableList.of(subPlan.getRoot().getOutputSymbols()), AggregationNode.Step.SINGLE, Optional.empty(), Optional.empty())); } return subPlan; } private PlanBuilder sort(PlanBuilder subPlan, Query node) { return sort(subPlan, node.getOrderBy(), node.getLimit(), analysis.getOrderByExpressions(node)); } private PlanBuilder sort(PlanBuilder subPlan, QuerySpecification node) { return sort(subPlan, node.getOrderBy(), node.getLimit(), analysis.getOrderByExpressions(node)); } private PlanBuilder sort(PlanBuilder subPlan, Optional<OrderBy> orderBy, Optional<String> limit, List<Expression> orderByExpressions) { if (!orderBy.isPresent()) { return subPlan; } Iterator<SortItem> sortItems = orderBy.get().getSortItems().iterator(); ImmutableList.Builder<Symbol> orderBySymbols = ImmutableList.builder(); Map<Symbol, SortOrder> orderings = new HashMap<>(); for (Expression fieldOrExpression : orderByExpressions) { Symbol symbol = subPlan.translate(fieldOrExpression); SortItem sortItem = sortItems.next(); if (!orderings.containsKey(symbol)) { orderBySymbols.add(symbol); orderings.put(symbol, toSortOrder(sortItem)); } } PlanNode planNode; if (limit.isPresent() && !limit.get().equalsIgnoreCase("all")) { planNode = new TopNNode(idAllocator.getNextId(), subPlan.getRoot(), Long.parseLong(limit.get()), orderBySymbols.build(), orderings, false); } else { planNode = new SortNode(idAllocator.getNextId(), subPlan.getRoot(), orderBySymbols.build(), orderings); } return subPlan.withNewRoot(planNode); } private PlanBuilder limit(PlanBuilder subPlan, Query node) { return limit(subPlan, node.getOrderBy(), node.getLimit()); } private PlanBuilder limit(PlanBuilder subPlan, QuerySpecification node) { return limit(subPlan, node.getOrderBy(), node.getLimit()); } private PlanBuilder limit(PlanBuilder subPlan, Optional<OrderBy> orderBy, Optional<String> limit) { if (!orderBy.isPresent() && limit.isPresent()) { if (!limit.get().equalsIgnoreCase("all")) { long limitValue = Long.parseLong(limit.get()); subPlan = subPlan.withNewRoot(new LimitNode(idAllocator.getNextId(), subPlan.getRoot(), limitValue, false)); } } return subPlan; } private static List<Expression> toSymbolReferences(List<Symbol> symbols) { return symbols.stream() .map(Symbol::toSymbolReference) .collect(toImmutableList()); } private static Map<Expression, Symbol> symbolsForExpressions(PlanBuilder builder, Iterable<? extends Expression> expressions) { Set<Expression> added = new HashSet<>(); return StreamSupport.stream(expressions.spliterator(), false) .filter(added::add) .collect(toImmutableMap(expression -> expression, builder::translate)); } private static SortOrder toSortOrder(SortItem sortItem) { if (sortItem.getOrdering() == Ordering.ASCENDING) { if (sortItem.getNullOrdering() == NullOrdering.FIRST) { return SortOrder.ASC_NULLS_FIRST; } else { return SortOrder.ASC_NULLS_LAST; } } else { if (sortItem.getNullOrdering() == NullOrdering.FIRST) { return SortOrder.DESC_NULLS_FIRST; } else { return SortOrder.DESC_NULLS_LAST; } } } }