Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Next Next commit
[SPARK-29375][SQL] Exchange reuse across all subquery levels
  • Loading branch information
peter-toth committed Dec 15, 2019
commit da4381add89cd855c9176b9f55ebaaa3d91c337d
Original file line number Diff line number Diff line change
Expand Up @@ -239,7 +239,9 @@ object QueryExecution {
* are correct, insert whole stage code gen, and try to reduce the work done by reusing exchanges
* and subqueries.
*/
private[execution] def preparations(sparkSession: SparkSession): Seq[Rule[SparkPlan]] =
private[execution] def preparations(
sparkSession: SparkSession,
subQuery: Boolean = false): Seq[Rule[SparkPlan]] =
Seq(
// `AdaptiveSparkPlanExec` is a leaf node. If inserted, all the following rules will be no-op
// as the original plan is hidden behind `AdaptiveSparkPlanExec`.
Expand All @@ -249,10 +251,9 @@ object QueryExecution {
EnsureRequirements(sparkSession.sessionState.conf),
ApplyColumnarRulesAndInsertTransitions(sparkSession.sessionState.conf,
sparkSession.sessionState.columnarRules),
CollapseCodegenStages(sparkSession.sessionState.conf),
ReuseExchange(sparkSession.sessionState.conf),
CollapseCodegenStages(sparkSession.sessionState.conf)) ++
(if (subQuery) Nil else Seq(ReuseExchange(sparkSession.sessionState.conf))) :+
ReuseSubquery(sparkSession.sessionState.conf)
)

/**
* Prepares a planned [[SparkPlan]] for execution by inserting shuffle operations and internal
Expand Down Expand Up @@ -283,7 +284,7 @@ object QueryExecution {
* Prepare the [[SparkPlan]] for execution.
*/
def prepareExecutedPlan(spark: SparkSession, plan: SparkPlan): SparkPlan = {
prepareForExecution(preparations(spark), plan)
prepareForExecution(preparations(spark, true), plan)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.spark.sql.execution.exchange

import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer

import org.apache.spark.broadcast
import org.apache.spark.rdd.RDD
Expand Down Expand Up @@ -107,35 +106,39 @@ case class ReuseExchange(conf: SQLConf) extends Rule[SparkPlan] {
if (!conf.exchangeReuseEnabled) {
return plan
}
// Build a hash map using schema of exchanges to avoid O(N*N) sameResult calls.
val exchanges = mutable.HashMap[StructType, ArrayBuffer[Exchange]]()

// Replace a Exchange duplicate with a ReusedExchange
def reuse: PartialFunction[Exchange, SparkPlan] = {
case exchange: Exchange =>
val sameSchema = exchanges.getOrElseUpdate(exchange.schema, ArrayBuffer[Exchange]())
val samePlan = sameSchema.find { e =>
exchange.sameResult(e)
}
if (samePlan.isDefined) {
// Keep the output of this exchange, the following plans require that to resolve
// attributes.
ReusedExchangeExec(exchange.output, samePlan.get)
} else {
sameSchema += exchange
exchange
// To avoid costly canonicalization of an exchange:
// - we use its schema first to check if it can be replaced to a reused exchange at all
// - we insert an exchange into the map of canonicalized plans only when at least 2 exchange
// have the same schema
val exchanges = mutable.Map[StructType, (Exchange, mutable.Map[SparkPlan, Exchange])]()

def reuse(plan: SparkPlan): SparkPlan = {
plan.transformUp {
case exchange: Exchange =>
val (firstSameSchemaExchange, sameResultExchanges) =
exchanges.getOrElseUpdate(exchange.schema, (exchange, mutable.Map()))
if (firstSameSchemaExchange.ne(exchange)) {
if (sameResultExchanges.isEmpty) {
sameResultExchanges +=
firstSameSchemaExchange.canonicalized -> firstSameSchemaExchange
}
val sameResultExchange =
sameResultExchanges.getOrElseUpdate(exchange.canonicalized, exchange)
if (sameResultExchange.ne(exchange)) {
ReusedExchangeExec(exchange.output, sameResultExchange)
} else {
exchange
}
} else {
exchange
}
case other => other.transformExpressions {
case sub: ExecSubqueryExpression =>
sub.withNewPlan(reuse(sub.plan).asInstanceOf[BaseSubqueryExec])
}
}
}

plan transformUp {
case exchange: Exchange => reuse(exchange)
} transformAllExpressions {
// Lookup inside subqueries for duplicate exchanges
case in: InSubqueryExec =>
val newIn = in.plan.transformUp {
case exchange: Exchange => reuse(exchange)
}
in.copy(plan = newIn.asInstanceOf[BaseSubqueryExec])
}
reuse(plan)
}
}
57 changes: 57 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ 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.{ColumnarToRowExec, ExecSubqueryExpression, FileSourceScanExec, InputAdapter, ReusedSubqueryExec, ScalarSubquery, SubqueryExec, WholeStageCodegenExec}
import org.apache.spark.sql.execution.datasources.FileScanRDD
import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSparkSession

Expand Down Expand Up @@ -1389,6 +1390,62 @@ class SubquerySuite extends QueryTest with SharedSparkSession {
}
}

test("Exchange reuse across all subquery levels") {
Seq(true, false).foreach { reuse =>
withSQLConf(SQLConf.EXCHANGE_REUSE_ENABLED.key -> reuse.toString) {
val df = sql(
"""
|SELECT
| (SELECT max(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key),
| a.key
|FROM testData AS a
|JOIN testData AS b ON b.key = a.key
""".stripMargin)

val plan = df.queryExecution.executedPlan

val exchangeIds = plan.collectInPlanAndSubqueries { case e: Exchange => e.id }
val reusedExchangeIds = plan.collectInPlanAndSubqueries {
case re: ReusedExchangeExec => re.child.id
}

if (reuse) {
assert(exchangeIds.size == 2, "Exchange reusing not working correctly")
assert(reusedExchangeIds.size == 3, "Exchange reusing not working correctly")
assert(reusedExchangeIds.forall(exchangeIds.contains(_)),
"ReusedExchangeExec should reuse an existing exchange")
} else {
assert(exchangeIds.size == 5, "expect 5 Exchange when not reusing")
assert(reusedExchangeIds.size == 0, "expect 0 ReusedExchangeExec when not reusing")
}

val df2 = sql(
"""
SELECT
(SELECT min(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key),
(SELECT max(a.key) FROM testData AS a JOIN testData2 AS b ON b.a = a.key)
""".stripMargin)

val plan2 = df2.queryExecution.executedPlan

val exchangeIds2 = plan2.collectInPlanAndSubqueries { case e: Exchange => e.id }
val reusedExchangeIds2 = plan2.collectInPlanAndSubqueries {
case re: ReusedExchangeExec => re.child.id
}

if (reuse) {
assert(exchangeIds2.size == 4, "Exchange reusing not working correctly")
assert(reusedExchangeIds2.size == 2, "Exchange reusing not working correctly")
assert(reusedExchangeIds2.forall(exchangeIds2.contains(_)),
"ReusedExchangeExec should reuse an existing exchange")
} else {
assert(exchangeIds2.size == 6, "expect 6 Exchange when not reusing")
assert(reusedExchangeIds2.size == 0, "expect 0 ReusedExchangeExec when not reusing")
}
}
}
}

test("Scalar subquery name should start with scalar-subquery#") {
val df = sql("SELECT a FROM l WHERE a = (SELECT max(c) FROM r WHERE c = 1)".stripMargin)
var subqueryExecs: ArrayBuffer[SubqueryExec] = ArrayBuffer.empty
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -470,7 +470,7 @@ class PlannerSuite extends SharedSparkSession {
Inner,
None,
shuffle,
shuffle)
shuffle.copy())
Copy link
Contributor Author

Choose a reason for hiding this comment

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

If a plan contains the exact same instance of Exchange multiple times then it makes no sense to replace one of the instances to a ReusedExchange.


val outputPlan = ReuseExchange(spark.sessionState.conf).apply(inputPlan)
if (outputPlan.collect { case e: ReusedExchangeExec => true }.size != 1) {
Expand Down