Skip to content

Commit e430614

Browse files
sameeragarwalyhuai
authored andcommitted
[SPARK-13668][SQL] Reorder filter/join predicates to short-circuit isNotNull checks
## What changes were proposed in this pull request? If a filter predicate or a join condition consists of `IsNotNull` checks, we should reorder these checks such that these non-nullability checks are evaluated before the rest of the predicates. For e.g., if a filter predicate is of the form `a > 5 && isNotNull(b)`, we should rewrite this as `isNotNull(b) && a > 5` during physical plan generation. ## How was this patch tested? new unit tests that verify the physical plan for both filters and joins in `ReorderedPredicateSuite` Author: Sameer Agarwal <[email protected]> Closes #11511 from sameeragarwal/reorder-isnotnull.
1 parent 1e28840 commit e430614

File tree

3 files changed

+150
-14
lines changed

3 files changed

+150
-14
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala

Lines changed: 23 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,8 @@
1818
package org.apache.spark.sql.catalyst.planning
1919

2020
import org.apache.spark.Logging
21+
import org.apache.spark.sql.catalyst.expressions.{And, Expression, IsNotNull, PredicateHelper}
22+
import org.apache.spark.sql.catalyst.plans
2123
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2224
import org.apache.spark.sql.catalyst.trees.TreeNode
2325

