Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

QualificationTool. Add speedup information to AppSummaryInfo #5454

Merged
merged 2 commits into from
May 11, 2022
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import org.apache.spark.sql.rapids.tool.qualification.QualificationSummaryInfo
*/
class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, printStdout: Boolean) {

private val finalOutputDir = s"$outputDir/rapids_4_spark_qualification_output"
// a file extension will be added to this later
private val logFileName = "rapids_4_spark_qualification_output"

Expand All @@ -44,7 +43,7 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, printStdout
}

def writeCSV(sums: Seq[QualificationSummaryInfo]): Unit = {
val csvFileWriter = new ToolTextFileWriter(finalOutputDir, s"${logFileName}.csv", "CSV")
val csvFileWriter = new ToolTextFileWriter(outputDir, s"${logFileName}.csv", "CSV")
try {
val headersAndSizes = QualOutputWriter
.getDetailedHeaderStringsAndSizes(sums, reportReadSchema)
Expand All @@ -60,7 +59,7 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, printStdout

// write the text summary report
def writeReport(summaries: Seq[QualificationSummaryInfo], numOutputRows: Int) : Unit = {
val textFileWriter = new ToolTextFileWriter(finalOutputDir, s"${logFileName}.log",
val textFileWriter = new ToolTextFileWriter(outputDir, s"${logFileName}.log",
"Summary report")
try {
writeTextSummary(textFileWriter, summaries, numOutputRows)
Expand Down Expand Up @@ -114,9 +113,20 @@ object QualOutputWriter {
val COMPLEX_TYPES_STR = "Complex Types"
val NESTED_TYPES_STR = "Nested Complex Types"
val READ_SCHEMA_STR = "Read Schema"
val NONSQL_DUR_STR = "NONSQL Task Duration Plug Overhead"
amahussein marked this conversation as resolved.
Show resolved Hide resolved
val ESTIMATED_DURATION_STR = "Estimated Duration"
val UNSUPPORTED_DURATION_STR = "Unsupported Duration"
val SPEEDUP_DURATION_STR = "Speedup Duration"
val SPEEDUP_FACTOR_STR = "Speedup Factor"
val TOTAL_SPEEDUP_STR = "Total Speedup"
val SPEEDUP_BUCKET_STR = "Recommendation"
val LONGEST_SQL_DURATION_STR = "Longest SQL Duration"
val APP_DUR_STR_SIZE: Int = APP_DUR_STR.size
val SQL_DUR_STR_SIZE: Int = SQL_DUR_STR.size
val PROBLEM_DUR_SIZE: Int = PROBLEM_DUR_STR.size
val SPEEDUP_BUCKET_STR_SIZE: Int = SPEEDUP_BUCKET_STR.size
val TOTAL_SPEEDUP_STR_SIZE: Int = TOTAL_SPEEDUP_STR.size
val LONGEST_SQL_DURATION_STR_SIZE: Int = LONGEST_SQL_DURATION_STR.size

def getAppIdSize(sums: Seq[QualificationSummaryInfo]): Int = {
val sizes = sums.map(_.appId.size)
Expand Down Expand Up @@ -172,7 +182,9 @@ object QualOutputWriter {
APP_ID_STR -> appIdMaxSize,
APP_DUR_STR -> APP_DUR_STR_SIZE,
SQL_DUR_STR -> SQL_DUR_STR_SIZE,
PROBLEM_DUR_STR -> PROBLEM_DUR_SIZE
PROBLEM_DUR_STR -> PROBLEM_DUR_SIZE,
TOTAL_SPEEDUP_STR -> TOTAL_SPEEDUP_STR_SIZE,
SPEEDUP_BUCKET_STR -> SPEEDUP_BUCKET_STR_SIZE
)
}

Expand Down Expand Up @@ -205,7 +217,15 @@ object QualOutputWriter {
COMPLEX_TYPES_STR ->
getMaxSizeForHeader(appInfos.map(_.complexTypes.size), COMPLEX_TYPES_STR),
NESTED_TYPES_STR -> getMaxSizeForHeader(appInfos.map(_.nestedComplexTypes.size),
NESTED_TYPES_STR)
NESTED_TYPES_STR),
LONGEST_SQL_DURATION_STR -> LONGEST_SQL_DURATION_STR_SIZE,
NONSQL_DUR_STR -> NONSQL_DUR_STR.size,
ESTIMATED_DURATION_STR -> ESTIMATED_DURATION_STR.size,
UNSUPPORTED_DURATION_STR -> UNSUPPORTED_DURATION_STR.size,
SPEEDUP_DURATION_STR -> SPEEDUP_DURATION_STR.size,
SPEEDUP_FACTOR_STR -> SPEEDUP_FACTOR_STR.size,
TOTAL_SPEEDUP_STR -> TOTAL_SPEEDUP_STR.size,
SPEEDUP_BUCKET_STR -> SPEEDUP_BUCKET_STR.size
)
if (reportReadSchema) {
detailedHeadersAndFields +=
Expand All @@ -227,7 +247,9 @@ object QualOutputWriter {
sumInfo.appId -> appIdMaxSize,
sumInfo.appDuration.toString -> APP_DUR_STR_SIZE,
sumInfo.sqlDataFrameDuration.toString -> SQL_DUR_STR_SIZE,
sumInfo.sqlDurationForProblematic.toString -> PROBLEM_DUR_SIZE
sumInfo.sqlDurationForProblematic.toString -> PROBLEM_DUR_SIZE,
sumInfo.totalSpeedup.toString -> TOTAL_SPEEDUP_STR_SIZE,
sumInfo.speedupBucket -> SPEEDUP_BUCKET_STR_SIZE
)
constructOutputRow(data, delimiter, prettyPrint)
}
Expand Down Expand Up @@ -271,7 +293,16 @@ object QualOutputWriter {
readFileFormatsNotSupported -> headersAndSizes(READ_FILE_FORMAT_TYPES_STR),
dataWriteFormat -> headersAndSizes(WRITE_DATA_FORMAT_STR),
complexTypes -> headersAndSizes(COMPLEX_TYPES_STR),
nestedComplexTypes -> headersAndSizes(NESTED_TYPES_STR)
nestedComplexTypes -> headersAndSizes(NESTED_TYPES_STR),
appInfo.longestSqlDuration.toString -> headersAndSizes(LONGEST_SQL_DURATION_STR),
appInfo.nonSqlTaskDurationAndOverhead.toString -> headersAndSizes(NONSQL_DUR_STR),
appInfo.estimatedDuration.toString -> headersAndSizes(ESTIMATED_DURATION_STR),
appInfo.unsupportedDuration.toString ->
headersAndSizes(UNSUPPORTED_DURATION_STR),
appInfo.speedupDuration.toString -> headersAndSizes(SPEEDUP_DURATION_STR),
appInfo.speedupFactor.toString -> headersAndSizes(SPEEDUP_FACTOR_STR),
appInfo.totalSpeedup.toString -> headersAndSizes(TOTAL_SPEEDUP_STR),
stringIfempty(appInfo.speedupBucket) -> headersAndSizes(SPEEDUP_BUCKET_STR)
)

if (reportReadSchema) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ import org.apache.spark.sql.rapids.tool.qualification._
class Qualification(outputDir: String, numRows: Int, hadoopConf: Configuration,
timeout: Option[Long], nThreads: Int, order: String,
pluginTypeChecker: PluginTypeChecker, readScorePercent: Int,
reportReadSchema: Boolean, printStdout: Boolean) extends Logging {
reportReadSchema: Boolean, printStdout: Boolean, uiEnabled: Boolean = false) extends Logging {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would remove the uiEnabled option since its not included here.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree. uiEnabled option and it's related code can be added in a different PR where we integrate the UI.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

removed the new argument.


private val allApps = new ConcurrentLinkedQueue[QualificationSummaryInfo]()
// default is 24 hours
Expand Down Expand Up @@ -73,7 +73,7 @@ class Qualification(outputDir: String, numRows: Int, hadoopConf: Configuration,
val sortedDesc = allAppsSum.sortBy(sum => {
(-sum.score, -sum.sqlDataFrameDuration, -sum.appDuration)
})
val qWriter = new QualOutputWriter(outputDir, reportReadSchema, printStdout)
val qWriter = new QualOutputWriter(getReportOutputPath, reportReadSchema, printStdout)
qWriter.writeCSV(sortedDesc)

val sortedForReport = if (QualificationArgs.isOrderAsc(order)) {
Expand Down Expand Up @@ -119,4 +119,11 @@ class Qualification(outputDir: String, numRows: Int, hadoopConf: Configuration,
logWarning(s"Unexpected exception processing log ${path.eventLog.toString}, skipping!", e)
}
}

/**
* The outputPath of the current instance of the provider
*/
def getReportOutputPath: String = {
s"$outputDir/rapids_4_spark_qualification_output"
}
}
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright (c) 2021, NVIDIA CORPORATION.
* Copyright (c) 2021-2022, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand Down Expand Up @@ -130,6 +130,10 @@ Usage: java -cp rapids-4-spark-tools_2.12-<version>.jar:$SPARK_HOME/jars/*
val userName: ScallopOption[String] =
opt[String](required = false,
descr = "Applications which a particular user has submitted." )
val uiEnabled: ScallopOption[Boolean] =
opt[Boolean](required = false,
descr = "Whether to render the report into HTML pages. Default is False",
default = Some(false))

validate(order) {
case o if (QualificationArgs.isOrderAsc(o) || QualificationArgs.isOrderDesc(o)) => Right(Unit)
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright (c) 2021, NVIDIA CORPORATION.
* Copyright (c) 2021-2022, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand Down Expand Up @@ -54,7 +54,7 @@ object QualificationMain extends Logging {
val readScorePercent = appArgs.readScorePercent.getOrElse(20)
val reportReadSchema = appArgs.reportReadSchema.getOrElse(false)
val order = appArgs.order.getOrElse("desc")

val uiEnabled = appArgs.uiEnabled.getOrElse(false)
val hadoopConf = new Configuration()

val pluginTypeChecker = try {
Expand Down Expand Up @@ -86,7 +86,8 @@ object QualificationMain extends Logging {
}

val qual = new Qualification(outputDirectory, numOutputRows, hadoopConf, timeout,
nThreads, order, pluginTypeChecker, readScorePercent, reportReadSchema, printStdout)
nThreads, order, pluginTypeChecker, readScorePercent, reportReadSchema, printStdout,
uiEnabled)
val res = qual.qualifyApps(filteredLogs)
(0, res)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import scala.io.{Codec, Source}

import com.nvidia.spark.rapids.tool.{DatabricksEventLog, DatabricksRollingEventLogFilesFileReader, EventLogInfo}
import com.nvidia.spark.rapids.tool.planparser.ReadParser
import com.nvidia.spark.rapids.tool.profiling.{DataSourceCase, DriverAccumCase, StageInfoClass, TaskStageAccumCase}
import com.nvidia.spark.rapids.tool.profiling.{DataSourceCase, DriverAccumCase, JobInfoClass, StageInfoClass, TaskStageAccumCase}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.json4s.jackson.JsonMethods.parse
Expand All @@ -45,6 +45,9 @@ abstract class AppBase(
// The data source information
val dataSourceInfo: ArrayBuffer[DataSourceCase] = ArrayBuffer[DataSourceCase]()

// jobId to job info
val jobIdToInfo = new HashMap[Int, JobInfoClass]()

// SQL containing any Dataset operation or RDD to DataSet/DataFrame operation
val sqlIDToDataSetOrRDDCase: HashSet[Long] = HashSet[Long]()
val sqlIDtoProblematic: HashMap[Long, Set[String]] = HashMap[Long, Set[String]]()
Expand All @@ -59,6 +62,8 @@ abstract class AppBase(
var driverAccumMap: HashMap[Long, ArrayBuffer[DriverAccumCase]] =
HashMap[Long, ArrayBuffer[DriverAccumCase]]()

var gpuMode = false

def getOrCreateStage(info: StageInfo): StageInfoClass = {
val stage = stageIdToInfo.getOrElseUpdate((info.stageId, info.attemptNumber()),
new StageInfoClass(info))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,11 @@

package org.apache.spark.sql.rapids.tool

import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
import scala.util.control.NonFatal

import com.nvidia.spark.rapids.tool.profiling.{DriverAccumCase, ProfileUtils, TaskStageAccumCase}
import com.nvidia.spark.rapids.tool.profiling.{DriverAccumCase, JobInfoClass, ProfileUtils, TaskStageAccumCase}

import org.apache.spark.internal.Logging
import org.apache.spark.scheduler._
Expand Down Expand Up @@ -280,15 +281,77 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi

def doSparkListenerJobStart(
app: T,
event: SparkListenerJobStart): Unit = {}
event: SparkListenerJobStart): Unit = {
logDebug("Processing event: " + event.getClass)
val sqlIDString = event.properties.getProperty("spark.sql.execution.id")
val sqlID = ProfileUtils.stringToLong(sqlIDString)
// add jobInfoClass
val thisJob = new JobInfoClass(
event.jobId,
event.stageIds,
sqlID,
event.properties.asScala,
event.time,
None,
None,
None,
None,
ProfileUtils.isPluginEnabled(event.properties.asScala) || app.gpuMode
)
app.jobIdToInfo.put(event.jobId, thisJob)
}

override def onJobStart(jobStart: SparkListenerJobStart): Unit = {
doSparkListenerJobStart(app, jobStart)
}

def doSparkListenerJobEnd(
app: T,
event: SparkListenerJobEnd): Unit = {}
event: SparkListenerJobEnd): Unit = {
logDebug("Processing event: " + event.getClass)

def jobResult(res: JobResult): String = {
res match {
case JobSucceeded => "JobSucceeded"
case _: JobFailed => "JobFailed"
case _ => "Unknown"
}
}

def failedReason(res: JobResult): String = {
res match {
case JobSucceeded => ""
case jobFailed: JobFailed => jobFailed.exception.toString
case _ => ""
}
}

app.jobIdToInfo.get(event.jobId) match {
case Some(j) =>
j.endTime = Some(event.time)
j.duration = ProfileUtils.OptionLongMinusLong(j.endTime, j.startTime)
val thisJobResult = jobResult(event.jobResult)
j.jobResult = Some(thisJobResult)
val thisFailedReason = failedReason(event.jobResult)
j.failedReason = Some(thisFailedReason)
case None =>
val thisJobResult = jobResult(event.jobResult)
val thisFailedReason = failedReason(event.jobResult)
val thisJob = new JobInfoClass(
event.jobId,
Seq.empty,
None,
Map.empty,
event.time, // put end time as start time
Some(event.time),
Some(thisJobResult),
Some(thisFailedReason),
None,
app.gpuMode
)
app.jobIdToInfo.put(event.jobId, thisJob)
}
}

override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = {
doSparkListenerJobEnd(app, jobEnd)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -195,8 +195,6 @@ class ApplicationInfo(
// resourceprofile id to resource profile info
val resourceProfIdToInfo = new HashMap[Int, ResourceProfileInfoCase]()

// jobId to job info
val jobIdToInfo = new HashMap[Int, JobInfoClass]()
// sqlId to sql info
val sqlIdToInfo = new HashMap[Long, SQLExecutionInfoClass]()

Expand All @@ -206,7 +204,6 @@ class ApplicationInfo(
// From SparkListenerEnvironmentUpdate
var sparkProperties = Map.empty[String, String]
var classpathEntries = Map.empty[String, String]
var gpuMode = false

var appInfo: ApplicationCase = null
var appId: String = ""
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ package org.apache.spark.sql.rapids.tool.profiling

import java.util.concurrent.TimeUnit.NANOSECONDS

import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
import scala.util.control.NonFatal

Expand Down Expand Up @@ -269,75 +268,6 @@ class EventsProcessor(app: ApplicationInfo) extends EventProcessorBase[Applicati
}
}

override def doSparkListenerJobStart(
app: ApplicationInfo,
event: SparkListenerJobStart): Unit = {
logDebug("Processing event: " + event.getClass)
val sqlIDString = event.properties.getProperty("spark.sql.execution.id")
val sqlID = ProfileUtils.stringToLong(sqlIDString)
val thisJob = new JobInfoClass(
event.jobId,
event.stageIds,
sqlID,
event.properties.asScala,
event.time,
None,
None,
None,
None,
ProfileUtils.isPluginEnabled(event.properties.asScala) || app.gpuMode
)
app.jobIdToInfo.put(event.jobId, thisJob)
}

override def doSparkListenerJobEnd(
app: ApplicationInfo,
event: SparkListenerJobEnd): Unit = {
logDebug("Processing event: " + event.getClass)

def jobResult(res: JobResult): String = {
res match {
case JobSucceeded => "JobSucceeded"
case _: JobFailed => "JobFailed"
case _ => "Unknown"
}
}

def failedReason(res: JobResult): String = {
res match {
case JobSucceeded => ""
case jobFailed: JobFailed => jobFailed.exception.toString
case _ => ""
}
}

app.jobIdToInfo.get(event.jobId) match {
case Some(j) =>
j.endTime = Some(event.time)
j.duration = ProfileUtils.OptionLongMinusLong(j.endTime, j.startTime)
val thisJobResult = jobResult(event.jobResult)
j.jobResult = Some(thisJobResult)
val thisFailedReason = failedReason(event.jobResult)
j.failedReason = Some(thisFailedReason)
case None =>
val thisJobResult = jobResult(event.jobResult)
val thisFailedReason = failedReason(event.jobResult)
val thisJob = new JobInfoClass(
event.jobId,
Seq.empty,
None,
Map.empty,
event.time, // put end time as start time
Some(event.time),
Some(thisJobResult),
Some(thisFailedReason),
None,
app.gpuMode
)
app.jobIdToInfo.put(event.jobId, thisJob)
}
}

override def doSparkListenerStageCompleted(
app: ApplicationInfo,
event: SparkListenerStageCompleted): Unit = {
Expand Down
Loading