Skip to content

Commit cec78b5

Browse files
committed
Update OuterJoinEliminationSuite
1 parent a032106 commit cec78b5

File tree

4 files changed

+38
-44
lines changed

4 files changed

+38
-44
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala

Lines changed: 10 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -77,12 +77,12 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: SQLConf)
7777
// Operator push down
7878
PushProjectionThroughUnion,
7979
ReorderJoin(conf),
80-
EliminateOuterJoin(conf),
80+
EliminateOuterJoin,
8181
PushPredicateThroughJoin,
8282
PushDownPredicate,
8383
LimitPushDown(conf),
8484
ColumnPruning,
85-
InferFiltersFromConstraints(conf),
85+
InferFiltersFromConstraints,
8686
// Operator combine
8787
CollapseRepartition,
8888
CollapseProject,
@@ -619,14 +619,15 @@ object CollapseWindow extends Rule[LogicalPlan] {
619619
* Note: While this optimization is applicable to all types of join, it primarily benefits Inner and
620620
* LeftSemi joins.
621621
*/
622-
case class InferFiltersFromConstraints(conf: SQLConf)
623-
extends Rule[LogicalPlan] with PredicateHelper {
624-
def apply(plan: LogicalPlan): LogicalPlan = if (conf.constraintPropagationEnabled) {
625-
inferFilters(plan)
626-
} else {
627-
plan
628-
}
622+
object InferFiltersFromConstraints extends Rule[LogicalPlan] with PredicateHelper {
629623

624+
def apply(plan: LogicalPlan): LogicalPlan = {
625+
if (SQLConf.get.constraintPropagationEnabled) {
626+
inferFilters(plan)
627+
} else {
628+
plan
629+
}
630+
}
630631

631632
private def inferFilters(plan: LogicalPlan): LogicalPlan = plan transform {
632633
case filter @ Filter(condition, child) =>

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -113,7 +113,7 @@ case class ReorderJoin(conf: SQLConf) extends Rule[LogicalPlan] with PredicateHe
113113
*
114114
* This rule should be executed before pushing down the Filter
115115
*/
116-
case class EliminateOuterJoin(conf: SQLConf) extends Rule[LogicalPlan] with PredicateHelper {
116+
object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper {
117117

118118
/**
119119
* Returns whether the expression returns null or false when all inputs are nulls.

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala

Lines changed: 10 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._
2323
import org.apache.spark.sql.catalyst.plans._
2424
import org.apache.spark.sql.catalyst.plans.logical._
2525
import org.apache.spark.sql.catalyst.rules._
26-
import org.apache.spark.sql.internal.SQLConf.CONSTRAINT_PROPAGATION_ENABLED
26+
import org.apache.spark.sql.internal.SQLConf
2727

2828
class InferFiltersFromConstraintsSuite extends PlanTest {
2929

@@ -32,20 +32,11 @@ class InferFiltersFromConstraintsSuite extends PlanTest {
3232
Batch("InferAndPushDownFilters", FixedPoint(100),
3333
PushPredicateThroughJoin,
3434
PushDownPredicate,
35-
InferFiltersFromConstraints(conf),
35+
InferFiltersFromConstraints,
3636
CombineFilters,
3737
BooleanSimplification) :: Nil
3838
}
3939

40-
object OptimizeWithConstraintPropagationDisabled extends RuleExecutor[LogicalPlan] {
41-
val batches =
42-
Batch("InferAndPushDownFilters", FixedPoint(100),
43-
PushPredicateThroughJoin,
44-
PushDownPredicate,
45-
InferFiltersFromConstraints(conf.copy(CONSTRAINT_PROPAGATION_ENABLED -> false)),
46-
CombineFilters) :: Nil
47-
}
48-
4940
val testRelation = LocalRelation('a.int, 'b.int, 'c.int)
5041

5142
test("filter: filter out constraints in condition") {
@@ -215,8 +206,13 @@ class InferFiltersFromConstraintsSuite extends PlanTest {
215206
}
216207

217208
test("No inferred filter when constraint propagation is disabled") {
218-
val originalQuery = testRelation.where('a === 1 && 'a === 'b).analyze
219-
val optimized = OptimizeWithConstraintPropagationDisabled.execute(originalQuery)
220-
comparePlans(optimized, originalQuery)
209+
try {
210+
SQLConf.get.setConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED, false)
211+
val originalQuery = testRelation.where('a === 1 && 'a === 'b).analyze
212+
val optimized = Optimize.execute(originalQuery)
213+
comparePlans(optimized, originalQuery)
214+
} finally {
215+
SQLConf.get.unsetConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED)
216+
}
221217
}
222218
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala

Lines changed: 17 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -24,24 +24,15 @@ import org.apache.spark.sql.catalyst.expressions.{Coalesce, IsNotNull}
2424
import org.apache.spark.sql.catalyst.plans._
2525
import org.apache.spark.sql.catalyst.plans.logical._
2626
import org.apache.spark.sql.catalyst.rules._
27-
import org.apache.spark.sql.internal.SQLConf.CONSTRAINT_PROPAGATION_ENABLED
27+
import org.apache.spark.sql.internal.SQLConf
2828

2929
class OuterJoinEliminationSuite extends PlanTest {
3030
object Optimize extends RuleExecutor[LogicalPlan] {
3131
val batches =
3232
Batch("Subqueries", Once,
3333
EliminateSubqueryAliases) ::
3434
Batch("Outer Join Elimination", Once,
35-
EliminateOuterJoin(conf),
36-
PushPredicateThroughJoin) :: Nil
37-
}
38-
39-
object OptimizeWithConstraintPropagationDisabled extends RuleExecutor[LogicalPlan] {
40-
val batches =
41-
Batch("Subqueries", Once,
42-
EliminateSubqueryAliases) ::
43-
Batch("Outer Join Elimination", Once,
44-
EliminateOuterJoin(conf.copy(CONSTRAINT_PROPAGATION_ENABLED -> false)),
35+
EliminateOuterJoin,
4536
PushPredicateThroughJoin) :: Nil
4637
}
4738

@@ -243,19 +234,25 @@ class OuterJoinEliminationSuite extends PlanTest {
243234
}
244235

245236
test("no outer join elimination if constraint propagation is disabled") {
246-
val x = testRelation.subquery('x)
247-
val y = testRelation1.subquery('y)
237+
try {
238+
SQLConf.get.setConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED, false)
248239

249-
// The predicate "x.b + y.d >= 3" will be inferred constraints like:
250-
// "x.b != null" and "y.d != null", if constraint propagation is enabled.
251-
// When we disable it, the predicate can't be evaluated on left or right plan and used to
252-
// filter out nulls. So the Outer Join will not be eliminated.
253-
val originalQuery =
240+
val x = testRelation.subquery('x)
241+
val y = testRelation1.subquery('y)
242+
243+
// The predicate "x.b + y.d >= 3" will be inferred constraints like:
244+
// "x.b != null" and "y.d != null", if constraint propagation is enabled.
245+
// When we disable it, the predicate can't be evaluated on left or right plan and used to
246+
// filter out nulls. So the Outer Join will not be eliminated.
247+
val originalQuery =
254248
x.join(y, FullOuter, Option("x.a".attr === "y.d".attr))
255249
.where("x.b".attr + "y.d".attr >= 3)
256250

257-
val optimized = OptimizeWithConstraintPropagationDisabled.execute(originalQuery.analyze)
251+
val optimized = Optimize.execute(originalQuery.analyze)
258252

259-
comparePlans(optimized, originalQuery.analyze)
253+
comparePlans(optimized, originalQuery.analyze)
254+
} finally {
255+
SQLConf.get.unsetConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED)
256+
}
260257
}
261258
}

0 commit comments

Comments
 (0)