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

Profiling tool: Add support for driver log as input to generate unsupported operators report #654

Merged
merged 8 commits into from
Nov 15, 2023
Merged
Show file tree
Hide file tree
Changes from 4 commits
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 @@ -168,6 +168,20 @@ extends ProfileResult {
}
}

case class DriverLogUnsupportedOperators(
operatorName: String, count: Int, reason: String) extends ProfileResult {
override val outputHeaders = Seq("operatorName", "count", "reason")

override def convertToSeq: Seq[String] = {
Seq(operatorName, count.toString, reason)
}

override def convertToCSVSeq: Seq[String] = {
Seq(StringUtils.reformatCSVString(operatorName), count.toString,
StringUtils.reformatCSVString(reason))
}
}

class StageInfoClass(val info: StageInfo) {
var completionTime: Option[Long] = None
var failureReason: Option[String] = None
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* Copyright (c) 2023, 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.io.Source

import org.apache.spark.internal.Logging

class DriverLogProcessor(driverlogPath: String) extends Logging {
def processDriverLog(): Seq[DriverLogUnsupportedOperators] = {
val source = Source.fromFile(driverlogPath)
// Create a map to store the counts for each operator and reason
var countsMap = Map[(String, String), Int]().withDefaultValue(0)
try {
// Process each line in the file
for (line <- source.getLines()) {
// condition to check if the line contains unsupported operators
if (line.contains("cannot run on GPU") &&
!line.contains("not all expressions can be replaced")) {
val operatorName = line.split("<")(1).split(">")(0)
val reason = line.split("because")(1).trim()
val key = (operatorName, reason)
countsMap += key -> (countsMap(key) + 1)
}
}
} catch {
case e: Exception =>
logError(s"Unexpected exception processing driver log: $driverlogPath", e)
} finally {
source.close()
}
countsMap.map(x => DriverLogUnsupportedOperators(x._1._1, x._2, x._1._2)).toSeq
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ Profiling Tool for the RAPIDS Accelerator and Apache Spark

Usage: java -cp rapids-4-spark-tools_2.12-<version>.jar:$SPARK_HOME/jars/*
com.nvidia.spark.rapids.tool.profiling.ProfileMain [options]
<eventlogs | eventlog directories ...>
[eventlogs | eventlog directories ...]
""")

val outputDirectory: ScallopOption[String] =
Expand All @@ -38,8 +38,11 @@ Usage: java -cp rapids-4-spark-tools_2.12-<version>.jar:$SPARK_HOME/jars/*
" rapids_4_spark_profile. It will overwrite any existing files" +
" with the same name.",
default = Some("."))
val driverlog: ScallopOption[String] =
opt[String](required = false,
descr = "Driver log filename - eg: /path/to/driverlog. Default is empty.")
val eventlog: ScallopOption[List[String]] =
trailArg[List[String]](required = true,
trailArg[List[String]](required = false,
tgravescs marked this conversation as resolved.
Show resolved Hide resolved
descr = "Event log filenames(space separated) or directories containing event logs." +
" eg: s3a://<BUCKET>/eventlog1 /path/to/eventlog2")
val filterCriteria: ScallopOption[String] =
Expand Down Expand Up @@ -143,6 +146,11 @@ Usage: java -cp rapids-4-spark-tools_2.12-<version>.jar:$SPARK_HOME/jars/*
Right(Unit)
}

// verify that either driverlog or eventlog is specified
validateOpt(driverlog, eventlog) {
tgravescs marked this conversation as resolved.
Show resolved Hide resolved
case (None, None) => Left("Error, one of driverlog or eventlog must be specified")
case _ => Right(Unit)
}
verify()

override def onError(e: Throwable) = e match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,8 @@ object ProfileMain extends Logging {
def mainInternal(appArgs: ProfileArgs, enablePB: Boolean = false): (Int, Int) = {

// Parsing args
val eventlogPaths = appArgs.eventlog()
val eventlogPaths = appArgs.eventlog.getOrElse(List.empty[String])
val driverLog = appArgs.driverlog.getOrElse("")
val filterN = appArgs.filterCriteria
val matchEventLogs = appArgs.matchEventLogs
val hadoopConf = RapidsToolsConfUtil.newHadoopConf
Expand All @@ -62,13 +63,17 @@ object ProfileMain extends Logging {
eventLogFsFiltered
}

if (filteredLogs.isEmpty) {
logWarning("No event logs to process after checking paths, exiting!")
if (filteredLogs.isEmpty && driverLog.isEmpty) {
logWarning("No event logs to process after checking paths and no driver log " +
"to process, exiting!")
return (0, filteredLogs.size)
}

val profiler = new Profiler(hadoopConf, appArgs, enablePB)
profiler.profile(eventLogFsFiltered)
if (driverLog.nonEmpty){
profiler.profileDriver(driverLog)
}
(0, filteredLogs.size)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,20 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea
progressBar.foreach(_.finishAll())
}

def profileDriver(driverLogInfos: String): Unit = {
val profileOutputWriter = new ProfileOutputWriter(s"$outputDir/driver",
Profiler.DRIVER_LOG_NAME, numOutputRows, true)

try {
val driverLogProcessor = new DriverLogProcessor(driverLogInfos)
val unsupportedDrivers = driverLogProcessor.processDriverLog()
profileOutputWriter.write(s"Unsupported operators in driver log",
unsupportedDrivers)
} finally {
profileOutputWriter.close()
}
}

private def errorHandler(error: Throwable, path: EventLogInfo) = {
error match {
case oom: OutOfMemoryError =>
Expand Down Expand Up @@ -530,6 +544,7 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea
object Profiler {
// This tool's output log file name
val PROFILE_LOG_NAME = "profile"
val DRIVER_LOG_NAME = "driver"
val COMPARE_LOG_FILE_NAME_PREFIX = "rapids_4_spark_tools_compare"
val COMBINED_LOG_FILE_NAME_PREFIX = "rapids_4_spark_tools_combined"
val SUBDIR = "rapids_4_spark_profile"
Expand Down
Loading
Loading