Skip to content

Commit 737eecd

Browse files
EnricoMicloud-fan
authored andcommitted
[SPARK-41162][SQL] Fix anti- and semi-join for self-join with aggregations
### What changes were proposed in this pull request? Rule `PushDownLeftSemiAntiJoin` should not push an anti-join below an `Aggregate` when the join condition references an attribute that exists in its right plan and its left plan's child. This usually happens when the anti-join / semi-join is a self-join while `DeduplicateRelations` cannot deduplicate those attributes (in this example due to the projection of `value` to `id`). This behaviour already exists for `Project` and `Union`, but `Aggregate` lacks this safety guard. ### Why are the changes needed? Without this change, the optimizer creates an incorrect plan. This example fails with `distinct()` (an aggregation), and succeeds without `distinct()`, but both queries are identical: ```scala val ids = Seq(1, 2, 3).toDF("id").distinct() val result = ids.withColumn("id", $"id" + 1).join(ids, Seq("id"), "left_anti").collect() assert(result.length == 1) ``` With `distinct()`, rule `PushDownLeftSemiAntiJoin` creates a join condition `(value#907 + 1) = value#907`, which can never be true. This effectively removes the anti-join. **Before this PR:** The anti-join is fully removed from the plan. ``` == Physical Plan == AdaptiveSparkPlan (16) +- == Final Plan == LocalTableScan (1) (16) AdaptiveSparkPlan Output [1]: [id#900] Arguments: isFinalPlan=true ``` This is caused by `PushDownLeftSemiAntiJoin` adding join condition `(value#907 + 1) = value#907`, which is wrong as because `id#910` in `(id#910 + 1) AS id#912` exists in the right child of the join as well as in the left grandchild: ``` === Applying Rule org.apache.spark.sql.catalyst.optimizer.PushDownLeftSemiAntiJoin === !Join LeftAnti, (id#912 = id#910) Aggregate [id#910], [(id#910 + 1) AS id#912] !:- Aggregate [id#910], [(id#910 + 1) AS id#912] +- Project [value#907 AS id#910] !: +- Project [value#907 AS id#910] +- Join LeftAnti, ((value#907 + 1) = value#907) !: +- LocalRelation [value#907] :- LocalRelation [value#907] !+- Aggregate [id#910], [id#910] +- Aggregate [id#910], [id#910] ! +- Project [value#914 AS id#910] +- Project [value#914 AS id#910] ! +- LocalRelation [value#914] +- LocalRelation [value#914] ``` The right child of the join and in the left grandchild would become the children of the pushed-down join, which creates an invalid join condition. **After this PR:** Join condition `(id#910 + 1) AS id#912` is understood to become ambiguous as both sides of the prospect join contain `id#910`. Hence, the join is not pushed down. The rule is then not applied any more. The final plan contains the anti-join: ``` == Physical Plan == AdaptiveSparkPlan (24) +- == Final Plan == * BroadcastHashJoin LeftSemi BuildRight (14) :- * HashAggregate (7) : +- AQEShuffleRead (6) : +- ShuffleQueryStage (5), Statistics(sizeInBytes=48.0 B, rowCount=3) : +- Exchange (4) : +- * HashAggregate (3) : +- * Project (2) : +- * LocalTableScan (1) +- BroadcastQueryStage (13), Statistics(sizeInBytes=1024.0 KiB, rowCount=3) +- BroadcastExchange (12) +- * HashAggregate (11) +- AQEShuffleRead (10) +- ShuffleQueryStage (9), Statistics(sizeInBytes=48.0 B, rowCount=3) +- ReusedExchange (8) (8) ReusedExchange [Reuses operator id: 4] Output [1]: [id#898] (24) AdaptiveSparkPlan Output [1]: [id#900] Arguments: isFinalPlan=true ``` ### Does this PR introduce _any_ user-facing change? It fixes correctness. ### How was this patch tested? Unit tests in `DataFrameJoinSuite` and `LeftSemiAntiJoinPushDownSuite`. Closes #39131 from EnricoMi/branch-antijoin-selfjoin-fix. Authored-by: Enrico Minack <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
1 parent 272634b commit 737eecd

File tree

3 files changed

+63
-25
lines changed

3 files changed

+63
-25
lines changed

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

Lines changed: 7 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -61,9 +61,10 @@ object PushDownLeftSemiAntiJoin extends Rule[LogicalPlan]
6161
}
6262

