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)