Skip to content

Commit

Permalink
[SPARK-12503][SPARK-12505] Limit pushdown in UNION ALL and OUTER JOIN
Browse files Browse the repository at this point in the history
This patch adds a new optimizer rule for performing limit pushdown. Limits will now be pushed down in two cases:

- If a limit is on top of a `UNION ALL` operator, then a partition-local limit operator will be pushed to each of the union operator's children.
- If a limit is on top of an `OUTER JOIN` then a partition-local limit will be pushed to one side of the join. For `LEFT OUTER` and `RIGHT OUTER` joins, the limit will be pushed to the left and right side, respectively. For `FULL OUTER` join, we will only push limits when at most one of the inputs is already limited: if one input is limited we will push a smaller limit on top of it and if neither input is limited then we will limit the input which is estimated to be larger.

These optimizations were proposed previously by gatorsmile in #10451 and #10454, but those earlier PRs were closed and deferred for later because at that time Spark's physical `Limit` operator would trigger a full shuffle to perform global limits so there was a chance that pushdowns could actually harm performance by causing additional shuffles/stages. In #7334, we split the `Limit` operator into separate `LocalLimit` and `GlobalLimit` operators, so we can now push down only local limits (which don't require extra shuffles). This patch is based on both of gatorsmile's patches, with changes and simplifications due to partition-local-limiting.

When we push down the limit, we still keep the original limit in place, so we need a mechanism to ensure that the optimizer rule doesn't keep pattern-matching once the limit has been pushed down. In order to handle this, this patch adds a `maxRows` method to `SparkPlan` which returns the maximum number of rows that the plan can compute, then defines the pushdown rules to only push limits to children if the children's maxRows are greater than the limit's maxRows. This idea is carried over from #10451; see that patch for additional discussion.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11121 from JoshRosen/limit-pushdown-2.
  • Loading branch information
JoshRosen authored and rxin committed Feb 15, 2016
1 parent 7cb4d74 commit a8bbc4f
Show file tree
Hide file tree
Showing 6 changed files with 294 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ abstract class Optimizer extends RuleExecutor[LogicalPlan] {
PushPredicateThroughProject,
PushPredicateThroughGenerate,
PushPredicateThroughAggregate,
// LimitPushDown, // Disabled until we have whole-stage codegen for limit
ColumnPruning,
// Operator combine
CollapseRepartition,
Expand Down Expand Up @@ -129,6 +130,69 @@ object EliminateSerialization extends Rule[LogicalPlan] {
}
}

/**
* Pushes down [[LocalLimit]] beneath UNION ALL and beneath the streamed inputs of outer joins.
*/
object LimitPushDown extends Rule[LogicalPlan] {

private def stripGlobalLimitIfPresent(plan: LogicalPlan): LogicalPlan = {
plan match {
case GlobalLimit(expr, child) => child
case _ => plan
}
}

private def maybePushLimit(limitExp: Expression, plan: LogicalPlan): LogicalPlan = {
(limitExp, plan.maxRows) match {
case (IntegerLiteral(maxRow), Some(childMaxRows)) if maxRow < childMaxRows =>
LocalLimit(limitExp, stripGlobalLimitIfPresent(plan))
case (_, None) =>
LocalLimit(limitExp, stripGlobalLimitIfPresent(plan))
case _ => plan
}
}

def apply(plan: LogicalPlan): LogicalPlan = plan transform {
// Adding extra Limits below UNION ALL for children which are not Limit or do not have Limit
// descendants whose maxRow is larger. This heuristic is valid assuming there does not exist any
// Limit push-down rule that is unable to infer the value of maxRows.
// Note: right now Union means UNION ALL, which does not de-duplicate rows, so it is safe to
// pushdown Limit through it. Once we add UNION DISTINCT, however, we will not be able to
// pushdown Limit.
case LocalLimit(exp, Union(children)) =>
LocalLimit(exp, Union(children.map(maybePushLimit(exp, _))))
// Add extra limits below OUTER JOIN. For LEFT OUTER and FULL OUTER JOIN we push limits to the
// left and right sides, respectively. For FULL OUTER JOIN, we can only push limits to one side
// because we need to ensure that rows from the limited side still have an opportunity to match
// against all candidates from the non-limited side. We also need to ensure that this limit
// pushdown rule will not eventually introduce limits on both sides if it is applied multiple
// times. Therefore:
// - If one side is already limited, stack another limit on top if the new limit is smaller.
// The redundant limit will be collapsed by the CombineLimits rule.
// - If neither side is limited, limit the side that is estimated to be bigger.
case LocalLimit(exp, join @ Join(left, right, joinType, condition)) =>
val newJoin = joinType match {
case RightOuter => join.copy(right = maybePushLimit(exp, right))
case LeftOuter => join.copy(left = maybePushLimit(exp, left))
case FullOuter =>
(left.maxRows, right.maxRows) match {
case (None, None) =>
if (left.statistics.sizeInBytes >= right.statistics.sizeInBytes) {
join.copy(left = maybePushLimit(exp, left))
} else {
join.copy(right = maybePushLimit(exp, right))
}
case (Some(_), Some(_)) => join
case (Some(_), None) => join.copy(left = maybePushLimit(exp, left))
case (None, Some(_)) => join.copy(right = maybePushLimit(exp, right))

}
case _ => join
}
LocalLimit(exp, newJoin)
}
}