6363
// LeftSemi/LeftAnti over Aggregate, only push down if join can be planned as broadcast join.
64-
case join @ Join(agg: Aggregate, rightOp, LeftSemiOrAnti(_), _, _)
64+
case join @ Join(agg: Aggregate, rightOp, LeftSemiOrAnti(_), joinCond, _)
6565
if agg.aggregateExpressions.forall(_.deterministic) && agg.groupingExpressions.nonEmpty &&
6666
!agg.aggregateExpressions.exists(ScalarSubquery.hasCorrelatedScalarSubquery) &&
67+
canPushThroughCondition(agg.children, joinCond, rightOp) &&
6768
canPlanAsBroadcastHashJoin(join, conf) =>
6869
val aliasMap = getAliasMap(agg)
6970
val canPushDownPredicate = (predicate: Expression) => {
@@ -110,11 +111,11 @@ object PushDownLeftSemiAntiJoin extends Rule[LogicalPlan]
110111
}
111112

112113
/**
113-
* Check if we can safely push a join through a project or union by making sure that attributes
114-
* referred in join condition do not contain the same attributes as the plan they are moved
115-
* into. This can happen when both sides of join refers to the same source (self join). This
116-
* function makes sure that the join condition refers to attributes that are not ambiguous (i.e
117-
* present in both the legs of the join) or else the resultant plan will be invalid.
114+
* Check if we can safely push a join through a project, aggregate, or union by making sure that
115+
* attributes referred in join condition do not contain the same attributes as the plan they are
116+
* moved into. This can happen when both sides of join refers to the same source (self join).
117+
* This function makes sure that the join condition refers to attributes that are not ambiguous
118+
* (i.e present in both the legs of the join) or else the resultant plan will be invalid.
118119
*/
119120
private def canPushThroughCondition(
120121
plans: Seq[LogicalPlan],

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

Lines changed: 38 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.rules._
2727
import org.apache.spark.sql.internal.SQLConf
2828
import org.apache.spark.sql.types.IntegerType
2929

30-
class LeftSemiPushdownSuite extends PlanTest {
30+
class LeftSemiAntiJoinPushDownSuite extends PlanTest {
3131

3232
object Optimize extends RuleExecutor[LogicalPlan] {
3333
val batches =
@@ -46,7 +46,7 @@ class LeftSemiPushdownSuite extends PlanTest {
4646
val testRelation1 = LocalRelation($"d".int)
4747
val testRelation2 = LocalRelation($"e".int)
4848

49-
test("Project: LeftSemiAnti join pushdown") {
49+
test("Project: LeftSemi join pushdown") {
5050
val originalQuery = testRelation
5151
.select(star())
5252
.join(testRelation1, joinType = LeftSemi, condition = Some($"b" === $"d"))
@@ -59,7 +59,7 @@ class LeftSemiPushdownSuite extends PlanTest {
5959
comparePlans(optimized, correctAnswer)
6060
}
6161

62-
test("Project: LeftSemiAnti join no pushdown because of non-deterministic proj exprs") {
62+
test("Project: LeftSemi join no pushdown - non-deterministic proj exprs") {
6363
val originalQuery = testRelation
6464
.select(Rand(1), $"b", $"c")
6565
.join(testRelation1, joinType = LeftSemi, condition = Some($"b" === $"d"))
@@ -68,7 +68,7 @@ class LeftSemiPushdownSuite extends PlanTest {
6868
comparePlans(optimized, originalQuery.analyze)
6969
}
7070

71-
test("Project: LeftSemiAnti join non correlated scalar subq") {
71+
test("Project: LeftSemi join pushdown - non-correlated scalar subq") {
7272
val subq = ScalarSubquery(testRelation.groupBy($"b")(sum($"c").as("sum")).analyze)
7373
val originalQuery = testRelation
7474
.select(subq.as("sum"))
@@ -83,7 +83,7 @@ class LeftSemiPushdownSuite extends PlanTest {
8383
comparePlans(optimized, correctAnswer)
8484
}
8585

86-
test("Project: LeftSemiAnti join no pushdown - correlated scalar subq in projection list") {
86+
test("Project: LeftSemi join no pushdown - correlated scalar subq in projection list") {
8787
val testRelation2 = LocalRelation($"e".int, $"f".int)
8888
val subqPlan = testRelation2.groupBy($"e")(sum($"f").as("sum")).where($"e" === $"a")
8989
val subqExpr = ScalarSubquery(subqPlan)
@@ -95,7 +95,7 @@ class LeftSemiPushdownSuite extends PlanTest {
9595
comparePlans(optimized, originalQuery.analyze)
9696
}
9797

98-
test("Aggregate: LeftSemiAnti join pushdown") {
98+
test("Aggregate: LeftSemi join pushdown") {
9999
val originalQuery = testRelation
100100
.groupBy($"b")($"b", sum($"c"))
101101
.join(testRelation1, joinType = LeftSemi, condition = Some($"b" === $"d"))
@@ -109,7 +109,7 @@ class LeftSemiPushdownSuite extends PlanTest {
109109
comparePlans(optimized, correctAnswer)
110110
}
111111

112-
test("Aggregate: LeftSemiAnti join no pushdown due to non-deterministic aggr expressions") {
112+
test("Aggregate: LeftSemi join no pushdown - non-deterministic aggr expressions") {
113113
val originalQuery = testRelation
114114
.groupBy($"b")($"b", Rand(10).as("c"))
115115
.join(testRelation1, joinType = LeftSemi, condition = Some($"b" === $"d"))
@@ -142,7 +142,7 @@ class LeftSemiPushdownSuite extends PlanTest {
142142
comparePlans(optimized, originalQuery.analyze)
143143
}
144144

145-
test("LeftSemiAnti join over aggregate - no pushdown") {
145+
test("Aggregate: LeftSemi join no pushdown") {
146146
val originalQuery = testRelation
147147
.groupBy($"b")($"b", sum($"c").as("sum"))
148148
.join(testRelation1, joinType = LeftSemi, condition = Some($"b" === $"d" && $"sum" === $"d"))
@@ -151,7 +151,7 @@ class LeftSemiPushdownSuite extends PlanTest {
151151
comparePlans(optimized, originalQuery.analyze)
152152
}
153153

154-
test("Aggregate: LeftSemiAnti join non-correlated scalar subq aggr exprs") {
154+
test("Aggregate: LeftSemi join pushdown - non-correlated scalar subq aggr exprs") {
155155
val subq = ScalarSubquery(testRelation.groupBy($"b")(sum($"c").as("sum")).analyze)
156156
val originalQuery = testRelation
157157
.groupBy($"a") ($"a", subq.as("sum"))
@@ -166,7 +166,7 @@ class LeftSemiPushdownSuite extends PlanTest {
166166
comparePlans(optimized, correctAnswer)
167167
}
168168

169-
test("LeftSemiAnti join over Window") {
169+
test("Window: LeftSemi join pushdown") {
170170
val winExpr = windowExpr(count($"b"),
171171
windowSpec($"a" :: Nil, $"b".asc :: Nil, UnspecifiedFrame))
172172

@@ -185,7 +185,7 @@ class LeftSemiPushdownSuite extends PlanTest {
185185
comparePlans(optimized, correctAnswer)
186186
}
187187

188-
test("Window: LeftSemi partial pushdown") {
188+
test("Window: LeftSemi join partial pushdown") {
189189
// Attributes from join condition which does not refer to the window partition spec
190190
// are kept up in the plan as a Filter operator above Window.
191191
val winExpr = windowExpr(count($"b"),
@@ -227,7 +227,7 @@ class LeftSemiPushdownSuite extends PlanTest {
227227
comparePlans(optimized, correctAnswer)
228228
}
229229

230-
test("Union: LeftSemiAnti join pushdown") {
230+
test("Union: LeftSemi join pushdown") {
231231
val testRelation2 = LocalRelation($"x".int, $"y".int, $"z".int)
232232

233233
val originalQuery = Union(Seq(testRelation, testRelation2))
@@ -243,7 +243,7 @@ class LeftSemiPushdownSuite extends PlanTest {
243243
comparePlans(optimized, correctAnswer)
244244
}
245245

246-
test("Union: LeftSemiAnti join pushdown in self join scenario") {
246+
test("Union: LeftSemi join pushdown in self join scenario") {
247247
val testRelation2 = LocalRelation($"x".int, $"y".int, $"z".int)
248248
val attrX = testRelation2.output.head
249249

@@ -262,7 +262,7 @@ class LeftSemiPushdownSuite extends PlanTest {
262262
comparePlans(optimized, correctAnswer)
263263
}
264264

265-
test("Unary: LeftSemiAnti join pushdown") {
265+
test("Unary: LeftSemi join pushdown") {
266266
val originalQuery = testRelation
267267
.select(star())
268268
.repartition(1)
@@ -277,7 +277,7 @@ class LeftSemiPushdownSuite extends PlanTest {
277277
comparePlans(optimized, correctAnswer)
278278
}
279279

280-
test("Unary: LeftSemiAnti join pushdown - empty join condition") {
280+
test("Unary: LeftSemi join pushdown - empty join condition") {
281281
val originalQuery = testRelation
282282
.select(star())
283283
.repartition(1)
@@ -292,7 +292,7 @@ class LeftSemiPushdownSuite extends PlanTest {
292292
comparePlans(optimized, correctAnswer)
293293
}
294294

295-
test("Unary: LeftSemi join pushdown - partial pushdown") {
295+
test("Unary: LeftSemi join partial pushdown") {
296296
val testRelationWithArrayType = LocalRelation($"a".int, $"b".int, $"c_arr".array(IntegerType))
297297
val originalQuery = testRelationWithArrayType
298298
.generate(Explode($"c_arr"), alias = Some("arr"), outputNames = Seq("out_col"))
@@ -309,7 +309,7 @@ class LeftSemiPushdownSuite extends PlanTest {
309309
comparePlans(optimized, correctAnswer)
310310
}
311311

312-
test("Unary: LeftAnti join pushdown - no pushdown") {
312+
test("Unary: LeftAnti join no pushdown") {
313313
val testRelationWithArrayType = LocalRelation($"a".int, $"b".int, $"c_arr".array(IntegerType))
314314
val originalQuery = testRelationWithArrayType
315315
.generate(Explode($"c_arr"), alias = Some("arr"), outputNames = Seq("out_col"))
@@ -320,7 +320,7 @@ class LeftSemiPushdownSuite extends PlanTest {
320320
comparePlans(optimized, originalQuery.analyze)
321321
}
322322

323-
test("Unary: LeftSemiAnti join pushdown - no pushdown") {
323+
test("Unary: LeftSemi join - no pushdown") {
324324
val testRelationWithArrayType = LocalRelation($"a".int, $"b".int, $"c_arr".array(IntegerType))
325325
val originalQuery = testRelationWithArrayType
326326
.generate(Explode($"c_arr"), alias = Some("arr"), outputNames = Seq("out_col"))
@@ -331,7 +331,7 @@ class LeftSemiPushdownSuite extends PlanTest {
331331
comparePlans(optimized, originalQuery.analyze)
332332
}
333333

334-
test("Unary: LeftSemi join push down through Expand") {
334+
test("Unary: LeftSemi join pushdown through Expand") {
335335
val expand = Expand(Seq(Seq($"a", $"b", "null"), Seq($"a", "null", $"c")),
336336
Seq($"a", $"b", $"c"), testRelation)
337337
val originalQuery = expand
@@ -437,6 +437,25 @@ class LeftSemiPushdownSuite extends PlanTest {
437437
}
438438
}
439439

440+
Seq(LeftSemi, LeftAnti).foreach { case jt =>
441+
test(s"Aggregate: $jt join no pushdown - join condition refers left leg and right leg child") {
442+
val aggregation = testRelation
443+
.select($"b".as("id"), $"c")
444+
.groupBy($"id")($"id", sum($"c").as("sum"))
445+
446+
// reference "b" exists in left leg, and the children of the right leg of the join
447+
val originalQuery = aggregation.select(($"id" + 1).as("id_plus_1"), $"sum")
448+
.join(aggregation, joinType = jt, condition = Some($"id" === $"id_plus_1"))
449+
val optimized = Optimize.execute(originalQuery.analyze)
450+
val correctAnswer = testRelation
451+
.select($"b".as("id"), $"c")
452+
.groupBy($"id")(($"id" + 1).as("id_plus_1"), sum($"c").as("sum"))
453+
.join(aggregation, joinType = jt, condition = Some($"id" === $"id_plus_1"))
454+
.analyze
455+
comparePlans(optimized, correctAnswer)
456+
}
457+
}
458+
440459
Seq(LeftSemi, LeftAnti).foreach { case outerJT =>
441460
Seq(Inner, LeftOuter, RightOuter, Cross).foreach { case innerJT =>
442461
test(s"$outerJT no pushdown - join condition refers none of the leg - join type $innerJT") {

sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -344,6 +344,24 @@ class DataFrameJoinSuite extends QueryTest
344344
}
345345
}
346346

347+
Seq("left_semi", "left_anti").foreach { joinType =>
348+
test(s"SPARK-41162: $joinType self-joined aggregated dataframe") {
349+
// aggregated dataframe
350+
val ids = Seq(1, 2, 3).toDF("id").distinct()
351+
352+
// self-joined via joinType
353+
val result = ids.withColumn("id", $"id" + 1)
354+
.join(ids, "id", joinType).collect()
355+
356+
val expected = joinType match {
357+
case "left_semi" => 2
358+
case "left_anti" => 1
359+
case _ => -1 // unsupported test type, test will always fail
360+
}
361+
assert(result.length == expected)
362+
}
363+
}
364+
347365
def extractLeftDeepInnerJoins(plan: LogicalPlan): Seq[LogicalPlan] = plan match {
348366
case j @ Join(left, right, _: InnerLike, _, _) => right +: extractLeftDeepInnerJoins(left)
349367
case Filter(_, child) => extractLeftDeepInnerJoins(child)

0 commit comments

Comments
 (0)