Skip to content

Commit

Permalink
[SPARK-27393][SQL] Show ReusedSubquery in the plan when the subquery …
Browse files Browse the repository at this point in the history
…is reused

With this change, we can easily identify the plan difference when subquery is reused.

When the reuse is enabled, the plan looks like
```
== Physical Plan ==
CollectLimit 1
+- *(1) Project [(Subquery subquery240 + ReusedSubquery Subquery subquery240) AS (scalarsubquery() + scalarsubquery())apache#253]
   :  :- Subquery subquery240
   :  :  +- *(2) HashAggregate(keys=[], functions=[avg(cast(key#13 as bigint))], output=[avg(key)apache#250])
   :  :     +- Exchange SinglePartition
   :  :        +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(key#13 as bigint))], output=[sum#256, count#257L])
   :  :           +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13]
   :  :              +- Scan[obj#12]
   :  +- ReusedSubquery Subquery subquery240
   +- *(1) SerializeFromObject
      +- Scan[obj#12]
```

When the reuse is disabled, the plan looks like
```
== Physical Plan ==
CollectLimit 1
+- *(1) Project [(Subquery subquery286 + Subquery subquery287) AS (scalarsubquery() + scalarsubquery())apache#299]
   :  :- Subquery subquery286
   :  :  +- *(2) HashAggregate(keys=[], functions=[avg(cast(key#13 as bigint))], output=[avg(key)apache#296])
   :  :     +- Exchange SinglePartition
   :  :        +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(key#13 as bigint))], output=[sum#302, count#303L])
   :  :           +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13]
   :  :              +- Scan[obj#12]
   :  +- Subquery subquery287
   :     +- *(2) HashAggregate(keys=[], functions=[avg(cast(key#13 as bigint))], output=[avg(key)apache#298])
   :        +- Exchange SinglePartition
   :           +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(key#13 as bigint))], output=[sum#306, count#307L])
   :              +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13]
   :                 +- Scan[obj#12]
   +- *(1) SerializeFromObject
      +- Scan[obj#12]
```

Modified the existing test.

Closes apache#24258 from gatorsmile/followupSPARK-27279.

Authored-by: gatorsmile <[email protected]>
Signed-off-by: gatorsmile <[email protected]>
  • Loading branch information
gatorsmile authored and j-baker committed Jan 25, 2020
1 parent ee96f62 commit f67d752
Show file tree
Hide file tree
Showing 8 changed files with 99 additions and 17 deletions.
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 @@ -658,19 +658,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
def child: SparkPlan

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

override def outputPartitioning: Partitioning = child.outputPartitioning

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

/**
* 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 @@ -693,6 +702,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 @@ -711,3 +724,23 @@ object SubqueryExec {
private[execution] val executionContext = ExecutionContext.fromExecutorService(
ThreadUtils.newDaemonCachedThreadPool("subquery", THREADS))
}

/**
* 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]
}
}

/**
Expand All @@ -44,15 +49,15 @@ abstract class ExecSubqueryExpression extends PlanExpression[SubqueryExec] {
* 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 @@ -117,13 +122,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]())
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
Original file line number Diff line number Diff line change
Expand Up @@ -261,7 +261,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 @@ -23,7 +23,8 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, SubqueryExpression, UnsafeRow}
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort}
import org.apache.spark.sql.execution.{SparkPlan, SubqueryExec}
import org.apache.spark.sql.execution.{ReusedSubqueryExec, ScalarSubquery, SparkPlan, SubqueryExec}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext

class SubquerySuite extends QueryTest with SharedSQLContext {
Expand Down Expand Up @@ -1339,4 +1340,37 @@ class SubquerySuite extends QueryTest with SharedSQLContext {
spark.sparkContext.setLocalProperty("a", "2")
SubqueryExec("test", LocalPropertiesOperator("a", "2")).executeCollect()
}

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")
}
}
}
}
}

0 comments on commit f67d752

Please sign in to comment.