Skip to content

Commit

Permalink
Possibly fix up reuse of GpuShuffleExchange for AQE (NVIDIA#16)
Browse files Browse the repository at this point in the history
* Fix up reuse of GpuShuffleExchange for AQE

---------

Signed-off-by: Firestarman <[email protected]>
  • Loading branch information
firestarman authored Dec 18, 2023
1 parent e99c1b9 commit 5cd0dd6
Show file tree
Hide file tree
Showing 14 changed files with 134 additions and 69 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -550,6 +550,16 @@ object GpuOverrides extends Logging {
}.getOrElse(bqse)
case _ => bqse
}
case sqse: ShuffleQueryStageExec =>
sqse.plan match {
case ReusedExchangeExec(output, b: ShuffleExchangeExec) =>
val cpuCanonical = b.canonicalized.asInstanceOf[ShuffleExchangeExec]
val gpuExchange = ExchangeMappingCache.findGpuExchangeReplacement(cpuCanonical)
gpuExchange.map { g =>
SparkShimImpl.newShuffleQueryStageExec(sqse, ReusedExchangeExec(output, g))
}.getOrElse(sqse)
case _ => sqse
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanExecBase,
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, Exchange, ReusedExchangeExec, ShuffleExchangeLike}
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, HashedRelationBroadcastMode}
import org.apache.spark.sql.rapids.{GpuDataSourceScanExec, GpuFileSourceScanExec, GpuShuffleEnv, GpuTaskMetrics}
import org.apache.spark.sql.rapids.execution.{ExchangeMappingCache, GpuBroadcastExchangeExec, GpuBroadcastExchangeExecBase, GpuBroadcastToRowExec, GpuCustomShuffleReaderExec, GpuHashJoin, GpuShuffleExchangeExecBase}
import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuBroadcastExchangeExecBase, GpuBroadcastToRowExec, GpuCustomShuffleReaderExec, GpuHashJoin, GpuShuffleExchangeExecBase}
import org.apache.spark.sql.types.StructType

/**
Expand Down Expand Up @@ -668,22 +668,24 @@ class GpuTransitionOverrides extends Rule[SparkPlan] {
/** Mark nodes as GPU planning completed. */
private def markGpuPlanningComplete(plan: SparkPlan): SparkPlan = {
plan.foreach {
case g: GpuBroadcastExchangeExec => g.markGpuPlanningComplete()
// Only GpuBroadcastExchangeExec and GpuShuffleExchangedExec support SupportPlanningMark
case g: SupportPlanningMark => g.markGpuPlanningComplete()
case _ =>
}
plan
}

