Skip to content

[SPARK-27393][SQL] Show ReusedSubquery in the plan when the subquery is reused #24258

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 @@ -37,6 +37,9 @@ abstract class PlanExpression[T <: QueryPlan[_]] extends Expression {
/** Updates the expression with a new plan. */
def withNewPlan(plan: T): PlanExpression[T]

/** Defines how the canonicalization should work for this expression. */
def canonicalize(attrs: AttributeSeq): PlanExpression[T]

protected def conditionString: String = children.mkString("[", " && ", "]")
}

Expand All @@ -58,7 +61,7 @@ abstract class SubqueryExpression(
children.zip(p.children).forall(p => p._1.semanticEquals(p._2))
case _ => false
}
def canonicalize(attrs: AttributeSeq): SubqueryExpression = {
override def canonicalize(attrs: AttributeSeq): SubqueryExpression = {
// Normalize the outer references in the subquery plan.
val normalizedPlan = plan.transformAllExpressions {
case OuterReference(r) => OuterReference(QueryPlan.normalizeExprId(r, attrs))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -279,7 +279,7 @@ object QueryPlan extends PredicateHelper {
*/
def normalizeExprId[T <: Expression](e: T, input: AttributeSeq): T = {
e.transformUp {
case s: SubqueryExpression => s.canonicalize(input)
case s: PlanExpression[_] => s.canonicalize(input)
case ar: AttributeReference =>
val ordinal = input.indexOf(ar.exprId)
if (ordinal == -1) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ private[execution] object SparkPlanInfo {
def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = {
val children = plan match {
case ReusedExchangeExec(_, child) => child :: Nil
case ReusedSubqueryExec(child) => child :: Nil
case _ => plan.children ++ plan.subqueries
}
val metrics = plan.metrics.toSeq.map { case (key, metric) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -660,21 +660,28 @@ object CoalesceExec {
}

/**
* Physical plan for a subquery.
* Parent class for different types of subquery plans
*/
case class SubqueryExec(name: String, child: SparkPlan) extends UnaryExecNode {

override lazy val metrics = Map(
"dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"),
"collectTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to collect"))
abstract class BaseSubqueryExec extends SparkPlan {
def name: String
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need name in this base class for subquery exec?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

both ReusedSubqueryExec and SubqueryExec have the name

def child: SparkPlan

override def output: Seq[Attribute] = child.output

override def outputPartitioning: Partitioning = child.outputPartitioning

override def outputOrdering: Seq[SortOrder] = child.outputOrdering
}

override def doCanonicalize(): SparkPlan = child.canonicalized
/**
* Physical plan for a subquery.
*/
case class SubqueryExec(name: String, child: SparkPlan)
extends BaseSubqueryExec with UnaryExecNode {

override lazy val metrics = Map(
"dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"),
"collectTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to collect"))

@transient
private lazy val relationFuture: Future[Array[InternalRow]] = {
Expand All @@ -698,6 +705,10 @@ case class SubqueryExec(name: String, child: SparkPlan) extends UnaryExecNode {
}(SubqueryExec.executionContext)
}

protected override def doCanonicalize(): SparkPlan = {
SubqueryExec("Subquery", child.canonicalized)
}

protected override def doPrepare(): Unit = {
relationFuture
}
Expand All @@ -715,3 +726,23 @@ object SubqueryExec {
private[execution] val executionContext = ExecutionContext.fromExecutorService(
ThreadUtils.newDaemonCachedThreadPool("subquery", 16))
}

/**
* A wrapper for reused [[BaseSubqueryExec]].
*/
case class ReusedSubqueryExec(child: BaseSubqueryExec)
extends BaseSubqueryExec with LeafExecNode {

override def name: String = child.name

override def output: Seq[Attribute] = child.output
override def doCanonicalize(): SparkPlan = child.canonicalized
override def outputOrdering: Seq[SortOrder] = child.outputOrdering
override def outputPartitioning: Partitioning = child.outputPartitioning

protected override def doPrepare(): Unit = child.prepare()

protected override def doExecute(): RDD[InternalRow] = child.execute()

override def executeCollect(): Array[InternalRow] = child.executeCollect()
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.{expressions, InternalRow}
import org.apache.spark.sql.catalyst.expressions.{Expression, ExprId, InSet, Literal, PlanExpression}
import org.apache.spark.sql.catalyst.expressions.{AttributeSeq, Expression, ExprId, InSet, Literal, PlanExpression}
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.internal.SQLConf
Expand All @@ -31,11 +31,16 @@ import org.apache.spark.sql.types.{BooleanType, DataType, StructType}
/**
* The base class for subquery that is used in SparkPlan.
*/
abstract class ExecSubqueryExpression extends PlanExpression[SubqueryExec] {
abstract class ExecSubqueryExpression extends PlanExpression[BaseSubqueryExec] {
/**
* Fill the expression with collected result from executed plan.
*/
def updateResult(): Unit

override def canonicalize(attrs: AttributeSeq): ExecSubqueryExpression = {
withNewPlan(plan.canonicalized.asInstanceOf[BaseSubqueryExec])
.asInstanceOf[ExecSubqueryExpression]
}
}

object ExecSubqueryExpression {
Expand All @@ -56,15 +61,15 @@ object ExecSubqueryExpression {
* This is the physical copy of ScalarSubquery to be used inside SparkPlan.
*/
case class ScalarSubquery(
plan: SubqueryExec,
plan: BaseSubqueryExec,
exprId: ExprId)
extends ExecSubqueryExpression {

override def dataType: DataType = plan.schema.fields.head.dataType
override def children: Seq[Expression] = Nil
override def nullable: Boolean = true
override def toString: String = plan.simpleString(SQLConf.get.maxToStringFields)
override def withNewPlan(query: SubqueryExec): ScalarSubquery = copy(plan = query)
override def withNewPlan(query: BaseSubqueryExec): ScalarSubquery = copy(plan = query)

override def semanticEquals(other: Expression): Boolean = other match {
case s: ScalarSubquery => plan.sameResult(s.plan)
Expand Down Expand Up @@ -129,13 +134,14 @@ case class ReuseSubquery(conf: SQLConf) extends Rule[SparkPlan] {
return plan
}
// Build a hash map using schema of subqueries to avoid O(N*N) sameResult calls.
val subqueries = mutable.HashMap[StructType, ArrayBuffer[SubqueryExec]]()
val subqueries = mutable.HashMap[StructType, ArrayBuffer[BaseSubqueryExec]]()
plan transformAllExpressions {
case sub: ExecSubqueryExpression =>
val sameSchema = subqueries.getOrElseUpdate(sub.plan.schema, ArrayBuffer[SubqueryExec]())
val sameSchema =
subqueries.getOrElseUpdate(sub.plan.schema, ArrayBuffer[BaseSubqueryExec]())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

unnecessary change?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

change it to BaseSubqueryExec

val sameResult = sameSchema.find(_.sameResult(sub.plan))
if (sameResult.isDefined) {
sub.withNewPlan(sameResult.get)
sub.withNewPlan(ReusedSubqueryExec(sameResult.get))
} else {
sameSchema += sub.plan
sub
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,11 @@ object SparkPlanGraph {
// Point to the re-used subquery
val node = exchanges(planInfo)
edges += SparkPlanGraphEdge(node.id, parent.id)
case "ReusedSubquery" =>
// Re-used subquery might appear before the original subquery, so skip this node and let
// the previous `case` make sure the re-used and the original point to the same node.
buildSparkPlanGraphNode(
planInfo.children.head, nodeIdGenerator, nodes, edges, parent, subgraph, exchanges)
case "ReusedExchange" if exchanges.contains(planInfo.children.head) =>
// Point to the re-used exchange
val node = exchanges(planInfo.children.head)
Expand Down
30 changes: 1 addition & 29 deletions sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import java.util.concurrent.atomic.AtomicBoolean
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.{aggregate, ScalarSubquery, SubqueryExec}
import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec}
import org.apache.spark.sql.execution.datasources.FilePartition
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec}
Expand Down Expand Up @@ -113,33 +112,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
}
}

test("Reuse Subquery") {
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

moved to SubquerySuite.scala

Seq(true, false).foreach { reuse =>
withSQLConf(SQLConf.SUBQUERY_REUSE_ENABLED.key -> reuse.toString) {
val df = sql(
"""
|SELECT (SELECT avg(key) FROM testData) + (SELECT avg(key) FROM testData)
|FROM testData
|LIMIT 1
""".stripMargin)

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

if (reuse) {
assert(subqueries.distinct.size == 1, "Subquery reusing not working correctly")
} else {
assert(subqueries.distinct.size == 2, "There should be 2 subqueries when not reusing")
}
}
}
}

test("SPARK-6743: no columns from cache") {
Seq(
(83, 0, 38),
Expand Down Expand Up @@ -288,7 +260,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
val df = sql(sqlText)
// First, check if we have GeneratedAggregate.
val hasGeneratedAgg = df.queryExecution.sparkPlan
.collect { case _: aggregate.HashAggregateExec => true }
.collect { case _: HashAggregateExec => true }
.nonEmpty
if (!hasGeneratedAgg) {
fail(
Expand Down
36 changes: 35 additions & 1 deletion sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,9 @@ import scala.collection.mutable.ArrayBuffer

import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort}
import org.apache.spark.sql.execution.{ExecSubqueryExpression, FileSourceScanExec, WholeStageCodegenExec}
import org.apache.spark.sql.execution.{ExecSubqueryExpression, FileSourceScanExec, ReusedSubqueryExec, ScalarSubquery, SubqueryExec, WholeStageCodegenExec}
import org.apache.spark.sql.execution.datasources.FileScanRDD
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext

class SubquerySuite extends QueryTest with SharedSQLContext {
Expand Down Expand Up @@ -1337,4 +1338,37 @@ class SubquerySuite extends QueryTest with SharedSQLContext {
checkAnswer(df3, Seq(Row("a", 2, "a"), Row("a", 2, "b")))
}
}

test("SPARK-27279: Reuse Subquery") {
Seq(true, false).foreach { reuse =>
withSQLConf(SQLConf.SUBQUERY_REUSE_ENABLED.key -> reuse.toString) {
val df = sql(
"""
|SELECT (SELECT avg(key) FROM testData) + (SELECT avg(key) FROM testData)
|FROM testData
|LIMIT 1
""".stripMargin)

var countSubqueryExec = 0
var countReuseSubqueryExec = 0
df.queryExecution.executedPlan.transformAllExpressions {
case s @ ScalarSubquery(_: SubqueryExec, _) =>
countSubqueryExec = countSubqueryExec + 1
s
case s @ ScalarSubquery(_: ReusedSubqueryExec, _) =>
countReuseSubqueryExec = countReuseSubqueryExec + 1
s
}

if (reuse) {
assert(countSubqueryExec == 1, "Subquery reusing not working correctly")
assert(countReuseSubqueryExec == 1, "Subquery reusing not working correctly")
} else {
assert(countSubqueryExec == 2, "expect 2 SubqueryExec when not reusing")
assert(countReuseSubqueryExec == 0,
"expect 0 ReusedSubqueryExec when not reusing")
}
}
}
}
}