Skip to content

Commit 08ede46

Browse files
cloud-fangatorsmile
authored andcommitted
[SPARK-20897][SQL] cached self-join should not fail
## What changes were proposed in this pull request? The failed test case is, we have a `SortMergeJoinExec` for a self-join, which means we have a `ReusedExchange` node in the query plan. It works fine without caching, but throws an exception in `SortMergeJoinExec.outputPartitioning` if we cache it. The root cause is, `ReusedExchange` doesn't propagate the output partitioning from its child, so in `SortMergeJoinExec.outputPartitioning` we create `PartitioningCollection` with a hash partitioning and an unknown partitioning, and fail. This bug is mostly fine, because inserting the `ReusedExchange` is the last step to prepare the physical plan, we won't call `SortMergeJoinExec.outputPartitioning` anymore after this. However, if the dataframe is cached, the physical plan of it becomes `InMemoryTableScanExec`, which contains another physical plan representing the cached query, and it has gone through the entire planning phase and may have `ReusedExchange`. Then the planner call `InMemoryTableScanExec.outputPartitioning`, which then calls `SortMergeJoinExec.outputPartitioning` and trigger this bug. ## How was this patch tested? a new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #18121 from cloud-fan/bug.
1 parent 8faffc4 commit 08ede46

File tree

2 files changed

+30
-1
lines changed

2 files changed

+30
-1
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala

Lines changed: 20 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,8 @@ import scala.collection.mutable.ArrayBuffer
2323
import org.apache.spark.broadcast
2424
import org.apache.spark.rdd.RDD
2525
import org.apache.spark.sql.catalyst.InternalRow
26-
import org.apache.spark.sql.catalyst.expressions.Attribute
26+
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression, SortOrder}
27+
import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning}
2728
import org.apache.spark.sql.catalyst.rules.Rule
2829
import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, UnaryExecNode}
2930
import org.apache.spark.sql.internal.SQLConf
@@ -58,6 +59,24 @@ case class ReusedExchangeExec(override val output: Seq[Attribute], child: Exchan
5859
override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = {
5960
child.executeBroadcast()
6061
}
62+
63+
// `ReusedExchangeExec` can have distinct set of output attribute ids from its child, we need
64+
// to update the attribute ids in `outputPartitioning` and `outputOrdering`.
65+
private lazy val updateAttr: Expression => Expression = {
66+
val originalAttrToNewAttr = AttributeMap(child.output.zip(output))
67+
e => e.transform {
68+
case attr: Attribute => originalAttrToNewAttr.getOrElse(attr, attr)
69+
}
70+
}
71+
72+
override def outputPartitioning: Partitioning = child.outputPartitioning match {
73+
case h: HashPartitioning => h.copy(expressions = h.expressions.map(updateAttr))
74+
case other => other
75+
}
76+
77+
override def outputOrdering: Seq[SortOrder] = {
78+
child.outputOrdering.map(updateAttr(_).asInstanceOf[SortOrder])
79+
}
6180
}
6281

6382
/**

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

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1855,4 +1855,14 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
18551855
.foldLeft(lit(false))((e, index) => e.or(df.col(df.columns(index)) =!= "string"))
18561856
df.filter(filter).count
18571857
}
1858+
1859+
test("SPARK-20897: cached self-join should not fail") {
1860+
// force to plan sort merge join
1861+
withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0") {
1862+
val df = Seq(1 -> "a").toDF("i", "j")
1863+
val df1 = df.as("t1")
1864+
val df2 = df.as("t2")
1865+
assert(df1.join(df2, $"t1.i" === $"t2.i").cache().count() == 1)
1866+
}
1867+
}
18581868
}

0 commit comments

Comments
 (0)