/**
* On some Spark platforms, AQE planning ends up not reusing as many GPU exchanges as possible.
* This searches the plan for any GPU broadcast exchanges and checks if their original CPU plans
* match any other previously seen GPU broadcasts with the same CPU plan.
* This searches the plan for any GPU broadcast/shuffle exchanges and checks if their original
* CPU plans match any other previously seen GPU broadcasts/shuffles with the same CPU plan.
*/
private def fixupAdaptiveExchangeReuse(p: SparkPlan): SparkPlan = {
def doFixup(plan: SparkPlan): SparkPlan = {
plan.transformUp {
case g: GpuBroadcastExchangeExec =>
ExchangeMappingCache.findGpuExchangeReplacement(g.cpuCanonical).map { other =>
// Only GpuBroadcastExchangeExec and GpuShuffleExchangedExec support SupportPlanningMark
case g: SupportPlanningMark if g.cpuCanonicalExec.isDefined =>
ExchangeMappingCache.findGpuExchangeReplacement(g.cpuCanonicalExec.get).map { other =>
if (other eq g) {
g
} else {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* Copyright (c) 2023, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package com.nvidia.spark.rapids

import scala.collection.mutable
import scala.ref.WeakReference

import org.apache.spark.internal.Logging
import org.apache.spark.sql.execution.exchange.Exchange

/**A trait mixed with a GPU version of Exchange to support to mark if it is optimized by GPU*/
trait SupportPlanningMark extends Logging { this: Exchange =>

// Some GPU Shuffle exchanges are used internally without a CPU instance, it is an Option.
val cpuCanonicalExec: Option[Exchange]

private var _isGpuPlanningComplete = false

/**
* Returns true if this node and children are finished being optimized by the RAPIDS Accelerator.
*/
final def isGpuPlanningComplete: Boolean = _isGpuPlanningComplete

/**
* Method to call after all RAPIDS Accelerator optimizations have been applied
* to indicate this node and its children are done being planned by the RAPIDS Accelerator.
* Some optimizations, such as AQE exchange reuse fixup, need to know when a node will no longer
* be updated so it can be tracked for reuse.
*/
final def markGpuPlanningComplete(): Unit = {
if (!_isGpuPlanningComplete) {
_isGpuPlanningComplete = true
if (cpuCanonicalExec.isDefined) {
ExchangeMappingCache.trackExchangeMapping(cpuCanonicalExec.get, this)
} else {
logInfo("Trying to mark an exchange as GPU planning done without a CPU instance")
}
}
}
}

/** Caches the mappings from canonical CPU exchanges to the GPU exchanges that replaced them */
object ExchangeMappingCache extends Logging {
// Cache is a mapping from CPU broadcast plan to GPU broadcast plan. The cache should not
// artificially hold onto unused plans, so we make both the keys and values weak. The values
// point to their corresponding keys, so the keys will not be collected unless the value
// can be collected. The values will be held during normal Catalyst planning until those
// plans are no longer referenced, allowing both the key and value to be reaped at that point.
private val cache = new mutable.WeakHashMap[Exchange, WeakReference[Exchange]]

/** Try to find a recent GPU exchange that has replaced the specified CPU canonical plan. */
def findGpuExchangeReplacement(cpuCanonical: Exchange): Option[Exchange] = {
cache.get(cpuCanonical).flatMap(_.get)
}

/** Add a GPU exchange to the exchange cache */
def trackExchangeMapping(cpuCanonical: Exchange, gpuExchange: Exchange): Unit = {
val old = findGpuExchangeReplacement(cpuCanonical)
if (!old.exists(_.asInstanceOf[SupportPlanningMark].isGpuPlanningComplete)) {
cache.put(cpuCanonical, WeakReference(gpuExchange))
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,7 @@ import java.io._
import java.util.UUID
import java.util.concurrent._

import scala.collection.mutable
import scala.concurrent.ExecutionContext
import scala.ref.WeakReference
import scala.util.control.NonFatal

import ai.rapids.cudf.{HostMemoryBuffer, JCudfSerialization, NvtxColor, NvtxRange}
Expand Down Expand Up @@ -617,53 +615,13 @@ case class GpuBroadcastExchangeExec(
mode: BroadcastMode,
child: SparkPlan)
(val cpuCanonical: BroadcastExchangeExec)
extends GpuBroadcastExchangeExecBase(mode, child) {
override def otherCopyArgs: Seq[AnyRef] = Seq(cpuCanonical)

private var _isGpuPlanningComplete = false
extends GpuBroadcastExchangeExecBase(mode, child) with SupportPlanningMark {

/**
* Returns true if this node and children are finished being optimized by the RAPIDS Accelerator.
*/
def isGpuPlanningComplete: Boolean = _isGpuPlanningComplete
override val cpuCanonicalExec: Option[Exchange] = Some(cpuCanonical)

/**
* Method to call after all RAPIDS Accelerator optimizations have been applied
* to indicate this node and its children are done being planned by the RAPIDS Accelerator.
* Some optimizations, such as AQE exchange reuse fixup, need to know when a node will no longer
* be updated so it can be tracked for reuse.
*/
def markGpuPlanningComplete(): Unit = {
if (!_isGpuPlanningComplete) {
_isGpuPlanningComplete = true
ExchangeMappingCache.trackExchangeMapping(cpuCanonical, this)
}
}
override def otherCopyArgs: Seq[AnyRef] = Seq(cpuCanonical)

override def doCanonicalize(): SparkPlan = {
GpuBroadcastExchangeExec(mode.canonicalized, child.canonicalized)(cpuCanonical)
}
}

/** Caches the mappings from canonical CPU exchanges to the GPU exchanges that replaced them */
object ExchangeMappingCache extends Logging {
// Cache is a mapping from CPU broadcast plan to GPU broadcast plan. The cache should not
// artificially hold onto unused plans, so we make both the keys and values weak. The values
// point to their corresponding keys, so the keys will not be collected unless the value
// can be collected. The values will be held during normal Catalyst planning until those
// plans are no longer referenced, allowing both the key and value to be reaped at that point.
private val cache = new mutable.WeakHashMap[Exchange, WeakReference[Exchange]]

/** Try to find a recent GPU exchange that has replaced the specified CPU canonical plan. */
def findGpuExchangeReplacement(cpuCanonical: Exchange): Option[Exchange] = {
cache.get(cpuCanonical).flatMap(_.get)
}

/** Add a GPU exchange to the exchange cache */
def trackExchangeMapping(cpuCanonical: Exchange, gpuExchange: Exchange): Unit = {
val old = findGpuExchangeReplacement(cpuCanonical)
if (!old.exists(_.asInstanceOf[GpuBroadcastExchangeExec].isGpuPlanningComplete)) {
cache.put(cpuCanonical, WeakReference(gpuExchange))
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,7 @@ abstract class GpuShuffleMetaBase(
childParts.head.convertToGpu(),
newChild,
shuffle.shuffleOrigin
)(shuffle.outputPartitioning)
)(shuffle.outputPartitioning, Some(shuffle.canonicalized.asInstanceOf[ShuffleExchangeExec]))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,10 @@ abstract class Spark31XShims extends Spark31Xuntil33XShims with Logging {
old: BroadcastQueryStageExec,
newPlan: SparkPlan): BroadcastQueryStageExec = BroadcastQueryStageExec(old.id, newPlan)

final def newShuffleQueryStageExec(
old: ShuffleQueryStageExec,
newPlan: SparkPlan): ShuffleQueryStageExec = ShuffleQueryStageExec(old.id, newPlan)

override def getDateFormatter(): DateFormatter = {
DateFormatter(DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone))
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,25 +36,26 @@
spark-rapids-shim-json-lines ***/
package org.apache.spark.rapids.shims

import com.nvidia.spark.rapids.GpuPartitioning
import com.nvidia.spark.rapids.{GpuPartitioning, SupportPlanningMark}

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.plans.logical.Statistics
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.execution.{ShufflePartitionSpec, SparkPlan}
import org.apache.spark.sql.execution.exchange.{ShuffleExchangeLike, ShuffleOrigin}
import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeLike, ShuffleOrigin}
import org.apache.spark.sql.rapids.execution.{GpuShuffleExchangeExecBaseWithMetrics, ShuffledBatchRDD}

case class GpuShuffleExchangeExec(
gpuOutputPartitioning: GpuPartitioning,
child: SparkPlan,
shuffleOrigin: ShuffleOrigin,
advisoryPartitionSize: Option[Long] = None)(
cpuOutputPartitioning: Partitioning)
cpuOutputPartitioning: Partitioning,
override val cpuCanonicalExec: Option[Exchange] = None)
extends GpuShuffleExchangeExecBaseWithMetrics(gpuOutputPartitioning, child)
with ShuffleExchangeLike {
with ShuffleExchangeLike with SupportPlanningMark {

override def otherCopyArgs: Seq[AnyRef] = cpuOutputPartitioning :: Nil
override def otherCopyArgs: Seq[AnyRef] = Seq(cpuOutputPartitioning, cpuCanonicalExec)

override val outputPartitioning: Partitioning = cpuOutputPartitioning

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,11 @@ trait Spark320PlusNonDBShims extends SparkShims {
newPlan: SparkPlan): BroadcastQueryStageExec =
BroadcastQueryStageExec(old.id, newPlan, old._canonicalized)

final def newShuffleQueryStageExec(
old: ShuffleQueryStageExec,
newPlan: SparkPlan): ShuffleQueryStageExec =
ShuffleQueryStageExec(old.id, newPlan, old._canonicalized)

override final def filesFromFileIndex(fileIndex: PartitioningAwareFileIndex): Seq[FileStatus] = {
fileIndex.allFiles()
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,11 @@ trait Spark321PlusDBShims extends SparkShims
newPlan: SparkPlan): BroadcastQueryStageExec =
BroadcastQueryStageExec(old.id, newPlan, old.originalPlan, old.isSparkExchange)

final def newShuffleQueryStageExec(
old: ShuffleQueryStageExec,
newPlan: SparkPlan): ShuffleQueryStageExec =
ShuffleQueryStageExec(old.id, newPlan, old.originalPlan, old.isSparkExchange)

override def filesFromFileIndex(fileCatalog: PartitioningAwareFileIndex): Seq[FileStatus] = {
fileCatalog.allFiles().map(_.toFileStatus)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,23 +21,24 @@
spark-rapids-shim-json-lines ***/
package org.apache.spark.rapids.shims

import com.nvidia.spark.rapids.GpuPartitioning
import com.nvidia.spark.rapids.{GpuPartitioning, SupportPlanningMark}

import org.apache.spark.sql.catalyst.plans.logical.Statistics
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.exchange.{ShuffleExchangeLike, ShuffleOrigin}
import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeLike, ShuffleOrigin}
import org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBaseWithMetrics

abstract class GpuDatabricksShuffleExchangeExecBase(
gpuOutputPartitioning: GpuPartitioning,
child: SparkPlan,
shuffleOrigin: ShuffleOrigin)(
cpuOutputPartitioning: Partitioning)
cpuOutputPartitioning: Partitioning,
override val cpuCanonicalExec: Option[Exchange])
extends GpuShuffleExchangeExecBaseWithMetrics(gpuOutputPartitioning, child)
with ShuffleExchangeLike {
with ShuffleExchangeLike with SupportPlanningMark {

override def otherCopyArgs: Seq[AnyRef] = cpuOutputPartitioning :: Nil
override def otherCopyArgs: Seq[AnyRef] = Seq(cpuOutputPartitioning, cpuCanonicalExec)

override val outputPartitioning: Partitioning = cpuOutputPartitioning

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,16 +25,17 @@ import com.nvidia.spark.rapids.GpuPartitioning
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.execution.{ShufflePartitionSpec, SparkPlan}
import org.apache.spark.sql.execution.exchange.ShuffleOrigin
import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleOrigin}
import org.apache.spark.sql.rapids.execution.ShuffledBatchRDD

case class GpuShuffleExchangeExec(
gpuOutputPartitioning: GpuPartitioning,
child: SparkPlan,
shuffleOrigin: ShuffleOrigin)(
cpuOutputPartitioning: Partitioning)
cpuOutputPartitioning: Partitioning,
override val cpuCanonicalExec: Option[Exchange] = None)
extends GpuDatabricksShuffleExchangeExecBase(gpuOutputPartitioning,
child, shuffleOrigin)(cpuOutputPartitioning) {
child, shuffleOrigin)(cpuOutputPartitioning, cpuCanonicalExec) {

override def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[_] = {
new ShuffledBatchRDD(shuffleDependencyColumnar, metrics ++ readMetrics, partitionSpecs)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,5 +65,5 @@ class GpuShuffleMeta(
childParts.head.convertToGpu(),
newChild,
shuffle.shuffleOrigin
)(shuffle.outputPartitioning)
)(shuffle.outputPartitioning, Some(shuffle.canonicalized.asInstanceOf[ShuffleExchangeExec]))
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,10 @@ case class GpuShuffleExchangeExec(
gpuOutputPartitioning: GpuPartitioning,
child: SparkPlan,
shuffleOrigin: ShuffleOrigin)(
cpuOutputPartitioning: Partitioning)
cpuOutputPartitioning: Partitioning,
override val cpuCanonicalExec: Option[Exchange] = None)
extends GpuDatabricksShuffleExchangeExecBase(gpuOutputPartitioning, child, shuffleOrigin)(
cpuOutputPartitioning) {
cpuOutputPartitioning, cpuCanonicalExec) {

override def getShuffleRDD(
partitionSpecs: Array[ShufflePartitionSpec],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,5 +38,5 @@ class GpuShuffleMeta(
newChild,
shuffle.shuffleOrigin,
shuffle.advisoryPartitionSize
)(shuffle.outputPartitioning)
)(shuffle.outputPartitioning, Some(shuffle.canonicalized.asInstanceOf[ShuffleExchangeExec]))
}

0 comments on commit 5cd0dd6

Please sign in to comment.