Skip to content

Commit

Permalink
Pushdown dereference expressions in the query plan
Browse files Browse the repository at this point in the history
Co-authored-by: qqibrow <[email protected]>
Co-authored-by: Zhenxiao Luo <[email protected]>
  • Loading branch information
3 people authored and martint committed May 21, 2020
1 parent f00fae5 commit 69ef682
Show file tree
Hide file tree
Showing 27 changed files with 3,134 additions and 16 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import io.prestosql.sql.planner.iterative.rule.DetermineSemiJoinDistributionType;
import io.prestosql.sql.planner.iterative.rule.EliminateCrossJoins;
import io.prestosql.sql.planner.iterative.rule.EvaluateZeroSample;
import io.prestosql.sql.planner.iterative.rule.ExtractDereferencesFromFilterAboveScan;
import io.prestosql.sql.planner.iterative.rule.ExtractSpatialJoins;
import io.prestosql.sql.planner.iterative.rule.GatherAndMergeWindows;
import io.prestosql.sql.planner.iterative.rule.ImplementBernoulliSampleAsFilter;
Expand Down Expand Up @@ -105,6 +106,18 @@
import io.prestosql.sql.planner.iterative.rule.PruneWindowColumns;
import io.prestosql.sql.planner.iterative.rule.PushAggregationThroughOuterJoin;
import io.prestosql.sql.planner.iterative.rule.PushDeleteIntoConnector;
import io.prestosql.sql.planner.iterative.rule.PushDownDereferenceThroughFilter;
import io.prestosql.sql.planner.iterative.rule.PushDownDereferenceThroughJoin;
import io.prestosql.sql.planner.iterative.rule.PushDownDereferenceThroughProject;
import io.prestosql.sql.planner.iterative.rule.PushDownDereferenceThroughSemiJoin;
import io.prestosql.sql.planner.iterative.rule.PushDownDereferenceThroughUnnest;
import io.prestosql.sql.planner.iterative.rule.PushDownDereferencesThroughAssignUniqueId;
import io.prestosql.sql.planner.iterative.rule.PushDownDereferencesThroughLimit;
import io.prestosql.sql.planner.iterative.rule.PushDownDereferencesThroughRowNumber;
import io.prestosql.sql.planner.iterative.rule.PushDownDereferencesThroughSort;
import io.prestosql.sql.planner.iterative.rule.PushDownDereferencesThroughTopN;
import io.prestosql.sql.planner.iterative.rule.PushDownDereferencesThroughTopNRowNumber;
import io.prestosql.sql.planner.iterative.rule.PushDownDereferencesThroughWindow;
import io.prestosql.sql.planner.iterative.rule.PushLimitIntoTableScan;
import io.prestosql.sql.planner.iterative.rule.PushLimitThroughMarkDistinct;
import io.prestosql.sql.planner.iterative.rule.PushLimitThroughOffset;
Expand Down Expand Up @@ -299,7 +312,21 @@ public PlanOptimizers(
Set<Rule<?>> projectionPushdownRules = ImmutableSet.of(
new PushProjectionIntoTableScan(metadata, typeAnalyzer),
new PushProjectionThroughUnion(),
new PushProjectionThroughExchange());
new PushProjectionThroughExchange(),
// Dereference pushdown rules
new PushDownDereferenceThroughProject(typeAnalyzer),
new PushDownDereferenceThroughUnnest(typeAnalyzer),
new PushDownDereferenceThroughSemiJoin(typeAnalyzer),
new PushDownDereferenceThroughJoin(typeAnalyzer),
new PushDownDereferenceThroughFilter(typeAnalyzer),
new ExtractDereferencesFromFilterAboveScan(typeAnalyzer),
new PushDownDereferencesThroughLimit(typeAnalyzer),
new PushDownDereferencesThroughSort(typeAnalyzer),
new PushDownDereferencesThroughAssignUniqueId(typeAnalyzer),
new PushDownDereferencesThroughWindow(typeAnalyzer),
new PushDownDereferencesThroughTopN(typeAnalyzer),
new PushDownDereferencesThroughRowNumber(typeAnalyzer),
new PushDownDereferencesThroughTopNRowNumber(typeAnalyzer));

