Skip to content
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

[SPARK-35855][SQL] Unify reuse map data structures in non-AQE and AQE rules #33021

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

This file was deleted.

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,12 @@

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

import scala.collection.mutable

import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.TreePattern._
import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec}
import org.apache.spark.sql.util.ReuseMap

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

def apply(plan: SparkPlan): SparkPlan = {
if (conf.exchangeReuseEnabled || conf.subqueryReuseEnabled) {
val exchanges = new ReuseMap[Exchange, SparkPlan]()
val subqueries = new ReuseMap[BaseSubqueryExec, SparkPlan]()
val exchanges = mutable.Map.empty[SparkPlan, Exchange]
val subqueries = mutable.Map.empty[SparkPlan, BaseSubqueryExec]

def reuse(plan: SparkPlan): SparkPlan = {
plan.transformUpWithPruning(_.containsAnyPattern(EXCHANGE, PLAN_EXPRESSION)) {
case exchange: Exchange if conf.exchangeReuseEnabled =>
exchanges.reuseOrElseAdd(exchange, ReusedExchangeExec(exchange.output, _))
val cachedExchange = exchanges.getOrElseUpdate(exchange.canonicalized, exchange)
if (cachedExchange.ne(exchange)) {
ReusedExchangeExec(exchange.output, cachedExchange)
} else {
cachedExchange
}

case other =>
other.transformExpressionsUpWithPruning(_.containsPattern(PLAN_EXPRESSION)) {
case sub: ExecSubqueryExpression =>
val subquery = reuse(sub.plan).asInstanceOf[BaseSubqueryExec]
sub.withNewPlan(
if (conf.subqueryReuseEnabled) {
subqueries.reuseOrElseAdd(subquery, ReusedSubqueryExec(_))
val newSubquery = if (conf.subqueryReuseEnabled) {
val cachedSubquery = subqueries.getOrElseUpdate(subquery.canonicalized, subquery)
if (cachedSubquery.ne(subquery)) {
ReusedSubqueryExec(cachedSubquery)
} else {
subquery
})
cachedSubquery
}
} else {
subquery
}
sub.withNewPlan(newSubquery)
}
}
}
Expand Down