From 0cc500f688e7c4b6292401cfbecf592258424ac4 Mon Sep 17 00:00:00 2001
From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com>
Date: Thu, 16 May 2024 09:09:08 -0500
Subject: [PATCH] Refactor TaskEnd to be accessible by Q/P tools (#1000)
* Refactor TaskEnd to be accessible by Q/P tools
Signed-off-by: Ahmed Hussein (amahussein)
Contributes to #980
* Add Stage Accumulables to the accumulable objects for Q tool
Signed-off-by: Ahmed Hussein (amahussein)
* Refactor the code to allow Qual tool to generate same CSV files as Prof
Signed-off-by: Ahmed Hussein (amahussein)
* Cleaning up some naming conventions
Signed-off-by: Ahmed Hussein (amahussein)
* Fix typo in job/stage qual agg metrics
Signed-off-by: Ahmed Hussein (amahussein)
* Remove redundant sort function in skewshuffle analyzer
Signed-off-by: Ahmed Hussein (amahussein)
* Fix typos and remove unused classes from ProfileClassWarehouse
Signed-off-by: Ahmed Hussein (amahussein)
---------
Signed-off-by: Ahmed Hussein (amahussein)
---
.../tool/analysis/AggRawMetricsResult.scala | 47 +++
.../tool/analysis/AppAnalysisBase.scala | 31 ++
.../tool/analysis/AppIndexMapperTrait.scala | 43 ++
.../tool/analysis/AppSQLPlanAnalyzer.scala | 298 +++++++++++++
.../analysis/AppSparkMetricsAggTrait.scala | 65 +++
.../analysis/AppSparkMetricsAnalyzer.scala | 391 ++++++++++++++++++
.../analysis/ProfSparkMetricsAnalyzer.scala | 25 ++
.../analysis/QualSparkMetricsAnalyzer.scala | 22 +
.../spark/rapids/tool/analysis/package.scala | 26 ++
.../rapids/tool/profiling/Analysis.scala | 379 -----------------
.../tool/profiling/CollectInformation.scala | 160 +------
.../rapids/tool/profiling/GenerateDot.scala | 10 +-
.../tool/profiling/GenerateTimeline.scala | 4 +-
.../rapids/tool/profiling/HealthCheck.scala | 80 +---
.../profiling/ProfileClassWarehouse.scala | 45 --
.../rapids/tool/profiling/Profiler.scala | 72 ++--
.../tool/qualification/Qualification.scala | 3 +
.../tool/views/AggMetricsResultSorter.scala | 95 +++++
.../rapids/tool/views/ExecutorView.scala | 114 +++++
.../spark/rapids/tool/views/JobView.scala | 71 ++++
.../tool/views/QualRawReportGenerator.scala | 98 +++++
.../rapids/tool/views/RawMetricProfView.scala | 46 +++
.../spark/rapids/tool/views/SQLView.scala | 95 +++++
.../spark/rapids/tool/views/StageView.scala | 85 ++++
.../spark/rapids/tool/views/TaskView.scala | 49 +++
.../rapids/tool/views/ViewableTrait.scala | 42 ++
.../spark/rapids/tool/views/package.scala | 43 ++
.../spark/sql/rapids/tool/AppBase.scala | 19 +-
.../sql/rapids/tool/EventProcessorBase.scala | 51 ++-
.../rapids/tool/annotation/WallClock.scala | 2 +-
.../tool/profiling/ApplicationInfo.scala | 174 +-------
.../tool/profiling/EventsProcessor.scala | 128 +-----
.../sql/rapids/tool/store/TaskModel.scala | 119 ++++++
.../rapids/tool/store/TaskModelManager.scala | 109 +++++
.../rapids/tool/profiling/AnalysisSuite.scala | 30 +-
.../tool/profiling/ApplicationInfoSuite.scala | 9 +-
36 files changed, 2062 insertions(+), 1018 deletions(-)
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AggRawMetricsResult.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppAnalysisBase.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppIndexMapperTrait.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/ProfSparkMetricsAnalyzer.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/QualSparkMetricsAnalyzer.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/package.scala
delete mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Analysis.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/views/AggMetricsResultSorter.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/views/ExecutorView.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/views/JobView.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/views/QualRawReportGenerator.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/views/RawMetricProfView.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/views/StageView.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/views/TaskView.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/views/ViewableTrait.scala
create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/views/package.scala
create mode 100644 core/src/main/scala/org/apache/spark/sql/rapids/tool/store/TaskModel.scala
create mode 100644 core/src/main/scala/org/apache/spark/sql/rapids/tool/store/TaskModelManager.scala
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AggRawMetricsResult.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AggRawMetricsResult.scala
new file mode 100644
index 000000000..a27a08cf4
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AggRawMetricsResult.scala
@@ -0,0 +1,47 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.analysis
+
+import com.nvidia.spark.rapids.tool.profiling.{IOAnalysisProfileResult, JobStageAggTaskMetricsProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLMaxTaskInputSizes, SQLTaskAggMetricsProfileResult}
+
+/**
+ * The result of the aggregation of the raw metrics. It contains the aggregated metrics for an
+ * application. This case class is used to allow to separate the aggregation of the metrics from
+ * how the view are generated.
+ * For example, the profiler tool currently merges both job/stage-level in a single list.
+ * As a step toward separating the logic from the views, the analyzer returns
+ * AggRawMetricsResult that contains the aggregated metrics for jobs, stages, SQLs, and IOs.
+ * In later refactors, we can revisit *TaskMetricsProfileResult to have integer IDs instead of
+ * the current format "stage_ID" or "job_ID". We still use the old format to keep the compatibility
+ * with other modules.
+ *
+ * @param jobAggs the aggregated Spark metrics for jobs
+ * @param stageAggs the aggregated Spark metrics for stages
+ * @param taskShuffleSkew list of tasks that exhibit shuffle skewness
+ * @param sqlAggs the aggregated Spark metrics for SQLs
+ * @param ioAggs lists the SQLs along their IO metrics
+ * @param sqlDurAggs the aggregated duration and CPU time for SQLs
+ * @param maxTaskInputSizes a sequence of SQLMaxTaskInputSizes that contains the maximum input size
+ */
+case class AggRawMetricsResult(
+ jobAggs: Seq[JobStageAggTaskMetricsProfileResult],
+ stageAggs: Seq[JobStageAggTaskMetricsProfileResult],
+ taskShuffleSkew: Seq[ShuffleSkewProfileResult],
+ sqlAggs: Seq[SQLTaskAggMetricsProfileResult],
+ ioAggs: Seq[IOAnalysisProfileResult],
+ sqlDurAggs: Seq[SQLDurationExecutorTimeProfileResult],
+ maxTaskInputSizes: Seq[SQLMaxTaskInputSizes])
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppAnalysisBase.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppAnalysisBase.scala
new file mode 100644
index 000000000..138ca99c7
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppAnalysisBase.scala
@@ -0,0 +1,31 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.analysis
+
+import org.apache.spark.sql.rapids.tool.AppBase
+
+/**
+ * Base class for application analysis
+ * @param app the AppBase object to analyze
+ */
+abstract class AppAnalysisBase(app: AppBase) {
+ // Keep for future refactoring to use common methods for all Analysis classes.
+ // Ideally, we can common interface
+ // 1- caching layer
+ // 2- initializations
+ // 3- interface to pull information to generate views and reports
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppIndexMapperTrait.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppIndexMapperTrait.scala
new file mode 100644
index 000000000..0f874ce03
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppIndexMapperTrait.scala
@@ -0,0 +1,43 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.analysis
+
+import org.apache.spark.sql.rapids.tool.AppBase
+import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo
+
+trait AppIndexMapperTrait {
+ def zipAppsWithIndex(apps: Seq[AppBase]): Seq[(AppBase, Int)]
+}
+
+// Implementation used by Qualification components because AppBase has no appIndex field. Instead,
+// this implementation generates index based on the order of the apps.
+trait QualAppIndexMapperTrait extends AppIndexMapperTrait {
+ def zipAppsWithIndex(apps: Seq[AppBase]): Seq[(AppBase, Int)] = {
+ // we did not use zipWithIndex because we want to start from 1 instead of 0
+ apps.zip(Stream.from(1))
+ }
+}
+
+// Implementation used by Profiling components because ApplicationInfo has appIndex field which is
+// used in generating reports with multiple AppIds
+trait ProfAppIndexMapperTrait extends AppIndexMapperTrait {
+ override def zipAppsWithIndex(apps: Seq[AppBase]): Seq[(AppBase, Int)] = {
+ apps.collect {
+ case app: ApplicationInfo => (app, app.index)
+ }
+ }
+}
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
new file mode 100644
index 000000000..1efd2e0e2
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala
@@ -0,0 +1,298 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.analysis
+
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import com.nvidia.spark.rapids.tool.planparser.SQLPlanParser
+import com.nvidia.spark.rapids.tool.profiling.{SQLAccumProfileResults, SQLMetricInfoCase, SQLStageInfoProfileResult, UnsupportedSQLPlan, WholeStageCodeGenResults}
+
+import org.apache.spark.sql.execution.SparkPlanInfo
+import org.apache.spark.sql.execution.ui.SparkPlanGraph
+import org.apache.spark.sql.rapids.tool.{AppBase, RDDCheckHelper, SQLMetricsStats, SqlPlanInfoGraphBuffer, ToolUtils}
+import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo
+import org.apache.spark.sql.rapids.tool.util.ToolsPlanGraph
+
+/**
+ * This class processes SQL plan to build some information such as: metrics, wholestage nodes, and
+ * connecting operators to nodes. The implementation used to be directly under profiler's
+ * ApplicationInfo class. Moving the logic and the data structure to this new class is part of
+ * refactor to have a SQLPlan processor that can produce the same analysis for both the Prof/Qual
+ * tools.
+ * TODO: 1- Make the processor accepts appBase instead of applicationInfo. The tricky part here
+ * that Qual has its own way of reporting Problematic SQLs and identifying RDDs.
+ * 2- Restructure the implementation similar to AppSparkMetricsAnalysis to separate between
+ * analysis and views.
+ * @param app the Application into objects that contains the SQL plans to be processed
+ */
+class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(app) {
+ private val sqlPlanNodeIdToStageIds: mutable.HashMap[(Long, Long), Set[Int]] =
+ mutable.HashMap.empty[(Long, Long), Set[Int]]
+ var wholeStage: ArrayBuffer[WholeStageCodeGenResults] = ArrayBuffer[WholeStageCodeGenResults]()
+ var unsupportedSQLPlan: ArrayBuffer[UnsupportedSQLPlan] = ArrayBuffer[UnsupportedSQLPlan]()
+ var allSQLMetrics: ArrayBuffer[SQLMetricInfoCase] = ArrayBuffer[SQLMetricInfoCase]()
+
+ /**
+ * Connects Operators to Stages using AccumulatorIDs
+ * @param cb function that creates a SparkPlanGraph. This can be used as a cacheHolder for the
+ * object created to be used later.
+ */
+ private def connectOperatorToStage(cb: (Long, SparkPlanInfo) => SparkPlanGraph): Unit = {
+ for ((sqlId, planInfo) <- app.sqlPlans) {
+ val planGraph: SparkPlanGraph = cb.apply(sqlId, planInfo)
+ // Maps stages to operators by checking for non-zero intersection
+ // between nodeMetrics and stageAccumulateIDs
+ val nodeIdToStage = planGraph.allNodes.map { node =>
+ val mappedStages = SQLPlanParser.getStagesInSQLNode(node, app)
+ ((sqlId, node.id), mappedStages)
+ }.toMap
+ sqlPlanNodeIdToStageIds ++= nodeIdToStage
+ }
+ }
+
+ /**
+ * Function to process SQL Plan Metrics after all events are processed
+ */
+ def processSQLPlanMetrics(): Unit = {
+ // Define a buffer to cache the SQLPlanInfoGraphs
+ val sqlPlanInfoBuffer = SqlPlanInfoGraphBuffer()
+ // Define a function used to fill in the buffer while executing "connectOperatorToStage"
+ val createGraphFunc = (sqlId: Long, planInfo: SparkPlanInfo) => {
+ sqlPlanInfoBuffer.addSqlPlanInfoGraph(sqlId, planInfo).sparkPlanGraph
+ }
+ connectOperatorToStage(createGraphFunc)
+ for (sqlPIGEntry <- sqlPlanInfoBuffer.sqlPlanInfoGraphs) {
+ var sqlIsDsOrRDD = false
+ val potentialProblems = collection.mutable.Set[String]()
+ // store all datasources of the given SQL in a variable so that we won't have to iterate
+ // through the entire list
+ // get V1 dataSources for that SQLId
+ val sqlDataSources = app.checkMetadataForReadSchema(sqlPIGEntry)
+ for (node <- sqlPIGEntry.sparkPlanGraph.allNodes) {
+ var nodeIsDsOrRDD = false
+ if (node.isInstanceOf[org.apache.spark.sql.execution.ui.SparkPlanGraphCluster]) {
+ val ch = node.asInstanceOf[org.apache.spark.sql.execution.ui.SparkPlanGraphCluster].nodes
+ ch.foreach { c =>
+ wholeStage += WholeStageCodeGenResults(
+ appIndex, sqlPIGEntry.sqlID, node.id, node.name, c.name, c.id)
+ }
+ }
+ // get V2 dataSources for that node
+ val nodeV2Reads = app.checkGraphNodeForReads(sqlPIGEntry.sqlID, node)
+ if (nodeV2Reads.isDefined) {
+ sqlDataSources += nodeV2Reads.get
+ }
+ nodeIsDsOrRDD = RDDCheckHelper.isDatasetOrRDDPlan(node.name, node.desc).isRDD
+ if (nodeIsDsOrRDD) {
+ if (app.gpuMode) { // we want to report every node that is an RDD
+ val thisPlan = UnsupportedSQLPlan(sqlPIGEntry.sqlID, node.id, node.name, node.desc,
+ "Contains Dataset or RDD")
+ unsupportedSQLPlan += thisPlan
+ }
+ // If one node is RDD, the Sql should be set too
+ if (!sqlIsDsOrRDD) { // We need to set the flag only once for the given sqlID
+ sqlIsDsOrRDD = true
+ app.sqlIdToInfo.get(sqlPIGEntry.sqlID).foreach { sql =>
+ sql.setDsOrRdd(sqlIsDsOrRDD)
+ app.sqlIDToDataSetOrRDDCase += sqlPIGEntry.sqlID
+ // Clear the potential problems since it is an RDD to free memory
+ potentialProblems.clear()
+ }
+ }
+ }
+ if (!sqlIsDsOrRDD) {
+ // Append current node's potential problems to the Sql problems only if the SQL is not an
+ // RDD. This is an optimization since the potentialProblems won't be used any more.
+ potentialProblems ++= app.findPotentialIssues(node.desc)
+ }
+ // Then process SQL plan metric type
+ for (metric <- node.metrics) {
+ val stages =
+ sqlPlanNodeIdToStageIds.get((sqlPIGEntry.sqlID, node.id)).getOrElse(Set.empty)
+ val allMetric = SQLMetricInfoCase(sqlPIGEntry.sqlID, metric.name,
+ metric.accumulatorId, metric.metricType, node.id,
+ node.name, node.desc, stages)
+
+ allSQLMetrics += allMetric
+ if (app.sqlPlanMetricsAdaptive.nonEmpty) {
+ val adaptive = app.sqlPlanMetricsAdaptive.filter { adaptiveMetric =>
+ adaptiveMetric.sqlID == sqlPIGEntry.sqlID &&
+ adaptiveMetric.accumulatorId == metric.accumulatorId
+ }
+ adaptive.foreach { adaptiveMetric =>
+ val allMetric = SQLMetricInfoCase(sqlPIGEntry.sqlID, adaptiveMetric.name,
+ adaptiveMetric.accumulatorId, adaptiveMetric.metricType, node.id,
+ node.name, node.desc, stages)
+ // could make this more efficient but seems ok for now
+ val exists = allSQLMetrics.filter { a =>
+ ((a.accumulatorId == adaptiveMetric.accumulatorId) && (a.sqlID == sqlPIGEntry.sqlID)
+ && (a.nodeID == node.id && adaptiveMetric.metricType == a.metricType))
+ }
+ if (exists.isEmpty) {
+ allSQLMetrics += allMetric
+ }
+ }
+ }
+ }
+ }
+ if (app.isInstanceOf[ApplicationInfo]) {
+ // TODO: We should clean this in better way so that sqlItoProblematic is handled similar
+ // way in both Qual/Prof tools.
+ // This is a hack to get the processSQLPlanMetrics() method to work for both Qual/Prof
+ // - we check if the app is AppInfo, then we add the potential problems.
+ // - If not, then we do not set the problematic issues because this will cause
+ // records to be duplicated in the Qualification tool.
+ // Check if readsSchema is complex for the given sql
+ val sqlNestedComplexTypes =
+ AppBase.parseReadSchemaForNestedTypes(sqlDataSources.map { ds => ds.schema })
+ // Append problematic issues to the global variable for that SqlID
+ if (sqlNestedComplexTypes._2.nonEmpty) {
+ potentialProblems += "NESTED COMPLEX TYPE"
+ }
+ // Finally, add the local potentialProblems to the global data structure if any.
+ app.sqlIDtoProblematic(sqlPIGEntry.sqlID) = potentialProblems.toSet
+ // Convert the problematic issues to a string and update the SQLInfo
+ app.sqlIdToInfo.get(sqlPIGEntry.sqlID).foreach { sqlInfoClass =>
+ sqlInfoClass.problematic = ToolUtils.formatPotentialProblems(potentialProblems.toSeq)
+ }
+ }
+ }
+ }
+
+ def aggregateSQLStageInfo: Seq[SQLStageInfoProfileResult] = {
+ val jobsWithSQL = app.jobIdToInfo.filter { case (_, j) =>
+ j.sqlID.nonEmpty
+ }
+ val sqlToStages = jobsWithSQL.flatMap { case (jobId, j) =>
+ val stages = j.stageIds
+ val stagesInJob = app.stageManager.getStagesByIds(stages)
+ stagesInJob.map { sModel =>
+ val nodeIds = sqlPlanNodeIdToStageIds.filter { case (_, v) =>
+ v.contains(sModel.sId)
+ }.keys.toSeq
+ val nodeNames = app.sqlPlans.get(j.sqlID.get).map { planInfo =>
+ val nodes = ToolsPlanGraph(planInfo).allNodes
+ val validNodes = nodes.filter { n =>
+ nodeIds.contains((j.sqlID.get, n.id))
+ }
+ validNodes.map(n => s"${n.name}(${n.id.toString})")
+ }.getOrElse(Seq.empty)
+ SQLStageInfoProfileResult(appIndex, j.sqlID.get, jobId, sModel.sId,
+ sModel.attemptId, sModel.duration, nodeNames)
+ }
+ }
+ sqlToStages.toSeq
+ }
+
+ // Store (min, median, max, total) for a given metric
+ private case class statisticsMetrics(min: Long, med:Long, max:Long, total: Long)
+
+ def generateSQLAccums(): Seq[SQLAccumProfileResults] = {
+ allSQLMetrics.flatMap { metric =>
+ val jobsForSql = app.jobIdToInfo.filter { case (_, jc) =>
+ // Avoid getOrElse to reduce memory allocations
+ jc.sqlID.isDefined && jc.sqlID.get == metric.sqlID
+ }
+ val stageIdsForSQL = jobsForSql.flatMap(_._2.stageIds).toSet
+ val accumsOpt = app.taskStageAccumMap.get(metric.accumulatorId)
+ val taskMax = accumsOpt match {
+ case Some(accums) =>
+ val filtered = accums.filter { a =>
+ stageIdsForSQL.contains(a.stageId)
+ }
+ // If metricType is size, average or timing, we want to read field `update` value
+ // to get the min, median, max, and total. Otherwise, we want to use field `value`.
+ if (SQLMetricsStats.hasStats(metric.metricType)) {
+ val accumValues = filtered.map(_.update.getOrElse(0L)).sortWith(_ < _)
+ if (accumValues.isEmpty) {
+ None
+ }
+ else if (accumValues.length <= 1) {
+ Some(statisticsMetrics(0L, 0L, 0L, accumValues.sum))
+ } else {
+ Some(statisticsMetrics(accumValues(0), accumValues(accumValues.size / 2),
+ accumValues(accumValues.size - 1), accumValues.sum))
+ }
+ } else {
+ val accumValues = filtered.map(_.value.getOrElse(0L))
+ if (accumValues.isEmpty) {
+ None
+ } else {
+ Some(statisticsMetrics(0L, 0L, 0L, accumValues.max))
+ }
+ }
+ case None => None
+ }
+
+ // local mode driver gets updates
+ val driverAccumsOpt = app.driverAccumMap.get(metric.accumulatorId)
+ val driverMax = driverAccumsOpt match {
+ case Some(accums) =>
+ val filtered = accums.filter { a =>
+ a.sqlID == metric.sqlID
+ }
+ val accumValues = filtered.map(_.value).sortWith(_ < _)
+ if (accumValues.isEmpty) {
+ None
+ } else if (accumValues.length <= 1) {
+ Some(statisticsMetrics(0L, 0L, 0L, accumValues.sum))
+ } else {
+ Some(statisticsMetrics(accumValues(0), accumValues(accumValues.size / 2),
+ accumValues(accumValues.size - 1), accumValues.sum))
+ }
+ case None =>
+ None
+ }
+
+ if (taskMax.isDefined || driverMax.isDefined) {
+ val taskInfo = taskMax match {
+ case Some(task) => task
+ case None => statisticsMetrics(0L, 0L, 0L, 0L)
+ }
+ val driverInfo = driverMax match {
+ case Some(driver) => driver
+ case None => statisticsMetrics(0L, 0L, 0L, 0L)
+ }
+
+ val max = Math.max(taskInfo.max, driverInfo.max)
+ val min = Math.max(taskInfo.min, driverInfo.min)
+ val med = Math.max(taskInfo.med, driverInfo.med)
+ val total = Math.max(taskInfo.total, driverInfo.total)
+
+ Some(SQLAccumProfileResults(appIndex, metric.sqlID,
+ metric.nodeID, metric.nodeName, metric.accumulatorId, metric.name,
+ min, med, max, total, metric.metricType, metric.stageIds.mkString(",")))
+ } else {
+ None
+ }
+ }
+ }
+}
+
+object AppSQLPlanAnalyzer {
+ def processSQLPlan(app: ApplicationInfo): AppSQLPlanAnalyzer = {
+ val sqlProcessor = new AppSQLPlanAnalyzer(app, app.index)
+ sqlProcessor.processSQLPlanMetrics()
+ sqlProcessor
+ }
+ def apply(app: AppBase, appIndex: Int): AppSQLPlanAnalyzer = {
+ val sqlProcessor = new AppSQLPlanAnalyzer(app, appIndex)
+ sqlProcessor.processSQLPlanMetrics()
+ sqlProcessor
+ }
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala
new file mode 100644
index 000000000..e219ee46b
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala
@@ -0,0 +1,65 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.analysis
+
+import org.apache.spark.sql.rapids.tool.AppBase
+
+// A trait that provides the common methods used for Spark metrics aggregator
+// This is extended by the Qual/Prof aggregators
+trait AppSparkMetricsAggTrait extends AppIndexMapperTrait {
+ /**
+ * Given an application and its index, this methods creates a new appAnalysis
+ * object to aggregate the Raw metrics and returns the result
+ * @param app the AppBase to be analyzed
+ * @param index the application index
+ * @return a single record of AggRawMetricsResult containing all the raw aggregated Spark
+ * metrics
+ */
+ def getAggRawMetrics(app: AppBase, index: Int): AggRawMetricsResult = {
+ val analysisObj = new AppSparkMetricsAnalyzer(app)
+ AggRawMetricsResult(
+ analysisObj.aggregateSparkMetricsByJob(index),
+ analysisObj.aggregateSparkMetricsByStage(index),
+ analysisObj.shuffleSkewCheck(index),
+ analysisObj.aggregateSparkMetricsBySql(index),
+ analysisObj.aggregateIOMetricsBySql(analysisObj.aggregateSparkMetricsBySql(index)),
+ analysisObj.aggregateDurationAndCPUTimeBySql(index),
+ Seq(analysisObj.maxTaskInputSizeBytesPerSQL(index)))
+ }
+
+ /**
+ * Given a list of applications, this method aggregates the raw metrics for all the applications
+ * and returns the results as a single record
+ * @param apps the sequence of the apps to be analyzed
+ * @return a single record of all the aggregated metrics
+ */
+ def getAggregateRawMetrics(
+ apps: Seq[AppBase]): AggRawMetricsResult = {
+ zipAppsWithIndex(apps).map { case (app, index) =>
+ getAggRawMetrics(app, index)
+ }.reduce { (agg1, agg2) =>
+ AggRawMetricsResult(
+ agg1.jobAggs ++ agg2.jobAggs,
+ agg1.stageAggs ++ agg2.stageAggs,
+ agg1.taskShuffleSkew ++ agg2.taskShuffleSkew,
+ agg1.sqlAggs ++ agg2.sqlAggs,
+ agg1.ioAggs ++ agg2.ioAggs,
+ agg1.sqlDurAggs ++ agg2.sqlDurAggs,
+ agg1.maxTaskInputSizes ++ agg2.maxTaskInputSizes)
+ }
+ }
+}
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
new file mode 100644
index 000000000..9e3bf3134
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala
@@ -0,0 +1,391 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.analysis
+
+import scala.collection.mutable
+
+import com.nvidia.spark.rapids.tool.profiling.{IOAnalysisProfileResult, JobStageAggTaskMetricsProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLMaxTaskInputSizes, SQLTaskAggMetricsProfileResult}
+
+import org.apache.spark.sql.rapids.tool.{AppBase, ToolUtils}
+import org.apache.spark.sql.rapids.tool.store.TaskModel
+
+/**
+ * Does analysis on the DataFrames from object of AppBase.
+ * This class does the following:
+ * - aggregates SparkMetrics by Job, Stage, and SQL.
+ * - checks for shuffle skewness.
+ * - Find the max inputSizes for SQL.
+ *
+ * The implementation is tuned to improve the performance by reducing the number of times the
+ * analyzer visits the Tasks.
+ * 1- The assumption is that it is unlikely that the analysis will be aggregating metrics only for
+ * one of for SQL, jobs, or stages. Instead, any analysis is likely to do SQL/Stage levels.
+ * 2- The analyzer caches the stage level metrics to avoid recalculating the same metrics several
+ * times
+ * 3- The cached stage-level metrics are then used to calculate the aggregates for SQLs, and Jobs
+ * 4- It can be used by both Qual/Prof tools: this why it takes app-index as an argument to the
+ * aggregator methods. The index is a value used by the Profiler tool to list records from
+ * multiple applications.
+ *
+ * @param app the AppBase object to analyze
+ */
+class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) {
+ // Hashmap to cache the stage level metrics. It is initialized to None just in case the caller
+ // does not call methods in order starting with stage level metrics.
+ private var stageLevelCache:
+ Option[mutable.LinkedHashMap[Int, JobStageAggTaskMetricsProfileResult]] = None
+
+ // Getter method used to protect the cache from out-of-order calls.
+ // If the stage-level metrics are not generated yet, generates and add them to the cache
+ private def stageLevelSparkMetrics(
+ index: Int): mutable.LinkedHashMap[Int, JobStageAggTaskMetricsProfileResult] = {
+ if (stageLevelCache.isEmpty) {
+ stageLevelCache = Some(mutable.LinkedHashMap[Int, JobStageAggTaskMetricsProfileResult]())
+ aggregateSparkMetricsByStageInternal(index)
+ }
+ stageLevelCache.get
+ }
+
+ /**
+ * Aggregate the SparkMetrics by stage
+ * @param index the App-index (used by the profiler tool)
+ * @return sequence of JobStageAggTaskMetricsProfileResult that contains only Stage Ids
+ */
+ def aggregateSparkMetricsByStage(index: Int): Seq[JobStageAggTaskMetricsProfileResult] = {
+ stageLevelSparkMetrics(index).values.toSeq
+ }
+
+ /**
+ * Aggregate the SparkMetrics by Job
+ * @param index the App-index (used by the profiler tool)
+ * @return sequence of JobStageAggTaskMetricsProfileResult that contains only Job Ids
+ */
+ def aggregateSparkMetricsByJob(index: Int): Seq[JobStageAggTaskMetricsProfileResult] = {
+ val jobRows = app.jobIdToInfo.flatMap { case (id, jc) =>
+ if (jc.stageIds.isEmpty) {
+ None
+ } else {
+ val profResultsInJob = stageLevelSparkMetrics(index).filterKeys(jc.stageIds.contains).values
+ if (profResultsInJob.isEmpty) {
+ None
+ } else {
+ // Recalculate the duration sum, max, min, avg for the job based on the cached
+ // stage Profiling results
+ val tasksInJob = profResultsInJob.map(_.numTasks).sum
+ val durSum = profResultsInJob.map(_.durationSum).sum
+ val durMax =
+ AppSparkMetricsAnalyzer.maxWithEmptyHandling(profResultsInJob.map(_.durationMax))
+ val durMin =
+ AppSparkMetricsAnalyzer.minWithEmptyHandling(profResultsInJob.map(_.durationMin))
+ val durAvg = ToolUtils.calculateAverage(durSum, tasksInJob, 1)
+ Some(JobStageAggTaskMetricsProfileResult(index,
+ s"job_$id",
+ tasksInJob,
+ jc.duration,
+ profResultsInJob.map(_.diskBytesSpilledSum).sum,
+ durSum,
+ durMax,
+ durMin,
+ durAvg,
+ profResultsInJob.map(_.executorCPUTimeSum).sum,
+ profResultsInJob.map(_.executorDeserializeCpuTimeSum).sum,
+ profResultsInJob.map(_.executorDeserializeTimeSum).sum,
+ profResultsInJob.map(_.executorRunTimeSum).sum,
+ profResultsInJob.map(_.inputBytesReadSum).sum,
+ profResultsInJob.map(_.inputRecordsReadSum).sum,
+ profResultsInJob.map(_.jvmGCTimeSum).sum,
+ profResultsInJob.map(_.memoryBytesSpilledSum).sum,
+ profResultsInJob.map(_.outputBytesWrittenSum).sum,
+ profResultsInJob.map(_.outputRecordsWrittenSum).sum,
+ AppSparkMetricsAnalyzer.maxWithEmptyHandling(
+ profResultsInJob.map(_.peakExecutionMemoryMax)),
+ profResultsInJob.map(_.resultSerializationTimeSum).sum,
+ AppSparkMetricsAnalyzer.maxWithEmptyHandling(profResultsInJob.map(_.resultSizeMax)),
+ profResultsInJob.map(_.srFetchWaitTimeSum).sum,
+ profResultsInJob.map(_.srLocalBlocksFetchedSum).sum,
+ profResultsInJob.map(_.srcLocalBytesReadSum).sum,
+ profResultsInJob.map(_.srRemoteBlocksFetchSum).sum,
+ profResultsInJob.map(_.srRemoteBytesReadSum).sum,
+ profResultsInJob.map(_.srRemoteBytesReadToDiskSum).sum,
+ profResultsInJob.map(_.srTotalBytesReadSum).sum,
+ profResultsInJob.map(_.swBytesWrittenSum).sum,
+ profResultsInJob.map(_.swRecordsWrittenSum).sum,
+ profResultsInJob.map(_.swWriteTimeSum).sum))
+ }
+ }
+ }
+ jobRows.toSeq
+ }
+
+ private case class AverageStageInfo(avgDuration: Double, avgShuffleReadBytes: Double)
+
+ /**
+ * Scans tasks to identify if any exhibits shuffle skewness. If a task has input size larger than
+ * 3X the average shuffle read size and larger than 100MB, it is considered as a skew task.
+ * @param index the App-index (used by the profiler tool)
+ * @return sequence of ShuffleSkewProfileResult that contains only the skew tasks
+ */
+ def shuffleSkewCheck(index: Int): Seq[ShuffleSkewProfileResult] = {
+ // TODO: we can add averageShuffleRead as a field in JobStageAggTaskMetricsProfileResult instead
+ // of making an extra path on the StageAttempts
+ val avgStageInfos = app.taskManager.stageAttemptToTasks.collect {
+ // TODO: Should we only consider successful tasks?
+ case (stageId, attemptsToTasks) if attemptsToTasks.nonEmpty =>
+ attemptsToTasks.map { case (attemptId, tasks) =>
+ val sumDuration = tasks.map(_.duration).sum
+ val avgDuration = ToolUtils.calculateAverage(sumDuration, tasks.size, 2)
+ val sumShuffleReadBytes = tasks.map(_.sr_totalBytesRead).sum
+ val avgShuffleReadBytes = ToolUtils.calculateAverage(sumShuffleReadBytes, tasks.size, 2)
+ ((stageId, attemptId), AverageStageInfo(avgDuration, avgShuffleReadBytes))
+ }
+ }.flatten
+
+ avgStageInfos.flatMap { case ((stageId, attemptId), avg) =>
+ val definedTasks =
+ app.taskManager.getTasks(stageId, attemptId, Some(
+ tc => (tc.sr_totalBytesRead > 3 * avg.avgShuffleReadBytes)
+ && (tc.sr_totalBytesRead > 100 * 1024 * 1024)))
+ definedTasks.map { tc =>
+ ShuffleSkewProfileResult(index, stageId, attemptId,
+ tc.taskId, tc.attempt, tc.duration, avg.avgDuration, tc.sr_totalBytesRead,
+ avg.avgShuffleReadBytes, tc.peakExecutionMemory, tc.successful, tc.endReason)
+ }
+ }.toSeq
+ }
+
+ /**
+ * Aggregate the SparkMetrics by SQL
+ * @param index the App-index (used by the profiler tool)
+ * @return sequence of SQLTaskAggMetricsProfileResult
+ */
+ def aggregateSparkMetricsBySql(index: Int): Seq[SQLTaskAggMetricsProfileResult] = {
+ val sqlRows = app.sqlIdToInfo.flatMap { case (sqlId, sqlCase) =>
+ if (app.sqlIdToStages.contains(sqlId)) {
+ val stagesInSQL = app.sqlIdToStages(sqlId)
+ // TODO: Should we only consider successful tasks?
+ val cachedResBySQL = stageLevelSparkMetrics(index).filterKeys(stagesInSQL.contains).values
+ if (cachedResBySQL.isEmpty) {
+ None
+ } else {
+ // Recalculate the duration sum, max, min, avg for the job based on the cached
+ // stage Profiling results
+ val tasksInSql = cachedResBySQL.map(_.numTasks).sum
+ val durSum = cachedResBySQL.map(_.durationSum).sum
+ val durMax =
+ AppSparkMetricsAnalyzer.maxWithEmptyHandling(cachedResBySQL.map(_.durationMax))
+ val durMin =
+ AppSparkMetricsAnalyzer.minWithEmptyHandling(cachedResBySQL.map(_.durationMin))
+ val durAvg = ToolUtils.calculateAverage(durSum, tasksInSql, 1)
+ val diskBytes = cachedResBySQL.map(_.diskBytesSpilledSum).sum
+ val execCpuTime = cachedResBySQL.map(_.executorCPUTimeSum).sum
+ val execRunTime = cachedResBySQL.map(_.executorRunTimeSum).sum
+ val execCPURatio = ToolUtils.calculateDurationPercent(execCpuTime, execRunTime)
+ val inputBytesRead = cachedResBySQL.map(_.inputBytesReadSum).sum
+ // set this here, so make sure we don't get it again until later
+ sqlCase.sqlCpuTimePercent = execCPURatio
+
+ Some(SQLTaskAggMetricsProfileResult(index,
+ app.appId,
+ sqlId,
+ sqlCase.description,
+ tasksInSql,
+ sqlCase.duration,
+ execCpuTime,
+ execRunTime,
+ execCPURatio,
+ diskBytes,
+ durSum,
+ durMax,
+ durMin,
+ durAvg,
+ execCpuTime,
+ cachedResBySQL.map(_.executorDeserializeCpuTimeSum).sum,
+ cachedResBySQL.map(_.executorDeserializeTimeSum).sum,
+ execRunTime,
+ inputBytesRead,
+ inputBytesRead * 1.0 / tasksInSql,
+ cachedResBySQL.map(_.inputRecordsReadSum).sum,
+ cachedResBySQL.map(_.jvmGCTimeSum).sum,
+ cachedResBySQL.map(_.memoryBytesSpilledSum).sum,
+ cachedResBySQL.map(_.outputBytesWrittenSum).sum,
+ cachedResBySQL.map(_.outputRecordsWrittenSum).sum,
+ AppSparkMetricsAnalyzer.maxWithEmptyHandling(
+ cachedResBySQL.map(_.peakExecutionMemoryMax)),
+ cachedResBySQL.map(_.resultSerializationTimeSum).sum,
+ AppSparkMetricsAnalyzer.maxWithEmptyHandling(cachedResBySQL.map(_.resultSizeMax)),
+ cachedResBySQL.map(_.srFetchWaitTimeSum).sum,
+ cachedResBySQL.map(_.srLocalBlocksFetchedSum).sum,
+ cachedResBySQL.map(_.srcLocalBytesReadSum).sum,
+ cachedResBySQL.map(_.srRemoteBlocksFetchSum).sum,
+ cachedResBySQL.map(_.srRemoteBytesReadSum).sum,
+ cachedResBySQL.map(_.srRemoteBytesReadToDiskSum).sum,
+ cachedResBySQL.map(_.srTotalBytesReadSum).sum,
+ cachedResBySQL.map(_.swBytesWrittenSum).sum,
+ cachedResBySQL.map(_.swRecordsWrittenSum).sum,
+ cachedResBySQL.map(_.swWriteTimeSum).sum))
+ }
+ } else {
+ None
+ }
+ }
+ sqlRows.toSeq
+ }
+
+ /**
+ * Aggregates the IO metrics by SQL
+ * @param sqlMetricsAggs Spark metrics the aggregated by SQL. This is an optimization tuning to
+ * avoid recalculating those metrics twice.
+ * @return IOAnalysisProfileResult that contains the IO metrics aggregated by SQL
+ */
+ def aggregateIOMetricsBySql(
+ sqlMetricsAggs: Seq[SQLTaskAggMetricsProfileResult]): Seq[IOAnalysisProfileResult] = {
+ val sqlIORows = sqlMetricsAggs.map { sqlAgg =>
+ IOAnalysisProfileResult(sqlAgg.appIndex,
+ app.appId,
+ sqlAgg.sqlId,
+ sqlAgg.inputBytesReadSum,
+ sqlAgg.inputRecordsReadSum,
+ sqlAgg.outputBytesWrittenSum,
+ sqlAgg.outputRecordsWrittenSum,
+ sqlAgg.diskBytesSpilledSum,
+ sqlAgg.memoryBytesSpilledSum,
+ sqlAgg.srTotalBytesReadSum,
+ sqlAgg.swBytesWrittenSum)
+ }
+ sqlIORows
+ }
+
+ /**
+ * Find the maximum task input size
+ * @param index App index (used by the profiler tool)
+ * @return a single SQLMaxTaskInputSizes record that contains the maximum value. If none, it will
+ * be 0L
+ */
+ def maxTaskInputSizeBytesPerSQL(index: Int): SQLMaxTaskInputSizes = {
+ // TODO: We should keep maxInputSize as a field in the stageAggregate to avoid doing an
+ // extra path on the tasks
+ val maxOfSqls = app.sqlIdToStages.map { case (_, stageIds) =>
+ // TODO: Should we only consider successful tasks?
+ val tasksInSQL = app.taskManager.getTasksByStageIds(stageIds)
+ if (tasksInSQL.isEmpty) {
+ 0L
+ } else {
+ tasksInSQL.map(_.input_bytesRead).max
+ }
+ }
+ val maxVal = if (maxOfSqls.nonEmpty) {
+ maxOfSqls.max
+ } else {
+ 0L
+ }
+ SQLMaxTaskInputSizes(index, app.appId, maxVal)
+ }
+
+ /**
+ * Aggregates the duration and CPU time (milliseconds) by SQL
+ * @param index App index (used by the profiler tool)
+ * @return a sequence of SQLDurationExecutorTimeProfileResult or Empty if None.
+ */
+ def aggregateDurationAndCPUTimeBySql(index: Int): Seq[SQLDurationExecutorTimeProfileResult] = {
+ val sqlRows = app.sqlIdToInfo.map { case (sqlId, sqlCase) =>
+ SQLDurationExecutorTimeProfileResult(index, app.appId, sqlCase.rootExecutionID,
+ sqlId, sqlCase.duration, sqlCase.hasDatasetOrRDD,
+ app.getAppDuration.orElse(Option(0L)), sqlCase.problematic,
+ sqlCase.sqlCpuTimePercent)
+ }
+ sqlRows.toSeq
+ }
+
+ /**
+ * Aggregates the SparkMetrics by stage. This is an internal method to populate the cached metrics
+ * to be used by other aggregators.
+ * @param index AppIndex (used by the profiler tool)
+ */
+ private def aggregateSparkMetricsByStageInternal(index: Int): Unit = {
+ // TODO: this has stage attempts. we should handle different attempts
+ app.stageManager.getAllStages.foreach { sm =>
+ // TODO: Should we only consider successful tasks?
+ val tasksInStage = app.taskManager.getTasks(sm.sId, sm.attemptId)
+ // count duplicate task attempts
+ val numAttempts = tasksInStage.size
+ val (durSum, durMax, durMin, durAvg) = AppSparkMetricsAnalyzer.getDurations(tasksInStage)
+ val stageRow = JobStageAggTaskMetricsProfileResult(index,
+ s"stage_${sm.sId}",
+ numAttempts,
+ sm.duration,
+ tasksInStage.map(_.diskBytesSpilled).sum,
+ durSum,
+ durMax,
+ durMin,
+ durAvg,
+ tasksInStage.map(_.executorCPUTime).sum,
+ tasksInStage.map(_.executorDeserializeCPUTime).sum,
+ tasksInStage.map(_.executorDeserializeTime).sum,
+ tasksInStage.map(_.executorRunTime).sum,
+ tasksInStage.map(_.input_bytesRead).sum,
+ tasksInStage.map(_.input_recordsRead).sum,
+ tasksInStage.map(_.jvmGCTime).sum,
+ tasksInStage.map(_.memoryBytesSpilled).sum,
+ tasksInStage.map(_.output_bytesWritten).sum,
+ tasksInStage.map(_.output_recordsWritten).sum,
+ AppSparkMetricsAnalyzer.maxWithEmptyHandling(tasksInStage.map(_.peakExecutionMemory)),
+ tasksInStage.map(_.resultSerializationTime).sum,
+ AppSparkMetricsAnalyzer.maxWithEmptyHandling(tasksInStage.map(_.resultSize)),
+ tasksInStage.map(_.sr_fetchWaitTime).sum,
+ tasksInStage.map(_.sr_localBlocksFetched).sum,
+ tasksInStage.map(_.sr_localBytesRead).sum,
+ tasksInStage.map(_.sr_remoteBlocksFetched).sum,
+ tasksInStage.map(_.sr_remoteBytesRead).sum,
+ tasksInStage.map(_.sr_remoteBytesReadToDisk).sum,
+ tasksInStage.map(_.sr_totalBytesRead).sum,
+ tasksInStage.map(_.sw_bytesWritten).sum,
+ tasksInStage.map(_.sw_recordsWritten).sum,
+ tasksInStage.map(_.sw_writeTime).sum
+ )
+ stageLevelSparkMetrics(index).put(sm.sId, stageRow)
+ }
+ }
+}
+
+
+object AppSparkMetricsAnalyzer {
+ def getDurations(tcs: Iterable[TaskModel]): (Long, Long, Long, Double) = {
+ val durations = tcs.map(_.duration)
+ if (durations.nonEmpty) {
+ (durations.sum, durations.max, durations.min,
+ ToolUtils.calculateAverage(durations.sum, durations.size, 1))
+ } else {
+ (0L, 0L, 0L, 0.toDouble)
+ }
+ }
+
+ def maxWithEmptyHandling(arr: Iterable[Long]): Long = {
+ if (arr.isEmpty) {
+ 0L
+ } else {
+ arr.max
+ }
+ }
+
+ def minWithEmptyHandling(arr: Iterable[Long]): Long = {
+ if (arr.isEmpty) {
+ 0L
+ } else {
+ arr.min
+ }
+ }
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/ProfSparkMetricsAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/ProfSparkMetricsAnalyzer.scala
new file mode 100644
index 000000000..a173fa703
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/ProfSparkMetricsAnalyzer.scala
@@ -0,0 +1,25 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.analysis
+
+// This object is kept to provide the aggregation of the application data for the profiler.
+// Currently, the difference is not significant because `ProfAppIndexMapperTrait` handles the
+// extraction of AppIndex from ApplicationInfo. However, in the future this object can be used
+// to provide customized logic for the Profiler (i.e., handle metrics specific to GPU eventlogs)
+object ProfSparkMetricsAnalyzer extends AppSparkMetricsAggTrait with ProfAppIndexMapperTrait {
+
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/QualSparkMetricsAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/QualSparkMetricsAnalyzer.scala
new file mode 100644
index 000000000..1debfaa1b
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/QualSparkMetricsAnalyzer.scala
@@ -0,0 +1,22 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.nvidia.spark.rapids.tool.analysis
+
+object QualSparkMetricsAnalyzer extends AppSparkMetricsAggTrait with QualAppIndexMapperTrait {
+ // This object is kept to provide the aggregation of the application data for the Qualification.
+ // In future, we might need to provide customized logic for the Qualification
+ // (i.e., handle metrics; or filter; ..etc)
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/package.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/package.scala
new file mode 100644
index 000000000..fa352c1c5
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/package.scala
@@ -0,0 +1,26 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool
+
+/**
+ * RAPIDS tools analysis package holds the implementation of classes that process the information
+ * gathered from the eventlogs to transform it into meaningful data which is consumed by the
+ * reporting/heuristics engines.
+ */
+package object analysis {
+
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Analysis.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Analysis.scala
deleted file mode 100644
index fb6efee5d..000000000
--- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Analysis.scala
+++ /dev/null
@@ -1,379 +0,0 @@
-/*
- * Copyright (c) 2021-2024, NVIDIA CORPORATION.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package com.nvidia.spark.rapids.tool.profiling
-
-import scala.collection.mutable
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.spark.sql.rapids.tool.ToolUtils
-import org.apache.spark.sql.rapids.tool.profiling._
-
-/**
- * Does analysis on the DataFrames
- * from object of ApplicationInfo
- */
-class Analysis(apps: Seq[ApplicationInfo]) {
-
- def getDurations(tcs: ArrayBuffer[TaskCase]): (Long, Long, Long, Double) = {
- val durations = tcs.map(_.duration)
- if (durations.nonEmpty ) {
- (durations.sum, durations.max, durations.min,
- ToolUtils.calculateAverage(durations.sum, durations.size, 1))
- } else {
- (0L, 0L, 0L, 0.toDouble)
- }
- }
-
- private def maxWithEmptyHandling(arr: Iterable[Long]): Long = {
- if (arr.isEmpty) {
- 0L
- } else {
- arr.max
- }
- }
-
- private def minWithEmptyHandling(arr: Iterable[Long]): Long = {
- if (arr.isEmpty) {
- 0L
- } else {
- arr.min
- }
- }
-
- // Job + Stage Level TaskMetrics Aggregation
- def jobAndStageMetricsAggregation(): Seq[JobStageAggTaskMetricsProfileResult] = {
- // first get all stage aggregated levels
- val allRows = apps.flatMap { app =>
- // create a cache of stage rows to be used by the job aggregator
- val cachedStageRows = new mutable.LinkedHashMap[Int, JobStageAggTaskMetricsProfileResult]()
- app.stageManager.getAllStages.foreach { case sm =>
- val tasksInStage = app.taskEnd.filter { tc =>
- tc.stageId == sm.sId && tc.stageAttemptId == sm.attemptId
- }
- // count duplicate task attempts
- val numAttempts = tasksInStage.size
- val (durSum, durMax, durMin, durAvg) = getDurations(tasksInStage)
- val stageRow = JobStageAggTaskMetricsProfileResult(app.index,
- s"stage_${sm.sId}",
- numAttempts,
- sm.duration,
- tasksInStage.map(_.diskBytesSpilled).sum,
- durSum,
- durMax,
- durMin,
- durAvg,
- tasksInStage.map(_.executorCPUTime).sum,
- tasksInStage.map(_.executorDeserializeCPUTime).sum,
- tasksInStage.map(_.executorDeserializeTime).sum,
- tasksInStage.map(_.executorRunTime).sum,
- tasksInStage.map(_.input_bytesRead).sum,
- tasksInStage.map(_.input_recordsRead).sum,
- tasksInStage.map(_.jvmGCTime).sum,
- tasksInStage.map(_.memoryBytesSpilled).sum,
- tasksInStage.map(_.output_bytesWritten).sum,
- tasksInStage.map(_.output_recordsWritten).sum,
- maxWithEmptyHandling(tasksInStage.map(_.peakExecutionMemory)),
- tasksInStage.map(_.resultSerializationTime).sum,
- maxWithEmptyHandling(tasksInStage.map(_.resultSize)),
- tasksInStage.map(_.sr_fetchWaitTime).sum,
- tasksInStage.map(_.sr_localBlocksFetched).sum,
- tasksInStage.map(_.sr_localBytesRead).sum,
- tasksInStage.map(_.sr_remoteBlocksFetched).sum,
- tasksInStage.map(_.sr_remoteBytesRead).sum,
- tasksInStage.map(_.sr_remoteBytesReadToDisk).sum,
- tasksInStage.map(_.sr_totalBytesRead).sum,
- tasksInStage.map(_.sw_bytesWritten).sum,
- tasksInStage.map(_.sw_recordsWritten).sum,
- tasksInStage.map(_.sw_writeTime).sum
- )
- // cache the stage row to be used later
- cachedStageRows.put(sm.sId, stageRow)
- }
- // Aggregate all the stages by job
- val jobRows = app.jobIdToInfo.map { case (id, jc) =>
- if (jc.stageIds.isEmpty) {
- None
- } else {
- val profResultsInJob = cachedStageRows.filterKeys(jc.stageIds.contains).values
- if (profResultsInJob.isEmpty) {
- None
- } else {
- // Recalculate the duration sum, max, min, avg for the job based on the cached
- // stage Profiling results
- val tasksInJob = profResultsInJob.map(_.numTasks).sum
- val durSum = profResultsInJob.map(_.durationSum).sum
- val durMax = maxWithEmptyHandling(profResultsInJob.map(_.durationMax))
- val durMin = minWithEmptyHandling(profResultsInJob.map(_.durationMin))
- val durAvg = ToolUtils.calculateAverage(durSum, tasksInJob, 1)
- Some(JobStageAggTaskMetricsProfileResult(app.index,
- s"job_$id",
- tasksInJob,
- jc.duration,
- profResultsInJob.map(_.diskBytesSpilledSum).sum,
- durSum,
- durMax,
- durMin,
- durAvg,
- profResultsInJob.map(_.executorCPUTimeSum).sum,
- profResultsInJob.map(_.executorDeserializeCpuTimeSum).sum,
- profResultsInJob.map(_.executorDeserializeTimeSum).sum,
- profResultsInJob.map(_.executorRunTimeSum).sum,
- profResultsInJob.map(_.inputBytesReadSum).sum,
- profResultsInJob.map(_.inputRecordsReadSum).sum,
- profResultsInJob.map(_.jvmGCTimeSum).sum,
- profResultsInJob.map(_.memoryBytesSpilledSum).sum,
- profResultsInJob.map(_.outputBytesWrittenSum).sum,
- profResultsInJob.map(_.outputRecordsWrittenSum).sum,
- maxWithEmptyHandling(profResultsInJob.map(_.peakExecutionMemoryMax)),
- profResultsInJob.map(_.resultSerializationTimeSum).sum,
- maxWithEmptyHandling(profResultsInJob.map(_.resultSizeMax)),
- profResultsInJob.map(_.srFetchWaitTimeSum).sum,
- profResultsInJob.map(_.srLocalBlocksFetchedSum).sum,
- profResultsInJob.map(_.srcLocalBytesReadSum).sum,
- profResultsInJob.map(_.srRemoteBlocksFetchSum).sum,
- profResultsInJob.map(_.srRemoteBytesReadSum).sum,
- profResultsInJob.map(_.srRemoteBytesReadToDiskSum).sum,
- profResultsInJob.map(_.srTotalBytesReadSum).sum,
- profResultsInJob.map(_.swBytesWrittenSum).sum,
- profResultsInJob.map(_.swRecordsWrittenSum).sum,
- profResultsInJob.map(_.swWriteTimeSum).sum))
- }
- }
- }
- cachedStageRows.values ++ jobRows.flatMap(row => row)
- }
-
- if (allRows.nonEmpty) {
- val sortedRows = allRows.sortBy { cols =>
- val sortDur = cols.duration.getOrElse(0L)
- (cols.appIndex, -sortDur, cols.id)
- }
- sortedRows
- } else {
- Seq.empty
- }
- }
-
- // SQL Level TaskMetrics Aggregation(Only when SQL exists)
- def sqlMetricsAggregation(): Seq[SQLTaskAggMetricsProfileResult] = {
- val allRows = apps.flatMap { app =>
- app.sqlIdToInfo.map { case (sqlId, sqlCase) =>
- if (app.sqlIdToStages.contains(sqlId)) {
- val stagesInSQL = app.sqlIdToStages(sqlId)
- val tasksInSQL = app.taskEnd.filter { tc =>
- stagesInSQL.contains(tc.stageId)
- }
- if (tasksInSQL.isEmpty) {
- None
- } else {
- // count all attempts
- val numAttempts = tasksInSQL.size
-
- val diskBytes = tasksInSQL.map(_.diskBytesSpilled).sum
- val execCpuTime = tasksInSQL.map(_.executorCPUTime).sum
- val execRunTime = tasksInSQL.map(_.executorRunTime).sum
- val execCPURatio = ToolUtils.calculateDurationPercent(execCpuTime, execRunTime)
-
- // set this here, so make sure we don't get it again until later
- sqlCase.sqlCpuTimePercent = execCPURatio
-
- val (durSum, durMax, durMin, durAvg) = getDurations(tasksInSQL)
- Some(SQLTaskAggMetricsProfileResult(app.index,
- app.appId,
- sqlId,
- sqlCase.description,
- numAttempts,
- sqlCase.duration,
- execCpuTime,
- execRunTime,
- execCPURatio,
- diskBytes,
- durSum,
- durMax,
- durMin,
- durAvg,
- execCpuTime,
- tasksInSQL.map(_.executorDeserializeCPUTime).sum,
- tasksInSQL.map(_.executorDeserializeTime).sum,
- execRunTime,
- tasksInSQL.map(_.input_bytesRead).sum,
- tasksInSQL.map(_.input_bytesRead).sum * 1.0 / tasksInSQL.size,
- tasksInSQL.map(_.input_recordsRead).sum,
- tasksInSQL.map(_.jvmGCTime).sum,
- tasksInSQL.map(_.memoryBytesSpilled).sum,
- tasksInSQL.map(_.output_bytesWritten).sum,
- tasksInSQL.map(_.output_recordsWritten).sum,
- maxWithEmptyHandling(tasksInSQL.map(_.peakExecutionMemory)),
- tasksInSQL.map(_.resultSerializationTime).sum,
- maxWithEmptyHandling(tasksInSQL.map(_.resultSize)),
- tasksInSQL.map(_.sr_fetchWaitTime).sum,
- tasksInSQL.map(_.sr_localBlocksFetched).sum,
- tasksInSQL.map(_.sr_localBytesRead).sum,
- tasksInSQL.map(_.sr_remoteBlocksFetched).sum,
- tasksInSQL.map(_.sr_remoteBytesRead).sum,
- tasksInSQL.map(_.sr_remoteBytesReadToDisk).sum,
- tasksInSQL.map(_.sr_totalBytesRead).sum,
- tasksInSQL.map(_.sw_bytesWritten).sum,
- tasksInSQL.map(_.sw_recordsWritten).sum,
- tasksInSQL.map(_.sw_writeTime).sum
- ))
- }
- } else {
- None
- }
- }
- }
- val allFiltered = allRows.flatMap(row => row)
- if (allFiltered.size > 0) {
- val sortedRows = allFiltered.sortBy { cols =>
- val sortDur = cols.duration.getOrElse(0L)
- (cols.appIndex, -(sortDur), cols.sqlId, cols.executorCpuTime)
- }
- sortedRows
- } else {
- Seq.empty
- }
- }
-
- def ioAnalysis(): Seq[IOAnalysisProfileResult] = {
- val allRows = apps.flatMap { app =>
- app.sqlIdToStages.map {
- case (sqlId, stageIds) =>
- val tasksInSQL = app.taskEnd.filter { tc =>
- stageIds.contains(tc.stageId)
- }
- if (tasksInSQL.isEmpty) {
- None
- } else {
- val diskBytes = tasksInSQL.map(_.diskBytesSpilled).sum
- Some(IOAnalysisProfileResult(app.index,
- app.appId,
- sqlId,
- tasksInSQL.map(_.input_bytesRead).sum,
- tasksInSQL.map(_.input_recordsRead).sum,
- tasksInSQL.map(_.output_bytesWritten).sum,
- tasksInSQL.map(_.output_recordsWritten).sum,
- diskBytes,
- tasksInSQL.map(_.memoryBytesSpilled).sum,
- tasksInSQL.map(_.sr_totalBytesRead).sum,
- tasksInSQL.map(_.sw_bytesWritten).sum
- ))
- }
- }
- }
- val allFiltered = allRows.flatMap(row => row)
- if (allFiltered.size > 0) {
- val sortedRows = allFiltered.sortBy { cols =>
- (cols.appIndex, cols.sqlId)
- }
- sortedRows
- } else {
- Seq.empty
- }
- }
-
- def getMaxTaskInputSizeBytes(): Seq[SQLMaxTaskInputSizes] = {
- apps.map { app =>
- val maxOfSqls = app.sqlIdToStages.map {
- case (_, stageIds) =>
- val tasksInSQL = app.taskEnd.filter { tc =>
- stageIds.contains(tc.stageId)
- }
- if (tasksInSQL.isEmpty) {
- 0L
- } else {
- tasksInSQL.map(_.input_bytesRead).max
- }
- }
- val maxVal = if (maxOfSqls.nonEmpty) {
- maxOfSqls.max
- } else {
- 0
- }
- SQLMaxTaskInputSizes(app.index, app.appId, maxVal)
- }
- }
-
- def sqlMetricsAggregationDurationAndCpuTime(): Seq[SQLDurationExecutorTimeProfileResult] = {
- val allRows = apps.flatMap { app =>
- app.sqlIdToInfo.map { case (sqlId, sqlCase) =>
- SQLDurationExecutorTimeProfileResult(app.index, app.appId, sqlCase.rootExecutionID,
- sqlId, sqlCase.duration, sqlCase.hasDatasetOrRDD,
- app.getAppDuration.orElse(Option(0L)), sqlCase.problematic,
- sqlCase.sqlCpuTimePercent)
- }
- }
- if (allRows.nonEmpty) {
- val sortedRows = allRows.sortBy { cols =>
- val sortDur = cols.duration.getOrElse(0L)
- (cols.appIndex, cols.sqlID, sortDur)
- }
- sortedRows
- } else {
- Seq.empty
- }
- }
-
- private case class AverageStageInfo(avgDuration: Double, avgShuffleReadBytes: Double)
-
- def shuffleSkewCheck(): Seq[ShuffleSkewProfileResult] = {
- val allRows = apps.flatMap { app =>
- val tasksPerStageAttempt = app.taskEnd.groupBy { tc =>
- (tc.stageId, tc.stageAttemptId)
- }
- val avgsStageInfos = tasksPerStageAttempt.map { case ((sId, saId), tcArr) =>
- val sumDuration = tcArr.map(_.duration).sum
- val avgDuration = ToolUtils.calculateAverage(sumDuration, tcArr.size, 2)
- val sumShuffleReadBytes = tcArr.map(_.sr_totalBytesRead).sum
- val avgShuffleReadBytes = ToolUtils.calculateAverage(sumShuffleReadBytes, tcArr.size, 2)
- ((sId, saId), AverageStageInfo(avgDuration, avgShuffleReadBytes))
- }
-
- val tasksWithSkew = app.taskEnd.filter { tc =>
- val avgShuffleDur = avgsStageInfos.get((tc.stageId, tc.stageAttemptId))
- avgShuffleDur match {
- case Some(avg) =>
- (tc.sr_totalBytesRead > 3 * avg.avgShuffleReadBytes) &&
- (tc.sr_totalBytesRead > 100 * 1024 * 1024)
- case None => false
- }
- }
-
- tasksWithSkew.map { tc =>
- val avgShuffleDur = avgsStageInfos.get((tc.stageId, tc.stageAttemptId))
- avgShuffleDur match {
- case Some(avg) =>
- Some(ShuffleSkewProfileResult(app.index, tc.stageId, tc.stageAttemptId,
- tc.taskId, tc.attempt, tc.duration, avg.avgDuration, tc.sr_totalBytesRead,
- avg.avgShuffleReadBytes, tc.peakExecutionMemory, tc.successful, tc.endReason))
- case None =>
- None
- }
- }
- }
-
- val allNonEmptyRows = allRows.flatMap(row => row)
- if (allNonEmptyRows.nonEmpty) {
- val sortedRows = allNonEmptyRows.sortBy { cols =>
- (cols.appIndex, cols.stageId, cols.stageAttemptId, cols.taskId, cols.taskAttemptId)
- }
- sortedRows
- } else {
- Seq.empty
- }
- }
-}
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 99a28f64e..9d6486eb1 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
@@ -19,10 +19,10 @@ package com.nvidia.spark.rapids.tool.profiling
import scala.collection.mutable.{ArrayBuffer, HashMap}
import com.nvidia.spark.rapids.tool.ToolTextFileWriter
+import com.nvidia.spark.rapids.tool.views.{ProfExecutorView, ProfJobsView, ProfSQLCodeGenView, ProfSQLPlanMetricsView, ProfSQLToStageView}
import org.apache.spark.internal.Logging
-import org.apache.spark.resource.ResourceProfile
-import org.apache.spark.sql.rapids.tool.{SQLMetricsStats, ToolUtils}
+import org.apache.spark.sql.rapids.tool.ToolUtils
import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo
case class StageMetrics(numTasks: Int, duration: String)
@@ -146,67 +146,16 @@ class CollectInformation(apps: Seq[ApplicationInfo]) extends Logging {
// get executor related information
def getExecutorInfo: Seq[ExecutorInfoProfileResult] = {
- val allRows = apps.flatMap { app =>
- // first see if any executors have different resourceProfile ids
- val groupedExecs = app.executorIdToInfo.groupBy(_._2.resourceProfileId)
- groupedExecs.map { case (rpId, execs) =>
- val rp = app.resourceProfIdToInfo.get(rpId)
- val execMem = rp.map(_.executorResources.get(ResourceProfile.MEMORY)
- .map(_.amount).getOrElse(0L))
- val execGpus = rp.map(_.executorResources.get("gpu")
- .map(_.amount).getOrElse(0L))
- val taskCpus = rp.map(_.taskResources.get(ResourceProfile.CPUS)
- .map(_.amount).getOrElse(0.toDouble))
- val taskGpus = rp.map(_.taskResources.get("gpu").map(_.amount).getOrElse(0.toDouble))
- val execOffHeap = rp.map(_.executorResources.get(ResourceProfile.OFFHEAP_MEM)
- .map(_.amount).getOrElse(0L))
-
- val numExecutors = execs.size
- val exec = execs.head._2
- // We could print a lot more information here if we decided, more like the Spark UI
- // per executor info.
- ExecutorInfoProfileResult(app.index, rpId, numExecutors,
- exec.totalCores, exec.maxMemory, exec.totalOnHeap,
- exec.totalOffHeap, execMem, execGpus, execOffHeap, taskCpus, taskGpus)
- }
- }
-
- if (allRows.size > 0) {
- allRows.sortBy(cols => (cols.appIndex, cols.numExecutors))
- } else {
- Seq.empty
- }
+ ProfExecutorView.getRawView(apps)
}
// get job related information
def getJobInfo: Seq[JobInfoProfileResult] = {
- val allRows = apps.flatMap { app =>
- app.jobIdToInfo.map { case (_, j) =>
- JobInfoProfileResult(app.index, j.jobID, j.stageIds, j.sqlID, j.startTime, j.endTime)
- }
- }
- if (allRows.size > 0) {
- allRows.sortBy(cols => (cols.appIndex, cols.jobID))
- } else {
- Seq.empty
- }
+ ProfJobsView.getRawView(apps)
}
def getSQLToStage: Seq[SQLStageInfoProfileResult] = {
- val allRows = apps.flatMap { app =>
- app.aggregateSQLStageInfo
- }
- if (allRows.size > 0) {
- case class Reverse[T](t: T)
- implicit def ReverseOrdering[T: Ordering]: Ordering[Reverse[T]] =
- Ordering[T].reverse.on(_.t)
-
- // intentionally sort this table by the duration to be able to quickly
- // see the stage that took the longest
- allRows.sortBy(cols => (cols.appIndex, Reverse(cols.duration)))
- } else {
- Seq.empty
- }
+ ProfSQLToStageView.getRawView(apps)
}
/**
@@ -257,25 +206,12 @@ class CollectInformation(apps: Seq[ApplicationInfo]) extends Logging {
// Print SQL whole stage code gen mapping
def getWholeStageCodeGenMapping: Seq[WholeStageCodeGenResults] = {
- val allWholeStages = apps.flatMap { app =>
- app.wholeStage
- }
- if (allWholeStages.nonEmpty) {
- allWholeStages.sortBy(cols => (cols.appIndex, cols.sqlID, cols.nodeID))
- } else {
- Seq.empty
- }
+ ProfSQLCodeGenView.getRawView(apps)
}
// Print SQL Plan Metrics
def getSQLPlanMetrics: Seq[SQLAccumProfileResults] = {
- val sqlAccums = CollectInformation.generateSQLAccums(apps)
- if (sqlAccums.nonEmpty) {
- sqlAccums.sortBy(cols => (cols.appIndex, cols.sqlID, cols.nodeID,
- cols.nodeName, cols.accumulatorId, cols.metricType))
- } else {
- Seq.empty
- }
+ ProfSQLPlanMetricsView.getRawView(apps)
}
}
@@ -285,87 +221,9 @@ object CollectInformation extends Logging {
case class statisticsMetrics(min: Long, med:Long, max:Long, total: Long)
def generateSQLAccums(apps: Seq[ApplicationInfo]): Seq[SQLAccumProfileResults] = {
- val allRows = apps.flatMap { app =>
- app.allSQLMetrics.map { metric =>
- val jobsForSql = app.jobIdToInfo.filter { case (_, jc) =>
- // Avoid getOrElse to reduce memory allocations
- jc.sqlID.isDefined && jc.sqlID.get == metric.sqlID
- }
- val stageIdsForSQL = jobsForSql.flatMap(_._2.stageIds).toSet
- val accumsOpt = app.taskStageAccumMap.get(metric.accumulatorId)
- val taskMax = accumsOpt match {
- case Some(accums) =>
- val filtered = accums.filter { a =>
- stageIdsForSQL.contains(a.stageId)
- }
- // If metricType is size, average or timing, we want to read field `update` value
- // to get the min, median, max, and total. Otherwise, we want to use field `value`.
- if (SQLMetricsStats.hasStats(metric.metricType)) {
- val accumValues = filtered.map(_.update.getOrElse(0L)).sortWith(_ < _)
- if (accumValues.isEmpty) {
- None
- }
- else if (accumValues.length <= 1) {
- Some(statisticsMetrics(0L, 0L, 0L, accumValues.sum))
- } else {
- Some(statisticsMetrics(accumValues(0), accumValues(accumValues.size / 2),
- accumValues(accumValues.size - 1), accumValues.sum))
- }
- } else {
- val accumValues = filtered.map(_.value.getOrElse(0L))
- if (accumValues.isEmpty) {
- None
- } else {
- Some(statisticsMetrics(0L, 0L, 0L, accumValues.max))
- }
- }
- case None => None
- }
-
- // local mode driver gets updates
- val driverAccumsOpt = app.driverAccumMap.get(metric.accumulatorId)
- val driverMax = driverAccumsOpt match {
- case Some(accums) =>
- val filtered = accums.filter { a =>
- a.sqlID == metric.sqlID
- }
- val accumValues = filtered.map(_.value).sortWith(_ < _)
- if (accumValues.isEmpty) {
- None
- } else if (accumValues.length <= 1) {
- Some(statisticsMetrics(0L, 0L, 0L, accumValues.sum))
- } else {
- Some(statisticsMetrics(accumValues(0), accumValues(accumValues.size / 2),
- accumValues(accumValues.size - 1), accumValues.sum))
- }
- case None =>
- None
- }
-
- if (taskMax.isDefined || driverMax.isDefined) {
- val taskInfo = taskMax match {
- case Some(task) => task
- case None => statisticsMetrics(0L, 0L, 0L, 0L)
- }
- val driverInfo = driverMax match {
- case Some(driver) => driver
- case None => statisticsMetrics(0L, 0L, 0L, 0L)
- }
-
- val max = Math.max(taskInfo.max, driverInfo.max)
- val min = Math.max(taskInfo.min, driverInfo.min)
- val med = Math.max(taskInfo.med, driverInfo.med)
- val total = Math.max(taskInfo.total, driverInfo.total)
-
- Some(SQLAccumProfileResults(app.index, metric.sqlID,
- metric.nodeID, metric.nodeName, metric.accumulatorId, metric.name,
- min, med, max, total, metric.metricType, metric.stageIds.mkString(",")))
- } else {
- None
- }
- }
+ apps.flatMap { app =>
+ app.planMetricProcessor.generateSQLAccums()
}
- allRows.filter(_.isDefined).map(_.get)
}
def printSQLPlans(apps: Seq[ApplicationInfo], outputDir: String): Unit = {
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateDot.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateDot.scala
index a981d82c8..fc1963dfe 100644
--- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateDot.scala
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateDot.scala
@@ -90,18 +90,18 @@ object GenerateDot {
}
val accumIdToStageId = app.stageManager.getAccumToSingleStage()
val formatter = java.text.NumberFormat.getIntegerInstance
- val stageIdToStageMetrics = app.taskEnd.groupBy(task => task.stageId).mapValues { tasks =>
+ val stageIdToStageMetrics = app.taskManager.stageAttemptToTasks.collect { case (stageId, _) =>
+ val tasks = app.taskManager.getAllTasksStageAttempt(stageId)
val durations = tasks.map(_.duration)
- val numTasks = durations.length
+ val numTasks = tasks.size
val minDur = durations.min
val maxDur = durations.max
val meanDur = durations.sum / numTasks.toDouble
- StageMetrics(numTasks,
+ stageId -> StageMetrics(numTasks,
s"MIN: ${formatter.format(minDur)} ms " +
s"MAX: ${formatter.format(maxDur)} ms " +
s"AVG: ${formatter.format(meanDur)} ms")
- }
-
+ }.toMap
val sqlIdToMaxMetric = new mutable.HashMap[Long, ArrayBuffer[(Long, Long)]]()
for (row <- accumSummary) {
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimeline.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimeline.scala
index e4f0aef73..25b80ff1b 100644
--- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimeline.scala
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimeline.scala
@@ -259,7 +259,7 @@ object GenerateTimeline {
val readTimeIds = new mutable.HashSet[Long]()
val opTimeIds = new mutable.HashSet[Long]()
val writeTimeIds = new mutable.HashSet[Long]()
- app.allSQLMetrics.foreach { f =>
+ app.planMetricProcessor.allSQLMetrics.foreach { f =>
f.name match {
case "op time" | "GPU decode time" | "GPU Time" if f.metricType == "nsTiming" =>
opTimeIds += f.accumulatorId
@@ -295,7 +295,7 @@ object GenerateTimeline {
app.taskStageAccumMap.get(id)
}.flatten
- app.taskEnd.foreach { tc =>
+ app.taskManager.getAllTasks().foreach { tc =>
val host = tc.host
val execId = tc.executorId
val stageId = tc.stageId
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/HealthCheck.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/HealthCheck.scala
index dbbda465a..6548e6bfb 100644
--- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/HealthCheck.scala
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/HealthCheck.scala
@@ -16,6 +16,8 @@
package com.nvidia.spark.rapids.tool.profiling
+import com.nvidia.spark.rapids.tool.views.{ProfFailedJobsView, ProfFailedStageView, ProfFailedTaskView, ProfRemovedBLKMgrView, ProfRemovedExecutorView}
+
import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo
/**
@@ -25,95 +27,29 @@ class HealthCheck(apps: Seq[ApplicationInfo]) {
// Function to list all failed tasks , stages and jobs.
def getFailedTasks: Seq[FailedTaskProfileResults] = {
- val failed = apps.flatMap { app =>
- val tasksFailed = app.taskEnd.filter(_.successful == false)
- tasksFailed.map { t =>
- FailedTaskProfileResults(app.index, t.stageId, t.stageAttemptId,
- t.taskId, t.attempt, ProfileUtils.truncateFailureStr(t.endReason))
- }
- }
- if (failed.size > 0) {
- val sortedRows = failed.sortBy(cols =>
- (cols.appIndex, cols.stageId, cols.stageAttemptId, cols.taskId, cols.taskAttemptId))
- sortedRows
- } else {
- Seq.empty
- }
+ ProfFailedTaskView.getRawView(apps)
}
def getFailedStages: Seq[FailedStagesProfileResults] = {
- val failed = apps.flatMap { app =>
- val stagesFailed = app.stageManager.getFailedStages
- stagesFailed.map { case fsm =>
- val failureStr = fsm.getFailureReason
- FailedStagesProfileResults(app.index, fsm.sId, fsm.attemptId,
- fsm.sInfo.name, fsm.sInfo.numTasks,
- ProfileUtils.truncateFailureStr(failureStr))
- }
- }
- if (failed.size > 0) {
- val sortedRows = failed.sortBy(cols => (cols.appIndex, cols.stageId,
- cols.stageAttemptId))
- sortedRows
- } else {
- Seq.empty
- }
+ ProfFailedStageView.getRawView(apps)
}
def getFailedJobs: Seq[FailedJobsProfileResults] = {
- val failed = apps.flatMap { app =>
- val jobsFailed = app.jobIdToInfo.filter { case (_, jc) =>
- jc.jobResult.nonEmpty && !jc.jobResult.get.equals("JobSucceeded")
- }
- jobsFailed.map { case (id, jc) =>
- val failureStr = jc.failedReason.getOrElse("")
- FailedJobsProfileResults(app.index, id, jc.jobResult.getOrElse("Unknown"),
- ProfileUtils.truncateFailureStr(failureStr))
- }
- }
- if (failed.size > 0) {
- val sortedRows = failed.sortBy { cols =>
- (cols.appIndex, cols.jobId, cols.jobResult)
- }
- sortedRows
- } else {
- Seq.empty
- }
+ ProfFailedJobsView.getRawView(apps)
}
def getRemovedBlockManager: Seq[BlockManagerRemovedProfileResult] = {
- val res = apps.flatMap { app =>
- app.blockManagersRemoved.map { bm =>
- BlockManagerRemovedProfileResult(app.index, bm.executorId, bm.time)
- }
- }
- if (res.size > 0) {
- res.sortBy(cols => (cols.appIndex, cols.executorId))
- } else {
- Seq.empty
- }
+ ProfRemovedBLKMgrView.getRawView(apps)
}
def getRemovedExecutors: Seq[ExecutorsRemovedProfileResult] = {
- val res = apps.flatMap { app =>
- val execsRemoved = app.executorIdToInfo.filter { case (_, exec) =>
- exec.isActive == false
- }
- execsRemoved.map { case (id, exec) =>
- ExecutorsRemovedProfileResult(app.index, id, exec.removeTime, exec.removeReason)
- }
- }
- if (res.size > 0) {
- res.sortBy(cols => (cols.appIndex, cols.executorId))
- } else {
- Seq.empty
- }
+ ProfRemovedExecutorView.getRawView(apps)
}
//Function to list all *possible* not-supported plan nodes if GPU Mode=on
def getPossibleUnsupportedSQLPlan: Seq[UnsupportedOpsProfileResult] = {
val res = apps.flatMap { app =>
- app.unsupportedSQLplan.map { unsup =>
+ app.planMetricProcessor.unsupportedSQLPlan.map { unsup =>
UnsupportedOpsProfileResult(app.index, unsup.sqlID, unsup.nodeID, unsup.nodeName,
ProfileUtils.truncateFailureStr(unsup.nodeDesc), unsup.reason)
}
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 2ee5bf3c2..320b51026 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
@@ -32,9 +32,6 @@ trait ProfileResult {
def convertToCSVSeq: Seq[String]
}
-case class DriverInfo(val executorId: String, maxMemory: Long, totalOnHeap: Long,
- totalOffHeap: Long)
-
case class ExecutorInfoProfileResult(appIndex: Int, resourceProfileId: Int,
numExecutors: Int, executorCores: Int, maxMem: Long, maxOnHeapMem: Long,
maxOffHeapMem: Long, executorMemory: Option[Long], numGpusPerExecutor: Option[Long],
@@ -326,48 +323,6 @@ case class TaskStageAccumCase(
update: Option[Long],
isInternal: Boolean)
-// Note: sr = Shuffle Read; sw = Shuffle Write
-case class TaskCase(
- stageId: Int,
- stageAttemptId: Int,
- taskType: String,
- endReason: String,
- taskId: Long,
- attempt: Int,
- launchTime: Long,
- finishTime: Long,
- duration: Long,
- successful: Boolean,
- executorId: String,
- host: String,
- taskLocality: String,
- speculative: Boolean,
- gettingResultTime: Long,
- executorDeserializeTime: Long,
- executorDeserializeCPUTime: Long,
- executorRunTime: Long,
- executorCPUTime: Long,
- peakExecutionMemory: Long,
- resultSize: Long,
- jvmGCTime: Long,
- resultSerializationTime: Long,
- memoryBytesSpilled: Long,
- diskBytesSpilled: Long,
- sr_remoteBlocksFetched: Long,
- sr_localBlocksFetched: Long,
- sr_fetchWaitTime: Long,
- sr_remoteBytesRead: Long,
- sr_remoteBytesReadToDisk: Long,
- sr_localBytesRead: Long,
- sr_totalBytesRead: Long,
- sw_bytesWritten: Long,
- sw_writeTime: Long,
- sw_recordsWritten: Long,
- input_bytesRead: Long,
- input_recordsRead: Long,
- output_bytesWritten: Long,
- output_recordsWritten: Long)
-
case class UnsupportedSQLPlan(sqlID: Long, nodeID: Long, nodeName: String,
nodeDesc: String, reason: String)
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 f87474eef..1217cf9eb 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
@@ -24,6 +24,7 @@ import scala.util.control.NonFatal
import com.nvidia.spark.rapids.ThreadFactoryBuilder
import com.nvidia.spark.rapids.tool.{EventLogInfo, EventLogPathProcessor, PlatformFactory}
+import com.nvidia.spark.rapids.tool.views._
import org.apache.hadoop.conf.Configuration
import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo
@@ -343,9 +344,6 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea
}
val appLogPath = collect.getAppLogPath
val dsInfo = collect.getDataSourceInfo
- val execInfo = collect.getExecutorInfo
- val jobInfo = collect.getJobInfo
- val sqlStageInfo = collect.getSQLToStage
val rapidsProps = collect.getRapidsProperties
val sparkProps = collect.getSparkProperties
val systemProps = collect.getSystemProperties
@@ -362,18 +360,6 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea
None
}
- val analysis = new Analysis(apps)
- val jsMetAgg = analysis.jobAndStageMetricsAggregation()
- val sqlTaskAggMetrics = analysis.sqlMetricsAggregation()
- val ioAnalysisMetrics = analysis.ioAnalysis()
- val durAndCpuMet = analysis.sqlMetricsAggregationDurationAndCpuTime()
- val skewInfo = analysis.shuffleSkewCheck()
- val maxTaskInputInfo = if (useAutoTuner) {
- analysis.getMaxTaskInputSizeBytes()
- } else {
- Seq.empty
- }
-
val healthCheck = new HealthCheck(apps)
val failedTasks = healthCheck.getFailedTasks
val failedStages = healthCheck.getFailedStages
@@ -411,13 +397,21 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea
s"to $outputDir in $duration second(s)\n")
}
}
+ val analysis = RawMetricProfilerView.getAggMetrics(apps)
+ val maxTaskInputInfo = if (useAutoTuner) {
+ analysis.maxTaskInputSizes
+ } else {
+ Seq.empty
+ }
val endTime = System.currentTimeMillis()
logInfo(s"Took ${endTime - startTime}ms to Process [${appInfo.head.appId}]")
- (ApplicationSummaryInfo(appInfo, dsInfo, execInfo, jobInfo, rapidsProps,
- rapidsJar, sqlMetrics, jsMetAgg, sqlTaskAggMetrics, durAndCpuMet, skewInfo,
+ (ApplicationSummaryInfo(appInfo, dsInfo,
+ collect.getExecutorInfo, collect.getJobInfo, rapidsProps,
+ rapidsJar, sqlMetrics, analysis.jobStageAggs,
+ analysis.sqlAggs, analysis.sqlDurAggs, analysis.taskShuffleSkew,
failedTasks, failedStages, failedJobs, removedBMs, removedExecutors,
- unsupportedOps, sparkProps, sqlStageInfo, wholeStage, maxTaskInputInfo,
- appLogPath, ioAnalysisMetrics, systemProps), compareRes)
+ unsupportedOps, sparkProps, collect.getSQLToStage, wholeStage, maxTaskInputInfo,
+ appLogPath, analysis.ioAggs, systemProps), compareRes)
}
/**
@@ -511,9 +505,9 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea
profileOutputWriter.write("Application Information", app.appInfo)
profileOutputWriter.write("Application Log Path Mapping", app.appLogPath)
profileOutputWriter.write("Data Source Information", app.dsInfo)
- profileOutputWriter.write("Executor Information", app.execInfo)
- profileOutputWriter.write("Job Information", app.jobInfo)
- profileOutputWriter.write("SQL to Stage Information", app.sqlStageInfo)
+ profileOutputWriter.write(ProfExecutorView.getLabel, app.execInfo)
+ profileOutputWriter.write(ProfJobsView.getLabel, app.jobInfo)
+ profileOutputWriter.write(ProfSQLToStageView.getLabel, app.sqlStageInfo)
profileOutputWriter.write("Spark Rapids parameters set explicitly", app.rapidsProps,
Some("Spark Rapids parameters"))
profileOutputWriter.write("Spark Properties", app.sparkProps,
@@ -522,10 +516,10 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea
Some("System Properties"))
profileOutputWriter.write("Rapids Accelerator Jar and cuDF Jar", app.rapidsJar,
Some("Rapids 4 Spark Jars"))
- profileOutputWriter.write("SQL Plan Metrics for Application", app.sqlMetrics,
- Some("SQL Plan Metrics"))
- profileOutputWriter.write("WholeStageCodeGen Mapping", app.wholeStage,
- Some("WholeStagecodeGen Mapping"))
+ profileOutputWriter.write(ProfSQLPlanMetricsView.getLabel, app.sqlMetrics,
+ Some(ProfSQLPlanMetricsView.getDescription))
+ profileOutputWriter.write(ProfSQLCodeGenView.getLabel, app.wholeStage,
+ Some(ProfSQLCodeGenView.getDescription))
comparedRes.foreach { compareSum =>
val matchingSqlIds = compareSum.matchingSqlIds
val matchingStageIds = compareSum.matchingStageIds
@@ -534,22 +528,22 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea
}
profileOutputWriter.writeText("\n### B. Analysis ###\n")
- profileOutputWriter.write("Job + Stage level aggregated task metrics", app.jsMetAgg,
- Some("Job/Stage Metrics"))
- profileOutputWriter.write("SQL level aggregated task metrics", app.sqlTaskAggMetrics,
- Some("SQL Metrics"))
- profileOutputWriter.write("IO Metrics", app.ioMetrics)
- profileOutputWriter.write("SQL Duration and Executor CPU Time Percent", app.durAndCpuMet)
- val skewHeader = "Shuffle Skew Check" // +
- val skewTableDesc = "(When task's Shuffle Read Size > 3 * Avg Stage-level size)"
+ profileOutputWriter.write(JOB_AND_STAGE_AGG_LABEL, app.jsMetAgg,
+ Some(AGG_DESCRIPTION(JOB_AND_STAGE_AGG_LABEL)))
+ profileOutputWriter.write(SQL_AGG_LABEL, app.sqlTaskAggMetrics,
+ Some(AGG_DESCRIPTION(SQL_AGG_LABEL)))
+ profileOutputWriter.write(IO_LABEL, app.ioMetrics)
+ profileOutputWriter.write(SQL_DUR_LABEL, app.durAndCpuMet)
+ val skewHeader = TASK_SHUFFLE_SKEW
+ val skewTableDesc = AGG_DESCRIPTION(TASK_SHUFFLE_SKEW)
profileOutputWriter.write(skewHeader, app.skewInfo, tableDesc = Some(skewTableDesc))
profileOutputWriter.writeText("\n### C. Health Check###\n")
- profileOutputWriter.write("Failed Tasks", app.failedTasks)
- profileOutputWriter.write("Failed Stages", app.failedStages)
- profileOutputWriter.write("Failed Jobs", app.failedJobs)
- profileOutputWriter.write("Removed BlockManagers", app.removedBMs)
- profileOutputWriter.write("Removed Executors", app.removedExecutors)
+ profileOutputWriter.write(QualFailedTaskView.getLabel, app.failedTasks)
+ profileOutputWriter.write(ProfFailedStageView.getLabel, app.failedStages)
+ profileOutputWriter.write(ProfFailedJobsView.getLabel, app.failedJobs)
+ profileOutputWriter.write(ProfRemovedBLKMgrView.getLabel, app.removedBMs)
+ profileOutputWriter.write(ProfRemovedExecutorView.getLabel, app.removedExecutors)
profileOutputWriter.write("Unsupported SQL Plan", app.unsupportedOps,
Some("Unsupported SQL Ops"))
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala
index 8404c6d40..027bde53f 100644
--- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala
@@ -24,6 +24,7 @@ import com.nvidia.spark.rapids.ThreadFactoryBuilder
import com.nvidia.spark.rapids.tool.EventLogInfo
import com.nvidia.spark.rapids.tool.qualification.QualOutputWriter.DEFAULT_JOB_FREQUENCY
import com.nvidia.spark.rapids.tool.tuning.TunerContext
+import com.nvidia.spark.rapids.tool.views.QualRawReportGenerator
import org.apache.hadoop.conf.Configuration
import org.apache.spark.sql.rapids.tool.qualification._
@@ -159,6 +160,8 @@ class Qualification(outputPath: String, numRows: Int, hadoopConf: Configuration,
UnknownQualAppResult(pathStr, "", errorMessage)
case Right(app: QualificationAppInfo) =>
// Case with successful creation of QualificationAppInfo
+ // First, generate the Raw metrics view
+ QualRawReportGenerator.generateRawMetricQualView(outputDir, app)
val qualSumInfo = app.aggregateStats()
tunerContext.foreach { tuner =>
// Run the autotuner if it is enabled.
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/AggMetricsResultSorter.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/AggMetricsResultSorter.scala
new file mode 100644
index 000000000..da0bfa967
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/AggMetricsResultSorter.scala
@@ -0,0 +1,95 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.views
+
+import com.nvidia.spark.rapids.tool.profiling.{IOAnalysisProfileResult, JobStageAggTaskMetricsProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLTaskAggMetricsProfileResult}
+
+/**
+ * Contains the sort logic for the aggregated Spark RawMetrics.
+ * Note: This implementation needs to be cleaner and to allow for different sort strategies to
+ * be used by overriding the sorting methods.
+ * Having that in one place has the following pros:
+ * 1- makes the code easier to read, because otherwise the sorting code was spread everywhere in
+ * the aggregation methods.
+ * 2- Easier to maintain the code and extend it to implement different sort strategies for different
+ * tools/reports. For example, the qualification tool can define object that executes a different
+ * sorting logic compared to the profiler.
+ */
+object AggMetricsResultSorter {
+ // TODO: The implementation needs to be cleaner to allow for different sort strategies.
+ // Also, it will be better to use some generics to allow for different argument types.
+ def sortJobSparkMetrics(
+ rows: Seq[JobStageAggTaskMetricsProfileResult]): Seq[JobStageAggTaskMetricsProfileResult] = {
+ if (rows.isEmpty) {
+ Seq.empty
+ } else {
+ rows.sortBy { cols =>
+ val sortDur = cols.duration.getOrElse(0L)
+ (cols.appIndex, -sortDur, cols.id)
+ }
+ }
+ }
+
+ def sortSqlAgg(
+ rows: Seq[SQLTaskAggMetricsProfileResult]): Seq[SQLTaskAggMetricsProfileResult] = {
+ if (rows.isEmpty) {
+ Seq.empty
+ } else {
+ rows.sortBy { cols =>
+ val sortDur = cols.duration.getOrElse(0L)
+ (cols.appIndex, -sortDur, cols.sqlId, cols.executorCpuTime)
+ }
+ }
+ }
+
+ def sortSqlDurationAgg(
+ rows: Seq[SQLDurationExecutorTimeProfileResult]):
+ Seq[SQLDurationExecutorTimeProfileResult] = {
+ if (rows.isEmpty) {
+ Seq.empty
+ } else {
+ rows.sortBy { cols =>
+ val sortDur = cols.duration.getOrElse(0L)
+ (cols.appIndex, cols.sqlID, sortDur)
+ }
+ }
+ }
+
+ def sortShuffleSkew(
+ rows: Seq[ShuffleSkewProfileResult]):
+ Seq[ShuffleSkewProfileResult] = {
+ if (rows.isEmpty) {
+ Seq.empty
+ } else {
+ rows.sortBy { cols =>
+ (cols.appIndex, cols.stageId, cols.stageAttemptId, cols.taskId, cols.taskAttemptId)
+ }
+ }
+ }
+
+ def sortIO(
+ rows: Seq[IOAnalysisProfileResult]):
+ Seq[IOAnalysisProfileResult] = {
+ if (rows.isEmpty) {
+ Seq.empty
+ } else {
+ rows.sortBy { cols =>
+ (cols.appIndex, cols.sqlId)
+ }
+ }
+ }
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/ExecutorView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/ExecutorView.scala
new file mode 100644
index 000000000..76d673e4b
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/ExecutorView.scala
@@ -0,0 +1,114 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.views
+
+import com.nvidia.spark.rapids.tool.analysis.{ProfAppIndexMapperTrait, QualAppIndexMapperTrait}
+import com.nvidia.spark.rapids.tool.profiling.{BlockManagerRemovedProfileResult, ExecutorInfoProfileResult, ExecutorsRemovedProfileResult}
+
+import org.apache.spark.resource.ResourceProfile
+import org.apache.spark.sql.rapids.tool.AppBase
+
+
+trait AppExecutorViewTrait extends ViewableTrait[ExecutorInfoProfileResult] {
+ override def getLabel: String = "Executor Information"
+
+ override def getRawView(app: AppBase, index: Int): Seq[ExecutorInfoProfileResult] = {
+ // first see if any executors have different resourceProfile ids
+ val groupedExecs = app.executorIdToInfo.groupBy(_._2.resourceProfileId)
+ groupedExecs.map { case (rpId, execs) =>
+ val rp = app.resourceProfIdToInfo.get(rpId)
+ val execMem = rp.map(_.executorResources.get(ResourceProfile.MEMORY)
+ .map(_.amount).getOrElse(0L))
+ val execGpus = rp.map(_.executorResources.get("gpu")
+ .map(_.amount).getOrElse(0L))
+ val taskCpus = rp.map(_.taskResources.get(ResourceProfile.CPUS)
+ .map(_.amount).getOrElse(0.toDouble))
+ val taskGpus = rp.map(_.taskResources.get("gpu").map(_.amount).getOrElse(0.toDouble))
+ val execOffHeap = rp.map(_.executorResources.get(ResourceProfile.OFFHEAP_MEM)
+ .map(_.amount).getOrElse(0L))
+
+ val numExecutors = execs.size
+ val exec = execs.head._2
+ // We could print a lot more information here if we decided, more like the Spark UI
+ // per executor info.
+ ExecutorInfoProfileResult(index, rpId, numExecutors,
+ exec.totalCores, exec.maxMemory, exec.totalOnHeap,
+ exec.totalOffHeap, execMem, execGpus, execOffHeap, taskCpus, taskGpus)
+ }.toSeq
+ }
+
+ override def sortView(rows: Seq[ExecutorInfoProfileResult]): Seq[ExecutorInfoProfileResult] = {
+ rows.sortBy(cols => (cols.appIndex, cols.resourceProfileId))
+ }
+}
+
+trait AppRemovedExecutorView extends ViewableTrait[ExecutorsRemovedProfileResult] {
+ override def getLabel: String = "Removed Executors"
+
+ override def getRawView(app: AppBase, index: Int): Seq[ExecutorsRemovedProfileResult] = {
+ val execsRemoved = app.executorIdToInfo.filter { case (_, exec) =>
+ !exec.isActive
+ }
+ execsRemoved.map { case (id, exec) =>
+ ExecutorsRemovedProfileResult(index, id, exec.removeTime, exec.removeReason)
+ }.toSeq
+ }
+
+ override def sortView(
+ rows: Seq[ExecutorsRemovedProfileResult]): Seq[ExecutorsRemovedProfileResult] = {
+ rows.sortBy(cols => (cols.appIndex, cols.executorId))
+ }
+}
+
+trait AppRemovedBlockManagerView extends ViewableTrait[BlockManagerRemovedProfileResult] {
+ override def getLabel: String = "Removed BlockManagers"
+
+ override def getRawView(app: AppBase, index: Int): Seq[BlockManagerRemovedProfileResult] = {
+ app.blockManagersRemoved.map { bm =>
+ BlockManagerRemovedProfileResult(index, bm.executorId, bm.time)
+ }
+ }
+
+ override def sortView(
+ rows: Seq[BlockManagerRemovedProfileResult]): Seq[BlockManagerRemovedProfileResult] = {
+ rows.sortBy(cols => (cols.appIndex, cols.executorId))
+ }
+}
+
+object QualExecutorView extends AppExecutorViewTrait with QualAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
+
+object ProfExecutorView extends AppExecutorViewTrait with ProfAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
+
+object QualRemovedExecutorView extends AppRemovedExecutorView with QualAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
+
+object ProfRemovedExecutorView extends AppRemovedExecutorView with ProfAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
+
+object QualRemovedBLKMgrView extends AppRemovedBlockManagerView with QualAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
+
+object ProfRemovedBLKMgrView extends AppRemovedBlockManagerView with ProfAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/JobView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/JobView.scala
new file mode 100644
index 000000000..a9ed28fc7
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/JobView.scala
@@ -0,0 +1,71 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.views
+
+import com.nvidia.spark.rapids.tool.analysis.{ProfAppIndexMapperTrait, QualAppIndexMapperTrait}
+import com.nvidia.spark.rapids.tool.profiling.{FailedJobsProfileResults, JobInfoProfileResult, ProfileUtils}
+
+import org.apache.spark.sql.rapids.tool.AppBase
+
+
+trait AppJobsViewTrait extends ViewableTrait[JobInfoProfileResult] {
+ override def getLabel: String = "Job Information"
+
+ def getRawView(app: AppBase, index: Int): Seq[JobInfoProfileResult] = {
+ app.jobIdToInfo.map { case (_, j) =>
+ JobInfoProfileResult(index, j.jobID, j.stageIds, j.sqlID, j.startTime, j.endTime)
+ }.toSeq
+ }
+ override def sortView(rows: Seq[JobInfoProfileResult]): Seq[JobInfoProfileResult] = {
+ rows.sortBy(cols => (cols.appIndex, cols.jobID))
+ }
+}
+
+trait AppFailedJobsViewTrait extends ViewableTrait[FailedJobsProfileResults] {
+ override def getLabel: String = "Failed Jobs"
+
+ def getRawView(app: AppBase, index: Int): Seq[FailedJobsProfileResults] = {
+ val jobsFailed = app.jobIdToInfo.filter { case (_, jc) =>
+ jc.jobResult.nonEmpty && !jc.jobResult.get.equals("JobSucceeded")
+ }
+ jobsFailed.map { case (id, jc) =>
+ val failureStr = jc.failedReason.getOrElse("")
+ FailedJobsProfileResults(index, id, jc.jobResult.getOrElse("Unknown"),
+ ProfileUtils.truncateFailureStr(failureStr))
+ }.toSeq
+ }
+
+ override def sortView(rows: Seq[FailedJobsProfileResults]): Seq[FailedJobsProfileResults] = {
+ rows.sortBy(cols => (cols.appIndex, cols.jobId, cols.jobResult))
+ }
+}
+
+object QualAppJobView extends AppJobsViewTrait with QualAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
+
+object ProfJobsView extends AppJobsViewTrait with ProfAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
+
+object QualAppFailedJobView extends AppFailedJobsViewTrait with QualAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
+
+object ProfFailedJobsView extends AppFailedJobsViewTrait with ProfAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/QualRawReportGenerator.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/QualRawReportGenerator.scala
new file mode 100644
index 000000000..f9e200377
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/QualRawReportGenerator.scala
@@ -0,0 +1,98 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.views
+
+import com.nvidia.spark.rapids.tool.analysis.{AggRawMetricsResult, AppSQLPlanAnalyzer, QualSparkMetricsAnalyzer}
+import com.nvidia.spark.rapids.tool.profiling.{ProfileOutputWriter, ProfileResult}
+
+import org.apache.spark.sql.rapids.tool.qualification.QualificationAppInfo
+
+/**
+ * This object generates the raw metrics view for the qualification tool. It is used to generate
+ * the CSV files without applying any heuristics or estimation.
+ */
+object QualRawReportGenerator {
+
+ private def constructLabelsMaps(
+ aggRawResult: AggRawMetricsResult): Map[String, Seq[ProfileResult]] = {
+ val sortedRes = AggRawMetricsResult(
+ AggMetricsResultSorter.sortJobSparkMetrics(aggRawResult.jobAggs),
+ AggMetricsResultSorter.sortJobSparkMetrics(aggRawResult.stageAggs),
+ AggMetricsResultSorter.sortShuffleSkew(aggRawResult.taskShuffleSkew),
+ AggMetricsResultSorter.sortSqlAgg(aggRawResult.sqlAggs),
+ AggMetricsResultSorter.sortIO(aggRawResult.ioAggs),
+ AggMetricsResultSorter.sortSqlDurationAgg(aggRawResult.sqlDurAggs),
+ aggRawResult.maxTaskInputSizes)
+ Map(
+ STAGE_AGG_LABEL -> sortedRes.stageAggs,
+ JOB_AGG_LABEL -> sortedRes.jobAggs,
+ TASK_SHUFFLE_SKEW -> sortedRes.taskShuffleSkew,
+ SQL_AGG_LABEL -> sortedRes.sqlAggs,
+ IO_LABEL -> sortedRes.ioAggs,
+ SQL_DUR_LABEL -> sortedRes.sqlDurAggs)
+ }
+
+ private def generateSQLProcessingView(
+ pWriter: ProfileOutputWriter, app: QualificationAppInfo,
+ appIndex: Int): Unit = {
+ // We only need the SQL analyzer here to generate the output. It is not saved in the AppBase to
+ // save memory
+ val sqlPlanAnalyzer = AppSQLPlanAnalyzer(app, appIndex)
+ pWriter.write(QualSQLToStageView.getLabel,
+ QualSQLToStageView.getRawViewFromSqlProcessor(sqlPlanAnalyzer))
+ pWriter.write(QualSQLPlanMetricsView.getLabel,
+ QualSQLPlanMetricsView.getRawViewFromSqlProcessor(sqlPlanAnalyzer),
+ Some(QualSQLPlanMetricsView.getDescription))
+ pWriter.write(QualSQLCodeGenView.getLabel,
+ QualSQLCodeGenView.getRawViewFromSqlProcessor(sqlPlanAnalyzer),
+ Some(QualSQLCodeGenView.getDescription))
+ }
+
+ def generateRawMetricQualView(
+ rootDir: String,
+ app: QualificationAppInfo,
+ appIndex: Int = 1): Unit = {
+ val metricsDirectory = s"$rootDir/raw_metrics/${app.appId}"
+ val pWriter =
+ new ProfileOutputWriter(metricsDirectory, "raw_information",
+ 10000000, outputCSV = true)
+ try {
+ pWriter.writeText("### A. Information Collected ###")
+ pWriter.write(QualExecutorView.getLabel, QualExecutorView.getRawView(Seq(app)))
+ pWriter.write(QualAppJobView.getLabel, QualAppJobView.getRawView(Seq(app)))
+ generateSQLProcessingView(pWriter, app, appIndex)
+ pWriter.writeText("\n### B. Analysis ###\n")
+ constructLabelsMaps(
+ QualSparkMetricsAnalyzer.getAggRawMetrics(app, appIndex)).foreach { case (label, metrics) =>
+ pWriter.write(label,
+ metrics,
+ AGG_DESCRIPTION.get(label))
+ }
+ pWriter.writeText("\n### C. Health Check###\n")
+ pWriter.write(QualFailedTaskView.getLabel, QualFailedTaskView.getRawView(Seq(app)))
+ pWriter.write(QualFailedStageView.getLabel, QualFailedStageView.getRawView(Seq(app)))
+ pWriter.write(QualAppFailedJobView.getLabel, QualAppFailedJobView.getRawView(Seq(app)))
+ pWriter.write(QualRemovedBLKMgrView.getLabel, QualRemovedBLKMgrView.getRawView(Seq(app)))
+ pWriter.write(QualRemovedExecutorView.getLabel, QualRemovedExecutorView.getRawView(Seq(app)))
+ } catch {
+ case e: Exception =>
+ println(s"Error generating raw metrics for ${app.appId}: ${e.getMessage}")
+ } finally {
+ pWriter.close()
+ }
+ }
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/RawMetricProfView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/RawMetricProfView.scala
new file mode 100644
index 000000000..abbf1b6eb
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/RawMetricProfView.scala
@@ -0,0 +1,46 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.views
+
+import com.nvidia.spark.rapids.tool.analysis.ProfSparkMetricsAnalyzer
+import com.nvidia.spark.rapids.tool.profiling.{IOAnalysisProfileResult, JobStageAggTaskMetricsProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLMaxTaskInputSizes, SQLTaskAggMetricsProfileResult}
+
+import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo
+
+// The profiling shows a single combined view for both Stage/Job-levels which is different from
+// the default view that separates between the two.
+case class ProfilerAggregatedView(
+ jobStageAggs: Seq[JobStageAggTaskMetricsProfileResult],
+ taskShuffleSkew: Seq[ShuffleSkewProfileResult],
+ sqlAggs: Seq[SQLTaskAggMetricsProfileResult],
+ ioAggs: Seq[IOAnalysisProfileResult],
+ sqlDurAggs: Seq[SQLDurationExecutorTimeProfileResult],
+ maxTaskInputSizes: Seq[SQLMaxTaskInputSizes])
+
+object RawMetricProfilerView {
+ def getAggMetrics(apps: Seq[ApplicationInfo]): ProfilerAggregatedView = {
+ val aggMetricsResults = ProfSparkMetricsAnalyzer.getAggregateRawMetrics(apps)
+ ProfilerAggregatedView(
+ AggMetricsResultSorter.sortJobSparkMetrics(
+ aggMetricsResults.stageAggs ++ aggMetricsResults.jobAggs),
+ AggMetricsResultSorter.sortShuffleSkew(aggMetricsResults.taskShuffleSkew),
+ AggMetricsResultSorter.sortSqlAgg(aggMetricsResults.sqlAggs),
+ AggMetricsResultSorter.sortIO(aggMetricsResults.ioAggs),
+ AggMetricsResultSorter.sortSqlDurationAgg(aggMetricsResults.sqlDurAggs),
+ aggMetricsResults.maxTaskInputSizes)
+ }
+}
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
new file mode 100644
index 000000000..0f3cc830a
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala
@@ -0,0 +1,95 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.views
+
+import com.nvidia.spark.rapids.tool.analysis.{AppSQLPlanAnalyzer, ProfAppIndexMapperTrait, QualAppIndexMapperTrait}
+import com.nvidia.spark.rapids.tool.profiling.{SQLAccumProfileResults, WholeStageCodeGenResults}
+
+import org.apache.spark.sql.rapids.tool.AppBase
+import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo
+
+trait AppSQLCodeGenViewTrait extends ViewableTrait[WholeStageCodeGenResults] {
+ override def getLabel: String = "WholeStageCodeGen Mapping"
+ override def getDescription: String = "WholeStagecodeGen Mapping"
+
+ override def sortView(
+ rows: Seq[WholeStageCodeGenResults]): Seq[WholeStageCodeGenResults] = {
+ rows.sortBy(cols => (cols.appIndex, cols.sqlID, cols.nodeID))
+ }
+}
+
+trait AppSQLPlanMetricsViewTrait extends ViewableTrait[SQLAccumProfileResults] {
+ override def getLabel: String = "SQL Plan Metrics for Application"
+ override def getDescription: String = "SQL Plan Metrics"
+
+ override def sortView(
+ rows: Seq[SQLAccumProfileResults]): Seq[SQLAccumProfileResults] = {
+ rows.sortBy(cols => (cols.appIndex, cols.sqlID, cols.nodeID,
+ cols.nodeName, cols.accumulatorId, cols.metricType))
+ }
+}
+
+object ProfSQLCodeGenView extends AppSQLCodeGenViewTrait with ProfAppIndexMapperTrait {
+
+ override def getRawView(app: AppBase, index: Int): Seq[WholeStageCodeGenResults] = {
+ app match {
+ case app: ApplicationInfo =>
+ app.planMetricProcessor.wholeStage
+ case _ => Seq.empty
+ }
+ }
+}
+
+object QualSQLCodeGenView extends AppSQLCodeGenViewTrait with QualAppIndexMapperTrait {
+
+ override def getRawView(app: AppBase, index: Int): Seq[WholeStageCodeGenResults] = {
+ // TODO: Fix this implementation when we have a better way to get bind between App and
+ // SQLProcessor but for now, we do not store SQLPlan Processor in QualificationSummaryInfo
+ // to save
+ Seq.empty
+ }
+
+ def getRawViewFromSqlProcessor(
+ sqlAnalyzer: AppSQLPlanAnalyzer): Seq[WholeStageCodeGenResults] = {
+ sortView(sqlAnalyzer.wholeStage)
+ }
+}
+
+object ProfSQLPlanMetricsView extends AppSQLPlanMetricsViewTrait with ProfAppIndexMapperTrait {
+
+ override def getRawView(app: AppBase, index: Int): Seq[SQLAccumProfileResults] = {
+ app match {
+ case app: ApplicationInfo =>
+ app.planMetricProcessor.generateSQLAccums()
+ 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
+ // SQLProcessor but for now, we do not store SQLPlan Processor in QualificationSummaryInfo
+ // to save
+ Seq.empty
+ }
+
+ def getRawViewFromSqlProcessor(
+ sqlAnalyzer: AppSQLPlanAnalyzer): Seq[SQLAccumProfileResults] = {
+ sortView(sqlAnalyzer.generateSQLAccums())
+ }
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/StageView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/StageView.scala
new file mode 100644
index 000000000..a208b6ef0
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/StageView.scala
@@ -0,0 +1,85 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.views
+
+import com.nvidia.spark.rapids.tool.analysis.{AppSQLPlanAnalyzer, ProfAppIndexMapperTrait, QualAppIndexMapperTrait}
+import com.nvidia.spark.rapids.tool.profiling.{FailedStagesProfileResults, ProfileUtils, SQLStageInfoProfileResult}
+
+import org.apache.spark.sql.rapids.tool.AppBase
+import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo
+
+trait AppFailedStageViewTrait extends ViewableTrait[FailedStagesProfileResults] {
+ override def getLabel: String = "Failed Stages"
+
+ override def getRawView(app: AppBase, index: Int): Seq[FailedStagesProfileResults] = {
+ app.stageManager.getFailedStages.map { fsm =>
+ val failureStr = fsm.getFailureReason
+ FailedStagesProfileResults(index, fsm.sId, fsm.attemptId,
+ fsm.sInfo.name, fsm.sInfo.numTasks, ProfileUtils.truncateFailureStr(failureStr))
+ }.toSeq
+ }
+
+ override def sortView(
+ rows: Seq[FailedStagesProfileResults]): Seq[FailedStagesProfileResults] = {
+ rows.sortBy(cols => (cols.appIndex, cols.stageId, cols.stageAttemptId))
+ }
+}
+
+
+trait AppSQLToStageViewTrait extends ViewableTrait[SQLStageInfoProfileResult] {
+ override def getLabel: String = "SQL to Stage Information"
+
+ override def sortView(
+ rows: Seq[SQLStageInfoProfileResult]): Seq[SQLStageInfoProfileResult] = {
+ case class Reverse[T](t: T)
+ implicit def ReverseOrdering[T: Ordering]: Ordering[Reverse[T]] =
+ Ordering[T].reverse.on(_.t)
+
+ // intentionally sort this table by the duration to be able to quickly
+ // see the stage that took the longest
+ rows.sortBy(cols => (cols.appIndex, Reverse(cols.duration)))
+ }
+}
+
+
+object QualFailedStageView extends AppFailedStageViewTrait with QualAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
+
+object ProfFailedStageView extends AppFailedStageViewTrait with ProfAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
+
+object QualSQLToStageView extends AppSQLToStageViewTrait with QualAppIndexMapperTrait {
+ override def getRawView(app: AppBase, index: Int): Seq[SQLStageInfoProfileResult] = {
+ // TODO: Fix this implementation when we have a better way to get bind between App and
+ // SQLProcessor but for now, we do not store SQLPlan Processor in QualificationSummaryInfo
+ // to save
+ Seq.empty
+ }
+
+ def getRawViewFromSqlProcessor(
+ sqlAnalyzer: AppSQLPlanAnalyzer): Seq[SQLStageInfoProfileResult] = {
+ sortView(sqlAnalyzer.aggregateSQLStageInfo)
+ }
+}
+
+object ProfSQLToStageView extends AppSQLToStageViewTrait with ProfAppIndexMapperTrait {
+ override def getRawView(app: AppBase, index: Int): Seq[SQLStageInfoProfileResult] = {
+ app.asInstanceOf[ApplicationInfo].planMetricProcessor.aggregateSQLStageInfo
+ }
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/TaskView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/TaskView.scala
new file mode 100644
index 000000000..d68962e28
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/TaskView.scala
@@ -0,0 +1,49 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.views
+
+import com.nvidia.spark.rapids.tool.analysis.{ProfAppIndexMapperTrait, QualAppIndexMapperTrait}
+import com.nvidia.spark.rapids.tool.profiling.{FailedTaskProfileResults, ProfileUtils}
+
+import org.apache.spark.sql.rapids.tool.AppBase
+
+
+trait AppFailedTaskViewTrait extends ViewableTrait[FailedTaskProfileResults] {
+ override def getLabel: String = "Failed Tasks"
+
+ override def getRawView(app: AppBase, index: Int): Seq[FailedTaskProfileResults] = {
+ app.taskManager.getAllFailedTasks.map { t =>
+ FailedTaskProfileResults(index, t.stageId, t.stageAttemptId,
+ t.taskId, t.attempt, ProfileUtils.truncateFailureStr(t.endReason))
+ }.toSeq
+ }
+
+ override def sortView(
+ rows: Seq[FailedTaskProfileResults]): Seq[FailedTaskProfileResults] = {
+ rows.sortBy(
+ cols => (cols.appIndex, cols.stageId, cols.stageAttemptId, cols.taskId, cols.taskAttemptId))
+ }
+}
+
+object QualFailedTaskView extends AppFailedTaskViewTrait with QualAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
+
+object ProfFailedTaskView extends AppFailedTaskViewTrait with ProfAppIndexMapperTrait {
+ // Keep for the following refactor stages to customize the view based on the app type (Qual/Prof)
+}
+
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/ViewableTrait.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/ViewableTrait.scala
new file mode 100644
index 000000000..8ee761ea7
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/ViewableTrait.scala
@@ -0,0 +1,42 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool.views
+
+import com.nvidia.spark.rapids.tool.analysis.AppIndexMapperTrait
+import com.nvidia.spark.rapids.tool.profiling.ProfileResult
+
+import org.apache.spark.sql.rapids.tool.AppBase
+
+trait ViewableTrait[R <: ProfileResult] extends AppIndexMapperTrait {
+ def getLabel: String
+ def getDescription: String = ""
+
+ def getRawView(app: AppBase, index: Int): Seq[R]
+
+ def getRawView(apps: Seq[AppBase]): Seq[R] = {
+ val allRows = zipAppsWithIndex(apps).flatMap { case (app, index) =>
+ getRawView(app, index)
+ }
+ if (allRows.isEmpty) {
+ allRows
+ } else {
+ sortView(allRows)
+ }
+ }
+
+ def sortView(rows: Seq[R]): Seq[R] = rows
+}
diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/package.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/package.scala
new file mode 100644
index 000000000..3fff97020
--- /dev/null
+++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/package.scala
@@ -0,0 +1,43 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.nvidia.spark.rapids.tool
+
+/**
+ * Package that contains implementation specific to generate the tools views. This is part of the
+ * refactor code to separate the internal object representation from the final output format.
+ */
+package object views {
+ val JOB_AND_STAGE_AGG_LABEL = "Job + Stage level aggregated task metrics"
+ val STAGE_AGG_LABEL = "Stage level aggregated task metrics"
+ val JOB_AGG_LABEL = "Job level aggregated task metrics"
+ val TASK_SHUFFLE_SKEW = "Shuffle Skew Check"
+ val SQL_AGG_LABEL = "SQL level aggregated task metrics"
+ val IO_LABEL = "IO Metrics"
+ val SQL_DUR_LABEL = "SQL Duration and Executor CPU Time Percent"
+ val SQL_MAX_INPUT_SIZE = "SQL Max Task Input Size"
+
+ val AGG_DESCRIPTION = Map(
+ JOB_AND_STAGE_AGG_LABEL -> "Job/Stage Metrics",
+ STAGE_AGG_LABEL -> "Stage metrics",
+ JOB_AGG_LABEL -> "Job metrics",
+ SQL_AGG_LABEL -> "SQL metrics",
+ IO_LABEL -> "IO Metrics per SQL",
+ SQL_DUR_LABEL -> "Total duration and CPUTime per SQL",
+ TASK_SHUFFLE_SKEW ->
+ "(When task's Shuffle Read Size > 3 * Avg Stage-level size)"
+ )
+}
diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala
index 80982983f..7e6b3a1d3 100644
--- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala
+++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala
@@ -26,7 +26,7 @@ import scala.io.{Codec, Source}
import com.nvidia.spark.rapids.tool.{DatabricksEventLog, DatabricksRollingEventLogFilesFileReader, EventLogInfo}
import com.nvidia.spark.rapids.tool.planparser.{HiveParseHelper, ReadParser}
import com.nvidia.spark.rapids.tool.planparser.HiveParseHelper.isHiveTableScanNode
-import com.nvidia.spark.rapids.tool.profiling.{DataSourceCase, DriverAccumCase, JobInfoClass, SQLExecutionInfoClass, TaskStageAccumCase}
+import com.nvidia.spark.rapids.tool.profiling.{BlockManagerRemovedCase, DataSourceCase, DriverAccumCase, JobInfoClass, ResourceProfileInfoCase, SQLExecutionInfoClass, SQLPlanMetricsCase, TaskStageAccumCase}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
@@ -35,7 +35,7 @@ import org.apache.spark.internal.Logging
import org.apache.spark.scheduler.{SparkListenerEvent, StageInfo}
import org.apache.spark.sql.execution.SparkPlanInfo
import org.apache.spark.sql.execution.ui.SparkPlanGraphNode
-import org.apache.spark.sql.rapids.tool.store.{StageModel, StageModelManager}
+import org.apache.spark.sql.rapids.tool.store.{StageModel, StageModelManager, TaskModelManager}
import org.apache.spark.sql.rapids.tool.util.{EventUtils, RapidsToolsConfUtil, ToolsPlanGraph}
import org.apache.spark.util.Utils
@@ -55,7 +55,10 @@ abstract class AppBase(
// Store map of executorId to executor info
val executorIdToInfo = new HashMap[String, ExecutorInfoClass]()
-
+ // resourceProfile id to resource profile info
+ val resourceProfIdToInfo = new HashMap[Int, ResourceProfileInfoCase]()
+ var blockManagersRemoved: ArrayBuffer[BlockManagerRemovedCase] =
+ ArrayBuffer[BlockManagerRemovedCase]()
// The data source information
val dataSourceInfo: ArrayBuffer[DataSourceCase] = ArrayBuffer[DataSourceCase]()
@@ -71,12 +74,16 @@ abstract class AppBase(
val sqlIdToStages = new HashMap[Long, ArrayBuffer[Int]]()
// sqlPlans stores HashMap (sqlID <-> SparkPlanInfo)
var sqlPlans: HashMap[Long, SparkPlanInfo] = HashMap.empty[Long, SparkPlanInfo]
+ var sqlPlanMetricsAdaptive: ArrayBuffer[SQLPlanMetricsCase] = ArrayBuffer[SQLPlanMetricsCase]()
// accum id to task stage accum info
var taskStageAccumMap: HashMap[Long, ArrayBuffer[TaskStageAccumCase]] =
HashMap[Long, ArrayBuffer[TaskStageAccumCase]]()
lazy val stageManager: StageModelManager = new StageModelManager()
+ // Container that manages TaskIno including SparkMetrics.
+ // A task is added during a TaskEnd eventLog
+ lazy val taskManager: TaskModelManager = new TaskModelManager()
var driverAccumMap: HashMap[Long, ArrayBuffer[DriverAccumCase]] =
HashMap[Long, ArrayBuffer[DriverAccumCase]]()
@@ -265,7 +272,7 @@ abstract class AppBase(
".*second\\(.*\\).*" -> "TIMEZONE second()"
)
- protected def findPotentialIssues(desc: String): Set[String] = {
+ def findPotentialIssues(desc: String): Set[String] = {
val potentialIssuesRegexs = potentialIssuesRegexMap
val issues = potentialIssuesRegexs.filterKeys(desc.matches(_))
issues.values.toSet
@@ -284,7 +291,7 @@ abstract class AppBase(
}
// The ReadSchema metadata is only in the eventlog for DataSource V1 readers
- protected def checkMetadataForReadSchema(
+ def checkMetadataForReadSchema(
sqlPlanInfoGraph: SqlPlanInfoGraphEntry): ArrayBuffer[DataSourceCase] = {
// check if planInfo has ReadSchema
val allMetaWithSchema = AppBase.getPlanMetaWithSchema(sqlPlanInfoGraph.planInfo)
@@ -338,7 +345,7 @@ abstract class AppBase(
// This will find scans for DataSource V2, if the schema is very large it
// will likely be incomplete and have ... at the end.
- protected def checkGraphNodeForReads(
+ def checkGraphNodeForReads(
sqlID: Long, node: SparkPlanGraphNode): Option[DataSourceCase] = {
if (ReadParser.isDataSourceV2Node(node)) {
val res = ReadParser.parseReadNode(node)
diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala
index 40507cb50..230af18ab 100644
--- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala
+++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.rapids.tool
import scala.collection.mutable.ArrayBuffer
import scala.util.control.NonFatal
-import com.nvidia.spark.rapids.tool.profiling.{DriverAccumCase, JobInfoClass, ProfileUtils, SQLExecutionInfoClass, TaskStageAccumCase}
+import com.nvidia.spark.rapids.tool.profiling.{BlockManagerRemovedCase, DriverAccumCase, JobInfoClass, ProfileUtils, ResourceProfileInfoCase, SQLExecutionInfoClass, SQLPlanMetricsCase, TaskStageAccumCase}
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler._
@@ -185,7 +185,15 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi
def doSparkListenerSQLAdaptiveSQLMetricUpdates(
app: T,
- event: SparkListenerSQLAdaptiveSQLMetricUpdates): Unit = {}
+ event: SparkListenerSQLAdaptiveSQLMetricUpdates): Unit = {
+ logDebug("Processing event: " + event.getClass)
+ val SparkListenerSQLAdaptiveSQLMetricUpdates(sqlID, sqlPlanMetrics) = event
+ val metrics = sqlPlanMetrics.map { metric =>
+ SQLPlanMetricsCase(sqlID, metric.name,
+ metric.accumulatorId, metric.metricType)
+ }
+ app.sqlPlanMetricsAdaptive ++= metrics
+ }
override def onOtherEvent(event: SparkListenerEvent): Unit = event match {
case e: SparkListenerSQLExecutionStart =>
@@ -207,7 +215,12 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi
def doSparkListenerResourceProfileAdded(
app: T,
- event: SparkListenerResourceProfileAdded): Unit = {}
+ event: SparkListenerResourceProfileAdded): Unit = {
+ // leave off maxTasks for now
+ val rp = ResourceProfileInfoCase(event.resourceProfile.id,
+ event.resourceProfile.executorResources, event.resourceProfile.taskResources)
+ app.resourceProfIdToInfo(event.resourceProfile.id) = rp
+ }
override def onResourceProfileAdded(event: SparkListenerResourceProfileAdded): Unit = {
doSparkListenerResourceProfileAdded(app, event)
@@ -247,7 +260,15 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi
def doSparkListenerBlockManagerRemoved(
app: T,
- event: SparkListenerBlockManagerRemoved): Unit = {}
+ event: SparkListenerBlockManagerRemoved): Unit = {
+ val thisBlockManagerRemoved = BlockManagerRemovedCase(
+ event.blockManagerId.executorId,
+ event.blockManagerId.host,
+ event.blockManagerId.port,
+ event.time
+ )
+ app.blockManagersRemoved += thisBlockManagerRemoved
+ }
override def onBlockManagerRemoved(
blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = {
@@ -351,6 +372,8 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi
+ res.name + ",value=" + res.value + ",update=" + res.update)
}
}
+ // Create Task Objects and update the taskEnd dataStructure.
+ app.taskManager.addTaskFromEvent(event)
}
override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = {
@@ -442,6 +465,26 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi
event: SparkListenerStageCompleted): Unit = {
logDebug("Processing event: " + event.getClass)
app.getOrCreateStage(event.stageInfo)
+ // TODO: Should accumulators be added only if the stage is successful?
+ // Parse stage accumulables
+ for (res <- event.stageInfo.accumulables) {
+ try {
+ val accumInfo = res._2
+ EventUtils.buildTaskStageAccumFromAccumInfo(accumInfo,
+ event.stageInfo.stageId, event.stageInfo.attemptNumber()).foreach { thisMetric =>
+ val arrBuf = app.taskStageAccumMap.getOrElseUpdate(accumInfo.id,
+ ArrayBuffer[TaskStageAccumCase]())
+ arrBuf += thisMetric
+ }
+ } catch {
+ case NonFatal(e) =>
+ logWarning("Exception when parsing accumulables on stage-completed " +
+ "stageID=" + event.stageInfo.stageId + ": ")
+ logWarning(e.toString)
+ logWarning("The problematic accumulable is: name="
+ + res._2.name + ",value=" + res._2.value + ",update=" + res._2.update)
+ }
+ }
}
override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = {
diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/annotation/WallClock.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/annotation/WallClock.scala
index b90c9a89b..669828dca 100644
--- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/annotation/WallClock.scala
+++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/annotation/WallClock.scala
@@ -21,7 +21,7 @@ import scala.annotation.meta.{beanGetter, beanSetter, field, getter, param, sett
/**
- * A Scala annotation that specifies whether the type of duration: wallClockTime Vs. CPUTime
+ * A Scala annotation that specifies whether the type of duration: wallClockTime Vs. TaskDuration
*/
@param @field @getter @setter @beanGetter @beanSetter
class WallClock extends StaticAnnotation
diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala
index 96f47ab44..1bc172795 100644
--- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala
+++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala
@@ -17,21 +17,16 @@
package org.apache.spark.sql.rapids.tool.profiling
import scala.collection.{mutable, Map}
-import scala.collection.mutable.{ArrayBuffer, HashMap}
import com.nvidia.spark.rapids.tool.EventLogInfo
-import com.nvidia.spark.rapids.tool.planparser.SQLPlanParser
-import com.nvidia.spark.rapids.tool.profiling._
+import com.nvidia.spark.rapids.tool.analysis.AppSQLPlanAnalyzer
import org.apache.hadoop.conf.Configuration
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler._
import org.apache.spark.sql.execution.SparkPlanInfo
import org.apache.spark.sql.execution.metric.SQLMetricInfo
-import org.apache.spark.sql.execution.ui.SparkPlanGraph
-import org.apache.spark.sql.rapids.tool.{AppBase, RDDCheckHelper, ToolUtils}
-import org.apache.spark.sql.rapids.tool.SqlPlanInfoGraphBuffer
-import org.apache.spark.sql.rapids.tool.util.ToolsPlanGraph
+import org.apache.spark.sql.rapids.tool.AppBase
class SparkPlanInfoWithStage(
@@ -192,33 +187,20 @@ class ApplicationInfo(
val index: Int)
extends AppBase(Some(eLogInfo), Some(hadoopConf)) with Logging {
- // resourceprofile id to resource profile info
- val resourceProfIdToInfo = new HashMap[Int, ResourceProfileInfoCase]()
-
- var blockManagersRemoved: ArrayBuffer[BlockManagerRemovedCase] =
- ArrayBuffer[BlockManagerRemovedCase]()
-
// physicalPlanDescription stores HashMap (sqlID <-> physicalPlanDescription)
var physicalPlanDescription: mutable.HashMap[Long, String] = mutable.HashMap.empty[Long, String]
- var allSQLMetrics: ArrayBuffer[SQLMetricInfoCase] = ArrayBuffer[SQLMetricInfoCase]()
- var sqlPlanMetricsAdaptive: ArrayBuffer[SQLPlanMetricsCase] = ArrayBuffer[SQLPlanMetricsCase]()
-
- var taskEnd: ArrayBuffer[TaskCase] = ArrayBuffer[TaskCase]()
- var unsupportedSQLplan: ArrayBuffer[UnsupportedSQLPlan] = ArrayBuffer[UnsupportedSQLPlan]()
- var wholeStage: ArrayBuffer[WholeStageCodeGenResults] = ArrayBuffer[WholeStageCodeGenResults]()
- val sqlPlanNodeIdToStageIds: mutable.HashMap[(Long, Long), Set[Int]] =
- mutable.HashMap.empty[(Long, Long), Set[Int]]
-
private lazy val eventProcessor = new EventsProcessor(this)
// Process all events
processEvents()
+
// Process SQL Plan Metrics after all events are processed
- processSQLPlanMetrics()
+ val planMetricProcessor: AppSQLPlanAnalyzer = AppSQLPlanAnalyzer.processSQLPlan(this)
+ // finally aggregate the Info
aggregateAppInfo
- override def processEvent(event: SparkListenerEvent) = {
+ override def processEvent(event: SparkListenerEvent): Boolean = {
eventProcessor.processAnyEvent(event)
false
}
@@ -227,150 +209,6 @@ class ApplicationInfo(
clusterInfo = buildClusterInfo
}
- /**
- * Connects Operators to Stages using AccumulatorIDs
- * @param cb function that creates a SparkPlanGraph. This can be used as a cacheHolder for the
- * object created to be used later.
- */
- private def connectOperatorToStage(cb: (Long, SparkPlanInfo) => SparkPlanGraph): Unit = {
- for ((sqlId, planInfo) <- sqlPlans) {
- val planGraph: SparkPlanGraph = cb.apply(sqlId, planInfo)
- // Maps stages to operators by checking for non-zero intersection
- // between nodeMetrics and stageAccumulateIDs
- val nodeIdToStage = planGraph.allNodes.map { node =>
- val mappedStages = SQLPlanParser.getStagesInSQLNode(node, this)
- ((sqlId, node.id), mappedStages)
- }.toMap
- sqlPlanNodeIdToStageIds ++= nodeIdToStage
- }
- }
-
- /**
- * Function to process SQL Plan Metrics after all events are processed
- */
- def processSQLPlanMetrics(): Unit = {
- // Define a buffer to cache the SQLPlanInfoGraphs
- val sqlPlanInfoBuffer = SqlPlanInfoGraphBuffer()
- // Define a function used to fill in the buffer while executing "connectOperatorToStage"
- val createGraphFunc = (sqlId: Long, planInfo: SparkPlanInfo) => {
- sqlPlanInfoBuffer.addSqlPlanInfoGraph(sqlId, planInfo).sparkPlanGraph
- }
- connectOperatorToStage(createGraphFunc)
- for (sqlPIGEntry <- sqlPlanInfoBuffer.sqlPlanInfoGraphs) {
- var sqlIsDsOrRDD = false
- val potentialProblems = collection.mutable.Set[String]()
- // store all datasources of the given SQL in a variable so that we won't have to iterate
- // through the entire list
- // get V1 dataSources for that SQLId
- val sqlDataSources = checkMetadataForReadSchema(sqlPIGEntry)
- for (node <- sqlPIGEntry.sparkPlanGraph.allNodes) {
- var nodeIsDsOrRDD = false
- if (node.isInstanceOf[org.apache.spark.sql.execution.ui.SparkPlanGraphCluster]) {
- val ch = node.asInstanceOf[org.apache.spark.sql.execution.ui.SparkPlanGraphCluster].nodes
- ch.foreach { c =>
- wholeStage += WholeStageCodeGenResults(
- index, sqlPIGEntry.sqlID, node.id, node.name, c.name, c.id)
- }
- }
- // get V2 dataSources for that node
- val nodeV2Reads = checkGraphNodeForReads(sqlPIGEntry.sqlID, node)
- if (nodeV2Reads.isDefined) {
- sqlDataSources += nodeV2Reads.get
- }
- nodeIsDsOrRDD = RDDCheckHelper.isDatasetOrRDDPlan(node.name, node.desc).isRDD
- if (nodeIsDsOrRDD) {
- if (gpuMode) { // we want to report every node that is an RDD
- val thisPlan = UnsupportedSQLPlan(sqlPIGEntry.sqlID, node.id, node.name, node.desc,
- "Contains Dataset or RDD")
- unsupportedSQLplan += thisPlan
- }
- // If one node is RDD, the Sql should be set too
- if (!sqlIsDsOrRDD) { // We need to set the flag only once for the given sqlID
- sqlIsDsOrRDD = true
- sqlIdToInfo.get(sqlPIGEntry.sqlID).foreach { sql =>
- sql.setDsOrRdd(sqlIsDsOrRDD)
- sqlIDToDataSetOrRDDCase += sqlPIGEntry.sqlID
- // Clear the potential problems since it is an RDD to free memory
- potentialProblems.clear()
- }
- }
- }
- if (!sqlIsDsOrRDD) {
- // Append current node's potential problems to the Sql problems only if the SQL is not an
- // RDD. This is an optimization since the potentialProblems won't be used any more.
- potentialProblems ++= findPotentialIssues(node.desc)
- }
- // Then process SQL plan metric type
- for (metric <- node.metrics) {
- val stages =
- sqlPlanNodeIdToStageIds.get((sqlPIGEntry.sqlID, node.id)).getOrElse(Set.empty)
- val allMetric = SQLMetricInfoCase(sqlPIGEntry.sqlID, metric.name,
- metric.accumulatorId, metric.metricType, node.id,
- node.name, node.desc, stages)
-
- allSQLMetrics += allMetric
- if (this.sqlPlanMetricsAdaptive.nonEmpty) {
- val adaptive = sqlPlanMetricsAdaptive.filter { adaptiveMetric =>
- adaptiveMetric.sqlID == sqlPIGEntry.sqlID &&
- adaptiveMetric.accumulatorId == metric.accumulatorId
- }
- adaptive.foreach { adaptiveMetric =>
- val allMetric = SQLMetricInfoCase(sqlPIGEntry.sqlID, adaptiveMetric.name,
- adaptiveMetric.accumulatorId, adaptiveMetric.metricType, node.id,
- node.name, node.desc, stages)
- // could make this more efficient but seems ok for now
- val exists = allSQLMetrics.filter { a =>
- ((a.accumulatorId == adaptiveMetric.accumulatorId) && (a.sqlID == sqlPIGEntry.sqlID)
- && (a.nodeID == node.id && adaptiveMetric.metricType == a.metricType))
- }
- if (exists.isEmpty) {
- allSQLMetrics += allMetric
- }
- }
- }
- }
- }
- // Check if readsSchema is complex for the given sql
- val sqlNestedComplexTypes =
- AppBase.parseReadSchemaForNestedTypes(sqlDataSources.map { ds => ds.schema })
- // Append problematic issues to the global variable for that SqlID
- if (sqlNestedComplexTypes._2.nonEmpty) {
- potentialProblems += "NESTED COMPLEX TYPE"
- }
- // Finally, add the local potentialProblems to the global data structure if any.
- sqlIDtoProblematic(sqlPIGEntry.sqlID) = potentialProblems.toSet
- // Convert the problematic issues to a string and update the SQLInfo
- sqlIdToInfo.get(sqlPIGEntry.sqlID).foreach { sqlInfoClass =>
- sqlInfoClass.problematic = ToolUtils.formatPotentialProblems(potentialProblems.toSeq)
- }
- }
- }
-
- def aggregateSQLStageInfo: Seq[SQLStageInfoProfileResult] = {
- val jobsWithSQL = jobIdToInfo.filter { case (_, j) =>
- j.sqlID.nonEmpty
- }
- val sqlToStages = jobsWithSQL.flatMap { case (jobId, j) =>
- val stages = j.stageIds
- val stagesInJob = stageManager.getStagesByIds(stages)
- stagesInJob.map { sModel =>
- val nodeIds = sqlPlanNodeIdToStageIds.filter { case (_, v) =>
- v.contains(sModel.sId)
- }.keys.toSeq
- val nodeNames = sqlPlans.get(j.sqlID.get).map { planInfo =>
- val nodes = ToolsPlanGraph(planInfo).allNodes
- val validNodes = nodes.filter { n =>
- nodeIds.contains((j.sqlID.get, n.id))
- }
- validNodes.map(n => s"${n.name}(${n.id.toString})")
- }.getOrElse(Seq.empty)
- SQLStageInfoProfileResult(index, j.sqlID.get, jobId, sModel.sId,
- sModel.attemptId, sModel.duration, nodeNames)
- }
- }
- sqlToStages.toSeq
- }
-
private def aggregateAppInfo: Unit = {
estimateAppEndTime { () =>
val jobEndTimes = jobIdToInfo.map { case (_, jc) => jc.endTime }.filter(_.isDefined)
diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/EventsProcessor.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/EventsProcessor.scala
index 50818f338..edebb3548 100644
--- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/EventsProcessor.scala
+++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/EventsProcessor.scala
@@ -16,20 +16,15 @@
package org.apache.spark.sql.rapids.tool.profiling
-import java.util.concurrent.TimeUnit
-
import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-import scala.util.control.NonFatal
import com.nvidia.spark.rapids.tool.profiling._
-import org.apache.spark.TaskFailedReason
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler._
-import org.apache.spark.sql.execution.ui.{SparkListenerSQLAdaptiveExecutionUpdate, SparkListenerSQLAdaptiveSQLMetricUpdates, SparkListenerSQLExecutionStart}
+import org.apache.spark.sql.execution.ui.{SparkListenerSQLAdaptiveExecutionUpdate, SparkListenerSQLExecutionStart}
import org.apache.spark.sql.rapids.tool.EventProcessorBase
-import org.apache.spark.sql.rapids.tool.util.{EventUtils, StringUtils}
+import org.apache.spark.sql.rapids.tool.util.StringUtils
/**
* This class is to process all events and do validation in the end.
@@ -84,30 +79,6 @@ class EventsProcessor(app: ApplicationInfo) extends EventProcessorBase[Applicati
}
}
- override def doSparkListenerResourceProfileAdded(
- app: ApplicationInfo,
- event: SparkListenerResourceProfileAdded): Unit = {
-
- logDebug("Processing event: " + event.getClass)
- // leave off maxTasks for now
- val rp = ResourceProfileInfoCase(event.resourceProfile.id,
- event.resourceProfile.executorResources, event.resourceProfile.taskResources)
- app.resourceProfIdToInfo(event.resourceProfile.id) = rp
- }
-
- override def doSparkListenerBlockManagerRemoved(
- app: ApplicationInfo,
- event: SparkListenerBlockManagerRemoved): Unit = {
- logDebug("Processing event: " + event.getClass)
- val thisBlockManagerRemoved = BlockManagerRemovedCase(
- event.blockManagerId.executorId,
- event.blockManagerId.host,
- event.blockManagerId.port,
- event.time
- )
- app.blockManagersRemoved += thisBlockManagerRemoved
- }
-
override def doSparkListenerEnvironmentUpdate(
app: ApplicationInfo,
event: SparkListenerEnvironmentUpdate): Unit = {
@@ -117,62 +88,6 @@ class EventsProcessor(app: ApplicationInfo) extends EventProcessorBase[Applicati
logDebug(s"App's GPU Mode = ${app.gpuMode}")
}
- override def doSparkListenerTaskEnd(
- app: ApplicationInfo,
- event: SparkListenerTaskEnd): Unit = {
- logDebug("Processing event: " + event.getClass)
- super.doSparkListenerTaskEnd(app, event)
- val reason = event.reason match {
- case failed: TaskFailedReason =>
- failed.toErrorString
- case _ =>
- event.reason.toString
- }
-
- val thisTask = TaskCase(
- event.stageId,
- event.stageAttemptId,
- event.taskType,
- reason,
- event.taskInfo.taskId,
- event.taskInfo.attemptNumber,
- event.taskInfo.launchTime,
- event.taskInfo.finishTime,
- event.taskInfo.duration,
- event.taskInfo.successful,
- event.taskInfo.executorId,
- event.taskInfo.host,
- event.taskInfo.taskLocality.toString,
- event.taskInfo.speculative,
- event.taskInfo.gettingResultTime,
- event.taskMetrics.executorDeserializeTime,
- TimeUnit.NANOSECONDS.toMillis(event.taskMetrics.executorDeserializeCpuTime),
- event.taskMetrics.executorRunTime,
- TimeUnit.NANOSECONDS.toMillis(event.taskMetrics.executorCpuTime),
- event.taskMetrics.peakExecutionMemory,
- event.taskMetrics.resultSize,
- event.taskMetrics.jvmGCTime,
- event.taskMetrics.resultSerializationTime,
- event.taskMetrics.memoryBytesSpilled,
- event.taskMetrics.diskBytesSpilled,
- event.taskMetrics.shuffleReadMetrics.remoteBlocksFetched,
- event.taskMetrics.shuffleReadMetrics.localBlocksFetched,
- event.taskMetrics.shuffleReadMetrics.fetchWaitTime,
- event.taskMetrics.shuffleReadMetrics.remoteBytesRead,
- event.taskMetrics.shuffleReadMetrics.remoteBytesReadToDisk,
- event.taskMetrics.shuffleReadMetrics.localBytesRead,
- event.taskMetrics.shuffleReadMetrics.totalBytesRead,
- event.taskMetrics.shuffleWriteMetrics.bytesWritten,
- TimeUnit.NANOSECONDS.toMillis(event.taskMetrics.shuffleWriteMetrics.writeTime),
- event.taskMetrics.shuffleWriteMetrics.recordsWritten,
- event.taskMetrics.inputMetrics.bytesRead,
- event.taskMetrics.inputMetrics.recordsRead,
- event.taskMetrics.outputMetrics.bytesWritten,
- event.taskMetrics.outputMetrics.recordsWritten
- )
- app.taskEnd += thisTask
- }
-
override def doSparkListenerSQLExecutionStart(
app: ApplicationInfo,
event: SparkListenerSQLExecutionStart): Unit = {
@@ -180,33 +95,6 @@ class EventsProcessor(app: ApplicationInfo) extends EventProcessorBase[Applicati
app.physicalPlanDescription += (event.executionId -> event.physicalPlanDescription)
}
- override def doSparkListenerStageCompleted(
- app: ApplicationInfo,
- event: SparkListenerStageCompleted): Unit = {
- logDebug("Processing event: " + event.getClass)
- super.doSparkListenerStageCompleted(app, event)
-
- // Parse stage accumulables
- for (res <- event.stageInfo.accumulables) {
- try {
- val accumInfo = res._2
- EventUtils.buildTaskStageAccumFromAccumInfo(accumInfo,
- event.stageInfo.stageId, event.stageInfo.attemptNumber()).foreach { thisMetric =>
- val arrBuf = app.taskStageAccumMap.getOrElseUpdate(accumInfo.id,
- ArrayBuffer[TaskStageAccumCase]())
- arrBuf += thisMetric
- }
- } catch {
- case NonFatal(e) =>
- logWarning("Exception when parsing accumulables on stage-completed " +
- "stageID=" + event.stageInfo.stageId + ": ")
- logWarning(e.toString)
- logWarning("The problematic accumulable is: name="
- + res._2.name + ",value=" + res._2.value + ",update=" + res._2.update)
- }
- }
- }
-
override def doSparkListenerTaskGettingResult(
app: ApplicationInfo,
event: SparkListenerTaskGettingResult): Unit = {
@@ -222,18 +110,6 @@ class EventsProcessor(app: ApplicationInfo) extends EventProcessorBase[Applicati
super.doSparkListenerSQLAdaptiveExecutionUpdate(app, event)
}
- override def doSparkListenerSQLAdaptiveSQLMetricUpdates(
- app: ApplicationInfo,
- event: SparkListenerSQLAdaptiveSQLMetricUpdates): Unit = {
- logDebug("Processing event: " + event.getClass)
- val SparkListenerSQLAdaptiveSQLMetricUpdates(sqlID, sqlPlanMetrics) = event
- val metrics = sqlPlanMetrics.map { metric =>
- SQLPlanMetricsCase(sqlID, metric.name,
- metric.accumulatorId, metric.metricType)
- }
- app.sqlPlanMetricsAdaptive ++= metrics
- }
-
// To process all other unknown events
override def doOtherEvent(app: ApplicationInfo, event: SparkListenerEvent): Unit = {
logDebug("Skipping unhandled event: " + event.getClass)
diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/TaskModel.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/TaskModel.scala
new file mode 100644
index 000000000..39b396bfc
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/TaskModel.scala
@@ -0,0 +1,119 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.rapids.tool.store
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.spark.TaskFailedReason
+import org.apache.spark.scheduler.SparkListenerTaskEnd
+import org.apache.spark.sql.rapids.tool.annotation.Since
+
+@Since("24.04.1")
+case class TaskModel(
+ stageId: Int,
+ stageAttemptId: Int,
+ taskType: String,
+ endReason: String,
+ taskId: Long,
+ attempt: Int,
+ launchTime: Long,
+ finishTime: Long,
+ duration: Long,
+ successful: Boolean,
+ executorId: String,
+ host: String,
+ taskLocality: String,
+ speculative: Boolean,
+ gettingResultTime: Long,
+ executorDeserializeTime: Long,
+ executorDeserializeCPUTime: Long,
+ executorRunTime: Long,
+ executorCPUTime: Long,
+ peakExecutionMemory: Long,
+ resultSize: Long,
+ jvmGCTime: Long,
+ resultSerializationTime: Long,
+ memoryBytesSpilled: Long,
+ diskBytesSpilled: Long,
+ // Note: sr stands for ShuffleRead
+ sr_remoteBlocksFetched: Long,
+ sr_localBlocksFetched: Long,
+ sr_fetchWaitTime: Long,
+ sr_remoteBytesRead: Long,
+ sr_remoteBytesReadToDisk: Long,
+ sr_localBytesRead: Long,
+ sr_totalBytesRead: Long,
+ // Note: sw stands for ShuffleWrite
+ sw_bytesWritten: Long,
+ sw_writeTime: Long,
+ sw_recordsWritten: Long,
+ input_bytesRead: Long,
+ input_recordsRead: Long,
+ output_bytesWritten: Long,
+ output_recordsWritten: Long)
+
+object TaskModel {
+ def apply(event: SparkListenerTaskEnd): TaskModel = {
+ val reason = event.reason match {
+ case failed: TaskFailedReason =>
+ failed.toErrorString
+ case _ =>
+ event.reason.toString
+ }
+
+ TaskModel(
+ event.stageId,
+ event.stageAttemptId,
+ event.taskType,
+ reason,
+ event.taskInfo.taskId,
+ event.taskInfo.attemptNumber,
+ event.taskInfo.launchTime,
+ event.taskInfo.finishTime,
+ event.taskInfo.duration,
+ event.taskInfo.successful,
+ event.taskInfo.executorId,
+ event.taskInfo.host,
+ event.taskInfo.taskLocality.toString,
+ event.taskInfo.speculative,
+ event.taskInfo.gettingResultTime,
+ event.taskMetrics.executorDeserializeTime,
+ TimeUnit.NANOSECONDS.toMillis(event.taskMetrics.executorDeserializeCpuTime),
+ event.taskMetrics.executorRunTime,
+ TimeUnit.NANOSECONDS.toMillis(event.taskMetrics.executorCpuTime),
+ event.taskMetrics.peakExecutionMemory,
+ event.taskMetrics.resultSize,
+ event.taskMetrics.jvmGCTime,
+ event.taskMetrics.resultSerializationTime,
+ event.taskMetrics.memoryBytesSpilled,
+ event.taskMetrics.diskBytesSpilled,
+ event.taskMetrics.shuffleReadMetrics.remoteBlocksFetched,
+ event.taskMetrics.shuffleReadMetrics.localBlocksFetched,
+ event.taskMetrics.shuffleReadMetrics.fetchWaitTime,
+ event.taskMetrics.shuffleReadMetrics.remoteBytesRead,
+ event.taskMetrics.shuffleReadMetrics.remoteBytesReadToDisk,
+ event.taskMetrics.shuffleReadMetrics.localBytesRead,
+ event.taskMetrics.shuffleReadMetrics.totalBytesRead,
+ event.taskMetrics.shuffleWriteMetrics.bytesWritten,
+ TimeUnit.NANOSECONDS.toMillis(event.taskMetrics.shuffleWriteMetrics.writeTime),
+ event.taskMetrics.shuffleWriteMetrics.recordsWritten,
+ event.taskMetrics.inputMetrics.bytesRead,
+ event.taskMetrics.inputMetrics.recordsRead,
+ event.taskMetrics.outputMetrics.bytesWritten,
+ event.taskMetrics.outputMetrics.recordsWritten)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/TaskModelManager.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/TaskModelManager.scala
new file mode 100644
index 000000000..ae1f6bd3f
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/TaskModelManager.scala
@@ -0,0 +1,109 @@
+/*
+ * Copyright (c) 2024, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.rapids.tool.store
+
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.scheduler.SparkListenerTaskEnd
+import org.apache.spark.sql.rapids.tool.annotation.Since
+
+/**
+ * A class to maintain the tasks.
+ * There is an alternative design that consists of using StageModel as a parent holding a list of
+ * tasks.
+ * However, the decision to use a standalone TaskModelManager at the moment is to achieve the
+ * following targets:
+ * 1- separation between managing the Stages and their Tasks. It is known that Tasks represent the
+ * highest percentage of allocated objects in the memory. This allows us to drop the entire tasks
+ * management if we dice to aggregate the metrics through the Acccumulables list.
+ * 2- flexibility in refactoring the TaskManager to use a permanent storage toward future
+ * improvements.
+ */
+@Since("24.04.1")
+class TaskModelManager {
+ // A nested HashMap to map between ((Int: stageId, Int: attemptId) -> ArrayBuffer[TaskModel]).
+ // We keep track of the attemptId to allow improvement down the road if we decide to handle
+ // different Attempts.
+ // A new Task is added by TaskEnd event handler.
+ // - 1st level maps between [Int: stageId -> 2nd Level]
+ // - 2nd level maps between [Int: attemptId -> ArrayBuffer[TaskModel]]
+ // Use Nested Maps to store taskModels which should be faster to retrieve than a map of
+ // composite key (i.e., Tuple).
+ // Composite keys would cost more because it implicitly allocates a new object every time there
+ // is a read operation from the map.
+ val stageAttemptToTasks: mutable.HashMap[Int, mutable.HashMap[Int, ArrayBuffer[TaskModel]]] =
+ new mutable.HashMap[Int, mutable.HashMap[Int, ArrayBuffer[TaskModel]]]()
+
+ // Given a Spark taskEnd event, create a new Task and add it to the Map.
+ def addTaskFromEvent(event: SparkListenerTaskEnd): Unit = {
+ val taskModel = TaskModel(event)
+ val stageAttempts =
+ stageAttemptToTasks.getOrElseUpdate(event.stageId,
+ new mutable.HashMap[Int, ArrayBuffer[TaskModel]]())
+ val attemptToTasks =
+ stageAttempts.getOrElseUpdate(event.stageAttemptId, ArrayBuffer[TaskModel]())
+ attemptToTasks += taskModel
+ }
+
+ // Given a stageID and stageAttemptID, return all tasks or Empty iterable.
+ // The returned tasks will be filtered by the the predicate func if the latter exists
+ def getTasks(stageID: Int, stageAttemptID: Int,
+ predicateFunc: Option[TaskModel => Boolean] = None): Iterable[TaskModel] = {
+ stageAttemptToTasks.get(stageID).flatMap { stageAttempts =>
+ stageAttempts.get(stageAttemptID).map { tasks =>
+ if (predicateFunc.isDefined) {
+ tasks.filter(predicateFunc.get)
+ } else {
+ tasks
+ }
+ }
+ }.getOrElse(Iterable.empty)
+ }
+
+ // Returns the combined list of tasks that belong to a specific stageID.
+ // This includes tasks belonging to different stageAttempts.
+ // This is mainly supporting callers that use stageID (without attemptID).
+ def getAllTasksStageAttempt(stageID: Int): Iterable[TaskModel] = {
+ stageAttemptToTasks.get(stageID).map { stageAttempts =>
+ stageAttempts.values.flatten
+ }.getOrElse(Iterable.empty)
+ }
+
+ // Returns the combined list of all tasks that satisfy the predicate function if it exists.
+ def getAllTasks(predicateFunc: Option[TaskModel => Boolean] = None): Iterable[TaskModel] = {
+ stageAttemptToTasks.collect {
+ case (_, attemptsToTasks) if attemptsToTasks.nonEmpty =>
+ if (predicateFunc.isDefined) {
+ attemptsToTasks.values.flatten.filter(predicateFunc.get)
+ } else {
+ attemptsToTasks.values.flatten
+ }
+ }.flatten
+ }
+
+ // Return a list of tasks that failed within all the stageAttempts
+ def getAllFailedTasks: Iterable[TaskModel] = {
+ getAllTasks(Some(!_.successful))
+ }
+
+ // Given an iterable of StageIDs, return all the tasks that belong to these stages. Note that
+ // this include tasks within multiple stageAttempts.
+ // This is implemented to support callers that do not use stageAttemptID in their logic.
+ def getTasksByStageIds(stageIds: Iterable[Int]): Iterable[TaskModel] = {
+ stageIds.flatMap(getAllTasksStageAttempt)
+ }
+}
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 0adc438e6..ba5efe441 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
@@ -19,6 +19,7 @@ package com.nvidia.spark.rapids.tool.profiling
import java.io.File
import com.nvidia.spark.rapids.tool.ToolTestUtils
+import com.nvidia.spark.rapids.tool.views.RawMetricProfilerView
import org.scalatest.FunSuite
import org.apache.spark.sql.SparkSession
@@ -63,16 +64,15 @@ class AnalysisSuite extends FunSuite {
expectFileJS: String): Unit = {
val apps = ToolTestUtils.processProfileApps(logs, sparkSession)
assert(apps.size == logs.size)
- val analysis = new Analysis(apps)
-
- val sqlTaskMetrics = analysis.sqlMetricsAggregation()
+ val aggResults = RawMetricProfilerView.getAggMetrics(apps)
+ val sqlTaskMetrics = aggResults.sqlAggs
val resultExpectation = new File(expRoot, expectFile)
import sparkSession.implicits._
val actualDf = sqlTaskMetrics.toDF.drop(skippedColumnsInSqlAggProfile:_*)
val dfExpect = ToolTestUtils.readExpectationCSV(sparkSession, resultExpectation.getPath())
ToolTestUtils.compareDataFrames(actualDf, dfExpect)
- val jobStageMetrics = analysis.jobAndStageMetricsAggregation()
+ val jobStageMetrics = aggResults.jobStageAggs
val resultExpectationJS = new File(expRoot, expectFileJS)
val actualDfJS = jobStageMetrics.toDF
val dfExpectJS = ToolTestUtils.readExpectationCSV(sparkSession, resultExpectationJS.getPath())
@@ -84,11 +84,9 @@ class AnalysisSuite extends FunSuite {
val expectFile = "rapids_duration_and_cpu_expectation.csv"
val apps = ToolTestUtils.processProfileApps(logs, sparkSession)
- val analysis = new Analysis(apps)
- // have to call this to set all the fields properly
- analysis.sqlMetricsAggregation()
+ val aggResults = RawMetricProfilerView.getAggMetrics(apps)
import sparkSession.implicits._
- val sqlAggDurCpu = analysis.sqlMetricsAggregationDurationAndCpuTime()
+ val sqlAggDurCpu = aggResults.sqlDurAggs
val resultExpectation = new File(expRoot, expectFile)
val schema = new StructType()
.add("appIndex",IntegerType,true)
@@ -113,8 +111,8 @@ class AnalysisSuite extends FunSuite {
ToolTestUtils.processProfileApps(Array(s"$logDir/rapids_join_eventlog.zstd"), sparkSession)
assert(apps.size == 1)
- val analysis = new Analysis(apps)
- val shuffleSkewInfo = analysis.shuffleSkewCheck()
+ val aggResults = RawMetricProfilerView.getAggMetrics(apps)
+ val shuffleSkewInfo = aggResults.taskShuffleSkew
assert(shuffleSkewInfo.isEmpty)
}
@@ -123,8 +121,8 @@ class AnalysisSuite extends FunSuite {
val logs = Array(s"$qualLogDir/nds_q86_test")
val apps = ToolTestUtils.processProfileApps(logs, sparkSession)
- val analysis = new Analysis(apps)
- val sqlDurAndCpu = analysis.sqlMetricsAggregationDurationAndCpuTime()
+ val aggResults = RawMetricProfilerView.getAggMetrics(apps)
+ val sqlDurAndCpu = aggResults.sqlDurAggs
val containsDs = sqlDurAndCpu.filter(_.containsDataset === true)
assert(containsDs.isEmpty)
}
@@ -134,8 +132,8 @@ class AnalysisSuite extends FunSuite {
val logs = Array(s"$qualLogDir/dataset_eventlog")
val apps = ToolTestUtils.processProfileApps(logs, sparkSession)
- val analysis = new Analysis(apps)
- val sqlDurAndCpu = analysis.sqlMetricsAggregationDurationAndCpuTime()
+ val aggResults = RawMetricProfilerView.getAggMetrics(apps)
+ val sqlDurAndCpu = aggResults.sqlDurAggs
val containsDs = sqlDurAndCpu.filter(_.containsDataset === true)
assert(containsDs.size == 1)
}
@@ -148,8 +146,8 @@ class AnalysisSuite extends FunSuite {
apps.foreach { app =>
app.appMetaData = None
}
- val analysis = new Analysis(apps)
- val metrics = analysis.sqlMetricsAggregationDurationAndCpuTime()
+ val aggResults = RawMetricProfilerView.getAggMetrics(apps)
+ val metrics = aggResults.sqlDurAggs
metrics.foreach(m => assert(m.appDuration.get == 0L))
}
}
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 302b729d0..53f2185e2 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
@@ -22,6 +22,7 @@ import java.nio.file.{Files, Paths, StandardOpenOption}
import scala.collection.mutable.ArrayBuffer
import com.nvidia.spark.rapids.tool.{EventLogPathProcessor, ToolTestUtils}
+import com.nvidia.spark.rapids.tool.views.RawMetricProfilerView
import org.apache.hadoop.io.IOUtils
import org.scalatest.FunSuite
@@ -98,8 +99,8 @@ class ApplicationInfoSuite extends FunSuite with Logging {
val stageInfo = firstApp.stageManager.getStage(0, 0)
assert(stageInfo.isDefined && stageInfo.get.sInfo.numTasks.equals(1))
assert(firstApp.stageManager.getStage(2, 0).isDefined)
- assert(firstApp.taskEnd(firstApp.index).successful.equals(true))
- assert(firstApp.taskEnd(firstApp.index).endReason.equals("Success"))
+ assert(firstApp.taskManager.getTasks(firstApp.index, 0).head.successful.equals(true))
+ assert(firstApp.taskManager.getTasks(firstApp.index, 0).head.endReason.equals("Success"))
val execInfo = firstApp.executorIdToInfo.get(firstApp.executorIdToInfo.keys.head)
assert(execInfo.isDefined && execInfo.get.totalCores.equals(8))
val rp = firstApp.resourceProfIdToInfo.get(firstApp.resourceProfIdToInfo.keys.head)
@@ -376,8 +377,8 @@ class ApplicationInfoSuite extends FunSuite with Logging {
index += 1
}
assert(apps.size == 1)
- val analysis = new Analysis(apps)
- val ioMetrics = analysis.ioAnalysis()
+ val aggResults = RawMetricProfilerView.getAggMetrics(apps)
+ val ioMetrics = aggResults.ioAggs
assert(ioMetrics.size == 5)
val metricsSqlId1 = ioMetrics.filter(metrics => metrics.sqlId == 1)
assert(metricsSqlId1.size == 1)