From cf435dc87d58fe1437daa366f457df24fb561e96 Mon Sep 17 00:00:00 2001 From: Ray Ortigas Date: Wed, 15 Feb 2017 15:15:22 -0800 Subject: [PATCH 01/11] Use old Spark fetcher as a fallback. --- .../linkedin/drelephant/ElephantContext.java | 1 + .../spark/fetchers/SparkFetcher.scala | 75 ++-- .../spark/fetchers/SparkLogClient.scala | 142 ++------ .../legacydata/LegacyDataConverters.scala | 186 ++++++++++ .../legacydata/SparkApplicationData.java | 38 ++ .../legacydata/SparkEnvironmentData.java | 70 ++++ .../spark/legacydata/SparkExecutorData.java | 70 ++++ .../spark/legacydata/SparkGeneralData.java | 89 +++++ .../legacydata/SparkJobProgressData.java | 273 +++++++++++++++ .../spark/legacydata/SparkStorageData.java | 46 +++ .../spark/legacyfetchers/LegacyFetchers.scala | 33 ++ .../BestPropertiesConventionHeuristic.java | 146 ++++++++ .../EventLogLimitHeuristic.java | 61 ++++ .../ExecutorLoadHeuristic.java | 217 ++++++++++++ .../legacyheuristics/JobRuntimeHeuristic.java | 135 +++++++ .../MemoryLimitHeuristic.java | 207 +++++++++++ .../StageRuntimeHeuristic.java | 160 +++++++++ .../drelephant/util/HadoopUtils.scala | 94 +++++ .../linkedin/drelephant/util/SparkUtils.scala | 183 +++++++++- .../apache/spark/SparkMetricsAggregator.java | 89 +++++ .../deploy/history/SparkDataCollection.scala | 330 ++++++++++++++++++ .../spark/deploy/history/SparkFSFetcher.scala | 119 +++++++ .../StorageStatusTrackingListener.scala | 110 ++++++ .../spark/fetchers/SparkFetcherTest.scala | 102 ++++-- .../spark/fetchers/SparkLogClientTest.scala | 97 ++--- .../spark/fetchers/SparkRestClientTest.scala | 22 +- .../legacydata/LegacyDataConvertersTest.scala | 320 +++++++++++++++++ .../legacydata/MockSparkApplicationData.java | 92 +++++ ...BestPropertiesConventionHeuristicTest.java | 93 +++++ .../JobRuntimeHeuristicTest.java | 30 ++ .../MemoryLimitHeuristicTest.java | 138 ++++++++ .../drelephant/util/HadoopUtilsTest.scala | 131 +++++++ .../drelephant/util/SparkUtilsTest.scala | 267 ++++++++++++++ .../spark/TestSparkAggregatedMetrics.java | 95 +++++ .../history/SparkDataCollectionTest.java | 55 +++ .../deploy/history/SparkFsFetcherTest.scala | 122 +++++++ .../fetcher/FetcherConfTest5.xml | 3 +- .../fetcher/FetcherConfTest6.xml | 26 ++ .../fetcher/FetcherConfTest7.xml | 25 ++ .../fetcher/FetcherConfTest8.xml | 26 ++ test/resources/core-site.xml | 9 +- test/resources/spark_event_logs/event_log_1 | 32 ++ 42 files changed, 4298 insertions(+), 261 deletions(-) create mode 100644 app/com/linkedin/drelephant/spark/legacydata/LegacyDataConverters.scala create mode 100644 app/com/linkedin/drelephant/spark/legacydata/SparkApplicationData.java create mode 100644 app/com/linkedin/drelephant/spark/legacydata/SparkEnvironmentData.java create mode 100644 app/com/linkedin/drelephant/spark/legacydata/SparkExecutorData.java create mode 100644 app/com/linkedin/drelephant/spark/legacydata/SparkGeneralData.java create mode 100644 app/com/linkedin/drelephant/spark/legacydata/SparkJobProgressData.java create mode 100644 app/com/linkedin/drelephant/spark/legacydata/SparkStorageData.java create mode 100644 app/com/linkedin/drelephant/spark/legacyfetchers/LegacyFetchers.scala create mode 100644 app/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristic.java create mode 100644 app/com/linkedin/drelephant/spark/legacyheuristics/EventLogLimitHeuristic.java create mode 100644 app/com/linkedin/drelephant/spark/legacyheuristics/ExecutorLoadHeuristic.java create mode 100644 app/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristic.java create mode 100644 app/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristic.java create mode 100644 app/com/linkedin/drelephant/spark/legacyheuristics/StageRuntimeHeuristic.java create mode 100644 app/com/linkedin/drelephant/util/HadoopUtils.scala create mode 100644 app/org/apache/spark/SparkMetricsAggregator.java create mode 100644 app/org/apache/spark/deploy/history/SparkDataCollection.scala create mode 100644 app/org/apache/spark/deploy/history/SparkFSFetcher.scala create mode 100644 app/org/apache/spark/storage/StorageStatusTrackingListener.scala create mode 100644 test/com/linkedin/drelephant/spark/legacydata/LegacyDataConvertersTest.scala create mode 100644 test/com/linkedin/drelephant/spark/legacydata/MockSparkApplicationData.java create mode 100644 test/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristicTest.java create mode 100644 test/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristicTest.java create mode 100644 test/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristicTest.java create mode 100644 test/com/linkedin/drelephant/util/HadoopUtilsTest.scala create mode 100644 test/com/linkedin/drelephant/util/SparkUtilsTest.scala create mode 100644 test/org/apache/spark/TestSparkAggregatedMetrics.java create mode 100644 test/org/apache/spark/deploy/history/SparkDataCollectionTest.java create mode 100644 test/org/apache/spark/deploy/history/SparkFsFetcherTest.scala create mode 100644 test/resources/configurations/fetcher/FetcherConfTest6.xml create mode 100644 test/resources/configurations/fetcher/FetcherConfTest7.xml create mode 100644 test/resources/configurations/fetcher/FetcherConfTest8.xml create mode 100644 test/resources/spark_event_logs/event_log_1 diff --git a/app/com/linkedin/drelephant/ElephantContext.java b/app/com/linkedin/drelephant/ElephantContext.java index fe6ff3026..225f6b53e 100644 --- a/app/com/linkedin/drelephant/ElephantContext.java +++ b/app/com/linkedin/drelephant/ElephantContext.java @@ -47,6 +47,7 @@ import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.log4j.Logger; +import org.apache.spark.SparkMetricsAggregator; import org.w3c.dom.Document; import play.api.templates.Html; diff --git a/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala b/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala index 0cf703f23..e6e4cb24f 100644 --- a/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala +++ b/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala @@ -19,7 +19,7 @@ package com.linkedin.drelephant.spark.fetchers import scala.async.Async import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration.{Duration, SECONDS} -import scala.util.Try +import scala.util.{Try, Success, Failure} import scala.util.control.NonFatal import com.linkedin.drelephant.analysis.{AnalyticJob, ElephantFetcher} @@ -37,6 +37,7 @@ import org.apache.spark.SparkConf class SparkFetcher(fetcherConfigurationData: FetcherConfigurationData) extends ElephantFetcher[SparkApplicationData] { import SparkFetcher._ + import Async.{async, await} import ExecutionContext.Implicits.global private val logger: Logger = Logger.getLogger(classOf[SparkFetcher]) @@ -47,7 +48,7 @@ class SparkFetcher(fetcherConfigurationData: FetcherConfigurationData) private[fetchers] lazy val sparkConf: SparkConf = { val sparkConf = new SparkConf() - sparkUtils.getDefaultPropertiesFile(sparkUtils.defaultEnv) match { + sparkUtils.getDefaultPropertiesFile() match { case Some(filename) => sparkConf.setAll(sparkUtils.getPropertiesFromFile(filename)) case None => throw new IllegalStateException("can't find Spark conf; please set SPARK_HOME or SPARK_CONF_DIR") } @@ -61,32 +62,40 @@ class SparkFetcher(fetcherConfigurationData: FetcherConfigurationData) if (eventLogEnabled) Some(new SparkLogClient(hadoopConfiguration, sparkConf)) else None } + private[fetchers] lazy val backupFetcher: ElephantFetcher[SparkApplicationData] = + new LegacyFetcher(fetcherConfigurationData) + override def fetchData(analyticJob: AnalyticJob): SparkApplicationData = { + doFetchData(analyticJob) match { + case Success(data) => data + case Failure(e) => throw e + } + } + + private def doFetchData(analyticJob: AnalyticJob): Try[SparkApplicationData] = { val appId = analyticJob.getAppId logger.info(s"Fetching data for ${appId}") - try { - Await.result(doFetchData(sparkRestClient, sparkLogClient, appId), DEFAULT_TIMEOUT) - } catch { - case NonFatal(e) => + Try { + Await.result(doFetchDataUsingRestAndLogClients(analyticJob), DEFAULT_TIMEOUT) + }.recover { + case e => { + logger.warn("Exception fetching data. Will make another attempt with backup fetcher instead.", e) + Await.result(doFetchDataUsingBackupFetcher(analyticJob), DEFAULT_TIMEOUT) + } + }.transform( + data => { + logger.info(s"Succeeded fetching data for ${appId}") + Success(data) + }, + e => { logger.error(s"Failed fetching data for ${appId}", e) - throw e - } + Failure(e) + } + ) } -} - -object SparkFetcher { - import Async.{async, await} - val SPARK_EVENT_LOG_ENABLED_KEY = "spark.eventLog.enabled" - val DEFAULT_TIMEOUT = Duration(30, SECONDS) - - private def doFetchData( - sparkRestClient: SparkRestClient, - sparkLogClient: Option[SparkLogClient], - appId: String - )( - implicit ec: ExecutionContext - ): Future[SparkApplicationData] = async { + private def doFetchDataUsingRestAndLogClients(analyticJob: AnalyticJob): Future[SparkApplicationData] = async { + val appId = analyticJob.getAppId val restDerivedData = await(sparkRestClient.fetchData(appId)) val lastAttemptId = restDerivedData.applicationInfo.attempts.maxBy { _.startTime }.attemptId @@ -98,4 +107,26 @@ object SparkFetcher { SparkApplicationData(appId, restDerivedData, logDerivedData) } + + private def doFetchDataUsingBackupFetcher(analyticJob: AnalyticJob): Future[SparkApplicationData] = async { + backupFetcher.fetchData(analyticJob) + } +} + +object SparkFetcher { + import org.apache.spark.deploy.history.SparkFSFetcher + import com.linkedin.drelephant.spark.legacydata.LegacyDataConverters + + private[fetchers] class LegacyFetcher(fetcherConfigurationData: FetcherConfigurationData) + extends ElephantFetcher[SparkApplicationData] { + lazy val legacyFetcher = new SparkFSFetcher(fetcherConfigurationData) + + override def fetchData(analyticJob: AnalyticJob): SparkApplicationData = { + val legacyData = legacyFetcher.fetchData(analyticJob) + LegacyDataConverters.convert(legacyData) + } + } + + val SPARK_EVENT_LOG_ENABLED_KEY = "spark.eventLog.enabled" + val DEFAULT_TIMEOUT = Duration(60, SECONDS) } diff --git a/app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala b/app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala index 75fcbff78..5f64bb9f9 100644 --- a/app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala +++ b/app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala @@ -16,32 +16,26 @@ package com.linkedin.drelephant.spark.fetchers -import java.io.{BufferedInputStream, FileNotFoundException, InputStream} -import java.net.URI +import java.io.InputStream +import java.security.PrivilegedAction import scala.async.Async -import scala.collection.mutable.HashMap import scala.concurrent.{ExecutionContext, Future} import scala.io.Source +import com.linkedin.drelephant.security.HadoopSecurity import com.linkedin.drelephant.spark.data.SparkLogDerivedData +import com.linkedin.drelephant.util.SparkUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.log4j.Logger import org.apache.spark.SparkConf -import org.apache.spark.io.{CompressionCodec, LZ4CompressionCodec, LZFCompressionCodec, SnappyCompressionCodec} import org.apache.spark.scheduler.{SparkListenerEnvironmentUpdate, SparkListenerEvent} import org.json4s.{DefaultFormats, JsonAST} import org.json4s.jackson.JsonMethods /** - * A client for getting data from the Spark event logs, using the location configured for spark.eventLog.dir. - * - * This client uses webhdfs to access the location, even if spark.eventLog.dir is provided as an hdfs URL. - * - * The codecs used by this client use JNI, which results in some weird classloading issues (at least when testing in the console), - * so some of the client's implementation is non-lazy or synchronous when needed. + * A client for getting data from the Spark event logs. */ class SparkLogClient(hadoopConfiguration: Configuration, sparkConf: SparkConf) { import SparkLogClient._ @@ -49,39 +43,29 @@ class SparkLogClient(hadoopConfiguration: Configuration, sparkConf: SparkConf) { private val logger: Logger = Logger.getLogger(classOf[SparkLogClient]) - private[fetchers] val webhdfsEventLogUri: URI = { - val eventLogUri = sparkConf.getOption(SPARK_EVENT_LOG_DIR_KEY).map(new URI(_)) - val dfsNamenodeHttpAddress = Option(hadoopConfiguration.get(HADOOP_DFS_NAMENODE_HTTP_ADDRESS_KEY)) - (eventLogUri, dfsNamenodeHttpAddress) match { - case (Some(eventLogUri), _) if eventLogUri.getScheme == "webhdfs" => - eventLogUri - case (Some(eventLogUri), Some(dfsNamenodeHttpAddress)) if eventLogUri.getScheme == "hdfs" => - val dfsNamenodeHttpUri = new URI(null, dfsNamenodeHttpAddress, null, null, null) - new URI(s"webhdfs://${eventLogUri.getHost}:${dfsNamenodeHttpUri.getPort}${eventLogUri.getPath}") - case _ => - throw new IllegalArgumentException( - s"""|${SPARK_EVENT_LOG_DIR_KEY} must be provided as webhdfs:// or hdfs://; - |if hdfs, ${HADOOP_DFS_NAMENODE_HTTP_ADDRESS_KEY} must also be provided for port""".stripMargin.replaceAll("\n", " ") - ) - } - } + private lazy val security: HadoopSecurity = new HadoopSecurity() - private[fetchers] lazy val fs: FileSystem = FileSystem.get(webhdfsEventLogUri, hadoopConfiguration) + protected lazy val sparkUtils: SparkUtils = SparkUtils - private lazy val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", defaultValue = false) - private lazy val compressionCodec = if (shouldCompress) Some(compressionCodecFromConf(sparkConf)) else None - private lazy val compressionCodecShortName = compressionCodec.map(shortNameOfCompressionCodec) + def fetchData(appId: String, attemptId: Option[String])(implicit ec: ExecutionContext): Future[SparkLogDerivedData] = + doAsPrivilegedAction { () => doFetchData(appId, attemptId) } - def fetchData(appId: String, attemptId: Option[String])(implicit ec: ExecutionContext): Future[SparkLogDerivedData] = { - val logPath = getLogPath(webhdfsEventLogUri, appId, attemptId, compressionCodecShortName) - logger.info(s"looking for logs at ${logPath}") + protected def doAsPrivilegedAction[T](action: () => T): T = + security.doAs[T](new PrivilegedAction[T] { override def run(): T = action() }) - val codec = compressionCodecForLogPath(sparkConf, logPath) + protected def doFetchData( + appId: String, + attemptId: Option[String] + )( + implicit ec: ExecutionContext + ): Future[SparkLogDerivedData] = { + val (eventLogFileSystem, baseEventLogPath) = + sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + val (eventLogPath, eventLogCodec) = + sparkUtils.pathAndCodecforEventLog(sparkConf, eventLogFileSystem, baseEventLogPath, appId, attemptId) - // Limit scope of async. async { - resource.managed { openEventLog(sparkConf, logPath, fs) } - .acquireAndGet { in => findDerivedData(codec.map { _.compressedInputStream(in) }.getOrElse(in)) } + sparkUtils.withEventLog(eventLogFileSystem, eventLogPath, eventLogCodec)(findDerivedData(_)) } } } @@ -89,9 +73,6 @@ class SparkLogClient(hadoopConfiguration: Configuration, sparkConf: SparkConf) { object SparkLogClient { import JsonAST._ - val SPARK_EVENT_LOG_DIR_KEY = "spark.eventLog.dir" - val HADOOP_DFS_NAMENODE_HTTP_ADDRESS_KEY = "dfs.namenode.http-address" - private implicit val formats: DefaultFormats = DefaultFormats def findDerivedData(in: InputStream, eventsLimit: Option[Int] = None): SparkLogDerivedData = { @@ -123,85 +104,6 @@ object SparkLogClient { // https://github.com/apache/spark/blob/v1.4.1/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala // https://github.com/apache/spark/blob/v1.4.1/core/src/main/scala/org/apache/spark/util/Utils.scala - private val IN_PROGRESS = ".inprogress" - private val DEFAULT_COMPRESSION_CODEC = "snappy" - - private val compressionCodecClassNamesByShortName = Map( - "lz4" -> classOf[LZ4CompressionCodec].getName, - "lzf" -> classOf[LZFCompressionCodec].getName, - "snappy" -> classOf[SnappyCompressionCodec].getName - ) - - // A cache for compression codecs to avoid creating the same codec many times - private val compressionCodecMap = HashMap.empty[String, CompressionCodec] - - private def compressionCodecFromConf(conf: SparkConf): CompressionCodec = { - val codecName = conf.get("spark.io.compression.codec", DEFAULT_COMPRESSION_CODEC) - loadCompressionCodec(conf, codecName) - } - - private def loadCompressionCodec(conf: SparkConf, codecName: String): CompressionCodec = { - val codecClass = compressionCodecClassNamesByShortName.getOrElse(codecName.toLowerCase, codecName) - val classLoader = Option(Thread.currentThread().getContextClassLoader).getOrElse(getClass.getClassLoader) - val codec = try { - val ctor = Class.forName(codecClass, true, classLoader).getConstructor(classOf[SparkConf]) - Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec]) - } catch { - case e: ClassNotFoundException => None - case e: IllegalArgumentException => None - } - codec.getOrElse(throw new IllegalArgumentException(s"Codec [$codecName] is not available. ")) - } - - private def shortNameOfCompressionCodec(compressionCodec: CompressionCodec): String = { - val codecName = compressionCodec.getClass.getName - if (compressionCodecClassNamesByShortName.contains(codecName)) { - codecName - } else { - compressionCodecClassNamesByShortName - .collectFirst { case (k, v) if v == codecName => k } - .getOrElse { throw new IllegalArgumentException(s"No short name for codec $codecName.") } - } - } - - private def getLogPath( - logBaseDir: URI, - appId: String, - appAttemptId: Option[String], - compressionCodecName: Option[String] = None - ): Path = { - val base = logBaseDir.toString.stripSuffix("/") + "/" + sanitize(appId) - val codec = compressionCodecName.map("." + _).getOrElse("") - if (appAttemptId.isDefined) { - new Path(base + "_" + sanitize(appAttemptId.get) + codec) - } else { - new Path(base + codec) - } - } - - private def openEventLog(conf: SparkConf, logPath: Path, fs: FileSystem): InputStream = { - // It's not clear whether FileSystem.open() throws FileNotFoundException or just plain - // IOException when a file does not exist, so try our best to throw a proper exception. - if (!fs.exists(logPath)) { - throw new FileNotFoundException(s"File ${logPath} does not exist.") - } - - new BufferedInputStream(fs.open(logPath)) - } - - private def compressionCodecForLogPath(conf: SparkConf, logPath: Path): Option[CompressionCodec] = { - // Compression codec is encoded as an extension, e.g. app_123.lzf - // Since we sanitize the app ID to not include periods, it is safe to split on it - val logBaseName = logPath.getName.stripSuffix(IN_PROGRESS) - logBaseName.split("\\.").tail.lastOption.map { codecName => - compressionCodecMap.getOrElseUpdate(codecName, loadCompressionCodec(conf, codecName)) - } - } - - private def sanitize(str: String): String = { - str.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase - } - private def sparkEventFromJson(json: JValue): Option[SparkListenerEvent] = { val environmentUpdate = getFormattedClassName(SparkListenerEnvironmentUpdate) diff --git a/app/com/linkedin/drelephant/spark/legacydata/LegacyDataConverters.scala b/app/com/linkedin/drelephant/spark/legacydata/LegacyDataConverters.scala new file mode 100644 index 000000000..2276a00f7 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacydata/LegacyDataConverters.scala @@ -0,0 +1,186 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacydata + +import java.util.Date + +import scala.collection.JavaConverters +import scala.util.Try + +import com.linkedin.drelephant.spark.fetchers.statusapiv1._ +import org.apache.spark.JobExecutionStatus +import org.apache.spark.status.api.v1.StageStatus + +/** + * Converters for legacy SparkApplicationData to current SparkApplicationData. + * + * The converters make a best effort, providing default values for attributes the legacy data doesn't provide. + * In practice, the Dr. Elephant Spark heuristics end up using a relatively small subset of the converted data. + */ +object LegacyDataConverters { + import JavaConverters._ + + def convert(legacyData: SparkApplicationData): com.linkedin.drelephant.spark.data.SparkApplicationData = { + com.linkedin.drelephant.spark.data.SparkApplicationData( + legacyData.getAppId, + extractAppConfigurationProperties(legacyData), + extractApplicationInfo(legacyData), + extractJobDatas(legacyData), + extractStageDatas(legacyData), + extractExecutorSummaries(legacyData) + ) + } + + def extractAppConfigurationProperties(legacyData: SparkApplicationData): Map[String, String] = + legacyData.getEnvironmentData.getSparkProperties.asScala.toMap + + def extractApplicationInfo(legacyData: SparkApplicationData): ApplicationInfo = { + val generalData = legacyData.getGeneralData + new ApplicationInfo( + generalData.getApplicationId, + generalData.getApplicationName, + Seq( + new ApplicationAttemptInfo( + Some("1"), + new Date(generalData.getStartTime), + new Date(generalData.getEndTime), + generalData.getSparkUser, + completed = true + ) + ) + ) + } + + def extractJobDatas(legacyData: SparkApplicationData): Seq[JobData] = { + val jobProgressData = legacyData.getJobProgressData + + def extractJobData(jobId: Int): JobData = { + val jobInfo = jobProgressData.getJobInfo(jobId) + new JobData( + jobInfo.jobId, + jobInfo.jobId.toString, + description = None, + submissionTime = None, + completionTime = None, + jobInfo.stageIds.asScala.map { _.toInt }, + Option(jobInfo.jobGroup), + extractJobExecutionStatus(jobId), + jobInfo.numTasks, + jobInfo.numActiveTasks, + jobInfo.numCompletedTasks, + jobInfo.numSkippedTasks, + jobInfo.numFailedTasks, + jobInfo.numActiveStages, + jobInfo.completedStageIndices.size(), + jobInfo.numSkippedStages, + jobInfo.numFailedStages + ) + } + + def extractJobExecutionStatus(jobId: Int): JobExecutionStatus = { + if (jobProgressData.getCompletedJobs.contains(jobId)) { + JobExecutionStatus.SUCCEEDED + } else if (jobProgressData.getFailedJobs.contains(jobId)) { + JobExecutionStatus.FAILED + } else { + JobExecutionStatus.UNKNOWN + } + } + + val sortedJobIds = jobProgressData.getJobIds.asScala.toSeq.sorted + sortedJobIds.map { jobId => extractJobData(jobId) } + } + + def extractStageDatas(legacyData: SparkApplicationData): Seq[StageData] = { + val jobProgressData = legacyData.getJobProgressData + + def extractStageData(stageAttemptId: SparkJobProgressData.StageAttemptId): StageData = { + val stageInfo = jobProgressData.getStageInfo(stageAttemptId.stageId, stageAttemptId.attemptId) + new StageData( + extractStageStatus(stageAttemptId), + stageAttemptId.stageId, + stageAttemptId.attemptId, + stageInfo.numActiveTasks, + stageInfo.numCompleteTasks, + stageInfo.numFailedTasks, + stageInfo.executorRunTime, + stageInfo.inputBytes, + inputRecords = 0, + stageInfo.outputBytes, + outputRecords = 0, + stageInfo.shuffleReadBytes, + shuffleReadRecords = 0, + stageInfo.shuffleWriteBytes, + shuffleWriteRecords = 0, + stageInfo.memoryBytesSpilled, + stageInfo.diskBytesSpilled, + stageInfo.name, + stageInfo.description, + schedulingPool = "", + accumulatorUpdates = Seq.empty, + tasks = None, + executorSummary = None + ) + } + + def extractStageStatus(stageAttemptId: SparkJobProgressData.StageAttemptId): StageStatus = { + if (jobProgressData.getCompletedStages.contains(stageAttemptId)) { + StageStatus.COMPLETE + } else if (jobProgressData.getFailedStages.contains(stageAttemptId)) { + StageStatus.FAILED + } else { + StageStatus.PENDING + } + } + + val sortedStageAttemptIds = jobProgressData.getStageAttemptIds.asScala.toSeq.sortBy { stageAttemptId => + (stageAttemptId.stageId, stageAttemptId.attemptId) + } + sortedStageAttemptIds.map { stageAttemptId => extractStageData(stageAttemptId) } + } + + def extractExecutorSummaries(legacyData: SparkApplicationData): Seq[ExecutorSummary] = { + val executorData = legacyData.getExecutorData + + def extractExecutorSummary(executorId: String): ExecutorSummary = { + val executorInfo = executorData.getExecutorInfo(executorId) + new ExecutorSummary( + executorInfo.execId, + executorInfo.hostPort, + executorInfo.rddBlocks, + executorInfo.memUsed, + executorInfo.diskUsed, + executorInfo.activeTasks, + executorInfo.failedTasks, + executorInfo.completedTasks, + executorInfo.totalTasks, + executorInfo.duration, + executorInfo.inputBytes, + executorInfo.shuffleRead, + executorInfo.shuffleWrite, + executorInfo.maxMem, + executorLogs = Map.empty + ) + } + + val sortedExecutorIds = { + val executorIds = executorData.getExecutors.asScala.toSeq + Try(executorIds.sortBy { _.toInt }).getOrElse(executorIds.sorted) + } + sortedExecutorIds.map { executorId => extractExecutorSummary(executorId) } + } +} diff --git a/app/com/linkedin/drelephant/spark/legacydata/SparkApplicationData.java b/app/com/linkedin/drelephant/spark/legacydata/SparkApplicationData.java new file mode 100644 index 000000000..dfb5b9d3f --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacydata/SparkApplicationData.java @@ -0,0 +1,38 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacydata; + +import com.linkedin.drelephant.analysis.HadoopApplicationData; + + +/** + * This holds a collection of all SparkApplicationData + */ +public interface SparkApplicationData extends HadoopApplicationData { + + public boolean isThrottled(); + + public SparkGeneralData getGeneralData(); + + public SparkEnvironmentData getEnvironmentData(); + + public SparkExecutorData getExecutorData(); + + public SparkJobProgressData getJobProgressData(); + + public SparkStorageData getStorageData(); +} diff --git a/app/com/linkedin/drelephant/spark/legacydata/SparkEnvironmentData.java b/app/com/linkedin/drelephant/spark/legacydata/SparkEnvironmentData.java new file mode 100644 index 000000000..1afc7f1b2 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacydata/SparkEnvironmentData.java @@ -0,0 +1,70 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacydata; + +import java.util.Properties; + + +/** + * This data class holds Spark environment data (Spark properties, JVM properties and etc.) + */ +public class SparkEnvironmentData { + private final Properties _sparkProperties; + private final Properties _systemProperties; + + public SparkEnvironmentData() { + _sparkProperties = new Properties(); + _systemProperties = new Properties(); + } + + public void addSparkProperty(String key, String value) { + _sparkProperties.put(key, value); + } + + public void addSystemProperty(String key, String value) { + _systemProperties.put(key, value); + } + + public String getSparkProperty(String key) { + return _sparkProperties.getProperty(key); + } + + public String getSparkProperty(String key, String defaultValue) { + String val = getSparkProperty(key); + if (val == null) { + return defaultValue; + } + return val; + } + + public String getSystemProperty(String key) { + return _systemProperties.getProperty(key); + } + + public Properties getSparkProperties() { + return _sparkProperties; + } + + public Properties getSystemProperties() { + return _systemProperties; + } + + @Override + public String toString() { + return _sparkProperties.toString() + "\n\n\n" + _systemProperties.toString(); + } +} diff --git a/app/com/linkedin/drelephant/spark/legacydata/SparkExecutorData.java b/app/com/linkedin/drelephant/spark/legacydata/SparkExecutorData.java new file mode 100644 index 000000000..7b0fcb5c2 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacydata/SparkExecutorData.java @@ -0,0 +1,70 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacydata; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + + +/** + * This class contains Spark executor information. + */ +public class SparkExecutorData { + public static final String EXECUTOR_DRIVER_NAME = "driver"; + + public static class ExecutorInfo { + public String execId; + public String hostPort; + public int rddBlocks = 0; + public long memUsed = 0L; + public long maxMem = 0L; + public long diskUsed = 0L; + + public int activeTasks = 0; + public int completedTasks = 0; + public int failedTasks = 0; + public int totalTasks = 0; + public long duration = 0L; + public long inputBytes = 0L; + public long outputBytes = 0L; + public long shuffleRead = 0L; + public long shuffleWrite = 0L; + + public String toString() { + return "{execId: " + execId + ", hostPort:" + hostPort + " , rddBlocks: " + rddBlocks + ", memUsed: " + memUsed + + ", maxMem: " + maxMem + ", diskUsed: " + diskUsed + ", totalTasks" + totalTasks + ", tasksActive: " + + activeTasks + ", tasksComplete: " + completedTasks + ", tasksFailed: " + failedTasks + ", duration: " + + duration + ", inputBytes: " + inputBytes + ", outputBytes:" + outputBytes + ", shuffleRead: " + shuffleRead + + ", shuffleWrite: " + shuffleWrite + "}"; + } + } + + private final Map _executorInfoMap = new HashMap(); + + public void setExecutorInfo(String executorId, ExecutorInfo info) { + _executorInfoMap.put(executorId, info); + } + + public ExecutorInfo getExecutorInfo(String executorId) { + return _executorInfoMap.get(executorId); + } + + public Set getExecutors() { + return _executorInfoMap.keySet(); + } +} diff --git a/app/com/linkedin/drelephant/spark/legacydata/SparkGeneralData.java b/app/com/linkedin/drelephant/spark/legacydata/SparkGeneralData.java new file mode 100644 index 000000000..ed251446a --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacydata/SparkGeneralData.java @@ -0,0 +1,89 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacydata; + +import java.util.Set; + + +/** + * This class holds Spark application information + */ +public class SparkGeneralData { + private Set _adminAcls; + private Set _viewAcls; + private String _applicationId; + private String _applicationName; + private String _sparkUser; + private long _startTime; + private long _endTime; + + public Set getAdminAcls() { + return _adminAcls; + } + + public void setAdminAcls(Set adminAcls) { + _adminAcls = adminAcls; + } + + public Set getViewAcls() { + return _viewAcls; + } + + public void setViewAcls(Set viewAcls) { + _viewAcls = viewAcls; + } + + public String getApplicationId() { + return _applicationId; + } + + public void setApplicationId(String applicationId) { + _applicationId = applicationId; + } + + public String getApplicationName() { + return _applicationName; + } + + public void setApplicationName(String applicationName) { + _applicationName = applicationName; + } + + public String getSparkUser() { + return _sparkUser; + } + + public void setSparkUser(String sparkUser) { + _sparkUser = sparkUser; + } + + public long getStartTime() { + return _startTime; + } + + public void setStartTime(long startTime) { + _startTime = startTime; + } + + public long getEndTime() { + return _endTime; + } + + public void setEndTime(long endTime) { + _endTime = endTime; + } +} diff --git a/app/com/linkedin/drelephant/spark/legacydata/SparkJobProgressData.java b/app/com/linkedin/drelephant/spark/legacydata/SparkJobProgressData.java new file mode 100644 index 000000000..81a0f269c --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacydata/SparkJobProgressData.java @@ -0,0 +1,273 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacydata; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang.StringUtils; +import org.apache.log4j.Logger; + + +/** + * This class represents information contained in a job runtime process. + */ +public class SparkJobProgressData { + private static final Logger logger = Logger.getLogger(SparkJobProgressData.class); + private final Map _jobIdToInfo = new HashMap(); + private final Set _completedJobs = new HashSet(); + private final Set _failedJobs = new HashSet(); + + private final Map _stageIdToInfo = new HashMap(); + private final Set _completedStages = new HashSet(); + private final Set _failedStages = new HashSet(); + + public void addJobInfo(int jobId, JobInfo info) { + _jobIdToInfo.put(jobId, info); + } + + public void addCompletedJob(int jobId) { + _completedJobs.add(jobId); + } + + public void addFailedJob(int jobId) { + _failedJobs.add(jobId); + } + + public void addStageInfo(int stageId, int attemptId, StageInfo info) { + _stageIdToInfo.put(new StageAttemptId(stageId, attemptId), info); + } + + public void addCompletedStages(int stageId, int attemptId) { + _completedStages.add(new StageAttemptId(stageId, attemptId)); + } + + public void addFailedStages(int stageId, int attemptId) { + _failedStages.add(new StageAttemptId(stageId, attemptId)); + } + + public Set getJobIds() { + return _jobIdToInfo.keySet(); + } + + public Set getStageAttemptIds() { + return _stageIdToInfo.keySet(); + } + + public Set getCompletedJobs() { + return _completedJobs; + } + + public Set getFailedJobs() { + return _failedJobs; + } + + private static double getFailureRate(int numCompleted, int numFailed) { + int num = numCompleted + numFailed; + + if (num == 0) { + return 0d; + } + + return numFailed * 1.0d / num; + } + + public double getJobFailureRate() { + return getFailureRate(_completedJobs.size(), _failedJobs.size()); + } + + public double getStageFailureRate() { + return getFailureRate(_completedStages.size(), _failedStages.size()); + } + + public JobInfo getJobInfo(int jobId) { + return _jobIdToInfo.get(jobId); + } + + public StageInfo getStageInfo(int stageId, int attemptId) { + return _stageIdToInfo.get(new StageAttemptId(stageId, attemptId)); + } + + public Set getCompletedStages() { + return _completedStages; + } + + public Set getFailedStages() { + return _failedStages; + } + + /** + * Job itself does not have a name, it will use its latest stage as the name. + * + * @param jobId + * @return + */ + public String getJobDescription(int jobId) { + List stageIds = _jobIdToInfo.get(jobId).stageIds; + int id = -1; + for (int stageId : stageIds) { + id = Math.max(id, stageId); + } + if (id == -1) { + logger.error("Spark Job id [" + jobId + "] does not contain any stage."); + return null; + } + return _stageIdToInfo.get(new StageAttemptId(id, 0)).name; + } + + public List getFailedJobDescriptions() { + List result = new ArrayList(); + for (int id : _failedJobs) { + result.add(getJobDescription(id)); + } + return result; + } + + // For debug purpose + public String toString() { + StringBuilder s = new StringBuilder(); + s.append("JobInfo: ["); + + for (Map.Entry entry : _jobIdToInfo.entrySet()) { + s.append("{id:" + entry.getKey() + ", value: " + entry.getValue() + "}"); + } + + s.append("]\nStageInfo: ["); + for (Map.Entry entry : _stageIdToInfo.entrySet()) { + s.append("{id:" + entry.getKey() + ", value: " + entry.getValue() + "}"); + } + s.append("]"); + + return s.toString(); + } + + public static class StageAttemptId { + public int stageId; + public int attemptId; + + public StageAttemptId(int stageId, int attemptId) { + this.stageId = stageId; + this.attemptId = attemptId; + } + + @Override + public int hashCode() { + return new Integer(stageId).hashCode() * 31 + new Integer(attemptId).hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof StageAttemptId) { + StageAttemptId other = (StageAttemptId) obj; + return stageId == other.stageId && attemptId == other.attemptId; + } + return false; + } + + public String toString() { + return "id: " + stageId + " # attemptId: " + attemptId; + } + } + + public static class JobInfo { + public int jobId; + public String jobGroup; + public long startTime; + public long endTime; + public final List stageIds = new ArrayList(); + + /* Tasks */ + public int numTasks = 0; + public int numActiveTasks = 0; + public int numCompletedTasks = 0; + public int numSkippedTasks = 0; + public int numFailedTasks = 0; + + /* Stages */ + public int numActiveStages = 0; + // This needs to be a set instead of a simple count to prevent double-counting of rerun stages: + public final Set completedStageIndices = new HashSet(); + public int numSkippedStages = 0; + public int numFailedStages = 0; + + public void addStageId(int stageId) { + stageIds.add(stageId); + } + + public double getFailureRate() { + return SparkJobProgressData.getFailureRate(numCompletedTasks, numFailedTasks); + } + + public String toString() { + return String.format("{jobId:%s, jobGroup:%s, startTime:%s, endTime:%s, numTask:%s, numActiveTasks:%s, " + + "numCompletedTasks:%s, numSkippedTasks:%s, numFailedTasks:%s, numActiveStages:%s, " + + "completedStageIndices:%s, stages:%s, numSkippedStages:%s, numFailedStages:%s}", jobId, jobGroup, + startTime, endTime, numTasks, numActiveTasks, numCompletedTasks, numSkippedTasks, numFailedTasks, + numActiveStages, getListString(completedStageIndices), getListString(stageIds), numSkippedStages, + numFailedStages); + } + } + + public static class StageInfo { + public int numActiveTasks; + public int numCompleteTasks; + public final Set completedIndices = new HashSet(); + public int numFailedTasks; + + // Total accumulated executor runtime + public long executorRunTime; + // Total stage duration + public long duration; + + // Note, currently calculating I/O speed on stage level does not make sense + // since we do not have information about specific I/O time. + public long inputBytes = 0; + public long outputBytes = 0; + public long shuffleReadBytes = 0; + public long shuffleWriteBytes = 0; + public long memoryBytesSpilled = 0; + public long diskBytesSpilled = 0; + + public String name; + public String description; + + public double getFailureRate() { + return SparkJobProgressData.getFailureRate(numCompleteTasks, numFailedTasks); + } + + // TODO: accumulables info seem to be unnecessary, might might be useful later on + // sample code from Spark source: var accumulables = new HashMap[Long, AccumulableInfo] + + @Override + public String toString() { + return String.format("{numActiveTasks:%s, numCompleteTasks:%s, completedIndices:%s, numFailedTasks:%s," + + " executorRunTime:%s, inputBytes:%s, outputBytes:%s, shuffleReadBytes:%s, shuffleWriteBytes:%s," + + " memoryBytesSpilled:%s, diskBytesSpilled:%s, name:%s, description:%s}", + numActiveTasks, numCompleteTasks, getListString(completedIndices), numFailedTasks, executorRunTime, + inputBytes, outputBytes, shuffleReadBytes, shuffleWriteBytes, memoryBytesSpilled, diskBytesSpilled, name, + description); + } + } + + private static String getListString(Collection collection) { + return "[" + StringUtils.join(collection, ",") + "]"; + } +} diff --git a/app/com/linkedin/drelephant/spark/legacydata/SparkStorageData.java b/app/com/linkedin/drelephant/spark/legacydata/SparkStorageData.java new file mode 100644 index 000000000..0145848a3 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacydata/SparkStorageData.java @@ -0,0 +1,46 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacydata; + +import java.util.List; +import org.apache.spark.storage.RDDInfo; +import org.apache.spark.storage.StorageStatus; + + +/** + * This class holds information related to Spark storage (RDDs specifically) information. + */ +public class SparkStorageData { + private List _rddInfoList; + private List _storageStatusList; + + public List getRddInfoList() { + return _rddInfoList; + } + + public void setRddInfoList(List rddInfoList) { + _rddInfoList = rddInfoList; + } + + public List getStorageStatusList() { + return _storageStatusList; + } + + public void setStorageStatusList(List storageStatusList) { + _storageStatusList = storageStatusList; + } +} diff --git a/app/com/linkedin/drelephant/spark/legacyfetchers/LegacyFetchers.scala b/app/com/linkedin/drelephant/spark/legacyfetchers/LegacyFetchers.scala new file mode 100644 index 000000000..ac0a3754c --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacyfetchers/LegacyFetchers.scala @@ -0,0 +1,33 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacyfetchers + +import com.linkedin.drelephant.analysis.AnalyticJob +import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData +import com.linkedin.drelephant.spark.legacydata.SparkApplicationData +import org.apache.spark.deploy.history.SparkFSFetcher + + +object LegacyFetchers { + def fetchDataUsingLegacyFetcher( + fetcherConfigurationData: FetcherConfigurationData, + analyticJob: AnalyticJob + ): SparkApplicationData = { + val legacyFetcher = new SparkFSFetcher(fetcherConfigurationData) + legacyFetcher.fetchData(analyticJob) + } +} diff --git a/app/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristic.java b/app/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristic.java new file mode 100644 index 000000000..d2de28f63 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristic.java @@ -0,0 +1,146 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacyheuristics; + +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; +import com.linkedin.drelephant.spark.legacydata.SparkEnvironmentData; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.util.MemoryFormatUtils; +import com.linkedin.drelephant.util.Utils; +import java.util.Arrays; +import java.util.Map; +import org.apache.log4j.Logger; + + +/** + * This heuristic rule check some of the most commonly set spark properties and make sure the user is following + * a best convention of them. + */ +public class BestPropertiesConventionHeuristic implements Heuristic { + private static final Logger logger = Logger.getLogger(BestPropertiesConventionHeuristic.class); + + public static final String SPARK_SERIALIZER = "spark.serializer"; + public static final String SPARK_DRIVER_MEMORY = "spark.driver.memory"; + public static final String SPARK_SHUFFLE_MANAGER = "spark.shuffle.manager"; + public static final String SPARK_EXECUTOR_CORES = "spark.executor.cores"; + + // Severity parameters. + private static final String NUM_CORE_SEVERITY = "num_core_severity"; + private static final String DRIVER_MEM_SEVERITY = "driver_memory_severity_in_gb"; + + // Default value of parameters + private double[] numCoreLimit= {2d}; // Spark Executor Cores + private double[] driverMemLimits = {4d, 4d, 8d, 8d}; // Spark Driver Memory + + private HeuristicConfigurationData _heuristicConfData; + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confNumCoreLimit = Utils.getParam(paramMap.get(NUM_CORE_SEVERITY), numCoreLimit.length); + if (confNumCoreLimit != null) { + numCoreLimit = confNumCoreLimit; + } + logger.info(heuristicName + " will use " + NUM_CORE_SEVERITY + " with the following threshold settings: " + + Arrays.toString(numCoreLimit)); + + double[] confDriverMemLimits = Utils.getParam(paramMap.get(DRIVER_MEM_SEVERITY), driverMemLimits.length); + if (confDriverMemLimits != null) { + driverMemLimits = confDriverMemLimits; + } + logger.info(heuristicName + " will use " + DRIVER_MEM_SEVERITY + " with the following threshold settings: " + + Arrays.toString(driverMemLimits)); + for (int i = 0; i < driverMemLimits.length; i++) { + driverMemLimits[i] = (double) MemoryFormatUtils.stringToBytes(Double.toString(driverMemLimits[i]) + "G"); + } + } + + public BestPropertiesConventionHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + @Override + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + @Override + public HeuristicResult apply(SparkApplicationData data) { + SparkEnvironmentData env = data.getEnvironmentData(); + String sparkSerializer = env.getSparkProperty(SPARK_SERIALIZER); + String sparkDriverMemory = env.getSparkProperty(SPARK_DRIVER_MEMORY); + String sparkShuffleManager = env.getSparkProperty(SPARK_SHUFFLE_MANAGER); + String sparkExecutorCores = env.getSparkProperty(SPARK_EXECUTOR_CORES); + int coreNum = sparkExecutorCores == null ? 1 : Integer.parseInt(sparkExecutorCores); + + Severity kryoSeverity = + binarySeverity("org.apache.spark.serializer.KryoSerializer", sparkSerializer, true, Severity.MODERATE); + Severity driverMemSeverity = getDriverMemorySeverity(MemoryFormatUtils.stringToBytes(sparkDriverMemory)); + Severity sortSeverity = binarySeverity("sort", sparkShuffleManager, true, Severity.MODERATE); + Severity executorCoreSeverity = getCoreNumSeverity(coreNum); + + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), Severity.max(kryoSeverity, driverMemSeverity, sortSeverity, + executorCoreSeverity), 0); + + result.addResultDetail(SPARK_SERIALIZER, propertyToString(sparkSerializer)); + result.addResultDetail(SPARK_DRIVER_MEMORY, propertyToString(sparkDriverMemory)); + result.addResultDetail(SPARK_SHUFFLE_MANAGER, propertyToString(sparkShuffleManager)); + result.addResultDetail(SPARK_EXECUTOR_CORES, propertyToString(sparkExecutorCores)); + + return result; + } + + private Severity getCoreNumSeverity(int cores) { + if (cores > numCoreLimit[0]) { + return Severity.CRITICAL; + } else { + return Severity.NONE; + } + } + + private Severity getDriverMemorySeverity(long mem) { + return Severity.getSeverityAscending( + mem, driverMemLimits[0], driverMemLimits[1], driverMemLimits[2], driverMemLimits[3]); + } + + private static Severity binarySeverity(String expectedValue, String actualValue, boolean ignoreNull, + Severity severity) { + if (actualValue == null) { + if (ignoreNull) { + return Severity.NONE; + } else { + return severity; + } + } + + if (actualValue.equals(expectedValue)) { + return Severity.NONE; + } else { + return severity; + } + } + + private static String propertyToString(String val) { + return val == null ? "Not presented. Using default" : val; + } +} diff --git a/app/com/linkedin/drelephant/spark/legacyheuristics/EventLogLimitHeuristic.java b/app/com/linkedin/drelephant/spark/legacyheuristics/EventLogLimitHeuristic.java new file mode 100644 index 000000000..b61fdbae7 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacyheuristics/EventLogLimitHeuristic.java @@ -0,0 +1,61 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacyheuristics; + +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; + + +/** + * This is a safeguard heuristic rule that makes sure if a log size passes the limit, we do not automatically + * approve it. + */ +public class EventLogLimitHeuristic implements Heuristic { + private HeuristicConfigurationData _heuristicConfData; + + public EventLogLimitHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + } + + @Override + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + @Override + public HeuristicResult apply(SparkApplicationData data) { + Severity severity = getSeverity(data); + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, 0); + if (severity == Severity.CRITICAL) { + result.addResultDetail("Large Log File", "Spark job's event log passes the limit. No actual log data is fetched." + + " All other heuristic rules will not make sense.", null); + } + return result; + } + + private Severity getSeverity(SparkApplicationData data) { + if (data.isThrottled()) { + return Severity.CRITICAL; + } else { + return Severity.NONE; + } + } +} diff --git a/app/com/linkedin/drelephant/spark/legacyheuristics/ExecutorLoadHeuristic.java b/app/com/linkedin/drelephant/spark/legacyheuristics/ExecutorLoadHeuristic.java new file mode 100644 index 000000000..c7eb887dc --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacyheuristics/ExecutorLoadHeuristic.java @@ -0,0 +1,217 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacyheuristics; + +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.math.Statistics; +import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; +import com.linkedin.drelephant.spark.legacydata.SparkExecutorData; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.util.MemoryFormatUtils; +import com.linkedin.drelephant.util.Utils; +import java.util.Arrays; +import java.util.Map; +import java.util.Set; +import org.apache.log4j.Logger; + +import static com.linkedin.drelephant.spark.legacydata.SparkExecutorData.EXECUTOR_DRIVER_NAME; + + +/** + * This heuristic rule observes load details of executors + */ +public class ExecutorLoadHeuristic implements Heuristic { + private static final Logger logger = Logger.getLogger(ExecutorLoadHeuristic.class); + private static final long MEMORY_OBSERVATION_THRESHOLD = MemoryFormatUtils.stringToBytes("1 MB"); + + // Severity parameters. + private static final String LOOSER_METRIC_DEV_SEVERITY = "looser_metric_deviation_severity"; + private static final String METRIC_DEV_SEVERITY = "metric_deviation_severity"; + + // Default value of parameters + private double[] looserMetDevLimits = {0.8d, 1d, 1.2d, 1.4d}; // Max deviation from avg. + private double[] metDevLimits = {0.4d, 0.6d, 0.8d, 1.0d}; + + private HeuristicConfigurationData _heuristicConfData; + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confLooserMetDevLimits = Utils.getParam(paramMap.get(LOOSER_METRIC_DEV_SEVERITY), + looserMetDevLimits.length); + if (confLooserMetDevLimits != null) { + looserMetDevLimits = confLooserMetDevLimits; + } + logger.info(heuristicName + " will use " + LOOSER_METRIC_DEV_SEVERITY + " with the following threshold settings: " + + Arrays.toString(looserMetDevLimits)); + + double[] confMetDevLimits = Utils.getParam(paramMap.get(METRIC_DEV_SEVERITY), metDevLimits.length); + if (confMetDevLimits != null) { + metDevLimits = confMetDevLimits; + } + logger.info(heuristicName + " will use " + METRIC_DEV_SEVERITY + " with the following threshold settings: " + + Arrays.toString(metDevLimits)); + } + + public ExecutorLoadHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + private class ValueObserver { + private final long[] _values; + private Long _min; + private Long _max; + private Long _avg; + + public ValueObserver(int size) { + _values = new long[size]; + } + + public void set(int index, long value) { + _values[index] = value; + if (_min == null) { + _min = value; + } else { + _min = Math.min(value, _min); + } + + if (_max == null) { + _max = value; + } else { + _max = Math.max(value, _max); + } + } + + public long getMin() { + return _min == null ? 0L : _min; + } + + public long getMax() { + return _max == null ? 0L : _max; + } + + public long getAvg() { + if (_avg == null) { + if (_values == null) { + return 0L; + } + _avg = Statistics.average(_values); + } + return _avg; + } + + /** + * Max(|max-avg|, |min-avg|) / avg + * + * @return + */ + public double getDeviationFactor() { + long avg = getAvg(); + if (avg == 0) { + return 0d; + } + long diff = Math.max(getMax() - avg, avg - getMin()); + return diff * 1.0d / avg; + } + } + + @Override + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + @Override + public HeuristicResult apply(SparkApplicationData data) { + SparkExecutorData executorData = data.getExecutorData(); + Set executors = executorData.getExecutors(); + + int numNonDriverExe = executors.size(); + if (executors.contains(EXECUTOR_DRIVER_NAME)) { + numNonDriverExe -= 1; + } + ValueObserver peakMems = new ValueObserver(numNonDriverExe); + ValueObserver durations = new ValueObserver(numNonDriverExe); + ValueObserver inputBytes = new ValueObserver(numNonDriverExe); + ValueObserver outputBytes = new ValueObserver(numNonDriverExe); + + int i = 0; + for (String exeId : executors) { + if (!exeId.equals(EXECUTOR_DRIVER_NAME)) { + SparkExecutorData.ExecutorInfo info = executorData.getExecutorInfo(exeId); + + /* Ignore the memory variation and consider it as 0 if it is too small + * The deviation of memory usage in KB level is too fluctuating to track. + */ + if (info.memUsed < MEMORY_OBSERVATION_THRESHOLD) { + peakMems.set(i, 0L); + } else { + peakMems.set(i, info.memUsed); + } + + durations.set(i, info.duration); + inputBytes.set(i, info.inputBytes); + outputBytes.set(i, info.outputBytes); + i += 1; + } + } + + Severity severity = Severity.max(getLooserMetricDeviationSeverity(peakMems), getMetricDeviationSeverity(durations), + getMetricDeviationSeverity(inputBytes), getLooserMetricDeviationSeverity(outputBytes)); + + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, 0); + + result.addResultDetail("Average peak storage memory", + String.format("%s (%s~%s)", MemoryFormatUtils.bytesToString(peakMems.getAvg()), + MemoryFormatUtils.bytesToString(peakMems.getMin()), MemoryFormatUtils.bytesToString(peakMems.getMax()))); + result.addResultDetail("Average runtime", + String.format("%s (%s~%s)", Statistics.readableTimespan(durations.getAvg()), + Statistics.readableTimespan(durations.getMin()), Statistics.readableTimespan(durations.getMax()))); + result.addResultDetail("Average input size", + String.format("%s (%s~%s)", MemoryFormatUtils.bytesToString(inputBytes.getAvg()), + MemoryFormatUtils.bytesToString(inputBytes.getMin()), MemoryFormatUtils.bytesToString(inputBytes.getMax()))); + result.addResultDetail("Average output size", + String.format("%s (%s~%s)", MemoryFormatUtils.bytesToString(outputBytes.getAvg()), + MemoryFormatUtils.bytesToString(outputBytes.getMin()), MemoryFormatUtils.bytesToString(outputBytes.getMax()))); + + return result; + } + + /** + * Some metrics by nature could deviate a bit more than other metrics. This method basically allows some metrics + * to have looser severity thresholds. + * + * @param ob + * @return the corresponding severity + */ + private Severity getLooserMetricDeviationSeverity(ValueObserver ob) { + double diffFactor = ob.getDeviationFactor(); + return Severity.getSeverityAscending( + diffFactor, looserMetDevLimits[0], looserMetDevLimits[1], looserMetDevLimits[2], looserMetDevLimits[3]); + } + + private Severity getMetricDeviationSeverity(ValueObserver ob) { + double diffFactor = ob.getDeviationFactor(); + return Severity.getSeverityAscending( + diffFactor, metDevLimits[0], metDevLimits[1], metDevLimits[2], metDevLimits[3]); + } + +} diff --git a/app/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristic.java b/app/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristic.java new file mode 100644 index 000000000..8daad4d1f --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristic.java @@ -0,0 +1,135 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacyheuristics; + +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; +import com.linkedin.drelephant.spark.legacydata.SparkJobProgressData; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.util.Utils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang.StringUtils; +import org.apache.log4j.Logger; + + +/** + * This heuristic rule observes the runtime characteristics of the spark application run. + */ +public class JobRuntimeHeuristic implements Heuristic { + private static final Logger logger = Logger.getLogger(JobRuntimeHeuristic.class); + + // Severity parameters. + private static final String AVG_JOB_FAILURE_SEVERITY = "avg_job_failure_rate_severity"; + private static final String SINGLE_JOB_FAILURE_SEVERITY = "single_job_failure_rate_severity"; + + // Default value of parameters + private double[] avgJobFailureLimits = {0.1d, 0.3d, 0.5d, 0.5d}; // The avg job failure rate + private double[] jobFailureLimits = {0.0d, 0.3d, 0.5d, 0.5d}; + + private HeuristicConfigurationData _heuristicConfData; + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confAvgJobFailureLimits = Utils.getParam(paramMap.get(AVG_JOB_FAILURE_SEVERITY), + avgJobFailureLimits.length); + if (confAvgJobFailureLimits != null) { + avgJobFailureLimits = confAvgJobFailureLimits; + } + logger.info(heuristicName + " will use " + AVG_JOB_FAILURE_SEVERITY + " with the following threshold settings: " + + Arrays.toString(avgJobFailureLimits)); + + double[] confJobFailureLimits = Utils.getParam(paramMap.get(SINGLE_JOB_FAILURE_SEVERITY), + jobFailureLimits.length); + if (confJobFailureLimits != null) { + jobFailureLimits = confJobFailureLimits; + } + logger.info(heuristicName + " will use " + SINGLE_JOB_FAILURE_SEVERITY + " with the following threshold settings: " + + Arrays.toString(jobFailureLimits)); + } + + public JobRuntimeHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + @Override + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + @Override + public HeuristicResult apply(SparkApplicationData data) { + SparkJobProgressData jobProgressData = data.getJobProgressData(); + Severity endSeverity = Severity.NONE; + + Set completedJobs = jobProgressData.getCompletedJobs(); + Set failedJobs = jobProgressData.getFailedJobs(); + + // Average job failure rate + double avgJobFailureRate = jobProgressData.getJobFailureRate(); + Severity jobFailureRateSeverity = getAvgJobFailureRateSeverity(avgJobFailureRate); + endSeverity = Severity.max(endSeverity, jobFailureRateSeverity); + + // For each completed individual job + List highFailureRateJobs = new ArrayList(); + for (int jobId : completedJobs) { + SparkJobProgressData.JobInfo job = jobProgressData.getJobInfo(jobId); + double jobFailureRate = job.getFailureRate(); + Severity severity = getSingleJobFailureRateSeverity(jobFailureRate); + if (severity.getValue() > Severity.MODERATE.getValue()) { + highFailureRateJobs.add( + jobProgressData.getJobDescription(jobId) + " (task failure rate:" + String.format("%1.3f", jobFailureRate) + + ")"); + } + endSeverity = Severity.max(endSeverity, severity); + } + + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), endSeverity, 0); + + result.addResultDetail("Spark completed jobs number", String.valueOf(completedJobs.size())); + result.addResultDetail("Spark failed jobs number", String.valueOf(failedJobs.size())); + result.addResultDetail("Spark failed jobs list", getJobsAsString(jobProgressData.getFailedJobDescriptions())); + result.addResultDetail("Spark average job failure rate", String.format("%.3f", avgJobFailureRate)); + result.addResultDetail("Spark jobs with high task failure rate", getJobsAsString(highFailureRateJobs)); + + return result; + } + + private Severity getAvgJobFailureRateSeverity(double rate) { + return Severity.getSeverityAscending( + rate, avgJobFailureLimits[0], avgJobFailureLimits[1], avgJobFailureLimits[2], avgJobFailureLimits[3]); + } + + private Severity getSingleJobFailureRateSeverity(double rate) { + return Severity.getSeverityAscending( + rate, jobFailureLimits[0], jobFailureLimits[1], jobFailureLimits[2], jobFailureLimits[3]); + } + + private static String getJobsAsString(Collection names) { + return StringUtils.join(names, "\n"); + } +} diff --git a/app/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristic.java b/app/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristic.java new file mode 100644 index 000000000..66f511204 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristic.java @@ -0,0 +1,207 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacyheuristics; + +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; +import com.linkedin.drelephant.spark.legacydata.SparkEnvironmentData; +import com.linkedin.drelephant.spark.legacydata.SparkExecutorData; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.util.MemoryFormatUtils; +import com.linkedin.drelephant.util.Utils; +import java.util.Arrays; +import java.util.Map; +import org.apache.log4j.Logger; +import org.apache.spark.SparkConf; + +import static com.linkedin.drelephant.spark.legacydata.SparkExecutorData.EXECUTOR_DRIVER_NAME; + + +/** + * This heuristic checks for memory consumption. + */ +public class MemoryLimitHeuristic implements Heuristic { + private static final Logger logger = Logger.getLogger(MemoryLimitHeuristic.class); + + public static final String SPARK_EXECUTOR_MEMORY = "spark.executor.memory"; + public static final String SPARK_DRIVER_MEMORY = "spark.driver.memory"; + public static final String SPARK_EXECUTOR_INSTANCES = "spark.executor.instances"; + + public static final String SPARK_STORAGE_MEMORY_FRACTION = "spark.storage.memoryFraction"; + public static final double DEFAULT_SPARK_STORAGE_MEMORY_FRACTION = 0.6d; + + // Severity parameters. + private static final String MEM_UTILIZATION_SEVERITY = "mem_util_severity"; + private static final String TOTAL_MEM_SEVERITY = "total_mem_severity_in_tb"; + + // Default value of parameters + private double[] memUtilLimits = {0.8d, 0.6d, 0.4d, 0.2d}; + private double[] totalMemLimits = {0.5d, 1d, 1.5d, 2d}; // Peak Memory / Total Storage Memory + + private HeuristicConfigurationData _heuristicConfData; + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + if(paramMap.get(MEM_UTILIZATION_SEVERITY) != null) { + double[] confMemUtilLimits = Utils.getParam(paramMap.get(MEM_UTILIZATION_SEVERITY), memUtilLimits.length); + if (confMemUtilLimits != null) { + memUtilLimits = confMemUtilLimits; + } + } + logger.info(heuristicName + " will use " + MEM_UTILIZATION_SEVERITY + " with the following threshold settings: " + + Arrays.toString(memUtilLimits)); + + if(paramMap.get(TOTAL_MEM_SEVERITY) != null) { + double[] confTotalMemLimits = Utils.getParam(paramMap.get(TOTAL_MEM_SEVERITY), totalMemLimits.length); + if (confTotalMemLimits != null) { + totalMemLimits = confTotalMemLimits; + } + } + logger.info(heuristicName + " will use " + TOTAL_MEM_SEVERITY + " with the following threshold settings: " + + Arrays.toString(totalMemLimits)); + for (int i = 0; i < totalMemLimits.length; i++) { + totalMemLimits[i] = MemoryFormatUtils.stringToBytes(totalMemLimits[i] + "T"); + } + } + + public MemoryLimitHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + @Override + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + @Override + public HeuristicResult apply(SparkApplicationData data) { + + int executorNum = Integer.parseInt(data.getEnvironmentData().getSparkProperty(SPARK_EXECUTOR_INSTANCES, "0")); + long perExecutorMem = + MemoryFormatUtils.stringToBytes(data.getEnvironmentData().getSparkProperty(SPARK_EXECUTOR_MEMORY, "0")); + + long totalExecutorMem = executorNum * perExecutorMem; + + long totalStorageMem = getTotalStorageMem(data); + long totalDriverMem = getTotalDriverMem(data); + long peakMem = getStoragePeakMemory(data); + + Severity totalMemorySeverity = getTotalMemorySeverity(totalExecutorMem); + Severity memoryUtilizationServerity = getMemoryUtilizationSeverity(peakMem, totalStorageMem); + + HeuristicResult result = + new HeuristicResult(_heuristicConfData.getClassName(), _heuristicConfData.getHeuristicName(), + Severity.max(totalMemorySeverity, memoryUtilizationServerity), 0); + + result.addResultDetail("Total executor memory allocated", String + .format("%s (%s x %s)", MemoryFormatUtils.bytesToString(totalExecutorMem), + MemoryFormatUtils.bytesToString(perExecutorMem), executorNum)); + result.addResultDetail("Total driver memory allocated", MemoryFormatUtils.bytesToString(totalDriverMem)); + result.addResultDetail("Total memory allocated for storage", MemoryFormatUtils.bytesToString(totalStorageMem)); + result.addResultDetail("Total memory used at peak", MemoryFormatUtils.bytesToString(peakMem)); + result.addResultDetail("Memory utilization rate", String.format("%1.3f", peakMem * 1.0 / totalStorageMem)); + return result; + } + + /** + * Get the total driver memory + * + * @param data The spark application data that contains the information + * @return The memory in bytes + */ + private static long getTotalDriverMem(SparkApplicationData data) { + long bytes = MemoryFormatUtils.stringToBytes(data.getEnvironmentData().getSparkProperty(SPARK_DRIVER_MEMORY)); + // spark.driver.memory might not be present, in which case we would infer it from the executor data + if (bytes == 0L) { + SparkExecutorData.ExecutorInfo info = data.getExecutorData().getExecutorInfo(EXECUTOR_DRIVER_NAME); + if (info == null) { + logger.error("Application id [" + data.getGeneralData().getApplicationId() + + "] does not contain driver memory configuration info and also does not contain executor driver info." + + " Unable to detect is driver memory usage."); + return 0L; + } + // This maxmium memory only counts in memory for storage + bytes = (long) (info.maxMem / getStorageMemoryFraction(data.getEnvironmentData())); + } + + return bytes; + } + + /** + * Get the storage memory fraction ratio used for storage + * + * @param data The spark environment data + * @return the memory fraction + */ + private static double getStorageMemoryFraction(SparkEnvironmentData data) { + String ratio = data.getSparkProperty(SPARK_STORAGE_MEMORY_FRACTION); + if (ratio == null) { + ratio = new SparkConf().get(SPARK_STORAGE_MEMORY_FRACTION, String.valueOf(DEFAULT_SPARK_STORAGE_MEMORY_FRACTION)); + } + return Double.parseDouble(ratio); + } + + /** + * Get the peak storage memory used during all running time of the spark application + * + * @param data The spark application data that contains the information + * @return The memory in bytes + */ + private static long getStoragePeakMemory(SparkApplicationData data) { + SparkExecutorData executorData = data.getExecutorData(); + long mem = 0L; + for (String id : executorData.getExecutors()) { + mem += executorData.getExecutorInfo(id).memUsed; + } + return mem; + } + + /** + * Get the total memory allocated for storage + * + * @param data The spark application data that contains the information + * @return The memory in bytes + */ + private static long getTotalStorageMem(SparkApplicationData data) { + SparkExecutorData executorData = data.getExecutorData(); + long totalStorageMem = 0L; + for (String id : executorData.getExecutors()) { + totalStorageMem += executorData.getExecutorInfo(id).maxMem; + } + return totalStorageMem; + } + + public Severity getTotalMemorySeverity(long memory) { + return Severity.getSeverityAscending(memory, totalMemLimits[0], totalMemLimits[1], totalMemLimits[2], + totalMemLimits[3]); + } + + private Severity getMemoryUtilizationSeverity(long peakMemory, long totalStorageMemory) { + double fraction = peakMemory * 1.0 / totalStorageMemory; + if (totalStorageMemory < MemoryFormatUtils.stringToBytes("500 GB")) { + return Severity.NONE; + } else { + return Severity.getSeverityDescending( + fraction, memUtilLimits[0], memUtilLimits[1], memUtilLimits[2], memUtilLimits[3]); + } + } +} diff --git a/app/com/linkedin/drelephant/spark/legacyheuristics/StageRuntimeHeuristic.java b/app/com/linkedin/drelephant/spark/legacyheuristics/StageRuntimeHeuristic.java new file mode 100644 index 000000000..d564fd4c5 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/legacyheuristics/StageRuntimeHeuristic.java @@ -0,0 +1,160 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacyheuristics; + +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.math.Statistics; +import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; +import com.linkedin.drelephant.spark.legacydata.SparkJobProgressData; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.util.Utils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang.StringUtils; +import org.apache.log4j.Logger; + + +/** + * Spark heuristic that detects stage runtime anomalies. + * + */ +public class StageRuntimeHeuristic implements Heuristic { + private static final Logger logger = Logger.getLogger(StageRuntimeHeuristic.class); + + // Severity parameters + private static final String STAGE_FAILURE_SEVERITY = "stage_failure_rate_severity"; + private static final String SINGLE_STAGE_FAILURE_SEVERITY = "single_stage_tasks_failure_rate_severity"; + private static final String STAGE_RUNTIME_SEVERITY = "stage_runtime_severity_in_min"; + + // Default value of parameters + private double[] stageFailRateLimits = {0.3d, 0.3d, 0.5d, 0.5d}; + private double[] singleStageFailLimits = {0.0d, 0.3d, 0.5d, 0.5d}; + private double[] stageRuntimeLimits = {15, 30, 60, 60}; + + private HeuristicConfigurationData _heuristicConfData; + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confStageFailRateLimits = Utils.getParam(paramMap.get(STAGE_FAILURE_SEVERITY), + stageFailRateLimits.length); + if (confStageFailRateLimits != null) { + stageFailRateLimits = confStageFailRateLimits; + } + logger.info(heuristicName + " will use " + STAGE_FAILURE_SEVERITY + " with the following threshold settings: " + + Arrays.toString(stageFailRateLimits)); + + double[] confSingleFailLimits = Utils.getParam(paramMap.get(SINGLE_STAGE_FAILURE_SEVERITY), + singleStageFailLimits.length); + if (confSingleFailLimits != null) { + singleStageFailLimits = confSingleFailLimits; + } + logger.info(heuristicName + " will use " + SINGLE_STAGE_FAILURE_SEVERITY + " with the following threshold" + + " settings: " + Arrays.toString(singleStageFailLimits)); + + double[] confStageRuntimeLimits = Utils.getParam(paramMap.get(STAGE_RUNTIME_SEVERITY), stageRuntimeLimits.length); + if (confStageRuntimeLimits != null) { + stageRuntimeLimits = confStageRuntimeLimits; + } + logger.info(heuristicName + " will use " + STAGE_RUNTIME_SEVERITY + " with the following threshold settings: " + + Arrays.toString(stageRuntimeLimits)); + for (int i = 0; i < stageRuntimeLimits.length; i++) { + stageRuntimeLimits[i] = stageRuntimeLimits[i] * Statistics.MINUTE_IN_MS; + } + } + + public StageRuntimeHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + @Override + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + @Override + public HeuristicResult apply(SparkApplicationData data) { + SparkJobProgressData jobProgressData = data.getJobProgressData(); + Severity endSeverity = Severity.NONE; + + Set completedStages = jobProgressData.getCompletedStages(); + Set failedStages = jobProgressData.getFailedStages(); + + // Average stage failure rate + double avgStageFailureRate = jobProgressData.getStageFailureRate(); + endSeverity = Severity.max(endSeverity, getStageFailureRateSeverity(avgStageFailureRate)); + + // For each completed stage, the task failure rate + List problematicStages = new ArrayList(); + + for (SparkJobProgressData.StageAttemptId id : completedStages) { + SparkJobProgressData.StageInfo info = jobProgressData.getStageInfo(id.stageId, id.attemptId); + double stageTasksFailureRate = info.getFailureRate(); + Severity tasksFailureRateSeverity = getSingleStageTasksFailureRate(stageTasksFailureRate); + + if (tasksFailureRateSeverity.getValue() > Severity.MODERATE.getValue()) { + problematicStages.add(String.format("%s (task failure rate: %1.3f)", id, stageTasksFailureRate)); + } + + long duration = info.duration; + Severity runtimeSeverity = getStageRuntimeSeverity(duration); + if (runtimeSeverity.getValue() > Severity.MODERATE.getValue()) { + problematicStages + .add(String.format("%s (runtime: %s)", id, Statistics.readableTimespan(duration))); + } + + endSeverity = Severity.max(endSeverity, tasksFailureRateSeverity, runtimeSeverity); + } + + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), endSeverity, 0); + + result.addResultDetail("Spark stage completed", String.valueOf(completedStages.size())); + result.addResultDetail("Spark stage failed", String.valueOf(failedStages.size())); + result.addResultDetail("Spark average stage failure rate", String.format("%.3f", avgStageFailureRate)); + result.addResultDetail("Spark problematic stages", getStagesAsString(problematicStages)); + + return result; + } + + private Severity getStageRuntimeSeverity(long runtime) { + return Severity.getSeverityAscending( + runtime, stageRuntimeLimits[0], stageRuntimeLimits[1], stageRuntimeLimits[2], stageRuntimeLimits[3]); + } + + private Severity getStageFailureRateSeverity(double rate) { + return Severity.getSeverityAscending( + rate, stageFailRateLimits[0], stageFailRateLimits[1], stageFailRateLimits[2], stageFailRateLimits[3]); + } + + private Severity getSingleStageTasksFailureRate(double rate) { + return Severity.getSeverityAscending( + rate, singleStageFailLimits[0], singleStageFailLimits[1], singleStageFailLimits[2], singleStageFailLimits[3]); + } + + private static String getStagesAsString(Collection names) { + return StringUtils.join(names, "\n"); + } +} diff --git a/app/com/linkedin/drelephant/util/HadoopUtils.scala b/app/com/linkedin/drelephant/util/HadoopUtils.scala new file mode 100644 index 000000000..8f37b4a32 --- /dev/null +++ b/app/com/linkedin/drelephant/util/HadoopUtils.scala @@ -0,0 +1,94 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.util + +import java.io.InputStream +import java.net.{HttpURLConnection, URL} + +import com.fasterxml.jackson.databind.{JsonNode, ObjectMapper} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.authentication.client.AuthenticatedURL +import org.apache.log4j.Logger + +trait HadoopUtils { + val DFS_NAMESERVICES_KEY = "dfs.nameservices" + val DFS_HA_NAMENODES_KEY = "dfs.ha.namenodes" + val DFS_NAMENODE_HTTP_ADDRESS_KEY = "dfs.namenode.http-address" + + protected def logger: Logger + + def findHaNameNodeAddress(conf: Configuration): Option[String] = { + + def findNameNodeAddressInNameServices(nameServices: Array[String]): Option[String] = nameServices match { + case Array(nameService) => { + val ids = Option(conf.get(s"${DFS_HA_NAMENODES_KEY}.${nameService}")).map { _.split(",") } + val namenodeAddress = ids.flatMap { findNameNodeAddressInNameService(nameService, _) } + namenodeAddress match { + case Some(address) => logger.info(s"Active namenode for ${nameService}: ${address}") + case None => logger.info(s"No active namenode for ${nameService}.") + } + namenodeAddress + } + case Array() => { + logger.info("No name services found.") + None + } + case _ => { + logger.info("Multiple name services found. HDFS federation is not supported right now.") + None + } + } + + def findNameNodeAddressInNameService(nameService: String, nameNodeIds: Array[String]): Option[String] = + nameNodeIds + .flatMap { id => Option(conf.get(s"${DFS_NAMENODE_HTTP_ADDRESS_KEY}.${nameService}.${id}")) } + .find(isActiveNameNode) + + val nameServices = Option(conf.get(DFS_NAMESERVICES_KEY)).map { _.split(",") } + nameServices.flatMap(findNameNodeAddressInNameServices) + } + + def httpNameNodeAddress(conf: Configuration): Option[String] = Option(conf.get(DFS_NAMENODE_HTTP_ADDRESS_KEY)) + + def isActiveNameNode(hostAndPort: String): Boolean = { + val url = new URL(s"http://${hostAndPort}/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus") + val conn = newAuthenticatedConnection(url) + try { + val in = conn.getInputStream() + try { + isActiveNameNode(in) + } finally { + in.close() + } + } finally { + conn.disconnect() + } + } + + protected def isActiveNameNode(in: InputStream): Boolean = + new ObjectMapper().readTree(in).path("beans").get(0).path("State").textValue() == "active" + + protected def newAuthenticatedConnection(url: URL): HttpURLConnection = { + val token = new AuthenticatedURL.Token() + val authenticatedURL = new AuthenticatedURL() + authenticatedURL.openConnection(url, token) + } +} + +object HadoopUtils extends HadoopUtils { + override protected lazy val logger = Logger.getLogger(classOf[HadoopUtils]) +} diff --git a/app/com/linkedin/drelephant/util/SparkUtils.scala b/app/com/linkedin/drelephant/util/SparkUtils.scala index 3a0354070..64cae4285 100644 --- a/app/com/linkedin/drelephant/util/SparkUtils.scala +++ b/app/com/linkedin/drelephant/util/SparkUtils.scala @@ -16,16 +16,112 @@ package com.linkedin.drelephant.util -import java.io.{File, FileInputStream, InputStreamReader} +import java.io.{BufferedInputStream, File, FileInputStream, FileNotFoundException, InputStream, InputStreamReader} +import java.net.URI import java.util.Properties import scala.collection.JavaConverters +import scala.collection.mutable.HashMap + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.log4j.Logger +import org.apache.spark.SparkConf +import org.apache.spark.io.{CompressionCodec, LZ4CompressionCodec, LZFCompressionCodec, SnappyCompressionCodec} trait SparkUtils { import JavaConverters._ - def defaultEnv: Map[String, String] + protected def logger: Logger + + protected def hadoopUtils: HadoopUtils + + protected def defaultEnv: Map[String, String] + + val SPARK_EVENT_LOG_DIR_KEY = "spark.eventLog.dir" + val SPARK_EVENT_LOG_COMPRESS_KEY = "spark.eventLog.compress" + val DFS_HTTP_PORT = 50070 + + /** + * Returns the webhdfs FileSystem and Path for the configured Spark event log directory and optionally the + * configured Hadoop namenode. + * + * Primarily the FileSystem and Path are based on spark.eventLog.dir but if spark.eventLog.dir is a simple path, + * then it is combined with the namenode info from the Hadoop configuration. + * + * @param hadoopConfiguration a Hadoop configuration containing namenode info + * @param sparkConf a Spark configuration with the Spark event log directory setting + * @return a tuple (FileSystem, Path) for the configured Spark event log directory + */ + def fileSystemAndPathForEventLogDir(hadoopConfiguration: Configuration, sparkConf: SparkConf): (FileSystem, Path) = { + val eventLogUri = sparkConf.getOption(SPARK_EVENT_LOG_DIR_KEY).map(new URI(_)) + eventLogUri match { + case Some(uri) if uri.getScheme == "webhdfs" => + (FileSystem.get(uri, hadoopConfiguration), new Path(uri.getPath)) + case Some(uri) if uri.getScheme == "hdfs" => + (FileSystem.get(new URI(s"webhdfs://${uri.getHost}:${DFS_HTTP_PORT}${uri.getPath}"), hadoopConfiguration), new Path(uri.getPath)) + case Some(uri) => + val nameNodeAddress + = hadoopUtils.findHaNameNodeAddress(hadoopConfiguration) + .orElse(hadoopUtils.httpNameNodeAddress(hadoopConfiguration)) + nameNodeAddress match { + case Some(address) => + (FileSystem.get(new URI(s"webhdfs://${address}${uri.getPath}"), hadoopConfiguration), new Path(uri.getPath)) + case None => + throw new IllegalArgumentException("Couldn't find configured namenode") + } + case None => + throw new IllegalArgumentException("${SPARK_EVENT_LOG_DIR_KEY} not provided") + } + } + + /** + * Returns the path and codec for the event log for the given app and attempt. + * + * This invokes JNI to get the codec, so it must be done synchronously, otherwise weird classloading issues will + * manifest (at least they manifest during testing). + * + * The path and codec can then be passed to withEventLog, which can be called asynchronously. + * + * @param sparkConf the Spark configuration with the setting for whether Spark event logs are compressed + * @param fs the filesystem which contains the logs + * @param basePath the base path for logs on the given filesystem + * @param appId the app identifier to use for the specific log file + * @param attemptId the attempt identifier to use for the specific log file + * @return a tuple (Path, Option[CompressionCodec]) for the specific event log file and the codec to use + */ + def pathAndCodecforEventLog( + sparkConf: SparkConf, + fs: FileSystem, + basePath: Path, + appId: String, + attemptId: Option[String] + ): (Path, Option[CompressionCodec]) = { + val path = { + val shouldUseCompression = sparkConf.getBoolean(SPARK_EVENT_LOG_COMPRESS_KEY, defaultValue = false) + val compressionCodecShortName = + if (shouldUseCompression) Some(shortNameOfCompressionCodec(compressionCodecFromConf(sparkConf))) else None + getLogPath(fs.getUri.resolve(basePath.toUri), appId, attemptId, compressionCodecShortName) + } + val codec = compressionCodecForLogPath(sparkConf, path) + (path, codec) + } + + /** + * A loan method that performs the given function on the loaned event log inputstream, and closes it after use. + * + * The method arguments should have been attained from fileSystemAndPathForEventLogDir and pathAndCodecforEventLog. + * + * @param fs the filesystem which contains the log + * @param path the full path to the log + * @param codec the codec to use for the log + */ + def withEventLog[T](fs: FileSystem, path: Path, codec: Option[CompressionCodec])(f: InputStream => T): T = { + resource.managed { openEventLog(path, fs) } + .map { in => codec.map { _.compressedInputStream(in) }.getOrElse(in) } + .acquireAndGet(f) + } // Below this line are modified utility methods from // https://github.com/apache/spark/blob/v1.4.1/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -55,8 +151,89 @@ trait SparkUtils { inReader.close() } } + + private val IN_PROGRESS = ".inprogress" + private val DEFAULT_COMPRESSION_CODEC = "snappy" + + private val compressionCodecClassNamesByShortName = Map( + "lz4" -> classOf[LZ4CompressionCodec].getName, + "lzf" -> classOf[LZFCompressionCodec].getName, + "snappy" -> classOf[SnappyCompressionCodec].getName + ) + + // A cache for compression codecs to avoid creating the same codec many times + private val compressionCodecMap = HashMap.empty[String, CompressionCodec] + + private def compressionCodecFromConf(conf: SparkConf): CompressionCodec = { + val codecName = conf.get("spark.io.compression.codec", DEFAULT_COMPRESSION_CODEC) + loadCompressionCodec(conf, codecName) + } + + private def loadCompressionCodec(conf: SparkConf, codecName: String): CompressionCodec = { + val codecClass = compressionCodecClassNamesByShortName.getOrElse(codecName.toLowerCase, codecName) + val classLoader = Option(Thread.currentThread().getContextClassLoader).getOrElse(getClass.getClassLoader) + val codec = try { + val ctor = Class.forName(codecClass, true, classLoader).getConstructor(classOf[SparkConf]) + Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec]) + } catch { + case e: ClassNotFoundException => None + case e: IllegalArgumentException => None + } + codec.getOrElse(throw new IllegalArgumentException(s"Codec [$codecName] is not available. ")) + } + + private def shortNameOfCompressionCodec(compressionCodec: CompressionCodec): String = { + val codecName = compressionCodec.getClass.getName + if (compressionCodecClassNamesByShortName.contains(codecName)) { + codecName + } else { + compressionCodecClassNamesByShortName + .collectFirst { case (k, v) if v == codecName => k } + .getOrElse { throw new IllegalArgumentException(s"No short name for codec $codecName.") } + } + } + + private def getLogPath( + logBaseDir: URI, + appId: String, + appAttemptId: Option[String], + compressionCodecName: Option[String] = None + ): Path = { + val base = logBaseDir.toString.stripSuffix("/") + "/" + sanitize(appId) + val codec = compressionCodecName.map("." + _).getOrElse("") + if (appAttemptId.isDefined) { + new Path(base + "_" + sanitize(appAttemptId.get) + codec) + } else { + new Path(base + codec) + } + } + + private def openEventLog(logPath: Path, fs: FileSystem): InputStream = { + // It's not clear whether FileSystem.open() throws FileNotFoundException or just plain + // IOException when a file does not exist, so try our best to throw a proper exception. + if (!fs.exists(logPath)) { + throw new FileNotFoundException(s"File ${logPath} does not exist.") + } + + new BufferedInputStream(fs.open(logPath)) + } + + private def compressionCodecForLogPath(conf: SparkConf, logPath: Path): Option[CompressionCodec] = { + // Compression codec is encoded as an extension, e.g. app_123.lzf + // Since we sanitize the app ID to not include periods, it is safe to split on it + val logBaseName = logPath.getName.stripSuffix(IN_PROGRESS) + logBaseName.split("\\.").tail.lastOption.map { codecName => + compressionCodecMap.getOrElseUpdate(codecName, loadCompressionCodec(conf, codecName)) + } + } + + private def sanitize(str: String): String = { + str.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase + } } object SparkUtils extends SparkUtils { - override val defaultEnv = sys.env + override protected lazy val logger = Logger.getLogger(classOf[SparkUtils]) + override protected lazy val hadoopUtils = HadoopUtils + override protected lazy val defaultEnv = sys.env } diff --git a/app/org/apache/spark/SparkMetricsAggregator.java b/app/org/apache/spark/SparkMetricsAggregator.java new file mode 100644 index 000000000..98783977f --- /dev/null +++ b/app/org/apache/spark/SparkMetricsAggregator.java @@ -0,0 +1,89 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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; +import com.linkedin.drelephant.math.Statistics; +import org.apache.commons.io.FileUtils; + +import com.linkedin.drelephant.analysis.HadoopApplicationData; +import com.linkedin.drelephant.analysis.HadoopMetricsAggregator; +import com.linkedin.drelephant.analysis.HadoopAggregatedData; +import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData; +import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; +import com.linkedin.drelephant.spark.legacydata.SparkExecutorData; +import com.linkedin.drelephant.util.MemoryFormatUtils; +import java.util.Iterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class SparkMetricsAggregator implements HadoopMetricsAggregator { + + private static final Logger logger = LoggerFactory.getLogger(SparkMetricsAggregator.class); + + private AggregatorConfigurationData _aggregatorConfigurationData; + private double _storageMemWastageBuffer = 0.5; + + private static final String SPARK_EXECUTOR_MEMORY = "spark.executor.memory"; + private static final String STORAGE_MEM_WASTAGE_BUFFER = "storage_mem_wastage_buffer"; + + private HadoopAggregatedData _hadoopAggregatedData = new HadoopAggregatedData(); + + + public SparkMetricsAggregator(AggregatorConfigurationData _aggregatorConfigurationData) { + this._aggregatorConfigurationData = _aggregatorConfigurationData; + String configValue = _aggregatorConfigurationData.getParamMap().get(STORAGE_MEM_WASTAGE_BUFFER); + if(configValue != null) { + _storageMemWastageBuffer = Double.parseDouble(configValue); + } + } + + @Override + public void aggregate(HadoopApplicationData data) { + long resourceUsed = 0; + long resourceWasted = 0; + SparkApplicationData applicationData = (SparkApplicationData) data; + + long perExecutorMem = + MemoryFormatUtils.stringToBytes(applicationData.getEnvironmentData().getSparkProperty(SPARK_EXECUTOR_MEMORY, "0")); + + Iterator executorIds = applicationData.getExecutorData().getExecutors().iterator(); + + while(executorIds.hasNext()) { + String executorId = executorIds.next(); + SparkExecutorData.ExecutorInfo executorInfo = applicationData.getExecutorData().getExecutorInfo(executorId); + // store the resourceUsed in MBSecs + resourceUsed += (executorInfo.duration / Statistics.SECOND_IN_MS) * (perExecutorMem / FileUtils.ONE_MB); + // maxMem is the maximum available storage memory + // memUsed is how much storage memory is used. + // any difference is wasted after a buffer of 50% is wasted + long excessMemory = (long) (executorInfo.maxMem - (executorInfo.memUsed * (1.0 + _storageMemWastageBuffer))); + if( excessMemory > 0) { + resourceWasted += (executorInfo.duration / Statistics.SECOND_IN_MS) * (excessMemory / FileUtils.ONE_MB); + } + } + + _hadoopAggregatedData.setResourceUsed(resourceUsed); + _hadoopAggregatedData.setResourceWasted(resourceWasted); + // TODO: to find a way to calculate the delay + _hadoopAggregatedData.setTotalDelay(0L); + } + + @Override + public HadoopAggregatedData getResult() { + return _hadoopAggregatedData; + } +} diff --git a/app/org/apache/spark/deploy/history/SparkDataCollection.scala b/app/org/apache/spark/deploy/history/SparkDataCollection.scala new file mode 100644 index 000000000..f60fcfa19 --- /dev/null +++ b/app/org/apache/spark/deploy/history/SparkDataCollection.scala @@ -0,0 +1,330 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.deploy.history + +import java.io.InputStream +import java.util.{Set => JSet, Properties, List => JList, HashSet => JHashSet, ArrayList => JArrayList} + +import scala.collection.mutable + +import com.linkedin.drelephant.analysis.ApplicationType +import com.linkedin.drelephant.spark.legacydata._ +import com.linkedin.drelephant.spark.legacydata.SparkExecutorData.ExecutorInfo +import com.linkedin.drelephant.spark.legacydata.SparkJobProgressData.JobInfo + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.{ApplicationEventListener, ReplayListenerBus, StageInfo} +import org.apache.spark.storage.{RDDInfo, StorageStatus, StorageStatusListener, StorageStatusTrackingListener} +import org.apache.spark.ui.env.EnvironmentListener +import org.apache.spark.ui.exec.ExecutorsListener +import org.apache.spark.ui.jobs.JobProgressListener +import org.apache.spark.ui.storage.StorageListener +import org.apache.spark.util.collection.OpenHashSet + +/** + * This class wraps the logic of collecting the data in SparkEventListeners into the + * HadoopApplicationData instances. + * + * Notice: + * This has to live in Spark's scope because ApplicationEventListener is in private[spark] scope. And it is problematic + * to compile if written in Java. + */ +class SparkDataCollection extends SparkApplicationData { + import SparkDataCollection._ + + lazy val applicationEventListener = new ApplicationEventListener() + lazy val jobProgressListener = new JobProgressListener(new SparkConf()) + lazy val environmentListener = new EnvironmentListener() + lazy val storageStatusListener = new StorageStatusListener() + lazy val executorsListener = new ExecutorsListener(storageStatusListener) + lazy val storageListener = new StorageListener(storageStatusListener) + + // This is a customized listener that tracks peak used memory + // The original listener only tracks the current in use memory which is useless in offline scenario. + lazy val storageStatusTrackingListener = new StorageStatusTrackingListener() + + private var _applicationData: SparkGeneralData = null; + private var _jobProgressData: SparkJobProgressData = null; + private var _environmentData: SparkEnvironmentData = null; + private var _executorData: SparkExecutorData = null; + private var _storageData: SparkStorageData = null; + private var _isThrottled: Boolean = false; + + def throttle(): Unit = { + _isThrottled = true + } + + override def isThrottled(): Boolean = _isThrottled + + override def getApplicationType(): ApplicationType = APPLICATION_TYPE + + override def getConf(): Properties = getEnvironmentData().getSparkProperties() + + override def isEmpty(): Boolean = !isThrottled() && getExecutorData().getExecutors.isEmpty() + + override def getGeneralData(): SparkGeneralData = { + if (_applicationData == null) { + _applicationData = new SparkGeneralData() + + applicationEventListener.adminAcls match { + case Some(s: String) => { + _applicationData.setAdminAcls(stringToSet(s)) + } + case None => { + // do nothing + } + } + + applicationEventListener.viewAcls match { + case Some(s: String) => { + _applicationData.setViewAcls(stringToSet(s)) + } + case None => { + // do nothing + } + } + + applicationEventListener.appId match { + case Some(s: String) => { + _applicationData.setApplicationId(s) + } + case None => { + // do nothing + } + } + + applicationEventListener.appName match { + case Some(s: String) => { + _applicationData.setApplicationName(s) + } + case None => { + // do nothing + } + } + + applicationEventListener.sparkUser match { + case Some(s: String) => { + _applicationData.setSparkUser(s) + } + case None => { + // do nothing + } + } + + applicationEventListener.startTime match { + case Some(s: Long) => { + _applicationData.setStartTime(s) + } + case None => { + // do nothing + } + } + + applicationEventListener.endTime match { + case Some(s: Long) => { + _applicationData.setEndTime(s) + } + case None => { + // do nothing + } + } + } + _applicationData + } + + override def getEnvironmentData(): SparkEnvironmentData = { + if (_environmentData == null) { + // Notice: we ignore jvmInformation and classpathEntries, because they are less likely to be used by any analyzer. + _environmentData = new SparkEnvironmentData() + environmentListener.systemProperties.foreach { case (name, value) => + _environmentData.addSystemProperty(name, value) + } + environmentListener.sparkProperties.foreach { case (name, value) => + _environmentData.addSparkProperty(name, value) + } + } + _environmentData + } + + override def getExecutorData(): SparkExecutorData = { + if (_executorData == null) { + _executorData = new SparkExecutorData() + + for (statusId <- 0 until executorsListener.storageStatusList.size) { + val info = new ExecutorInfo() + + val status = executorsListener.storageStatusList(statusId) + + info.execId = status.blockManagerId.executorId + info.hostPort = status.blockManagerId.hostPort + info.rddBlocks = status.numBlocks + + // Use a customized listener to fetch the peak memory used, the data contained in status are + // the current used memory that is not useful in offline settings. + info.memUsed = storageStatusTrackingListener.executorIdToMaxUsedMem.getOrElse(info.execId, 0L) + info.maxMem = status.maxMem + info.diskUsed = status.diskUsed + info.activeTasks = executorsListener.executorToTasksActive.getOrElse(info.execId, 0) + info.failedTasks = executorsListener.executorToTasksFailed.getOrElse(info.execId, 0) + info.completedTasks = executorsListener.executorToTasksComplete.getOrElse(info.execId, 0) + info.totalTasks = info.activeTasks + info.failedTasks + info.completedTasks + info.duration = executorsListener.executorToDuration.getOrElse(info.execId, 0L) + info.inputBytes = executorsListener.executorToInputBytes.getOrElse(info.execId, 0L) + info.shuffleRead = executorsListener.executorToShuffleRead.getOrElse(info.execId, 0L) + info.shuffleWrite = executorsListener.executorToShuffleWrite.getOrElse(info.execId, 0L) + + _executorData.setExecutorInfo(info.execId, info) + } + } + _executorData + } + + override def getJobProgressData(): SparkJobProgressData = { + if (_jobProgressData == null) { + _jobProgressData = new SparkJobProgressData() + + // Add JobInfo + jobProgressListener.jobIdToData.foreach { case (id, data) => + val jobInfo = new JobInfo() + + jobInfo.jobId = data.jobId + jobInfo.jobGroup = data.jobGroup.getOrElse("") + jobInfo.numActiveStages = data.numActiveStages + jobInfo.numActiveTasks = data.numActiveTasks + jobInfo.numCompletedTasks = data.numCompletedTasks + jobInfo.numFailedStages = data.numFailedStages + jobInfo.numFailedTasks = data.numFailedTasks + jobInfo.numSkippedStages = data.numSkippedStages + jobInfo.numSkippedTasks = data.numSkippedTasks + jobInfo.numTasks = data.numTasks + + jobInfo.startTime = data.submissionTime.getOrElse(0) + jobInfo.endTime = data.completionTime.getOrElse(0) + + data.stageIds.foreach{ case (id: Int) => jobInfo.addStageId(id)} + addIntSetToJSet(data.completedStageIndices, jobInfo.completedStageIndices) + + _jobProgressData.addJobInfo(id, jobInfo) + } + + // Add Stage Info + jobProgressListener.stageIdToData.foreach { case (id, data) => + val stageInfo = new SparkJobProgressData.StageInfo() + val sparkStageInfo = jobProgressListener.stageIdToInfo.get(id._1) + stageInfo.name = sparkStageInfo match { + case Some(info: StageInfo) => { + info.name + } + case None => { + "" + } + } + stageInfo.description = data.description.getOrElse("") + stageInfo.diskBytesSpilled = data.diskBytesSpilled + stageInfo.executorRunTime = data.executorRunTime + stageInfo.duration = sparkStageInfo match { + case Some(info: StageInfo) => { + val submissionTime = info.submissionTime.getOrElse(0L) + info.completionTime.getOrElse(submissionTime) - submissionTime + } + case _ => 0L + } + stageInfo.inputBytes = data.inputBytes + stageInfo.memoryBytesSpilled = data.memoryBytesSpilled + stageInfo.numActiveTasks = data.numActiveTasks + stageInfo.numCompleteTasks = data.numCompleteTasks + stageInfo.numFailedTasks = data.numFailedTasks + stageInfo.outputBytes = data.outputBytes + stageInfo.shuffleReadBytes = data.shuffleReadTotalBytes + stageInfo.shuffleWriteBytes = data.shuffleWriteBytes + addIntSetToJSet(data.completedIndices, stageInfo.completedIndices) + + _jobProgressData.addStageInfo(id._1, id._2, stageInfo) + } + + // Add completed jobs + jobProgressListener.completedJobs.foreach { case (data) => _jobProgressData.addCompletedJob(data.jobId) } + // Add failed jobs + jobProgressListener.failedJobs.foreach { case (data) => _jobProgressData.addFailedJob(data.jobId) } + // Add completed stages + jobProgressListener.completedStages.foreach { case (data) => + _jobProgressData.addCompletedStages(data.stageId, data.attemptId) + } + // Add failed stages + jobProgressListener.failedStages.foreach { case (data) => + _jobProgressData.addFailedStages(data.stageId, data.attemptId) + } + } + _jobProgressData + } + + // This method returns a combined information from StorageStatusListener and StorageListener + override def getStorageData(): SparkStorageData = { + if (_storageData == null) { + _storageData = new SparkStorageData() + _storageData.setRddInfoList(toJList[RDDInfo](storageListener.rddInfoList)) + _storageData.setStorageStatusList(toJList[StorageStatus](storageStatusListener.storageStatusList)) + } + _storageData + } + + override def getAppId: String = { + getGeneralData().getApplicationId + } + + def load(in: InputStream, sourceName: String): Unit = { + val replayBus = new ReplayListenerBus() + replayBus.addListener(applicationEventListener) + replayBus.addListener(jobProgressListener) + replayBus.addListener(environmentListener) + replayBus.addListener(storageStatusListener) + replayBus.addListener(executorsListener) + replayBus.addListener(storageListener) + replayBus.addListener(storageStatusTrackingListener) + replayBus.replay(in, sourceName, maybeTruncated = false) + } +} + +object SparkDataCollection { + private val APPLICATION_TYPE = new ApplicationType("SPARK") + + def stringToSet(str: String): JSet[String] = { + val set = new JHashSet[String]() + str.split(",").foreach { case t: String => set.add(t)} + set + } + + def toJList[T](seq: Seq[T]): JList[T] = { + val list = new JArrayList[T]() + seq.foreach { case (item: T) => list.add(item)} + list + } + + def addIntSetToJSet(set: OpenHashSet[Int], jset: JSet[Integer]): Unit = { + val it = set.iterator + while (it.hasNext) { + jset.add(it.next()) + } + } + + def addIntSetToJSet(set: mutable.HashSet[Int], jset: JSet[Integer]): Unit = { + val it = set.iterator + while (it.hasNext) { + jset.add(it.next()) + } + } +} diff --git a/app/org/apache/spark/deploy/history/SparkFSFetcher.scala b/app/org/apache/spark/deploy/history/SparkFSFetcher.scala new file mode 100644 index 000000000..e0b3b336d --- /dev/null +++ b/app/org/apache/spark/deploy/history/SparkFSFetcher.scala @@ -0,0 +1,119 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.deploy.history + +import java.io.InputStream +import java.security.PrivilegedAction + +import com.linkedin.drelephant.analysis.{AnalyticJob, ElephantFetcher} +import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData +import com.linkedin.drelephant.security.HadoopSecurity +import com.linkedin.drelephant.spark.legacydata.SparkApplicationData +import com.linkedin.drelephant.util.{HadoopUtils, SparkUtils, Utils} +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.log4j.Logger +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.{ApplicationEventListener, ReplayListenerBus} +import org.apache.spark.storage.{StorageStatusListener, StorageStatusTrackingListener} +import org.apache.spark.ui.env.EnvironmentListener +import org.apache.spark.ui.exec.ExecutorsListener +import org.apache.spark.ui.jobs.JobProgressListener +import org.apache.spark.ui.storage.StorageListener + + +/** + * A wrapper that replays Spark event history from files and then fill proper data objects. + */ +class SparkFSFetcher(fetcherConfData: FetcherConfigurationData) extends ElephantFetcher[SparkApplicationData] { + import SparkFSFetcher._ + + val eventLogSizeLimitMb = + Option(fetcherConfData.getParamMap.get(LOG_SIZE_XML_FIELD)) + .flatMap { x => Option(Utils.getParam(x, 1)) } + .map { _(0) } + .getOrElse(DEFAULT_EVENT_LOG_SIZE_LIMIT_MB) + logger.info("The event log limit of Spark application is set to " + eventLogSizeLimitMb + " MB") + + private lazy val security = new HadoopSecurity() + + protected lazy val hadoopUtils: HadoopUtils = HadoopUtils + + protected lazy val sparkUtils: SparkUtils = SparkUtils + + protected lazy val hadoopConfiguration: Configuration = new Configuration() + + protected lazy val sparkConf: SparkConf = { + val sparkConf = new SparkConf() + sparkUtils.getDefaultPropertiesFile() match { + case Some(filename) => sparkConf.setAll(sparkUtils.getPropertiesFromFile(filename)) + case None => throw new IllegalStateException("can't find Spark conf; please set SPARK_HOME or SPARK_CONF_DIR") + } + sparkConf + } + + def fetchData(analyticJob: AnalyticJob): SparkApplicationData = { + val appId = analyticJob.getAppId() + doAsPrivilegedAction { () => doFetchData(appId) } + } + + protected def doAsPrivilegedAction[T](action: () => T): T = + security.doAs[T](new PrivilegedAction[T] { override def run(): T = action() }) + + protected def doFetchData(appId: String): SparkDataCollection = { + val dataCollection = new SparkDataCollection() + + val (eventLogFileSystem, baseEventLogPath) = + sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + val (eventLogPath, eventLogCodec) = + sparkUtils.pathAndCodecforEventLog(sparkConf, eventLogFileSystem, baseEventLogPath, appId, DEFAULT_ATTEMPT_ID) + + // Check if the log parser should be throttled when the file is too large. + val shouldThrottle = eventLogFileSystem.getFileStatus(eventLogPath).getLen() > (eventLogSizeLimitMb * FileUtils.ONE_MB) + if (shouldThrottle) { + dataCollection.throttle() + // Since the data set is empty, we need to set the application id, + // so that we could detect this is Spark job type + dataCollection.getGeneralData().setApplicationId(appId) + dataCollection.getConf().setProperty("spark.app.id", appId) + + logger.info("The event log of Spark application: " + appId + " is over the limit size of " + + eventLogSizeLimitMb + " MB, the parsing process gets throttled.") + } else { + logger.info("Replaying Spark logs for application: " + appId) + + sparkUtils.withEventLog(eventLogFileSystem, eventLogPath, eventLogCodec) { in => + dataCollection.load(in, eventLogPath.toString()) + } + + logger.info("Replay completed for application: " + appId) + } + + dataCollection + } +} + +object SparkFSFetcher { + private val logger = Logger.getLogger(SparkFSFetcher.getClass) + + val DEFAULT_EVENT_LOG_SIZE_LIMIT_MB = 100d; // 100MB + + val LOG_SIZE_XML_FIELD = "event_log_size_limit_in_mb" + + val DEFAULT_ATTEMPT_ID = Some("1") +} diff --git a/app/org/apache/spark/storage/StorageStatusTrackingListener.scala b/app/org/apache/spark/storage/StorageStatusTrackingListener.scala new file mode 100644 index 000000000..5d30a2887 --- /dev/null +++ b/app/org/apache/spark/storage/StorageStatusTrackingListener.scala @@ -0,0 +1,110 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.storage + + +import scala.collection.mutable + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.scheduler._ + + +/** + * :: DeveloperApi :: + * A modified version of StorageStatusListener that tracks the peak memory usage during the entire application runtime. + * + * NOTICE: this class copies StorageStatusListener's codes instead of extending from it, because the methods that + * require being overridden are all in private scope. + */ +@DeveloperApi +class StorageStatusTrackingListener extends SparkListener { + // This maintains only blocks that are cached (i.e. storage level is not StorageLevel.NONE) + private[storage] val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() + + def storageStatusList = executorIdToStorageStatus.values.toSeq + + val executorIdToMaxUsedMem = mutable.Map[String, Long]() + + /** Update storage status list to reflect updated block statuses */ + private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = { + executorIdToStorageStatus.get(execId).foreach { storageStatus => + updatedBlocks.foreach { case (blockId, updatedStatus) => + if (updatedStatus.storageLevel == StorageLevel.NONE) { + storageStatus.removeBlock(blockId) + } else { + storageStatus.updateBlock(blockId, updatedStatus) + } + } + } + updateUsedMem() + } + + /** Update storage status list to reflect the removal of an RDD from the cache */ + private def updateStorageStatus(unpersistedRDDId: Int): Unit = { + storageStatusList.foreach { storageStatus => + storageStatus.rddBlocksById(unpersistedRDDId).foreach { case (blockId, _) => + storageStatus.removeBlock(blockId) + } + } + updateUsedMem() + } + + private def updateUsedMem(): Unit = { + executorIdToStorageStatus.foreach { case (execId, storageStatus) => + val currentMemUsed = storageStatus.memUsed + if (currentMemUsed > executorIdToMaxUsedMem.getOrElse(execId, 0L)) { + executorIdToMaxUsedMem(execId) = currentMemUsed + } + } + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + synchronized { + val info = taskEnd.taskInfo + val metrics = taskEnd.taskMetrics + if (info != null && metrics != null) { + val updatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) + if (updatedBlocks.length > 0) { + updateStorageStatus(info.executorId, updatedBlocks) + } + } + } + } + + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = { + synchronized { + updateStorageStatus(unpersistRDD.rddId) + } + } + + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { + synchronized { + val blockManagerId = blockManagerAdded.blockManagerId + val executorId = blockManagerId.executorId + val maxMem = blockManagerAdded.maxMem + val storageStatus = new StorageStatus(blockManagerId, maxMem) + executorIdToStorageStatus(executorId) = storageStatus + } + } + + override def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = { + synchronized { + val executorId = blockManagerRemoved.blockManagerId.executorId + executorIdToStorageStatus.remove(executorId) + } + } +} diff --git a/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala b/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala index 7d37168a6..384eff475 100644 --- a/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala +++ b/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala @@ -16,24 +16,29 @@ package com.linkedin.drelephant.spark.fetchers -import java.io.{File, FileOutputStream, InputStream, OutputStream} +import java.io.InputStream +import java.nio.file.Files import java.util.Date import scala.collection.JavaConverters import scala.concurrent.{ExecutionContext, Future} -import com.google.common.io.Files import com.linkedin.drelephant.analysis.{AnalyticJob, ApplicationType} import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData import com.linkedin.drelephant.spark.data.{SparkLogDerivedData, SparkRestDerivedData} import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo} -import com.linkedin.drelephant.util.SparkUtils +import com.linkedin.drelephant.spark.legacydata.{MockSparkApplicationData, SparkGeneralData} +import com.linkedin.drelephant.util.{SparkUtils, HadoopUtils} +import org.apache.hadoop.fs.Path +import org.apache.log4j.Logger import org.apache.spark.SparkConf +import org.apache.spark.deploy.history.SparkFSFetcher import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate import org.mockito.Mockito import org.scalatest.{FunSpec, Matchers} +import org.scalatest.mockito.MockitoSugar -class SparkFetcherTest extends FunSpec with Matchers { +class SparkFetcherTest extends FunSpec with Matchers with MockitoSugar { import SparkFetcherTest._ describe("SparkFetcher") { @@ -75,6 +80,38 @@ class SparkFetcherTest extends FunSpec with Matchers { data.appId should be(appId) } + it("returns data from the legacy fetchers if the REST client fails") { + val sharedSparkConf = new SparkConf() + sharedSparkConf.set("spark.eventLog.dir", "/logs/spark") + + val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { + override lazy val sparkConf = sharedSparkConf + override lazy val sparkRestClient = newFakeSparkRestClient(appId, Future { throw new Exception() }) + override lazy val sparkLogClient = Some(newFakeSparkLogClient(appId, Some("2"), Future { throw new Exception() })) + override lazy val backupFetcher = new SparkFetcher.LegacyFetcher(fetcherConfigurationData) { + override lazy val legacyFetcher = new SparkFSFetcher(fetcherConfigurationData) { + override lazy val sparkConf = sharedSparkConf + override def fetchData(analyticJob: AnalyticJob) = new MockSparkApplicationData() { + val generalData = { + val generalData = new SparkGeneralData() + generalData.setApplicationId("application_1") + generalData.setApplicationName("app") + generalData.setStartTime(1000L) + generalData.setEndTime(2000L) + generalData.setSparkUser("foo") + generalData + } + + override def getGeneralData(): SparkGeneralData = generalData + } + } + } + } + + val data = sparkFetcher.fetchData(analyticJob) + data.appId should be("application_1") + } + it("throws an exception if the REST client fails") { val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { override lazy val sparkConf = new SparkConf() @@ -96,22 +133,25 @@ class SparkFetcherTest extends FunSpec with Matchers { } it("gets its SparkConf when SPARK_CONF_DIR is set") { - val tempDir = Files.createTempDir() + val tempDir = Files.createTempDirectory(null) val testResourceIn = getClass.getClassLoader.getResourceAsStream("spark-defaults.conf") - val testResourceFile = new File(tempDir, "spark-defaults.conf") - val testResourceOut = new FileOutputStream(testResourceFile) - managedCopyInputStreamToOutputStream(testResourceIn, testResourceOut) + val testResourceFile = tempDir.resolve("spark-defaults.conf") + Files.copy(testResourceIn, testResourceFile) val fetcherConfigurationData = newFakeFetcherConfigurationData() val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { override lazy val sparkUtils = new SparkUtils() { - override val defaultEnv = Map("SPARK_CONF_DIR" -> tempDir.toString) + override lazy val logger = mock[Logger] + override lazy val hadoopUtils = mock[HadoopUtils] + override lazy val defaultEnv = Map("SPARK_CONF_DIR" -> tempDir.toString) } } val sparkConf = sparkFetcher.sparkConf - tempDir.delete() + testResourceIn.close() + Files.delete(testResourceFile) + Files.delete(tempDir) sparkConf.get("spark.yarn.historyServer.address") should be("jh1.grid.example.com:18080") sparkConf.get("spark.eventLog.enabled") should be("true") @@ -120,24 +160,27 @@ class SparkFetcherTest extends FunSpec with Matchers { } it("gets its SparkConf when SPARK_HOME is set") { - val tempDir = Files.createTempDir() - val tempConfDir = new File(tempDir, "conf") - tempConfDir.mkdir() + val tempDir = Files.createTempDirectory(null) + val tempConfDir = Files.createDirectory(tempDir.resolve("conf")) val testResourceIn = getClass.getClassLoader.getResourceAsStream("spark-defaults.conf") - val testResourceFile = new File(tempConfDir, "spark-defaults.conf") - val testResourceOut = new FileOutputStream(testResourceFile) - managedCopyInputStreamToOutputStream(testResourceIn, testResourceOut) + val testResourceFile = tempConfDir.resolve("spark-defaults.conf") + Files.copy(testResourceIn, testResourceFile) val fetcherConfigurationData = newFakeFetcherConfigurationData() val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { override lazy val sparkUtils = new SparkUtils() { - override val defaultEnv = Map("SPARK_HOME" -> tempDir.toString) + override lazy val logger = mock[Logger] + override lazy val hadoopUtils = mock[HadoopUtils] + override lazy val defaultEnv = Map("SPARK_HOME" -> tempDir.toString) } } val sparkConf = sparkFetcher.sparkConf - tempDir.delete() + testResourceIn.close() + Files.delete(testResourceFile) + Files.delete(tempConfDir) + Files.delete(tempDir) sparkConf.get("spark.yarn.historyServer.address") should be("jh1.grid.example.com:18080") sparkConf.get("spark.eventLog.enabled") should be("true") @@ -148,7 +191,11 @@ class SparkFetcherTest extends FunSpec with Matchers { it("throws an exception if neither SPARK_CONF_DIR nor SPARK_HOME are set") { val fetcherConfigurationData = newFakeFetcherConfigurationData() val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { - override lazy val sparkUtils = new SparkUtils() { override val defaultEnv = Map.empty[String, String] } + override lazy val sparkUtils = new SparkUtils() { + override lazy val logger = mock[Logger] + override lazy val hadoopUtils = mock[HadoopUtils] + override lazy val defaultEnv = Map.empty[String, String] + } } an[IllegalStateException] should be thrownBy { sparkFetcher.sparkConf } } @@ -195,21 +242,4 @@ object SparkFetcherTest { Mockito.when(sparkLogClient.fetchData(appId, attemptId)).thenReturn(logDerivedData) sparkLogClient } - - def managedCopyInputStreamToOutputStream(in: => InputStream, out: => OutputStream): Unit = { - for { - input <- resource.managed(in) - output <- resource.managed(out) - } { - val buffer = new Array[Byte](512) - def read(): Unit = input.read(buffer) match { - case -1 => () - case bytesRead => { - output.write(buffer, 0, bytesRead) - read() - } - } - read() - } - } } diff --git a/test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala b/test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala index d5fd38927..e4ef1877d 100644 --- a/test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala +++ b/test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala @@ -16,73 +16,54 @@ package com.linkedin.drelephant.spark.fetchers -import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream, OutputStream} +import java.io.{ByteArrayOutputStream, InputStream} import java.net.URI import scala.concurrent.ExecutionContext +import com.linkedin.drelephant.util.{SparkUtils, SparkUtilsTest} +import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FSDataInputStream, FileSystem, Path, PositionedReadable} -import org.apache.hadoop.io.compress.CompressionInputStream +import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf -import org.mockito.BDDMockito import org.scalatest.{AsyncFunSpec, Matchers} import org.scalatest.mockito.MockitoSugar import org.xerial.snappy.SnappyOutputStream class SparkLogClientTest extends AsyncFunSpec with Matchers with MockitoSugar { - import SparkLogClientTest._ - describe("SparkLogClient") { - it("throws an exception if spark.eventLog.dir is missing") { - an[IllegalArgumentException] should be thrownBy { new SparkLogClient(new Configuration(), new SparkConf()) } - } - - it("uses spark.eventLog.dir if it is already an webhdfs URI") { - val hadoopConfiguration = new Configuration() - val sparkConf = new SparkConf().set("spark.eventLog.dir", "webhdfs://nn1.grid.example.com:50070/logs/spark") - val sparkLogClient = new SparkLogClient(hadoopConfiguration, sparkConf) - sparkLogClient.webhdfsEventLogUri should be(new URI("webhdfs://nn1.grid.example.com:50070/logs/spark")) - } - - it("uses a webhdfs URI constructed from spark.eventLog.dir and dfs.namenode.http-address if spark.eventLog.dir is an hdfs URI") { - val hadoopConfiguration = new Configuration() - hadoopConfiguration.set("dfs.namenode.http-address", "0.0.0.0:50070") - val sparkConf = new SparkConf().set("spark.eventLog.dir", "hdfs://nn1.grid.example.com:9000/logs/spark") - val sparkLogClient = new SparkLogClient(hadoopConfiguration, sparkConf) - sparkLogClient.webhdfsEventLogUri should be(new URI("webhdfs://nn1.grid.example.com:50070/logs/spark")) - } - - it("returns the desired data from the Spark event logs") { - import ExecutionContext.Implicits.global - - val hadoopConfiguration = new Configuration() - hadoopConfiguration.set("dfs.namenode.http-address", "0.0.0.0:50070") + it("returns log-derived data") { + val hadoopConfiguration = new Configuration(false) val sparkConf = new SparkConf() - .set("spark.eventLog.dir", "hdfs://nn1.grid.example.com:9000/logs/spark") + .set("spark.eventLog.dir", "webhdfs://nn1.grid.example.com:50070/logs/spark") .set("spark.eventLog.compress", "true") .set("spark.io.compression.codec", "snappy") val appId = "application_1" val attemptId = Some("1") - val testResourceIn = getClass.getClassLoader.getResourceAsStream("spark_event_logs/event_log_2") - val byteOut = new ByteArrayOutputStream() - val snappyOut = new SnappyOutputStream(byteOut) - managedCopyInputStreamToOutputStream(testResourceIn, snappyOut) + val eventLogBytes = { + val bout = new ByteArrayOutputStream() + for { + in <- resource.managed(getClass.getClassLoader.getResourceAsStream("spark_event_logs/event_log_2")) + out <- resource.managed(new SnappyOutputStream(bout)) + } { + IOUtils.copy(in, out) + } + bout.toByteArray + } val sparkLogClient = new SparkLogClient(hadoopConfiguration, sparkConf) { - override lazy val fs: FileSystem = { - val fs = mock[FileSystem] - val expectedPath = new Path("webhdfs://nn1.grid.example.com:50070/logs/spark/application_1_1.snappy") - BDDMockito.given(fs.exists(expectedPath)).willReturn(true) - BDDMockito.given(fs.open(expectedPath)).willReturn( - new FSDataInputStream(new FakeCompressionInputStream(new ByteArrayInputStream(byteOut.toByteArray))) - ) - fs - } + override lazy val sparkUtils = SparkUtilsTest.newFakeSparkUtilsForEventLog( + new URI("webhdfs://nn1.grid.example.com:50070"), + new Path("/logs/spark"), + new Path("application_1_1.snappy"), + eventLogBytes + ) + + override protected def doAsPrivilegedAction[T](action: () => T): T = action() } sparkLogClient.fetchData(appId, attemptId).map { logDerivedData => @@ -100,31 +81,3 @@ class SparkLogClientTest extends AsyncFunSpec with Matchers with MockitoSugar { } } } - -object SparkLogClientTest { - class FakeCompressionInputStream(in: InputStream) extends CompressionInputStream(in) with PositionedReadable { - override def read(): Int = in.read() - override def read(b: Array[Byte], off: Int, len: Int): Int = in.read(b, off, len) - override def read(pos: Long, buffer: Array[Byte], off: Int, len: Int): Int = ??? - override def readFully(pos: Long, buffer: Array[Byte], off: Int, len: Int): Unit = ??? - override def readFully(pos: Long, buffer: Array[Byte]): Unit = ??? - override def resetState(): Unit = ??? - } - - def managedCopyInputStreamToOutputStream(in: => InputStream, out: => OutputStream): Unit = { - for { - input <- resource.managed(in) - output <- resource.managed(out) - } { - val buffer = new Array[Byte](512) - def read(): Unit = input.read(buffer) match { - case -1 => () - case bytesRead => { - output.write(buffer, 0, bytesRead) - read() - } - } - read() - } - } -} diff --git a/test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala b/test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala index 5c48e42aa..71a401d32 100644 --- a/test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala +++ b/test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala @@ -41,10 +41,6 @@ class SparkRestClientTest extends AsyncFunSpec with Matchers { import SparkRestClientTest._ describe("SparkRestClient") { - it("throws an exception if spark.eventLog.dir is missing") { - an[IllegalArgumentException] should be thrownBy(new SparkRestClient(new SparkConf())) - } - it("returns the desired data from the Spark REST API for cluster mode application") { import ExecutionContext.Implicits.global val fakeJerseyServer = new FakeJerseyServer() { @@ -146,6 +142,24 @@ class SparkRestClientTest extends AsyncFunSpec with Matchers { assertion } } + + it("throws an exception if spark.yarn.historyServer.address is missing") { + an[IllegalArgumentException] should be thrownBy(new SparkRestClient(new SparkConf())) + } + + it("handles unrecognized fields gracefully when parsing") { + val objectMapper = SparkRestClient.SparkRestObjectMapper + val json = s"""{ + "startTime" : "2016-09-12T19:30:18.101GMT", + "endTime" : "1969-12-31T23:59:59.999GMT", + "sparkUser" : "foo", + "completed" : false, + "unrecognized" : "bar" + }""" + + val applicationAttemptInfo = objectMapper.readValue[ApplicationAttemptInfo](json) + applicationAttemptInfo.sparkUser should be("foo") + } } } diff --git a/test/com/linkedin/drelephant/spark/legacydata/LegacyDataConvertersTest.scala b/test/com/linkedin/drelephant/spark/legacydata/LegacyDataConvertersTest.scala new file mode 100644 index 000000000..ad8e7511c --- /dev/null +++ b/test/com/linkedin/drelephant/spark/legacydata/LegacyDataConvertersTest.scala @@ -0,0 +1,320 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacydata + +import java.util.Date + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.status.api.v1.StageStatus +import org.scalatest.{FunSpec, Matchers} + + +class LegacyDataConvertersTest extends FunSpec with Matchers { + describe("LegacyDataConverters") { + describe(".convert") { + } + + describe(".extractAppConfigurationProperties") { + it("returns a Map of Spark properties extracted from the given legacy SparkApplicationData") { + val legacyData = new MockSparkApplicationData() { + val environmentData = { + val environmentData = new SparkEnvironmentData() + environmentData.addSparkProperty("a", "b") + environmentData.addSparkProperty("c", "d") + environmentData + } + + override def getEnvironmentData(): SparkEnvironmentData = environmentData + } + + val appConfigurationProperties = LegacyDataConverters.extractAppConfigurationProperties(legacyData) + appConfigurationProperties should contain theSameElementsAs Map("a" -> "b", "c" -> "d") + } + } + + describe(".extractApplicationInfo") { + it("returns an ApplicationInfo extracted from the given legacy SparkApplicationData") { + val legacyData = new MockSparkApplicationData() { + val generalData = { + val generalData = new SparkGeneralData() + generalData.setApplicationId("application_1") + generalData.setApplicationName("app") + generalData.setStartTime(1000L) + generalData.setEndTime(2000L) + generalData.setSparkUser("foo") + generalData + } + + override def getGeneralData(): SparkGeneralData = generalData + } + + val applicationInfo = LegacyDataConverters.extractApplicationInfo(legacyData) + applicationInfo.id should be("application_1") + applicationInfo.name should be("app") + applicationInfo.attempts.size should be(1) + + val applicationAttemptInfo = applicationInfo.attempts.last + applicationAttemptInfo.attemptId should be(Some("1")) + applicationAttemptInfo.startTime should be(new Date(1000L)) + applicationAttemptInfo.endTime should be(new Date(2000L)) + applicationAttemptInfo.sparkUser should be("foo") + applicationAttemptInfo.completed should be(true) + } + } + + describe(".extractJobDatas") { + it("returns JobDatas extracted from the given legacy SparkApplicationData") { + val legacyData = new MockSparkApplicationData() { + val jobProgressData = { + val jobProgressData = new SparkJobProgressData() + + val jobInfo1 = { + val jobInfo = new SparkJobProgressData.JobInfo() + jobInfo.jobId = 1 + + jobInfo.numTasks = 10 + jobInfo.numActiveTasks = 1 + jobInfo.numCompletedTasks = 2 + jobInfo.numSkippedTasks = 3 + jobInfo.numFailedTasks = 4 + + for (i <- 1 to 100) { jobInfo.stageIds.add(i) } + jobInfo.numActiveStages = 10 + for (i <- 1 to 20) { jobInfo.completedStageIndices.add(i) } + jobInfo.numSkippedStages = 30 + jobInfo.numFailedStages = 40 + + jobInfo + } + jobProgressData.addJobInfo(1, jobInfo1) + jobProgressData.addCompletedJob(1) + + val jobInfo2 = { + val jobInfo = new SparkJobProgressData.JobInfo() + jobInfo.jobId = 2 + jobInfo + } + jobProgressData.addJobInfo(2, jobInfo2) + jobProgressData.addFailedJob(2) + + jobProgressData + } + + override def getJobProgressData(): SparkJobProgressData = jobProgressData + } + + val jobDatas = LegacyDataConverters.extractJobDatas(legacyData) + jobDatas.size should be(2) + + val jobData1 = jobDatas(0) + jobData1.jobId should be(1) + jobData1.name should be("1") + jobData1.description should be(None) + jobData1.submissionTime should be(None) + jobData1.completionTime should be(None) + jobData1.stageIds should be((1 to 100).toSeq) + jobData1.jobGroup should be(None) + jobData1.status should be(JobExecutionStatus.SUCCEEDED) + jobData1.numTasks should be(10) + jobData1.numActiveTasks should be(1) + jobData1.numCompletedTasks should be(2) + jobData1.numSkippedTasks should be(3) + jobData1.numFailedTasks should be(4) + jobData1.numActiveStages should be(10) + jobData1.numCompletedStages should be(20) + jobData1.numSkippedStages should be(30) + jobData1.numFailedStages should be(40) + + val jobData2 = jobDatas(1) + jobData2.jobId should be(2) + jobData2.name should be("2") + jobData2.status should be(JobExecutionStatus.FAILED) + } + } + + describe(".extractStageDatas") { + it("returns StageDatas extracted from the given legacy SparkApplicationData") { + val legacyData = new MockSparkApplicationData() { + val jobProgressData = { + val jobProgressData = new SparkJobProgressData() + + val stageInfoS1A1 = { + val stageInfo = new SparkJobProgressData.StageInfo() + + stageInfo.numActiveTasks = 1 + stageInfo.numCompleteTasks = 2 + stageInfo.numFailedTasks = 3 + + stageInfo.executorRunTime = 1000L + + stageInfo.inputBytes = 10000L + stageInfo.outputBytes = 20000L + stageInfo.shuffleReadBytes = 30000L + stageInfo.shuffleWriteBytes = 40000L + stageInfo.memoryBytesSpilled = 50000L + stageInfo.diskBytesSpilled = 60000L + + stageInfo.name = "1,1" + stageInfo.description = "a" + + stageInfo + } + jobProgressData.addStageInfo(1, 1, stageInfoS1A1) + jobProgressData.addCompletedStages(1, 1) + + val stageInfoS1A2 = { + val stageInfo = new SparkJobProgressData.StageInfo() + stageInfo.name = "1,2" + stageInfo + } + jobProgressData.addStageInfo(1, 2, stageInfoS1A2) + jobProgressData.addCompletedStages(1, 2) + + val stageInfoS2A1 = { + val stageInfo = new SparkJobProgressData.StageInfo() + stageInfo.name = "2,1" + stageInfo + } + jobProgressData.addStageInfo(2, 1, stageInfoS2A1) + jobProgressData.addFailedStages(2, 1) + + jobProgressData + } + + override def getJobProgressData(): SparkJobProgressData = jobProgressData + } + + val stageDatas = LegacyDataConverters.extractStageDatas(legacyData) + stageDatas.size should be(3) + + val stageDataS1A1 = stageDatas(0) + stageDataS1A1.status should be(StageStatus.COMPLETE) + stageDataS1A1.stageId should be(1) + stageDataS1A1.attemptId should be(1) + stageDataS1A1.numActiveTasks should be(1) + stageDataS1A1.numCompleteTasks should be(2) + stageDataS1A1.numFailedTasks should be(3) + stageDataS1A1.executorRunTime should be(1000L) + stageDataS1A1.inputBytes should be(10000L) + stageDataS1A1.inputRecords should be(0L) + stageDataS1A1.outputBytes should be(20000L) + stageDataS1A1.outputRecords should be(0L) + stageDataS1A1.shuffleReadBytes should be(30000L) + stageDataS1A1.shuffleReadRecords should be(0L) + stageDataS1A1.shuffleWriteBytes should be(40000L) + stageDataS1A1.shuffleWriteRecords should be(0L) + stageDataS1A1.memoryBytesSpilled should be(50000L) + stageDataS1A1.diskBytesSpilled should be(60000L) + stageDataS1A1.name should be("1,1") + stageDataS1A1.details should be("a") + stageDataS1A1.schedulingPool should be("") + stageDataS1A1.accumulatorUpdates should be(Seq.empty) + stageDataS1A1.tasks should be(None) + stageDataS1A1.executorSummary should be(None) + + val stageDataS1A2 = stageDatas(1) + stageDataS1A2.status should be(StageStatus.COMPLETE) + stageDataS1A2.stageId should be(1) + stageDataS1A2.attemptId should be(2) + stageDataS1A2.name should be("1,2") + + val stageDataS2A1 = stageDatas(2) + stageDataS2A1.status should be(StageStatus.FAILED) + stageDataS2A1.stageId should be(2) + stageDataS2A1.attemptId should be(1) + stageDataS2A1.name should be("2,1") + } + } + + describe(".extractExecutorSummaries") { + it("returns ExecutorSummaries extracted from the given legacy SparkApplicationData") { + val legacyData = new MockSparkApplicationData() { + val executorData = { + val executorData = new SparkExecutorData() + + val executorInfo1 = { + val executorInfo = new SparkExecutorData.ExecutorInfo() + + executorInfo.execId = "1" + executorInfo.hostPort = "9090" + + executorInfo.rddBlocks = 10 + executorInfo.memUsed = 10000L + executorInfo.maxMem = 20000L + executorInfo.diskUsed = 30000L + + executorInfo.activeTasks = 1 + executorInfo.completedTasks = 2 + executorInfo.failedTasks = 3 + executorInfo.totalTasks = 6 + + executorInfo.duration = 1000L + + executorInfo.inputBytes = 100000L + executorInfo.shuffleRead = 200000L + executorInfo.shuffleWrite = 300000L + + executorInfo + } + executorData.setExecutorInfo("1", executorInfo1) + + val executorInfo2 = { + val executorInfo = new SparkExecutorData.ExecutorInfo() + executorInfo.execId = "2" + executorInfo + } + executorData.setExecutorInfo("2", executorInfo2) + + executorData + } + + override def getExecutorData(): SparkExecutorData = executorData + } + + val executorSummaries = LegacyDataConverters.extractExecutorSummaries(legacyData) + executorSummaries.size should be(2) + + val executorSummary1 = executorSummaries(0) + executorSummary1.id should be("1") + executorSummary1.hostPort should be("9090") + executorSummary1.rddBlocks should be(10) + executorSummary1.memoryUsed should be(10000L) + executorSummary1.diskUsed should be(30000L) + executorSummary1.activeTasks should be(1) + executorSummary1.failedTasks should be(3) + executorSummary1.completedTasks should be(2) + executorSummary1.totalTasks should be(6) + executorSummary1.totalDuration should be(1000L) + executorSummary1.totalInputBytes should be(100000L) + executorSummary1.totalShuffleRead should be(200000L) + executorSummary1.totalShuffleWrite should be(300000L) + executorSummary1.maxMemory should be(20000L) + executorSummary1.executorLogs should be(Map.empty) + + val executorSummary2 = executorSummaries(1) + executorSummary2.id should be("2") + } + } + + describe(".") { + } + } +} + +object LegacyDataConvertersTest { + +} diff --git a/test/com/linkedin/drelephant/spark/legacydata/MockSparkApplicationData.java b/test/com/linkedin/drelephant/spark/legacydata/MockSparkApplicationData.java new file mode 100644 index 000000000..34917f6a4 --- /dev/null +++ b/test/com/linkedin/drelephant/spark/legacydata/MockSparkApplicationData.java @@ -0,0 +1,92 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacydata; + +import com.linkedin.drelephant.analysis.ApplicationType; +import java.util.Properties; + + +/** + * This is a pseudo local implementation of SparkApplicationData interface, supposed to be used for test purpose. + */ +public class MockSparkApplicationData implements SparkApplicationData { + private static final ApplicationType APPLICATION_TYPE = new ApplicationType("SPARK"); + + private final SparkGeneralData _sparkGeneralData; + private final SparkEnvironmentData _sparkEnvironmentData; + private final SparkExecutorData _sparkExecutorData; + private final SparkJobProgressData _sparkJobProgressData; + private final SparkStorageData _sparkStorageData; + + public MockSparkApplicationData() { + _sparkGeneralData = new SparkGeneralData(); + _sparkEnvironmentData = new SparkEnvironmentData(); + _sparkExecutorData = new SparkExecutorData(); + _sparkJobProgressData = new SparkJobProgressData(); + _sparkStorageData = new SparkStorageData(); + } + + @Override + public boolean isThrottled() { + return false; + } + + @Override + public SparkGeneralData getGeneralData() { + return _sparkGeneralData; + } + + @Override + public SparkEnvironmentData getEnvironmentData() { + return _sparkEnvironmentData; + } + + @Override + public SparkExecutorData getExecutorData() { + return _sparkExecutorData; + } + + @Override + public SparkJobProgressData getJobProgressData() { + return _sparkJobProgressData; + } + + @Override + public SparkStorageData getStorageData() { + return _sparkStorageData; + } + + @Override + public Properties getConf() { + return getEnvironmentData().getSparkProperties(); + } + + @Override + public String getAppId() { + return getGeneralData().getApplicationId(); + } + + @Override + public ApplicationType getApplicationType() { + return APPLICATION_TYPE; + } + + @Override + public boolean isEmpty() { + return getExecutorData().getExecutors().isEmpty(); + } +} diff --git a/test/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristicTest.java b/test/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristicTest.java new file mode 100644 index 000000000..3666500e9 --- /dev/null +++ b/test/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristicTest.java @@ -0,0 +1,93 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacyheuristics; + +import com.linkedin.drelephant.analysis.ApplicationType; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.spark.legacydata.MockSparkApplicationData; +import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import junit.framework.TestCase; + +import static com.linkedin.drelephant.spark.legacyheuristics.BestPropertiesConventionHeuristic.SPARK_DRIVER_MEMORY; +import static com.linkedin.drelephant.spark.legacyheuristics.BestPropertiesConventionHeuristic.SPARK_EXECUTOR_CORES; +import static com.linkedin.drelephant.spark.legacyheuristics.BestPropertiesConventionHeuristic.SPARK_SERIALIZER; +import static com.linkedin.drelephant.spark.legacyheuristics.BestPropertiesConventionHeuristic.SPARK_SHUFFLE_MANAGER; + + +/** + * This class test the BestPropertiesConventionHeuristic + * + */ +public class BestPropertiesConventionHeuristicTest extends TestCase { + public void testPropertiesCheck() { + assertEquals(analyzeJob(getDefaultGoodProperteis()), Severity.NONE); + + assertEquals(Severity.MODERATE, analyzeJob(getPropertiesAndOverideOne(SPARK_DRIVER_MEMORY, "7G"))); + assertEquals(Severity.CRITICAL, analyzeJob(getPropertiesAndOverideOne(SPARK_DRIVER_MEMORY, "8G"))); + assertEquals(Severity.CRITICAL, analyzeJob(getPropertiesAndOverideOne(SPARK_DRIVER_MEMORY, "9G"))); + + assertEquals(Severity.NONE, analyzeJob(getPropertiesAndOverideOne(SPARK_EXECUTOR_CORES, "1"))); + assertEquals(Severity.NONE, analyzeJob(getPropertiesAndOverideOne(SPARK_EXECUTOR_CORES, "2"))); + assertEquals(Severity.CRITICAL, analyzeJob(getPropertiesAndOverideOne(SPARK_EXECUTOR_CORES, "4"))); + + + assertEquals(Severity.MODERATE, analyzeJob(getPropertiesAndOverideOne(SPARK_SERIALIZER, "foo"))); + assertEquals(Severity.MODERATE, analyzeJob(getPropertiesAndOverideOne(SPARK_SHUFFLE_MANAGER, "hash"))); + } + + public void testNullSettings() { + assertEquals(Severity.NONE, analyzeJob(getPropertiesAndRemove(SPARK_SERIALIZER))); + assertEquals(Severity.NONE, analyzeJob(getPropertiesAndRemove(SPARK_SHUFFLE_MANAGER))); + assertEquals(Severity.NONE, analyzeJob(getPropertiesAndRemove(SPARK_EXECUTOR_CORES))); + } + + private static Properties getDefaultGoodProperteis() { + Properties properties = new Properties(); + properties.put(SPARK_DRIVER_MEMORY, "1G"); + properties.put(SPARK_EXECUTOR_CORES, "1"); + properties.put(SPARK_SERIALIZER, "org.apache.spark.serializer.KryoSerializer"); + properties.put(SPARK_SHUFFLE_MANAGER, "sort"); + + return properties; + } + + private static Properties getPropertiesAndOverideOne(String key, String value) { + Properties properties = getDefaultGoodProperteis(); + properties.put(key, value); + return properties; + } + + private static Properties getPropertiesAndRemove(String key) { + Properties properties = getDefaultGoodProperteis(); + properties.remove(key); + return properties; + } + + private Severity analyzeJob(Properties sparkProperties) { + SparkApplicationData data = new MockSparkApplicationData(); + for (String key : sparkProperties.stringPropertyNames()) { + data.getEnvironmentData().addSparkProperty(key, sparkProperties.getProperty(key)); + } + Map paramsMap = new HashMap(); + return new BestPropertiesConventionHeuristic(new HeuristicConfigurationData("test_heuristic", "test_class", + "test_view", new ApplicationType("test_apptype"), paramsMap)).apply(data).getSeverity(); + } +} diff --git a/test/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristicTest.java b/test/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristicTest.java new file mode 100644 index 000000000..7ab245eed --- /dev/null +++ b/test/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristicTest.java @@ -0,0 +1,30 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacyheuristics; + +import junit.framework.TestCase; + + +/** + * This class tests JobRuntimeHeuristic + */ +public class JobRuntimeHeuristicTest extends TestCase { + + public void test() { + + } +} diff --git a/test/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristicTest.java b/test/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristicTest.java new file mode 100644 index 000000000..3c926d1a4 --- /dev/null +++ b/test/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristicTest.java @@ -0,0 +1,138 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.spark.legacyheuristics; + +import com.linkedin.drelephant.analysis.ApplicationType; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.HeuristicResultDetails; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.spark.legacydata.MockSparkApplicationData; +import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; +import com.linkedin.drelephant.spark.legacydata.SparkEnvironmentData; +import com.linkedin.drelephant.spark.legacydata.SparkExecutorData; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.util.MemoryFormatUtils; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import junit.framework.TestCase; + +import static com.linkedin.drelephant.spark.legacydata.SparkExecutorData.EXECUTOR_DRIVER_NAME; +import static com.linkedin.drelephant.spark.legacyheuristics.MemoryLimitHeuristic.SPARK_DRIVER_MEMORY; +import static com.linkedin.drelephant.spark.legacyheuristics.MemoryLimitHeuristic.SPARK_EXECUTOR_INSTANCES; +import static com.linkedin.drelephant.spark.legacyheuristics.MemoryLimitHeuristic.SPARK_EXECUTOR_MEMORY; +import static com.linkedin.drelephant.spark.legacyheuristics.MemoryLimitHeuristic.DEFAULT_SPARK_STORAGE_MEMORY_FRACTION; + + +/** + * This class tests the heuristic rule: MemoryLimitHeuristic + */ +public class MemoryLimitHeuristicTest extends TestCase { + public void testTotalMemoryRule() { + // Test if the total memory limit is working, set all peak memory to arbirarity 100% + assertEquals(Severity.NONE, analyzeJob(1, "1G", "1G", "1G")); + assertEquals(Severity.NONE, analyzeJob(100, "1G", "1G", "100G")); + assertEquals(Severity.NONE, analyzeJob(10, "10G", "1G", "100G")); + assertEquals(Severity.LOW, analyzeJob(600, "1G", "1G", "600G")); + assertEquals(Severity.MODERATE, analyzeJob(2400, "512M", "1G", "1.2T")); + assertEquals(Severity.SEVERE, analyzeJob(1600, "1G", "1G", "1.6T")); + assertEquals(Severity.CRITICAL, analyzeJob(4200, "512M", "1G", "2.1T")); + } + + public void testMemoryUtilizationRule() { + // Test if the total memory utilization is working + + // When the total memory is too low, ignore the ratio calculation + assertEquals(Severity.NONE, analyzeJob(1, "1G", "1G", "0B")); + // When we barely pass the safe zone + assertEquals(Severity.CRITICAL, analyzeJob(1000, "1G", "1G", "0B")); + + // Normal situations + assertEquals(Severity.LOW, analyzeJob(1000, "1G", "1G", getPeakMemory(0.7d, 1000, "1G"))); + assertEquals(Severity.MODERATE, analyzeJob(1000, "1G", "1G", getPeakMemory(0.5d, 1000, "1G"))); + assertEquals(Severity.SEVERE, analyzeJob(1000, "1G", "1G", getPeakMemory(0.3d, 1000, "1G"))); + assertEquals(Severity.CRITICAL, analyzeJob(1000, "1G", "1G", getPeakMemory(0.1d, 1000, "1G"))); + } + + public void testCombinedRules() { + // Mix multiple rules together, majorly check the combined logic + assertEquals(Severity.CRITICAL, analyzeJob(1, "1G", "10T", "0B")); + assertEquals(Severity.CRITICAL, analyzeJob(2500, "1G", "1G", getPeakMemory(0.7d, 2500, "1G"))); + } + + public void testMissingSparkDriverMemoryProperty() { + HeuristicResult result = getJobresult(100, "1G", "700M", getPeakMemory(1.0d, 100, "1G"), SPARK_DRIVER_MEMORY); + for (HeuristicResultDetails detail : result.getHeuristicResultDetails()) { + if (detail.getName().startsWith("\"Total driver memory allocated")) { + assertEquals("\"Total driver memory allocated\",\"700 MB\"", detail.getName()); + } + } + } + + private static String getPeakMemory(double utilRatio, int executors, String memPerExecutor) { + long totalMem = + (long) (executors * MemoryFormatUtils.stringToBytes(memPerExecutor) * DEFAULT_SPARK_STORAGE_MEMORY_FRACTION); + return MemoryFormatUtils.bytesToString((long) (utilRatio * totalMem)); + } + + private Severity analyzeJob(int executors, String memPerExecutor, String driverMem, String peakTotalMem, + String... skippedProperties) { + return getJobresult(executors, memPerExecutor, driverMem, peakTotalMem, skippedProperties).getSeverity(); + } + + private HeuristicResult getJobresult(int executors, String memPerExecutor, String driverMem, String peakTotalMem, + String... skippedProperties) { + Set filters = new HashSet(); + for (int i = 0; i < skippedProperties.length; i++) { + filters.add(skippedProperties[i]); + } + + SparkApplicationData data = new MockSparkApplicationData(); + SparkEnvironmentData env = data.getEnvironmentData(); + if (!filters.contains(SPARK_EXECUTOR_INSTANCES)) { + env.addSparkProperty(SPARK_EXECUTOR_INSTANCES, String.valueOf(executors)); + } + if (!filters.contains(SPARK_EXECUTOR_MEMORY)) { + env.addSparkProperty(SPARK_EXECUTOR_MEMORY, memPerExecutor); + } + if (!filters.contains(SPARK_DRIVER_MEMORY)) { + env.addSparkProperty(SPARK_DRIVER_MEMORY, driverMem); + } + + SparkExecutorData exe = data.getExecutorData(); + SparkExecutorData.ExecutorInfo driverInfo = new SparkExecutorData.ExecutorInfo(); + driverInfo.maxMem = (long) (MemoryFormatUtils.stringToBytes(driverMem) * DEFAULT_SPARK_STORAGE_MEMORY_FRACTION); + exe.setExecutorInfo(EXECUTOR_DRIVER_NAME, driverInfo); + + long bytesPerExecutor = MemoryFormatUtils.stringToBytes(memPerExecutor); + + /* Assign evenly the peak memory to each executor, in practical cases, we might observe the executor peak memory + * can vary a bit due to data skewness and imperfect partitioning. + */ + long peakMemToGenerate = MemoryFormatUtils.stringToBytes(peakTotalMem) / executors; + for (int i = 0; i < executors; i++) { + SparkExecutorData.ExecutorInfo info = new SparkExecutorData.ExecutorInfo(); + info.maxMem = (long) (bytesPerExecutor * DEFAULT_SPARK_STORAGE_MEMORY_FRACTION); + info.memUsed = peakMemToGenerate; + exe.setExecutorInfo(String.valueOf(i), info); + } + Map paramsMap = new HashMap(); + return new MemoryLimitHeuristic(new HeuristicConfigurationData("test_heuristic", "test_class", "test_view", + new ApplicationType("test_apptype"), paramsMap)).apply(data); + } +} diff --git a/test/com/linkedin/drelephant/util/HadoopUtilsTest.scala b/test/com/linkedin/drelephant/util/HadoopUtilsTest.scala new file mode 100644 index 000000000..753e087a7 --- /dev/null +++ b/test/com/linkedin/drelephant/util/HadoopUtilsTest.scala @@ -0,0 +1,131 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.util + +import java.io.{ByteArrayInputStream, IOException} +import java.net.{HttpURLConnection, URL} + +import com.fasterxml.jackson.databind.JsonNode +import com.fasterxml.jackson.databind.node.JsonNodeFactory +import org.apache.hadoop.conf.Configuration +import org.apache.log4j.Logger +import org.mockito.Mockito +import org.scalatest.{FunSpec, Matchers} +import org.scalatest.mockito.MockitoSugar + +class HadoopUtilsTest extends FunSpec with Matchers { + import HadoopUtilsTest._ + + describe("HadoopUtils") { + describe(".findHaNameNodeAddress") { + it("returns the first active HA name node it can find") { + val hadoopUtils = HadoopUtilsTest.newFakeHadoopUtilsForNameNode( + ("sample-ha1.grid.example.com", ("sample-ha1.grid.example.com", "standby")), + ("sample-ha2.grid.example.com", ("sample-ha2.grid.example.com", "active")) + ) + val conf = new Configuration(false) + conf.addResource("core-site.xml") + val haNameNodeAddress = hadoopUtils.findHaNameNodeAddress(conf) + haNameNodeAddress should be(Some("sample-ha2.grid.example.com:50070")) + } + + it("returns no HA name node if one isn't configured") { + val hadoopUtils = HadoopUtilsTest.newFakeHadoopUtilsForNameNode( + ("sample-ha1.grid.example.com", ("sample-ha1.grid.example.com", "standby")), + ("sample-ha2.grid.example.com", ("sample-ha2.grid.example.com", "active")) + ) + val conf = new Configuration(false) + val haNameNodeAddress = hadoopUtils.findHaNameNodeAddress(conf) + haNameNodeAddress should be(None) + } + } + + describe(".httpNameNodeAddress") { + it("returns the default name node") { + val hadoopUtils = HadoopUtilsTest.newFakeHadoopUtilsForNameNode( + ("sample-ha1.grid.example.com", ("sample-ha1.grid.example.com", "standby")), + ("sample-ha2.grid.example.com", ("sample-ha2.grid.example.com", "active")) + ) + val conf = new Configuration(false) + conf.addResource("core-site.xml") + val haNameNodeAddress = hadoopUtils.httpNameNodeAddress(conf) + haNameNodeAddress should be(Some("sample.grid.example.com:50070")) + } + } + + describe(".isActiveNameNode") { + it("returns true for active name nodes") { + val hadoopUtils = + newFakeHadoopUtilsForNameNode(Map(("nn1.grid.example.com", ("nn1-ha1.grid.example.com", "active")))) + hadoopUtils.isActiveNameNode("nn1.grid.example.com") should be(true) + } + + it("returns false for standby name nodes") { + val hadoopUtils = + newFakeHadoopUtilsForNameNode(Map(("nn1.grid.example.com", ("nn1-ha1.grid.example.com", "standby")))) + hadoopUtils.isActiveNameNode("nn1.grid.example.com") should be(false) + } + } + } +} + +object HadoopUtilsTest extends MockitoSugar { + import scala.annotation.varargs + + @varargs + def newFakeHadoopUtilsForNameNode(nameNodeHostsAndStatesByJmxHost: (String, (String, String))*): HadoopUtils = + newFakeHadoopUtilsForNameNode(nameNodeHostsAndStatesByJmxHost.toMap) + + def newFakeHadoopUtilsForNameNode(nameNodeHostsAndStatesByJmxHost: Map[String, (String, String)]): HadoopUtils = + new HadoopUtils { + override lazy val logger = mock[Logger] + + override def newAuthenticatedConnection(url: URL): HttpURLConnection = { + val conn = mock[HttpURLConnection] + val jmxHost = url.getHost + nameNodeHostsAndStatesByJmxHost.get(jmxHost) match { + case Some((host, state)) => { + val jsonNode = newFakeNameNodeStatus(host, state) + val bytes = jsonNode.toString.getBytes("UTF-8") + Mockito.when(conn.getInputStream()).thenReturn(new ByteArrayInputStream(bytes)) + } + case None => { + Mockito.when(conn.getInputStream()).thenThrow(new IOException()) + } + } + conn + } + } + + def newFakeNameNodeStatus(host: String, state: String): JsonNode = { + val jsonNodeFactory = JsonNodeFactory.instance; + + val beanJsonNode = + jsonNodeFactory.objectNode() + .put("name", "Hadoop:service=NameNode, name=NameNodeStatus") + .put("modelerType", "org.apache.hadoop.hdfs.server.namenode.NameNode") + .put("NNRole", "NameNode") + .put("HostAndPort", "s${host}:9000") + .put("SecurityEnabled", "true") + .put("State", state) + + val beansJsonNode = + jsonNodeFactory.arrayNode().add(beanJsonNode) + + jsonNodeFactory.objectNode().set("beans", beansJsonNode) + } +} diff --git a/test/com/linkedin/drelephant/util/SparkUtilsTest.scala b/test/com/linkedin/drelephant/util/SparkUtilsTest.scala new file mode 100644 index 000000000..532a6b4df --- /dev/null +++ b/test/com/linkedin/drelephant/util/SparkUtilsTest.scala @@ -0,0 +1,267 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.linkedin.drelephant.util + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream} +import java.net.URI + +import org.apache.commons.io.IOUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FSDataInputStream, FileStatus, FileSystem, Path, PositionedReadable} +import org.apache.hadoop.io.compress.CompressionInputStream +import org.apache.log4j.Logger +import org.apache.spark.SparkConf +import org.apache.spark.io.SnappyCompressionCodec +import org.mockito.BDDMockito +import org.scalatest.{FunSpec, Matchers, OptionValues} +import org.scalatest.mockito.MockitoSugar +import org.xerial.snappy.SnappyOutputStream + + +class SparkUtilsTest extends FunSpec with Matchers with OptionValues with MockitoSugar { + describe("SparkUtils") { + describe(".fileSystemAndPathForEventLogDir") { + it("returns a webhdfs filesystem + path based on spark.eventLog.dir when it is a webhdfs URL") { + val hadoopConfiguration = new Configuration(false) + val sparkConf = new SparkConf().set("spark.eventLog.dir", "webhdfs://nn1.grid.example.com:50070/logs/spark") + val sparkUtils = new SparkUtils { + override lazy val logger = mock[Logger] + override lazy val hadoopUtils = mock[HadoopUtils] + override lazy val defaultEnv = Map.empty[String, String] + } + + val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + fs.getUri.toString should be("webhdfs://nn1.grid.example.com:50070") + path should be(new Path("/logs/spark")) + } + + it("returns a webhdfs filesystem + path based on spark.eventLog.dir when it is an hdfs URL") { + val hadoopConfiguration = new Configuration(false) + val sparkConf = new SparkConf().set("spark.eventLog.dir", "hdfs://nn1.grid.example.com:9000/logs/spark") + val sparkUtils = new SparkUtils { + override lazy val logger = mock[Logger] + override lazy val hadoopUtils = mock[HadoopUtils] + override lazy val defaultEnv = Map.empty[String, String] + } + + val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + fs.getUri.toString should be("webhdfs://nn1.grid.example.com:50070") + path should be(new Path("/logs/spark")) + } + + it("returns a webhdfs filesystem + path based on dfs.nameservices and spark.eventLog.dir when the latter is a path and the dfs.nameservices is configured and available") { + val hadoopConfiguration = new Configuration(false) + hadoopConfiguration.set("dfs.nameservices", "sample") + hadoopConfiguration.set("dfs.ha.namenodes.sample", "ha1,ha2") + hadoopConfiguration.set("dfs.namenode.http-address.sample.ha1", "sample-ha1.grid.example.com:50070") + hadoopConfiguration.set("dfs.namenode.http-address.sample.ha2", "sample-ha2.grid.example.com:50070") + + val sparkConf = new SparkConf().set("spark.eventLog.dir", "/logs/spark") + + val sparkUtils = new SparkUtils { + override lazy val logger = mock[Logger] + + override lazy val hadoopUtils = HadoopUtilsTest.newFakeHadoopUtilsForNameNode( + ("sample-ha1.grid.example.com", ("sample-ha1.grid.example.com", "standby")), + ("sample-ha2.grid.example.com", ("sample-ha2.grid.example.com", "active")) + ) + + override lazy val defaultEnv = Map.empty[String, String] + } + + val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + fs.getUri.toString should be("webhdfs://sample-ha2.grid.example.com:50070") + path should be(new Path("/logs/spark")) + } + + it("returns a webhdfs filesystem + path based on dfs.nameservices and spark.eventLog.dir when the latter is a path and the dfs.nameservices is configured but unavailable") { + val hadoopConfiguration = new Configuration(false) + hadoopConfiguration.set("dfs.nameservices", "sample") + hadoopConfiguration.set("dfs.ha.namenodes.sample", "ha1,ha2") + hadoopConfiguration.set("dfs.namenode.http-address.sample.ha1", "sample-ha1.grid.example.com:50070") + hadoopConfiguration.set("dfs.namenode.http-address.sample.ha2", "sample-ha2.grid.example.com:50070") + hadoopConfiguration.set("dfs.namenode.http-address", "sample.grid.example.com:50070") + + val sparkConf = new SparkConf().set("spark.eventLog.dir", "/logs/spark") + + val sparkUtils = new SparkUtils { + override lazy val logger = mock[Logger] + + override lazy val hadoopUtils = HadoopUtilsTest.newFakeHadoopUtilsForNameNode( + ("sample-ha1.grid.example.com", ("sample-ha1.grid.example.com", "standby")), + ("sample-ha2.grid.example.com", ("sample-ha2.grid.example.com", "standby")) + ) + + override lazy val defaultEnv = Map.empty[String, String] + } + + val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + fs.getUri.toString should be("webhdfs://sample.grid.example.com:50070") + path should be(new Path("/logs/spark")) + } + + it("returns a webhdfs filesystem + path based on dfs.namenode.http-address and spark.eventLog.dir when the latter is a path and dfs.nameservices is not configured") { + val hadoopConfiguration = new Configuration(false) + hadoopConfiguration.set("dfs.namenode.http-address", "sample.grid.example.com:50070") + + val sparkConf = new SparkConf().set("spark.eventLog.dir", "/logs/spark") + + val sparkUtils = new SparkUtils { + override lazy val logger = mock[Logger] + + override lazy val hadoopUtils = HadoopUtilsTest.newFakeHadoopUtilsForNameNode( + ("sample-ha1.grid.example.com", ("sample-ha1.grid.example.com", "standby")), + ("sample-ha2.grid.example.com", ("sample-ha2.grid.example.com", "active")) + ) + + override lazy val defaultEnv = Map.empty[String, String] + } + + val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + fs.getUri.toString should be("webhdfs://sample.grid.example.com:50070") + path should be(new Path("/logs/spark")) + } + + it("throws an exception when spark.eventLog.dir is a path and no namenode is configured at all") { + val hadoopConfiguration = new Configuration(false) + + val sparkConf = new SparkConf().set("spark.eventLog.dir", "/logs/spark") + + val sparkUtils = new SparkUtils { + override lazy val logger = mock[Logger] + override lazy val hadoopUtils = mock[HadoopUtils] + override lazy val defaultEnv = Map.empty[String, String] + } + + an[Exception] should be thrownBy { sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) } + } + } + + describe(".pathAndCodecforEventLog") { + it("returns the path and codec for the event log, given the base path and app/attempt information") { + val hadoopConfiguration = new Configuration(false) + + val sparkConf = + new SparkConf() + .set("spark.eventLog.dir", "/logs/spark") + .set("spark.eventLog.compress", "true") + + val sparkUtils = SparkUtilsTest.newFakeSparkUtilsForEventLog( + new URI("webhdfs://nn1.grid.example.com:50070"), + new Path("/logs/spark"), + new Path("application_1_1.snappy"), + Array.empty[Byte] + ) + + val (fs, basePath) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + + val (path, codec) = + sparkUtils.pathAndCodecforEventLog(sparkConf: SparkConf, fs: FileSystem, basePath: Path, "application_1", Some("1")) + + path should be(new Path("webhdfs://nn1.grid.example.com:50070/logs/spark/application_1_1.snappy")) + codec.value should be(a[SnappyCompressionCodec]) + } + } + + describe(".withEventLog") { + it("loans the input stream for the event log") { + val expectedLog = + """{"Event":"SparkListenerApplicationStart","App Name":"app","App ID":"application_1","Timestamp":1,"User":"foo"}""" + + val eventLogBytes = { + val bout = new ByteArrayOutputStream() + for { + in <- resource.managed(new ByteArrayInputStream(expectedLog.getBytes("UTF-8"))) + out <- resource.managed(new SnappyOutputStream(bout)) + } { + IOUtils.copy(in, out) + } + bout.toByteArray + } + + val hadoopConfiguration = new Configuration(false) + + val sparkConf = + new SparkConf() + .set("spark.eventLog.dir", "/logs/spark") + .set("spark.eventLog.compress", "true") + + val sparkUtils = SparkUtilsTest.newFakeSparkUtilsForEventLog( + new URI("webhdfs://nn1.grid.example.com:50070"), + new Path("/logs/spark"), + new Path("application_1_1.snappy"), + eventLogBytes + ) + + val (fs, basePath) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + + val (path, codec) = + sparkUtils.pathAndCodecforEventLog(sparkConf: SparkConf, fs: FileSystem, basePath: Path, "application_1", Some("1")) + + sparkUtils.withEventLog(fs, path, codec) { in => + val bout = new ByteArrayOutputStream() + IOUtils.copy(in, bout) + + val actualLog = new String(bout.toByteArray, "UTF-8") + actualLog should be(expectedLog) + } + } + } + } +} + +object SparkUtilsTest extends MockitoSugar { + def newFakeSparkUtilsForEventLog( + fileSystemUri: URI, + basePath: Path, + filename: Path, + bytes: Array[Byte] + ): SparkUtils = new SparkUtils() { + override lazy val logger = mock[Logger] + override lazy val hadoopUtils = mock[HadoopUtils] + override lazy val defaultEnv = Map.empty[String, String] + + override def fileSystemAndPathForEventLogDir( + hadoopConfiguration: Configuration, + sparkConf: SparkConf + ): (FileSystem, Path) = { + val fs = mock[FileSystem] + val expectedPath = new Path(new Path(fileSystemUri), new Path(basePath, filename)) + val expectedFileStatus = { + val fileStatus = mock[FileStatus] + BDDMockito.given(fileStatus.getLen).willReturn(bytes.length.toLong) + fileStatus + } + BDDMockito.given(fs.getUri).willReturn(fileSystemUri) + BDDMockito.given(fs.exists(expectedPath)).willReturn(true) + BDDMockito.given(fs.getFileStatus(expectedPath)).willReturn(expectedFileStatus) + BDDMockito.given(fs.open(expectedPath)).willReturn( + new FSDataInputStream(new FakeCompressionInputStream(new ByteArrayInputStream(bytes))) + ) + (fs, basePath) + } + } + + class FakeCompressionInputStream(in: InputStream) extends CompressionInputStream(in) with PositionedReadable { + override def read(): Int = in.read() + override def read(b: Array[Byte], off: Int, len: Int): Int = in.read(b, off, len) + override def read(pos: Long, buffer: Array[Byte], off: Int, len: Int): Int = ??? + override def readFully(pos: Long, buffer: Array[Byte], off: Int, len: Int): Unit = ??? + override def readFully(pos: Long, buffer: Array[Byte]): Unit = ??? + override def resetState(): Unit = ??? + } +} diff --git a/test/org/apache/spark/TestSparkAggregatedMetrics.java b/test/org/apache/spark/TestSparkAggregatedMetrics.java new file mode 100644 index 000000000..3d8496d36 --- /dev/null +++ b/test/org/apache/spark/TestSparkAggregatedMetrics.java @@ -0,0 +1,95 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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; + +import com.linkedin.drelephant.analysis.ApplicationType; +import com.linkedin.drelephant.analysis.HadoopApplicationData; +import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData; +import com.linkedin.drelephant.spark.legacydata.MockSparkApplicationData; +import com.linkedin.drelephant.spark.legacydata.SparkExecutorData; +import org.apache.commons.io.FileUtils; +import org.junit.Assert; +import org.apache.spark.SparkMetricsAggregator; +import org.apache.spark.deploy.history.SparkDataCollection; +import org.junit.Test; + + +public class TestSparkAggregatedMetrics { + private static final String SPARK_EXECUTOR_MEMORY = "spark.executor.memory"; + + private SparkExecutorData.ExecutorInfo mockExecutorInfo(long maxMem, long memUsed, long duration) { + SparkExecutorData.ExecutorInfo executorInfo = new SparkExecutorData.ExecutorInfo(); + executorInfo.maxMem = maxMem; + executorInfo.memUsed = memUsed; + executorInfo.duration = duration; + + return executorInfo; + } + @Test + public void TestNullExecutors() { + ApplicationType appType = new ApplicationType("SPARK"); + AggregatorConfigurationData conf = + new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", appType, null); + SparkMetricsAggregator metrics = new SparkMetricsAggregator(conf); + + MockSparkApplicationData appData = new MockSparkApplicationData(); + + metrics.aggregate(appData); + + Assert.assertEquals(metrics.getResult().getResourceUsed() , 0L); + Assert.assertEquals(metrics.getResult().getResourceWasted() , 0L); + Assert.assertEquals(metrics.getResult().getTotalDelay() , 0L); + } + @Test + public void TestValidExecutorsWithNoEnvironmentData() { + ApplicationType appType = new ApplicationType("SPARK"); + AggregatorConfigurationData conf = + new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", appType, null); + SparkMetricsAggregator metrics = new SparkMetricsAggregator(conf); + + MockSparkApplicationData appData = new MockSparkApplicationData(); + appData.getExecutorData().setExecutorInfo("1", mockExecutorInfo(100 * FileUtils.ONE_MB, 60 * FileUtils.ONE_MB, 1000)); + appData.getExecutorData().setExecutorInfo("2", mockExecutorInfo(100 * FileUtils.ONE_MB, 60 * FileUtils.ONE_MB, 1000)); + + metrics.aggregate(appData); + + Assert.assertEquals(0L, metrics.getResult().getResourceUsed()); + Assert.assertEquals(20L, metrics.getResult().getResourceWasted()); + Assert.assertEquals(0L, metrics.getResult().getTotalDelay()); + } + + @Test + public void TestValidExecutorsAndValidEnvironmentData() { + ApplicationType appType = new ApplicationType("SPARK"); + AggregatorConfigurationData conf = + new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", appType, null); + SparkMetricsAggregator metrics = new SparkMetricsAggregator(conf); + + MockSparkApplicationData appData = new MockSparkApplicationData(); + appData.getExecutorData().setExecutorInfo("1", mockExecutorInfo(100 * FileUtils.ONE_MB, 60 * FileUtils.ONE_MB, 1000)); + appData.getExecutorData().setExecutorInfo("2", mockExecutorInfo(100 * FileUtils.ONE_MB, 60 * FileUtils.ONE_MB, 1000)); + + appData.getEnvironmentData().addSparkProperty(SPARK_EXECUTOR_MEMORY, "1048576000"); + + metrics.aggregate(appData); + + Assert.assertEquals(2000L, metrics.getResult().getResourceUsed()); + Assert.assertEquals(20L, metrics.getResult().getResourceWasted()); + Assert.assertEquals(0L, metrics.getResult().getTotalDelay()); + } + +} diff --git a/test/org/apache/spark/deploy/history/SparkDataCollectionTest.java b/test/org/apache/spark/deploy/history/SparkDataCollectionTest.java new file mode 100644 index 000000000..0ed76b2e5 --- /dev/null +++ b/test/org/apache/spark/deploy/history/SparkDataCollectionTest.java @@ -0,0 +1,55 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.deploy.history; + +import com.linkedin.drelephant.spark.legacydata.SparkJobProgressData; +import java.io.IOException; +import org.apache.spark.SparkConf; +import org.apache.spark.scheduler.ApplicationEventListener; +import org.apache.spark.scheduler.ReplayListenerBus; +import org.apache.spark.storage.StorageStatusListener; +import org.apache.spark.storage.StorageStatusTrackingListener; +import org.apache.spark.ui.env.EnvironmentListener; +import org.apache.spark.ui.exec.ExecutorsListener; +import org.apache.spark.ui.jobs.JobProgressListener; +import org.apache.spark.ui.storage.StorageListener; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.BufferedInputStream; +import java.io.InputStream; + +import static org.junit.Assert.assertNotNull; + +public class SparkDataCollectionTest { + + private static final String event_log_dir = "spark_event_logs/"; + + @Test + public void testCollectJobProgressData() throws IOException { + SparkDataCollection dataCollection = new SparkDataCollection(); + + InputStream in = new BufferedInputStream( + SparkDataCollectionTest.class.getClassLoader().getResourceAsStream(event_log_dir + "event_log_1")); + dataCollection.load(in, in.toString()); + in.close(); + + SparkJobProgressData jobProgressData = dataCollection.getJobProgressData(); + assertNotNull("can't get job progress data", jobProgressData); + } + +} diff --git a/test/org/apache/spark/deploy/history/SparkFsFetcherTest.scala b/test/org/apache/spark/deploy/history/SparkFsFetcherTest.scala new file mode 100644 index 000000000..50995b2a8 --- /dev/null +++ b/test/org/apache/spark/deploy/history/SparkFsFetcherTest.scala @@ -0,0 +1,122 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * 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.deploy.history + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} +import java.net.URI + +import com.linkedin.drelephant.analysis.AnalyticJob +import com.linkedin.drelephant.configurations.fetcher.{FetcherConfiguration, FetcherConfigurationData} +import com.linkedin.drelephant.util.{SparkUtils, SparkUtilsTest} +import javax.xml.parsers.DocumentBuilderFactory +import org.apache.commons.io.IOUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.spark.SparkConf +import org.scalatest.{FunSpec, Matchers} +import org.scalatest.mockito.MockitoSugar +import org.w3c.dom.Document +import org.xerial.snappy.SnappyOutputStream + +class SparkFsFetcherTest extends FunSpec with Matchers with MockitoSugar { + import SparkFsFetcherTest._ + + describe("SparkFsFetcher") { + describe("constructor") { + it("handles fetcher configurations with supplied values") { + val fetcher = newFetcher("configurations/fetcher/FetcherConfTest5.xml") + fetcher.eventLogSizeLimitMb should be(50) + } + + it("handles fetcher configurations with empty values") { + val fetcher = newFetcher("configurations/fetcher/FetcherConfTest6.xml") + fetcher.eventLogSizeLimitMb should be(SparkFSFetcher.DEFAULT_EVENT_LOG_SIZE_LIMIT_MB) + } + + it("handles fetcher configurations with missing values") { + val fetcher = newFetcher("configurations/fetcher/FetcherConfTest7.xml") + fetcher.eventLogSizeLimitMb should be(SparkFSFetcher.DEFAULT_EVENT_LOG_SIZE_LIMIT_MB) + } + } + + describe(".fetchData") { + it("returns the data collected from the Spark event log for the given analytic job") { + val eventLogBytes = { + val eventLog = + """{"Event":"SparkListenerApplicationStart","App Name":"app","App ID":"application_1","Timestamp":1,"User":"foo"}""" + val bout = new ByteArrayOutputStream() + for { + in <- resource.managed(new ByteArrayInputStream(eventLog.getBytes("UTF-8"))) + out <- resource.managed(new SnappyOutputStream(bout)) + } { + IOUtils.copy(in, out) + } + bout.toByteArray + } + + val fetcherConfigurationData = newFetcherConfigurationData("configurations/fetcher/FetcherConfTest7.xml") + val fetcher = new SparkFSFetcher(fetcherConfigurationData) { + override lazy val hadoopConfiguration = new Configuration(false) + + override lazy val sparkConf = + new SparkConf() + .set("spark.eventLog.dir", "webhdfs://nn1.grid.example.com:50070/logs/spark") + .set("spark.eventLog.compress", "true") + .set("spark.io.compression.codec", "snappy") + + override lazy val sparkUtils = SparkUtilsTest.newFakeSparkUtilsForEventLog( + new URI("webhdfs://nn1.grid.example.com:50070"), + new Path("/logs/spark"), + new Path("application_1_1.snappy"), + eventLogBytes + ) + + override protected def doAsPrivilegedAction[T](action: () => T): T = action() + } + val analyticJob = new AnalyticJob().setAppId("application_1") + + val data = fetcher.fetchData(analyticJob) + data.getAppId should be("application_1") + + val generalData = data.getGeneralData + generalData.getApplicationId should be("application_1") + generalData.getApplicationName should be("app") + generalData.getSparkUser should be("foo") + } + } + } +} + +object SparkFsFetcherTest { + def newFetcher(confResourcePath: String): SparkFSFetcher = { + val fetcherConfData = newFetcherConfigurationData(confResourcePath) + val fetcherClass = getClass.getClassLoader.loadClass(fetcherConfData.getClassName) + fetcherClass.getConstructor(classOf[FetcherConfigurationData]).newInstance(fetcherConfData).asInstanceOf[SparkFSFetcher] + } + + def newFetcherConfigurationData(confResourcePath: String): FetcherConfigurationData = { + val document = parseDocument(confResourcePath) + val fetcherConf = new FetcherConfiguration(document.getDocumentElement()) + fetcherConf.getFetchersConfigurationData().get(0) + } + + def parseDocument(resourcePath: String): Document = { + val factory = DocumentBuilderFactory.newInstance() + val builder = factory.newDocumentBuilder() + builder.parse(getClass.getClassLoader.getResourceAsStream(resourcePath)) + } +} diff --git a/test/resources/configurations/fetcher/FetcherConfTest5.xml b/test/resources/configurations/fetcher/FetcherConfTest5.xml index 2372f0828..4004c5e3e 100644 --- a/test/resources/configurations/fetcher/FetcherConfTest5.xml +++ b/test/resources/configurations/fetcher/FetcherConfTest5.xml @@ -18,8 +18,9 @@ spark - com.linkedin.drelephant.spark.fetchers.SparkFetcher + org.apache.spark.deploy.history.SparkFSFetcher + 50 diff --git a/test/resources/configurations/fetcher/FetcherConfTest6.xml b/test/resources/configurations/fetcher/FetcherConfTest6.xml new file mode 100644 index 000000000..a09588dfc --- /dev/null +++ b/test/resources/configurations/fetcher/FetcherConfTest6.xml @@ -0,0 +1,26 @@ + + + + + + spark + org.apache.spark.deploy.history.SparkFSFetcher + + + + + diff --git a/test/resources/configurations/fetcher/FetcherConfTest7.xml b/test/resources/configurations/fetcher/FetcherConfTest7.xml new file mode 100644 index 000000000..1564cb313 --- /dev/null +++ b/test/resources/configurations/fetcher/FetcherConfTest7.xml @@ -0,0 +1,25 @@ + + + + + + spark + org.apache.spark.deploy.history.SparkFSFetcher + + + + diff --git a/test/resources/configurations/fetcher/FetcherConfTest8.xml b/test/resources/configurations/fetcher/FetcherConfTest8.xml new file mode 100644 index 000000000..00fe9c055 --- /dev/null +++ b/test/resources/configurations/fetcher/FetcherConfTest8.xml @@ -0,0 +1,26 @@ + + + + + + spark + org.apache.spark.deploy.history.SparkFSFetcher + + sample-ha3.grid.example.com:50070,sample-ha4.grid.example.com:50070 + + + diff --git a/test/resources/core-site.xml b/test/resources/core-site.xml index 403589589..7c26750fb 100644 --- a/test/resources/core-site.xml +++ b/test/resources/core-site.xml @@ -41,13 +41,16 @@ dfs.namenode.http-address.sample.ha1 - sample-ha1.grid.company.com:50070 + sample-ha1.grid.example.com:50070 dfs.namenode.http-address.sample.ha2 - sample-ha2.grid.company.com:50070 + sample-ha2.grid.example.com:50070 + + dfs.namenode.http-address + sample.grid.example.com:50070 + - diff --git a/test/resources/spark_event_logs/event_log_1 b/test/resources/spark_event_logs/event_log_1 new file mode 100644 index 000000000..0ed97efaf --- /dev/null +++ b/test/resources/spark_event_logs/event_log_1 @@ -0,0 +1,32 @@ +{"Event":"SparkListenerLogStart","Spark Version":"1.5.1"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"10.20.0.71","Port":58838},"Maximum Memory":1111794647,"Timestamp":1458126388757} +{"Event":"SparkListenerApplicationStart","App Name":"PythonPi","App ID":"application_1457600942802_0093","Timestamp":1458126354336,"User":"hdfs"} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1458126390170,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at pi.py:39","Number of Tasks":10,"RDD Info":[{"RDD ID":1,"Name":"PythonRDD","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Use ExternalBlockStore":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"ExternalBlockStore Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Use ExternalBlockStore":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"ExternalBlockStore Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"","Accumulables":[]}],"Stage IDs":[0],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"collect\"}","callSite.short":"reduce at pi.py:39"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at pi.py:39","Number of Tasks":10,"RDD Info":[{"RDD ID":1,"Name":"PythonRDD","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Use ExternalBlockStore":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"ExternalBlockStore Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Use ExternalBlockStore":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"ExternalBlockStore Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"","Submission Time":1458126390256,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"collect\"}","callSite.short":"reduce at pi.py:39"}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1458126397624,"Executor ID":"2","Executor Info":{"Host":".hello.com","Total Cores":2,"Log Urls":{"stdout":"http://hello.com:8042/node/containerlogs/container_e38_1457600942802_0093_01_000003/hdfs/stdout?start=-4096","stderr":"http://hello.com:8042/node/containerlogs/container_e38_1457600942802_0093_01_000003/hdfs/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":".hello.com","Port":36478},"Maximum Memory":2223023063,"Timestamp":1458126398028} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1458126398712,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1458126398726,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1458126398962,"Executor ID":"1","Executor Info":{"Host":".hello.com","Total Cores":2,"Log Urls":{"stdout":"http://.hello.com:8042/node/containerlogs/container_e38_1457600942802_0093_01_000002/hdfs/stdout?start=-4096","stderr":"http://.hello.com:8042/node/containerlogs/container_e38_1457600942802_0093_01_000002/hdfs/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1458126398970,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1458126398973,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":".hello.com","Port":38464},"Maximum Memory":2223023063,"Timestamp":1458126399357} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1458126403532,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1458126398726,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126403558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"hello.com","Executor Deserialize Time":1091,"Executor Run Time":408,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1458126403563,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1458126398712,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126403565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"hello.com","Executor Deserialize Time":2605,"Executor Run Time":411,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1458126404784,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1458126398973,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126404787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":".hello.com","Executor Deserialize Time":1375,"Executor Run Time":473,"Result Size":1018,"JVM GC Time":48,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1458126404791,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1458126398970,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126404793,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":".hello.com","Executor Deserialize Time":1385,"Executor Run Time":473,"Result Size":1018,"JVM GC Time":48,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1458126405270,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1458126403532,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126405273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"hello.com","Executor Deserialize Time":16,"Executor Run Time":115,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1458126406523,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1458126404784,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126406526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":".hello.com","Executor Deserialize Time":14,"Executor Run Time":115,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1458126403563,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126406809,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"hello.com","Executor Deserialize Time":14,"Executor Run Time":125,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1458126404791,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126408068,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":".hello.com","Executor Deserialize Time":15,"Executor Run Time":118,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1458126405270,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126408323,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"hello.com","Executor Deserialize Time":13,"Executor Run Time":116,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1458126406523,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126409598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":".hello.com","Executor Deserialize Time":12,"Executor Run Time":116,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at pi.py:39","Number of Tasks":10,"RDD Info":[{"RDD ID":1,"Name":"PythonRDD","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Use ExternalBlockStore":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"ExternalBlockStore Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Use ExternalBlockStore":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"ExternalBlockStore Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"","Submission Time":1458126390256,"Completion Time":1458126409599,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1458126409602,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1458126409609} From 7c74161ff4b9b8e838bec6a07813ad072e54e063 Mon Sep 17 00:00:00 2001 From: shankar Date: Tue, 4 Apr 2017 11:37:56 +0530 Subject: [PATCH 02/11] remove old heuristics and aggregators brought back unnecessarily --- .../linkedin/drelephant/ElephantContext.java | 1 - .../BestPropertiesConventionHeuristic.java | 146 ------------ .../EventLogLimitHeuristic.java | 61 ----- .../ExecutorLoadHeuristic.java | 217 ------------------ .../legacyheuristics/JobRuntimeHeuristic.java | 135 ----------- .../MemoryLimitHeuristic.java | 207 ----------------- .../StageRuntimeHeuristic.java | 160 ------------- .../apache/spark/SparkMetricsAggregator.java | 89 ------- ...BestPropertiesConventionHeuristicTest.java | 93 -------- .../JobRuntimeHeuristicTest.java | 30 --- .../MemoryLimitHeuristicTest.java | 138 ----------- .../spark/TestSparkAggregatedMetrics.java | 95 -------- 12 files changed, 1372 deletions(-) delete mode 100644 app/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristic.java delete mode 100644 app/com/linkedin/drelephant/spark/legacyheuristics/EventLogLimitHeuristic.java delete mode 100644 app/com/linkedin/drelephant/spark/legacyheuristics/ExecutorLoadHeuristic.java delete mode 100644 app/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristic.java delete mode 100644 app/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristic.java delete mode 100644 app/com/linkedin/drelephant/spark/legacyheuristics/StageRuntimeHeuristic.java delete mode 100644 app/org/apache/spark/SparkMetricsAggregator.java delete mode 100644 test/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristicTest.java delete mode 100644 test/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristicTest.java delete mode 100644 test/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristicTest.java delete mode 100644 test/org/apache/spark/TestSparkAggregatedMetrics.java diff --git a/app/com/linkedin/drelephant/ElephantContext.java b/app/com/linkedin/drelephant/ElephantContext.java index 225f6b53e..fe6ff3026 100644 --- a/app/com/linkedin/drelephant/ElephantContext.java +++ b/app/com/linkedin/drelephant/ElephantContext.java @@ -47,7 +47,6 @@ import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.log4j.Logger; -import org.apache.spark.SparkMetricsAggregator; import org.w3c.dom.Document; import play.api.templates.Html; diff --git a/app/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristic.java b/app/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristic.java deleted file mode 100644 index d2de28f63..000000000 --- a/app/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristic.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * 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.linkedin.drelephant.spark.legacyheuristics; - -import com.linkedin.drelephant.analysis.Heuristic; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; -import com.linkedin.drelephant.spark.legacydata.SparkEnvironmentData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import com.linkedin.drelephant.util.MemoryFormatUtils; -import com.linkedin.drelephant.util.Utils; -import java.util.Arrays; -import java.util.Map; -import org.apache.log4j.Logger; - - -/** - * This heuristic rule check some of the most commonly set spark properties and make sure the user is following - * a best convention of them. - */ -public class BestPropertiesConventionHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(BestPropertiesConventionHeuristic.class); - - public static final String SPARK_SERIALIZER = "spark.serializer"; - public static final String SPARK_DRIVER_MEMORY = "spark.driver.memory"; - public static final String SPARK_SHUFFLE_MANAGER = "spark.shuffle.manager"; - public static final String SPARK_EXECUTOR_CORES = "spark.executor.cores"; - - // Severity parameters. - private static final String NUM_CORE_SEVERITY = "num_core_severity"; - private static final String DRIVER_MEM_SEVERITY = "driver_memory_severity_in_gb"; - - // Default value of parameters - private double[] numCoreLimit= {2d}; // Spark Executor Cores - private double[] driverMemLimits = {4d, 4d, 8d, 8d}; // Spark Driver Memory - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confNumCoreLimit = Utils.getParam(paramMap.get(NUM_CORE_SEVERITY), numCoreLimit.length); - if (confNumCoreLimit != null) { - numCoreLimit = confNumCoreLimit; - } - logger.info(heuristicName + " will use " + NUM_CORE_SEVERITY + " with the following threshold settings: " - + Arrays.toString(numCoreLimit)); - - double[] confDriverMemLimits = Utils.getParam(paramMap.get(DRIVER_MEM_SEVERITY), driverMemLimits.length); - if (confDriverMemLimits != null) { - driverMemLimits = confDriverMemLimits; - } - logger.info(heuristicName + " will use " + DRIVER_MEM_SEVERITY + " with the following threshold settings: " - + Arrays.toString(driverMemLimits)); - for (int i = 0; i < driverMemLimits.length; i++) { - driverMemLimits[i] = (double) MemoryFormatUtils.stringToBytes(Double.toString(driverMemLimits[i]) + "G"); - } - } - - public BestPropertiesConventionHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); - } - - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } - - @Override - public HeuristicResult apply(SparkApplicationData data) { - SparkEnvironmentData env = data.getEnvironmentData(); - String sparkSerializer = env.getSparkProperty(SPARK_SERIALIZER); - String sparkDriverMemory = env.getSparkProperty(SPARK_DRIVER_MEMORY); - String sparkShuffleManager = env.getSparkProperty(SPARK_SHUFFLE_MANAGER); - String sparkExecutorCores = env.getSparkProperty(SPARK_EXECUTOR_CORES); - int coreNum = sparkExecutorCores == null ? 1 : Integer.parseInt(sparkExecutorCores); - - Severity kryoSeverity = - binarySeverity("org.apache.spark.serializer.KryoSerializer", sparkSerializer, true, Severity.MODERATE); - Severity driverMemSeverity = getDriverMemorySeverity(MemoryFormatUtils.stringToBytes(sparkDriverMemory)); - Severity sortSeverity = binarySeverity("sort", sparkShuffleManager, true, Severity.MODERATE); - Severity executorCoreSeverity = getCoreNumSeverity(coreNum); - - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), Severity.max(kryoSeverity, driverMemSeverity, sortSeverity, - executorCoreSeverity), 0); - - result.addResultDetail(SPARK_SERIALIZER, propertyToString(sparkSerializer)); - result.addResultDetail(SPARK_DRIVER_MEMORY, propertyToString(sparkDriverMemory)); - result.addResultDetail(SPARK_SHUFFLE_MANAGER, propertyToString(sparkShuffleManager)); - result.addResultDetail(SPARK_EXECUTOR_CORES, propertyToString(sparkExecutorCores)); - - return result; - } - - private Severity getCoreNumSeverity(int cores) { - if (cores > numCoreLimit[0]) { - return Severity.CRITICAL; - } else { - return Severity.NONE; - } - } - - private Severity getDriverMemorySeverity(long mem) { - return Severity.getSeverityAscending( - mem, driverMemLimits[0], driverMemLimits[1], driverMemLimits[2], driverMemLimits[3]); - } - - private static Severity binarySeverity(String expectedValue, String actualValue, boolean ignoreNull, - Severity severity) { - if (actualValue == null) { - if (ignoreNull) { - return Severity.NONE; - } else { - return severity; - } - } - - if (actualValue.equals(expectedValue)) { - return Severity.NONE; - } else { - return severity; - } - } - - private static String propertyToString(String val) { - return val == null ? "Not presented. Using default" : val; - } -} diff --git a/app/com/linkedin/drelephant/spark/legacyheuristics/EventLogLimitHeuristic.java b/app/com/linkedin/drelephant/spark/legacyheuristics/EventLogLimitHeuristic.java deleted file mode 100644 index b61fdbae7..000000000 --- a/app/com/linkedin/drelephant/spark/legacyheuristics/EventLogLimitHeuristic.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * 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.linkedin.drelephant.spark.legacyheuristics; - -import com.linkedin.drelephant.analysis.Heuristic; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; - - -/** - * This is a safeguard heuristic rule that makes sure if a log size passes the limit, we do not automatically - * approve it. - */ -public class EventLogLimitHeuristic implements Heuristic { - private HeuristicConfigurationData _heuristicConfData; - - public EventLogLimitHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - } - - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } - - @Override - public HeuristicResult apply(SparkApplicationData data) { - Severity severity = getSeverity(data); - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), severity, 0); - if (severity == Severity.CRITICAL) { - result.addResultDetail("Large Log File", "Spark job's event log passes the limit. No actual log data is fetched." - + " All other heuristic rules will not make sense.", null); - } - return result; - } - - private Severity getSeverity(SparkApplicationData data) { - if (data.isThrottled()) { - return Severity.CRITICAL; - } else { - return Severity.NONE; - } - } -} diff --git a/app/com/linkedin/drelephant/spark/legacyheuristics/ExecutorLoadHeuristic.java b/app/com/linkedin/drelephant/spark/legacyheuristics/ExecutorLoadHeuristic.java deleted file mode 100644 index c7eb887dc..000000000 --- a/app/com/linkedin/drelephant/spark/legacyheuristics/ExecutorLoadHeuristic.java +++ /dev/null @@ -1,217 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * 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.linkedin.drelephant.spark.legacyheuristics; - -import com.linkedin.drelephant.analysis.Heuristic; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.math.Statistics; -import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; -import com.linkedin.drelephant.spark.legacydata.SparkExecutorData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import com.linkedin.drelephant.util.MemoryFormatUtils; -import com.linkedin.drelephant.util.Utils; -import java.util.Arrays; -import java.util.Map; -import java.util.Set; -import org.apache.log4j.Logger; - -import static com.linkedin.drelephant.spark.legacydata.SparkExecutorData.EXECUTOR_DRIVER_NAME; - - -/** - * This heuristic rule observes load details of executors - */ -public class ExecutorLoadHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(ExecutorLoadHeuristic.class); - private static final long MEMORY_OBSERVATION_THRESHOLD = MemoryFormatUtils.stringToBytes("1 MB"); - - // Severity parameters. - private static final String LOOSER_METRIC_DEV_SEVERITY = "looser_metric_deviation_severity"; - private static final String METRIC_DEV_SEVERITY = "metric_deviation_severity"; - - // Default value of parameters - private double[] looserMetDevLimits = {0.8d, 1d, 1.2d, 1.4d}; // Max deviation from avg. - private double[] metDevLimits = {0.4d, 0.6d, 0.8d, 1.0d}; - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confLooserMetDevLimits = Utils.getParam(paramMap.get(LOOSER_METRIC_DEV_SEVERITY), - looserMetDevLimits.length); - if (confLooserMetDevLimits != null) { - looserMetDevLimits = confLooserMetDevLimits; - } - logger.info(heuristicName + " will use " + LOOSER_METRIC_DEV_SEVERITY + " with the following threshold settings: " - + Arrays.toString(looserMetDevLimits)); - - double[] confMetDevLimits = Utils.getParam(paramMap.get(METRIC_DEV_SEVERITY), metDevLimits.length); - if (confMetDevLimits != null) { - metDevLimits = confMetDevLimits; - } - logger.info(heuristicName + " will use " + METRIC_DEV_SEVERITY + " with the following threshold settings: " - + Arrays.toString(metDevLimits)); - } - - public ExecutorLoadHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); - } - - private class ValueObserver { - private final long[] _values; - private Long _min; - private Long _max; - private Long _avg; - - public ValueObserver(int size) { - _values = new long[size]; - } - - public void set(int index, long value) { - _values[index] = value; - if (_min == null) { - _min = value; - } else { - _min = Math.min(value, _min); - } - - if (_max == null) { - _max = value; - } else { - _max = Math.max(value, _max); - } - } - - public long getMin() { - return _min == null ? 0L : _min; - } - - public long getMax() { - return _max == null ? 0L : _max; - } - - public long getAvg() { - if (_avg == null) { - if (_values == null) { - return 0L; - } - _avg = Statistics.average(_values); - } - return _avg; - } - - /** - * Max(|max-avg|, |min-avg|) / avg - * - * @return - */ - public double getDeviationFactor() { - long avg = getAvg(); - if (avg == 0) { - return 0d; - } - long diff = Math.max(getMax() - avg, avg - getMin()); - return diff * 1.0d / avg; - } - } - - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } - - @Override - public HeuristicResult apply(SparkApplicationData data) { - SparkExecutorData executorData = data.getExecutorData(); - Set executors = executorData.getExecutors(); - - int numNonDriverExe = executors.size(); - if (executors.contains(EXECUTOR_DRIVER_NAME)) { - numNonDriverExe -= 1; - } - ValueObserver peakMems = new ValueObserver(numNonDriverExe); - ValueObserver durations = new ValueObserver(numNonDriverExe); - ValueObserver inputBytes = new ValueObserver(numNonDriverExe); - ValueObserver outputBytes = new ValueObserver(numNonDriverExe); - - int i = 0; - for (String exeId : executors) { - if (!exeId.equals(EXECUTOR_DRIVER_NAME)) { - SparkExecutorData.ExecutorInfo info = executorData.getExecutorInfo(exeId); - - /* Ignore the memory variation and consider it as 0 if it is too small - * The deviation of memory usage in KB level is too fluctuating to track. - */ - if (info.memUsed < MEMORY_OBSERVATION_THRESHOLD) { - peakMems.set(i, 0L); - } else { - peakMems.set(i, info.memUsed); - } - - durations.set(i, info.duration); - inputBytes.set(i, info.inputBytes); - outputBytes.set(i, info.outputBytes); - i += 1; - } - } - - Severity severity = Severity.max(getLooserMetricDeviationSeverity(peakMems), getMetricDeviationSeverity(durations), - getMetricDeviationSeverity(inputBytes), getLooserMetricDeviationSeverity(outputBytes)); - - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), severity, 0); - - result.addResultDetail("Average peak storage memory", - String.format("%s (%s~%s)", MemoryFormatUtils.bytesToString(peakMems.getAvg()), - MemoryFormatUtils.bytesToString(peakMems.getMin()), MemoryFormatUtils.bytesToString(peakMems.getMax()))); - result.addResultDetail("Average runtime", - String.format("%s (%s~%s)", Statistics.readableTimespan(durations.getAvg()), - Statistics.readableTimespan(durations.getMin()), Statistics.readableTimespan(durations.getMax()))); - result.addResultDetail("Average input size", - String.format("%s (%s~%s)", MemoryFormatUtils.bytesToString(inputBytes.getAvg()), - MemoryFormatUtils.bytesToString(inputBytes.getMin()), MemoryFormatUtils.bytesToString(inputBytes.getMax()))); - result.addResultDetail("Average output size", - String.format("%s (%s~%s)", MemoryFormatUtils.bytesToString(outputBytes.getAvg()), - MemoryFormatUtils.bytesToString(outputBytes.getMin()), MemoryFormatUtils.bytesToString(outputBytes.getMax()))); - - return result; - } - - /** - * Some metrics by nature could deviate a bit more than other metrics. This method basically allows some metrics - * to have looser severity thresholds. - * - * @param ob - * @return the corresponding severity - */ - private Severity getLooserMetricDeviationSeverity(ValueObserver ob) { - double diffFactor = ob.getDeviationFactor(); - return Severity.getSeverityAscending( - diffFactor, looserMetDevLimits[0], looserMetDevLimits[1], looserMetDevLimits[2], looserMetDevLimits[3]); - } - - private Severity getMetricDeviationSeverity(ValueObserver ob) { - double diffFactor = ob.getDeviationFactor(); - return Severity.getSeverityAscending( - diffFactor, metDevLimits[0], metDevLimits[1], metDevLimits[2], metDevLimits[3]); - } - -} diff --git a/app/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristic.java b/app/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristic.java deleted file mode 100644 index 8daad4d1f..000000000 --- a/app/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristic.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * 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.linkedin.drelephant.spark.legacyheuristics; - -import com.linkedin.drelephant.analysis.Heuristic; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; -import com.linkedin.drelephant.spark.legacydata.SparkJobProgressData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import com.linkedin.drelephant.util.Utils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.commons.lang.StringUtils; -import org.apache.log4j.Logger; - - -/** - * This heuristic rule observes the runtime characteristics of the spark application run. - */ -public class JobRuntimeHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(JobRuntimeHeuristic.class); - - // Severity parameters. - private static final String AVG_JOB_FAILURE_SEVERITY = "avg_job_failure_rate_severity"; - private static final String SINGLE_JOB_FAILURE_SEVERITY = "single_job_failure_rate_severity"; - - // Default value of parameters - private double[] avgJobFailureLimits = {0.1d, 0.3d, 0.5d, 0.5d}; // The avg job failure rate - private double[] jobFailureLimits = {0.0d, 0.3d, 0.5d, 0.5d}; - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confAvgJobFailureLimits = Utils.getParam(paramMap.get(AVG_JOB_FAILURE_SEVERITY), - avgJobFailureLimits.length); - if (confAvgJobFailureLimits != null) { - avgJobFailureLimits = confAvgJobFailureLimits; - } - logger.info(heuristicName + " will use " + AVG_JOB_FAILURE_SEVERITY + " with the following threshold settings: " - + Arrays.toString(avgJobFailureLimits)); - - double[] confJobFailureLimits = Utils.getParam(paramMap.get(SINGLE_JOB_FAILURE_SEVERITY), - jobFailureLimits.length); - if (confJobFailureLimits != null) { - jobFailureLimits = confJobFailureLimits; - } - logger.info(heuristicName + " will use " + SINGLE_JOB_FAILURE_SEVERITY + " with the following threshold settings: " - + Arrays.toString(jobFailureLimits)); - } - - public JobRuntimeHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); - } - - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } - - @Override - public HeuristicResult apply(SparkApplicationData data) { - SparkJobProgressData jobProgressData = data.getJobProgressData(); - Severity endSeverity = Severity.NONE; - - Set completedJobs = jobProgressData.getCompletedJobs(); - Set failedJobs = jobProgressData.getFailedJobs(); - - // Average job failure rate - double avgJobFailureRate = jobProgressData.getJobFailureRate(); - Severity jobFailureRateSeverity = getAvgJobFailureRateSeverity(avgJobFailureRate); - endSeverity = Severity.max(endSeverity, jobFailureRateSeverity); - - // For each completed individual job - List highFailureRateJobs = new ArrayList(); - for (int jobId : completedJobs) { - SparkJobProgressData.JobInfo job = jobProgressData.getJobInfo(jobId); - double jobFailureRate = job.getFailureRate(); - Severity severity = getSingleJobFailureRateSeverity(jobFailureRate); - if (severity.getValue() > Severity.MODERATE.getValue()) { - highFailureRateJobs.add( - jobProgressData.getJobDescription(jobId) + " (task failure rate:" + String.format("%1.3f", jobFailureRate) - + ")"); - } - endSeverity = Severity.max(endSeverity, severity); - } - - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), endSeverity, 0); - - result.addResultDetail("Spark completed jobs number", String.valueOf(completedJobs.size())); - result.addResultDetail("Spark failed jobs number", String.valueOf(failedJobs.size())); - result.addResultDetail("Spark failed jobs list", getJobsAsString(jobProgressData.getFailedJobDescriptions())); - result.addResultDetail("Spark average job failure rate", String.format("%.3f", avgJobFailureRate)); - result.addResultDetail("Spark jobs with high task failure rate", getJobsAsString(highFailureRateJobs)); - - return result; - } - - private Severity getAvgJobFailureRateSeverity(double rate) { - return Severity.getSeverityAscending( - rate, avgJobFailureLimits[0], avgJobFailureLimits[1], avgJobFailureLimits[2], avgJobFailureLimits[3]); - } - - private Severity getSingleJobFailureRateSeverity(double rate) { - return Severity.getSeverityAscending( - rate, jobFailureLimits[0], jobFailureLimits[1], jobFailureLimits[2], jobFailureLimits[3]); - } - - private static String getJobsAsString(Collection names) { - return StringUtils.join(names, "\n"); - } -} diff --git a/app/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristic.java b/app/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristic.java deleted file mode 100644 index 66f511204..000000000 --- a/app/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristic.java +++ /dev/null @@ -1,207 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * 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.linkedin.drelephant.spark.legacyheuristics; - -import com.linkedin.drelephant.analysis.Heuristic; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; -import com.linkedin.drelephant.spark.legacydata.SparkEnvironmentData; -import com.linkedin.drelephant.spark.legacydata.SparkExecutorData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import com.linkedin.drelephant.util.MemoryFormatUtils; -import com.linkedin.drelephant.util.Utils; -import java.util.Arrays; -import java.util.Map; -import org.apache.log4j.Logger; -import org.apache.spark.SparkConf; - -import static com.linkedin.drelephant.spark.legacydata.SparkExecutorData.EXECUTOR_DRIVER_NAME; - - -/** - * This heuristic checks for memory consumption. - */ -public class MemoryLimitHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(MemoryLimitHeuristic.class); - - public static final String SPARK_EXECUTOR_MEMORY = "spark.executor.memory"; - public static final String SPARK_DRIVER_MEMORY = "spark.driver.memory"; - public static final String SPARK_EXECUTOR_INSTANCES = "spark.executor.instances"; - - public static final String SPARK_STORAGE_MEMORY_FRACTION = "spark.storage.memoryFraction"; - public static final double DEFAULT_SPARK_STORAGE_MEMORY_FRACTION = 0.6d; - - // Severity parameters. - private static final String MEM_UTILIZATION_SEVERITY = "mem_util_severity"; - private static final String TOTAL_MEM_SEVERITY = "total_mem_severity_in_tb"; - - // Default value of parameters - private double[] memUtilLimits = {0.8d, 0.6d, 0.4d, 0.2d}; - private double[] totalMemLimits = {0.5d, 1d, 1.5d, 2d}; // Peak Memory / Total Storage Memory - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - if(paramMap.get(MEM_UTILIZATION_SEVERITY) != null) { - double[] confMemUtilLimits = Utils.getParam(paramMap.get(MEM_UTILIZATION_SEVERITY), memUtilLimits.length); - if (confMemUtilLimits != null) { - memUtilLimits = confMemUtilLimits; - } - } - logger.info(heuristicName + " will use " + MEM_UTILIZATION_SEVERITY + " with the following threshold settings: " - + Arrays.toString(memUtilLimits)); - - if(paramMap.get(TOTAL_MEM_SEVERITY) != null) { - double[] confTotalMemLimits = Utils.getParam(paramMap.get(TOTAL_MEM_SEVERITY), totalMemLimits.length); - if (confTotalMemLimits != null) { - totalMemLimits = confTotalMemLimits; - } - } - logger.info(heuristicName + " will use " + TOTAL_MEM_SEVERITY + " with the following threshold settings: " - + Arrays.toString(totalMemLimits)); - for (int i = 0; i < totalMemLimits.length; i++) { - totalMemLimits[i] = MemoryFormatUtils.stringToBytes(totalMemLimits[i] + "T"); - } - } - - public MemoryLimitHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); - } - - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } - - @Override - public HeuristicResult apply(SparkApplicationData data) { - - int executorNum = Integer.parseInt(data.getEnvironmentData().getSparkProperty(SPARK_EXECUTOR_INSTANCES, "0")); - long perExecutorMem = - MemoryFormatUtils.stringToBytes(data.getEnvironmentData().getSparkProperty(SPARK_EXECUTOR_MEMORY, "0")); - - long totalExecutorMem = executorNum * perExecutorMem; - - long totalStorageMem = getTotalStorageMem(data); - long totalDriverMem = getTotalDriverMem(data); - long peakMem = getStoragePeakMemory(data); - - Severity totalMemorySeverity = getTotalMemorySeverity(totalExecutorMem); - Severity memoryUtilizationServerity = getMemoryUtilizationSeverity(peakMem, totalStorageMem); - - HeuristicResult result = - new HeuristicResult(_heuristicConfData.getClassName(), _heuristicConfData.getHeuristicName(), - Severity.max(totalMemorySeverity, memoryUtilizationServerity), 0); - - result.addResultDetail("Total executor memory allocated", String - .format("%s (%s x %s)", MemoryFormatUtils.bytesToString(totalExecutorMem), - MemoryFormatUtils.bytesToString(perExecutorMem), executorNum)); - result.addResultDetail("Total driver memory allocated", MemoryFormatUtils.bytesToString(totalDriverMem)); - result.addResultDetail("Total memory allocated for storage", MemoryFormatUtils.bytesToString(totalStorageMem)); - result.addResultDetail("Total memory used at peak", MemoryFormatUtils.bytesToString(peakMem)); - result.addResultDetail("Memory utilization rate", String.format("%1.3f", peakMem * 1.0 / totalStorageMem)); - return result; - } - - /** - * Get the total driver memory - * - * @param data The spark application data that contains the information - * @return The memory in bytes - */ - private static long getTotalDriverMem(SparkApplicationData data) { - long bytes = MemoryFormatUtils.stringToBytes(data.getEnvironmentData().getSparkProperty(SPARK_DRIVER_MEMORY)); - // spark.driver.memory might not be present, in which case we would infer it from the executor data - if (bytes == 0L) { - SparkExecutorData.ExecutorInfo info = data.getExecutorData().getExecutorInfo(EXECUTOR_DRIVER_NAME); - if (info == null) { - logger.error("Application id [" + data.getGeneralData().getApplicationId() - + "] does not contain driver memory configuration info and also does not contain executor driver info." - + " Unable to detect is driver memory usage."); - return 0L; - } - // This maxmium memory only counts in memory for storage - bytes = (long) (info.maxMem / getStorageMemoryFraction(data.getEnvironmentData())); - } - - return bytes; - } - - /** - * Get the storage memory fraction ratio used for storage - * - * @param data The spark environment data - * @return the memory fraction - */ - private static double getStorageMemoryFraction(SparkEnvironmentData data) { - String ratio = data.getSparkProperty(SPARK_STORAGE_MEMORY_FRACTION); - if (ratio == null) { - ratio = new SparkConf().get(SPARK_STORAGE_MEMORY_FRACTION, String.valueOf(DEFAULT_SPARK_STORAGE_MEMORY_FRACTION)); - } - return Double.parseDouble(ratio); - } - - /** - * Get the peak storage memory used during all running time of the spark application - * - * @param data The spark application data that contains the information - * @return The memory in bytes - */ - private static long getStoragePeakMemory(SparkApplicationData data) { - SparkExecutorData executorData = data.getExecutorData(); - long mem = 0L; - for (String id : executorData.getExecutors()) { - mem += executorData.getExecutorInfo(id).memUsed; - } - return mem; - } - - /** - * Get the total memory allocated for storage - * - * @param data The spark application data that contains the information - * @return The memory in bytes - */ - private static long getTotalStorageMem(SparkApplicationData data) { - SparkExecutorData executorData = data.getExecutorData(); - long totalStorageMem = 0L; - for (String id : executorData.getExecutors()) { - totalStorageMem += executorData.getExecutorInfo(id).maxMem; - } - return totalStorageMem; - } - - public Severity getTotalMemorySeverity(long memory) { - return Severity.getSeverityAscending(memory, totalMemLimits[0], totalMemLimits[1], totalMemLimits[2], - totalMemLimits[3]); - } - - private Severity getMemoryUtilizationSeverity(long peakMemory, long totalStorageMemory) { - double fraction = peakMemory * 1.0 / totalStorageMemory; - if (totalStorageMemory < MemoryFormatUtils.stringToBytes("500 GB")) { - return Severity.NONE; - } else { - return Severity.getSeverityDescending( - fraction, memUtilLimits[0], memUtilLimits[1], memUtilLimits[2], memUtilLimits[3]); - } - } -} diff --git a/app/com/linkedin/drelephant/spark/legacyheuristics/StageRuntimeHeuristic.java b/app/com/linkedin/drelephant/spark/legacyheuristics/StageRuntimeHeuristic.java deleted file mode 100644 index d564fd4c5..000000000 --- a/app/com/linkedin/drelephant/spark/legacyheuristics/StageRuntimeHeuristic.java +++ /dev/null @@ -1,160 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * 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.linkedin.drelephant.spark.legacyheuristics; - -import com.linkedin.drelephant.analysis.Heuristic; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.math.Statistics; -import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; -import com.linkedin.drelephant.spark.legacydata.SparkJobProgressData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import com.linkedin.drelephant.util.Utils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.commons.lang.StringUtils; -import org.apache.log4j.Logger; - - -/** - * Spark heuristic that detects stage runtime anomalies. - * - */ -public class StageRuntimeHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(StageRuntimeHeuristic.class); - - // Severity parameters - private static final String STAGE_FAILURE_SEVERITY = "stage_failure_rate_severity"; - private static final String SINGLE_STAGE_FAILURE_SEVERITY = "single_stage_tasks_failure_rate_severity"; - private static final String STAGE_RUNTIME_SEVERITY = "stage_runtime_severity_in_min"; - - // Default value of parameters - private double[] stageFailRateLimits = {0.3d, 0.3d, 0.5d, 0.5d}; - private double[] singleStageFailLimits = {0.0d, 0.3d, 0.5d, 0.5d}; - private double[] stageRuntimeLimits = {15, 30, 60, 60}; - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confStageFailRateLimits = Utils.getParam(paramMap.get(STAGE_FAILURE_SEVERITY), - stageFailRateLimits.length); - if (confStageFailRateLimits != null) { - stageFailRateLimits = confStageFailRateLimits; - } - logger.info(heuristicName + " will use " + STAGE_FAILURE_SEVERITY + " with the following threshold settings: " - + Arrays.toString(stageFailRateLimits)); - - double[] confSingleFailLimits = Utils.getParam(paramMap.get(SINGLE_STAGE_FAILURE_SEVERITY), - singleStageFailLimits.length); - if (confSingleFailLimits != null) { - singleStageFailLimits = confSingleFailLimits; - } - logger.info(heuristicName + " will use " + SINGLE_STAGE_FAILURE_SEVERITY + " with the following threshold" - + " settings: " + Arrays.toString(singleStageFailLimits)); - - double[] confStageRuntimeLimits = Utils.getParam(paramMap.get(STAGE_RUNTIME_SEVERITY), stageRuntimeLimits.length); - if (confStageRuntimeLimits != null) { - stageRuntimeLimits = confStageRuntimeLimits; - } - logger.info(heuristicName + " will use " + STAGE_RUNTIME_SEVERITY + " with the following threshold settings: " - + Arrays.toString(stageRuntimeLimits)); - for (int i = 0; i < stageRuntimeLimits.length; i++) { - stageRuntimeLimits[i] = stageRuntimeLimits[i] * Statistics.MINUTE_IN_MS; - } - } - - public StageRuntimeHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); - } - - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } - - @Override - public HeuristicResult apply(SparkApplicationData data) { - SparkJobProgressData jobProgressData = data.getJobProgressData(); - Severity endSeverity = Severity.NONE; - - Set completedStages = jobProgressData.getCompletedStages(); - Set failedStages = jobProgressData.getFailedStages(); - - // Average stage failure rate - double avgStageFailureRate = jobProgressData.getStageFailureRate(); - endSeverity = Severity.max(endSeverity, getStageFailureRateSeverity(avgStageFailureRate)); - - // For each completed stage, the task failure rate - List problematicStages = new ArrayList(); - - for (SparkJobProgressData.StageAttemptId id : completedStages) { - SparkJobProgressData.StageInfo info = jobProgressData.getStageInfo(id.stageId, id.attemptId); - double stageTasksFailureRate = info.getFailureRate(); - Severity tasksFailureRateSeverity = getSingleStageTasksFailureRate(stageTasksFailureRate); - - if (tasksFailureRateSeverity.getValue() > Severity.MODERATE.getValue()) { - problematicStages.add(String.format("%s (task failure rate: %1.3f)", id, stageTasksFailureRate)); - } - - long duration = info.duration; - Severity runtimeSeverity = getStageRuntimeSeverity(duration); - if (runtimeSeverity.getValue() > Severity.MODERATE.getValue()) { - problematicStages - .add(String.format("%s (runtime: %s)", id, Statistics.readableTimespan(duration))); - } - - endSeverity = Severity.max(endSeverity, tasksFailureRateSeverity, runtimeSeverity); - } - - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), endSeverity, 0); - - result.addResultDetail("Spark stage completed", String.valueOf(completedStages.size())); - result.addResultDetail("Spark stage failed", String.valueOf(failedStages.size())); - result.addResultDetail("Spark average stage failure rate", String.format("%.3f", avgStageFailureRate)); - result.addResultDetail("Spark problematic stages", getStagesAsString(problematicStages)); - - return result; - } - - private Severity getStageRuntimeSeverity(long runtime) { - return Severity.getSeverityAscending( - runtime, stageRuntimeLimits[0], stageRuntimeLimits[1], stageRuntimeLimits[2], stageRuntimeLimits[3]); - } - - private Severity getStageFailureRateSeverity(double rate) { - return Severity.getSeverityAscending( - rate, stageFailRateLimits[0], stageFailRateLimits[1], stageFailRateLimits[2], stageFailRateLimits[3]); - } - - private Severity getSingleStageTasksFailureRate(double rate) { - return Severity.getSeverityAscending( - rate, singleStageFailLimits[0], singleStageFailLimits[1], singleStageFailLimits[2], singleStageFailLimits[3]); - } - - private static String getStagesAsString(Collection names) { - return StringUtils.join(names, "\n"); - } -} diff --git a/app/org/apache/spark/SparkMetricsAggregator.java b/app/org/apache/spark/SparkMetricsAggregator.java deleted file mode 100644 index 98783977f..000000000 --- a/app/org/apache/spark/SparkMetricsAggregator.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * 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; -import com.linkedin.drelephant.math.Statistics; -import org.apache.commons.io.FileUtils; - -import com.linkedin.drelephant.analysis.HadoopApplicationData; -import com.linkedin.drelephant.analysis.HadoopMetricsAggregator; -import com.linkedin.drelephant.analysis.HadoopAggregatedData; -import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData; -import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; -import com.linkedin.drelephant.spark.legacydata.SparkExecutorData; -import com.linkedin.drelephant.util.MemoryFormatUtils; -import java.util.Iterator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -public class SparkMetricsAggregator implements HadoopMetricsAggregator { - - private static final Logger logger = LoggerFactory.getLogger(SparkMetricsAggregator.class); - - private AggregatorConfigurationData _aggregatorConfigurationData; - private double _storageMemWastageBuffer = 0.5; - - private static final String SPARK_EXECUTOR_MEMORY = "spark.executor.memory"; - private static final String STORAGE_MEM_WASTAGE_BUFFER = "storage_mem_wastage_buffer"; - - private HadoopAggregatedData _hadoopAggregatedData = new HadoopAggregatedData(); - - - public SparkMetricsAggregator(AggregatorConfigurationData _aggregatorConfigurationData) { - this._aggregatorConfigurationData = _aggregatorConfigurationData; - String configValue = _aggregatorConfigurationData.getParamMap().get(STORAGE_MEM_WASTAGE_BUFFER); - if(configValue != null) { - _storageMemWastageBuffer = Double.parseDouble(configValue); - } - } - - @Override - public void aggregate(HadoopApplicationData data) { - long resourceUsed = 0; - long resourceWasted = 0; - SparkApplicationData applicationData = (SparkApplicationData) data; - - long perExecutorMem = - MemoryFormatUtils.stringToBytes(applicationData.getEnvironmentData().getSparkProperty(SPARK_EXECUTOR_MEMORY, "0")); - - Iterator executorIds = applicationData.getExecutorData().getExecutors().iterator(); - - while(executorIds.hasNext()) { - String executorId = executorIds.next(); - SparkExecutorData.ExecutorInfo executorInfo = applicationData.getExecutorData().getExecutorInfo(executorId); - // store the resourceUsed in MBSecs - resourceUsed += (executorInfo.duration / Statistics.SECOND_IN_MS) * (perExecutorMem / FileUtils.ONE_MB); - // maxMem is the maximum available storage memory - // memUsed is how much storage memory is used. - // any difference is wasted after a buffer of 50% is wasted - long excessMemory = (long) (executorInfo.maxMem - (executorInfo.memUsed * (1.0 + _storageMemWastageBuffer))); - if( excessMemory > 0) { - resourceWasted += (executorInfo.duration / Statistics.SECOND_IN_MS) * (excessMemory / FileUtils.ONE_MB); - } - } - - _hadoopAggregatedData.setResourceUsed(resourceUsed); - _hadoopAggregatedData.setResourceWasted(resourceWasted); - // TODO: to find a way to calculate the delay - _hadoopAggregatedData.setTotalDelay(0L); - } - - @Override - public HadoopAggregatedData getResult() { - return _hadoopAggregatedData; - } -} diff --git a/test/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristicTest.java b/test/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristicTest.java deleted file mode 100644 index 3666500e9..000000000 --- a/test/com/linkedin/drelephant/spark/legacyheuristics/BestPropertiesConventionHeuristicTest.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * 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.linkedin.drelephant.spark.legacyheuristics; - -import com.linkedin.drelephant.analysis.ApplicationType; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.spark.legacydata.MockSparkApplicationData; -import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; -import junit.framework.TestCase; - -import static com.linkedin.drelephant.spark.legacyheuristics.BestPropertiesConventionHeuristic.SPARK_DRIVER_MEMORY; -import static com.linkedin.drelephant.spark.legacyheuristics.BestPropertiesConventionHeuristic.SPARK_EXECUTOR_CORES; -import static com.linkedin.drelephant.spark.legacyheuristics.BestPropertiesConventionHeuristic.SPARK_SERIALIZER; -import static com.linkedin.drelephant.spark.legacyheuristics.BestPropertiesConventionHeuristic.SPARK_SHUFFLE_MANAGER; - - -/** - * This class test the BestPropertiesConventionHeuristic - * - */ -public class BestPropertiesConventionHeuristicTest extends TestCase { - public void testPropertiesCheck() { - assertEquals(analyzeJob(getDefaultGoodProperteis()), Severity.NONE); - - assertEquals(Severity.MODERATE, analyzeJob(getPropertiesAndOverideOne(SPARK_DRIVER_MEMORY, "7G"))); - assertEquals(Severity.CRITICAL, analyzeJob(getPropertiesAndOverideOne(SPARK_DRIVER_MEMORY, "8G"))); - assertEquals(Severity.CRITICAL, analyzeJob(getPropertiesAndOverideOne(SPARK_DRIVER_MEMORY, "9G"))); - - assertEquals(Severity.NONE, analyzeJob(getPropertiesAndOverideOne(SPARK_EXECUTOR_CORES, "1"))); - assertEquals(Severity.NONE, analyzeJob(getPropertiesAndOverideOne(SPARK_EXECUTOR_CORES, "2"))); - assertEquals(Severity.CRITICAL, analyzeJob(getPropertiesAndOverideOne(SPARK_EXECUTOR_CORES, "4"))); - - - assertEquals(Severity.MODERATE, analyzeJob(getPropertiesAndOverideOne(SPARK_SERIALIZER, "foo"))); - assertEquals(Severity.MODERATE, analyzeJob(getPropertiesAndOverideOne(SPARK_SHUFFLE_MANAGER, "hash"))); - } - - public void testNullSettings() { - assertEquals(Severity.NONE, analyzeJob(getPropertiesAndRemove(SPARK_SERIALIZER))); - assertEquals(Severity.NONE, analyzeJob(getPropertiesAndRemove(SPARK_SHUFFLE_MANAGER))); - assertEquals(Severity.NONE, analyzeJob(getPropertiesAndRemove(SPARK_EXECUTOR_CORES))); - } - - private static Properties getDefaultGoodProperteis() { - Properties properties = new Properties(); - properties.put(SPARK_DRIVER_MEMORY, "1G"); - properties.put(SPARK_EXECUTOR_CORES, "1"); - properties.put(SPARK_SERIALIZER, "org.apache.spark.serializer.KryoSerializer"); - properties.put(SPARK_SHUFFLE_MANAGER, "sort"); - - return properties; - } - - private static Properties getPropertiesAndOverideOne(String key, String value) { - Properties properties = getDefaultGoodProperteis(); - properties.put(key, value); - return properties; - } - - private static Properties getPropertiesAndRemove(String key) { - Properties properties = getDefaultGoodProperteis(); - properties.remove(key); - return properties; - } - - private Severity analyzeJob(Properties sparkProperties) { - SparkApplicationData data = new MockSparkApplicationData(); - for (String key : sparkProperties.stringPropertyNames()) { - data.getEnvironmentData().addSparkProperty(key, sparkProperties.getProperty(key)); - } - Map paramsMap = new HashMap(); - return new BestPropertiesConventionHeuristic(new HeuristicConfigurationData("test_heuristic", "test_class", - "test_view", new ApplicationType("test_apptype"), paramsMap)).apply(data).getSeverity(); - } -} diff --git a/test/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristicTest.java b/test/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristicTest.java deleted file mode 100644 index 7ab245eed..000000000 --- a/test/com/linkedin/drelephant/spark/legacyheuristics/JobRuntimeHeuristicTest.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * 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.linkedin.drelephant.spark.legacyheuristics; - -import junit.framework.TestCase; - - -/** - * This class tests JobRuntimeHeuristic - */ -public class JobRuntimeHeuristicTest extends TestCase { - - public void test() { - - } -} diff --git a/test/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristicTest.java b/test/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristicTest.java deleted file mode 100644 index 3c926d1a4..000000000 --- a/test/com/linkedin/drelephant/spark/legacyheuristics/MemoryLimitHeuristicTest.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * 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.linkedin.drelephant.spark.legacyheuristics; - -import com.linkedin.drelephant.analysis.ApplicationType; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.HeuristicResultDetails; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.spark.legacydata.MockSparkApplicationData; -import com.linkedin.drelephant.spark.legacydata.SparkApplicationData; -import com.linkedin.drelephant.spark.legacydata.SparkEnvironmentData; -import com.linkedin.drelephant.spark.legacydata.SparkExecutorData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import com.linkedin.drelephant.util.MemoryFormatUtils; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import junit.framework.TestCase; - -import static com.linkedin.drelephant.spark.legacydata.SparkExecutorData.EXECUTOR_DRIVER_NAME; -import static com.linkedin.drelephant.spark.legacyheuristics.MemoryLimitHeuristic.SPARK_DRIVER_MEMORY; -import static com.linkedin.drelephant.spark.legacyheuristics.MemoryLimitHeuristic.SPARK_EXECUTOR_INSTANCES; -import static com.linkedin.drelephant.spark.legacyheuristics.MemoryLimitHeuristic.SPARK_EXECUTOR_MEMORY; -import static com.linkedin.drelephant.spark.legacyheuristics.MemoryLimitHeuristic.DEFAULT_SPARK_STORAGE_MEMORY_FRACTION; - - -/** - * This class tests the heuristic rule: MemoryLimitHeuristic - */ -public class MemoryLimitHeuristicTest extends TestCase { - public void testTotalMemoryRule() { - // Test if the total memory limit is working, set all peak memory to arbirarity 100% - assertEquals(Severity.NONE, analyzeJob(1, "1G", "1G", "1G")); - assertEquals(Severity.NONE, analyzeJob(100, "1G", "1G", "100G")); - assertEquals(Severity.NONE, analyzeJob(10, "10G", "1G", "100G")); - assertEquals(Severity.LOW, analyzeJob(600, "1G", "1G", "600G")); - assertEquals(Severity.MODERATE, analyzeJob(2400, "512M", "1G", "1.2T")); - assertEquals(Severity.SEVERE, analyzeJob(1600, "1G", "1G", "1.6T")); - assertEquals(Severity.CRITICAL, analyzeJob(4200, "512M", "1G", "2.1T")); - } - - public void testMemoryUtilizationRule() { - // Test if the total memory utilization is working - - // When the total memory is too low, ignore the ratio calculation - assertEquals(Severity.NONE, analyzeJob(1, "1G", "1G", "0B")); - // When we barely pass the safe zone - assertEquals(Severity.CRITICAL, analyzeJob(1000, "1G", "1G", "0B")); - - // Normal situations - assertEquals(Severity.LOW, analyzeJob(1000, "1G", "1G", getPeakMemory(0.7d, 1000, "1G"))); - assertEquals(Severity.MODERATE, analyzeJob(1000, "1G", "1G", getPeakMemory(0.5d, 1000, "1G"))); - assertEquals(Severity.SEVERE, analyzeJob(1000, "1G", "1G", getPeakMemory(0.3d, 1000, "1G"))); - assertEquals(Severity.CRITICAL, analyzeJob(1000, "1G", "1G", getPeakMemory(0.1d, 1000, "1G"))); - } - - public void testCombinedRules() { - // Mix multiple rules together, majorly check the combined logic - assertEquals(Severity.CRITICAL, analyzeJob(1, "1G", "10T", "0B")); - assertEquals(Severity.CRITICAL, analyzeJob(2500, "1G", "1G", getPeakMemory(0.7d, 2500, "1G"))); - } - - public void testMissingSparkDriverMemoryProperty() { - HeuristicResult result = getJobresult(100, "1G", "700M", getPeakMemory(1.0d, 100, "1G"), SPARK_DRIVER_MEMORY); - for (HeuristicResultDetails detail : result.getHeuristicResultDetails()) { - if (detail.getName().startsWith("\"Total driver memory allocated")) { - assertEquals("\"Total driver memory allocated\",\"700 MB\"", detail.getName()); - } - } - } - - private static String getPeakMemory(double utilRatio, int executors, String memPerExecutor) { - long totalMem = - (long) (executors * MemoryFormatUtils.stringToBytes(memPerExecutor) * DEFAULT_SPARK_STORAGE_MEMORY_FRACTION); - return MemoryFormatUtils.bytesToString((long) (utilRatio * totalMem)); - } - - private Severity analyzeJob(int executors, String memPerExecutor, String driverMem, String peakTotalMem, - String... skippedProperties) { - return getJobresult(executors, memPerExecutor, driverMem, peakTotalMem, skippedProperties).getSeverity(); - } - - private HeuristicResult getJobresult(int executors, String memPerExecutor, String driverMem, String peakTotalMem, - String... skippedProperties) { - Set filters = new HashSet(); - for (int i = 0; i < skippedProperties.length; i++) { - filters.add(skippedProperties[i]); - } - - SparkApplicationData data = new MockSparkApplicationData(); - SparkEnvironmentData env = data.getEnvironmentData(); - if (!filters.contains(SPARK_EXECUTOR_INSTANCES)) { - env.addSparkProperty(SPARK_EXECUTOR_INSTANCES, String.valueOf(executors)); - } - if (!filters.contains(SPARK_EXECUTOR_MEMORY)) { - env.addSparkProperty(SPARK_EXECUTOR_MEMORY, memPerExecutor); - } - if (!filters.contains(SPARK_DRIVER_MEMORY)) { - env.addSparkProperty(SPARK_DRIVER_MEMORY, driverMem); - } - - SparkExecutorData exe = data.getExecutorData(); - SparkExecutorData.ExecutorInfo driverInfo = new SparkExecutorData.ExecutorInfo(); - driverInfo.maxMem = (long) (MemoryFormatUtils.stringToBytes(driverMem) * DEFAULT_SPARK_STORAGE_MEMORY_FRACTION); - exe.setExecutorInfo(EXECUTOR_DRIVER_NAME, driverInfo); - - long bytesPerExecutor = MemoryFormatUtils.stringToBytes(memPerExecutor); - - /* Assign evenly the peak memory to each executor, in practical cases, we might observe the executor peak memory - * can vary a bit due to data skewness and imperfect partitioning. - */ - long peakMemToGenerate = MemoryFormatUtils.stringToBytes(peakTotalMem) / executors; - for (int i = 0; i < executors; i++) { - SparkExecutorData.ExecutorInfo info = new SparkExecutorData.ExecutorInfo(); - info.maxMem = (long) (bytesPerExecutor * DEFAULT_SPARK_STORAGE_MEMORY_FRACTION); - info.memUsed = peakMemToGenerate; - exe.setExecutorInfo(String.valueOf(i), info); - } - Map paramsMap = new HashMap(); - return new MemoryLimitHeuristic(new HeuristicConfigurationData("test_heuristic", "test_class", "test_view", - new ApplicationType("test_apptype"), paramsMap)).apply(data); - } -} diff --git a/test/org/apache/spark/TestSparkAggregatedMetrics.java b/test/org/apache/spark/TestSparkAggregatedMetrics.java deleted file mode 100644 index 3d8496d36..000000000 --- a/test/org/apache/spark/TestSparkAggregatedMetrics.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * 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; - -import com.linkedin.drelephant.analysis.ApplicationType; -import com.linkedin.drelephant.analysis.HadoopApplicationData; -import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData; -import com.linkedin.drelephant.spark.legacydata.MockSparkApplicationData; -import com.linkedin.drelephant.spark.legacydata.SparkExecutorData; -import org.apache.commons.io.FileUtils; -import org.junit.Assert; -import org.apache.spark.SparkMetricsAggregator; -import org.apache.spark.deploy.history.SparkDataCollection; -import org.junit.Test; - - -public class TestSparkAggregatedMetrics { - private static final String SPARK_EXECUTOR_MEMORY = "spark.executor.memory"; - - private SparkExecutorData.ExecutorInfo mockExecutorInfo(long maxMem, long memUsed, long duration) { - SparkExecutorData.ExecutorInfo executorInfo = new SparkExecutorData.ExecutorInfo(); - executorInfo.maxMem = maxMem; - executorInfo.memUsed = memUsed; - executorInfo.duration = duration; - - return executorInfo; - } - @Test - public void TestNullExecutors() { - ApplicationType appType = new ApplicationType("SPARK"); - AggregatorConfigurationData conf = - new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", appType, null); - SparkMetricsAggregator metrics = new SparkMetricsAggregator(conf); - - MockSparkApplicationData appData = new MockSparkApplicationData(); - - metrics.aggregate(appData); - - Assert.assertEquals(metrics.getResult().getResourceUsed() , 0L); - Assert.assertEquals(metrics.getResult().getResourceWasted() , 0L); - Assert.assertEquals(metrics.getResult().getTotalDelay() , 0L); - } - @Test - public void TestValidExecutorsWithNoEnvironmentData() { - ApplicationType appType = new ApplicationType("SPARK"); - AggregatorConfigurationData conf = - new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", appType, null); - SparkMetricsAggregator metrics = new SparkMetricsAggregator(conf); - - MockSparkApplicationData appData = new MockSparkApplicationData(); - appData.getExecutorData().setExecutorInfo("1", mockExecutorInfo(100 * FileUtils.ONE_MB, 60 * FileUtils.ONE_MB, 1000)); - appData.getExecutorData().setExecutorInfo("2", mockExecutorInfo(100 * FileUtils.ONE_MB, 60 * FileUtils.ONE_MB, 1000)); - - metrics.aggregate(appData); - - Assert.assertEquals(0L, metrics.getResult().getResourceUsed()); - Assert.assertEquals(20L, metrics.getResult().getResourceWasted()); - Assert.assertEquals(0L, metrics.getResult().getTotalDelay()); - } - - @Test - public void TestValidExecutorsAndValidEnvironmentData() { - ApplicationType appType = new ApplicationType("SPARK"); - AggregatorConfigurationData conf = - new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", appType, null); - SparkMetricsAggregator metrics = new SparkMetricsAggregator(conf); - - MockSparkApplicationData appData = new MockSparkApplicationData(); - appData.getExecutorData().setExecutorInfo("1", mockExecutorInfo(100 * FileUtils.ONE_MB, 60 * FileUtils.ONE_MB, 1000)); - appData.getExecutorData().setExecutorInfo("2", mockExecutorInfo(100 * FileUtils.ONE_MB, 60 * FileUtils.ONE_MB, 1000)); - - appData.getEnvironmentData().addSparkProperty(SPARK_EXECUTOR_MEMORY, "1048576000"); - - metrics.aggregate(appData); - - Assert.assertEquals(2000L, metrics.getResult().getResourceUsed()); - Assert.assertEquals(20L, metrics.getResult().getResourceWasted()); - Assert.assertEquals(0L, metrics.getResult().getTotalDelay()); - } - -} From ec508e7af3b6a0390ee400f5c32f23de408a84f9 Mon Sep 17 00:00:00 2001 From: shankar Date: Tue, 4 Apr 2017 13:45:44 +0530 Subject: [PATCH 03/11] Refactor to add a FSFetcher that wraps the legacy FSFetcher to confirm to the new interface --- .../spark/fetchers/SparkFetcher.scala | 15 ++----------- .../{LegacyFetchers.scala => FSFetcher.scala} | 21 +++++++++++-------- .../spark/fetchers/SparkFetcherTest.scala | 3 ++- 3 files changed, 16 insertions(+), 23 deletions(-) rename app/com/linkedin/drelephant/spark/legacyfetchers/{LegacyFetchers.scala => FSFetcher.scala} (57%) diff --git a/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala b/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala index e6e4cb24f..87e4de9a3 100644 --- a/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala +++ b/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala @@ -25,6 +25,7 @@ import scala.util.control.NonFatal import com.linkedin.drelephant.analysis.{AnalyticJob, ElephantFetcher} import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData} +import com.linkedin.drelephant.spark.legacyfetchers.FSFetcher import com.linkedin.drelephant.util.SparkUtils import org.apache.hadoop.conf.Configuration import org.apache.log4j.Logger @@ -63,7 +64,7 @@ class SparkFetcher(fetcherConfigurationData: FetcherConfigurationData) } private[fetchers] lazy val backupFetcher: ElephantFetcher[SparkApplicationData] = - new LegacyFetcher(fetcherConfigurationData) + new FSFetcher(fetcherConfigurationData) override def fetchData(analyticJob: AnalyticJob): SparkApplicationData = { doFetchData(analyticJob) match { @@ -114,18 +115,6 @@ class SparkFetcher(fetcherConfigurationData: FetcherConfigurationData) } object SparkFetcher { - import org.apache.spark.deploy.history.SparkFSFetcher - import com.linkedin.drelephant.spark.legacydata.LegacyDataConverters - - private[fetchers] class LegacyFetcher(fetcherConfigurationData: FetcherConfigurationData) - extends ElephantFetcher[SparkApplicationData] { - lazy val legacyFetcher = new SparkFSFetcher(fetcherConfigurationData) - - override def fetchData(analyticJob: AnalyticJob): SparkApplicationData = { - val legacyData = legacyFetcher.fetchData(analyticJob) - LegacyDataConverters.convert(legacyData) - } - } val SPARK_EVENT_LOG_ENABLED_KEY = "spark.eventLog.enabled" val DEFAULT_TIMEOUT = Duration(60, SECONDS) diff --git a/app/com/linkedin/drelephant/spark/legacyfetchers/LegacyFetchers.scala b/app/com/linkedin/drelephant/spark/legacyfetchers/FSFetcher.scala similarity index 57% rename from app/com/linkedin/drelephant/spark/legacyfetchers/LegacyFetchers.scala rename to app/com/linkedin/drelephant/spark/legacyfetchers/FSFetcher.scala index ac0a3754c..f786ef855 100644 --- a/app/com/linkedin/drelephant/spark/legacyfetchers/LegacyFetchers.scala +++ b/app/com/linkedin/drelephant/spark/legacyfetchers/FSFetcher.scala @@ -16,18 +16,21 @@ package com.linkedin.drelephant.spark.legacyfetchers -import com.linkedin.drelephant.analysis.AnalyticJob +import com.linkedin.drelephant.analysis.{AnalyticJob, ElephantFetcher} import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData -import com.linkedin.drelephant.spark.legacydata.SparkApplicationData +import com.linkedin.drelephant.spark.data.SparkApplicationData +import com.linkedin.drelephant.spark.legacydata.LegacyDataConverters import org.apache.spark.deploy.history.SparkFSFetcher +class FSFetcher(fetcherConfigurationData: FetcherConfigurationData) + extends ElephantFetcher[SparkApplicationData] { + lazy val legacyFetcher = new SparkFSFetcher(fetcherConfigurationData) -object LegacyFetchers { - def fetchDataUsingLegacyFetcher( - fetcherConfigurationData: FetcherConfigurationData, - analyticJob: AnalyticJob - ): SparkApplicationData = { - val legacyFetcher = new SparkFSFetcher(fetcherConfigurationData) - legacyFetcher.fetchData(analyticJob) + override def fetchData(analyticJob: AnalyticJob): SparkApplicationData = { + val legacyData = legacyFetcher.fetchData(analyticJob) + LegacyDataConverters.convert(legacyData) } } + +object FSFetcher { +} diff --git a/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala b/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala index 384eff475..99c9b33be 100644 --- a/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala +++ b/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala @@ -28,6 +28,7 @@ import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData import com.linkedin.drelephant.spark.data.{SparkLogDerivedData, SparkRestDerivedData} import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo} import com.linkedin.drelephant.spark.legacydata.{MockSparkApplicationData, SparkGeneralData} +import com.linkedin.drelephant.spark.legacyfetchers.FSFetcher import com.linkedin.drelephant.util.{SparkUtils, HadoopUtils} import org.apache.hadoop.fs.Path import org.apache.log4j.Logger @@ -88,7 +89,7 @@ class SparkFetcherTest extends FunSpec with Matchers with MockitoSugar { override lazy val sparkConf = sharedSparkConf override lazy val sparkRestClient = newFakeSparkRestClient(appId, Future { throw new Exception() }) override lazy val sparkLogClient = Some(newFakeSparkLogClient(appId, Some("2"), Future { throw new Exception() })) - override lazy val backupFetcher = new SparkFetcher.LegacyFetcher(fetcherConfigurationData) { + override lazy val backupFetcher = new FSFetcher(fetcherConfigurationData) { override lazy val legacyFetcher = new SparkFSFetcher(fetcherConfigurationData) { override lazy val sparkConf = sharedSparkConf override def fetchData(analyticJob: AnalyticJob) = new MockSparkApplicationData() { From 35b6e334aa6028e949477163f7c5de22c0f88b35 Mon Sep 17 00:00:00 2001 From: shankar Date: Tue, 4 Apr 2017 14:52:51 +0530 Subject: [PATCH 04/11] Remove backup for Rest Fetcher and make Legacy FSFetcher as top level fetcher. Change the default fetcher in the config --- app-conf/FetcherConf.xml | 5 +-- .../spark/fetchers/SparkFetcher.scala | 11 ------- .../spark/fetchers/SparkFetcherTest.scala | 32 ------------------- 3 files changed, 3 insertions(+), 45 deletions(-) diff --git a/app-conf/FetcherConf.xml b/app-conf/FetcherConf.xml index d06ce8bf7..5290d5e92 100644 --- a/app-conf/FetcherConf.xml +++ b/app-conf/FetcherConf.xml @@ -29,6 +29,7 @@ --> + - + To work properly, this fetcher should use the same timezone with the job history server. + If not set, the local timezone will be used. + --> + spark - com.linkedin.drelephant.spark.fetchers.SparkFetcher + com.linkedin.drelephant.spark.fetchers.FSFetcher diff --git a/app/com/linkedin/drelephant/spark/legacyfetchers/FSFetcher.scala b/app/com/linkedin/drelephant/spark/fetchers/FSFetcher.scala similarity index 87% rename from app/com/linkedin/drelephant/spark/legacyfetchers/FSFetcher.scala rename to app/com/linkedin/drelephant/spark/fetchers/FSFetcher.scala index f786ef855..e85196c2c 100644 --- a/app/com/linkedin/drelephant/spark/legacyfetchers/FSFetcher.scala +++ b/app/com/linkedin/drelephant/spark/fetchers/FSFetcher.scala @@ -14,7 +14,7 @@ * the License. */ -package com.linkedin.drelephant.spark.legacyfetchers +package com.linkedin.drelephant.spark.fetchers import com.linkedin.drelephant.analysis.{AnalyticJob, ElephantFetcher} import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData @@ -22,6 +22,10 @@ import com.linkedin.drelephant.spark.data.SparkApplicationData import com.linkedin.drelephant.spark.legacydata.LegacyDataConverters import org.apache.spark.deploy.history.SparkFSFetcher +/** + * Wraps the SparkFSFetcher which has the actual logic to comply to the new SparkApplicationData interface + * @param fetcherConfigurationData + */ class FSFetcher(fetcherConfigurationData: FetcherConfigurationData) extends ElephantFetcher[SparkApplicationData] { lazy val legacyFetcher = new SparkFSFetcher(fetcherConfigurationData) From 14083adb06e67e56adfecc54410f9102f0b46359 Mon Sep 17 00:00:00 2001 From: shankar Date: Tue, 11 Apr 2017 09:59:54 +0530 Subject: [PATCH 07/11] Misc Fetcher fixes to make it more reliable Fixed up existing tests. Need to add a couple more tests. --- .../spark/SparkMetricsAggregator.scala | 19 ++- .../spark/fetchers/SparkFetcher.scala | 7 +- .../spark/fetchers/SparkLogClient.scala | 4 +- .../linkedin/drelephant/util/SparkUtils.scala | 139 ++++++++++++++---- .../spark/deploy/history/SparkFSFetcher.scala | 13 +- .../spark/SparkMetricsAggregatorTest.scala | 2 +- .../spark/fetchers/SparkFetcherTest.scala | 2 +- .../spark/fetchers/SparkLogClientTest.scala | 2 +- .../drelephant/util/SparkUtilsTest.scala | 54 +++++-- 9 files changed, 186 insertions(+), 56 deletions(-) diff --git a/app/com/linkedin/drelephant/spark/SparkMetricsAggregator.scala b/app/com/linkedin/drelephant/spark/SparkMetricsAggregator.scala index 9dc1ac5ae..6053f3d54 100644 --- a/app/com/linkedin/drelephant/spark/SparkMetricsAggregator.scala +++ b/app/com/linkedin/drelephant/spark/SparkMetricsAggregator.scala @@ -63,10 +63,18 @@ class SparkMetricsAggregator(private val aggregatorConfigurationData: Aggregator case false => 0.0 } //allocated is the total used resource from the cluster. - if (resourcesAllocatedForUse.isValidLong) { + if (resourcesAllocatedForUse.isValidLong && resourcesAllocatedForUse > 0) { hadoopAggregatedData.setResourceUsed(resourcesAllocatedForUse.toLong) } else { - logger.info(s"resourcesAllocatedForUse exceeds Long.MaxValue: ${resourcesAllocatedForUse }") + logger.warn(s"resourcesAllocatedForUse exceeds Long.MaxValue: ${resourcesAllocatedForUse}") + logger.warn(s"ResourceUsed: ${resourcesAllocatedForUse}") + logger.warn(s"executorInstances: ${executorInstances}") + logger.warn(s"executorMemoryBytes:${executorMemoryBytes}") + logger.warn(s"applicationDurationMillis:${applicationDurationMillis}") + logger.warn(s"totalExecutorTaskTimeMillis:${totalExecutorTaskTimeMillis}") + logger.warn(s"resourcesActuallyUsedWithBuffer:${resourcesActuallyUsedWithBuffer}") + logger.warn(s"resourcesWastedMBSeconds:${resourcesWastedMBSeconds}") + logger.warn(s"allocatedMemoryWasteBufferPercentage:${allocatedMemoryWasteBufferPercentage}") } hadoopAggregatedData.setResourceWasted(resourcesWastedMBSeconds.toLong) @@ -99,7 +107,12 @@ class SparkMetricsAggregator(private val aggregatorConfigurationData: Aggregator private def applicationDurationMillisOf(data: SparkApplicationData): Long = { require(data.applicationInfo.attempts.nonEmpty) val lastApplicationAttemptInfo = data.applicationInfo.attempts.last - lastApplicationAttemptInfo.endTime.getTime - lastApplicationAttemptInfo.startTime.getTime + if(lastApplicationAttemptInfo.endTime.getTime < lastApplicationAttemptInfo.startTime.getTime) { + logger.info(s"Negative duration:${lastApplicationAttemptInfo.attemptId.get} startTime:${lastApplicationAttemptInfo.startTime.getTime} endTime:${lastApplicationAttemptInfo.endTime.getTime} ") + 0L + } else { + lastApplicationAttemptInfo.endTime.getTime - lastApplicationAttemptInfo.startTime.getTime + } } private def totalExecutorTaskTimeMillisOf(data: SparkApplicationData): BigInt = { diff --git a/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala b/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala index 4c6896f61..3c128efe0 100644 --- a/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala +++ b/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala @@ -25,7 +25,6 @@ import scala.util.control.NonFatal import com.linkedin.drelephant.analysis.{AnalyticJob, ElephantFetcher} import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData} -import com.linkedin.drelephant.spark.legacyfetchers.FSFetcher import com.linkedin.drelephant.util.SparkUtils import org.apache.hadoop.conf.Configuration import org.apache.log4j.Logger @@ -43,6 +42,9 @@ class SparkFetcher(fetcherConfigurationData: FetcherConfigurationData) private val logger: Logger = Logger.getLogger(classOf[SparkFetcher]) + val eventLogUri = Option(fetcherConfigurationData.getParamMap.get(LOG_LOCATION_URI_XML_FIELD)) + logger.info("The event log location of Spark application is set to " + eventLogUri) + private[fetchers] lazy val hadoopConfiguration: Configuration = new Configuration() private[fetchers] lazy val sparkUtils: SparkUtils = SparkUtils @@ -60,7 +62,7 @@ class SparkFetcher(fetcherConfigurationData: FetcherConfigurationData) private[fetchers] lazy val sparkLogClient: Option[SparkLogClient] = { val eventLogEnabled = sparkConf.getBoolean(SPARK_EVENT_LOG_ENABLED_KEY, false) - if (eventLogEnabled) Some(new SparkLogClient(hadoopConfiguration, sparkConf)) else None + if (eventLogEnabled) Some(new SparkLogClient(hadoopConfiguration, sparkConf, eventLogUri)) else None } override def fetchData(analyticJob: AnalyticJob): SparkApplicationData = { @@ -107,4 +109,5 @@ object SparkFetcher { val SPARK_EVENT_LOG_ENABLED_KEY = "spark.eventLog.enabled" val DEFAULT_TIMEOUT = Duration(60, SECONDS) + val LOG_LOCATION_URI_XML_FIELD = "event_log_location_uri" } diff --git a/app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala b/app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala index 5f64bb9f9..fcd05bf04 100644 --- a/app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala +++ b/app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala @@ -37,7 +37,7 @@ import org.json4s.jackson.JsonMethods /** * A client for getting data from the Spark event logs. */ -class SparkLogClient(hadoopConfiguration: Configuration, sparkConf: SparkConf) { +class SparkLogClient(hadoopConfiguration: Configuration, sparkConf: SparkConf, eventLogUri: Option[String]) { import SparkLogClient._ import Async.async @@ -60,7 +60,7 @@ class SparkLogClient(hadoopConfiguration: Configuration, sparkConf: SparkConf) { implicit ec: ExecutionContext ): Future[SparkLogDerivedData] = { val (eventLogFileSystem, baseEventLogPath) = - sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf, eventLogUri) val (eventLogPath, eventLogCodec) = sparkUtils.pathAndCodecforEventLog(sparkConf, eventLogFileSystem, baseEventLogPath, appId, attemptId) diff --git a/app/com/linkedin/drelephant/util/SparkUtils.scala b/app/com/linkedin/drelephant/util/SparkUtils.scala index 64cae4285..351968032 100644 --- a/app/com/linkedin/drelephant/util/SparkUtils.scala +++ b/app/com/linkedin/drelephant/util/SparkUtils.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters import scala.collection.mutable.HashMap import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.{FileSystem, Path, PathFilter, FileStatus} import org.apache.log4j.Logger import org.apache.spark.SparkConf import org.apache.spark.io.{CompressionCodec, LZ4CompressionCodec, LZFCompressionCodec, SnappyCompressionCodec} @@ -54,25 +54,33 @@ trait SparkUtils { * @param sparkConf a Spark configuration with the Spark event log directory setting * @return a tuple (FileSystem, Path) for the configured Spark event log directory */ - def fileSystemAndPathForEventLogDir(hadoopConfiguration: Configuration, sparkConf: SparkConf): (FileSystem, Path) = { - val eventLogUri = sparkConf.getOption(SPARK_EVENT_LOG_DIR_KEY).map(new URI(_)) - eventLogUri match { - case Some(uri) if uri.getScheme == "webhdfs" => - (FileSystem.get(uri, hadoopConfiguration), new Path(uri.getPath)) - case Some(uri) if uri.getScheme == "hdfs" => - (FileSystem.get(new URI(s"webhdfs://${uri.getHost}:${DFS_HTTP_PORT}${uri.getPath}"), hadoopConfiguration), new Path(uri.getPath)) - case Some(uri) => - val nameNodeAddress + def fileSystemAndPathForEventLogDir(hadoopConfiguration: Configuration, + sparkConf: SparkConf, + uriFromFetcherConf : Option[String]): (FileSystem, Path) = { + if(uriFromFetcherConf.isDefined) { + logger.info(s"Using log location from FetcherConf ${uriFromFetcherConf}") + val uri = new URI(uriFromFetcherConf.get) + (FileSystem.get(uri, hadoopConfiguration), new Path(uri.getPath)) + } else { + val eventLogUri = sparkConf.getOption(SPARK_EVENT_LOG_DIR_KEY).map(new URI(_)) + eventLogUri match { + case Some(uri) if uri.getScheme == "webhdfs" => + (FileSystem.get(uri, hadoopConfiguration), new Path(uri.getPath)) + case Some(uri) if uri.getScheme == "hdfs" => + (FileSystem.get(new URI(s"webhdfs://${uri.getHost}:${DFS_HTTP_PORT}${uri.getPath}"), hadoopConfiguration), new Path(uri.getPath)) + case Some(uri) => + val nameNodeAddress = hadoopUtils.findHaNameNodeAddress(hadoopConfiguration) .orElse(hadoopUtils.httpNameNodeAddress(hadoopConfiguration)) - nameNodeAddress match { - case Some(address) => - (FileSystem.get(new URI(s"webhdfs://${address}${uri.getPath}"), hadoopConfiguration), new Path(uri.getPath)) - case None => - throw new IllegalArgumentException("Couldn't find configured namenode") - } - case None => - throw new IllegalArgumentException("${SPARK_EVENT_LOG_DIR_KEY} not provided") + nameNodeAddress match { + case Some(address) => + (FileSystem.get(new URI(s"webhdfs://${address}${uri.getPath}"), hadoopConfiguration), new Path(uri.getPath)) + case None => + throw new IllegalArgumentException("Couldn't find configured namenode") + } + case None => + throw new IllegalArgumentException("${SPARK_EVENT_LOG_DIR_KEY} not provided") + } } } @@ -98,14 +106,24 @@ trait SparkUtils { appId: String, attemptId: Option[String] ): (Path, Option[CompressionCodec]) = { - val path = { - val shouldUseCompression = sparkConf.getBoolean(SPARK_EVENT_LOG_COMPRESS_KEY, defaultValue = false) - val compressionCodecShortName = - if (shouldUseCompression) Some(shortNameOfCompressionCodec(compressionCodecFromConf(sparkConf))) else None - getLogPath(fs.getUri.resolve(basePath.toUri), appId, attemptId, compressionCodecShortName) + attemptId match { + // if attemptid is given, use the existing method + case x: Some[String] => { val path = { + val shouldUseCompression = sparkConf.getBoolean(SPARK_EVENT_LOG_COMPRESS_KEY, defaultValue = false) + val compressionCodecShortName = + if (shouldUseCompression) Some(shortNameOfCompressionCodec(compressionCodecFromConf(sparkConf))) else None + getLogPath(fs.getUri.resolve(basePath.toUri), appId, attemptId, compressionCodecShortName) + } + val codec = compressionCodecForLogPath(sparkConf, path) + (path, codec) + } + case None => { + val (logPath, codecName) = getLogPathAndCodecName(fs, fs.getUri.resolve(basePath.toUri), appId) + + (logPath, Some(compressionCodecMap.getOrElseUpdate(codecName, loadCompressionCodec(sparkConf, codecName)))) + } } - val codec = compressionCodecForLogPath(sparkConf, path) - (path, codec) + } /** @@ -193,12 +211,72 @@ trait SparkUtils { } } + private def splitLogPath( logPath: String) : (Option[String],Option[String],Option[String]) = { + var extension: Option[String] = None + var attempt: Option[String] = None + var appId: Option[String] = None + val nameAndExtension = logPath.split('.') + if( nameAndExtension.length == 2 ) { + extension = Some(nameAndExtension(1)) + val name = nameAndExtension(0) + val appIdAndAttempt = name.split('_') + if( appIdAndAttempt.length == 4 ) { + attempt = Some(appIdAndAttempt(3)) + appId = Some(appIdAndAttempt.dropRight(1).mkString("_")) + } else { + appId = Some(name) + } + } + (appId, attempt, extension) + } + private def getLogPathAndCodecName( + fs: FileSystem, + logBaseDir: URI, + appId: String + ): (Path, String) = { + val base = logBaseDir.toString.stripSuffix("/"); + val filter = new PathFilter() { + override def accept(file: Path): Boolean = { + file.getName().startsWith(appId); + } + } + val attemptsList = fs.listStatus(new Path(base), filter) + val finalAttempt = attemptsList.length match { + case 0 => throw new FileNotFoundException(s"logfile does not exist for ${appId}.") + case 1 => splitLogPath(attemptsList(0).getPath().getName()) + case _ => attemptsList. + map( x => splitLogPath(x.getPath().getName())). + sortWith( (x,y) => x._2.getOrElse("-1").toInt > y._2.getOrElse("-1").toInt ). + head + } + + finalAttempt match { + // if attemptId is none and the codec is available, use the appid with no attemptid suffix + case noAttempt if noAttempt._1 != None & noAttempt._2 == None & noAttempt._3 != None => + (new Path(base + + "/" + finalAttempt._1.get + + "." + finalAttempt._3.get), finalAttempt._3.get) + // if attemptId is available and the codec is available, use the appid with attemptid suffix + case attempt if attempt._1 != None & attempt._2 != None & attempt._3 != None => + (new Path(base + + "/" + attempt._1.get + + "_" + sanitize(finalAttempt._2.get) + + "." + finalAttempt._3.get), finalAttempt._3.get) + // if codec is not available, but we found a file match with appId, use the actual file Path from the first match + case nocodec if nocodec._1 != None & nocodec._3 == None => (attemptsList(0).getPath(), DEFAULT_COMPRESSION_CODEC) + + // This should be reached only if we can't parse the filename in the path. + // Try to construct a general path in that case. + case _ => (new Path(base + "/" + appId + "." + DEFAULT_COMPRESSION_CODEC), DEFAULT_COMPRESSION_CODEC) + } + } + private def getLogPath( - logBaseDir: URI, - appId: String, - appAttemptId: Option[String], - compressionCodecName: Option[String] = None - ): Path = { + logBaseDir: URI, + appId: String, + appAttemptId: Option[String], + compressionCodecName: Option[String] = None + ): Path = { val base = logBaseDir.toString.stripSuffix("/") + "/" + sanitize(appId) val codec = compressionCodecName.map("." + _).getOrElse("") if (appAttemptId.isDefined) { @@ -207,7 +285,6 @@ trait SparkUtils { new Path(base + codec) } } - private def openEventLog(logPath: Path, fs: FileSystem): InputStream = { // It's not clear whether FileSystem.open() throws FileNotFoundException or just plain // IOException when a file does not exist, so try our best to throw a proper exception. diff --git a/app/org/apache/spark/deploy/history/SparkFSFetcher.scala b/app/org/apache/spark/deploy/history/SparkFSFetcher.scala index e0b3b336d..6788cccf5 100644 --- a/app/org/apache/spark/deploy/history/SparkFSFetcher.scala +++ b/app/org/apache/spark/deploy/history/SparkFSFetcher.scala @@ -49,6 +49,8 @@ class SparkFSFetcher(fetcherConfData: FetcherConfigurationData) extends Elephant .map { _(0) } .getOrElse(DEFAULT_EVENT_LOG_SIZE_LIMIT_MB) logger.info("The event log limit of Spark application is set to " + eventLogSizeLimitMb + " MB") + val eventLogUri = Option(fetcherConfData.getParamMap.get(LOG_LOCATION_URI_XML_FIELD)) + logger.info("The event log location of Spark application is set to " + eventLogUri) private lazy val security = new HadoopSecurity() @@ -79,9 +81,9 @@ class SparkFSFetcher(fetcherConfData: FetcherConfigurationData) extends Elephant val dataCollection = new SparkDataCollection() val (eventLogFileSystem, baseEventLogPath) = - sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf, eventLogUri) val (eventLogPath, eventLogCodec) = - sparkUtils.pathAndCodecforEventLog(sparkConf, eventLogFileSystem, baseEventLogPath, appId, DEFAULT_ATTEMPT_ID) + sparkUtils.pathAndCodecforEventLog(sparkConf, eventLogFileSystem, baseEventLogPath, appId, None) // Check if the log parser should be throttled when the file is too large. val shouldThrottle = eventLogFileSystem.getFileStatus(eventLogPath).getLen() > (eventLogSizeLimitMb * FileUtils.ONE_MB) @@ -95,7 +97,9 @@ class SparkFSFetcher(fetcherConfData: FetcherConfigurationData) extends Elephant logger.info("The event log of Spark application: " + appId + " is over the limit size of " + eventLogSizeLimitMb + " MB, the parsing process gets throttled.") } else { - logger.info("Replaying Spark logs for application: " + appId) + logger.info("Replaying Spark logs for application: " + appId + + " withlogPath: " + eventLogPath + + " with codec:" + eventLogCodec) sparkUtils.withEventLog(eventLogFileSystem, eventLogPath, eventLogCodec) { in => dataCollection.load(in, eventLogPath.toString()) @@ -105,6 +109,7 @@ class SparkFSFetcher(fetcherConfData: FetcherConfigurationData) extends Elephant } dataCollection + } } @@ -115,5 +120,7 @@ object SparkFSFetcher { val LOG_SIZE_XML_FIELD = "event_log_size_limit_in_mb" + val LOG_LOCATION_URI_XML_FIELD = "event_log_location_uri" + val DEFAULT_ATTEMPT_ID = Some("1") } diff --git a/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala b/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala index 954000fa0..0990534a0 100644 --- a/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala +++ b/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala @@ -134,7 +134,7 @@ object SparkMetricsAggregatorTest { import JavaConverters._ def newFakeAggregatorConfigurationData(params: Map[String, String] = Map.empty): AggregatorConfigurationData = - new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", new ApplicationType("SPARK"), params.asJava) + new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", new ApplicationType("SPARK"), params.asJava) def newFakeSparkListenerEnvironmentUpdate(appConfigurationProperties: Map[String, String]): SparkListenerEnvironmentUpdate = SparkListenerEnvironmentUpdate(Map("Spark Properties" -> appConfigurationProperties.toSeq)) diff --git a/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala b/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala index aafd9e128..457080682 100644 --- a/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala +++ b/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala @@ -28,7 +28,7 @@ import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData import com.linkedin.drelephant.spark.data.{SparkLogDerivedData, SparkRestDerivedData} import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo} import com.linkedin.drelephant.spark.legacydata.{MockSparkApplicationData, SparkGeneralData} -import com.linkedin.drelephant.spark.legacyfetchers.FSFetcher +import com.linkedin.drelephant.spark.fetchers.FSFetcher import com.linkedin.drelephant.util.{SparkUtils, HadoopUtils} import org.apache.hadoop.fs.Path import org.apache.log4j.Logger diff --git a/test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala b/test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala index e4ef1877d..994af486f 100644 --- a/test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala +++ b/test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala @@ -55,7 +55,7 @@ class SparkLogClientTest extends AsyncFunSpec with Matchers with MockitoSugar { bout.toByteArray } - val sparkLogClient = new SparkLogClient(hadoopConfiguration, sparkConf) { + val sparkLogClient = new SparkLogClient(hadoopConfiguration, sparkConf, None) { override lazy val sparkUtils = SparkUtilsTest.newFakeSparkUtilsForEventLog( new URI("webhdfs://nn1.grid.example.com:50070"), new Path("/logs/spark"), diff --git a/test/com/linkedin/drelephant/util/SparkUtilsTest.scala b/test/com/linkedin/drelephant/util/SparkUtilsTest.scala index 532a6b4df..110bad160 100644 --- a/test/com/linkedin/drelephant/util/SparkUtilsTest.scala +++ b/test/com/linkedin/drelephant/util/SparkUtilsTest.scala @@ -21,20 +21,37 @@ import java.net.URI import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FSDataInputStream, FileStatus, FileSystem, Path, PositionedReadable} +import org.apache.hadoop.fs.{FSDataInputStream, FileStatus, FileSystem, Path, PathFilter, PositionedReadable} import org.apache.hadoop.io.compress.CompressionInputStream import org.apache.log4j.Logger import org.apache.spark.SparkConf import org.apache.spark.io.SnappyCompressionCodec import org.mockito.BDDMockito +import org.mockito.Matchers import org.scalatest.{FunSpec, Matchers, OptionValues} import org.scalatest.mockito.MockitoSugar import org.xerial.snappy.SnappyOutputStream -class SparkUtilsTest extends FunSpec with Matchers with OptionValues with MockitoSugar { +class SparkUtilsTest extends FunSpec with org.scalatest.Matchers with OptionValues with MockitoSugar { describe("SparkUtils") { describe(".fileSystemAndPathForEventLogDir") { + it("returns a filesystem + path based on uri from fetcherConfg") { + val hadoopConfiguration = new Configuration(false) + val sparkConf = new SparkConf() + val sparkUtils = new SparkUtils { + override lazy val logger = mock[Logger] + override lazy val hadoopUtils = mock[HadoopUtils] + override lazy val defaultEnv = Map.empty[String, String] + } + + val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, + sparkConf, + Some("webhdfs://nn1.grid.example.com:50070/logs/spark")) + fs.getUri.toString should be("webhdfs://nn1.grid.example.com:50070") + path should be(new Path("/logs/spark")) + } + it("returns a webhdfs filesystem + path based on spark.eventLog.dir when it is a webhdfs URL") { val hadoopConfiguration = new Configuration(false) val sparkConf = new SparkConf().set("spark.eventLog.dir", "webhdfs://nn1.grid.example.com:50070/logs/spark") @@ -44,7 +61,7 @@ class SparkUtilsTest extends FunSpec with Matchers with OptionValues with Mockit override lazy val defaultEnv = Map.empty[String, String] } - val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf, None) fs.getUri.toString should be("webhdfs://nn1.grid.example.com:50070") path should be(new Path("/logs/spark")) } @@ -58,7 +75,7 @@ class SparkUtilsTest extends FunSpec with Matchers with OptionValues with Mockit override lazy val defaultEnv = Map.empty[String, String] } - val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf, None) fs.getUri.toString should be("webhdfs://nn1.grid.example.com:50070") path should be(new Path("/logs/spark")) } @@ -83,7 +100,7 @@ class SparkUtilsTest extends FunSpec with Matchers with OptionValues with Mockit override lazy val defaultEnv = Map.empty[String, String] } - val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf, None) fs.getUri.toString should be("webhdfs://sample-ha2.grid.example.com:50070") path should be(new Path("/logs/spark")) } @@ -109,7 +126,7 @@ class SparkUtilsTest extends FunSpec with Matchers with OptionValues with Mockit override lazy val defaultEnv = Map.empty[String, String] } - val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf, None) fs.getUri.toString should be("webhdfs://sample.grid.example.com:50070") path should be(new Path("/logs/spark")) } @@ -131,7 +148,7 @@ class SparkUtilsTest extends FunSpec with Matchers with OptionValues with Mockit override lazy val defaultEnv = Map.empty[String, String] } - val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + val (fs, path) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf, None) fs.getUri.toString should be("webhdfs://sample.grid.example.com:50070") path should be(new Path("/logs/spark")) } @@ -147,7 +164,7 @@ class SparkUtilsTest extends FunSpec with Matchers with OptionValues with Mockit override lazy val defaultEnv = Map.empty[String, String] } - an[Exception] should be thrownBy { sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) } + an[Exception] should be thrownBy { sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf, None) } } } @@ -167,7 +184,7 @@ class SparkUtilsTest extends FunSpec with Matchers with OptionValues with Mockit Array.empty[Byte] ) - val (fs, basePath) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + val (fs, basePath) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf, None) val (path, codec) = sparkUtils.pathAndCodecforEventLog(sparkConf: SparkConf, fs: FileSystem, basePath: Path, "application_1", Some("1")) @@ -207,10 +224,10 @@ class SparkUtilsTest extends FunSpec with Matchers with OptionValues with Mockit eventLogBytes ) - val (fs, basePath) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf) + val (fs, basePath) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf, None) val (path, codec) = - sparkUtils.pathAndCodecforEventLog(sparkConf: SparkConf, fs: FileSystem, basePath: Path, "application_1", Some("1")) + sparkUtils.pathAndCodecforEventLog(sparkConf: SparkConf, fs: FileSystem, basePath: Path, "application_1", None) sparkUtils.withEventLog(fs, path, codec) { in => val bout = new ByteArrayOutputStream() @@ -237,18 +254,31 @@ object SparkUtilsTest extends MockitoSugar { override def fileSystemAndPathForEventLogDir( hadoopConfiguration: Configuration, - sparkConf: SparkConf + sparkConf: SparkConf, + uriFromFetcherConf: Option[String] ): (FileSystem, Path) = { val fs = mock[FileSystem] val expectedPath = new Path(new Path(fileSystemUri), new Path(basePath, filename)) val expectedFileStatus = { val fileStatus = mock[FileStatus] BDDMockito.given(fileStatus.getLen).willReturn(bytes.length.toLong) + BDDMockito.given(fileStatus.getPath()).willReturn(expectedPath) fileStatus } + val expectedStatusArray = Array(expectedFileStatus) + + val filter = new PathFilter() { + override def accept(file: Path): Boolean = { + file.getName().startsWith("mockAppId"); + } + } + BDDMockito.given(fs.getUri).willReturn(fileSystemUri) BDDMockito.given(fs.exists(expectedPath)).willReturn(true) BDDMockito.given(fs.getFileStatus(expectedPath)).willReturn(expectedFileStatus) + BDDMockito.given(fs.listStatus(org.mockito.Matchers.refEq(new Path( new Path(fileSystemUri), basePath)), + org.mockito.Matchers.any(filter.getClass))). + willReturn(expectedStatusArray) BDDMockito.given(fs.open(expectedPath)).willReturn( new FSDataInputStream(new FakeCompressionInputStream(new ByteArrayInputStream(bytes))) ) From 926c9749f649a9359b8355fe14fc118a116d6dcd Mon Sep 17 00:00:00 2001 From: shankar Date: Tue, 11 Apr 2017 10:35:36 +0530 Subject: [PATCH 08/11] Add More tests --- .../spark/SparkMetricsAggregatorTest.scala | 33 ++++++++++++++++--- .../drelephant/util/SparkUtilsTest.scala | 23 +++++++++++++ 2 files changed, 52 insertions(+), 4 deletions(-) diff --git a/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala b/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala index 0990534a0..c25e7915c 100644 --- a/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala +++ b/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala @@ -46,11 +46,11 @@ class SparkMetricsAggregatorTest extends FunSpec with Matchers { new ApplicationInfo(appId, name = "app", Seq(applicationAttemptInfo)) } + val executorSummaries = Seq( + newFakeExecutorSummary(id = "1", totalDuration = 1000000L), + newFakeExecutorSummary(id = "2", totalDuration = 3000000L) + ) val restDerivedData = { - val executorSummaries = Seq( - newFakeExecutorSummary(id = "1", totalDuration = 1000000L), - newFakeExecutorSummary(id = "2", totalDuration = 3000000L) - ) SparkRestDerivedData( applicationInfo, jobDatas = Seq.empty, @@ -105,6 +105,31 @@ class SparkMetricsAggregatorTest extends FunSpec with Matchers { it("doesn't calculate total delay") { result.getTotalDelay should be(0L) } + it("sets resourceused as 0 when duration is negative") { + //make the duration negative + val applicationInfo = { + val applicationAttemptInfo = { + val now = System.currentTimeMillis + val duration = -8000000L + newFakeApplicationAttemptInfo(Some("1"), startTime = new Date(now - duration), endTime = new Date(now)) + } + new ApplicationInfo(appId, name = "app", Seq(applicationAttemptInfo)) + } + val restDerivedData = SparkRestDerivedData( + applicationInfo, + jobDatas = Seq.empty, + stageDatas = Seq.empty, + executorSummaries + ) + + val data = SparkApplicationData(appId, restDerivedData, Some(logDerivedData)) + + val aggregator = new SparkMetricsAggregator(aggregatorConfigurationData) + aggregator.aggregate(data) + + val result = aggregator.getResult + result.getResourceUsed should be(0L) + } } describe("when it doesn't have log-derived data") { diff --git a/test/com/linkedin/drelephant/util/SparkUtilsTest.scala b/test/com/linkedin/drelephant/util/SparkUtilsTest.scala index 110bad160..632b49536 100644 --- a/test/com/linkedin/drelephant/util/SparkUtilsTest.scala +++ b/test/com/linkedin/drelephant/util/SparkUtilsTest.scala @@ -192,6 +192,29 @@ class SparkUtilsTest extends FunSpec with org.scalatest.Matchers with OptionValu path should be(new Path("webhdfs://nn1.grid.example.com:50070/logs/spark/application_1_1.snappy")) codec.value should be(a[SnappyCompressionCodec]) } + it("returns the path and codec for the event log, given the base path and appid. Extracts attempt and codec from path") { + val hadoopConfiguration = new Configuration(false) + + val sparkConf = + new SparkConf() + .set("spark.eventLog.dir", "/logs/spark") + .set("spark.eventLog.compress", "true") + + val sparkUtils = SparkUtilsTest.newFakeSparkUtilsForEventLog( + new URI("webhdfs://nn1.grid.example.com:50070"), + new Path("/logs/spark"), + new Path("application_1_1.snappy"), + Array.empty[Byte] + ) + + val (fs, basePath) = sparkUtils.fileSystemAndPathForEventLogDir(hadoopConfiguration, sparkConf, None) + + val (path, codec) = + sparkUtils.pathAndCodecforEventLog(sparkConf: SparkConf, fs: FileSystem, basePath: Path, "application_1", None) + + path should be(new Path("webhdfs://nn1.grid.example.com:50070/logs/spark/application_1_1.snappy")) + codec.value should be(a[SnappyCompressionCodec]) + } } describe(".withEventLog") { From 516b4c7a09133673b38635ff0ce53f59dd6fccc3 Mon Sep 17 00:00:00 2001 From: shankar Date: Wed, 12 Apr 2017 09:06:45 +0530 Subject: [PATCH 09/11] Remove MetricsAggregator fix to put it in separate PR --- .../spark/SparkMetricsAggregator.scala | 19 ++--------- .../spark/SparkMetricsAggregatorTest.scala | 33 +++---------------- 2 files changed, 7 insertions(+), 45 deletions(-) diff --git a/app/com/linkedin/drelephant/spark/SparkMetricsAggregator.scala b/app/com/linkedin/drelephant/spark/SparkMetricsAggregator.scala index 6053f3d54..9dc1ac5ae 100644 --- a/app/com/linkedin/drelephant/spark/SparkMetricsAggregator.scala +++ b/app/com/linkedin/drelephant/spark/SparkMetricsAggregator.scala @@ -63,18 +63,10 @@ class SparkMetricsAggregator(private val aggregatorConfigurationData: Aggregator case false => 0.0 } //allocated is the total used resource from the cluster. - if (resourcesAllocatedForUse.isValidLong && resourcesAllocatedForUse > 0) { + if (resourcesAllocatedForUse.isValidLong) { hadoopAggregatedData.setResourceUsed(resourcesAllocatedForUse.toLong) } else { - logger.warn(s"resourcesAllocatedForUse exceeds Long.MaxValue: ${resourcesAllocatedForUse}") - logger.warn(s"ResourceUsed: ${resourcesAllocatedForUse}") - logger.warn(s"executorInstances: ${executorInstances}") - logger.warn(s"executorMemoryBytes:${executorMemoryBytes}") - logger.warn(s"applicationDurationMillis:${applicationDurationMillis}") - logger.warn(s"totalExecutorTaskTimeMillis:${totalExecutorTaskTimeMillis}") - logger.warn(s"resourcesActuallyUsedWithBuffer:${resourcesActuallyUsedWithBuffer}") - logger.warn(s"resourcesWastedMBSeconds:${resourcesWastedMBSeconds}") - logger.warn(s"allocatedMemoryWasteBufferPercentage:${allocatedMemoryWasteBufferPercentage}") + logger.info(s"resourcesAllocatedForUse exceeds Long.MaxValue: ${resourcesAllocatedForUse }") } hadoopAggregatedData.setResourceWasted(resourcesWastedMBSeconds.toLong) @@ -107,12 +99,7 @@ class SparkMetricsAggregator(private val aggregatorConfigurationData: Aggregator private def applicationDurationMillisOf(data: SparkApplicationData): Long = { require(data.applicationInfo.attempts.nonEmpty) val lastApplicationAttemptInfo = data.applicationInfo.attempts.last - if(lastApplicationAttemptInfo.endTime.getTime < lastApplicationAttemptInfo.startTime.getTime) { - logger.info(s"Negative duration:${lastApplicationAttemptInfo.attemptId.get} startTime:${lastApplicationAttemptInfo.startTime.getTime} endTime:${lastApplicationAttemptInfo.endTime.getTime} ") - 0L - } else { - lastApplicationAttemptInfo.endTime.getTime - lastApplicationAttemptInfo.startTime.getTime - } + lastApplicationAttemptInfo.endTime.getTime - lastApplicationAttemptInfo.startTime.getTime } private def totalExecutorTaskTimeMillisOf(data: SparkApplicationData): BigInt = { diff --git a/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala b/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala index c25e7915c..0990534a0 100644 --- a/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala +++ b/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala @@ -46,11 +46,11 @@ class SparkMetricsAggregatorTest extends FunSpec with Matchers { new ApplicationInfo(appId, name = "app", Seq(applicationAttemptInfo)) } - val executorSummaries = Seq( - newFakeExecutorSummary(id = "1", totalDuration = 1000000L), - newFakeExecutorSummary(id = "2", totalDuration = 3000000L) - ) val restDerivedData = { + val executorSummaries = Seq( + newFakeExecutorSummary(id = "1", totalDuration = 1000000L), + newFakeExecutorSummary(id = "2", totalDuration = 3000000L) + ) SparkRestDerivedData( applicationInfo, jobDatas = Seq.empty, @@ -105,31 +105,6 @@ class SparkMetricsAggregatorTest extends FunSpec with Matchers { it("doesn't calculate total delay") { result.getTotalDelay should be(0L) } - it("sets resourceused as 0 when duration is negative") { - //make the duration negative - val applicationInfo = { - val applicationAttemptInfo = { - val now = System.currentTimeMillis - val duration = -8000000L - newFakeApplicationAttemptInfo(Some("1"), startTime = new Date(now - duration), endTime = new Date(now)) - } - new ApplicationInfo(appId, name = "app", Seq(applicationAttemptInfo)) - } - val restDerivedData = SparkRestDerivedData( - applicationInfo, - jobDatas = Seq.empty, - stageDatas = Seq.empty, - executorSummaries - ) - - val data = SparkApplicationData(appId, restDerivedData, Some(logDerivedData)) - - val aggregator = new SparkMetricsAggregator(aggregatorConfigurationData) - aggregator.aggregate(data) - - val result = aggregator.getResult - result.getResourceUsed should be(0L) - } } describe("when it doesn't have log-derived data") { From 8b98ae3a3e97a06a9990a11087040423f7469e64 Mon Sep 17 00:00:00 2001 From: shankar Date: Thu, 13 Apr 2017 21:52:23 +0530 Subject: [PATCH 10/11] Merge remote-tracking branch 'linkedin/master' into FixRaysSparkFetcherPR # Conflicts: # app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala # app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala --- .../spark/data/SparkRestDerivedData.scala | 4 +- .../spark/fetchers/SparkFetcher.scala | 42 +++++++--- .../spark/fetchers/SparkRestClient.scala | 62 +++++++++++--- .../spark/SparkMetricsAggregatorTest.scala | 2 +- .../spark/fetchers/SparkFetcherTest.scala | 58 +++++++++++-- .../spark/fetchers/SparkRestClientTest.scala | 81 ++++++++++++++++--- .../heuristics/ExecutorsHeuristicTest.scala | 2 +- .../heuristics/StagesHeuristicTest.scala | 2 +- 8 files changed, 212 insertions(+), 41 deletions(-) diff --git a/app/com/linkedin/drelephant/spark/data/SparkRestDerivedData.scala b/app/com/linkedin/drelephant/spark/data/SparkRestDerivedData.scala index 721c4f3c9..1b3a66232 100644 --- a/app/com/linkedin/drelephant/spark/data/SparkRestDerivedData.scala +++ b/app/com/linkedin/drelephant/spark/data/SparkRestDerivedData.scala @@ -23,5 +23,5 @@ case class SparkRestDerivedData( applicationInfo: ApplicationInfo, jobDatas: Seq[JobData], stageDatas: Seq[StageData], - executorSummaries: Seq[ExecutorSummary] -) + executorSummaries: Seq[ExecutorSummary], + private[spark] val logDerivedData: Option[SparkLogDerivedData] = None) diff --git a/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala b/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala index 3c128efe0..2ab8b40ac 100644 --- a/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala +++ b/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala @@ -24,7 +24,7 @@ import scala.util.control.NonFatal import com.linkedin.drelephant.analysis.{AnalyticJob, ElephantFetcher} import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData -import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData} +import com.linkedin.drelephant.spark.data.SparkApplicationData import com.linkedin.drelephant.util.SparkUtils import org.apache.hadoop.conf.Configuration import org.apache.log4j.Logger @@ -58,11 +58,30 @@ class SparkFetcher(fetcherConfigurationData: FetcherConfigurationData) sparkConf } + private[fetchers] lazy val eventLogSource: EventLogSource = { + val eventLogEnabled = sparkConf.getBoolean(SPARK_EVENT_LOG_ENABLED_KEY, false) + val useRestForLogs = Option(fetcherConfigurationData.getParamMap.get("use_rest_for_eventlogs")) + .exists(_.toBoolean) + if (!eventLogEnabled) { + EventLogSource.None + } else if (useRestForLogs) EventLogSource.Rest else EventLogSource.WebHdfs + } + private[fetchers] lazy val sparkRestClient: SparkRestClient = new SparkRestClient(sparkConf) - private[fetchers] lazy val sparkLogClient: Option[SparkLogClient] = { - val eventLogEnabled = sparkConf.getBoolean(SPARK_EVENT_LOG_ENABLED_KEY, false) - if (eventLogEnabled) Some(new SparkLogClient(hadoopConfiguration, sparkConf, eventLogUri)) else None + private[fetchers] lazy val sparkLogClient: SparkLogClient = { + new SparkLogClient(hadoopConfiguration, sparkConf) + } + + sealed trait EventLogSource + + object EventLogSource { + /** Fetch event logs through REST API. */ + case object Rest extends EventLogSource + /** Fetch event logs through WebHDFS. */ + case object WebHdfs extends EventLogSource + /** Event logs are not available. */ + case object None extends EventLogSource } override def fetchData(analyticJob: AnalyticJob): SparkApplicationData = { @@ -91,13 +110,14 @@ class SparkFetcher(fetcherConfigurationData: FetcherConfigurationData) private def doFetchDataUsingRestAndLogClients(analyticJob: AnalyticJob): Future[SparkApplicationData] = async { val appId = analyticJob.getAppId - val restDerivedData = await(sparkRestClient.fetchData(appId)) - val lastAttemptId = restDerivedData.applicationInfo.attempts.maxBy { _.startTime }.attemptId - - // Would use .map but await doesn't like that construction. - val logDerivedData = sparkLogClient match { - case Some(sparkLogClient) => Some(await(sparkLogClient.fetchData(appId, lastAttemptId))) - case None => None + val restDerivedData = await(sparkRestClient.fetchData(appId, eventLogSource == EventLogSource.Rest)) + + val logDerivedData = eventLogSource match { + case EventLogSource.None => None + case EventLogSource.Rest => restDerivedData.logDerivedData + case EventLogSource.WebHdfs => + val lastAttemptId = restDerivedData.applicationInfo.attempts.maxBy { _.startTime }.attemptId + Some(await(sparkLogClient.fetchData(appId, lastAttemptId))) } SparkApplicationData(appId, restDerivedData, logDerivedData) diff --git a/app/com/linkedin/drelephant/spark/fetchers/SparkRestClient.scala b/app/com/linkedin/drelephant/spark/fetchers/SparkRestClient.scala index 584748916..a5c1bb31e 100644 --- a/app/com/linkedin/drelephant/spark/fetchers/SparkRestClient.scala +++ b/app/com/linkedin/drelephant/spark/fetchers/SparkRestClient.scala @@ -16,25 +16,26 @@ package com.linkedin.drelephant.spark.fetchers +import java.io.BufferedInputStream import java.net.URI import java.text.SimpleDateFormat +import java.util.zip.ZipInputStream import java.util.{Calendar, SimpleTimeZone} import scala.async.Async import scala.concurrent.{ExecutionContext, Future} import scala.util.control.NonFatal - import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} import com.fasterxml.jackson.module.scala.DefaultScalaModule import com.fasterxml.jackson.module.scala.experimental.ScalaObjectMapper -import com.linkedin.drelephant.spark.data.SparkRestDerivedData -import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo, ExecutorSummary, JobData, StageData} +import com.linkedin.drelephant.spark.data.{SparkLogDerivedData, SparkRestDerivedData} +import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationInfo, ExecutorSummary, JobData, StageData} import javax.ws.rs.client.{Client, ClientBuilder, WebTarget} import javax.ws.rs.core.MediaType + import org.apache.log4j.Logger import org.apache.spark.SparkConf - /** * A client for getting data from the Spark monitoring REST API, e.g. . * @@ -67,24 +68,34 @@ class SparkRestClient(sparkConf: SparkConf) { private val apiTarget: WebTarget = client.target(historyServerUri).path(API_V1_MOUNT_PATH) - def fetchData(appId: String)(implicit ec: ExecutionContext): Future[SparkRestDerivedData] = { + def fetchData(appId: String, fetchLogs: Boolean = false)( + implicit ec: ExecutionContext + ): Future[SparkRestDerivedData] = { val appTarget = apiTarget.path(s"applications/${appId}") logger.info(s"calling REST API at ${appTarget.getUri}") val applicationInfo = getApplicationInfo(appTarget) - // Limit scope of async. + // These are pure and cannot fail, therefore it is safe to have + // them outside of the async block. + val lastAttemptId = applicationInfo.attempts.maxBy {_.startTime}.attemptId + val attemptTarget = lastAttemptId.map(appTarget.path).getOrElse(appTarget) + + // Limit the scope of async. async { - val lastAttemptId = applicationInfo.attempts.maxBy {_.startTime}.attemptId - val attemptTarget = lastAttemptId.map(appTarget.path).getOrElse(appTarget) val futureJobDatas = async { getJobDatas(attemptTarget) } val futureStageDatas = async { getStageDatas(attemptTarget) } val futureExecutorSummaries = async { getExecutorSummaries(attemptTarget) } + val futureLogData = if (fetchLogs) { + async { getLogData(attemptTarget)} + } else Future.successful(None) + SparkRestDerivedData( applicationInfo, await(futureJobDatas), await(futureStageDatas), - await(futureExecutorSummaries) + await(futureExecutorSummaries), + await(futureLogData) ) } } @@ -100,6 +111,39 @@ class SparkRestClient(sparkConf: SparkConf) { } } + private def getLogData(attemptTarget: WebTarget): Option[SparkLogDerivedData] = { + val target = attemptTarget.path("logs") + logger.info(s"calling REST API at ${target.getUri} to get eventlogs") + + // The logs are stored in a ZIP archive with a single entry. + // It should be named as "$logPrefix.$archiveExtension", but + // we trust Spark to get it right. + resource.managed { getApplicationLogs(target) }.acquireAndGet { zis => + val entry = zis.getNextEntry + if (entry == null) { + logger.warn(s"failed to resolve log for ${target.getUri}") + None + } else { + val codec = SparkLogClient.compressionCodecForLogName(sparkConf, entry.getName) + Some(SparkLogClient.findDerivedData( + codec.map { _.compressedInputStream(zis) }.getOrElse(zis))) + } + } + } + + private def getApplicationLogs(logTarget: WebTarget): ZipInputStream = { + try { + val is = logTarget.request(MediaType.APPLICATION_OCTET_STREAM) + .get(classOf[java.io.InputStream]) + new ZipInputStream(new BufferedInputStream(is)) + } catch { + case NonFatal(e) => { + logger.error(s"error reading logs ${logTarget.getUri}", e) + throw e + } + } + } + private def getJobDatas(attemptTarget: WebTarget): Seq[JobData] = { val target = attemptTarget.path("jobs") try { diff --git a/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala b/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala index 0990534a0..a3c0e1cf2 100644 --- a/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala +++ b/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala @@ -55,7 +55,7 @@ class SparkMetricsAggregatorTest extends FunSpec with Matchers { applicationInfo, jobDatas = Seq.empty, stageDatas = Seq.empty, - executorSummaries + executorSummaries = executorSummaries ) } diff --git a/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala b/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala index 457080682..e422b2499 100644 --- a/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala +++ b/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala @@ -20,12 +20,12 @@ import java.io.InputStream import java.nio.file.Files import java.util.Date -import scala.collection.JavaConverters import scala.concurrent.{ExecutionContext, Future} import com.linkedin.drelephant.analysis.{AnalyticJob, ApplicationType} import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData import com.linkedin.drelephant.spark.data.{SparkLogDerivedData, SparkRestDerivedData} +import com.linkedin.drelephant.spark.fetchers.SparkFetcher.EventLogSource import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo} import com.linkedin.drelephant.spark.legacydata.{MockSparkApplicationData, SparkGeneralData} import com.linkedin.drelephant.spark.fetchers.FSFetcher @@ -75,7 +75,7 @@ class SparkFetcherTest extends FunSpec with Matchers with MockitoSugar { val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { override lazy val sparkConf = new SparkConf() override lazy val sparkRestClient = newFakeSparkRestClient(appId, Future(restDerivedData)) - override lazy val sparkLogClient = Some(newFakeSparkLogClient(appId, Some("2"), Future(logDerivedData))) + override lazy val sparkLogClient = newFakeSparkLogClient(appId, Some("2"), Future(logDerivedData)) } val data = sparkFetcher.fetchData(analyticJob) data.appId should be(appId) @@ -85,7 +85,7 @@ class SparkFetcherTest extends FunSpec with Matchers with MockitoSugar { val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { override lazy val sparkConf = new SparkConf() override lazy val sparkRestClient = newFakeSparkRestClient(appId, Future { throw new Exception() }) - override lazy val sparkLogClient = Some(newFakeSparkLogClient(appId, Some("2"), Future(logDerivedData))) + override lazy val sparkLogClient = newFakeSparkLogClient(appId, Some("2"), Future(logDerivedData)) } an[Exception] should be thrownBy { sparkFetcher.fetchData(analyticJob) } @@ -94,8 +94,9 @@ class SparkFetcherTest extends FunSpec with Matchers with MockitoSugar { it("throws an exception if the log client fails") { val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { override lazy val sparkConf = new SparkConf() + .set(SparkFetcher.SPARK_EVENT_LOG_ENABLED_KEY, "true") override lazy val sparkRestClient = newFakeSparkRestClient(appId, Future(restDerivedData)) - override lazy val sparkLogClient = Some(newFakeSparkLogClient(appId, Some("2"), Future { throw new Exception() })) + override lazy val sparkLogClient = newFakeSparkLogClient(appId, Some("2"), Future { throw new Exception() }) } an[Exception] should be thrownBy { sparkFetcher.fetchData(analyticJob) } @@ -166,16 +167,59 @@ class SparkFetcherTest extends FunSpec with Matchers with MockitoSugar { override lazy val defaultEnv = Map.empty[String, String] } } + an[IllegalStateException] should be thrownBy { sparkFetcher.sparkConf } } + + it("eventlog source defaults to WebHDFS") { + val fetcherConfigurationData = newFakeFetcherConfigurationData() + val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { + override lazy val sparkConf: SparkConf = new SparkConf() + .set(SparkFetcher.SPARK_EVENT_LOG_ENABLED_KEY, "true") + } + + sparkFetcher.eventLogSource should be(EventLogSource.WebHdfs) + } + + it("eventlog source is WebHDFS if use_rest_for_eventlogs is false") { + val fetcherConfigurationData = newFakeFetcherConfigurationData( + Map("use_rest_for_eventlogs" -> "false")) + val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { + override lazy val sparkConf: SparkConf = new SparkConf() + .set(SparkFetcher.SPARK_EVENT_LOG_ENABLED_KEY, "true") + } + + sparkFetcher.eventLogSource should be(EventLogSource.WebHdfs) + } + + it("eventlog source is REST if use_rest_for_eventlogs is true") { + val fetcherConfigurationData = newFakeFetcherConfigurationData( + Map("use_rest_for_eventlogs" -> "true")) + val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { + override lazy val sparkConf: SparkConf = new SparkConf() + .set(SparkFetcher.SPARK_EVENT_LOG_ENABLED_KEY, "true") + } + + sparkFetcher.eventLogSource should be(EventLogSource.Rest) + } + + it("eventlog fetching is disabled when spark.eventLog is false") { + val fetcherConfigurationData = newFakeFetcherConfigurationData() + val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { + override lazy val sparkConf: SparkConf = new SparkConf() + .set(SparkFetcher.SPARK_EVENT_LOG_ENABLED_KEY, "false") + } + + sparkFetcher.eventLogSource should be(EventLogSource.None) + } } } object SparkFetcherTest { - import JavaConverters._ + import scala.collection.JavaConverters._ - def newFakeFetcherConfigurationData(): FetcherConfigurationData = - new FetcherConfigurationData(classOf[SparkFetcher].getName, new ApplicationType("SPARK"), Map.empty.asJava) + def newFakeFetcherConfigurationData(paramMap: Map[String, String] = Map.empty): FetcherConfigurationData = + new FetcherConfigurationData(classOf[SparkFetcher].getName, new ApplicationType("SPARK"), paramMap.asJava) def newFakeApplicationAttemptInfo( attemptId: Option[String], diff --git a/test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala b/test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala index 71a401d32..f428902c8 100644 --- a/test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala +++ b/test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala @@ -16,24 +16,27 @@ package com.linkedin.drelephant.spark.fetchers -import java.net.URI +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream} import java.text.SimpleDateFormat +import java.util.zip.{ZipEntry, ZipOutputStream} import java.util.{Calendar, Date, SimpleTimeZone} import scala.concurrent.ExecutionContext import scala.util.Try - import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo, ExecutorSummary, JobData, StageData} import javax.ws.rs.{GET, Path, PathParam, Produces} -import javax.ws.rs.client.WebTarget -import javax.ws.rs.core.{Application, MediaType} +import javax.ws.rs.core.{Application, MediaType, Response} import javax.ws.rs.ext.ContextResolver + +import com.google.common.io.Resources +import com.ning.compress.lzf.LZFEncoder import org.apache.spark.SparkConf import org.glassfish.jersey.client.ClientConfig import org.glassfish.jersey.server.ResourceConfig import org.glassfish.jersey.test.{JerseyTest, TestProperties} +import org.json4s.DefaultFormats import org.scalatest.{AsyncFunSpec, Matchers} import org.scalatest.compatible.Assertion @@ -52,6 +55,7 @@ class SparkRestClientTest extends AsyncFunSpec with Matchers { .register(classOf[FetchClusterModeDataFixtures.JobsResource]) .register(classOf[FetchClusterModeDataFixtures.StagesResource]) .register(classOf[FetchClusterModeDataFixtures.ExecutorsResource]) + .register(classOf[FetchClusterModeDataFixtures.LogsResource]) case config => config } } @@ -66,12 +70,21 @@ class SparkRestClientTest extends AsyncFunSpec with Matchers { sparkRestClient.fetchData(FetchClusterModeDataFixtures.APP_ID) map { restDerivedData => restDerivedData.applicationInfo.id should be(FetchClusterModeDataFixtures.APP_ID) restDerivedData.applicationInfo.name should be(FetchClusterModeDataFixtures.APP_NAME) - restDerivedData.jobDatas should not be(None) - restDerivedData.stageDatas should not be(None) - restDerivedData.executorSummaries should not be(None) + restDerivedData.jobDatas should not be (None) + restDerivedData.stageDatas should not be (None) + restDerivedData.executorSummaries should not be (None) + restDerivedData.logDerivedData should be(None) + } flatMap { + case assertion: Try[Assertion] => assertion + case _ => + val expectedLogDerivedData = + SparkLogClient.findDerivedData(new ByteArrayInputStream(EVENT_LOG_2)) + + sparkRestClient.fetchData(FetchClusterModeDataFixtures.APP_ID, fetchLogs = true) + .map { _.logDerivedData should be(Some(expectedLogDerivedData)) } } andThen { case assertion: Try[Assertion] => - fakeJerseyServer.tearDown() - assertion + fakeJerseyServer.tearDown() + assertion } } @@ -86,6 +99,7 @@ class SparkRestClientTest extends AsyncFunSpec with Matchers { .register(classOf[FetchClientModeDataFixtures.JobsResource]) .register(classOf[FetchClientModeDataFixtures.StagesResource]) .register(classOf[FetchClientModeDataFixtures.ExecutorsResource]) + .register(classOf[FetchClientModeDataFixtures.LogsResource]) case config => config } } @@ -103,6 +117,15 @@ class SparkRestClientTest extends AsyncFunSpec with Matchers { restDerivedData.jobDatas should not be(None) restDerivedData.stageDatas should not be(None) restDerivedData.executorSummaries should not be(None) + restDerivedData.logDerivedData should be(None) + } flatMap { + case assertion: Try[Assertion] => assertion + case _ => + val expectedLogDerivedData = + SparkLogClient.findDerivedData(new ByteArrayInputStream(EVENT_LOG_2)) + + sparkRestClient.fetchData(FetchClientModeDataFixtures.APP_ID, fetchLogs = true) + .map { _.logDerivedData should be(Some(expectedLogDerivedData)) } } andThen { case assertion: Try[Assertion] => fakeJerseyServer.tearDown() assertion @@ -214,6 +237,9 @@ object SparkRestClientTest { @Path("applications/{appId}/{attemptId}/executors") def getExecutors(): ExecutorsResource = new ExecutorsResource() + + @Path("applications/{appId}/{attemptId}/logs") + def getLogs(): LogsResource = new LogsResource() } @Produces(Array(MediaType.APPLICATION_JSON)) @@ -254,6 +280,16 @@ object SparkRestClientTest { def getExecutors(@PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): Seq[ExecutorSummary] = if (attemptId == "2") Seq.empty else throw new Exception() } + + @Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) + class LogsResource { + @GET + def getLogs(@PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): Response = { + if (attemptId == "2") { + Response.ok(newFakeLog(appId, Some(attemptId))).build() + } else throw new Exception() + } + } } object FetchClientModeDataFixtures { @@ -273,6 +309,9 @@ object SparkRestClientTest { @Path("applications/{appId}/executors") def getExecutors(): ExecutorsResource = new ExecutorsResource() + + @Path("applications/{appId}/logs") + def getLogs(): LogsResource = new LogsResource() } @Produces(Array(MediaType.APPLICATION_JSON)) @@ -313,6 +352,14 @@ object SparkRestClientTest { def getExecutors(@PathParam("appId") appId: String): Seq[ExecutorSummary] = Seq.empty } + + @Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) + class LogsResource { + @GET + def getLogs(@PathParam("appId") appId: String): Response = { + Response.ok(newFakeLog(appId, None)).build() + } + } } def newFakeApplicationAttemptInfo( @@ -326,4 +373,20 @@ object SparkRestClientTest { sparkUser = "foo", completed = true ) + + private val EVENT_LOG_2 = Resources.toByteArray( + Resources.getResource("spark_event_logs/event_log_2")) + + def newFakeLog(appId: String, attemptId: Option[String]): InputStream = { + val os = new ByteArrayOutputStream() + val zos = new ZipOutputStream(os) + val name = attemptId.map(id => s"${appId}_$id").getOrElse(appId) + ".lzf" + zos.putNextEntry(new ZipEntry(name)) + // LZFEncoder instead of Snappy, because of xerial/snappy-java#76. + zos.write(LZFEncoder.encode(EVENT_LOG_2)) + zos.closeEntry() + zos.close() + + new ByteArrayInputStream(os.toByteArray) + } } diff --git a/test/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristicTest.scala b/test/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristicTest.scala index 90e360caf..459cdeab2 100644 --- a/test/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristicTest.scala +++ b/test/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristicTest.scala @@ -259,7 +259,7 @@ object ExecutorsHeuristicTest { new ApplicationInfo(appId, name = "app", Seq.empty), jobDatas = Seq.empty, stageDatas = Seq.empty, - executorSummaries + executorSummaries = executorSummaries ) SparkApplicationData(appId, restDerivedData, logDerivedData = None) diff --git a/test/com/linkedin/drelephant/spark/heuristics/StagesHeuristicTest.scala b/test/com/linkedin/drelephant/spark/heuristics/StagesHeuristicTest.scala index 6794df530..0b980393b 100644 --- a/test/com/linkedin/drelephant/spark/heuristics/StagesHeuristicTest.scala +++ b/test/com/linkedin/drelephant/spark/heuristics/StagesHeuristicTest.scala @@ -178,7 +178,7 @@ object StagesHeuristicTest { val restDerivedData = SparkRestDerivedData( new ApplicationInfo(appId, name = "app", Seq.empty), jobDatas = Seq.empty, - stageDatas, + stageDatas = stageDatas, executorSummaries = Seq.empty ) From 529d5dc2b9f60472dfc89901f1308a9b4b919635 Mon Sep 17 00:00:00 2001 From: shankar Date: Thu, 13 Apr 2017 22:44:32 +0530 Subject: [PATCH 11/11] Fix build errors --- .../spark/fetchers/SparkFetcher.scala | 13 +----------- .../spark/fetchers/SparkRestClient.scala | 3 ++- .../linkedin/drelephant/util/SparkUtils.scala | 20 +++++++++---------- 3 files changed, 13 insertions(+), 23 deletions(-) diff --git a/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala b/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala index 998d6cb82..698064ac6 100644 --- a/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala +++ b/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala @@ -70,18 +70,7 @@ class SparkFetcher(fetcherConfigurationData: FetcherConfigurationData) private[fetchers] lazy val sparkRestClient: SparkRestClient = new SparkRestClient(sparkConf) private[fetchers] lazy val sparkLogClient: SparkLogClient = { - new SparkLogClient(hadoopConfiguration, sparkConf) - } - - sealed trait EventLogSource - - object EventLogSource { - /** Fetch event logs through REST API. */ - case object Rest extends EventLogSource - /** Fetch event logs through WebHDFS. */ - case object WebHdfs extends EventLogSource - /** Event logs are not available. */ - case object None extends EventLogSource + new SparkLogClient(hadoopConfiguration, sparkConf, eventLogUri) } override def fetchData(analyticJob: AnalyticJob): SparkApplicationData = { diff --git a/app/com/linkedin/drelephant/spark/fetchers/SparkRestClient.scala b/app/com/linkedin/drelephant/spark/fetchers/SparkRestClient.scala index a5c1bb31e..55381831c 100644 --- a/app/com/linkedin/drelephant/spark/fetchers/SparkRestClient.scala +++ b/app/com/linkedin/drelephant/spark/fetchers/SparkRestClient.scala @@ -30,6 +30,7 @@ import com.fasterxml.jackson.module.scala.DefaultScalaModule import com.fasterxml.jackson.module.scala.experimental.ScalaObjectMapper import com.linkedin.drelephant.spark.data.{SparkLogDerivedData, SparkRestDerivedData} import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationInfo, ExecutorSummary, JobData, StageData} +import com.linkedin.drelephant.util.SparkUtils import javax.ws.rs.client.{Client, ClientBuilder, WebTarget} import javax.ws.rs.core.MediaType @@ -124,7 +125,7 @@ class SparkRestClient(sparkConf: SparkConf) { logger.warn(s"failed to resolve log for ${target.getUri}") None } else { - val codec = SparkLogClient.compressionCodecForLogName(sparkConf, entry.getName) + val codec = SparkUtils.compressionCodecForLogName(sparkConf, entry.getName) Some(SparkLogClient.findDerivedData( codec.map { _.compressedInputStream(zis) }.getOrElse(zis))) } diff --git a/app/com/linkedin/drelephant/util/SparkUtils.scala b/app/com/linkedin/drelephant/util/SparkUtils.scala index 351968032..e7efd9d84 100644 --- a/app/com/linkedin/drelephant/util/SparkUtils.scala +++ b/app/com/linkedin/drelephant/util/SparkUtils.scala @@ -114,7 +114,7 @@ trait SparkUtils { if (shouldUseCompression) Some(shortNameOfCompressionCodec(compressionCodecFromConf(sparkConf))) else None getLogPath(fs.getUri.resolve(basePath.toUri), appId, attemptId, compressionCodecShortName) } - val codec = compressionCodecForLogPath(sparkConf, path) + val codec = compressionCodecForLogName(sparkConf, path.getName()) (path, codec) } case None => { @@ -170,6 +170,15 @@ trait SparkUtils { } } + def compressionCodecForLogName(conf: SparkConf, logName: String): Option[CompressionCodec] = { + // Compression codec is encoded as an extension, e.g. app_123.lzf + // Since we sanitize the app ID to not include periods, it is safe to split on it + val logBaseName = logName.stripSuffix(IN_PROGRESS) + logBaseName.split("\\.").tail.lastOption.map { codecName => + compressionCodecMap.getOrElseUpdate(codecName, loadCompressionCodec(conf, codecName)) + } + } + private val IN_PROGRESS = ".inprogress" private val DEFAULT_COMPRESSION_CODEC = "snappy" @@ -295,15 +304,6 @@ trait SparkUtils { new BufferedInputStream(fs.open(logPath)) } - private def compressionCodecForLogPath(conf: SparkConf, logPath: Path): Option[CompressionCodec] = { - // Compression codec is encoded as an extension, e.g. app_123.lzf - // Since we sanitize the app ID to not include periods, it is safe to split on it - val logBaseName = logPath.getName.stripSuffix(IN_PROGRESS) - logBaseName.split("\\.").tail.lastOption.map { codecName => - compressionCodecMap.getOrElseUpdate(codecName, loadCompressionCodec(conf, codecName)) - } - } - private def sanitize(str: String): String = { str.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase }