Skip to content

Commit 2af656a

Browse files
committed
[SPARK-26078][SQL] Dedup self-join attributes on subqueries
1 parent 4ac8f9b commit 2af656a

File tree

2 files changed

+59
-8
lines changed

2 files changed

+59
-8
lines changed

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

Lines changed: 28 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -54,10 +54,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper {
5454
val aliasMap = AttributeMap(duplicates.map { dup =>
5555
dup -> Alias(dup, dup.toString)()
5656
}.toSeq)
57-
val aliasedExpressions = right.output.map { ref =>
58-
aliasMap.getOrElse(ref, ref)
59-
}
60-
val newRight = Project(aliasedExpressions, right)
57+
val newRight = rewriteDedupPlan(right, aliasMap)
6158
val newJoinCond = joinCond.map { condExpr =>
6259
condExpr transform {
6360
case a: Attribute => aliasMap.getOrElse(a, a).toAttribute
@@ -70,6 +67,27 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper {
7067
case _ => joinPlan
7168
}
7269

70+
private def rewriteDedupPlan(plan: LogicalPlan, rewrites: AttributeMap[Alias]): LogicalPlan = {
71+
val aliasedExpressions = plan.output.map { ref =>
72+
rewrites.getOrElse(ref, ref)
73+
}
74+
Project(aliasedExpressions, plan)
75+
}
76+
77+
private def dedupSubqueryOnSelfJoin(values: Seq[Expression], sub: LogicalPlan): LogicalPlan = {
78+
val leftRefs = AttributeSet.fromAttributeSets(values.map(_.references))
79+
val rightRefs = AttributeSet(sub.output)
80+
val duplicates = leftRefs.intersect(rightRefs)
81+
if (duplicates.isEmpty) {
82+
sub
83+
} else {
84+
val aliasMap = AttributeMap(duplicates.map { dup =>
85+
dup -> Alias(dup, dup.toString)()
86+
}.toSeq)
87+
rewriteDedupPlan(sub, aliasMap)
88+
}
89+
}
90+
7391
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
7492
case Filter(condition, child) =>
7593
val (withSubquery, withoutSubquery) =
@@ -92,18 +110,20 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper {
92110
// Deduplicate conflicting attributes if any.
93111
dedupJoin(Join(outerPlan, sub, LeftAnti, joinCond))
94112
case (p, InSubquery(values, ListQuery(sub, conditions, _, _))) =>
95-
val inConditions = values.zip(sub.output).map(EqualTo.tupled)
113+
val newSub = dedupSubqueryOnSelfJoin(values, sub)
114+
val inConditions = values.zip(newSub.output).map(EqualTo.tupled)
96115
val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions ++ conditions, p)
97116
// Deduplicate conflicting attributes if any.
98-
dedupJoin(Join(outerPlan, sub, LeftSemi, joinCond))
117+
dedupJoin(Join(outerPlan, newSub, LeftSemi, joinCond))
99118
case (p, Not(InSubquery(values, ListQuery(sub, conditions, _, _)))) =>
100119
// This is a NULL-aware (left) anti join (NAAJ) e.g. col NOT IN expr
101120
// Construct the condition. A NULL in one of the conditions is regarded as a positive
102121
// result; such a row will be filtered out by the Anti-Join operator.
103122

104123
// Note that will almost certainly be planned as a Broadcast Nested Loop join.
105124
// Use EXISTS if performance matters to you.
106-
val inConditions = values.zip(sub.output).map(EqualTo.tupled)
125+
val newSub = dedupSubqueryOnSelfJoin(values, sub)
126+
val inConditions = values.zip(newSub.output).map(EqualTo.tupled)
107127
val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions, p)
108128
// Expand the NOT IN expression with the NULL-aware semantic
109129
// to its full form. That is from:
@@ -119,7 +139,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper {
119139
// (A.A1 = B.B1 OR ISNULL(A.A1 = B.B1)) AND (B.B2 = A.A2) AND B.B3 > 1
120140
val finalJoinCond = (nullAwareJoinConds ++ conditions).reduceLeft(And)
121141
// Deduplicate conflicting attributes if any.
122-
dedupJoin(Join(outerPlan, sub, LeftAnti, Option(finalJoinCond)))
142+
dedupJoin(Join(outerPlan, newSub, LeftAnti, Option(finalJoinCond)))
123143
case (p, predicate) =>
124144
val (newCond, inputPlan) = rewriteExistentialExpr(Seq(predicate), p)
125145
Project(p.output, Filter(newCond.get, inputPlan))

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

Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer
2121

2222
import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
2323
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort}
24+
import org.apache.spark.sql.types._
2425
import org.apache.spark.sql.test.SharedSQLContext
2526

2627
class SubquerySuite extends QueryTest with SharedSQLContext {
@@ -1280,4 +1281,34 @@ class SubquerySuite extends QueryTest with SharedSQLContext {
12801281
assert(subqueries.length == 1)
12811282
}
12821283
}
1284+
1285+
test("SPARK-26078: deduplicate fake self joins for IN subqueries") {
1286+
withTempView("a", "b") {
1287+
val a = spark.createDataFrame(spark.sparkContext.parallelize(Seq(Row("a", 2), Row("b", 1))),
1288+
StructType(Seq(StructField("id", StringType), StructField("num", IntegerType))))
1289+
val b = spark.createDataFrame(spark.sparkContext.parallelize(Seq(Row("a", 2), Row("b", 1))),
1290+
StructType(Seq(StructField("id", StringType), StructField("num", IntegerType))))
1291+
a.createOrReplaceTempView("a")
1292+
b.createOrReplaceTempView("b")
1293+
1294+
val df1 = spark.sql(
1295+
"""
1296+
|SELECT id,num,source FROM (
1297+
| SELECT id, num, 'a' as source FROM a
1298+
| UNION ALL
1299+
| SELECT id, num, 'b' as source FROM b
1300+
|) AS c WHERE c.id IN (SELECT id FROM b WHERE num = 2)
1301+
""".stripMargin)
1302+
checkAnswer(df1, Seq(Row("a", 2, "a"), Row("a", 2, "b")))
1303+
val df2 = spark.sql(
1304+
"""
1305+
|SELECT id,num,source FROM (
1306+
| SELECT id, num, 'a' as source FROM a
1307+
| UNION ALL
1308+
| SELECT id, num, 'b' as source FROM b
1309+
|) AS c WHERE c.id NOT IN (SELECT id FROM b WHERE num = 2)
1310+
""".stripMargin)
1311+
checkAnswer(df2, Seq(Row("b", 1, "a"), Row("b", 1, "b")))
1312+
}
1313+
}
12831314
}

0 commit comments

Comments
 (0)