From 5d367430e2d66f5d3196299e23d1931dfb65d0f6 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Thu, 14 Nov 2024 18:00:47 -0800 Subject: [PATCH 01/23] diagnostic view 2 start Signed-off-by: cindyyuanjiang --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 48 +++++++++++++++++++ .../profiling/ProfileClassWarehouse.scala | 39 +++++++++++++-- 2 files changed, 84 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index e32061726..1a9268a45 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -300,6 +300,13 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap val med = Math.max(taskInfo.med, driverInfo.med) val total = Math.max(taskInfo.total, driverInfo.total) + metric.stageIds.map { stageId => + val accumInfo = app.accumManager.accumInfoMap.get(metric.accumulatorId) + val taskIdsInStage = + accumInfo.taskUpdatesMap.keys.filter(id => app.taskManager.getAllTasksStageAttempt(stageId).contains(id)) + // TODO: calculate stats of above set of task ids, this is the result for this stage + } + Some(SQLAccumProfileResults(appIndex, metric.sqlID, metric.nodeID, metric.nodeName, metric.accumulatorId, metric.name, min, med, max, total, metric.metricType, metric.stageIds.mkString(","))) @@ -309,6 +316,47 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap } } + def generateIODiagnosticAccums(sqlAccums: Seq[SQLAccumProfileResults]): + Seq[IODiagnosticProfileResult] = { + // TODO: filter node names only for io + sqlAccums.flatMap { sqlAccum => + sqlAccum.stageIds.split(",").map { stageIdStr => + val stageId = stageIdStr.toInt + val stageTaskIds = app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet + val accumInfo = app.accumManager.accumInfoMap.get(sqlAccum.accumulatorId) + // val taskAccumsInStage = accumInfo.taskUpdatesMap.collect { + // case (id, accum) if app.taskManager.getAllTasksStageAttempt(stageId).contains(id) => + // accum + // } + val taskUpatesSubset = accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted + // val sortedTaskUpdates = taskAccumsInStage.toSeq.sorted + if (taskUpatesSubset.isEmpty) { + None + } else { + val min = taskUpatesSubset.head + val max = taskUpatesSubset.last + val sum = taskUpatesSubset.sum + val median = if (taskUpatesSubset.size % 2 == 0) { + val mid = taskUpatesSubset.size / 2 + (taskUpatesSubset(mid) + taskUpatesSubset(mid - 1)) / 2 + } else { + taskUpatesSubset(taskUpatesSubset.size / 2) + } + Some(IODiagnosticProfileResult( + appIndex, + app.getAppName, + app.appId, + stageId, + sqlAccum.sqlID, + app.stageManager.getDurationById(stageId), + sqlAccum.nodeName, + // TODO + )) + } + + } + } + /** * Generate the stage level metrics for the SQL plan including GPU metrics if applicable. * Along with Spark defined metrics, below is the list of GPU metrics that are collected if they diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index b98d36aab..314fe6888 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -206,9 +206,19 @@ class SQLExecutionInfoClass( } } -case class SQLAccumProfileResults(appIndex: Int, sqlID: Long, nodeID: Long, - nodeName: String, accumulatorId: Long, name: String, min: Long, median:Long, - max: Long, total: Long, metricType: String, stageIds: String) extends ProfileResult { +case class SQLAccumProfileResults( + appIndex: Int, + sqlID: Long, + nodeID: Long, + nodeName: String, + accumulatorId: Long, + name: String, + min: Long, + median:Long, + max: Long, + total: Long, + metricType: String, + stageIds: String) extends ProfileResult { override val outputHeaders = Seq("appIndex", "sqlID", "nodeID", "nodeName", "accumulatorId", "name", "min", "median", "max", "total", "metricType", "stageIds") override def convertToSeq: Seq[String] = { @@ -753,6 +763,29 @@ case class SQLTaskAggMetricsProfileResult( } } +case class IODiagnosticProfileResult( + appIndex: Int, + appName: String, + appId: String, + stageId: Long, + sqlId: Long, + stageDuration: Option[Long], + nodeName: String, + outputRows: SQLAccumProfileResults, + scanTime: SQLAccumProfileResults, + serializationTime: SQLAccumProfileResults, + outputBatches: SQLAccumProfileResults, + buffeTime: SQLAccumProfileResults, + shuffleWriteTime: SQLAccumProfileResults, + gpuDecodeTime: SQLAccumProfileResults) extends ProfileResult { + + val durStr = duration match { + case Some(dur) => dur.toString + case None => "null" + } + +} + case class IOAnalysisProfileResult( appIndex: Int, appId: String, From 6978f79166020d6afbea10031106deb1a7434ea2 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Tue, 26 Nov 2024 11:54:42 -0800 Subject: [PATCH 02/23] update IODiagnosticProfileResult Signed-off-by: cindyyuanjiang --- .../profiling/ProfileClassWarehouse.scala | 87 ++++++++++++++++++- 1 file changed, 86 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index ad6d50b54..23b9e28b5 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -894,7 +894,7 @@ case class IODiagnosticProfileResult( appId: String, stageId: Long, sqlId: Long, - stageDuration: Option[Long], + duration: Option[Long], nodeName: String, outputRows: SQLAccumProfileResults, scanTime: SQLAccumProfileResults, @@ -902,6 +902,7 @@ case class IODiagnosticProfileResult( outputBatches: SQLAccumProfileResults, buffeTime: SQLAccumProfileResults, shuffleWriteTime: SQLAccumProfileResults, + fetchWaitTime: SQLAccumProfileResults, gpuDecodeTime: SQLAccumProfileResults) extends ProfileResult { val durStr = duration match { @@ -909,6 +910,90 @@ case class IODiagnosticProfileResult( case None => "null" } + override val outputHeaders = { + Seq("appIndex", + "appName", + "appId", + "stageId", + "sqlId" + "stageDurationMs", + "nodeName", + "outputRowsMin", + "outputRowsMedian", + "outputRowsMax", + "outputRowsTotal", + "scanTimeMin", + "scanTimeMedian", + "scanTimeMax", + "scanTimeTotal", + "serializationTimeMin", + "serializationTimeMedian", + "serializationTimeMax", + "serializationTimeTotal", + "outputBatchesMin", + "outputBatchesMedian", + "outputBatchesMax", + "outputBatchesTotal", + "buffeTimeMin", + "buffeTimeMedian", + "buffeTimeMax", + "buffeTimeTotal", + "shuffleWriteTimeMin", + "shuffleWriteTimeMedian", + "shuffleWriteTimeMax", + "shuffleWriteTimeTotal", + "fetchWaitTimeMin", + "fetchWaitTimeMedian", + "fetchWaitTimeMax", + "fetchWaitTimeTotal", + "gpuDecodeTimeMin", + "gpuDecodeTimeMedian", + "gpuDecodeTimeMax", + "gpuDecodeTimeTotal") + } + + override def convertToSeq: Seq[String] = { + Seq(appIndex.toString, + appName, + appId, + stageId.toString, + sqlId.toString, + durStr, + nodeName, + outputRows.min.toString, + outputRows.median.toString, + outputRows.max.toString, + outputRows.total.toString, + scanTime.min.toString, + scanTime.median.toString, + scanTime.max.toString, + scanTime.total.toString, + serializationTime.min.toString, + serializationTime.median.toString, + serializationTime.max.toString, + serializationTime.total.toString, + outputBatches.min.toString, + outputBatches.median.toString, + outputBatches.max.toString, + outputBatches.total.toString, + buffeTime.min.toString, + buffeTime.median.toString, + buffeTime.max.toString, + buffeTime.total.toString, + shuffleWriteTime.min.toString, + shuffleWriteTime.median.toString, + shuffleWriteTime.max.toString, + shuffleWriteTime.total.toString, + fetchWaitTime.min.toString, + fetchWaitTime.median.toString, + fetchWaitTime.max.toString, + fetchWaitTime.total.toString, + gpuDecodeTime.min.toString, + gpuDecodeTime.median.toString, + gpuDecodeTime.max.toString) + } + + override def convertToCSVSeq: Seq[String] = convertToSeq } case class IOAnalysisProfileResult( From e0fe6928d7a74c57418397b0c27ce8a04ffa6a57 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Tue, 3 Dec 2024 16:54:58 -0800 Subject: [PATCH 03/23] working implementation of view 2 Signed-off-by: cindyyuanjiang --- .../rapids/tool/analysis/AnalysisUtils.scala | 24 ++++ .../tool/analysis/AppSQLPlanAnalyzer.scala | 121 +++++++++++------- .../tool/profiling/CollectInformation.scala | 4 + .../profiling/DiagnosticSummaryInfo.scala | 3 +- .../profiling/ProfileClassWarehouse.scala | 91 ++++++++----- .../rapids/tool/profiling/Profiler.scala | 6 +- .../spark/rapids/tool/views/SQLView.scala | 20 ++- 7 files changed, 186 insertions(+), 83 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala index 01f6989c7..527174e81 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala @@ -34,3 +34,27 @@ object StageAccumDiagnosticMetrics { SW_TOTAL_BYTES_METRIC, SR_FETCH_WAIT_TIME_METRIC, SW_WRITE_TIME_METRIC, GPU_SEMAPHORE_WAIT_METRIC) } + +object IOAccumDiagnosticMetrics { + val OUTPUT_ROWS_METRIC = "output rows" // other names: join output rows, number of output rows + val SCAN_TIME_METRIC = "scan time" + val OUTPUT_BATCHES_METRIC = "output columnar batches" + val BUFFER_TIME_METRIC = "buffer time" + val SHUFFLE_WRITE_TIME_METRIC = "shuffle write time" + val FETCH_WAIT_TIME_METRIC = "fetch wait time" + val GPU_DECODE_TIME_METRIC = "GPU decode time" + + /** + * Get all IO diagnostic metrics + */ + def getAllIODiagnosticMetrics: Set[String] = Set(OUTPUT_ROWS_METRIC, + SCAN_TIME_METRIC, OUTPUT_BATCHES_METRIC, BUFFER_TIME_METRIC, + SHUFFLE_WRITE_TIME_METRIC, FETCH_WAIT_TIME_METRIC, GPU_DECODE_TIME_METRIC) + + /** + * Check if input is an IO diagnostic metric + */ + def isIODiagnosticMetric(metric: String): Boolean = { + getAllIODiagnosticMetrics.contains(metric) || metric.contains(OUTPUT_ROWS_METRIC) + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index d9b826ce4..852dd9259 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -18,8 +18,9 @@ package com.nvidia.spark.rapids.tool.analysis import scala.collection.mutable.{AbstractSet, ArrayBuffer, HashMap, LinkedHashSet} +import com.nvidia.spark.rapids.tool.analysis.IOAccumDiagnosticMetrics._ import com.nvidia.spark.rapids.tool.planparser.SQLPlanParser -import com.nvidia.spark.rapids.tool.profiling.{AccumProfileResults, SQLAccumProfileResults, SQLMetricInfoCase, SQLStageInfoProfileResult, UnsupportedSQLPlan, WholeStageCodeGenResults} +import com.nvidia.spark.rapids.tool.profiling.{AccumProfileResults, IODiagnosticProfileResult, SQLAccumProfileResults, SQLMetricInfoCase, SQLStageInfoProfileResult, UnsupportedSQLPlan, WholeStageCodeGenResults} import com.nvidia.spark.rapids.tool.qualification.QualSQLPlanAnalyzer import org.apache.spark.sql.execution.SparkPlanInfo @@ -27,7 +28,7 @@ import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SparkPlanGraphCluster, import org.apache.spark.sql.rapids.tool.{AppBase, RDDCheckHelper, SqlPlanInfoGraphBuffer, SqlPlanInfoGraphEntry} import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo import org.apache.spark.sql.rapids.tool.qualification.QualificationAppInfo -import org.apache.spark.sql.rapids.tool.store.DataSourceRecord +import org.apache.spark.sql.rapids.tool.store.{AccumInfo, AccumMetaRef, AccumNameRef, DataSourceRecord} import org.apache.spark.sql.rapids.tool.util.ToolsPlanGraph /** @@ -62,6 +63,11 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap // and AccumProfileResults) val stageToDiagnosticMetrics: HashMap[Long, HashMap[String, AccumProfileResults]] = HashMap.empty[Long, HashMap[String, AccumProfileResults]] + // A map between (sql ID, node name, stage IDs) and a list of IO diagnostic metrics results + // (stored as an array of SQLAccumProfileResults) - we need all info in the key to uniquely + // identify each SQLAccumProfileResults + val IODiagnosticMetrics: HashMap[(Long, String, String), ArrayBuffer[SQLAccumProfileResults]] = + HashMap.empty[(Long, String, String), ArrayBuffer[SQLAccumProfileResults]] /** * Given an input diagnostic metric result, update stageToDiagnosticMetrics mapping @@ -75,6 +81,18 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap stageToDiagnosticMetrics(stageId)(accum.accMetaRef.getName()) = accum } + /** + * Given an input IO diagnostic metric result, update IODiagnosticMetrics mapping + * @param accum SQLAccumProfileResults to be analyzed + */ + private def updateIODiagnosticMetrics(accum: SQLAccumProfileResults): Unit = { + val key = (accum.sqlID, accum.nodeName, accum.stageIds) + if (!IODiagnosticMetrics.contains(key)) { + IODiagnosticMetrics(key) = ArrayBuffer[SQLAccumProfileResults]() + } + IODiagnosticMetrics(key) += accum + } + /** * Connects Operators to Stages using AccumulatorIDs. * TODO: This function can be fused in the visitNode function to avoid the extra iteration. @@ -319,61 +337,72 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap val med = Math.max(taskInfo.med, driverInfo.med) val total = Math.max(taskInfo.total, driverInfo.total) - metric.stageIds.map { stageId => - val accumInfo = app.accumManager.accumInfoMap.get(metric.accumulatorId) - val taskIdsInStage = - accumInfo.taskUpdatesMap.keys.filter(id => app.taskManager.getAllTasksStageAttempt(stageId).contains(id)) - // TODO: calculate stats of above set of task ids, this is the result for this stage + val sqlAccumProileResult = SQLAccumProfileResults(appIndex, metric.sqlID, + metric.nodeID, metric.nodeName, metric.accumulatorId, metric.name, + min, med, max, total, metric.metricType, metric.stageIds.mkString(",")) + + if (isIODiagnosticMetric(metric.name)) { + updateIODiagnosticMetrics(sqlAccumProileResult) } - Some(SQLAccumProfileResults(appIndex, metric.sqlID, - metric.nodeID, metric.nodeName, metric.accumulatorId, metric.name, - min, med, max, total, metric.metricType, metric.stageIds.mkString(","))) + Some(sqlAccumProileResult) } else { None } } } - def generateIODiagnosticAccums(sqlAccums: Seq[SQLAccumProfileResults]): - Seq[IODiagnosticProfileResult] = { - // TODO: filter node names only for io - sqlAccums.flatMap { sqlAccum => - sqlAccum.stageIds.split(",").map { stageIdStr => - val stageId = stageIdStr.toInt - val stageTaskIds = app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet - val accumInfo = app.accumManager.accumInfoMap.get(sqlAccum.accumulatorId) - // val taskAccumsInStage = accumInfo.taskUpdatesMap.collect { - // case (id, accum) if app.taskManager.getAllTasksStageAttempt(stageId).contains(id) => - // accum - // } - val taskUpatesSubset = accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted - // val sortedTaskUpdates = taskAccumsInStage.toSeq.sorted - if (taskUpatesSubset.isEmpty) { + def generateIODiagnosticAccums(): Seq[IODiagnosticProfileResult] = { + val zeroRecord = StatisticsMetrics.ZERO_RECORD + IODiagnosticMetrics.toSeq.flatMap { case ((sqlId, nodeName, stageIds), sqlAccums) => + stageIds.split(",").map(_.toInt).flatMap { stageId => + val stageDiagnosticInfo = HashMap.empty[String, StatisticsMetrics] + + sqlAccums.foreach { sqlAccum => + val stageTaskIds = app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet + val accumInfo = app.accumManager.accumInfoMap.getOrElse(sqlAccum.accumulatorId, + new AccumInfo(AccumMetaRef(0L, AccumNameRef("")))) + val taskUpatesSubset = accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted + if (!taskUpatesSubset.isEmpty) { + val min = taskUpatesSubset.head + val max = taskUpatesSubset.last + val sum = taskUpatesSubset.sum + val median = if (taskUpatesSubset.size % 2 == 0) { + val mid = taskUpatesSubset.size / 2 + (taskUpatesSubset(mid) + taskUpatesSubset(mid - 1)) / 2 + } else { + taskUpatesSubset(taskUpatesSubset.size / 2) + } + val metricName = if (sqlAccum.name.contains(OUTPUT_ROWS_METRIC)) { + OUTPUT_ROWS_METRIC + } else { + sqlAccum.name + } + stageDiagnosticInfo(metricName) = StatisticsMetrics(min, median, max, sum) + } + } + + if (stageDiagnosticInfo.isEmpty) { None } else { - val min = taskUpatesSubset.head - val max = taskUpatesSubset.last - val sum = taskUpatesSubset.sum - val median = if (taskUpatesSubset.size % 2 == 0) { - val mid = taskUpatesSubset.size / 2 - (taskUpatesSubset(mid) + taskUpatesSubset(mid - 1)) / 2 - } else { - taskUpatesSubset(taskUpatesSubset.size / 2) - } - Some(IODiagnosticProfileResult( - appIndex, - app.getAppName, - app.appId, - stageId, - sqlAccum.sqlID, - app.stageManager.getDurationById(stageId), - sqlAccum.nodeName, - // TODO - )) + Some(IODiagnosticProfileResult( + appIndex, + app.getAppName, + app.appId, + stageId, + sqlId, + app.stageManager.getDurationById(stageId), + nodeName, + stageDiagnosticInfo.getOrElse(OUTPUT_ROWS_METRIC, zeroRecord), + stageDiagnosticInfo.getOrElse(SCAN_TIME_METRIC, zeroRecord), + stageDiagnosticInfo.getOrElse(OUTPUT_BATCHES_METRIC, zeroRecord), + stageDiagnosticInfo.getOrElse(BUFFER_TIME_METRIC, zeroRecord), + stageDiagnosticInfo.getOrElse(SHUFFLE_WRITE_TIME_METRIC, zeroRecord), + stageDiagnosticInfo.getOrElse(FETCH_WAIT_TIME_METRIC, zeroRecord), + stageDiagnosticInfo.getOrElse(GPU_DECODE_TIME_METRIC, zeroRecord))) + } } - - } + }.toSeq } /** diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CollectInformation.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CollectInformation.scala index b9c622334..a4a507d8b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CollectInformation.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CollectInformation.scala @@ -95,6 +95,10 @@ class CollectInformation(apps: Seq[ApplicationInfo]) extends Logging { ProfStageMetricView.getRawView(apps) } + def getIODiagnosticMetrics: Seq[IODiagnosticProfileResult] = { + ProfIODiagnosticMetricsView.getRawView(apps) + } + /** * This function is meant to clean up Delta log execs so that you could align * SQL ids between CPU and GPU eventlogs. It attempts to remove any delta log diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala index 6ba34f7f5..79a2a9582 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala @@ -21,5 +21,6 @@ package com.nvidia.spark.rapids.tool.profiling * TODO: We plan to add two more fields/views in upcoming PRs. */ case class DiagnosticSummaryInfo( - stageDiagnostics: Seq[StageDiagnosticResult] + stageDiagnostics: Seq[StageDiagnosticResult], + IODiagnostics: Seq[IODiagnosticProfileResult] ) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index 23b9e28b5..75de75dd3 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -18,6 +18,8 @@ package com.nvidia.spark.rapids.tool.profiling import scala.collection.Map +import com.nvidia.spark.rapids.tool.analysis.StatisticsMetrics + import org.apache.spark.resource.{ExecutorResourceRequest, TaskResourceRequest} import org.apache.spark.sql.rapids.tool.store.AccumMetaRef import org.apache.spark.sql.rapids.tool.util.{SparkRuntime, StringUtils} @@ -894,28 +896,22 @@ case class IODiagnosticProfileResult( appId: String, stageId: Long, sqlId: Long, - duration: Option[Long], + duration: Long, nodeName: String, - outputRows: SQLAccumProfileResults, - scanTime: SQLAccumProfileResults, - serializationTime: SQLAccumProfileResults, - outputBatches: SQLAccumProfileResults, - buffeTime: SQLAccumProfileResults, - shuffleWriteTime: SQLAccumProfileResults, - fetchWaitTime: SQLAccumProfileResults, - gpuDecodeTime: SQLAccumProfileResults) extends ProfileResult { - - val durStr = duration match { - case Some(dur) => dur.toString - case None => "null" - } + outputRows: StatisticsMetrics, + scanTime: StatisticsMetrics, + outputBatches: StatisticsMetrics, + buffeTime: StatisticsMetrics, + shuffleWriteTime: StatisticsMetrics, + fetchWaitTime: StatisticsMetrics, + gpuDecodeTime: StatisticsMetrics) extends ProfileResult { override val outputHeaders = { Seq("appIndex", "appName", "appId", "stageId", - "sqlId" + "sqlId", "stageDurationMs", "nodeName", "outputRowsMin", @@ -926,10 +922,6 @@ case class IODiagnosticProfileResult( "scanTimeMedian", "scanTimeMax", "scanTimeTotal", - "serializationTimeMin", - "serializationTimeMedian", - "serializationTimeMax", - "serializationTimeTotal", "outputBatchesMin", "outputBatchesMedian", "outputBatchesMax", @@ -958,42 +950,75 @@ case class IODiagnosticProfileResult( appId, stageId.toString, sqlId.toString, - durStr, + duration.toString, nodeName, outputRows.min.toString, - outputRows.median.toString, + outputRows.med.toString, outputRows.max.toString, outputRows.total.toString, scanTime.min.toString, - scanTime.median.toString, + scanTime.med.toString, scanTime.max.toString, scanTime.total.toString, - serializationTime.min.toString, - serializationTime.median.toString, - serializationTime.max.toString, - serializationTime.total.toString, outputBatches.min.toString, - outputBatches.median.toString, + outputBatches.med.toString, outputBatches.max.toString, outputBatches.total.toString, buffeTime.min.toString, - buffeTime.median.toString, + buffeTime.med.toString, buffeTime.max.toString, buffeTime.total.toString, shuffleWriteTime.min.toString, - shuffleWriteTime.median.toString, + shuffleWriteTime.med.toString, shuffleWriteTime.max.toString, shuffleWriteTime.total.toString, fetchWaitTime.min.toString, - fetchWaitTime.median.toString, + fetchWaitTime.med.toString, fetchWaitTime.max.toString, fetchWaitTime.total.toString, gpuDecodeTime.min.toString, - gpuDecodeTime.median.toString, - gpuDecodeTime.max.toString) + gpuDecodeTime.med.toString, + gpuDecodeTime.max.toString, + gpuDecodeTime.total.toString) } - override def convertToCSVSeq: Seq[String] = convertToSeq + override def convertToCSVSeq: Seq[String] = { + Seq(appIndex.toString, + appName, + appId, + stageId.toString, + sqlId.toString, + duration.toString, + StringUtils.reformatCSVString(nodeName), + outputRows.min.toString, + outputRows.med.toString, + outputRows.max.toString, + outputRows.total.toString, + scanTime.min.toString, + scanTime.med.toString, + scanTime.max.toString, + scanTime.total.toString, + outputBatches.min.toString, + outputBatches.med.toString, + outputBatches.max.toString, + outputBatches.total.toString, + buffeTime.min.toString, + buffeTime.med.toString, + buffeTime.max.toString, + buffeTime.total.toString, + shuffleWriteTime.min.toString, + shuffleWriteTime.med.toString, + shuffleWriteTime.max.toString, + shuffleWriteTime.total.toString, + fetchWaitTime.min.toString, + fetchWaitTime.med.toString, + fetchWaitTime.max.toString, + fetchWaitTime.total.toString, + gpuDecodeTime.min.toString, + gpuDecodeTime.med.toString, + gpuDecodeTime.max.toString, + gpuDecodeTime.total.toString) + } } case class IOAnalysisProfileResult( diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala index 75b4c4590..9a1baac7e 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala @@ -403,7 +403,7 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea failedTasks, failedStages, failedJobs, removedBMs, removedExecutors, unsupportedOps, sparkProps, collect.getSQLToStage, wholeStage, maxTaskInputInfo, appLogPath, analysis.ioAggs, systemProps, sqlIdAlign, sparkRapidsBuildInfo), - compareRes, DiagnosticSummaryInfo(analysis.stageDiagnostics)) + compareRes, DiagnosticSummaryInfo(analysis.stageDiagnostics, collect.getIODiagnosticMetrics)) } /** @@ -570,12 +570,14 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea } // Write diagnostic related results to CSV files val diagnostics = if (outputCombined) { - Seq(DiagnosticSummaryInfo(diagnosticSum.flatMap(_.stageDiagnostics))) + Seq(DiagnosticSummaryInfo(diagnosticSum.flatMap(_.stageDiagnostics), + diagnosticSum.flatMap(_.IODiagnostics))) } else { diagnosticSum } diagnostics.foreach { diagnostoic => profileOutputWriter.writeCSVTable(STAGE_DIAGNOSTICS_LABEL, diagnostoic.stageDiagnostics) + profileOutputWriter.writeCSVTable(ProfIODiagnosticMetricsView.getLabel, diagnostoic.IODiagnostics) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala index e52340dce..fad2ba45d 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala @@ -17,7 +17,7 @@ package com.nvidia.spark.rapids.tool.views import com.nvidia.spark.rapids.tool.analysis.{AppSQLPlanAnalyzer, ProfAppIndexMapperTrait, QualAppIndexMapperTrait} -import com.nvidia.spark.rapids.tool.profiling.{SQLAccumProfileResults, SQLCleanAndAlignIdsProfileResult, SQLPlanClassifier, WholeStageCodeGenResults} +import com.nvidia.spark.rapids.tool.profiling.{IODiagnosticProfileResult, SQLAccumProfileResults, SQLCleanAndAlignIdsProfileResult, SQLPlanClassifier, WholeStageCodeGenResults} import org.apache.spark.sql.rapids.tool.AppBase import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo @@ -109,6 +109,24 @@ object ProfSQLPlanMetricsView extends AppSQLPlanMetricsViewTrait with ProfAppInd } } +object ProfIODiagnosticMetricsView extends ViewableTrait[IODiagnosticProfileResult] with ProfAppIndexMapperTrait { + override def getLabel: String = "IO Diagnostic Metrics" + override def getDescription: String = "IO Diagnostic Metrics" + + override def sortView( + rows: Seq[IODiagnosticProfileResult]): Seq[IODiagnosticProfileResult] = { + rows.sortBy(cols => (cols.appIndex, cols.duration, cols.stageId, cols.sqlId)) + } + + override def getRawView(app: AppBase, index: Int): Seq[IODiagnosticProfileResult] = { + app match { + case app: ApplicationInfo => + sortView(app.planMetricProcessor.generateIODiagnosticAccums()) + case _ => Seq.empty + } + } +} + object QualSQLPlanMetricsView extends AppSQLPlanMetricsViewTrait with QualAppIndexMapperTrait { override def getRawView(app: AppBase, index: Int): Seq[SQLAccumProfileResults] = { // TODO: Fix this implementation when we have a better way to get bind between App and From b2a8c0efaa6ce9fae94394a22ece11f9b8fbb523 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Thu, 5 Dec 2024 14:07:24 -0800 Subject: [PATCH 04/23] add node name in view 2 with some print statements Signed-off-by: cindyyuanjiang --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 30 ++++++++++++++----- .../profiling/ProfileClassWarehouse.scala | 4 +++ .../spark/rapids/tool/views/SQLView.scala | 2 +- 3 files changed, 27 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 852dd9259..f82069a54 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -63,11 +63,12 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap // and AccumProfileResults) val stageToDiagnosticMetrics: HashMap[Long, HashMap[String, AccumProfileResults]] = HashMap.empty[Long, HashMap[String, AccumProfileResults]] - // A map between (sql ID, node name, stage IDs) and a list of IO diagnostic metrics results - // (stored as an array of SQLAccumProfileResults) - we need all info in the key to uniquely - // identify each SQLAccumProfileResults - val IODiagnosticMetrics: HashMap[(Long, String, String), ArrayBuffer[SQLAccumProfileResults]] = - HashMap.empty[(Long, String, String), ArrayBuffer[SQLAccumProfileResults]] + // A map between (sql ID, node ID, node name, stage IDs) and a list of IO diagnostic metrics + // results (stored as an array of SQLAccumProfileResults) - we need all info in the key to + // uniquely identify each SQLAccumProfileResults + val IODiagnosticMetrics: + HashMap[(Long, Long, String, String), ArrayBuffer[SQLAccumProfileResults]] = + HashMap.empty[(Long, Long, String, String), ArrayBuffer[SQLAccumProfileResults]] /** * Given an input diagnostic metric result, update stageToDiagnosticMetrics mapping @@ -86,7 +87,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap * @param accum SQLAccumProfileResults to be analyzed */ private def updateIODiagnosticMetrics(accum: SQLAccumProfileResults): Unit = { - val key = (accum.sqlID, accum.nodeName, accum.stageIds) + val key = (accum.sqlID, accum.nodeID, accum.nodeName, accum.stageIds) if (!IODiagnosticMetrics.contains(key)) { IODiagnosticMetrics(key) = ArrayBuffer[SQLAccumProfileResults]() } @@ -352,9 +353,17 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap } } + /** + * Generate IO-related diagnostic metrics for the SQL plan: + * output rows, scan time, output batches, buffer time, shuffle write time, fetch wait time, GPU + * decode time. + * @return a sequence of IODiagnosticProfileResult + */ def generateIODiagnosticAccums(): Seq[IODiagnosticProfileResult] = { val zeroRecord = StatisticsMetrics.ZERO_RECORD - IODiagnosticMetrics.toSeq.flatMap { case ((sqlId, nodeName, stageIds), sqlAccums) => + IODiagnosticMetrics.toSeq.flatMap { case ((sqlId, nodeId, nodeName, stageIds), sqlAccums) => + // System.err.println(s"nodeId = $nodeId") + // System.err.println(s"stageIds = $stageIds\n") stageIds.split(",").map(_.toInt).flatMap { stageId => val stageDiagnosticInfo = HashMap.empty[String, StatisticsMetrics] @@ -363,7 +372,11 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap val accumInfo = app.accumManager.accumInfoMap.getOrElse(sqlAccum.accumulatorId, new AccumInfo(AccumMetaRef(0L, AccumNameRef("")))) val taskUpatesSubset = accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted - if (!taskUpatesSubset.isEmpty) { + // System.err.println(s"node name = $nodeName") + // System.err.println(s"stage id = $stageId") + // System.err.println(s"metricName = ${sqlAccum.name}") + // System.err.println(s"taskUpatesSubset = ${taskUpatesSubset.toList}") + if (taskUpatesSubset.nonEmpty) { val min = taskUpatesSubset.head val max = taskUpatesSubset.last val sum = taskUpatesSubset.sum @@ -392,6 +405,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap stageId, sqlId, app.stageManager.getDurationById(stageId), + nodeId, nodeName, stageDiagnosticInfo.getOrElse(OUTPUT_ROWS_METRIC, zeroRecord), stageDiagnosticInfo.getOrElse(SCAN_TIME_METRIC, zeroRecord), diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index 75de75dd3..c7d56c10d 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -897,6 +897,7 @@ case class IODiagnosticProfileResult( stageId: Long, sqlId: Long, duration: Long, + nodeId: Long, nodeName: String, outputRows: StatisticsMetrics, scanTime: StatisticsMetrics, @@ -913,6 +914,7 @@ case class IODiagnosticProfileResult( "stageId", "sqlId", "stageDurationMs", + "nodeId", "nodeName", "outputRowsMin", "outputRowsMedian", @@ -951,6 +953,7 @@ case class IODiagnosticProfileResult( stageId.toString, sqlId.toString, duration.toString, + nodeId.toString, nodeName, outputRows.min.toString, outputRows.med.toString, @@ -989,6 +992,7 @@ case class IODiagnosticProfileResult( stageId.toString, sqlId.toString, duration.toString, + nodeId.toString, StringUtils.reformatCSVString(nodeName), outputRows.min.toString, outputRows.med.toString, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala index fad2ba45d..1106ae387 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala @@ -115,7 +115,7 @@ object ProfIODiagnosticMetricsView extends ViewableTrait[IODiagnosticProfileResu override def sortView( rows: Seq[IODiagnosticProfileResult]): Seq[IODiagnosticProfileResult] = { - rows.sortBy(cols => (cols.appIndex, cols.duration, cols.stageId, cols.sqlId)) + rows.sortBy(cols => (cols.appIndex, cols.duration, cols.stageId, cols.sqlId, cols.nodeId)) } override def getRawView(app: AppBase, index: Int): Seq[IODiagnosticProfileResult] = { From e0d6a0c3957b754eb021a219019c0c74d7465000 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Thu, 5 Dec 2024 17:38:18 -0800 Subject: [PATCH 05/23] fixe unit tests Signed-off-by: cindyyuanjiang --- .../rapids/tool/analysis/AppSQLPlanAnalyzer.scala | 13 ++++--------- .../tool/profiling/ProfileClassWarehouse.scala | 8 ++++---- .../spark/rapids/tool/profiling/Profiler.scala | 3 ++- .../nvidia/spark/rapids/tool/views/SQLView.scala | 3 ++- .../tool/profiling/ApplicationInfoSuite.scala | 8 ++++---- 5 files changed, 16 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index f82069a54..f5561c6ec 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -362,20 +362,15 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap def generateIODiagnosticAccums(): Seq[IODiagnosticProfileResult] = { val zeroRecord = StatisticsMetrics.ZERO_RECORD IODiagnosticMetrics.toSeq.flatMap { case ((sqlId, nodeId, nodeName, stageIds), sqlAccums) => - // System.err.println(s"nodeId = $nodeId") - // System.err.println(s"stageIds = $stageIds\n") - stageIds.split(",").map(_.toInt).flatMap { stageId => + stageIds.split(",").filter(_.nonEmpty).map(_.toInt).flatMap { stageId => val stageDiagnosticInfo = HashMap.empty[String, StatisticsMetrics] sqlAccums.foreach { sqlAccum => val stageTaskIds = app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet val accumInfo = app.accumManager.accumInfoMap.getOrElse(sqlAccum.accumulatorId, new AccumInfo(AccumMetaRef(0L, AccumNameRef("")))) - val taskUpatesSubset = accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted - // System.err.println(s"node name = $nodeName") - // System.err.println(s"stage id = $stageId") - // System.err.println(s"metricName = ${sqlAccum.name}") - // System.err.println(s"taskUpatesSubset = ${taskUpatesSubset.toList}") + val taskUpatesSubset = + accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted if (taskUpatesSubset.nonEmpty) { val min = taskUpatesSubset.head val max = taskUpatesSubset.last @@ -402,8 +397,8 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap appIndex, app.getAppName, app.appId, - stageId, sqlId, + stageId, app.stageManager.getDurationById(stageId), nodeId, nodeName, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index c7d56c10d..f535672ff 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -894,8 +894,8 @@ case class IODiagnosticProfileResult( appIndex: Int, appName: String, appId: String, - stageId: Long, sqlId: Long, + stageId: Long, duration: Long, nodeId: Long, nodeName: String, @@ -911,8 +911,8 @@ case class IODiagnosticProfileResult( Seq("appIndex", "appName", "appId", - "stageId", "sqlId", + "stageId", "stageDurationMs", "nodeId", "nodeName", @@ -950,8 +950,8 @@ case class IODiagnosticProfileResult( Seq(appIndex.toString, appName, appId, - stageId.toString, sqlId.toString, + stageId.toString, duration.toString, nodeId.toString, nodeName, @@ -989,8 +989,8 @@ case class IODiagnosticProfileResult( Seq(appIndex.toString, appName, appId, - stageId.toString, sqlId.toString, + stageId.toString, duration.toString, nodeId.toString, StringUtils.reformatCSVString(nodeName), diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala index 9a1baac7e..b05447d3d 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala @@ -577,7 +577,8 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea } diagnostics.foreach { diagnostoic => profileOutputWriter.writeCSVTable(STAGE_DIAGNOSTICS_LABEL, diagnostoic.stageDiagnostics) - profileOutputWriter.writeCSVTable(ProfIODiagnosticMetricsView.getLabel, diagnostoic.IODiagnostics) + profileOutputWriter.writeCSVTable(ProfIODiagnosticMetricsView.getLabel, + diagnostoic.IODiagnostics) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala index 1106ae387..7c978d49e 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala @@ -109,7 +109,8 @@ object ProfSQLPlanMetricsView extends AppSQLPlanMetricsViewTrait with ProfAppInd } } -object ProfIODiagnosticMetricsView extends ViewableTrait[IODiagnosticProfileResult] with ProfAppIndexMapperTrait { +object ProfIODiagnosticMetricsView extends ViewableTrait[IODiagnosticProfileResult] + with ProfAppIndexMapperTrait { override def getLabel: String = "IO Diagnostic Metrics" override def getDescription: String = "IO Diagnostic Metrics" diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala index 7ff03a943..fb7fbab05 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala @@ -841,7 +841,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { f.endsWith(".csv") }) // compare the number of files generated - assert(dotDirs.length === 21) + assert(dotDirs.length === 22) for (file <- dotDirs) { assert(file.getAbsolutePath.endsWith(".csv")) // just load each one to make sure formatted properly @@ -875,7 +875,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { f.endsWith(".csv") }) // compare the number of files generated - assert(dotDirs.length === 17) + assert(dotDirs.length === 18) for (file <- dotDirs) { assert(file.getAbsolutePath.endsWith(".csv")) // just load each one to make sure formatted properly @@ -912,7 +912,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { f.endsWith(".csv") }) // compare the number of files generated - assert(dotDirs.length === 21) + assert(dotDirs.length === 22) for (file <- dotDirs) { assert(file.getAbsolutePath.endsWith(".csv")) // just load each one to make sure formatted properly @@ -949,7 +949,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { f.endsWith(".csv") }) // compare the number of files generated - assert(dotDirs.length === 19) + assert(dotDirs.length === 20) for (file <- dotDirs) { assert(file.getAbsolutePath.endsWith(".csv")) // just load each one to make sure formatted properly From c0e44b2b83473d49890fad55dba7d892322480ff Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Thu, 5 Dec 2024 18:28:47 -0800 Subject: [PATCH 06/23] fix merge conflict Signed-off-by: cindyyuanjiang --- .../nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 51da10391..e762721a2 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -19,7 +19,6 @@ package com.nvidia.spark.rapids.tool.analysis import scala.collection.mutable.{AbstractSet, ArrayBuffer, HashMap, LinkedHashSet} import com.nvidia.spark.rapids.tool.analysis.IOAccumDiagnosticMetrics._ -import com.nvidia.spark.rapids.tool.planparser.SQLPlanParser import com.nvidia.spark.rapids.tool.profiling.{AccumProfileResults, IODiagnosticProfileResult, SQLAccumProfileResults, SQLMetricInfoCase, SQLStageInfoProfileResult, UnsupportedSQLPlan, WholeStageCodeGenResults} import com.nvidia.spark.rapids.tool.qualification.QualSQLPlanAnalyzer From 66c9762d8228384d48956f084f487a4aa0ea44e5 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Fri, 6 Dec 2024 15:34:32 -0800 Subject: [PATCH 07/23] add unit test and clean up Signed-off-by: cindyyuanjiang --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 8 +- .../tool/profiling/CollectInformation.scala | 2 +- .../profiling/DiagnosticSummaryInfo.scala | 2 +- .../profiling/ProfileClassWarehouse.scala | 52 ++++-- .../spark/rapids/tool/views/SQLView.scala | 8 +- ...entlog_iodiagnosticmetrics_expectation.csv | 16 ++ ...og_stagediagnosticmetrics_expectation.csv} | 0 .../rapids/tool/profiling/AnalysisSuite.scala | 163 +++++++++++++++--- 8 files changed, 210 insertions(+), 41 deletions(-) create mode 100644 core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_iodiagnosticmetrics_expectation.csv rename core/src/test/resources/ProfilingExpectations/{rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv => rapids_join_eventlog_stagediagnosticmetrics_expectation.csv} (100%) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index e762721a2..e2367cf2e 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -19,7 +19,7 @@ package com.nvidia.spark.rapids.tool.analysis import scala.collection.mutable.{AbstractSet, ArrayBuffer, HashMap, LinkedHashSet} import com.nvidia.spark.rapids.tool.analysis.IOAccumDiagnosticMetrics._ -import com.nvidia.spark.rapids.tool.profiling.{AccumProfileResults, IODiagnosticProfileResult, SQLAccumProfileResults, SQLMetricInfoCase, SQLStageInfoProfileResult, UnsupportedSQLPlan, WholeStageCodeGenResults} +import com.nvidia.spark.rapids.tool.profiling.{AccumProfileResults, IODiagnosticResult, SQLAccumProfileResults, SQLMetricInfoCase, SQLStageInfoProfileResult, UnsupportedSQLPlan, WholeStageCodeGenResults} import com.nvidia.spark.rapids.tool.qualification.QualSQLPlanAnalyzer import org.apache.spark.sql.execution.SparkPlanInfo @@ -357,9 +357,9 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap * Generate IO-related diagnostic metrics for the SQL plan: * output rows, scan time, output batches, buffer time, shuffle write time, fetch wait time, GPU * decode time. - * @return a sequence of IODiagnosticProfileResult + * @return a sequence of IODiagnosticResult */ - def generateIODiagnosticAccums(): Seq[IODiagnosticProfileResult] = { + def generateIODiagnosticAccums(): Seq[IODiagnosticResult] = { val zeroRecord = StatisticsMetrics.ZERO_RECORD IODiagnosticMetrics.toSeq.flatMap { case ((sqlId, nodeId, nodeName, stageIds), sqlAccums) => stageIds.split(",").filter(_.nonEmpty).map(_.toInt).flatMap { stageId => @@ -393,7 +393,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap if (stageDiagnosticInfo.isEmpty) { None } else { - Some(IODiagnosticProfileResult( + Some(IODiagnosticResult( appIndex, app.getAppName, app.appId, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CollectInformation.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CollectInformation.scala index a4a507d8b..6467f7feb 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CollectInformation.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CollectInformation.scala @@ -95,7 +95,7 @@ class CollectInformation(apps: Seq[ApplicationInfo]) extends Logging { ProfStageMetricView.getRawView(apps) } - def getIODiagnosticMetrics: Seq[IODiagnosticProfileResult] = { + def getIODiagnosticMetrics: Seq[IODiagnosticResult] = { ProfIODiagnosticMetricsView.getRawView(apps) } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala index 79a2a9582..e714b123b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala @@ -22,5 +22,5 @@ package com.nvidia.spark.rapids.tool.profiling */ case class DiagnosticSummaryInfo( stageDiagnostics: Seq[StageDiagnosticResult], - IODiagnostics: Seq[IODiagnosticProfileResult] + IODiagnostics: Seq[IODiagnosticResult] ) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index f535672ff..eee89cb35 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -212,22 +212,54 @@ case class SQLAccumProfileResults( accumulatorId: Long, name: String, min: Long, - median:Long, + median: Long, max: Long, total: Long, metricType: String, stageIds: String) extends ProfileResult { - override val outputHeaders = Seq("appIndex", "sqlID", "nodeID", "nodeName", "accumulatorId", - "name", "min", "median", "max", "total", "metricType", "stageIds") + + override val outputHeaders = { + Seq("appIndex", + "sqlID", + "nodeID", + "nodeName", + "accumulatorId", + "name", + "min", + "median", + "max", + "total", + "metricType", + "stageIds") + } + override def convertToSeq: Seq[String] = { - Seq(appIndex.toString, sqlID.toString, nodeID.toString, nodeName, accumulatorId.toString, - name, min.toString, median.toString, max.toString, total.toString, metricType, stageIds) + Seq(appIndex.toString, + sqlID.toString, + nodeID.toString, + nodeName, + accumulatorId.toString, + name, + min.toString, + median.toString, + max.toString, + total.toString, + metricType, + stageIds) } + override def convertToCSVSeq: Seq[String] = { - Seq(appIndex.toString, sqlID.toString, nodeID.toString, - StringUtils.reformatCSVString(nodeName), accumulatorId.toString, - StringUtils.reformatCSVString(name), min.toString, median.toString, max.toString, - total.toString, StringUtils.reformatCSVString(metricType), + Seq(appIndex.toString, + sqlID.toString, + nodeID.toString, + StringUtils.reformatCSVString(nodeName), + accumulatorId.toString, + StringUtils.reformatCSVString(name), + min.toString, + median.toString, + max.toString, + total.toString, + StringUtils.reformatCSVString(metricType), StringUtils.reformatCSVString(stageIds)) } } @@ -890,7 +922,7 @@ case class SQLTaskAggMetricsProfileResult( } } -case class IODiagnosticProfileResult( +case class IODiagnosticResult( appIndex: Int, appName: String, appId: String, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala index 7c978d49e..154f820f7 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala @@ -17,7 +17,7 @@ package com.nvidia.spark.rapids.tool.views import com.nvidia.spark.rapids.tool.analysis.{AppSQLPlanAnalyzer, ProfAppIndexMapperTrait, QualAppIndexMapperTrait} -import com.nvidia.spark.rapids.tool.profiling.{IODiagnosticProfileResult, SQLAccumProfileResults, SQLCleanAndAlignIdsProfileResult, SQLPlanClassifier, WholeStageCodeGenResults} +import com.nvidia.spark.rapids.tool.profiling.{IODiagnosticResult, SQLAccumProfileResults, SQLCleanAndAlignIdsProfileResult, SQLPlanClassifier, WholeStageCodeGenResults} import org.apache.spark.sql.rapids.tool.AppBase import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo @@ -109,17 +109,17 @@ object ProfSQLPlanMetricsView extends AppSQLPlanMetricsViewTrait with ProfAppInd } } -object ProfIODiagnosticMetricsView extends ViewableTrait[IODiagnosticProfileResult] +object ProfIODiagnosticMetricsView extends ViewableTrait[IODiagnosticResult] with ProfAppIndexMapperTrait { override def getLabel: String = "IO Diagnostic Metrics" override def getDescription: String = "IO Diagnostic Metrics" override def sortView( - rows: Seq[IODiagnosticProfileResult]): Seq[IODiagnosticProfileResult] = { + rows: Seq[IODiagnosticResult]): Seq[IODiagnosticResult] = { rows.sortBy(cols => (cols.appIndex, cols.duration, cols.stageId, cols.sqlId, cols.nodeId)) } - override def getRawView(app: AppBase, index: Int): Seq[IODiagnosticProfileResult] = { + override def getRawView(app: AppBase, index: Int): Seq[IODiagnosticResult] = { app match { case app: ApplicationInfo => sortView(app.planMetricProcessor.generateIODiagnosticAccums()) diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_iodiagnosticmetrics_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_iodiagnosticmetrics_expectation.csv new file mode 100644 index 000000000..094b8b2fb --- /dev/null +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_iodiagnosticmetrics_expectation.csv @@ -0,0 +1,16 @@ +appIndex,appName,appId,sqlId,stageId,stageDurationMs,nodeId,nodeName,outputRowsMin,outputRowsMedian,outputRowsMax,outputRowsTotal,scanTimeMin,scanTimeMedian,scanTimeMax,scanTimeTotal,outputBatchesMin,outputBatchesMedian,outputBatchesMax,outputBatchesTotal,buffeTimeMin,buffeTimeMedian,buffeTimeMax,buffeTimeTotal,shuffleWriteTimeMin,shuffleWriteTimeMedian,shuffleWriteTimeMax,shuffleWriteTimeTotal,fetchWaitTimeMin,fetchWaitTimeMedian,fetchWaitTimeMax,fetchWaitTimeTotal,gpuDecodeTimeMin,gpuDecodeTimeMedian,gpuDecodeTimeMax,gpuDecodeTimeTotal +1,Spark shell,local-1622814619968,0,3,83,1,"GpuHashAggregate",1,1,1,1,0,0,0,0,1,1,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,3,83,2,"GpuShuffleCoalesce",200,200,200,200,0,0,0,0,1,1,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,3,83,3,"GpuColumnarExchange",0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,3,"GpuColumnarExchange",1,1,1,200,0,0,0,0,1,1,1,200,0,0,0,0,139875,230038,9747416,93193331,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,4,"GpuHashAggregate",1,1,1,200,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,6,"GpuShuffledHashJoin",49480,50017,50559,10000000,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,7,"GpuShuffleCoalesce",49480,50017,50559,10000000,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,8,"GpuColumnarExchange",0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,14,"GpuCoalesceBatches",49480,50017,50559,10000000,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,15,"GpuShuffleCoalesce",49480,50017,50559,10000000,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,16,"GpuColumnarExchange",0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,1,1631,8,"GpuColumnarExchange",1666666,1666667,1666667,10000000,0,0,0,0,200,200,200,1200,0,0,0,0,37444140,92128351,108992798,508750471,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,1,1631,13,"Scan",1666666,1666667,1666667,10000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,0,1743,16,"GpuColumnarExchange",1666666,1666667,1666667,10000000,0,0,0,0,200,200,200,1200,0,0,0,0,41434653,60830365,100858775,400284505,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,0,1743,21,"Scan",1666666,1666667,1666667,10000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagediagnosticmetrics_expectation.csv similarity index 100% rename from core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv rename to core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagediagnosticmetrics_expectation.csv diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala index 2b8c3bf12..3a18f3e10 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala @@ -67,30 +67,135 @@ case class TestStageDiagnosticResult( gpuSemaphoreWaitSum: Long, nodeNames: Seq[String]) +case class TestIODiagnosticResult( + appIndex: Int, + appName: String, + appId: String, + sqlId: Long, + stageId: Long, + duration: Long, + nodeId: Long, + nodeName: String, + outputRowsMin: Long, + outputRowsMed: Long, + outputRowsMax: Long, + outputRowsSum: Long, + scanTimeMin: Long, + scanTimeMed: Long, + scanTimeMax: Long, + scanTimeSum: Long, + outputBatchesMin: Long, + outputBatchesMed: Long, + outputBatchesMax: Long, + outputBatchesSum: Long, + buffeTimeMin: Long, + buffeTimeMed: Long, + buffeTimeMax: Long, + buffeTimeSum: Long, + shuffleWriteTimeMin: Long, + shuffleWriteTimeMed: Long, + shuffleWriteTimeMax: Long, + shuffleWriteTimeSum: Long, + fetchWaitTimeMin: Long, + fetchWaitTimeMed: Long, + fetchWaitTimeMax: Long, + fetchWaitTimeSum: Long, + gpuDecodeTimeMin: Long, + gpuDecodeTimeMed: Long, + gpuDecodeTimeMax: Long, + gpuDecodeTimeSum: Long) + class AnalysisSuite extends FunSuite { private def createTestStageDiagnosticResult(diagnosticsResults: Seq[StageDiagnosticResult]): Seq[TestStageDiagnosticResult] = { def bytesToMB(numBytes: Long): Long = numBytes / (1024 * 1024) def nanoToMilliSec(numNano: Long): Long = numNano / 1000000 + diagnosticsResults.map { result => + TestStageDiagnosticResult( + result.appIndex, + result.appName, + result.appId, + result.stageId, + result.duration, + result.numTasks, + bytesToMB(result.memoryBytesSpilled.min), + bytesToMB(result.memoryBytesSpilled.median), + bytesToMB(result.memoryBytesSpilled.max), + bytesToMB(result.memoryBytesSpilled.total), + bytesToMB(result.diskBytesSpilled.min), + bytesToMB(result.diskBytesSpilled.median), + bytesToMB(result.diskBytesSpilled.max), + bytesToMB(result.diskBytesSpilled.total), + result.inputBytesRead.min, + result.inputBytesRead.median, + result.inputBytesRead.max, + result.inputBytesRead.total, + result.outputBytesWritten.min, + result.outputBytesWritten.median, + result.outputBytesWritten.max, + result.outputBytesWritten.total, + result.srTotalBytesReadMin, + result.srTotalBytesReadMed, + result.srTotalBytesReadMax, + result.srTotalBytesReadSum, + result.swBytesWritten.min, + result.swBytesWritten.median, + result.swBytesWritten.max, + result.swBytesWritten.total, + nanoToMilliSec(result.srFetchWaitTime.min), + nanoToMilliSec(result.srFetchWaitTime.median), + nanoToMilliSec(result.srFetchWaitTime.max), + nanoToMilliSec(result.srFetchWaitTime.total), + nanoToMilliSec(result.swWriteTime.min), + nanoToMilliSec(result.swWriteTime.median), + nanoToMilliSec(result.swWriteTime.max), + nanoToMilliSec(result.swWriteTime.total), + result.gpuSemaphoreWait.total, + result.nodeNames) + } + } + + private def createTestIODiagnosticResult(diagnosticsResults: Seq[IODiagnosticResult]): + Seq[TestIODiagnosticResult] = { diagnosticsResults.map {result => - TestStageDiagnosticResult(result.appIndex, result.appName, result.appId, result.stageId, - result.duration, result.numTasks, bytesToMB(result.memoryBytesSpilled.min), - bytesToMB(result.memoryBytesSpilled.median), bytesToMB(result.memoryBytesSpilled.max), - bytesToMB(result.memoryBytesSpilled.total), bytesToMB(result.diskBytesSpilled.min), - bytesToMB(result.diskBytesSpilled.median), bytesToMB(result.diskBytesSpilled.max), - bytesToMB(result.diskBytesSpilled.total), result.inputBytesRead.min, - result.inputBytesRead.median, result.inputBytesRead.max, result.inputBytesRead.total, - result.outputBytesWritten.min, result.outputBytesWritten.median, - result.outputBytesWritten.max, result.outputBytesWritten.total, - result.srTotalBytesReadMin, result.srTotalBytesReadMed, result.srTotalBytesReadMax, - result.srTotalBytesReadSum, result.swBytesWritten.min, result.swBytesWritten.median, - result.swBytesWritten.max, result.swBytesWritten.total, - nanoToMilliSec(result.srFetchWaitTime.min), nanoToMilliSec(result.srFetchWaitTime.median), - nanoToMilliSec(result.srFetchWaitTime.max), nanoToMilliSec(result.srFetchWaitTime.total), - nanoToMilliSec(result.swWriteTime.min), nanoToMilliSec(result.swWriteTime.median), - nanoToMilliSec(result.swWriteTime.max), nanoToMilliSec(result.swWriteTime.total), - result.gpuSemaphoreWait.total, result.nodeNames) + TestIODiagnosticResult( + result.appIndex, + result.appName, + result.appId, + result.sqlId, + result.stageId, + result.duration, + result.nodeId, + result.nodeName, + result.outputRows.min, + result.outputRows.med, + result.outputRows.max, + result.outputRows.total, + result.scanTime.min, + result.scanTime.med, + result.scanTime.max, + result.scanTime.total, + result.outputBatches.min, + result.outputBatches.med, + result.outputBatches.max, + result.outputBatches.total, + result.buffeTime.min, + result.buffeTime.med, + result.buffeTime.max, + result.buffeTime.total, + result.shuffleWriteTime.min, + result.shuffleWriteTime.med, + result.shuffleWriteTime.max, + result.shuffleWriteTime.total, + result.fetchWaitTime.min, + result.fetchWaitTime.med, + result.fetchWaitTime.max, + result.fetchWaitTime.total, + result.gpuDecodeTime.min, + result.gpuDecodeTime.med, + result.gpuDecodeTime.max, + result.gpuDecodeTime.total) } } @@ -142,8 +247,8 @@ class AnalysisSuite extends FunSuite { expectFile("sql"), expectFile("job"), expectFile("stage")) } - test("test stage-level diagnostic aggregation simple") { - val expectFile = "rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv" + test("test stage-level diagnostic metrics") { + val expectFile = "rapids_join_eventlog_stagediagnosticmetrics_expectation.csv" val logs = Array(s"$logDir/rapids_join_eventlog.zstd") val apps = ToolTestUtils.processProfileApps(logs, sparkSession) assert(apps.size == logs.size) @@ -154,14 +259,30 @@ class AnalysisSuite extends FunSuite { collect.getSQLToStage collect.getStageLevelMetrics - val aggResults = RawMetricProfilerView.getAggMetrics(apps) + val diagnosticResults = RawMetricProfilerView.getAggMetrics(apps) import org.apache.spark.sql.functions._ import sparkSession.implicits._ - val actualDf = createTestStageDiagnosticResult(aggResults.stageDiagnostics).toDF. + val actualDf = createTestStageDiagnosticResult(diagnosticResults.stageDiagnostics).toDF. withColumn("nodeNames", concat_ws(",", col("nodeNames"))) compareMetrics(actualDf, expectFile) } + test("test IO diagnostic metrics") { + val expectFile = "rapids_join_eventlog_iodiagnosticmetrics_expectation.csv" + val logs = Array(s"$logDir/rapids_join_eventlog.zstd") + val apps = ToolTestUtils.processProfileApps(logs, sparkSession) + assert(apps.size == logs.size) + + val collect = new CollectInformation(apps) + // Computes IO diagnostic metrics mapping which is later used in getIODiagnosticMetrics + collect.getSQLPlanMetrics + val diagnosticResults = collect.getIODiagnosticMetrics + + import sparkSession.implicits._ + val actualDf = createTestIODiagnosticResult(diagnosticResults).toDF + compareMetrics(actualDf, expectFile) + } + private def testSqlMetricsAggregation(logs: Array[String], expectFileSQL: String, expectFileJob: String, expectFileStage: String): Unit = { val apps = ToolTestUtils.processProfileApps(logs, sparkSession) From 45712e2f83352551505594c5ef5245e712279a5e Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Mon, 9 Dec 2024 17:12:57 -0800 Subject: [PATCH 08/23] address review feedback Signed-off-by: cindyyuanjiang --- .../rapids/tool/analysis/AnalysisUtils.scala | 28 +++- .../tool/analysis/AppSQLPlanAnalyzer.scala | 154 +++++++++++------- .../profiling/ProfileClassWarehouse.scala | 44 ++++- 3 files changed, 150 insertions(+), 76 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala index 527174e81..35ffea283 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala @@ -45,16 +45,32 @@ object IOAccumDiagnosticMetrics { val GPU_DECODE_TIME_METRIC = "GPU decode time" /** - * Get all IO diagnostic metrics + * Get all IO diagnostic metrics names */ - def getAllIODiagnosticMetrics: Set[String] = Set(OUTPUT_ROWS_METRIC, - SCAN_TIME_METRIC, OUTPUT_BATCHES_METRIC, BUFFER_TIME_METRIC, - SHUFFLE_WRITE_TIME_METRIC, FETCH_WAIT_TIME_METRIC, GPU_DECODE_TIME_METRIC) + def getAllIODiagnosticMetrics: Set[String] = Set( + OUTPUT_ROWS_METRIC, + SCAN_TIME_METRIC, + OUTPUT_BATCHES_METRIC, + BUFFER_TIME_METRIC, + SHUFFLE_WRITE_TIME_METRIC, + FETCH_WAIT_TIME_METRIC, + GPU_DECODE_TIME_METRIC) /** - * Check if input is an IO diagnostic metric + * Check if a metric name belongs to IO diagnostic metrics */ - def isIODiagnosticMetric(metric: String): Boolean = { + def isIODiagnosticMetricName(metric: String): Boolean = { getAllIODiagnosticMetrics.contains(metric) || metric.contains(OUTPUT_ROWS_METRIC) } + + /** + * Normalize a metric name to its IO diagnostic metric constant + */ + def normalizeToIODiagnosticMetric(metric: String): String = { + if (metric.contains(OUTPUT_ROWS_METRIC)) { + OUTPUT_ROWS_METRIC + } else { + metric + } + } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index e2367cf2e..29bc6974e 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -58,39 +58,96 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap var allSQLMetrics: ArrayBuffer[SQLMetricInfoCase] = ArrayBuffer[SQLMetricInfoCase]() // A map between stage ID and a set of node names val stageToNodeNames: HashMap[Long, Seq[String]] = HashMap.empty[Long, Seq[String]] - // A map between stage ID and diagnostic metrics results (stored as a map between metric name - // and AccumProfileResults) + + // A mapping from stage ID to diagnostic metrics results. + // Each stage ID maps to another HashMap, where: + // - The key is the diagnostic metric name (String). + // - The value is an AccumProfileResults object containing the diagnostic data for that metric. val stageToDiagnosticMetrics: HashMap[Long, HashMap[String, AccumProfileResults]] = HashMap.empty[Long, HashMap[String, AccumProfileResults]] - // A map between (sql ID, node ID, node name, stage IDs) and a list of IO diagnostic metrics - // results (stored as an array of SQLAccumProfileResults) - we need all info in the key to - // uniquely identify each SQLAccumProfileResults - val IODiagnosticMetrics: - HashMap[(Long, Long, String, String), ArrayBuffer[SQLAccumProfileResults]] = - HashMap.empty[(Long, Long, String, String), ArrayBuffer[SQLAccumProfileResults]] + + // A mapping from a unique combination of SQL execution identifiers to a list of IO diagnostic + // metrics results. + // The key is a tuple consisting of: + // - sqlID (Long): The unique identifier for the SQL query. + // - nodeID (Long): The unique identifier for the node. + // - stageIDs (String): A comma-separated string representing the stage IDs involved. + // The value is an ArrayBuffer of SQLAccumProfileResults objects, storing the IO diagnostic + // metrics for the given key. + val IODiagnosticMetricsMap: HashMap[(Long, Long, String), ArrayBuffer[SQLAccumProfileResults]] = + HashMap.empty[(Long, Long, String), ArrayBuffer[SQLAccumProfileResults]] /** - * Given an input diagnostic metric result, update stageToDiagnosticMetrics mapping - * @param accum AccumProfileResults to be analyzed + * Updates the stageToDiagnosticMetrics mapping with the provided AccumProfileResults. + * @param accum AccumProfileResults instance containing diagnostic metrics to be added + * to stageToDiagnosticMetrics mapping. */ private def updateStageDiagnosticMetrics(accum: AccumProfileResults): Unit = { val stageId = accum.stageId + + // Initialize an empty mapping for the stage if it doesn't already exist if (!stageToDiagnosticMetrics.contains(stageId)) { stageToDiagnosticMetrics(stageId) = HashMap.empty[String, AccumProfileResults] } + stageToDiagnosticMetrics(stageId)(accum.accMetaRef.getName()) = accum } /** - * Given an input IO diagnostic metric result, update IODiagnosticMetrics mapping - * @param accum SQLAccumProfileResults to be analyzed + * Updates the IODiagnosticMetricsMap with the provided SQLAccumProfileResults. + * @param accum SQLAccumProfileResults instance containing IO diagnostics metrics + * to be added to IODiagnosticMetricsMap. + */ + private def updateIODiagnosticMetricsMap(accum: SQLAccumProfileResults): Unit = { + val key = (accum.sqlID, accum.nodeID, accum.stageIds) + + // Initialize an entry if the key does not exist + if (!IODiagnosticMetricsMap.contains(key)) { + IODiagnosticMetricsMap(key) = ArrayBuffer[SQLAccumProfileResults]() + } + + IODiagnosticMetricsMap(key) += accum + } + + /** + * Retrieves the set of task IDs associated with a specific stage. + * + * @param stageId The ID of the stage. + * @return A set of task IDs corresponding to the given stage ID. + */ + private def getStageTaskIds(stageId: Int): Set[Long] = { + app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet + } + + /** + * Computes statistical metrics (min, median, max, sum) for task updates + * in a given stage based on a provided set of task IDs. + * @param accumInfo AccumInfo object containing task update data. + * @param stageTaskIds Set of task IDs corresponding to the stage. + * @return Option containing a tuple of (min, median, max, sum) + * if there are task updates, or None if no updates exist. */ - private def updateIODiagnosticMetrics(accum: SQLAccumProfileResults): Unit = { - val key = (accum.sqlID, accum.nodeID, accum.nodeName, accum.stageIds) - if (!IODiagnosticMetrics.contains(key)) { - IODiagnosticMetrics(key) = ArrayBuffer[SQLAccumProfileResults]() + private def getAccumStatisticsInStage(accumInfo: AccumInfo, stageTaskIds: Set[Long]): + Option[(Long, Long, Long, Long)] = { + // Filter task updates to only include those matching the stage's task IDs + val filteredTaskUpdates = + accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted + + if (filteredTaskUpdates.isEmpty) { + None + } else { + val min = filteredTaskUpdates.head + val max = filteredTaskUpdates.last + val sum = filteredTaskUpdates.sum + val median = if (filteredTaskUpdates.size % 2 == 0) { + val mid = filteredTaskUpdates.size / 2 + (filteredTaskUpdates(mid) + filteredTaskUpdates(mid - 1)) / 2 + } else { + filteredTaskUpdates(filteredTaskUpdates.size / 2) + } + + Some((min, median, max, sum)) } - IODiagnosticMetrics(key) += accum } /** @@ -342,8 +399,8 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap metric.nodeID, metric.nodeName, metric.accumulatorId, metric.name, min, med, max, total, metric.metricType, metric.stageIds.mkString(",")) - if (isIODiagnosticMetric(metric.name)) { - updateIODiagnosticMetrics(sqlAccumProileResult) + if (isIODiagnosticMetricName(metric.name)) { + updateIODiagnosticMetricsMap(sqlAccumProileResult) } Some(sqlAccumProileResult) @@ -361,31 +418,20 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap */ def generateIODiagnosticAccums(): Seq[IODiagnosticResult] = { val zeroRecord = StatisticsMetrics.ZERO_RECORD - IODiagnosticMetrics.toSeq.flatMap { case ((sqlId, nodeId, nodeName, stageIds), sqlAccums) => + IODiagnosticMetricsMap.toSeq.flatMap { case ((sqlId, nodeId, stageIds), sqlAccums) => stageIds.split(",").filter(_.nonEmpty).map(_.toInt).flatMap { stageId => + val nodeName = sqlAccums.head.nodeName val stageDiagnosticInfo = HashMap.empty[String, StatisticsMetrics] + val stageTaskIds = getStageTaskIds(stageId) sqlAccums.foreach { sqlAccum => - val stageTaskIds = app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet val accumInfo = app.accumManager.accumInfoMap.getOrElse(sqlAccum.accumulatorId, new AccumInfo(AccumMetaRef(0L, AccumNameRef("")))) - val taskUpatesSubset = - accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted - if (taskUpatesSubset.nonEmpty) { - val min = taskUpatesSubset.head - val max = taskUpatesSubset.last - val sum = taskUpatesSubset.sum - val median = if (taskUpatesSubset.size % 2 == 0) { - val mid = taskUpatesSubset.size / 2 - (taskUpatesSubset(mid) + taskUpatesSubset(mid - 1)) / 2 - } else { - taskUpatesSubset(taskUpatesSubset.size / 2) - } - val metricName = if (sqlAccum.name.contains(OUTPUT_ROWS_METRIC)) { - OUTPUT_ROWS_METRIC - } else { - sqlAccum.name - } + val accumStatistics = getAccumStatisticsInStage(accumInfo, stageTaskIds) + + if (accumStatistics.nonEmpty) { + val (min, median, max, sum) = accumStatistics.get + val metricName = normalizeToIODiagnosticMetric(sqlAccum.name) stageDiagnosticInfo(metricName) = StatisticsMetrics(min, median, max, sum) } } @@ -428,36 +474,22 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap app.accumManager.accumInfoMap.flatMap { accumMapEntry => val accumInfo = accumMapEntry._2 accumInfo.stageValuesMap.keySet.flatMap( stageId => { - val stageTaskIds = app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet - // get the task updates that belong to that stage - val taskUpatesSubset = - accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted - if (taskUpatesSubset.isEmpty) { - None - } else { - val min = taskUpatesSubset.head - val max = taskUpatesSubset.last - val sum = taskUpatesSubset.sum - val median = if (taskUpatesSubset.size % 2 == 0) { - val mid = taskUpatesSubset.size / 2 - (taskUpatesSubset(mid) + taskUpatesSubset(mid - 1)) / 2 - } else { - taskUpatesSubset(taskUpatesSubset.size / 2) - } - // reuse AccumProfileResults to avoid generating extra memory from allocating new objects + getAccumStatisticsInStage(accumInfo, getStageTaskIds(stageId)).map( stats => { + val (min, median, max, sum) = stats + // create and reuse AccumProfileResults object to avoid generating extra memory val accumProfileResults = AccumProfileResults( appIndex, stageId, accumInfo.infoRef, - min = min, - median = median, - max = max, - total = sum) + min, + median, + max, + sum) if (accumInfo.infoRef.name.isDiagnosticMetrics()) { updateStageDiagnosticMetrics(accumProfileResults) } - Some(accumProfileResults) - } + accumProfileResults + }) }) } }.toSeq diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index eee89cb35..4f2f9d47c 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -264,20 +264,46 @@ case class SQLAccumProfileResults( } } -case class AccumProfileResults(appIndex: Int, stageId: Int, accMetaRef: AccumMetaRef, - min: Long, median: Long, max: Long, total: Long) extends ProfileResult { - override val outputHeaders = Seq("appIndex", "stageId", "accumulatorId", "name", "min", - "median", "max", "total") +case class AccumProfileResults( + appIndex: Int, + stageId: Int, + accMetaRef: AccumMetaRef, + min: Long, + median: Long, + max: Long, + total: Long) extends ProfileResult { + + override val outputHeaders = { + Seq("appIndex", + "stageId", + "accumulatorId", + "name", + "min", + "median", + "max", + "total") + } override def convertToSeq: Seq[String] = { - Seq(appIndex.toString, stageId.toString, accMetaRef.id.toString, accMetaRef.getName(), - min.toString, median.toString, max.toString, total.toString) + Seq(appIndex.toString, + stageId.toString, + accMetaRef.id.toString, + accMetaRef.getName(), + min.toString, + median.toString, + max.toString, + total.toString) } override def convertToCSVSeq: Seq[String] = { - Seq(appIndex.toString, stageId.toString, accMetaRef.id.toString, - accMetaRef.name.csvValue, min.toString, - median.toString, max.toString, total.toString) + Seq(appIndex.toString, + stageId.toString, + accMetaRef.id.toString, + accMetaRef.name.csvValue, + min.toString, + median.toString, + max.toString, + total.toString) } } From 3786fd992c8d10693ef0c2c2a6e1776a1e999c58 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Mon, 9 Dec 2024 17:54:01 -0800 Subject: [PATCH 09/23] add comments and rename variables/functions Signed-off-by: cindyyuanjiang --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 60 ++++++++++++------- 1 file changed, 38 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 29bc6974e..9ab12bc54 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -127,7 +127,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap * @return Option containing a tuple of (min, median, max, sum) * if there are task updates, or None if no updates exist. */ - private def getAccumStatisticsInStage(accumInfo: AccumInfo, stageTaskIds: Set[Long]): + private def getAccumInfoStatisticsInStage(accumInfo: AccumInfo, stageTaskIds: Set[Long]): Option[(Long, Long, Long, Long)] = { // Filter task updates to only include those matching the stage's task IDs val filteredTaskUpdates = @@ -411,32 +411,47 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap } /** - * Generate IO-related diagnostic metrics for the SQL plan: - * output rows, scan time, output batches, buffer time, shuffle write time, fetch wait time, GPU - * decode time. - * @return a sequence of IODiagnosticResult + * Generate IO-related diagnostic metrics for the SQL plan. Metrics include: + * - Output rows + * - Scan time + * - Output batches + * - Buffer time + * - Shuffle write time + * - Fetch wait time + * - GPU decode time + * + * @return A sequence of `IODiagnosticResult` objects containing diagnostic metrics. */ def generateIODiagnosticAccums(): Seq[IODiagnosticResult] = { val zeroRecord = StatisticsMetrics.ZERO_RECORD + + // Transform the diagnostic metrics map into a sequence of results IODiagnosticMetricsMap.toSeq.flatMap { case ((sqlId, nodeId, stageIds), sqlAccums) => + // Process each stage ID and compute diagnostic results stageIds.split(",").filter(_.nonEmpty).map(_.toInt).flatMap { stageId => val nodeName = sqlAccums.head.nodeName - val stageDiagnosticInfo = HashMap.empty[String, StatisticsMetrics] val stageTaskIds = getStageTaskIds(stageId) + // A mapping from metric name to its statistical results (min, median, max, sum) + val metricNameToStatistics = HashMap.empty[String, StatisticsMetrics] + // Iterate through each IO metric sqlAccums.foreach { sqlAccum => - val accumInfo = app.accumManager.accumInfoMap.getOrElse(sqlAccum.accumulatorId, - new AccumInfo(AccumMetaRef(0L, AccumNameRef("")))) - val accumStatistics = getAccumStatisticsInStage(accumInfo, stageTaskIds) - - if (accumStatistics.nonEmpty) { - val (min, median, max, sum) = accumStatistics.get + val accumInfo = app.accumManager.accumInfoMap.getOrElse( + sqlAccum.accumulatorId, + new AccumInfo(AccumMetaRef(0L, AccumNameRef(""))) + ) + // Compute the metric's statistics (min, median, max, sum) for the given stage + val accumInfoStatistics = getAccumInfoStatisticsInStage(accumInfo, stageTaskIds) + // If statistics are available, store the results + if (accumInfoStatistics.nonEmpty) { + val (min, median, max, sum) = accumInfoStatistics.get val metricName = normalizeToIODiagnosticMetric(sqlAccum.name) - stageDiagnosticInfo(metricName) = StatisticsMetrics(min, median, max, sum) + metricNameToStatistics(metricName) = StatisticsMetrics(min, median, max, sum) } } - if (stageDiagnosticInfo.isEmpty) { + if (metricNameToStatistics.isEmpty) { + // metricNameToStatistics is not updated - there is no IO metrics result for this stage None } else { Some(IODiagnosticResult( @@ -448,13 +463,13 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap app.stageManager.getDurationById(stageId), nodeId, nodeName, - stageDiagnosticInfo.getOrElse(OUTPUT_ROWS_METRIC, zeroRecord), - stageDiagnosticInfo.getOrElse(SCAN_TIME_METRIC, zeroRecord), - stageDiagnosticInfo.getOrElse(OUTPUT_BATCHES_METRIC, zeroRecord), - stageDiagnosticInfo.getOrElse(BUFFER_TIME_METRIC, zeroRecord), - stageDiagnosticInfo.getOrElse(SHUFFLE_WRITE_TIME_METRIC, zeroRecord), - stageDiagnosticInfo.getOrElse(FETCH_WAIT_TIME_METRIC, zeroRecord), - stageDiagnosticInfo.getOrElse(GPU_DECODE_TIME_METRIC, zeroRecord))) + metricNameToStatistics.getOrElse(OUTPUT_ROWS_METRIC, zeroRecord), + metricNameToStatistics.getOrElse(SCAN_TIME_METRIC, zeroRecord), + metricNameToStatistics.getOrElse(OUTPUT_BATCHES_METRIC, zeroRecord), + metricNameToStatistics.getOrElse(BUFFER_TIME_METRIC, zeroRecord), + metricNameToStatistics.getOrElse(SHUFFLE_WRITE_TIME_METRIC, zeroRecord), + metricNameToStatistics.getOrElse(FETCH_WAIT_TIME_METRIC, zeroRecord), + metricNameToStatistics.getOrElse(GPU_DECODE_TIME_METRIC, zeroRecord))) } } }.toSeq @@ -474,7 +489,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap app.accumManager.accumInfoMap.flatMap { accumMapEntry => val accumInfo = accumMapEntry._2 accumInfo.stageValuesMap.keySet.flatMap( stageId => { - getAccumStatisticsInStage(accumInfo, getStageTaskIds(stageId)).map( stats => { + getAccumInfoStatisticsInStage(accumInfo, getStageTaskIds(stageId)).map( stats => { val (min, median, max, sum) = stats // create and reuse AccumProfileResults object to avoid generating extra memory val accumProfileResults = AccumProfileResults( @@ -485,6 +500,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap median, max, sum) + // update stageToDiagnosticMetrics mapping if accumInfo is a diagnostic metric if (accumInfo.infoRef.name.isDiagnosticMetrics()) { updateStageDiagnosticMetrics(accumProfileResults) } From 5dabc282974b5c1eac99fc8a0c5db0cd4c286f2d Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Thu, 12 Dec 2024 16:52:17 -0800 Subject: [PATCH 10/23] address review feedback Signed-off-by: cindyyuanjiang --- .../rapids/tool/analysis/AnalysisUtils.scala | 4 +- .../tool/analysis/AppSQLPlanAnalyzer.scala | 66 +++++++++---------- .../profiling/ProfileClassWarehouse.scala | 52 +++++++++++---- .../rapids/tool/profiling/AnalysisSuite.scala | 16 ++--- 4 files changed, 83 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala index 35ffea283..e6fc24376 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala @@ -64,7 +64,9 @@ object IOAccumDiagnosticMetrics { } /** - * Normalize a metric name to its IO diagnostic metric constant + * Normalize a metric name to its IO diagnostic metric constant because we want to + * support variations in metric naming, e.g. "join output rows", "number of output rows" + * are different names for output rows metric. */ def normalizeToIODiagnosticMetric(metric: String): String = { if (metric.contains(OUTPUT_ROWS_METRIC)) { diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 9ab12bc54..069540d92 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -423,8 +423,6 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap * @return A sequence of `IODiagnosticResult` objects containing diagnostic metrics. */ def generateIODiagnosticAccums(): Seq[IODiagnosticResult] = { - val zeroRecord = StatisticsMetrics.ZERO_RECORD - // Transform the diagnostic metrics map into a sequence of results IODiagnosticMetricsMap.toSeq.flatMap { case ((sqlId, nodeId, stageIds), sqlAccums) => // Process each stage ID and compute diagnostic results @@ -432,7 +430,8 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap val nodeName = sqlAccums.head.nodeName val stageTaskIds = getStageTaskIds(stageId) // A mapping from metric name to its statistical results (min, median, max, sum) - val metricNameToStatistics = HashMap.empty[String, StatisticsMetrics] + val metricNameToStatistics = HashMap.empty[String, StatisticsMetrics]. + withDefaultValue(StatisticsMetrics.ZERO_RECORD) // Iterate through each IO metric sqlAccums.foreach { sqlAccum => @@ -440,14 +439,14 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap sqlAccum.accumulatorId, new AccumInfo(AccumMetaRef(0L, AccumNameRef(""))) ) - // Compute the metric's statistics (min, median, max, sum) for the given stage - val accumInfoStatistics = getAccumInfoStatisticsInStage(accumInfo, stageTaskIds) - // If statistics are available, store the results - if (accumInfoStatistics.nonEmpty) { - val (min, median, max, sum) = accumInfoStatistics.get - val metricName = normalizeToIODiagnosticMetric(sqlAccum.name) - metricNameToStatistics(metricName) = StatisticsMetrics(min, median, max, sum) - } + // Compute the metric's statistics (min, median, max, sum) for the given stage. + // Store the results if available. + getAccumInfoStatisticsInStage(accumInfo, stageTaskIds).map( { + case (min, median, max, sum) => { + val metricName = normalizeToIODiagnosticMetric(sqlAccum.name) + metricNameToStatistics(metricName) = StatisticsMetrics(min, median, max, sum) + } + }) } if (metricNameToStatistics.isEmpty) { @@ -463,13 +462,13 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap app.stageManager.getDurationById(stageId), nodeId, nodeName, - metricNameToStatistics.getOrElse(OUTPUT_ROWS_METRIC, zeroRecord), - metricNameToStatistics.getOrElse(SCAN_TIME_METRIC, zeroRecord), - metricNameToStatistics.getOrElse(OUTPUT_BATCHES_METRIC, zeroRecord), - metricNameToStatistics.getOrElse(BUFFER_TIME_METRIC, zeroRecord), - metricNameToStatistics.getOrElse(SHUFFLE_WRITE_TIME_METRIC, zeroRecord), - metricNameToStatistics.getOrElse(FETCH_WAIT_TIME_METRIC, zeroRecord), - metricNameToStatistics.getOrElse(GPU_DECODE_TIME_METRIC, zeroRecord))) + metricNameToStatistics(OUTPUT_ROWS_METRIC), + metricNameToStatistics(SCAN_TIME_METRIC), + metricNameToStatistics(OUTPUT_BATCHES_METRIC), + metricNameToStatistics(BUFFER_TIME_METRIC), + metricNameToStatistics(SHUFFLE_WRITE_TIME_METRIC), + metricNameToStatistics(FETCH_WAIT_TIME_METRIC), + metricNameToStatistics(GPU_DECODE_TIME_METRIC))) } } }.toSeq @@ -489,22 +488,23 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap app.accumManager.accumInfoMap.flatMap { accumMapEntry => val accumInfo = accumMapEntry._2 accumInfo.stageValuesMap.keySet.flatMap( stageId => { - getAccumInfoStatisticsInStage(accumInfo, getStageTaskIds(stageId)).map( stats => { - val (min, median, max, sum) = stats - // create and reuse AccumProfileResults object to avoid generating extra memory - val accumProfileResults = AccumProfileResults( - appIndex, - stageId, - accumInfo.infoRef, - min, - median, - max, - sum) - // update stageToDiagnosticMetrics mapping if accumInfo is a diagnostic metric - if (accumInfo.infoRef.name.isDiagnosticMetrics()) { - updateStageDiagnosticMetrics(accumProfileResults) + getAccumInfoStatisticsInStage(accumInfo, getStageTaskIds(stageId)).map( { + case (min, median, max, sum) => { + // create and reuse AccumProfileResults object to avoid generating extra memory + val accumProfileResults = AccumProfileResults( + appIndex, + stageId, + accumInfo.infoRef, + min, + median, + max, + sum) + // update stageToDiagnosticMetrics mapping if accumInfo is a diagnostic metric + if (accumInfo.infoRef.name.isDiagnosticMetrics()) { + updateStageDiagnosticMetrics(accumProfileResults) + } + accumProfileResults } - accumProfileResults }) }) } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index 4f2f9d47c..6f06fca98 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -643,6 +643,20 @@ case class StageAggTaskMetricsProfileResult( override def idHeader = "stageId" } +/** + * Represents diagnostic metrics results at task/stage level in a Spark SQL execution plan. + * Output file: stage_level_diagnostic_metrics.csv. + * Collected metrics include: + * - Memory spilled (MB) + * - Disk spilled (MB) + * - Input bytes read + * - Output bytes written + * - Shuffle read total bytes (remote + local) + * - Shuffle write bytes + * - Shuffle read fetch wait time (ms) + * - Shuffle write time (ms) + * - GPU semaphore time (ns) + */ case class StageDiagnosticResult( appIndex: Int, appName: String, @@ -948,6 +962,18 @@ case class SQLTaskAggMetricsProfileResult( } } +/** + * Represents IO-related diagnostic metrics results in a Spark SQL execution plan. + * Output file: io_diagnostic_metrics.csv. + * Collected metrics include: + * - Output rows + * - Scan time (ns) + * - Output batches + * - Buffer time (ns) + * - Shuffle write time (ns) + * - Fetch wait time (ns) + * - GPU decode time (ns) + */ case class IODiagnosticResult( appIndex: Int, appName: String, @@ -960,7 +986,7 @@ case class IODiagnosticResult( outputRows: StatisticsMetrics, scanTime: StatisticsMetrics, outputBatches: StatisticsMetrics, - buffeTime: StatisticsMetrics, + bufferTime: StatisticsMetrics, shuffleWriteTime: StatisticsMetrics, fetchWaitTime: StatisticsMetrics, gpuDecodeTime: StatisticsMetrics) extends ProfileResult { @@ -986,10 +1012,10 @@ case class IODiagnosticResult( "outputBatchesMedian", "outputBatchesMax", "outputBatchesTotal", - "buffeTimeMin", - "buffeTimeMedian", - "buffeTimeMax", - "buffeTimeTotal", + "bufferTimeMin", + "bufferTimeMedian", + "bufferTimeMax", + "bufferTimeTotal", "shuffleWriteTimeMin", "shuffleWriteTimeMedian", "shuffleWriteTimeMax", @@ -1025,10 +1051,10 @@ case class IODiagnosticResult( outputBatches.med.toString, outputBatches.max.toString, outputBatches.total.toString, - buffeTime.min.toString, - buffeTime.med.toString, - buffeTime.max.toString, - buffeTime.total.toString, + bufferTime.min.toString, + bufferTime.med.toString, + bufferTime.max.toString, + bufferTime.total.toString, shuffleWriteTime.min.toString, shuffleWriteTime.med.toString, shuffleWriteTime.max.toString, @@ -1064,10 +1090,10 @@ case class IODiagnosticResult( outputBatches.med.toString, outputBatches.max.toString, outputBatches.total.toString, - buffeTime.min.toString, - buffeTime.med.toString, - buffeTime.max.toString, - buffeTime.total.toString, + bufferTime.min.toString, + bufferTime.med.toString, + bufferTime.max.toString, + bufferTime.total.toString, shuffleWriteTime.min.toString, shuffleWriteTime.med.toString, shuffleWriteTime.max.toString, diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala index 3a18f3e10..17b0623c0 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala @@ -88,10 +88,10 @@ case class TestIODiagnosticResult( outputBatchesMed: Long, outputBatchesMax: Long, outputBatchesSum: Long, - buffeTimeMin: Long, - buffeTimeMed: Long, - buffeTimeMax: Long, - buffeTimeSum: Long, + bufferTimeMin: Long, + bufferTimeMed: Long, + bufferTimeMax: Long, + bufferTimeSum: Long, shuffleWriteTimeMin: Long, shuffleWriteTimeMed: Long, shuffleWriteTimeMax: Long, @@ -180,10 +180,10 @@ class AnalysisSuite extends FunSuite { result.outputBatches.med, result.outputBatches.max, result.outputBatches.total, - result.buffeTime.min, - result.buffeTime.med, - result.buffeTime.max, - result.buffeTime.total, + result.bufferTime.min, + result.bufferTime.med, + result.bufferTime.max, + result.bufferTime.total, result.shuffleWriteTime.min, result.shuffleWriteTime.med, result.shuffleWriteTime.max, From f668dda7bbab3351cca6e0d0be233cb9111b23bb Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Mon, 16 Dec 2024 12:46:08 -0800 Subject: [PATCH 11/23] new output batches name Signed-off-by: cindyyuanjiang --- .../com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala index e6fc24376..0777a0096 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala @@ -38,7 +38,7 @@ object StageAccumDiagnosticMetrics { object IOAccumDiagnosticMetrics { val OUTPUT_ROWS_METRIC = "output rows" // other names: join output rows, number of output rows val SCAN_TIME_METRIC = "scan time" - val OUTPUT_BATCHES_METRIC = "output columnar batches" + val OUTPUT_BATCHES_METRIC = "output columnar batches" // other names: number of output batches val BUFFER_TIME_METRIC = "buffer time" val SHUFFLE_WRITE_TIME_METRIC = "shuffle write time" val FETCH_WAIT_TIME_METRIC = "fetch wait time" From e9c950821efaa061f448520955c28f824e95bfc2 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Tue, 17 Dec 2024 23:13:33 -0800 Subject: [PATCH 12/23] refactor io diagnostic metrics Signed-off-by: cindyyuanjiang --- .../rapids/tool/analysis/AnalysisUtils.scala | 84 +++++++++++++------ .../tool/analysis/AppSQLPlanAnalyzer.scala | 18 ++-- .../sql/rapids/tool/store/AccumNameRef.scala | 4 +- 3 files changed, 70 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala index 0777a0096..176d79c15 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala @@ -27,40 +27,75 @@ object StageAccumDiagnosticMetrics { val GPU_SEMAPHORE_WAIT_METRIC = "gpuSemaphoreWait" /** - * Get all diagnostic metrics + * Set of all diagnostic metrics */ - def getAllDiagnosticMetrics: Set[String] = Set(MEMORY_SPILLED_METRIC, + lazy val allDiagnosticMetrics: Set[String] = Set(MEMORY_SPILLED_METRIC, DISK_SPILLED_METRIC, INPUT_BYTES_READ_METRIC, OUTPUT_BYTES_WRITTEN_METRIC, SW_TOTAL_BYTES_METRIC, SR_FETCH_WAIT_TIME_METRIC, SW_WRITE_TIME_METRIC, GPU_SEMAPHORE_WAIT_METRIC) } object IOAccumDiagnosticMetrics { - val OUTPUT_ROWS_METRIC = "output rows" // other names: join output rows, number of output rows - val SCAN_TIME_METRIC = "scan time" - val OUTPUT_BATCHES_METRIC = "output columnar batches" // other names: number of output batches - val BUFFER_TIME_METRIC = "buffer time" - val SHUFFLE_WRITE_TIME_METRIC = "shuffle write time" - val FETCH_WAIT_TIME_METRIC = "fetch wait time" - val GPU_DECODE_TIME_METRIC = "GPU decode time" + // Metric keys to support variations in metric naming + val OUTPUT_ROWS_METRIC_KEY = "output rows" + val SCAN_TIME_METRIC_KEY = "scan time" + val OUTPUT_BATCHES_METRIC_KEY = "output batches" + val BUFFER_TIME_METRIC_KEY = "buffer time" + val SHUFFLE_WRITE_TIME_METRIC_KEY = "shuffle write time" + val FETCH_WAIT_TIME_METRIC_KEY = "fetch wait time" + val GPU_DECODE_TIME_METRIC_KEY = "GPU decode time" + + val OUTPUT_ROW_METRIC_NAMES = Set( + "number of output rows", // common across all Spark eventlogs + "output rows", // only in GPU eventlogs + "join output rows" // only in GPU eventlogs + ) + + val SCAN_TIME_METRIC_NAMES = Set( + "scan time" // common across all Spark eventlogs + ) + + val OUTPUT_BATCHES_METRIC_NAMES = Set( + "number of output batches", // only in Photon eventlogs + "output columnar batches" // only in GPU eventlogs + ) + + val BUFFER_TIME_METRIC_NAMES = Set( + "buffer time" // common across all Spark eventlogs + ) + + val SHUFFLE_WRITE_TIME_METRIC_NAMES = Set( + "shuffle write time", // common across all Spark eventlogs + "rs. shuffle write time" // only in GPU eventlogs + ) + + val FETCH_WAIT_TIME_METRIC_NAMES = Set( + "fetch wait time" // common across all Spark eventlogs + ) + + val GPU_DECODE_TIME_METRIC_NAMES = Set( + "GPU decode time" // only in GPU eventlogs + ) + + private val metricNamesToKeyMap: Map[Set[String], String] = Map( + OUTPUT_ROW_METRIC_NAMES -> OUTPUT_ROWS_METRIC_KEY, + SCAN_TIME_METRIC_NAMES -> SCAN_TIME_METRIC_KEY, + OUTPUT_BATCHES_METRIC_NAMES -> OUTPUT_BATCHES_METRIC_KEY, + BUFFER_TIME_METRIC_NAMES -> BUFFER_TIME_METRIC_KEY, + SHUFFLE_WRITE_TIME_METRIC_NAMES -> SHUFFLE_WRITE_TIME_METRIC_KEY, + FETCH_WAIT_TIME_METRIC_NAMES -> FETCH_WAIT_TIME_METRIC_KEY, + GPU_DECODE_TIME_METRIC_NAMES -> GPU_DECODE_TIME_METRIC_KEY) /** - * Get all IO diagnostic metrics names + * Set of all IO diagnostic metrics names */ - def getAllIODiagnosticMetrics: Set[String] = Set( - OUTPUT_ROWS_METRIC, - SCAN_TIME_METRIC, - OUTPUT_BATCHES_METRIC, - BUFFER_TIME_METRIC, - SHUFFLE_WRITE_TIME_METRIC, - FETCH_WAIT_TIME_METRIC, - GPU_DECODE_TIME_METRIC) + lazy val allIODiagnosticMetrics: Set[String] = metricNamesToKeyMap.keys.flatten.toSet /** * Check if a metric name belongs to IO diagnostic metrics */ def isIODiagnosticMetricName(metric: String): Boolean = { - getAllIODiagnosticMetrics.contains(metric) || metric.contains(OUTPUT_ROWS_METRIC) + allIODiagnosticMetrics.contains(metric) } /** @@ -68,11 +103,10 @@ object IOAccumDiagnosticMetrics { * support variations in metric naming, e.g. "join output rows", "number of output rows" * are different names for output rows metric. */ - def normalizeToIODiagnosticMetric(metric: String): String = { - if (metric.contains(OUTPUT_ROWS_METRIC)) { - OUTPUT_ROWS_METRIC - } else { - metric - } + def normalizeToIODiagnosticMetricKey(metric: String): String = { + // input metric is already known to be an IO diagnostic metric + metricNamesToKeyMap.collectFirst { + case (names, key) if names.contains(metric) => key + }.get } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 069540d92..95ddd026a 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -443,8 +443,8 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap // Store the results if available. getAccumInfoStatisticsInStage(accumInfo, stageTaskIds).map( { case (min, median, max, sum) => { - val metricName = normalizeToIODiagnosticMetric(sqlAccum.name) - metricNameToStatistics(metricName) = StatisticsMetrics(min, median, max, sum) + val metricKey = normalizeToIODiagnosticMetricKey(sqlAccum.name) + metricNameToStatistics(metricKey) = StatisticsMetrics(min, median, max, sum) } }) } @@ -462,13 +462,13 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap app.stageManager.getDurationById(stageId), nodeId, nodeName, - metricNameToStatistics(OUTPUT_ROWS_METRIC), - metricNameToStatistics(SCAN_TIME_METRIC), - metricNameToStatistics(OUTPUT_BATCHES_METRIC), - metricNameToStatistics(BUFFER_TIME_METRIC), - metricNameToStatistics(SHUFFLE_WRITE_TIME_METRIC), - metricNameToStatistics(FETCH_WAIT_TIME_METRIC), - metricNameToStatistics(GPU_DECODE_TIME_METRIC))) + metricNameToStatistics(OUTPUT_ROWS_METRIC_KEY), + metricNameToStatistics(SCAN_TIME_METRIC_KEY), + metricNameToStatistics(OUTPUT_BATCHES_METRIC_KEY), + metricNameToStatistics(BUFFER_TIME_METRIC_KEY), + metricNameToStatistics(SHUFFLE_WRITE_TIME_METRIC_KEY), + metricNameToStatistics(FETCH_WAIT_TIME_METRIC_KEY), + metricNameToStatistics(GPU_DECODE_TIME_METRIC_KEY))) } } }.toSeq diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala index 0172f5229..5969912ed 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.rapids.tool.store import java.util.concurrent.ConcurrentHashMap -import com.nvidia.spark.rapids.tool.analysis.StageAccumDiagnosticMetrics.getAllDiagnosticMetrics +import com.nvidia.spark.rapids.tool.analysis.StageAccumDiagnosticMetrics.allDiagnosticMetrics import org.apache.spark.sql.rapids.tool.util.EventUtils.normalizeMetricName import org.apache.spark.sql.rapids.tool.util.StringUtils @@ -36,7 +36,7 @@ case class AccumNameRef(value: String) { // create a new CSV string even though they represent the same AccumulatorName. val csvValue: String = StringUtils.reformatCSVString(value) - def isDiagnosticMetrics(): Boolean = getAllDiagnosticMetrics.contains(value) + def isDiagnosticMetrics(): Boolean = allDiagnosticMetrics.contains(value) } object AccumNameRef { From 0bd1e7fb019f182decd2a3faabf49ab910e89398 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Wed, 18 Dec 2024 14:50:49 -0800 Subject: [PATCH 13/23] remove function getAccumInfoStatisticsInStage Signed-off-by: cindyyuanjiang --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 98 +++++++++---------- 1 file changed, 45 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 95ddd026a..8cddf7f50 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -119,37 +119,6 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet } - /** - * Computes statistical metrics (min, median, max, sum) for task updates - * in a given stage based on a provided set of task IDs. - * @param accumInfo AccumInfo object containing task update data. - * @param stageTaskIds Set of task IDs corresponding to the stage. - * @return Option containing a tuple of (min, median, max, sum) - * if there are task updates, or None if no updates exist. - */ - private def getAccumInfoStatisticsInStage(accumInfo: AccumInfo, stageTaskIds: Set[Long]): - Option[(Long, Long, Long, Long)] = { - // Filter task updates to only include those matching the stage's task IDs - val filteredTaskUpdates = - accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted - - if (filteredTaskUpdates.isEmpty) { - None - } else { - val min = filteredTaskUpdates.head - val max = filteredTaskUpdates.last - val sum = filteredTaskUpdates.sum - val median = if (filteredTaskUpdates.size % 2 == 0) { - val mid = filteredTaskUpdates.size / 2 - (filteredTaskUpdates(mid) + filteredTaskUpdates(mid - 1)) / 2 - } else { - filteredTaskUpdates(filteredTaskUpdates.size / 2) - } - - Some((min, median, max, sum)) - } - } - /** * Connects Operators to Stages using AccumulatorIDs. * TODO: This function can be fused in the visitNode function to avoid the extra iteration. @@ -441,12 +410,22 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap ) // Compute the metric's statistics (min, median, max, sum) for the given stage. // Store the results if available. - getAccumInfoStatisticsInStage(accumInfo, stageTaskIds).map( { - case (min, median, max, sum) => { - val metricKey = normalizeToIODiagnosticMetricKey(sqlAccum.name) - metricNameToStatistics(metricKey) = StatisticsMetrics(min, median, max, sum) + val filteredTaskUpdates = + accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted + if (filteredTaskUpdates.nonEmpty) { + val min = filteredTaskUpdates.head + val max = filteredTaskUpdates.last + val sum = filteredTaskUpdates.sum + val median = if (filteredTaskUpdates.size % 2 == 0) { + val mid = filteredTaskUpdates.size / 2 + (filteredTaskUpdates(mid) + filteredTaskUpdates(mid - 1)) / 2 + } else { + filteredTaskUpdates(filteredTaskUpdates.size / 2) } - }) + + val metricKey = normalizeToIODiagnosticMetricKey(sqlAccum.name) + metricNameToStatistics(metricKey) = StatisticsMetrics(min, median, max, sum) + } } if (metricNameToStatistics.isEmpty) { @@ -488,24 +467,37 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap app.accumManager.accumInfoMap.flatMap { accumMapEntry => val accumInfo = accumMapEntry._2 accumInfo.stageValuesMap.keySet.flatMap( stageId => { - getAccumInfoStatisticsInStage(accumInfo, getStageTaskIds(stageId)).map( { - case (min, median, max, sum) => { - // create and reuse AccumProfileResults object to avoid generating extra memory - val accumProfileResults = AccumProfileResults( - appIndex, - stageId, - accumInfo.infoRef, - min, - median, - max, - sum) - // update stageToDiagnosticMetrics mapping if accumInfo is a diagnostic metric - if (accumInfo.infoRef.name.isDiagnosticMetrics()) { - updateStageDiagnosticMetrics(accumProfileResults) - } - accumProfileResults + val stageTaskIds = getStageTaskIds(stageId) + val filteredTaskUpdates = + accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted + + if (filteredTaskUpdates.isEmpty) { + None + } else { + val min = filteredTaskUpdates.head + val max = filteredTaskUpdates.last + val sum = filteredTaskUpdates.sum + val median = if (filteredTaskUpdates.size % 2 == 0) { + val mid = filteredTaskUpdates.size / 2 + (filteredTaskUpdates(mid) + filteredTaskUpdates(mid - 1)) / 2 + } else { + filteredTaskUpdates(filteredTaskUpdates.size / 2) } - }) + + val accumProfileResults = AccumProfileResults( + appIndex, + stageId, + accumInfo.infoRef, + min, + median, + max, + sum) + // update stageToDiagnosticMetrics mapping if accumInfo is a diagnostic metric + if (accumInfo.infoRef.name.isDiagnosticMetrics()) { + updateStageDiagnosticMetrics(accumProfileResults) + } + Some(accumProfileResults) + } }) } }.toSeq From 550406c69e8071913be7e1cf7b3b08d8a48ae9c6 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Wed, 18 Dec 2024 16:43:35 -0800 Subject: [PATCH 14/23] refactor due to new optimizations from dev Signed-off-by: cindyyuanjiang --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 51 +++++++++++-------- 1 file changed, 29 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 8cddf7f50..78f29adae 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SparkPlanGraphCluster, import org.apache.spark.sql.rapids.tool.{AppBase, RDDCheckHelper, SqlPlanInfoGraphBuffer, SqlPlanInfoGraphEntry} import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo import org.apache.spark.sql.rapids.tool.qualification.QualificationAppInfo -import org.apache.spark.sql.rapids.tool.store.{AccumInfo, AccumMetaRef, AccumNameRef, DataSourceRecord} +import org.apache.spark.sql.rapids.tool.store.{AccumInfo, AccumMetaRef, DataSourceRecord} import org.apache.spark.sql.rapids.tool.util.ToolsPlanGraph /** @@ -110,13 +110,19 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap } /** - * Retrieves the set of task IDs associated with a specific stage. + * Retrieves task update values from the accumulator info for the specified stage ID. * - * @param stageId The ID of the stage. - * @return A set of task IDs corresponding to the given stage ID. + * @param accumInfo AccumInfo object containing the task updates map. + * @param stageId The stage ID for which task updates need to be retrived. + * @return A sorted sequence of task update values (`Long`) corresponding to the tasks + * in the specified stage. */ - private def getStageTaskIds(stageId: Int): Set[Long] = { - app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet + private def filterAccumTaskUpdatesForStage(accumInfo: AccumInfo, stageId: Int): Seq[Long] = { + val stageTaskIds = app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet + stageTaskIds.toSeq.collect { + case taskId if accumInfo.taskUpdatesMap.contains(taskId) => + accumInfo.taskUpdatesMap(taskId) + }.toSeq.sorted } /** @@ -380,7 +386,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap } /** - * Generate IO-related diagnostic metrics for the SQL plan. Metrics include: + * Generates IO-related diagnostic metrics for the SQL plan. Metrics include: * - Output rows * - Scan time * - Output batches @@ -389,29 +395,32 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap * - Fetch wait time * - GPU decode time * - * @return A sequence of `IODiagnosticResult` objects containing diagnostic metrics. + * This method processes accumulator information for each SQL stage and node and + * computes statistical results (min, median, max, sum) for IO-related metrics. + * + * @return A sequence of `IODiagnosticResult` objects one per SQL stage and node. */ def generateIODiagnosticAccums(): Seq[IODiagnosticResult] = { + val emptyAccumInfo = new AccumInfo(AccumMetaRef.EMPTY_ACCUM_META_REF) // Transform the diagnostic metrics map into a sequence of results IODiagnosticMetricsMap.toSeq.flatMap { case ((sqlId, nodeId, stageIds), sqlAccums) => // Process each stage ID and compute diagnostic results stageIds.split(",").filter(_.nonEmpty).map(_.toInt).flatMap { stageId => val nodeName = sqlAccums.head.nodeName - val stageTaskIds = getStageTaskIds(stageId) - // A mapping from metric name to its statistical results (min, median, max, sum) + + // Initialize a map to store statistics for each IO metric val metricNameToStatistics = HashMap.empty[String, StatisticsMetrics]. withDefaultValue(StatisticsMetrics.ZERO_RECORD) - // Iterate through each IO metric + // Process each accumulator for the current SQL stage sqlAccums.foreach { sqlAccum => val accumInfo = app.accumManager.accumInfoMap.getOrElse( - sqlAccum.accumulatorId, - new AccumInfo(AccumMetaRef(0L, AccumNameRef(""))) - ) - // Compute the metric's statistics (min, median, max, sum) for the given stage. - // Store the results if available. - val filteredTaskUpdates = - accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted + sqlAccum.accumulatorId, emptyAccumInfo) + + // Retrieve and sort task updates correspond to the current stage + val filteredTaskUpdates = filterAccumTaskUpdatesForStage(accumInfo, stageId) + + // Compute the metric's statistics and store the results if available if (filteredTaskUpdates.nonEmpty) { val min = filteredTaskUpdates.head val max = filteredTaskUpdates.last @@ -429,7 +438,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap } if (metricNameToStatistics.isEmpty) { - // metricNameToStatistics is not updated - there is no IO metrics result for this stage + // No IO metric statistics were computed for this stage None } else { Some(IODiagnosticResult( @@ -467,9 +476,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap app.accumManager.accumInfoMap.flatMap { accumMapEntry => val accumInfo = accumMapEntry._2 accumInfo.stageValuesMap.keySet.flatMap( stageId => { - val stageTaskIds = getStageTaskIds(stageId) - val filteredTaskUpdates = - accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted + val filteredTaskUpdates = filterAccumTaskUpdatesForStage(accumInfo, stageId) if (filteredTaskUpdates.isEmpty) { None From 6d0d7989e160bc6580eae5eb650268a0b3b807b2 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Wed, 18 Dec 2024 17:03:23 -0800 Subject: [PATCH 15/23] add back getStageTaskIds to avoid computing stage ids multiple times when unnecessary Signed-off-by: cindyyuanjiang --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 78f29adae..74b793fa7 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -109,6 +109,16 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap IODiagnosticMetricsMap(key) += accum } + /** + * Retrieves the task IDs associated with a specific stage. + * + * @param stageId The ID of the stage. + * @return A seq of task IDs corresponding to the given stage ID. + */ + private def getStageTaskIds(stageId: Int): Seq[Long] = { + app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet.toSeq + } + /** * Retrieves task update values from the accumulator info for the specified stage ID. * @@ -117,9 +127,9 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap * @return A sorted sequence of task update values (`Long`) corresponding to the tasks * in the specified stage. */ - private def filterAccumTaskUpdatesForStage(accumInfo: AccumInfo, stageId: Int): Seq[Long] = { - val stageTaskIds = app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet - stageTaskIds.toSeq.collect { + private def filterAccumTaskUpdatesForStage(accumInfo: AccumInfo, stageTaskIds: Seq[Long]) + : Seq[Long] = { + stageTaskIds.collect { case taskId if accumInfo.taskUpdatesMap.contains(taskId) => accumInfo.taskUpdatesMap(taskId) }.toSeq.sorted @@ -406,6 +416,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap IODiagnosticMetricsMap.toSeq.flatMap { case ((sqlId, nodeId, stageIds), sqlAccums) => // Process each stage ID and compute diagnostic results stageIds.split(",").filter(_.nonEmpty).map(_.toInt).flatMap { stageId => + val stageTaskIds = getStageTaskIds(stageId) val nodeName = sqlAccums.head.nodeName // Initialize a map to store statistics for each IO metric @@ -418,7 +429,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap sqlAccum.accumulatorId, emptyAccumInfo) // Retrieve and sort task updates correspond to the current stage - val filteredTaskUpdates = filterAccumTaskUpdatesForStage(accumInfo, stageId) + val filteredTaskUpdates = filterAccumTaskUpdatesForStage(accumInfo, stageTaskIds) // Compute the metric's statistics and store the results if available if (filteredTaskUpdates.nonEmpty) { @@ -476,7 +487,8 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap app.accumManager.accumInfoMap.flatMap { accumMapEntry => val accumInfo = accumMapEntry._2 accumInfo.stageValuesMap.keySet.flatMap( stageId => { - val filteredTaskUpdates = filterAccumTaskUpdatesForStage(accumInfo, stageId) + val filteredTaskUpdates = + filterAccumTaskUpdatesForStage(accumInfo, getStageTaskIds(stageId)) if (filteredTaskUpdates.isEmpty) { None From be428e954ed9c5051b251bb15364fb6a8ceced50 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Thu, 19 Dec 2024 18:46:09 -0800 Subject: [PATCH 16/23] merged dev Signed-off-by: cindyyuanjiang --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 37 ++++++++----------- 1 file changed, 15 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index ba7a60b6a..7270c800b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -117,7 +117,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap * @return A seq of task IDs corresponding to the given stage ID. */ private def getStageTaskIds(stageId: Int): Seq[Long] = { - app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet.toSeq + app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId)(breakOut).distinct } /** @@ -125,15 +125,15 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap * * @param accumInfo AccumInfo object containing the task updates map. * @param stageId The stage ID for which task updates need to be retrived. - * @return A sorted sequence of task update values (`Long`) corresponding to the tasks + * @return An array of task update values (`Long`) corresponding to the tasks * in the specified stage. */ private def filterAccumTaskUpdatesForStage(accumInfo: AccumInfo, stageTaskIds: Seq[Long]) - : Seq[Long] = { + : Array[Long] = { stageTaskIds.collect { case taskId if accumInfo.taskUpdatesMap.contains(taskId) => accumInfo.taskUpdatesMap(taskId) - }.toSeq.sorted + }(breakOut) } /** @@ -419,23 +419,15 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap val accumInfo = app.accumManager.accumInfoMap.getOrElse( sqlAccum.accumulatorId, emptyAccumInfo) - // Retrieve and sort task updates correspond to the current stage + // Retrieve task updates correspond to the current stage val filteredTaskUpdates = filterAccumTaskUpdatesForStage(accumInfo, stageTaskIds) // Compute the metric's statistics and store the results if available - if (filteredTaskUpdates.nonEmpty) { - val min = filteredTaskUpdates.head - val max = filteredTaskUpdates.last - val sum = filteredTaskUpdates.sum - val median = if (filteredTaskUpdates.size % 2 == 0) { - val mid = filteredTaskUpdates.size / 2 - (filteredTaskUpdates(mid) + filteredTaskUpdates(mid - 1)) / 2 - } else { - filteredTaskUpdates(filteredTaskUpdates.size / 2) - } - - val metricKey = normalizeToIODiagnosticMetricKey(sqlAccum.name) - metricNameToStatistics(metricKey) = StatisticsMetrics(min, median, max, sum) + StatisticsMetrics.createOptionalFromArr(filteredTaskUpdates) match { + case Some(stat) => + val metricKey = normalizeToIODiagnosticMetricKey(sqlAccum.name) + metricNameToStatistics(metricKey) = stat + case _ => () } } @@ -478,11 +470,12 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap app.accumManager.accumInfoMap.flatMap { accumMapEntry => val accumInfo = accumMapEntry._2 accumInfo.stageValuesMap.keys.flatMap( stageId => { - val stageTaskIds: Set[Long] = - app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId)(breakOut) + // Retrieve task updates correspond to the current stage + val filteredTaskUpdates = + filterAccumTaskUpdatesForStage(accumInfo, getStageTaskIds(stageId)) + // Get the task updates that belong to that stage - StatisticsMetrics.createOptionalFromArr( - accumInfo.taskUpdatesMap.filterKeys(stageTaskIds).map(_._2)(breakOut)) match { + StatisticsMetrics.createOptionalFromArr(filteredTaskUpdates) match { case Some(stat) => // Reuse AccumProfileResults to avoid generating allocating new objects val accumProfileResults = AccumProfileResults( From e882518e65b9d28412838f868f946df3276ff49b Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Mon, 23 Dec 2024 14:29:56 -0800 Subject: [PATCH 17/23] optimized computation of metric stats Signed-off-by: cindyyuanjiang --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 39 ++++++++++++------- .../profiling/ProfileClassWarehouse.scala | 33 ++++++++++++---- .../spark/rapids/tool/views/SQLView.scala | 2 +- .../tool/profiling/ApplicationInfoSuite.scala | 7 ++-- 4 files changed, 53 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 7270c800b..3d9d20b9a 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SparkPlanGraphCluster, import org.apache.spark.sql.rapids.tool.{AppBase, RDDCheckHelper, SqlPlanInfoGraphBuffer, SqlPlanInfoGraphEntry} import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo import org.apache.spark.sql.rapids.tool.qualification.QualificationAppInfo -import org.apache.spark.sql.rapids.tool.store.{AccumInfo, AccumMetaRef, DataSourceRecord} +import org.apache.spark.sql.rapids.tool.store.{AccumInfo, DataSourceRecord} import org.apache.spark.sql.rapids.tool.util.ToolsPlanGraph /** @@ -72,11 +72,10 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap // The key is a tuple consisting of: // - sqlID (Long): The unique identifier for the SQL query. // - nodeID (Long): The unique identifier for the node. - // - stageIDs (String): A comma-separated string representing the stage IDs involved. // The value is an ArrayBuffer of SQLAccumProfileResults objects, storing the IO diagnostic // metrics for the given key. - val IODiagnosticMetricsMap: HashMap[(Long, Long, String), ArrayBuffer[SQLAccumProfileResults]] = - HashMap.empty[(Long, Long, String), ArrayBuffer[SQLAccumProfileResults]] + val IODiagnosticMetricsMap: HashMap[(Long, Long), ArrayBuffer[SQLAccumProfileResults]] = + HashMap.empty[(Long, Long), ArrayBuffer[SQLAccumProfileResults]] /** * Updates the stageToDiagnosticMetrics mapping with the provided AccumProfileResults. @@ -100,7 +99,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap * to be added to IODiagnosticMetricsMap. */ private def updateIODiagnosticMetricsMap(accum: SQLAccumProfileResults): Unit = { - val key = (accum.sqlID, accum.nodeID, accum.stageIds) + val key = (accum.sqlID, accum.nodeID) // Initialize an entry if the key does not exist if (!IODiagnosticMetricsMap.contains(key)) { @@ -373,7 +372,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap val sqlAccumProileResult = SQLAccumProfileResults(appIndex, metric.sqlID, metric.nodeID, metric.nodeName, metric.accumulatorId, metric.name, - min, med, max, total, metric.metricType, metric.stageIds.mkString(",")) + min, med, max, total, metric.metricType, metric.stageIds) if (isIODiagnosticMetricName(metric.name)) { updateIODiagnosticMetricsMap(sqlAccumProileResult) @@ -402,11 +401,13 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap * @return A sequence of `IODiagnosticResult` objects one per SQL stage and node. */ def generateIODiagnosticAccums(): Seq[IODiagnosticResult] = { - val emptyAccumInfo = new AccumInfo(AccumMetaRef.EMPTY_ACCUM_META_REF) + // val emptyAccumInfo = new AccumInfo(AccumMetaRef.EMPTY_ACCUM_META_REF) // Transform the diagnostic metrics map into a sequence of results - IODiagnosticMetricsMap.toSeq.flatMap { case ((sqlId, nodeId, stageIds), sqlAccums) => + IODiagnosticMetricsMap.toSeq.flatMap { case ((sqlId, nodeId), sqlAccums) => // Process each stage ID and compute diagnostic results - stageIds.split(",").filter(_.nonEmpty).map(_.toInt).flatMap { stageId => + val stageIds = sqlAccums.head.stageIds + val stageIdsSize = stageIds.size + stageIds.flatMap { stageId => val stageTaskIds = getStageTaskIds(stageId) val nodeName = sqlAccums.head.nodeName @@ -416,14 +417,22 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap // Process each accumulator for the current SQL stage sqlAccums.foreach { sqlAccum => - val accumInfo = app.accumManager.accumInfoMap.getOrElse( - sqlAccum.accumulatorId, emptyAccumInfo) - - // Retrieve task updates correspond to the current stage - val filteredTaskUpdates = filterAccumTaskUpdatesForStage(accumInfo, stageTaskIds) + // TODO: need to check if accum ID is in driverAccumMap, currently skipped + val accumInfo = app.accumManager.accumInfoMap.get(sqlAccum.accumulatorId) + + val metricStats: Option[StatisticsMetrics] = + if (accumInfo.isEmpty || !accumInfo.stageValuesMap.contains(stageId)) { + None + } else if (stageIdsSize == 1) { + Some(StatisticsMetrics(sqlAccum.min, sqlAccum.median, sqlAccum.max, sqlAccum.total)) + } else { + // Retrieve task updates correspond to the current stage + val filteredTaskUpdates = filterAccumTaskUpdatesForStage(accumInfo.get, stageTaskIds) + StatisticsMetrics.createOptionalFromArr(filteredTaskUpdates) + } // Compute the metric's statistics and store the results if available - StatisticsMetrics.createOptionalFromArr(filteredTaskUpdates) match { + metricStats match { case Some(stat) => val metricKey = normalizeToIODiagnosticMetricKey(sqlAccum.name) metricNameToStatistics(metricKey) = stat diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index 6f06fca98..4ed1c4911 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -76,16 +76,33 @@ case class JobInfoProfileResult( sqlID: Option[Long], startTime: Long, endTime: Option[Long]) extends ProfileResult { - override val outputHeaders = Seq("appIndex", "jobID", "stageIds", "sqlID", "startTime", "endTime") + + override val outputHeaders = { + Seq("appIndex", + "jobID", + "stageIds", + "sqlID", + "startTime", + "endTime") + } + override def convertToSeq: Seq[String] = { val stageIdStr = s"[${stageIds.mkString(",")}]" - Seq(appIndex.toString, jobID.toString, stageIdStr, sqlID.map(_.toString).getOrElse(null), - startTime.toString, endTime.map(_.toString).getOrElse(null)) + Seq(appIndex.toString, + jobID.toString, + stageIdStr, + sqlID.map(_.toString).getOrElse(null), + startTime.toString, + endTime.map(_.toString).getOrElse(null)) } + override def convertToCSVSeq: Seq[String] = { val stageIdStr = s"[${stageIds.mkString(",")}]" - Seq(appIndex.toString, jobID.toString, StringUtils.reformatCSVString(stageIdStr), - sqlID.map(_.toString).getOrElse(null), startTime.toString, + Seq(appIndex.toString, + jobID.toString, + StringUtils.reformatCSVString(stageIdStr), + sqlID.map(_.toString).getOrElse(null), + startTime.toString, endTime.map(_.toString).getOrElse(null)) } } @@ -216,7 +233,7 @@ case class SQLAccumProfileResults( max: Long, total: Long, metricType: String, - stageIds: String) extends ProfileResult { + stageIds: Set[Int]) extends ProfileResult { override val outputHeaders = { Seq("appIndex", @@ -245,7 +262,7 @@ case class SQLAccumProfileResults( max.toString, total.toString, metricType, - stageIds) + stageIds.mkString(",")) } override def convertToCSVSeq: Seq[String] = { @@ -260,7 +277,7 @@ case class SQLAccumProfileResults( max.toString, total.toString, StringUtils.reformatCSVString(metricType), - StringUtils.reformatCSVString(stageIds)) + StringUtils.reformatCSVString(stageIds.mkString(","))) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala index 154f820f7..c1f2b5675 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala @@ -116,7 +116,7 @@ object ProfIODiagnosticMetricsView extends ViewableTrait[IODiagnosticResult] override def sortView( rows: Seq[IODiagnosticResult]): Seq[IODiagnosticResult] = { - rows.sortBy(cols => (cols.appIndex, cols.duration, cols.stageId, cols.sqlId, cols.nodeId)) + rows.sortBy(cols => (cols.appIndex, -cols.duration, cols.stageId, cols.sqlId, cols.nodeId)) } override def getRawView(app: AppBase, index: Int): Seq[IODiagnosticResult] = { diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala index fb7fbab05..40c7ea99f 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala @@ -195,11 +195,10 @@ class ApplicationInfoSuite extends FunSuite with Logging { val resultExpectation = new File(expRoot, "rapids_join_eventlog_sqlmetrics_expectation.csv") assert(sqlMetrics.size == 83) - val sqlMetricsWithDelim = sqlMetrics.map { metrics => - metrics.copy(stageIds = ProfileUtils.replaceDelimiter(metrics.stageIds, ",")) - } + + import org.apache.spark.sql.functions._ import sparkSession.implicits._ - val df = sqlMetricsWithDelim.toDF + val df = sqlMetrics.toDF.withColumn("stageIds", concat_ws(",", col("stageIds"))) val dfExpect = ToolTestUtils.readExpectationCSV(sparkSession, resultExpectation.getPath()) ToolTestUtils.compareDataFrames(df, dfExpect) } From 735546a8d8ff46251b6092d581e20aeef993bc72 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Mon, 23 Dec 2024 15:05:11 -0800 Subject: [PATCH 18/23] change metricNamesToKeyMap from string to string Signed-off-by: cindyyuanjiang --- .../rapids/tool/analysis/AnalysisUtils.scala | 22 +++++++++---------- .../tool/analysis/AppSQLPlanAnalyzer.scala | 2 +- 2 files changed, 11 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala index 176d79c15..6bf42206d 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala @@ -77,19 +77,19 @@ object IOAccumDiagnosticMetrics { "GPU decode time" // only in GPU eventlogs ) - private val metricNamesToKeyMap: Map[Set[String], String] = Map( - OUTPUT_ROW_METRIC_NAMES -> OUTPUT_ROWS_METRIC_KEY, - SCAN_TIME_METRIC_NAMES -> SCAN_TIME_METRIC_KEY, - OUTPUT_BATCHES_METRIC_NAMES -> OUTPUT_BATCHES_METRIC_KEY, - BUFFER_TIME_METRIC_NAMES -> BUFFER_TIME_METRIC_KEY, - SHUFFLE_WRITE_TIME_METRIC_NAMES -> SHUFFLE_WRITE_TIME_METRIC_KEY, - FETCH_WAIT_TIME_METRIC_NAMES -> FETCH_WAIT_TIME_METRIC_KEY, - GPU_DECODE_TIME_METRIC_NAMES -> GPU_DECODE_TIME_METRIC_KEY) + private val metricNamesToKeyMap: Map[String, String] = ( + OUTPUT_ROW_METRIC_NAMES.map(_ -> OUTPUT_ROWS_METRIC_KEY) ++ + SCAN_TIME_METRIC_NAMES.map(_ -> SCAN_TIME_METRIC_KEY) ++ + OUTPUT_BATCHES_METRIC_NAMES.map(_ -> OUTPUT_BATCHES_METRIC_KEY) ++ + BUFFER_TIME_METRIC_NAMES.map(_ -> BUFFER_TIME_METRIC_KEY) ++ + SHUFFLE_WRITE_TIME_METRIC_NAMES.map(_ -> SHUFFLE_WRITE_TIME_METRIC_KEY) ++ + FETCH_WAIT_TIME_METRIC_NAMES.map(_ -> FETCH_WAIT_TIME_METRIC_KEY) ++ + GPU_DECODE_TIME_METRIC_NAMES.map(_ -> GPU_DECODE_TIME_METRIC_KEY)).toMap /** * Set of all IO diagnostic metrics names */ - lazy val allIODiagnosticMetrics: Set[String] = metricNamesToKeyMap.keys.flatten.toSet + lazy val allIODiagnosticMetrics: Set[String] = metricNamesToKeyMap.keys.toSet /** * Check if a metric name belongs to IO diagnostic metrics @@ -105,8 +105,6 @@ object IOAccumDiagnosticMetrics { */ def normalizeToIODiagnosticMetricKey(metric: String): String = { // input metric is already known to be an IO diagnostic metric - metricNamesToKeyMap.collectFirst { - case (names, key) if names.contains(metric) => key - }.get + metricNamesToKeyMap(metric) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 3d9d20b9a..e571e2f7a 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -421,7 +421,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap val accumInfo = app.accumManager.accumInfoMap.get(sqlAccum.accumulatorId) val metricStats: Option[StatisticsMetrics] = - if (accumInfo.isEmpty || !accumInfo.stageValuesMap.contains(stageId)) { + if (accumInfo.isEmpty || !accumInfo.get.stageValuesMap.contains(stageId)) { None } else if (stageIdsSize == 1) { Some(StatisticsMetrics(sqlAccum.min, sqlAccum.median, sqlAccum.max, sqlAccum.total)) From 43f5c84fc70ea0a775d9e30fc6f8973f88f7646a Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Mon, 23 Dec 2024 15:31:54 -0800 Subject: [PATCH 19/23] move AnalysisUtil file Signed-off-by: cindyyuanjiang --- .../nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala | 2 +- .../spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala | 2 +- .../{AnalysisUtils.scala => util/DiagnosticMetrics.scala} | 2 +- .../org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) rename core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/{AnalysisUtils.scala => util/DiagnosticMetrics.scala} (98%) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index e571e2f7a..1a0f0a35f 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -19,7 +19,7 @@ package com.nvidia.spark.rapids.tool.analysis import scala.collection.breakOut import scala.collection.mutable.{AbstractSet, ArrayBuffer, HashMap, LinkedHashSet} -import com.nvidia.spark.rapids.tool.analysis.IOAccumDiagnosticMetrics._ +import com.nvidia.spark.rapids.tool.analysis.util.IOAccumDiagnosticMetrics._ import com.nvidia.spark.rapids.tool.profiling.{AccumProfileResults, IODiagnosticResult, SQLAccumProfileResults, SQLMetricInfoCase, SQLStageInfoProfileResult, UnsupportedSQLPlan, WholeStageCodeGenResults} import com.nvidia.spark.rapids.tool.qualification.QualSQLPlanAnalyzer diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala index 6b8c3d5e5..990d9d21a 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala @@ -19,8 +19,8 @@ package com.nvidia.spark.rapids.tool.analysis import scala.collection.breakOut import scala.collection.mutable.{ArrayBuffer, HashMap, LinkedHashMap} -import com.nvidia.spark.rapids.tool.analysis.StageAccumDiagnosticMetrics._ import com.nvidia.spark.rapids.tool.analysis.util.{AggAccumHelper, AggAccumPhotonHelper} +import com.nvidia.spark.rapids.tool.analysis.util.StageAccumDiagnosticMetrics._ import com.nvidia.spark.rapids.tool.planparser.DatabricksParseHelper import com.nvidia.spark.rapids.tool.profiling._ diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/DiagnosticMetrics.scala similarity index 98% rename from core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala rename to core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/DiagnosticMetrics.scala index 6bf42206d..323d3a245 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/DiagnosticMetrics.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.tool.analysis +package com.nvidia.spark.rapids.tool.analysis.util object StageAccumDiagnosticMetrics { val MEMORY_SPILLED_METRIC = "internal.metrics.memoryBytesSpilled" diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala index 7c7aca6a9..75a55635d 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.rapids.tool.store import java.util.concurrent.ConcurrentHashMap -import com.nvidia.spark.rapids.tool.analysis.StageAccumDiagnosticMetrics.allDiagnosticMetrics +import com.nvidia.spark.rapids.tool.analysis.util.StageAccumDiagnosticMetrics.allDiagnosticMetrics import org.apache.spark.sql.rapids.tool.util.EventUtils.normalizeMetricName import org.apache.spark.sql.rapids.tool.util.StringUtils From e8c2814c45dd703aab9601a94ca3825385caad1a Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Mon, 23 Dec 2024 16:01:35 -0800 Subject: [PATCH 20/23] minor style updates Signed-off-by: cindyyuanjiang --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 1a0f0a35f..b5300295e 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -83,14 +83,12 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap * to stageToDiagnosticMetrics mapping. */ private def updateStageDiagnosticMetrics(accum: AccumProfileResults): Unit = { - val stageId = accum.stageId - // Initialize an empty mapping for the stage if it doesn't already exist - if (!stageToDiagnosticMetrics.contains(stageId)) { - stageToDiagnosticMetrics(stageId) = HashMap.empty[String, AccumProfileResults] + if (!stageToDiagnosticMetrics.contains(accum.stageId)) { + stageToDiagnosticMetrics(accum.stageId) = HashMap.empty[String, AccumProfileResults] } - stageToDiagnosticMetrics(stageId)(accum.accMetaRef.getName()) = accum + stageToDiagnosticMetrics(accum.stageId)(accum.accMetaRef.getName()) = accum } /** @@ -401,12 +399,10 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap * @return A sequence of `IODiagnosticResult` objects one per SQL stage and node. */ def generateIODiagnosticAccums(): Seq[IODiagnosticResult] = { - // val emptyAccumInfo = new AccumInfo(AccumMetaRef.EMPTY_ACCUM_META_REF) // Transform the diagnostic metrics map into a sequence of results IODiagnosticMetricsMap.toSeq.flatMap { case ((sqlId, nodeId), sqlAccums) => // Process each stage ID and compute diagnostic results val stageIds = sqlAccums.head.stageIds - val stageIdsSize = stageIds.size stageIds.flatMap { stageId => val stageTaskIds = getStageTaskIds(stageId) val nodeName = sqlAccums.head.nodeName @@ -417,16 +413,17 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap // Process each accumulator for the current SQL stage sqlAccums.foreach { sqlAccum => - // TODO: need to check if accum ID is in driverAccumMap, currently skipped + // TODO: check if accumulator ID is in driverAccumMap, currently skipped val accumInfo = app.accumManager.accumInfoMap.get(sqlAccum.accumulatorId) val metricStats: Option[StatisticsMetrics] = if (accumInfo.isEmpty || !accumInfo.get.stageValuesMap.contains(stageId)) { None - } else if (stageIdsSize == 1) { + } else if (stageIds.size == 1) { + // Skip computing statistics when there is only one stage Some(StatisticsMetrics(sqlAccum.min, sqlAccum.median, sqlAccum.max, sqlAccum.total)) } else { - // Retrieve task updates correspond to the current stage + // Retrieve task updates which correspond to the current stage val filteredTaskUpdates = filterAccumTaskUpdatesForStage(accumInfo.get, stageTaskIds) StatisticsMetrics.createOptionalFromArr(filteredTaskUpdates) } From b97e2db22c127f03e676a7d359b2aa1c65030e4d Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Mon, 23 Dec 2024 16:07:33 -0800 Subject: [PATCH 21/23] expected file update Signed-off-by: cindyyuanjiang --- ..._eventlog_iodiagnosticmetrics_expectation.csv | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_iodiagnosticmetrics_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_iodiagnosticmetrics_expectation.csv index 094b8b2fb..8b1d3093e 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_iodiagnosticmetrics_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_iodiagnosticmetrics_expectation.csv @@ -1,7 +1,8 @@ -appIndex,appName,appId,sqlId,stageId,stageDurationMs,nodeId,nodeName,outputRowsMin,outputRowsMedian,outputRowsMax,outputRowsTotal,scanTimeMin,scanTimeMedian,scanTimeMax,scanTimeTotal,outputBatchesMin,outputBatchesMedian,outputBatchesMax,outputBatchesTotal,buffeTimeMin,buffeTimeMedian,buffeTimeMax,buffeTimeTotal,shuffleWriteTimeMin,shuffleWriteTimeMedian,shuffleWriteTimeMax,shuffleWriteTimeTotal,fetchWaitTimeMin,fetchWaitTimeMedian,fetchWaitTimeMax,fetchWaitTimeTotal,gpuDecodeTimeMin,gpuDecodeTimeMedian,gpuDecodeTimeMax,gpuDecodeTimeTotal -1,Spark shell,local-1622814619968,0,3,83,1,"GpuHashAggregate",1,1,1,1,0,0,0,0,1,1,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 -1,Spark shell,local-1622814619968,0,3,83,2,"GpuShuffleCoalesce",200,200,200,200,0,0,0,0,1,1,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 -1,Spark shell,local-1622814619968,0,3,83,3,"GpuColumnarExchange",0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +appIndex,appName,appId,sqlId,stageId,stageDurationMs,nodeId,nodeName,outputRowsMin,outputRowsMedian,outputRowsMax,outputRowsTotal,scanTimeMin,scanTimeMedian,scanTimeMax,scanTimeTotal,outputBatchesMin,outputBatchesMedian,outputBatchesMax,outputBatchesTotal,bufferTimeMin,bufferTimeMedian,bufferTimeMax,bufferTimeTotal,shuffleWriteTimeMin,shuffleWriteTimeMedian,shuffleWriteTimeMax,shuffleWriteTimeTotal,fetchWaitTimeMin,fetchWaitTimeMedian,fetchWaitTimeMax,fetchWaitTimeTotal,gpuDecodeTimeMin,gpuDecodeTimeMedian,gpuDecodeTimeMax,gpuDecodeTimeTotal +1,Spark shell,local-1622814619968,0,0,1743,16,"GpuColumnarExchange",1666666,1666667,1666667,10000000,0,0,0,0,200,200,200,1200,0,0,0,0,41434653,60830365,100858775,400284505,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,0,1743,21,"Scan",1666666,1666667,1666667,10000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,1,1631,8,"GpuColumnarExchange",1666666,1666667,1666667,10000000,0,0,0,0,200,200,200,1200,0,0,0,0,37444140,92128351,108992798,508750471,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,1,1631,13,"Scan",1666666,1666667,1666667,10000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 1,Spark shell,local-1622814619968,0,2,688,3,"GpuColumnarExchange",1,1,1,200,0,0,0,0,1,1,1,200,0,0,0,0,139875,230038,9747416,93193331,0,0,0,0,0,0,0,0 1,Spark shell,local-1622814619968,0,2,688,4,"GpuHashAggregate",1,1,1,200,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 1,Spark shell,local-1622814619968,0,2,688,6,"GpuShuffledHashJoin",49480,50017,50559,10000000,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 @@ -10,7 +11,6 @@ appIndex,appName,appId,sqlId,stageId,stageDurationMs,nodeId,nodeName,outputRowsM 1,Spark shell,local-1622814619968,0,2,688,14,"GpuCoalesceBatches",49480,50017,50559,10000000,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 1,Spark shell,local-1622814619968,0,2,688,15,"GpuShuffleCoalesce",49480,50017,50559,10000000,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 1,Spark shell,local-1622814619968,0,2,688,16,"GpuColumnarExchange",0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 -1,Spark shell,local-1622814619968,0,1,1631,8,"GpuColumnarExchange",1666666,1666667,1666667,10000000,0,0,0,0,200,200,200,1200,0,0,0,0,37444140,92128351,108992798,508750471,0,0,0,0,0,0,0,0 -1,Spark shell,local-1622814619968,0,1,1631,13,"Scan",1666666,1666667,1666667,10000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 -1,Spark shell,local-1622814619968,0,0,1743,16,"GpuColumnarExchange",1666666,1666667,1666667,10000000,0,0,0,0,200,200,200,1200,0,0,0,0,41434653,60830365,100858775,400284505,0,0,0,0,0,0,0,0 -1,Spark shell,local-1622814619968,0,0,1743,21,"Scan",1666666,1666667,1666667,10000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,3,83,1,"GpuHashAggregate",1,1,1,1,0,0,0,0,1,1,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,3,83,2,"GpuShuffleCoalesce",200,200,200,200,0,0,0,0,1,1,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,3,83,3,"GpuColumnarExchange",0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 From 631b9c0311b9ae2e29bb7548a92f02271df80f99 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Mon, 23 Dec 2024 16:38:40 -0800 Subject: [PATCH 22/23] minor updates between seq conversion Signed-off-by: cindyyuanjiang --- .../spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index b5300295e..35ab0e323 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -400,7 +400,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap */ def generateIODiagnosticAccums(): Seq[IODiagnosticResult] = { // Transform the diagnostic metrics map into a sequence of results - IODiagnosticMetricsMap.toSeq.flatMap { case ((sqlId, nodeId), sqlAccums) => + IODiagnosticMetricsMap.flatMap { case ((sqlId, nodeId), sqlAccums) => // Process each stage ID and compute diagnostic results val stageIds = sqlAccums.head.stageIds stageIds.flatMap { stageId => @@ -459,7 +459,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap metricNameToStatistics(GPU_DECODE_TIME_METRIC_KEY))) } } - }.toSeq + }(breakOut) } /** From cc80b48d9e3c5e198cf2597a59f2a898d969d120 Mon Sep 17 00:00:00 2001 From: cindyyuanjiang Date: Mon, 30 Dec 2024 14:30:49 -0800 Subject: [PATCH 23/23] comment on empty stage ids Signed-off-by: cindyyuanjiang --- .../nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 35ab0e323..05c27648b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -402,6 +402,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap // Transform the diagnostic metrics map into a sequence of results IODiagnosticMetricsMap.flatMap { case ((sqlId, nodeId), sqlAccums) => // Process each stage ID and compute diagnostic results + // TODO: currently if stage IDs is empty, the result is skipped val stageIds = sqlAccums.head.stageIds stageIds.flatMap { stageId => val stageTaskIds = getStageTaskIds(stageId)