Skip to content

Commit

Permalink
Merge branch 'unionLimit' into unionLimit2
Browse files Browse the repository at this point in the history
  • Loading branch information
gatorsmile committed Dec 29, 2015
2 parents 10d570c + 2823a57 commit cfbeea7
Show file tree
Hide file tree
Showing 5 changed files with 121 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,6 @@ abstract class Optimizer extends RuleExecutor[LogicalPlan] {
// Operator combine
ProjectCollapsing,
CombineFilters,
CombineLimits,
// Constant folding
NullPropagation,
OptimizeIn,
Expand All @@ -64,6 +63,11 @@ abstract class Optimizer extends RuleExecutor[LogicalPlan] {
SimplifyFilters,
SimplifyCasts,
SimplifyCaseConversionExpressions) ::
Batch("Push Down Limits", FixedPoint(100),
PushDownLimit,
CombineLimits,
ConstantFolding,
BooleanSimplification) ::
Batch("Decimal Optimizations", FixedPoint(100),
DecimalAggregates) ::
Batch("LocalRelation", FixedPoint(100),
Expand All @@ -79,6 +83,36 @@ abstract class Optimizer extends RuleExecutor[LogicalPlan] {
*/
object DefaultOptimizer extends Optimizer

/**
* Pushes down Limit for reducing the amount of returned data.
*
* 1. Adding Extra Limit beneath the operations, including Union All.
* 2. Project is pushed through Limit in the rule ColumnPruning
*
* Any operator that a Limit can be pushed passed should override the maxRows function.
*
* Note: This rule has to be done when the logical plan is stable;
* Otherwise, it could impact the other rules.
*/
object PushDownLimit extends Rule[LogicalPlan] {

def apply(plan: LogicalPlan): LogicalPlan = plan transform {

// Adding extra Limit below UNION ALL iff both left and right childs are not Limit or
// do not have Limit descendants. 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, we will not be able to
// pushdown Limit.
case Limit(exp, Union(left, right))
if left.maxRows.isEmpty || right.maxRows.isEmpty =>
Limit(exp,
Union(
Limit(exp, left),
Limit(exp, right)))
}
}

/**
* Pushes operations down into a Sample.
*/
Expand All @@ -97,8 +131,8 @@ object SamplePushDown extends Rule[LogicalPlan] {
* Operations that are safe to pushdown are listed as follows.
* Union:
* Right now, Union means UNION ALL, which does not de-duplicate rows. So, it is
* safe to pushdown Filters and Projections through it. Once we add UNION DISTINCT,
* we will not be able to pushdown Projections.
* safe to pushdown Filters, Projections and Limits through it. Once we add UNION DISTINCT,
* we will not be able to pushdown Projections and Limits.
*
* Intersect:
* It is not safe to pushdown Projections through it because we need to get the
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,13 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging {
Statistics(sizeInBytes = children.map(_.statistics.sizeInBytes).product)
}

/**
* Returns the limited number of rows to be returned.
*
* Any operator that a Limit can be pushed passed should override this function.
*/
def maxRows: Option[Expression] = 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 @@ -28,6 +28,8 @@ import scala.collection.mutable.ArrayBuffer
case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode {
override def output: Seq[Attribute] = projectList.map(_.toAttribute)

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

override lazy val resolved: Boolean = {
val hasSpecialExpressions = projectList.exists ( _.collect {
case agg: AggregateExpression => agg
Expand Down Expand Up @@ -109,6 +111,9 @@ private[sql] object SetOperation {

case class Union(left: LogicalPlan, right: LogicalPlan) extends SetOperation(left, right) {

override def maxRows: Option[Expression] =
for (leftMax <- left.maxRows; rightMax <- right.maxRows) yield Add(leftMax, rightMax)

override def statistics: Statistics = {
val sizeInBytes = left.statistics.sizeInBytes + right.statistics.sizeInBytes
Statistics(sizeInBytes = sizeInBytes)
Expand Down Expand Up @@ -451,6 +456,8 @@ case class Pivot(
case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode {
override def output: Seq[Attribute] = child.output

override def maxRows: Option[Expression] = Option(limitExpr)

override lazy val statistics: Statistics = {
val limit = limitExpr.eval().asInstanceOf[Int]
val sizeInBytes = (limit: Long) * output.map(a => a.dataType.defaultSize).sum
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* 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.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.dsl.expressions._

class PushdownLimitsSuite extends PlanTest {
object Optimize extends RuleExecutor[LogicalPlan] {
val batches =
Batch("Subqueries", Once,
EliminateSubQueries) ::
Batch("Push Down Limit", Once,
PushDownLimit,
CombineLimits,
ConstantFolding,
BooleanSimplification) :: Nil
}

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

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

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

test("Union: no limit to both sides") {
val testLimitUnion = Union(testRelation.limit(2), testRelation2.select('d).limit(3))
val unionQuery = testLimitUnion.limit(2)
val unionOptimized = Optimize.execute(unionQuery.analyze)
val unionCorrectAnswer =
Limit(2, Union(testRelation.limit(2), testRelation2.select('d).limit(3))).analyze
comparePlans(unionOptimized, unionCorrectAnswer)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,9 @@ class SetOperationPushDownSuite extends PlanTest {
EliminateSubQueries) ::
Batch("Union Pushdown", Once,
SetOperationPushDown,
CombineLimits,
ConstantFolding,
BooleanSimplification,
SimplifyFilters) :: Nil
}

Expand Down

0 comments on commit cfbeea7

Please sign in to comment.