/**
* Pushes certain operations to both sides of a Union or Except operator.
* Operations that are safe to pushdown are listed as follows.
Expand Down Expand Up @@ -985,8 +1049,12 @@ object RemoveDispensableExpressions extends Rule[LogicalPlan] {
*/
object CombineLimits extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case ll @ GlobalLimit(le, nl @ GlobalLimit(ne, grandChild)) =>
GlobalLimit(Least(Seq(ne, le)), grandChild)
case ll @ LocalLimit(le, nl @ LocalLimit(ne, grandChild)) =>
LocalLimit(Least(Seq(ne, le)), grandChild)
case ll @ Limit(le, nl @ Limit(ne, grandChild)) =>
Limit(If(LessThan(ne, le), ne, le), grandChild)
Limit(Least(Seq(ne, le)), grandChild)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,14 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging {
Statistics(sizeInBytes = children.map(_.statistics.sizeInBytes).product)
}

/**
* Returns the maximum number of rows that this plan may compute.
*
* Any operator that a Limit can be pushed passed should override this function (e.g., Union).
* Any operator that can push through a Limit should override this function (e.g., Project).
*/
def maxRows: Option[Long] = None

/**
* Returns true if this expression and all its children have been resolved to a specific schema
* and false if it still contains any unresolved placeholders. Implementations of LogicalPlan
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ case class ReturnAnswer(child: LogicalPlan) extends UnaryNode {

case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode {
override def output: Seq[Attribute] = projectList.map(_.toAttribute)
override def maxRows: Option[Long] = child.maxRows

override lazy val resolved: Boolean = {
val hasSpecialExpressions = projectList.exists ( _.collect {
Expand All @@ -56,6 +57,7 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend
* output of each into a new stream of rows. This operation is similar to a `flatMap` in functional
* programming with one important additional feature, which allows the input rows to be joined with
* their output.
*
* @param generator the generator expression
* @param join when true, each output row is implicitly joined with the input tuple that produced
* it.
Expand Down Expand Up @@ -102,6 +104,8 @@ case class Filter(condition: Expression, child: LogicalPlan)
extends UnaryNode with PredicateHelper {
override def output: Seq[Attribute] = child.output

override def maxRows: Option[Long] = child.maxRows

override protected def validConstraints: Set[Expression] = {
child.constraints.union(splitConjunctivePredicates(condition).toSet)
}
Expand Down Expand Up @@ -144,6 +148,14 @@ case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation
left.output.length == right.output.length &&
left.output.zip(right.output).forall { case (l, r) => l.dataType == r.dataType } &&
duplicateResolved

override def maxRows: Option[Long] = {
if (children.exists(_.maxRows.isEmpty)) {
None
} else {
Some(children.flatMap(_.maxRows).min)
}
}
}

case class Except(left: LogicalPlan, right: LogicalPlan) extends SetOperation(left, right) {
Expand All @@ -166,6 +178,13 @@ object Union {
}

case class Union(children: Seq[LogicalPlan]) extends LogicalPlan {
override def maxRows: Option[Long] = {
if (children.exists(_.maxRows.isEmpty)) {
None
} else {
Some(children.flatMap(_.maxRows).sum)
}
}

// updating nullability to make all the children consistent
override def output: Seq[Attribute] =
Expand Down Expand Up @@ -305,6 +324,7 @@ case class InsertIntoTable(
/**
* A container for holding named common table expressions (CTEs) and a query plan.
* This operator will be removed during analysis and the relations will be substituted into child.
*
* @param child The final query of this CTE.
* @param cteRelations Queries that this CTE defined,
* key is the alias of the CTE definition,
Expand All @@ -331,6 +351,7 @@ case class Sort(
global: Boolean,
child: LogicalPlan) extends UnaryNode {
override def output: Seq[Attribute] = child.output
override def maxRows: Option[Long] = child.maxRows
}

/** Factory for constructing new `Range` nodes. */
Expand Down Expand Up @@ -384,6 +405,7 @@ case class Aggregate(
}