@@ -26,8 +28,28 @@ import org.apache.spark.sql.catalyst.trees.TreeNode
2628
* be used for execution. If this strategy does not apply to the give logical operation then an
2729
* empty list should be returned.
2830
*/
29-
abstract class GenericStrategy[PhysicalPlan <: TreeNode[PhysicalPlan]] extends Logging {
31+
abstract class GenericStrategy[PhysicalPlan <: TreeNode[PhysicalPlan]]
32+
extends PredicateHelper with Logging {
33+
3034
def apply(plan: LogicalPlan): Seq[PhysicalPlan]
35+
36+
// Attempts to re-order the individual conjunctive predicates in an expression to short circuit
37+
// the evaluation of relatively cheaper checks (e.g., checking for nullability) before others.
38+
protected def reorderPredicates(expr: Expression): Expression = {
39+
splitConjunctivePredicates(expr)
40+
.sortWith((x, _) => x.isInstanceOf[IsNotNull])
41+
.reduce(And)
42+
}
43+
44+
// Wrapper around reorderPredicates(expr: Expression) to reorder optional conditions in joins
45+
protected def reorderPredicates(exprOpt: Option[Expression]): Option[Expression] = {
46+
exprOpt match {
47+
case Some(expr) =>
48+
Option(reorderPredicates(expr))
49+
case None =>
50+
exprOpt
51+
}
52+
}
3153
}
3254

3355
/**

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala

Lines changed: 24 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -66,11 +66,13 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
6666
case ExtractEquiJoinKeys(
6767
LeftSemi, leftKeys, rightKeys, condition, left, CanBroadcast(right)) =>
6868
joins.BroadcastLeftSemiJoinHash(
69-
leftKeys, rightKeys, planLater(left), planLater(right), condition) :: Nil
69+
leftKeys, rightKeys, planLater(left), planLater(right),
70+
reorderPredicates(condition)) :: Nil
7071
// Find left semi joins where at least some predicates can be evaluated by matching join keys
7172
case ExtractEquiJoinKeys(LeftSemi, leftKeys, rightKeys, condition, left, right) =>
7273
joins.LeftSemiJoinHash(
73-
leftKeys, rightKeys, planLater(left), planLater(right), condition) :: Nil
74+
leftKeys, rightKeys, planLater(left), planLater(right),
75+
reorderPredicates(condition)) :: Nil
7476
case _ => Nil
7577
}
7678
}
@@ -111,33 +113,39 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
111113

112114
case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, CanBroadcast(right)) =>
113115
Seq(joins.BroadcastHashJoin(
114-
leftKeys, rightKeys, Inner, BuildRight, condition, planLater(left), planLater(right)))
116+
leftKeys, rightKeys, Inner, BuildRight, reorderPredicates(condition),
117+
planLater(left), planLater(right)))
115118

116119
case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, CanBroadcast(left), right) =>
117120
Seq(joins.BroadcastHashJoin(
118-
leftKeys, rightKeys, Inner, BuildLeft, condition, planLater(left), planLater(right)))
121+
leftKeys, rightKeys, Inner, BuildLeft, reorderPredicates(condition), planLater(left),
122+
planLater(right)))
119123

120124
case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right)
121125
if RowOrdering.isOrderable(leftKeys) =>
122126
joins.SortMergeJoin(
123-
leftKeys, rightKeys, condition, planLater(left), planLater(right)) :: Nil
127+
leftKeys, rightKeys, reorderPredicates(condition), planLater(left),
128+
planLater(right)) :: Nil
124129

125130
// --- Outer joins --------------------------------------------------------------------------
126131

127132
case ExtractEquiJoinKeys(
128133
LeftOuter, leftKeys, rightKeys, condition, left, CanBroadcast(right)) =>
129134
Seq(joins.BroadcastHashJoin(
130-
leftKeys, rightKeys, LeftOuter, BuildRight, condition, planLater(left), planLater(right)))
135+
leftKeys, rightKeys, LeftOuter, BuildRight, reorderPredicates(condition),
136+
planLater(left), planLater(right)))
131137

132138
case ExtractEquiJoinKeys(
133139
RightOuter, leftKeys, rightKeys, condition, CanBroadcast(left), right) =>
134140
Seq(joins.BroadcastHashJoin(
135-
leftKeys, rightKeys, RightOuter, BuildLeft, condition, planLater(left), planLater(right)))
141+
leftKeys, rightKeys, RightOuter, BuildLeft, reorderPredicates(condition),
142+
planLater(left), planLater(right)))
136143

137144
case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right)
138145
if RowOrdering.isOrderable(leftKeys) =>
139146
joins.SortMergeOuterJoin(
140-
leftKeys, rightKeys, joinType, condition, planLater(left), planLater(right)) :: Nil
147+
leftKeys, rightKeys, joinType, reorderPredicates(condition), planLater(left),
148+
planLater(right)) :: Nil
141149

142150
// --- Cases where this strategy does not apply ---------------------------------------------
143151

@@ -252,10 +260,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
252260
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
253261
case j @ logical.Join(CanBroadcast(left), right, Inner | RightOuter, condition) =>
254262
execution.joins.BroadcastNestedLoopJoin(
255-
planLater(left), planLater(right), joins.BuildLeft, j.joinType, condition) :: Nil
263+
planLater(left), planLater(right), joins.BuildLeft, j.joinType,
264+
reorderPredicates(condition)) :: Nil
256265
case j @ logical.Join(left, CanBroadcast(right), Inner | LeftOuter | LeftSemi, condition) =>
257266
execution.joins.BroadcastNestedLoopJoin(
258-
planLater(left), planLater(right), joins.BuildRight, j.joinType, condition) :: Nil
267+
planLater(left), planLater(right), joins.BuildRight, j.joinType,
268+
reorderPredicates(condition)) :: Nil
259269
case _ => Nil
260270
}
261271
}
@@ -265,7 +275,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
265275
case logical.Join(left, right, Inner, None) =>
266276
execution.joins.CartesianProduct(planLater(left), planLater(right)) :: Nil
267277
case logical.Join(left, right, Inner, Some(condition)) =>
268-
execution.Filter(condition,
278+
execution.Filter(reorderPredicates(condition),
269279
execution.joins.CartesianProduct(planLater(left), planLater(right))) :: Nil
270280
case _ => Nil
271281
}
@@ -282,7 +292,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
282292
}
283293
// This join could be very slow or even hang forever
284294
joins.BroadcastNestedLoopJoin(
285-
planLater(left), planLater(right), buildSide, joinType, condition) :: Nil
295+
planLater(left), planLater(right), buildSide, joinType,
296+
reorderPredicates(condition)) :: Nil
286297
case _ => Nil
287298
}
288299
}
@@ -341,7 +352,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
341352
case logical.Project(projectList, child) =>
342353
execution.Project(projectList, planLater(child)) :: Nil
343354
case logical.Filter(condition, child) =>
344-
execution.Filter(condition, planLater(child)) :: Nil
355+
execution.Filter(reorderPredicates(condition), planLater(child)) :: Nil
345356
case e @ logical.Expand(_, _, child) =>
346357
execution.Expand(e.projections, e.output, planLater(child)) :: Nil
347358
case logical.Window(projectList, windowExprs, partitionSpec, orderSpec, child) =>
Lines changed: 103 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,103 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.execution
19+
20+
import org.apache.spark.sql.catalyst.expressions.{Expression, IsNotNull, PredicateHelper}
21+
import org.apache.spark.sql.catalyst.plans.logical
22+
import org.apache.spark.sql.catalyst.plans.logical.Join
23+
import org.apache.spark.sql.execution
24+
import org.apache.spark.sql.execution.joins.LeftSemiJoinHash
25+
import org.apache.spark.sql.test.SharedSQLContext
26+
27+
28+
class ReorderedPredicateSuite extends SharedSQLContext with PredicateHelper {
29+
30+
setupTestData()
31+
32+
// Verifies that (a) In the new condition, the IsNotNull operators precede rest of the operators
33+
// and (b) The relative sort order of IsNotNull and !IsNotNull operators is still maintained
34+
private def verifyStableOrder(before: Expression, after: Expression): Unit = {
35+
val oldPredicates = splitConjunctivePredicates(before)
36+
splitConjunctivePredicates(after).sliding(2).foreach { case Seq(x, y) =>
37+
// Verify IsNotNull operator ordering
38+
assert(x.isInstanceOf[IsNotNull] || !y.isInstanceOf[IsNotNull])
39+
40+
// Verify stable sort order
41+
if ((x.isInstanceOf[IsNotNull] && y.isInstanceOf[IsNotNull]) ||
42+
(!x.isInstanceOf[IsNotNull] && !y.isInstanceOf[IsNotNull])) {
43+
assert(oldPredicates.indexOf(x) <= oldPredicates.indexOf(y))
44+
}
45+
}
46+
}
47+
48+
test("null ordering in filter predicates") {
49+
val query = sql(
50+
"""
51+
|SELECT * from testData
52+
|WHERE value != '5' AND value != '4' AND value IS NOT NULL AND key != 5
53+
""".stripMargin)
54+
.queryExecution
55+
56+
val logicalPlan = query.optimizedPlan
57+
val physicalPlan = query.sparkPlan
58+
assert(logicalPlan.find(_.isInstanceOf[logical.Filter]).isDefined)
59+
assert(physicalPlan.find(_.isInstanceOf[execution.Filter]).isDefined)
60+
61+
val logicalCondition = logicalPlan.collect {
62+
case logical.Filter(condition, _) =>
63+
condition
64+
}.head
65+
66+
val physicalCondition = physicalPlan.collect {
67+
case Filter(condition, _) =>
68+
condition
69+
}.head
70+
71+
verifyStableOrder(logicalCondition, physicalCondition)
72+
}
73+
74+
test("null ordering in join predicates") {
75+
sqlContext.cacheManager.clearCache()
76+
val query = sql(
77+
"""
78+
|SELECT * FROM testData t1
79+
|LEFT SEMI JOIN testData t2
80+
|ON t1.key = t2.key
81+
|AND t1.key + t2.key != 5
82+
|AND CONCAT(t1.value, t2.value) IS NOT NULL
83+
""".stripMargin)
84+
.queryExecution
85+
86+
val logicalPlan = query.optimizedPlan
87+
val physicalPlan = query.sparkPlan
88+
assert(logicalPlan.find(_.isInstanceOf[Join]).isDefined)
89+
assert(physicalPlan.find(_.isInstanceOf[LeftSemiJoinHash]).isDefined)
90+
91+
val logicalCondition = logicalPlan.collect {
92+
case Join(_, _, _, condition) =>
93+
condition.get
94+
}.head
95+
96+
val physicalCondition = physicalPlan.collect {
97+
case LeftSemiJoinHash(_, _, _, _, conditionOpt) =>
98+
conditionOpt.get
99+
}.head
100+
101+
verifyStableOrder(logicalCondition, physicalCondition)
102+
}
103+
}

0 commit comments

Comments
 (0)