Skip to content

Commit 79e3d0d

Browse files
peter-tothcloud-fan
authored andcommitted
[SPARK-35855][SQL] Unify reuse map data structures in non-AQE and AQE rules
### What changes were proposed in this pull request? This PR unifies reuse map data structures in non-AQE and AQE rules to a simple `Map[<canonicalized plan>, <plan>]` based on the discussion here: #28885 (comment) ### Why are the changes needed? The proposed `Map[<canonicalized plan>, <plan>]` is simpler than the currently used `Map[<schema>, ArrayBuffer[<plan>]]` in `ReuseMap`/`ReuseExchangeAndSubquery` (non-AQE) and consistent with the `ReuseAdaptiveSubquery` (AQE) subquery reuse rule. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing UTs. Closes #33021 from peter-toth/SPARK-35855-unify-reuse-map-data-structures. Authored-by: Peter Toth <peter.toth@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent 20edfdd commit 79e3d0d

File tree

4 files changed

+22
-157
lines changed

4 files changed

+22
-157
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/util/ReuseMap.scala

Lines changed: 0 additions & 73 deletions
This file was deleted.

sql/catalyst/src/test/scala/org/apache/spark/sql/util/ReuseMapSuite.scala

Lines changed: 0 additions & 73 deletions
This file was deleted.

sql/core/src/main/scala/org/apache/spark/sql/execution/reuse/ReuseExchangeAndSubquery.scala

Lines changed: 20 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -17,11 +17,12 @@
1717

1818
package org.apache.spark.sql.execution.reuse
1919

20+
import scala.collection.mutable
21+
2022
import org.apache.spark.sql.catalyst.rules.Rule
2123
import org.apache.spark.sql.catalyst.trees.TreePattern._
2224
import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
2325
import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
24-
import org.apache.spark.sql.util.ReuseMap
2526

2627
/**
2728
* Find out duplicated exchanges and subqueries in the whole spark plan including subqueries, then
@@ -36,24 +37,34 @@ case object ReuseExchangeAndSubquery extends Rule[SparkPlan] {
3637

3738
def apply(plan: SparkPlan): SparkPlan = {
3839
if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {
39-
val exchanges = new ReuseMap[Exchange, SparkPlan]()
40-
val subqueries = new ReuseMap[BaseSubqueryExec, SparkPlan]()
40+
val exchanges = mutable.Map.empty[SparkPlan, Exchange]
41+
val subqueries = mutable.Map.empty[SparkPlan, BaseSubqueryExec]
4142

4243
def reuse(plan: SparkPlan): SparkPlan = {
4344
plan.transformUpWithPruning(_.containsAnyPattern(EXCHANGE, PLAN_EXPRESSION)) {
4445
case exchange: Exchange if conf.exchangeReuseEnabled =>
45-
exchanges.reuseOrElseAdd(exchange, ReusedExchangeExec(exchange.output, _))
46+
val cachedExchange = exchanges.getOrElseUpdate(exchange.canonicalized, exchange)
47+
if (cachedExchange.ne(exchange)) {
48+
ReusedExchangeExec(exchange.output, cachedExchange)
49+
} else {
50+
cachedExchange
51+
}
4652

4753
case other =>
4854
other.transformExpressionsUpWithPruning(_.containsPattern(PLAN_EXPRESSION)) {
4955
case sub: ExecSubqueryExpression =>
5056
val subquery = reuse(sub.plan).asInstanceOf[BaseSubqueryExec]
51-
sub.withNewPlan(
52-
if (conf.subqueryReuseEnabled) {
53-
subqueries.reuseOrElseAdd(subquery, ReusedSubqueryExec(_))
57+
val newSubquery = if (conf.subqueryReuseEnabled) {
58+
val cachedSubquery = subqueries.getOrElseUpdate(subquery.canonicalized, subquery)
59+
if (cachedSubquery.ne(subquery)) {
60+
ReusedSubqueryExec(cachedSubquery)
5461
} else {
55-
subquery
56-
})
62+
cachedSubquery
63+
}
64+
} else {
65+
subquery
66+
}
67+
sub.withNewPlan(newSubquery)
5768
}
5869
}
5970
}

sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -479,9 +479,9 @@ abstract class BroadcastJoinSuiteBase extends QueryTest with SQLTestUtils
479479
test("broadcast join where streamed side's output partitioning is PartitioningCollection") {
480480
withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") {
481481
val t1 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i1", "j1")
482-
val t2 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i2", "j2")
482+
val t2 = (0 until 100).map(i => (i % 5, i % 14)).toDF("i2", "j2")
483483
val t3 = (0 until 20).map(i => (i % 7, i % 11)).toDF("i3", "j3")
484-
val t4 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i4", "j4")
484+
val t4 = (0 until 100).map(i => (i % 5, i % 15)).toDF("i4", "j4")
485485

486486
// join1 is a sort merge join (shuffle on the both sides).
487487
val join1 = t1.join(t2, t1("i1") === t2("i2"))

0 commit comments

Comments
 (0)