override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute)
override def maxRows: Option[Long] = child.maxRows
}

case class Window(
Expand Down Expand Up @@ -505,6 +527,7 @@ trait GroupingAnalytics extends UnaryNode {
* to generated by a UNION ALL of multiple simple GROUP BY clauses.
*
* We will transform GROUPING SETS into logical plan Aggregate(.., Expand) in Analyzer
*
* @param bitmasks A list of bitmasks, each of the bitmask indicates the selected
* GroupBy expressions
* @param groupByExprs The Group By expressions candidates, take effective only if the
Expand Down Expand Up @@ -537,9 +560,42 @@ case class Pivot(
}
}

case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode {
object Limit {
def apply(limitExpr: Expression, child: LogicalPlan): UnaryNode = {
GlobalLimit(limitExpr, LocalLimit(limitExpr, child))
}

def unapply(p: GlobalLimit): Option[(Expression, LogicalPlan)] = {
p match {
case GlobalLimit(le1, LocalLimit(le2, child)) if le1 == le2 => Some((le1, child))
case _ => None
}
}
}

case class GlobalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode {
override def output: Seq[Attribute] = child.output
override def maxRows: Option[Long] = {
limitExpr match {
case IntegerLiteral(limit) => Some(limit)
case _ => None
}
}
override lazy val statistics: Statistics = {
val limit = limitExpr.eval().asInstanceOf[Int]
val sizeInBytes = (limit: Long) * output.map(a => a.dataType.defaultSize).sum
Statistics(sizeInBytes = sizeInBytes)
}
}

case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode {
override def output: Seq[Attribute] = child.output
override def maxRows: Option[Long] = {
limitExpr match {
case IntegerLiteral(limit) => Some(limit)
case _ => None
}
}
override lazy val statistics: Statistics = {
val limit = limitExpr.eval().asInstanceOf[Int]
val sizeInBytes = (limit: Long) * output.map(a => a.dataType.defaultSize).sum
Expand Down Expand Up @@ -576,6 +632,7 @@ case class Sample(
* Returns a new logical plan that dedups input rows.
*/
case class Distinct(child: LogicalPlan) extends UnaryNode {
override def maxRows: Option[Long] = child.maxRows
override def output: Seq[Attribute] = child.output
}

Expand All @@ -594,6 +651,7 @@ case class Repartition(numPartitions: Int, shuffle: Boolean, child: LogicalPlan)
* A relation with one row. This is used in "SELECT ..." without a from clause.
*/
case object OneRowRelation extends LeafNode {
override def maxRows: Option[Long] = Some(1)
override def output: Seq[Attribute] = Nil

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,145 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.spark.sql.catalyst.optimizer

import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.expressions.Add
import org.apache.spark.sql.catalyst.plans.{FullOuter, LeftOuter, PlanTest, RightOuter}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._

class LimitPushdownSuite extends PlanTest {

private object Optimize extends RuleExecutor[LogicalPlan] {
val batches =
Batch("Subqueries", Once,
EliminateSubQueries) ::
Batch("Limit pushdown", FixedPoint(100),
LimitPushDown,
CombineLimits,
ConstantFolding,
BooleanSimplification) :: Nil
}

private val testRelation = LocalRelation('a.int, 'b.int, 'c.int)
private val testRelation2 = LocalRelation('d.int, 'e.int, 'f.int)
private val x = testRelation.subquery('x)
private val y = testRelation.subquery('y)

// Union ---------------------------------------------------------------------------------------

test("Union: limit to each side") {
val unionQuery = Union(testRelation, testRelation2).limit(1)
val unionOptimized = Optimize.execute(unionQuery.analyze)
val unionCorrectAnswer =
Limit(1, Union(LocalLimit(1, testRelation), LocalLimit(1, testRelation2))).analyze
comparePlans(unionOptimized, unionCorrectAnswer)
}

test("Union: limit to each side with constant-foldable limit expressions") {
val unionQuery = Union(testRelation, testRelation2).limit(Add(1, 1))
val unionOptimized = Optimize.execute(unionQuery.analyze)
val unionCorrectAnswer =
Limit(2, Union(LocalLimit(2, testRelation), LocalLimit(2, testRelation2))).analyze
comparePlans(unionOptimized, unionCorrectAnswer)
}

test("Union: limit to each side with the new limit number") {
val unionQuery = Union(testRelation, testRelation2.limit(3)).limit(1)
val unionOptimized = Optimize.execute(unionQuery.analyze)
val unionCorrectAnswer =
Limit(1, Union(LocalLimit(1, testRelation), LocalLimit(1, testRelation2))).analyze
comparePlans(unionOptimized, unionCorrectAnswer)
}

test("Union: no limit to both sides if children having smaller limit values") {
val unionQuery = Union(testRelation.limit(1), testRelation2.select('d).limit(1)).limit(2)
val unionOptimized = Optimize.execute(unionQuery.analyze)
val unionCorrectAnswer =
Limit(2, Union(testRelation.limit(1), testRelation2.select('d).limit(1))).analyze
comparePlans(unionOptimized, unionCorrectAnswer)
}

test("Union: limit to each sides if children having larger limit values") {
val testLimitUnion = Union(testRelation.limit(3), testRelation2.select('d).limit(4))
val unionQuery = testLimitUnion.limit(2)
val unionOptimized = Optimize.execute(unionQuery.analyze)
val unionCorrectAnswer =
Limit(2, Union(LocalLimit(2, testRelation), LocalLimit(2, testRelation2.select('d)))).analyze
comparePlans(unionOptimized, unionCorrectAnswer)
}

// Outer join ----------------------------------------------------------------------------------

test("left outer join") {
val originalQuery = x.join(y, LeftOuter).limit(1)
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = Limit(1, LocalLimit(1, y).join(y, LeftOuter)).analyze
comparePlans(optimized, correctAnswer)
}

test("right outer join") {
val originalQuery = x.join(y, RightOuter).limit(1)
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = Limit(1, x.join(LocalLimit(1, y), RightOuter)).analyze
comparePlans(optimized, correctAnswer)
}

test("larger limits are not pushed on top of smaller ones in right outer join") {
val originalQuery = x.join(y.limit(5), RightOuter).limit(10)
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = Limit(10, x.join(Limit(5, y), RightOuter)).analyze
comparePlans(optimized, correctAnswer)
}

test("full outer join where neither side is limited and both sides have same statistics") {
assert(x.statistics.sizeInBytes === y.statistics.sizeInBytes)
val originalQuery = x.join(y, FullOuter).limit(1)
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = Limit(1, LocalLimit(1, x).join(y, FullOuter)).analyze
comparePlans(optimized, correctAnswer)
}

test("full outer join where neither side is limited and left side has larger statistics") {
val xBig = testRelation.copy(data = Seq.fill(2)(null)).subquery('x)
assert(xBig.statistics.sizeInBytes > y.statistics.sizeInBytes)
val originalQuery = xBig.join(y, FullOuter).limit(1)
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = Limit(1, LocalLimit(1, xBig).join(y, FullOuter)).analyze
comparePlans(optimized, correctAnswer)
}

test("full outer join where neither side is limited and right side has larger statistics") {
val yBig = testRelation.copy(data = Seq.fill(2)(null)).subquery('y)
assert(x.statistics.sizeInBytes < yBig.statistics.sizeInBytes)
val originalQuery = x.join(yBig, FullOuter).limit(1)
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = Limit(1, x.join(LocalLimit(1, yBig), FullOuter)).analyze
comparePlans(optimized, correctAnswer)
}

test("full outer join where both sides are limited") {
val originalQuery = x.limit(2).join(y.limit(2), FullOuter).limit(1)
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = Limit(1, Limit(2, x).join(Limit(2, y), FullOuter)).analyze
comparePlans(optimized, correctAnswer)
}
}

Loading

0 comments on commit a8bbc4f

Please sign in to comment.