Skip to content

[SPARK-20941] [SQL] Fix SubqueryExec Reuse #18169

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -552,6 +552,12 @@ object SQLConf {
.booleanConf
.createWithDefault(true)

val SUBQUERY_REUSE_ENABLED = buildConf("spark.sql.subquery.reuse")
.internal()
.doc("When true, the planner will try to find out duplicated subqueries and re-use them.")
.booleanConf
.createWithDefault(true)

val STATE_STORE_PROVIDER_CLASS =
buildConf("spark.sql.streaming.stateStore.providerClass")
.internal()
Expand Down Expand Up @@ -932,6 +938,8 @@ class SQLConf extends Serializable with Logging {

def exchangeReuseEnabled: Boolean = getConf(EXCHANGE_REUSE_ENABLED)

def subqueryReuseEnabled: Boolean = getConf(SUBQUERY_REUSE_ENABLED)

def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE)

def constraintPropagationEnabled: Boolean = getConf(CONSTRAINT_PROPAGATION_ENABLED)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -595,6 +595,9 @@ case class OutputFakerExec(output: Seq[Attribute], child: SparkPlan) extends Spa
*/
case class SubqueryExec(name: String, child: SparkPlan) extends UnaryExecNode {

// Ignore this wrapper for canonicalizing.
override lazy val canonicalized: SparkPlan = child.canonicalized

override lazy val metrics = Map(
"dataSize" -> SQLMetrics.createMetric(sparkContext, "data size (bytes)"),
"collectTime" -> SQLMetrics.createMetric(sparkContext, "time to collect (ms)"))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -156,7 +156,7 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] {
case class ReuseSubquery(conf: SQLConf) extends Rule[SparkPlan] {

def apply(plan: SparkPlan): SparkPlan = {
if (!conf.exchangeReuseEnabled) {
if (!conf.subqueryReuseEnabled) {
return plan
}
// Build a hash map using schema of subqueries to avoid O(N*N) sameResult calls.
Expand Down
35 changes: 35 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -23,9 +23,12 @@ import java.net.{MalformedURLException, URL}
import java.sql.Timestamp
import java.util.concurrent.atomic.AtomicBoolean

import scala.collection.mutable.ArrayBuffer

import org.apache.spark.{AccumulatorSuite, SparkException}
import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart}
import org.apache.spark.sql.catalyst.util.StringUtils
import org.apache.spark.sql.execution.{ScalarSubquery, SubqueryExec}
import org.apache.spark.sql.execution.aggregate
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec}
import org.apache.spark.sql.functions._
Expand Down Expand Up @@ -700,6 +703,38 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
row => Seq.fill(16)(Row.merge(row, row))).collect().toSeq)
}

test("Verify spark.sql.subquery.reuse") {
Seq(true, false).foreach { reuse =>
withSQLConf(SQLConf.SUBQUERY_REUSE_ENABLED.key -> reuse.toString) {
val df = sql(
"""
|SELECT key, (SELECT avg(key) FROM testData)
|FROM testData
|WHERE key > (SELECT avg(key) FROM testData)
|ORDER BY key
|LIMIT 3
""".stripMargin)

checkAnswer(df, Row(51, 50.5) :: Row(52, 50.5) :: Row(53, 50.5) :: Nil)

val subqueries = ArrayBuffer.empty[SubqueryExec]
df.queryExecution.executedPlan.transformAllExpressions {
case s @ ScalarSubquery(plan: SubqueryExec, _) =>
subqueries += plan
s
}

assert(subqueries.size == 2, "Two ScalarSubquery are expected in the plan")

if (reuse) {
assert(subqueries.distinct.size == 1, "Only one ScalarSubquery exists in the plan")
} else {
assert(subqueries.distinct.size == 2, "Reuse is not expected")
}
}
}
}

test("cartesian product join") {
withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") {
checkAnswer(
Expand Down