IterativeOptimizer inlineProjections = new IterativeOptimizer(
ruleStats,
Expand Down Expand Up @@ -497,6 +524,16 @@ public PlanOptimizers(
inlineProjections,
simplifyOptimizer, // Re-run the SimplifyExpressions to simplify any recomposed expressions from other optimizations
projectionPushDown,
// Projection pushdown rules may push reducing projections (e.g. dereferences) below filters for potential
// pushdown into the connectors. We invoke PredicatePushdown and PushPredicateIntoTableScan after this
// to leverage predicate pushdown on projected columns.
new StatsRecordingPlanOptimizer(optimizerStats, new PredicatePushDown(metadata, typeAnalyzer, true, false)),
simplifyOptimizer, // Should be always run after PredicatePushDown
new IterativeOptimizer(
ruleStats,
statsCalculator,
estimatedExchangesCostCalculator,
ImmutableSet.of(new PushPredicateIntoTableScan(metadata, typeAnalyzer))),
new UnaliasSymbolReferences(metadata), // Run again because predicate pushdown and projection pushdown might add more projections
new PruneUnreferencedOutputs(metadata, typeAnalyzer), // Make sure to run this before index join. Filtered projections may not have all the columns.
new IndexJoinOptimizer(metadata), // Run this after projections and filters have been fully simplified and pushed down
Expand Down Expand Up @@ -539,6 +576,16 @@ public PlanOptimizers(
estimatedExchangesCostCalculator,
ImmutableSet.of(new PushPredicateIntoTableScan(metadata, typeAnalyzer))),
projectionPushDown,
// Projection pushdown rules may push reducing projections (e.g. dereferences) below filters for potential
// pushdown into the connectors. Invoke PredicatePushdown and PushPredicateIntoTableScan after this
// to leverage predicate pushdown on projected columns.
new StatsRecordingPlanOptimizer(optimizerStats, new PredicatePushDown(metadata, typeAnalyzer, true, false)),
simplifyOptimizer, // Should be always run after PredicatePushDown
new IterativeOptimizer(
ruleStats,
statsCalculator,
estimatedExchangesCostCalculator,
ImmutableSet.of(new PushPredicateIntoTableScan(metadata, typeAnalyzer))),
new PruneUnreferencedOutputs(metadata, typeAnalyzer),
new IterativeOptimizer(
ruleStats,
Expand Down Expand Up @@ -627,6 +674,17 @@ public PlanOptimizers(
costCalculator,
ImmutableSet.of(new RemoveRedundantTableScanPredicate(metadata))));
builder.add(projectionPushDown);
// Projection pushdown rules may push reducing projections (e.g. dereferences) below filters for potential
// pushdown into the connectors. Invoke PredicatePushdown and PushPredicateIntoTableScan after this
// to leverage predicate pushdown on projected columns.
builder.add(new StatsRecordingPlanOptimizer(optimizerStats, new PredicatePushDown(metadata, typeAnalyzer, true, true)));
builder.add(new RemoveUnsupportedDynamicFilters(metadata)); // Remove unsupported dynamic filters introduced by PredicatePushdown
builder.add(simplifyOptimizer); // Should always run after PredicatePushdown
new IterativeOptimizer(
ruleStats,
statsCalculator,
costCalculator,
ImmutableSet.of(new PushPredicateIntoTableScan(metadata, typeAnalyzer)));
builder.add(inlineProjections);
builder.add(new UnaliasSymbolReferences(metadata)); // Run unalias after merging projections to simplify projections more efficiently
builder.add(new PruneUnreferencedOutputs(metadata, typeAnalyzer));
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,130 @@
/*
* 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 io.prestosql.sql.planner.iterative.rule;

import com.google.common.collect.ImmutableList;
import io.prestosql.sql.planner.Symbol;
import io.prestosql.sql.tree.DefaultExpressionTraversalVisitor;
import io.prestosql.sql.tree.DereferenceExpression;
import io.prestosql.sql.tree.Expression;
import io.prestosql.sql.tree.LambdaExpression;
import io.prestosql.sql.tree.SymbolReference;

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

import static com.google.common.base.Verify.verify;
import static com.google.common.collect.Iterables.getOnlyElement;
import static io.prestosql.sql.planner.SymbolsExtractor.extractAll;

/**
* Provides helper methods to push down dereferences in the query plan.
*/
class DereferencePushdown
{
private DereferencePushdown() {}

public static Set<DereferenceExpression> extractDereferences(Collection<Expression> expressions, boolean allowOverlap)
{
Set<Expression> symbolReferencesAndDereferences = expressions.stream()
.flatMap(expression -> getSymbolReferencesAndDereferences(expression).stream())
.collect(Collectors.toSet());

// Remove overlap if required
Set<Expression> candidateExpressions = symbolReferencesAndDereferences;
if (!allowOverlap) {
candidateExpressions = symbolReferencesAndDereferences.stream()
.filter(expression -> !prefixExists(expression, symbolReferencesAndDereferences))
.collect(Collectors.toSet());
}

// Retain dereference expressions
return candidateExpressions.stream()
.filter(DereferenceExpression.class::isInstance)
.map(DereferenceExpression.class::cast)
.collect(Collectors.toSet());
}

public static boolean exclusiveDereferences(Set<Expression> projections)
{
return projections.stream()
.allMatch(expression -> expression instanceof SymbolReference ||
(expression instanceof DereferenceExpression &&
isDereferenceChain((DereferenceExpression) expression) &&
!prefixExists(expression, projections)));
}

public static Symbol getBase(DereferenceExpression expression)
{
return getOnlyElement(extractAll(expression));
}

/**
* Extract the sub-expressions of type {@link DereferenceExpression} or {@link SymbolReference} from the {@param expression}
* in a top-down manner. The expressions within the base of a valid {@link DereferenceExpression} sequence are not extracted.
*/
private static List<Expression> getSymbolReferencesAndDereferences(Expression expression)
{
ImmutableList.Builder<Expression> builder = ImmutableList.builder();

new DefaultExpressionTraversalVisitor<ImmutableList.Builder<Expression>>()
{
@Override
protected Void visitDereferenceExpression(DereferenceExpression node, ImmutableList.Builder<Expression> context)
{
if (isDereferenceChain(node)) {
context.add(node);
}
return null;
}

@Override
protected Void visitSymbolReference(SymbolReference node, ImmutableList.Builder<Expression> context)
{
context.add(node);
return null;
}

@Override
protected Void visitLambdaExpression(LambdaExpression node, ImmutableList.Builder<Expression> context)
{
return null;
}
}.process(expression, builder);

return builder.build();
}

private static boolean isDereferenceChain(DereferenceExpression expression)
{
return (expression.getBase() instanceof SymbolReference) ||
((expression.getBase() instanceof DereferenceExpression) && isDereferenceChain((DereferenceExpression) (expression.getBase())));
}

private static boolean prefixExists(Expression expression, Set<Expression> expressions)
{
Expression current = expression;
while (current instanceof DereferenceExpression) {
current = ((DereferenceExpression) current).getBase();
if (expressions.contains(current)) {
return true;
}
}

verify(current instanceof SymbolReference);
return false;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/*
* 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 io.prestosql.sql.planner.iterative.rule;

import com.google.common.collect.HashBiMap;
import com.google.common.collect.ImmutableList;
import io.prestosql.matching.Capture;
import io.prestosql.matching.Captures;
import io.prestosql.matching.Pattern;
import io.prestosql.sql.planner.TypeAnalyzer;
import io.prestosql.sql.planner.iterative.Rule;
import io.prestosql.sql.planner.plan.Assignments;
import io.prestosql.sql.planner.plan.FilterNode;
import io.prestosql.sql.planner.plan.PlanNode;
import io.prestosql.sql.planner.plan.ProjectNode;
import io.prestosql.sql.planner.plan.TableScanNode;
import io.prestosql.sql.tree.DereferenceExpression;
import io.prestosql.sql.tree.Expression;
import io.prestosql.sql.tree.SymbolReference;

import java.util.Map;
import java.util.Set;

import static com.google.common.collect.ImmutableMap.toImmutableMap;
import static io.prestosql.matching.Capture.newCapture;
import static io.prestosql.sql.planner.ExpressionNodeInliner.replaceExpression;
import static io.prestosql.sql.planner.iterative.rule.DereferencePushdown.extractDereferences;
import static io.prestosql.sql.planner.plan.Patterns.filter;
import static io.prestosql.sql.planner.plan.Patterns.source;
import static io.prestosql.sql.planner.plan.Patterns.tableScan;
import static java.util.Objects.requireNonNull;

/**
* Transforms:
* <pre>
* Filter(f1(A.x.y) = 1 AND f2(B.m) = 2 AND f3(A.x) = 6)
* Source(A, B, C)
* </pre>
* to:
* <pre>
* Project(A, B, C)
* Filter(f1(D) = 1 AND f2(E) = 2 AND f3(G) = 6)
* Project(A, B, C, D := A.x.y, E := B.m, G := A.x)
* Source(A, B, C)
* </pre>
*
* This optimizer extracts all dereference expressions from a filter node located above a table scan into a ProjectNode.
*
* Extracting dereferences from a filter (eg. FilterNode(a.x = 5)) can be suboptimal if full columns are being accessed up the
* plan tree (eg. a), because it can result in replicated shuffling of fields (eg. a.x). So it is safer to pushdown dereferences from
* Filter only when there's an explicit projection on top of the filter node (Ref PushDereferencesThroughFilter).
*
* In case of a FilterNode on top of TableScanNode, we want to push all dereferences into a new ProjectNode below, so that
* PushProjectionIntoTableScan optimizer can push those columns in the connector, and provide new column handles for the
* projected subcolumns. PushPredicateIntoTableScan optimizer can then push predicates on these subcolumns into the connector.
*/
public class ExtractDereferencesFromFilterAboveScan
implements Rule<FilterNode>
{
private static final Capture<TableScanNode> CHILD = newCapture();
private final TypeAnalyzer typeAnalyzer;

public ExtractDereferencesFromFilterAboveScan(TypeAnalyzer typeAnalyzer)
{
this.typeAnalyzer = requireNonNull(typeAnalyzer, "typeAnalyzer is null");
}

@Override
public Pattern<FilterNode> getPattern()
{
return filter()
.with(source().matching(tableScan().capturedAs(CHILD)));
}

@Override
public Result apply(FilterNode node, Captures captures, Context context)
{
Set<DereferenceExpression> dereferences = extractDereferences(ImmutableList.of(node.getPredicate()), true);
if (dereferences.isEmpty()) {
return Result.empty();
}

Assignments assignments = Assignments.of(dereferences, context.getSession(), context.getSymbolAllocator(), typeAnalyzer);
Map<Expression, SymbolReference> mappings = HashBiMap.create(assignments.getMap())
.inverse()
.entrySet().stream()
.collect(toImmutableMap(Map.Entry::getKey, entry -> entry.getValue().toSymbolReference()));

PlanNode source = node.getSource();
return Result.ofPlanNode(new ProjectNode(
context.getIdAllocator().getNextId(),
new FilterNode(
context.getIdAllocator().getNextId(),
new ProjectNode(
context.getIdAllocator().getNextId(),
source,
Assignments.builder()
.putIdentities(source.getOutputSymbols())
.putAll(assignments)
.build()),
replaceExpression(node.getPredicate(), mappings)),
Assignments.identity(node.getOutputSymbols())));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import io.prestosql.sql.planner.plan.Assignments;
import io.prestosql.sql.planner.plan.PlanNode;
import io.prestosql.sql.planner.plan.ProjectNode;
import io.prestosql.sql.tree.DereferenceExpression;
import io.prestosql.sql.tree.Expression;
import io.prestosql.sql.tree.Literal;
import io.prestosql.sql.tree.TryExpression;
Expand Down Expand Up @@ -172,6 +173,7 @@ private static Set<Symbol> extractInliningTargets(ProjectNode parent, ProjectNod
.filter(entry -> entry.getValue() == 1) // reference appears just once across all expressions in parent project node
.filter(entry -> !tryArguments.contains(entry.getKey())) // they are not inputs to TRY. Otherwise, inlining might change semantics
.filter(entry -> !child.getAssignments().isIdentity(entry.getKey())) // skip identities, otherwise, this rule will keep firing forever
.filter(entry -> !(child.getAssignments().get(entry.getKey()) instanceof DereferenceExpression)) // skip dereferences, otherwise, inlining can cause conflicts with PushdownDereferences
.map(Map.Entry::getKey)
.collect(toSet());

Expand Down
Loading

0 comments on commit 69ef682

Please sign in to comment.