From b4610cd6519e0d059bb78326db236cde69e7156e Mon Sep 17 00:00:00 2001 From: liurenjie1024 Date: Tue, 25 Jun 2024 17:55:36 +0800 Subject: [PATCH] Fix style --- docs/additional-functionality/advanced_configs.md | 10 ++++++++++ .../scala/com/nvidia/spark/rapids/lore/GpuLore.scala | 11 +++++++---- .../scala/com/nvidia/spark/rapids/lore/replay.scala | 2 +- .../rapids/execution/GpuBroadcastExchangeExec.scala | 4 ++-- 4 files changed, 20 insertions(+), 7 deletions(-) diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md index 941ab4046e6..7ef72a381d0 100644 --- a/docs/additional-functionality/advanced_configs.md +++ b/docs/additional-functionality/advanced_configs.md @@ -135,6 +135,16 @@ Name | Description | Default Value | Applicable at spark.rapids.sql.json.read.decimal.enabled|When reading a quoted string as a decimal Spark supports reading non-ascii unicode digits, and the RAPIDS Accelerator does not.|true|Runtime spark.rapids.sql.json.read.double.enabled|JSON reading is not 100% compatible when reading doubles.|true|Runtime spark.rapids.sql.json.read.float.enabled|JSON reading is not 100% compatible when reading floats.|true|Runtime +spark.rapids.sql.lore.dumpPath|The path to dump the lore nodes' input data. This must be set if spark.rapids.sql.lore.idsToDump has +been set.|None|Runtime +spark.rapids.sql.lore.idsToDump|Specify the lore ids of operators to dump. The format is a comma separated list of +lore ids. For example: "1,2,3" will dump the gpu exec nodes with lore ids 1, 2, and 3. +By default, all partitions of operators' input will be dumped. If you want to dump only +some partitions, you can specify the partition index after the lore id, e.g. 1[0-2 4-5 +7], 2[0 4 5-8] , will dump partitions 0, 1, 2, 4, 5 and 7 of the operator with lore id + 1, and partitions 0, 4, 5, 6, 7, 8 of the operator with lore id 2. +If this is not set, no lore nodes will be dumped.|None|Runtime +spark.rapids.sql.lore.tag.enabled|Enable add a lore id to each gpu plan node|true|Runtime spark.rapids.sql.mode|Set the mode for the Rapids Accelerator. The supported modes are explainOnly and executeOnGPU. This config can not be changed at runtime, you must restart the application for it to take affect. The default mode is executeOnGPU, which means the RAPIDS Accelerator plugin convert the Spark operations and execute them on the GPU when possible. The explainOnly mode allows running queries on the CPU and the RAPIDS Accelerator will evaluate the queries as if it was going to run on the GPU. The explanations of what would have run on the GPU and why are output in log messages. When using explainOnly mode, the default explain output is ALL, this can be changed by setting spark.rapids.sql.explain. See that config for more details.|executeongpu|Startup spark.rapids.sql.optimizer.joinReorder.enabled|When enabled, joins may be reordered for improved query performance|true|Runtime spark.rapids.sql.python.gpu.enabled|This is an experimental feature and is likely to change in the future. Enable (true) or disable (false) support for scheduling Python Pandas UDFs with GPU resources. When enabled, pandas UDFs are assumed to share the same GPU that the RAPIDs accelerator uses and will honor the python GPU configs|false|Runtime diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/GpuLore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/GpuLore.scala index bc156fe5636..be2c7968189 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/GpuLore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/GpuLore.scala @@ -23,11 +23,11 @@ import scala.reflect.ClassTag import com.nvidia.spark.rapids.{GpuColumnarToRowExec, GpuExec, GpuFilterExec, RapidsConf} import com.nvidia.spark.rapids.Arm.withResource -import com.nvidia.spark.rapids.shims.ShimLeafExecNode +import com.nvidia.spark.rapids.shims.{ShimLeafExecNode, SparkShimImpl} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.SparkEnv +import org.apache.spark.SparkEnv import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, SparkPlan, SQLExecution} @@ -80,7 +80,8 @@ object GpuLore { } def dumpPlan[T <: SparkPlan : ClassTag](plan: T, rootPath: Path): Unit = { - dumpObject(plan, pathOfRootPlanMeta(rootPath), plan.session.sparkContext.hadoopConfiguration) + dumpObject(plan, pathOfRootPlanMeta(rootPath), + SparkShimImpl.sessionFromPlan(plan).sparkContext.hadoopConfiguration) } def dumpObject[T: ClassTag](obj: T, path: Path, hadoopConf: Configuration): Unit = { @@ -158,7 +159,9 @@ object GpuLore { private def nextLoreIdOf(plan: SparkPlan): Option[Int] = { // When the execution id is not set, it means there is no actual execution happening, in this // case we don't need to generate lore id. - Option(plan.session.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)) + Option(SparkShimImpl.sessionFromPlan(plan) + .sparkContext + .getLocalProperty(SQLExecution.EXECUTION_ID_KEY)) .map { executionId => idGen.computeIfAbsent(executionId, _ => new AtomicInteger(0)).getAndIncrement() } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/replay.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/replay.scala index d5799349937..e58297c4ba6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/replay.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/replay.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.SerializableConfiguration case class GpuLoreReplayExec(idxInParent: Int, parentRootPath: Path) extends LeafExecNode with GpuExec { - private lazy val rdd = new GpuLoreReplayRDD(session.sparkContext, + private lazy val rdd = new GpuLoreReplayRDD(sparkSession.sparkContext, GpuLore.pathOfChild(parentRootPath, idxInParent)) override def output: Seq[Attribute] = rdd.loreRDDMeta.attrs diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala index e2ec7d99dbb..bd30459d63e 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala @@ -31,11 +31,11 @@ import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import com.nvidia.spark.rapids.lore.GpuLore.LORE_DUMP_RDD_TAG import com.nvidia.spark.rapids.lore.{GpuLoreDumpRDD, SimpleRDD} +import com.nvidia.spark.rapids.lore.GpuLore.LORE_DUMP_RDD_TAG import com.nvidia.spark.rapids.shims.{ShimBroadcastExchangeLike, ShimUnaryExecNode, SparkShimImpl} -import org.apache.spark.SparkException +import org.apache.spark.SparkException import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.launcher.SparkLauncher